Browse Source

Merge remote-tracking branch 'apache/trunk' into MR-2841

Todd Lipcon 10 years ago
parent
commit
4b3f1e2ce4
100 changed files with 2072 additions and 1676 deletions
  1. 0 0
      LICENSE.txt
  2. 0 0
      NOTICE.txt
  3. 0 0
      README.txt
  4. 19 5
      dev-support/create-release.sh
  5. 8 0
      hadoop-assemblies/src/main/resources/assemblies/hadoop-src.xml
  6. 20 0
      hadoop-common-project/hadoop-common/CHANGES.txt
  7. 1 1
      hadoop-common-project/hadoop-common/dev-support/findbugsExcludeFile.xml
  8. 26 10
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProviderFactory.java
  9. 2 55
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/KMSClientProvider.java
  10. 18 3
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/random/OsSecureRandom.java
  11. 11 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/authorize/AccessControlList.java
  12. 3 12
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticationFilter.java
  13. 4 2
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticationHandler.java
  14. 2 18
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticator.java
  15. 185 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/HttpExceptionUtils.java
  16. 51 18
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/TestCryptoCodec.java
  17. 1 1
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/TestCryptoStreams.java
  18. 13 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/key/TestKeyProviderFactory.java
  19. 15 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/random/TestOsSecureRandom.java
  20. 14 21
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/delegation/web/TestDelegationTokenAuthenticationHandlerWithMocks.java
  21. 167 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestHttpExceptionUtils.java
  22. 1 0
      hadoop-common-project/hadoop-kms/pom.xml
  23. 6 21
      hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMS.java
  24. 51 4
      hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSACLs.java
  25. 2 10
      hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSExceptionsProvider.java
  26. 86 2
      hadoop-common-project/hadoop-kms/src/site/apt/index.apt.vm
  27. 94 6
      hadoop-common-project/hadoop-kms/src/test/java/org/apache/hadoop/crypto/key/kms/server/TestKMS.java
  28. 1 1
      hadoop-common-project/hadoop-kms/src/test/java/org/apache/hadoop/crypto/key/kms/server/TestKMSACLs.java
  29. 3 0
      hadoop-dist/pom.xml
  30. 31 39
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSFileSystem.java
  31. 0 50
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSUtils.java
  32. 2 12
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/wsrs/ExceptionProvider.java
  33. 3 1
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/BaseTestHttpFSWith.java
  34. 4 6
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSServerNoACLs.java
  35. 3 3
      hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/OpenFileCtx.java
  36. 1 1
      hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/RpcProgramNfs3.java
  37. 25 0
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  38. 0 271
      hadoop-hdfs-project/hadoop-hdfs/LICENSE.txt
  39. 0 2
      hadoop-hdfs-project/hadoop-hdfs/NOTICE.txt
  40. 3 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
  41. 72 8
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
  42. 8 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
  43. 12 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/inotify/Event.java
  44. 4 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
  45. 6 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolManager.java
  46. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
  47. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java
  48. 35 9
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
  49. 4 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
  50. 6 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
  51. 18 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java
  52. 17 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageSerialization.java
  53. 39 21
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
  54. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeReference.java
  55. 1 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/InotifyFSEditLogOpTranslator.java
  56. 3 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeLayoutVersion.java
  57. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/DirectorySnapshottableFeature.java
  58. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/GetConf.java
  59. 1 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/proto/inotify.proto
  60. 18 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
  61. 15 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlocksScheduledCounter.java
  62. 2 6
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSInotifyEventInputStream.java
  63. 6 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSRename.java
  64. 26 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java
  65. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend4.java
  66. 119 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreation.java
  67. 28 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLocalDFS.java
  68. 22 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockPoolManager.java
  69. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/CreateEditsLog.java
  70. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java
  71. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStartup.java
  72. 36 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotDiffReport.java
  73. 22 4
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestGetConf.java
  74. BIN
      hadoop-hdfs-project/hadoop-hdfs/src/test/resources/editsStored
  75. 119 63
      hadoop-hdfs-project/hadoop-hdfs/src/test/resources/editsStored.xml
  76. 6 0
      hadoop-mapreduce-project/CHANGES.txt
  77. 0 341
      hadoop-mapreduce-project/LICENSE.txt
  78. 0 2
      hadoop-mapreduce-project/NOTICE.txt
  79. 2 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/JobImpl.java
  80. 1 3
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/MapTask.java
  81. 12 0
      hadoop-yarn-project/CHANGES.txt
  82. 0 341
      hadoop-yarn-project/LICENSE.txt
  83. 0 17
      hadoop-yarn-project/NOTICE.txt
  84. 8 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
  85. 15 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryServer.java
  86. 7 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/webapp/CrossOriginFilter.java
  87. 15 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/webapp/TestCrossOriginFilter.java
  88. 11 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/LinuxContainerExecutor.java
  89. 60 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestLinuxContainerExecutor.java
  90. 21 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/AllocationConfiguration.java
  91. 39 9
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/AllocationFileLoaderService.java
  92. 47 7
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSLeafQueue.java
  93. 3 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSParentQueue.java
  94. 19 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSQueue.java
  95. 18 42
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java
  96. 5 27
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/QueueManager.java
  97. 43 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestAllocationFileLoaderService.java
  98. 170 28
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFSLeafQueue.java
  99. 34 124
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java
  100. 10 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/FairScheduler.apt.vm

+ 0 - 0
hadoop-common-project/hadoop-common/LICENSE.txt → LICENSE.txt


+ 0 - 0
hadoop-common-project/hadoop-common/NOTICE.txt → NOTICE.txt


+ 0 - 0
hadoop-common-project/hadoop-common/README.txt → README.txt


+ 19 - 5
dev-support/create-release.sh

@@ -70,8 +70,10 @@ fi
 
 ARTIFACTS_DIR="target/artifacts"
 
-# Create staging dir for release artifacts
+# mvn clean for sanity
+run ${MVN} clean
 
+# Create staging dir for release artifacts
 run mkdir -p ${ARTIFACTS_DIR}
 
 # Create RAT report
@@ -80,10 +82,17 @@ run ${MVN} apache-rat:check
 # Create SRC and BIN tarballs for release,
 # Using 'install’ goal instead of 'package' so artifacts are available 
 # in the Maven local cache for the site generation
-run ${MVN} install -Pdist,docs,src,native -DskipTests -Dtar
+run ${MVN} install -Pdist,src,native -DskipTests -Dtar
 
 # Create site for release
 run ${MVN} site site:stage -Pdist -Psrc
+run mkdir -p target/staging/hadoop-project/hadoop-project-dist/hadoop-yarn
+run mkdir -p target/staging/hadoop-project/hadoop-project-dist/hadoop-mapreduce
+run cp ./hadoop-common-project/hadoop-common/src/main/docs/releasenotes.html target/staging/hadoop-project/hadoop-project-dist/hadoop-common/
+run cp ./hadoop-common-project/hadoop-common/CHANGES.txt target/staging/hadoop-project/hadoop-project-dist/hadoop-common/
+run cp ./hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt target/staging/hadoop-project/hadoop-project-dist/hadoop-hdfs/
+run cp ./hadoop-yarn-project/CHANGES.txt target/staging/hadoop-project/hadoop-project-dist/hadoop-yarn/
+run cp ./hadoop-mapreduce-project/CHANGES.txt target/staging/hadoop-project/hadoop-project-dist/hadoop-mapreduce/
 run mv target/staging/hadoop-project target/r${HADOOP_VERSION}/
 run cd target/
 run tar czf hadoop-site-${HADOOP_VERSION}.tar.gz r${HADOOP_VERSION}/*
@@ -94,14 +103,19 @@ find . -name rat.txt | xargs -I% cat % > ${ARTIFACTS_DIR}/hadoop-${HADOOP_VERSIO
 
 # Stage CHANGES.txt files
 run cp ./hadoop-common-project/hadoop-common/CHANGES.txt ${ARTIFACTS_DIR}/CHANGES-COMMON-${HADOOP_VERSION}${RC_LABEL}.txt
-run cp ./hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt ${ARTIFACTS_DIR}/CHANGES-HDFS--${HADOOP_VERSION}${RC_LABEL}.txt
+run cp ./hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt ${ARTIFACTS_DIR}/CHANGES-HDFS-${HADOOP_VERSION}${RC_LABEL}.txt
 run cp ./hadoop-mapreduce-project/CHANGES.txt ${ARTIFACTS_DIR}/CHANGES-MAPREDUCE-${HADOOP_VERSION}${RC_LABEL}.txt
 run cp ./hadoop-yarn-project/CHANGES.txt ${ARTIFACTS_DIR}/CHANGES-YARN-${HADOOP_VERSION}${RC_LABEL}.txt
 
-# Stage BIN tarball
+# Prepare and stage BIN tarball
+run cd hadoop-dist/target/
+run tar -xzf hadoop-${HADOOP_VERSION}.tar.gz
+run cp -r ../../target/r${HADOOP_VERSION}/* hadoop-${HADOOP_VERSION}/share/doc/hadoop/
+run tar -czf hadoop-${HADOOP_VERSION}.tar.gz hadoop-${HADOOP_VERSION}
+run cd ../..
 run mv hadoop-dist/target/hadoop-${HADOOP_VERSION}.tar.gz ${ARTIFACTS_DIR}/hadoop-${HADOOP_VERSION}${RC_LABEL}.tar.gz
 
-# State SRC tarball
+# Stage SRC tarball
 run mv hadoop-dist/target/hadoop-${HADOOP_VERSION}-src.tar.gz ${ARTIFACTS_DIR}/hadoop-${HADOOP_VERSION}${RC_LABEL}-src.tar.gz
 
 # Stage SITE tarball

+ 8 - 0
hadoop-assemblies/src/main/resources/assemblies/hadoop-src.xml

@@ -23,6 +23,14 @@
   </formats>
   <includeBaseDirectory>true</includeBaseDirectory>
   <fileSets>
+    <fileSet>
+      <directory>.</directory>
+      <includes>
+        <include>LICENCE.txt</include>
+        <include>README.txt</include>
+        <include>NOTICE.txt</include>
+      </includes>
+    </fileSet>
     <fileSet>
       <directory>.</directory>
       <useDefaultExcludes>true</useDefaultExcludes>

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

@@ -493,6 +493,17 @@ Release 2.6.0 - UNRELEASED
 
     HADOOP-10990. Add missed NFSv3 request and response classes (brandonli)
 
+    HADOOP-10863. KMS should have a blacklist for decrypting EEKs. 
+    (asuresh via tucu)
+
+    HADOOP-11054. Add a KeyProvider instantiation based on a URI. (tucu)
+
+    HADOOP-11015. Http server/client utils to propagate and recreate 
+    Exceptions from server to client. (tucu)
+
+    HADOOP-11060. Create a CryptoCodec test that verifies interoperability 
+    between the JCE and OpenSSL implementations. (hitliuyi via tucu)
+
   OPTIMIZATIONS
 
     HADOOP-10838. Byte array native checksumming. (James Thomas via todd)
@@ -681,6 +692,9 @@ Release 2.6.0 - UNRELEASED
     HADOOP-8815. RandomDatum needs to override hashCode().
     (Brandon Li via suresh)
 
+    HADOOP-11056. OsSecureRandom.setConf() might leak file descriptors (yzhang
+    via cmccabe)
+
     BREAKDOWN OF HDFS-6134 AND HADOOP-10150 SUBTASKS AND RELATED JIRAS
   
       HADOOP-10734. Implement high-performance secure random number sources.
@@ -740,6 +754,9 @@ Release 2.6.0 - UNRELEASED
     HADOOP-11012. hadoop fs -text of zero-length file causes EOFException
     (Eric Payne via jlowe)
 
+    HADOOP-11063. KMS cannot deploy on Windows, because class names are too long.
+    (cnauroth)
+
 Release 2.5.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES
@@ -747,6 +764,9 @@ Release 2.5.1 - UNRELEASED
   NEW FEATURES
 
   IMPROVEMENTS
+  
+    HADOOP-10956. Fix create-release script to include docs and necessary txt
+    files. (kasha)
 
   OPTIMIZATIONS
 

+ 1 - 1
hadoop-common-project/hadoop-common/dev-support/findbugsExcludeFile.xml

@@ -367,7 +367,7 @@
      </Match>
 
   <Match>
-    <Class name="org.apache.hadoop.crypto.key.kms.KMSClientProvider"/>
+    <Class name="org.apache.hadoop.util.HttpExceptionUtils"/>
     <Method name="validateResponse"/>
     <Bug pattern="REC_CATCH_EXCEPTION"/>
   </Match>

+ 26 - 10
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProviderFactory.java

@@ -63,16 +63,10 @@ public abstract class KeyProviderFactory {
     for(String path: conf.getStringCollection(KEY_PROVIDER_PATH)) {
       try {
         URI uri = new URI(path);
-        boolean found = false;
-        for(KeyProviderFactory factory: serviceLoader) {
-          KeyProvider kp = factory.createProvider(uri, conf);
-          if (kp != null) {
-            result.add(kp);
-            found = true;
-            break;
-          }
-        }
-        if (!found) {
+        KeyProvider kp = get(uri, conf);
+        if (kp != null) {
+          result.add(kp);
+        } else {
           throw new IOException("No KeyProviderFactory for " + uri + " in " +
               KEY_PROVIDER_PATH);
         }
@@ -83,4 +77,26 @@ public abstract class KeyProviderFactory {
     }
     return result;
   }
+
+  /**
+   * Create a KeyProvider based on a provided URI.
+   *
+   * @param uri key provider URI
+   * @param conf configuration to initialize the key provider
+   * @return the key provider for the specified URI, or <code>NULL</code> if
+   *         a provider for the specified URI scheme could not be found.
+   * @throws IOException thrown if the provider failed to initialize.
+   */
+  public static KeyProvider get(URI uri, Configuration conf)
+      throws IOException {
+    KeyProvider kp = null;
+    for (KeyProviderFactory factory : serviceLoader) {
+      kp = factory.createProvider(uri, conf);
+      if (kp != null) {
+        break;
+      }
+    }
+    return kp;
+  }
+
 }

+ 2 - 55
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/KMSClientProvider.java

@@ -34,6 +34,7 @@ import org.apache.hadoop.security.authentication.client.ConnectionConfigurator;
 import org.apache.hadoop.security.ssl.SSLFactory;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.delegation.web.DelegationTokenAuthenticatedURL;
+import org.apache.hadoop.util.HttpExceptionUtils;
 import org.apache.http.client.utils.URIBuilder;
 import org.codehaus.jackson.map.ObjectMapper;
 
@@ -44,7 +45,6 @@ import java.io.InputStream;
 import java.io.OutputStream;
 import java.io.OutputStreamWriter;
 import java.io.Writer;
-import java.lang.reflect.Constructor;
 import java.net.HttpURLConnection;
 import java.net.SocketTimeoutException;
 import java.net.URI;
@@ -54,7 +54,6 @@ import java.net.URLEncoder;
 import java.security.GeneralSecurityException;
 import java.security.NoSuchAlgorithmException;
 import java.security.PrivilegedExceptionAction;
-import java.text.MessageFormat;
 import java.util.ArrayList;
 import java.util.Date;
 import java.util.HashMap;
@@ -413,58 +412,6 @@ public class KMSClientProvider extends KeyProvider implements CryptoExtension,
     return conn;
   }
 
-  // trick, riding on generics to throw an undeclared exception
-
-  private static void throwEx(Throwable ex) {
-    KMSClientProvider.<RuntimeException>throwException(ex);
-  }
-
-  @SuppressWarnings("unchecked")
-  private static <E extends Throwable> void throwException(Throwable ex)
-      throws E {
-    throw (E) ex;
-  }
-
-  @SuppressWarnings("unchecked")
-  private static void validateResponse(HttpURLConnection conn, int expected)
-      throws IOException {
-    int status = conn.getResponseCode();
-    if (status != expected) {
-      InputStream es = null;
-      try {
-        Exception toThrow;
-        String contentType = conn.getHeaderField(CONTENT_TYPE);
-        if (contentType != null &&
-            contentType.toLowerCase().startsWith(APPLICATION_JSON_MIME)) {
-          es = conn.getErrorStream();
-          ObjectMapper mapper = new ObjectMapper();
-          Map json = mapper.readValue(es, Map.class);
-          String exClass = (String) json.get(
-              KMSRESTConstants.ERROR_EXCEPTION_JSON);
-          String exMsg = (String)
-              json.get(KMSRESTConstants.ERROR_MESSAGE_JSON);
-          try {
-            ClassLoader cl = KMSClientProvider.class.getClassLoader();
-            Class klass = cl.loadClass(exClass);
-            Constructor constr = klass.getConstructor(String.class);
-            toThrow = (Exception) constr.newInstance(exMsg);
-          } catch (Exception ex) {
-            toThrow = new IOException(MessageFormat.format(
-                "HTTP status [{0}], {1}", status, conn.getResponseMessage()));
-          }
-        } else {
-          toThrow = new IOException(MessageFormat.format(
-              "HTTP status [{0}], {1}", status, conn.getResponseMessage()));
-        }
-        throwEx(toThrow);
-      } finally {
-        if (es != null) {
-          es.close();
-        }
-      }
-    }
-  }
-
   private static <T> T call(HttpURLConnection conn, Map jsonOutput,
       int expectedResponse, Class<T> klass)
       throws IOException {
@@ -477,7 +424,7 @@ public class KMSClientProvider extends KeyProvider implements CryptoExtension,
       conn.getInputStream().close();
       throw ex;
     }
-    validateResponse(conn, expectedResponse);
+    HttpExceptionUtils.validateResponse(conn, expectedResponse);
     if (APPLICATION_JSON_MIME.equalsIgnoreCase(conn.getContentType())
         && klass != null) {
       ObjectMapper mapper = new ObjectMapper();

+ 18 - 3
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/random/OsSecureRandom.java

@@ -23,6 +23,8 @@ import java.io.FileInputStream;
 import java.io.IOException;
 import java.util.Random;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.conf.Configuration;
@@ -37,6 +39,8 @@ import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY
  */
 @InterfaceAudience.Private
 public class OsSecureRandom extends Random implements Closeable, Configurable {
+  public static final Log LOG = LogFactory.getLog(OsSecureRandom.class);
+  
   private static final long serialVersionUID = 6391500337172057900L;
 
   private transient Configuration conf;
@@ -72,12 +76,20 @@ public class OsSecureRandom extends Random implements Closeable, Configurable {
         HADOOP_SECURITY_SECURE_RANDOM_DEVICE_FILE_PATH_KEY,
         HADOOP_SECURITY_SECURE_RANDOM_DEVICE_FILE_PATH_DEFAULT);
     File randomDevFile = new File(randomDevPath);
+
     try {
+      close();
       this.stream = new FileInputStream(randomDevFile);
-      fillReservoir(0);
     } catch (IOException e) {
       throw new RuntimeException(e);
     }
+
+    try {
+      fillReservoir(0);
+    } catch (RuntimeException e) {
+      close();
+      throw e;
+    }
   }
 
   @Override
@@ -109,7 +121,10 @@ public class OsSecureRandom extends Random implements Closeable, Configurable {
   }
 
   @Override
-  synchronized public void close() throws IOException {
-    stream.close();
+  synchronized public void close() {
+    if (stream != null) {
+      IOUtils.cleanup(LOG, stream);
+      stream = null;
+    }
   }
 }

+ 11 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/authorize/AccessControlList.java

@@ -221,7 +221,13 @@ public class AccessControlList implements Writable {
     return groups;
   }
 
-  public boolean isUserAllowed(UserGroupInformation ugi) {
+  /**
+   * Checks if a user represented by the provided {@link UserGroupInformation}
+   * is a member of the Access Control List
+   * @param ugi UserGroupInformation to check if contained in the ACL
+   * @return true if ugi is member of the list
+   */
+  public final boolean isUserInList(UserGroupInformation ugi) {
     if (allAllowed || users.contains(ugi.getShortUserName())) {
       return true;
     } else {
@@ -234,6 +240,10 @@ public class AccessControlList implements Writable {
     return false;
   }
 
+  public boolean isUserAllowed(UserGroupInformation ugi) {
+    return isUserInList(ugi);
+  }
+
   /**
    * Returns descriptive way of users and groups that are part of this ACL.
    * Use {@link #getAclString()} to get the exact String that can be given to

+ 3 - 12
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticationFilter.java

@@ -31,6 +31,7 @@ import org.apache.hadoop.security.authentication.server.PseudoAuthenticationHand
 import org.apache.hadoop.security.authorize.AuthorizationException;
 import org.apache.hadoop.security.authorize.ProxyUsers;
 import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenSecretManager;
+import org.apache.hadoop.util.HttpExceptionUtils;
 import org.apache.http.NameValuePair;
 import org.apache.http.client.utils.URLEncodedUtils;
 import org.codehaus.jackson.map.ObjectMapper;
@@ -221,18 +222,8 @@ public class DelegationTokenAuthenticationFilter
           try {
             ProxyUsers.authorize(ugi, request.getRemoteHost());
           } catch (AuthorizationException ex) {
-            String msg = String.format(
-                "User '%s' from host '%s' not allowed to impersonate user '%s'",
-                realUser, request.getRemoteHost(), doAsUser);
-            response.setStatus(HttpServletResponse.SC_FORBIDDEN);
-            response.setContentType(APPLICATION_JSON_MIME);
-            Map<String, String> json = new HashMap<String, String>();
-            json.put(ERROR_EXCEPTION_JSON,
-                AuthorizationException.class.getName());
-            json.put(ERROR_MESSAGE_JSON, msg);
-            Writer writer = response.getWriter();
-            ObjectMapper jsonMapper = new ObjectMapper();
-            jsonMapper.writeValue(writer, json);
+            HttpExceptionUtils.createServletExceptionResponse(response,
+                HttpServletResponse.SC_FORBIDDEN, ex);
             requestCompleted = true;
           }
         }

+ 4 - 2
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticationHandler.java

@@ -29,6 +29,7 @@ import org.apache.hadoop.security.authentication.server.AuthenticationToken;
 import org.apache.hadoop.security.authentication.server.KerberosAuthenticationHandler;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenSecretManager;
+import org.apache.hadoop.util.HttpExceptionUtils;
 import org.codehaus.jackson.map.ObjectMapper;
 
 import javax.servlet.ServletException;
@@ -346,8 +347,9 @@ public abstract class DelegationTokenAuthenticationHandler
         token.setExpires(0);
         request.setAttribute(DELEGATION_TOKEN_UGI_ATTRIBUTE, ugi);
       } catch (Throwable ex) {
-        throw new AuthenticationException("Could not verify DelegationToken, " +
-            ex.toString(), ex);
+        token = null;
+        HttpExceptionUtils.createServletExceptionResponse(response,
+            HttpServletResponse.SC_FORBIDDEN, new AuthenticationException(ex));
       }
     } else {
       token = authHandler.authenticate(request, response);

+ 2 - 18
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticator.java

@@ -26,6 +26,7 @@ import org.apache.hadoop.security.authentication.client.Authenticator;
 import org.apache.hadoop.security.authentication.client.ConnectionConfigurator;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenIdentifier;
+import org.apache.hadoop.util.HttpExceptionUtils;
 import org.codehaus.jackson.map.ObjectMapper;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -217,7 +218,7 @@ public abstract class DelegationTokenAuthenticator implements Authenticator {
     AuthenticatedURL aUrl = new AuthenticatedURL(this);
     HttpURLConnection conn = aUrl.openConnection(url, token);
     conn.setRequestMethod(operation.getHttpMethod());
-    validateResponse(conn, HttpURLConnection.HTTP_OK);
+    HttpExceptionUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
     if (hasResponse) {
       String contentType = conn.getHeaderField(CONTENT_TYPE);
       contentType = (contentType != null) ? contentType.toLowerCase()
@@ -241,21 +242,4 @@ public abstract class DelegationTokenAuthenticator implements Authenticator {
     return ret;
   }
 
-  @SuppressWarnings("unchecked")
-  private static void validateResponse(HttpURLConnection conn, int expected)
-      throws IOException {
-    int status = conn.getResponseCode();
-    if (status != expected) {
-      try {
-        conn.getInputStream().close();
-      } catch (IOException ex) {
-        //NOP
-      }
-      String msg = String.format("HTTP status, expected [%d], got [%d]: %s", 
-          expected, status, conn.getResponseMessage());
-      LOG.debug(msg);
-      throw new IOException(msg);
-    }
-  }
-
 }

+ 185 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/HttpExceptionUtils.java

@@ -0,0 +1,185 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.util;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.codehaus.jackson.map.ObjectMapper;
+
+import javax.servlet.http.HttpServletResponse;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.Writer;
+import java.lang.reflect.Constructor;
+import java.net.HttpURLConnection;
+import java.util.LinkedHashMap;
+import java.util.Map;
+
+/**
+ * HTTP utility class to help propagate server side exception to the client
+ * over HTTP as a JSON payload.
+ * <p/>
+ * It creates HTTP Servlet and JAX-RPC error responses including details of the
+ * exception that allows a client to recreate the remote exception.
+ * <p/>
+ * It parses HTTP client connections and recreates the exception.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class HttpExceptionUtils {
+
+  public static final String ERROR_JSON = "RemoteException";
+  public static final String ERROR_EXCEPTION_JSON = "exception";
+  public static final String ERROR_CLASSNAME_JSON = "javaClassName";
+  public static final String ERROR_MESSAGE_JSON = "message";
+
+  private static final String APPLICATION_JSON_MIME = "application/json";
+
+  private static final String ENTER = System.getProperty("line.separator");
+
+  /**
+   * Creates a HTTP servlet response serializing the exception in it as JSON.
+   *
+   * @param response the servlet response
+   * @param status the error code to set in the response
+   * @param ex the exception to serialize in the response
+   * @throws IOException thrown if there was an error while creating the
+   * response
+   */
+  public static void createServletExceptionResponse(
+      HttpServletResponse response, int status, Throwable ex)
+      throws IOException {
+    response.setStatus(status);
+    response.setContentType(APPLICATION_JSON_MIME);
+    Map<String, Object> json = new LinkedHashMap<String, Object>();
+    json.put(ERROR_MESSAGE_JSON, getOneLineMessage(ex));
+    json.put(ERROR_EXCEPTION_JSON, ex.getClass().getSimpleName());
+    json.put(ERROR_CLASSNAME_JSON, ex.getClass().getName());
+    Map<String, Object> jsonResponse = new LinkedHashMap<String, Object>();
+    jsonResponse.put(ERROR_JSON, json);
+    ObjectMapper jsonMapper = new ObjectMapper();
+    Writer writer = response.getWriter();
+    jsonMapper.writerWithDefaultPrettyPrinter().writeValue(writer, jsonResponse);
+    writer.flush();
+  }
+
+  /**
+   * Creates a HTTP JAX-RPC response serializing the exception in it as JSON.
+   *
+   * @param status the error code to set in the response
+   * @param ex the exception to serialize in the response
+   * @return the JAX-RPC response with the set error and JSON encoded exception
+   */
+  public static Response createJerseyExceptionResponse(Response.Status status,
+      Throwable ex) {
+    Map<String, Object> json = new LinkedHashMap<String, Object>();
+    json.put(ERROR_MESSAGE_JSON, getOneLineMessage(ex));
+    json.put(ERROR_EXCEPTION_JSON, ex.getClass().getSimpleName());
+    json.put(ERROR_CLASSNAME_JSON, ex.getClass().getName());
+    Map<String, Object> response = new LinkedHashMap<String, Object>();
+    response.put(ERROR_JSON, json);
+    return Response.status(status).type(MediaType.APPLICATION_JSON).
+        entity(response).build();
+  }
+
+  private static String getOneLineMessage(Throwable exception) {
+    String message = exception.getMessage();
+    if (message != null) {
+      int i = message.indexOf(ENTER);
+      if (i > -1) {
+        message = message.substring(0, i);
+      }
+    }
+    return message;
+  }
+
+  // trick, riding on generics to throw an undeclared exception
+
+  private static void throwEx(Throwable ex) {
+    HttpExceptionUtils.<RuntimeException>throwException(ex);
+  }
+
+  @SuppressWarnings("unchecked")
+  private static <E extends Throwable> void throwException(Throwable ex)
+      throws E {
+    throw (E) ex;
+  }
+
+  /**
+   * Validates the status of an <code>HttpURLConnection</code> against an
+   * expected HTTP status code. If the current status code is not the expected
+   * one it throws an exception with a detail message using Server side error
+   * messages if available.
+   * <p/>
+   * <b>NOTE:</b> this method will throw the deserialized exception even if not
+   * declared in the <code>throws</code> of the method signature.
+   *
+   * @param conn the <code>HttpURLConnection</code>.
+   * @param expectedStatus the expected HTTP status code.
+   * @throws IOException thrown if the current status code does not match the
+   * expected one.
+   */
+  @SuppressWarnings("unchecked")
+  public static void validateResponse(HttpURLConnection conn,
+      int expectedStatus) throws IOException {
+    if (conn.getResponseCode() != expectedStatus) {
+      Exception toThrow;
+      InputStream es = null;
+      try {
+        es = conn.getErrorStream();
+        ObjectMapper mapper = new ObjectMapper();
+        Map json = mapper.readValue(es, Map.class);
+        json = (Map) json.get(ERROR_JSON);
+        String exClass = (String) json.get(ERROR_CLASSNAME_JSON);
+        String exMsg = (String) json.get(ERROR_MESSAGE_JSON);
+        if (exClass != null) {
+          try {
+            ClassLoader cl = HttpExceptionUtils.class.getClassLoader();
+            Class klass = cl.loadClass(exClass);
+            Constructor constr = klass.getConstructor(String.class);
+            toThrow = (Exception) constr.newInstance(exMsg);
+          } catch (Exception ex) {
+            toThrow = new IOException(String.format(
+                "HTTP status [%d], exception [%s], message [%s] ",
+                conn.getResponseCode(), exClass, exMsg));
+          }
+        } else {
+          String msg = (exMsg != null) ? exMsg : conn.getResponseMessage();
+          toThrow = new IOException(String.format(
+              "HTTP status [%d], message [%s]", conn.getResponseCode(), msg));
+        }
+      } catch (Exception ex) {
+        toThrow = new IOException(String.format(
+            "HTTP status [%d], message [%s]", conn.getResponseCode(),
+            conn.getResponseMessage()));
+      } finally {
+        if (es != null) {
+          try {
+            es.close();
+          } catch (IOException ex) {
+            //ignore
+          }
+        }
+      }
+      throwEx(toThrow);
+    }
+  }
+
+}

+ 51 - 18
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/TestCryptoCodec.java

@@ -52,35 +52,40 @@ public class TestCryptoCodec {
   private Configuration conf = new Configuration();
   private int count = 10000;
   private int seed = new Random().nextInt();
+  private final String jceCodecClass = 
+      "org.apache.hadoop.crypto.JceAesCtrCryptoCodec";
+  private final String opensslCodecClass = 
+      "org.apache.hadoop.crypto.OpensslAesCtrCryptoCodec";
   
   @Test(timeout=120000)
   public void testJceAesCtrCryptoCodec() throws Exception {
-    cryptoCodecTest(conf, seed, 0, 
-        "org.apache.hadoop.crypto.JceAesCtrCryptoCodec");
-    cryptoCodecTest(conf, seed, count, 
-        "org.apache.hadoop.crypto.JceAesCtrCryptoCodec");
+    Assume.assumeTrue(NativeCodeLoader.buildSupportsOpenssl());
+    Assert.assertEquals(null, OpensslCipher.getLoadingFailureReason());
+    cryptoCodecTest(conf, seed, 0, jceCodecClass, jceCodecClass);
+    cryptoCodecTest(conf, seed, count, jceCodecClass, jceCodecClass);
+    cryptoCodecTest(conf, seed, count, jceCodecClass, opensslCodecClass);
   }
   
-  @Test(timeout=1200000)
+  @Test(timeout=120000)
   public void testOpensslAesCtrCryptoCodec() throws Exception {
     Assume.assumeTrue(NativeCodeLoader.buildSupportsOpenssl());
     Assert.assertEquals(null, OpensslCipher.getLoadingFailureReason());
-    cryptoCodecTest(conf, seed, 0, 
-        "org.apache.hadoop.crypto.OpensslAesCtrCryptoCodec");
-    cryptoCodecTest(conf, seed, count, 
-        "org.apache.hadoop.crypto.OpensslAesCtrCryptoCodec");
+    cryptoCodecTest(conf, seed, 0, opensslCodecClass, opensslCodecClass);
+    cryptoCodecTest(conf, seed, count, opensslCodecClass, opensslCodecClass);
+    cryptoCodecTest(conf, seed, count, opensslCodecClass, jceCodecClass);
   }
   
   private void cryptoCodecTest(Configuration conf, int seed, int count, 
-      String codecClass) throws IOException, GeneralSecurityException {
-    CryptoCodec codec = null;
+      String encCodecClass, String decCodecClass) throws IOException, 
+      GeneralSecurityException {
+    CryptoCodec encCodec = null;
     try {
-      codec = (CryptoCodec)ReflectionUtils.newInstance(
-          conf.getClassByName(codecClass), conf);
+      encCodec = (CryptoCodec)ReflectionUtils.newInstance(
+          conf.getClassByName(encCodecClass), conf);
     } catch (ClassNotFoundException cnfe) {
       throw new IOException("Illegal crypto codec!");
     }
-    LOG.info("Created a Codec object of type: " + codecClass);
+    LOG.info("Created a Codec object of type: " + encCodecClass);
     
     // Generate data
     DataOutputBuffer data = new DataOutputBuffer();
@@ -98,18 +103,27 @@ public class TestCryptoCodec {
     // Encrypt data
     DataOutputBuffer encryptedDataBuffer = new DataOutputBuffer();
     CryptoOutputStream out = new CryptoOutputStream(encryptedDataBuffer, 
-        codec, bufferSize, key, iv);
+        encCodec, bufferSize, key, iv);
     out.write(data.getData(), 0, data.getLength());
     out.flush();
     out.close();
     LOG.info("Finished encrypting data");
     
+    CryptoCodec decCodec = null;
+    try {
+      decCodec = (CryptoCodec)ReflectionUtils.newInstance(
+          conf.getClassByName(decCodecClass), conf);
+    } catch (ClassNotFoundException cnfe) {
+      throw new IOException("Illegal crypto codec!");
+    }
+    LOG.info("Created a Codec object of type: " + decCodecClass);
+    
     // Decrypt data
     DataInputBuffer decryptedDataBuffer = new DataInputBuffer();
     decryptedDataBuffer.reset(encryptedDataBuffer.getData(), 0, 
         encryptedDataBuffer.getLength());
     CryptoInputStream in = new CryptoInputStream(decryptedDataBuffer, 
-        codec, bufferSize, key, iv);
+        decCodec, bufferSize, key, iv);
     DataInputStream dataIn = new DataInputStream(new BufferedInputStream(in));
     
     // Check
@@ -146,7 +160,7 @@ public class TestCryptoCodec {
     decryptedDataBuffer.reset(encryptedDataBuffer.getData(), 0, 
         encryptedDataBuffer.getLength());
     in = new CryptoInputStream(decryptedDataBuffer, 
-        codec, bufferSize, key, iv);
+        decCodec, bufferSize, key, iv);
 
     // Check
     originalIn = new DataInputStream(new BufferedInputStream(originalData));
@@ -156,11 +170,30 @@ public class TestCryptoCodec {
       assertEquals("Decrypted stream read by byte does not match",
         expected, in.read());
     } while (expected != -1);
+    
+    // Seek to a certain position and decrypt
+    originalData.reset(data.getData(), 0, data.getLength());
+    decryptedDataBuffer.reset(encryptedDataBuffer.getData(), 0,
+        encryptedDataBuffer.getLength());
+    in = new CryptoInputStream(new TestCryptoStreams.FakeInputStream(
+        decryptedDataBuffer), decCodec, bufferSize, key, iv);
+    int seekPos = data.getLength() / 3;
+    in.seek(seekPos);
+    
+    // Check
+    TestCryptoStreams.FakeInputStream originalInput = 
+        new TestCryptoStreams.FakeInputStream(originalData);
+    originalInput.seek(seekPos);
+    do {
+      expected = originalInput.read();
+      assertEquals("Decrypted stream read by byte does not match",
+        expected, in.read());
+    } while (expected != -1);
 
     LOG.info("SUCCESS! Completed checking " + count + " records");
     
     // Check secure random generator
-    testSecureRandom(codec);
+    testSecureRandom(encCodec);
   }
   
   /** Test secure random generator */

+ 1 - 1
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/TestCryptoStreams.java

@@ -159,7 +159,7 @@ public class TestCryptoStreams extends CryptoStreamsTestBase {
     }
   }
   
-  private class FakeInputStream extends InputStream implements 
+  public static class FakeInputStream extends InputStream implements 
       Seekable, PositionedReadable, ByteBufferReadable, HasFileDescriptor, 
       CanSetDropBehind, CanSetReadahead, HasEnhancedByteBufferAccess {
     private final byte[] oneByteBuf = new byte[1];

+ 13 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/key/TestKeyProviderFactory.java

@@ -357,4 +357,17 @@ public class TestKeyProviderFactory {
     }
   }
 
+  @Test
+  public void testGetProviderViaURI() throws Exception {
+    Configuration conf = new Configuration(false);
+    URI uri = new URI(JavaKeyStoreProvider.SCHEME_NAME + "://file" + tmpDir +
+        "/test.jks");
+    KeyProvider kp = KeyProviderFactory.get(uri, conf);
+    Assert.assertNotNull(kp);
+    Assert.assertEquals(JavaKeyStoreProvider.class, kp.getClass());
+    uri = new URI("foo://bar");
+    kp = KeyProviderFactory.get(uri, conf);
+    Assert.assertNull(kp);
+
+  }
 }

+ 15 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/random/TestOsSecureRandom.java

@@ -22,6 +22,7 @@ import java.util.Arrays;
 
 import org.apache.commons.lang.SystemUtils;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.util.Shell.ShellCommandExecutor;
 import org.junit.Assume;
 import org.junit.Test;
 
@@ -136,4 +137,18 @@ public class TestOsSecureRandom {
     }
     random.close();
   }
+  
+  @Test(timeout=120000)
+  public void testOsSecureRandomSetConf() throws IOException {
+    Assume.assumeTrue(SystemUtils.IS_OS_LINUX);
+    OsSecureRandom random = new OsSecureRandom();
+    for(int n = 0; n < 10; ++n) {
+      random.setConf(new Configuration());
+      String[] scmd = new String[] {"/bin/sh", "-c", "lsof | wc -l"};
+      ShellCommandExecutor sce = new ShellCommandExecutor(scmd);
+      sce.execute(); 
+      System.out.println("==lsof result " + n + ":");
+      System.out.println(sce.getOutput());
+    }
+  }
 }

+ 14 - 21
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/delegation/web/TestDelegationTokenAuthenticationHandlerWithMocks.java

@@ -25,6 +25,7 @@ import org.apache.hadoop.security.authentication.server.AuthenticationHandler;
 import org.apache.hadoop.security.authentication.server.AuthenticationToken;
 import org.apache.hadoop.security.token.SecretManager;
 import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.util.HttpExceptionUtils;
 import org.codehaus.jackson.map.ObjectMapper;
 import org.junit.After;
 import org.junit.Assert;
@@ -224,7 +225,8 @@ public class TestDelegationTokenAuthenticationHandlerWithMocks {
     Mockito.when(request.getQueryString()).thenReturn(
         DelegationTokenAuthenticator.OP_PARAM + "=" + op.toString() + "&" +
             DelegationTokenAuthenticator.TOKEN_PARAM + "=" +
-            token.encodeToUrlString());
+            token.encodeToUrlString()
+    );
     Assert.assertFalse(handler.managementOperation(null, request, response));
     Mockito.verify(response).setStatus(HttpServletResponse.SC_OK);
     try {
@@ -273,8 +275,8 @@ public class TestDelegationTokenAuthenticationHandlerWithMocks {
             UserGroupInformation.getCurrentUser(), "user");
     Mockito.when(request.getQueryString()).
         thenReturn(DelegationTokenAuthenticator.OP_PARAM + "=" + op.toString() +
-        "&" + DelegationTokenAuthenticator.TOKEN_PARAM + "=" +
-        dToken.encodeToUrlString());
+            "&" + DelegationTokenAuthenticator.TOKEN_PARAM + "=" +
+            dToken.encodeToUrlString());
     Assert.assertFalse(handler.managementOperation(token, request, response));
     Mockito.verify(response).setStatus(HttpServletResponse.SC_OK);
     pwriter.close();
@@ -333,15 +335,11 @@ public class TestDelegationTokenAuthenticationHandlerWithMocks {
     HttpServletResponse response = Mockito.mock(HttpServletResponse.class);
     Mockito.when(request.getQueryString()).thenReturn(
         DelegationTokenAuthenticator.DELEGATION_PARAM + "=invalid");
-
-    try {
-      handler.authenticate(request, response);
-      Assert.fail();
-    } catch (AuthenticationException ex) {
-      //NOP
-    } catch (Exception ex) {
-      Assert.fail();
-    }
+    StringWriter writer = new StringWriter();
+    Mockito.when(response.getWriter()).thenReturn(new PrintWriter(writer));
+    Assert.assertNull(handler.authenticate(request, response));
+    Mockito.verify(response).setStatus(HttpServletResponse.SC_FORBIDDEN);
+    Assert.assertTrue(writer.toString().contains("AuthenticationException"));
   }
 
   private void testInvalidDelegationTokenHeader() throws Exception {
@@ -350,15 +348,10 @@ public class TestDelegationTokenAuthenticationHandlerWithMocks {
     Mockito.when(request.getHeader(Mockito.eq(
         DelegationTokenAuthenticator.DELEGATION_TOKEN_HEADER))).thenReturn(
         "invalid");
-
-    try {
-      handler.authenticate(request, response);
-      Assert.fail();
-    } catch (AuthenticationException ex) {
-      //NOP
-    } catch (Exception ex) {
-      Assert.fail();
-    }
+    StringWriter writer = new StringWriter();
+    Mockito.when(response.getWriter()).thenReturn(new PrintWriter(writer));
+    Assert.assertNull(handler.authenticate(request, response));
+    Assert.assertTrue(writer.toString().contains("AuthenticationException"));
   }
 
 }

+ 167 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestHttpExceptionUtils.java

@@ -0,0 +1,167 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.util;
+
+import org.codehaus.jackson.map.ObjectMapper;
+import org.junit.Assert;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import javax.servlet.http.HttpServletResponse;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.PrintWriter;
+import java.io.StringWriter;
+import java.net.HttpURLConnection;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+
+public class TestHttpExceptionUtils {
+
+  @Test
+  public void testCreateServletException() throws IOException {
+    StringWriter writer = new StringWriter();
+    PrintWriter printWriter = new PrintWriter(writer);
+    HttpServletResponse response = Mockito.mock(HttpServletResponse.class);
+    Mockito.when(response.getWriter()).thenReturn(printWriter);
+    int status = HttpServletResponse.SC_INTERNAL_SERVER_ERROR;
+    Exception ex = new IOException("Hello IOEX");
+    HttpExceptionUtils.createServletExceptionResponse(response, status, ex);
+    Mockito.verify(response).setStatus(status);
+    Mockito.verify(response).setContentType(Mockito.eq("application/json"));
+    ObjectMapper mapper = new ObjectMapper();
+    Map json = mapper.readValue(writer.toString(), Map.class);
+    json = (Map) json.get(HttpExceptionUtils.ERROR_JSON);
+    Assert.assertEquals(IOException.class.getName(),
+        json.get(HttpExceptionUtils.ERROR_CLASSNAME_JSON));
+    Assert.assertEquals(IOException.class.getSimpleName(),
+        json.get(HttpExceptionUtils.ERROR_EXCEPTION_JSON));
+    Assert.assertEquals("Hello IOEX",
+        json.get(HttpExceptionUtils.ERROR_MESSAGE_JSON));
+  }
+
+  @Test
+  public void testCreateJerseyException() throws IOException {
+    Exception ex = new IOException("Hello IOEX");
+    Response response = HttpExceptionUtils.createJerseyExceptionResponse(
+        Response.Status.INTERNAL_SERVER_ERROR, ex);
+    Assert.assertEquals(Response.Status.INTERNAL_SERVER_ERROR.getStatusCode(),
+        response.getStatus());
+    Assert.assertArrayEquals(
+        Arrays.asList(MediaType.APPLICATION_JSON_TYPE).toArray(),
+        response.getMetadata().get("Content-Type").toArray());
+    Map entity = (Map) response.getEntity();
+    entity = (Map) entity.get(HttpExceptionUtils.ERROR_JSON);
+    Assert.assertEquals(IOException.class.getName(),
+        entity.get(HttpExceptionUtils.ERROR_CLASSNAME_JSON));
+    Assert.assertEquals(IOException.class.getSimpleName(),
+        entity.get(HttpExceptionUtils.ERROR_EXCEPTION_JSON));
+    Assert.assertEquals("Hello IOEX",
+        entity.get(HttpExceptionUtils.ERROR_MESSAGE_JSON));
+  }
+
+  @Test
+  public void testValidateResponseOK() throws IOException {
+    HttpURLConnection conn = Mockito.mock(HttpURLConnection.class);
+    Mockito.when(conn.getResponseCode()).thenReturn(
+        HttpURLConnection.HTTP_CREATED);
+    HttpExceptionUtils.validateResponse(conn, HttpURLConnection.HTTP_CREATED);
+  }
+
+  @Test(expected = IOException.class)
+  public void testValidateResponseFailNoErrorMessage() throws IOException {
+    HttpURLConnection conn = Mockito.mock(HttpURLConnection.class);
+    Mockito.when(conn.getResponseCode()).thenReturn(
+        HttpURLConnection.HTTP_BAD_REQUEST);
+    HttpExceptionUtils.validateResponse(conn, HttpURLConnection.HTTP_CREATED);
+  }
+
+  @Test
+  public void testValidateResponseNonJsonErrorMessage() throws IOException {
+    String msg = "stream";
+    InputStream is = new ByteArrayInputStream(msg.getBytes());
+    HttpURLConnection conn = Mockito.mock(HttpURLConnection.class);
+    Mockito.when(conn.getErrorStream()).thenReturn(is);
+    Mockito.when(conn.getResponseMessage()).thenReturn("msg");
+    Mockito.when(conn.getResponseCode()).thenReturn(
+        HttpURLConnection.HTTP_BAD_REQUEST);
+    try {
+      HttpExceptionUtils.validateResponse(conn, HttpURLConnection.HTTP_CREATED);
+      Assert.fail();
+    } catch (IOException ex) {
+      Assert.assertTrue(ex.getMessage().contains("msg"));
+      Assert.assertTrue(ex.getMessage().contains("" +
+          HttpURLConnection.HTTP_BAD_REQUEST));
+    }
+  }
+
+  @Test
+  public void testValidateResponseJsonErrorKnownException() throws IOException {
+    Map<String, Object> json = new HashMap<String, Object>();
+    json.put(HttpExceptionUtils.ERROR_EXCEPTION_JSON, IllegalStateException.class.getSimpleName());
+    json.put(HttpExceptionUtils.ERROR_CLASSNAME_JSON, IllegalStateException.class.getName());
+    json.put(HttpExceptionUtils.ERROR_MESSAGE_JSON, "EX");
+    Map<String, Object> response = new HashMap<String, Object>();
+    response.put(HttpExceptionUtils.ERROR_JSON, json);
+    ObjectMapper jsonMapper = new ObjectMapper();
+    String msg = jsonMapper.writeValueAsString(response);
+    InputStream is = new ByteArrayInputStream(msg.getBytes());
+    HttpURLConnection conn = Mockito.mock(HttpURLConnection.class);
+    Mockito.when(conn.getErrorStream()).thenReturn(is);
+    Mockito.when(conn.getResponseMessage()).thenReturn("msg");
+    Mockito.when(conn.getResponseCode()).thenReturn(
+        HttpURLConnection.HTTP_BAD_REQUEST);
+    try {
+      HttpExceptionUtils.validateResponse(conn, HttpURLConnection.HTTP_CREATED);
+      Assert.fail();
+    } catch (IllegalStateException ex) {
+      Assert.assertEquals("EX", ex.getMessage());
+    }
+  }
+
+  @Test
+  public void testValidateResponseJsonErrorUnknownException()
+      throws IOException {
+    Map<String, Object> json = new HashMap<String, Object>();
+    json.put(HttpExceptionUtils.ERROR_EXCEPTION_JSON, "FooException");
+    json.put(HttpExceptionUtils.ERROR_CLASSNAME_JSON, "foo.FooException");
+    json.put(HttpExceptionUtils.ERROR_MESSAGE_JSON, "EX");
+    Map<String, Object> response = new HashMap<String, Object>();
+    response.put(HttpExceptionUtils.ERROR_JSON, json);
+    ObjectMapper jsonMapper = new ObjectMapper();
+    String msg = jsonMapper.writeValueAsString(response);
+    InputStream is = new ByteArrayInputStream(msg.getBytes());
+    HttpURLConnection conn = Mockito.mock(HttpURLConnection.class);
+    Mockito.when(conn.getErrorStream()).thenReturn(is);
+    Mockito.when(conn.getResponseMessage()).thenReturn("msg");
+    Mockito.when(conn.getResponseCode()).thenReturn(
+        HttpURLConnection.HTTP_BAD_REQUEST);
+    try {
+      HttpExceptionUtils.validateResponse(conn, HttpURLConnection.HTTP_CREATED);
+      Assert.fail();
+    } catch (IOException ex) {
+      Assert.assertTrue(ex.getMessage().contains("EX"));
+      Assert.assertTrue(ex.getMessage().contains("foo.FooException"));
+    }
+  }
+
+}

+ 1 - 0
hadoop-common-project/hadoop-kms/pom.xml

@@ -243,6 +243,7 @@
               <goal>war</goal>
             </goals>
             <configuration>
+              <archiveClasses>true</archiveClasses>
               <warName>kms</warName>
               <webappDirectory>${project.build.directory}/kms
               </webappDirectory>

+ 6 - 21
hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMS.java

@@ -26,10 +26,10 @@ import org.apache.hadoop.crypto.key.KeyProviderCryptoExtension.EncryptedKeyVersi
 import org.apache.hadoop.crypto.key.kms.KMSRESTConstants;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.security.authorize.AuthorizationException;
 import org.apache.hadoop.crypto.key.kms.KMSClientProvider;
 import org.apache.hadoop.security.token.delegation.web.HttpUserGroupInformation;
 
+
 import javax.ws.rs.Consumes;
 import javax.ws.rs.DELETE;
 import javax.ws.rs.DefaultValue;
@@ -73,29 +73,14 @@ public class KMS {
     kmsAudit= KMSWebApp.getKMSAudit();
   }
 
-
-  private static final String UNAUTHORIZED_MSG_WITH_KEY = 
-      "User:%s not allowed to do '%s' on '%s'";
-  
-  private static final String UNAUTHORIZED_MSG_WITHOUT_KEY = 
-      "User:%s not allowed to do '%s'";
-
   private void assertAccess(KMSACLs.Type aclType, UserGroupInformation ugi,
       KMSOp operation) throws AccessControlException {
-    assertAccess(aclType, ugi, operation, null);
+    KMSWebApp.getACLs().assertAccess(aclType, ugi, operation, null);
   }
-
-  private void assertAccess(KMSACLs.Type aclType,
-      UserGroupInformation ugi, KMSOp operation, String key)
-      throws AccessControlException {
-    if (!KMSWebApp.getACLs().hasAccess(aclType, ugi)) {
-      KMSWebApp.getUnauthorizedCallsMeter().mark();
-      kmsAudit.unauthorized(ugi, operation, key);
-      throw new AuthorizationException(String.format(
-          (key != null) ? UNAUTHORIZED_MSG_WITH_KEY 
-                        : UNAUTHORIZED_MSG_WITHOUT_KEY,
-          ugi.getShortUserName(), operation, key));
-    }
+  
+  private void assertAccess(KMSACLs.Type aclType, UserGroupInformation ugi,
+      KMSOp operation, String key) throws AccessControlException {
+    KMSWebApp.getACLs().assertAccess(aclType, ugi, operation, key);
   }
 
   private static KeyProvider.KeyVersion removeKeyMaterial(

+ 51 - 4
hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSACLs.java

@@ -19,8 +19,11 @@ package org.apache.hadoop.crypto.key.kms.server;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.crypto.key.kms.server.KMS.KMSOp;
+import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authorize.AccessControlList;
+import org.apache.hadoop.security.authorize.AuthorizationException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -39,14 +42,23 @@ import java.util.concurrent.TimeUnit;
 public class KMSACLs implements Runnable {
   private static final Logger LOG = LoggerFactory.getLogger(KMSACLs.class);
 
+  private static final String UNAUTHORIZED_MSG_WITH_KEY =
+      "User:%s not allowed to do '%s' on '%s'";
+
+  private static final String UNAUTHORIZED_MSG_WITHOUT_KEY =
+      "User:%s not allowed to do '%s'";
 
   public enum Type {
     CREATE, DELETE, ROLLOVER, GET, GET_KEYS, GET_METADATA,
     SET_KEY_MATERIAL, GENERATE_EEK, DECRYPT_EEK;
 
-    public String getConfigKey() {
+    public String getAclConfigKey() {
       return KMSConfiguration.CONFIG_PREFIX + "acl." + this.toString();
     }
+
+    public String getBlacklistConfigKey() {
+      return KMSConfiguration.CONFIG_PREFIX + "blacklist." + this.toString();
+    }
   }
 
   public static final String ACL_DEFAULT = AccessControlList.WILDCARD_ACL_VALUE;
@@ -54,6 +66,7 @@ public class KMSACLs implements Runnable {
   public static final int RELOADER_SLEEP_MILLIS = 1000;
 
   private volatile Map<Type, AccessControlList> acls;
+  private volatile Map<Type, AccessControlList> blacklistedAcls;
   private ScheduledExecutorService executorService;
   private long lastReload;
 
@@ -70,12 +83,20 @@ public class KMSACLs implements Runnable {
 
   private void setACLs(Configuration conf) {
     Map<Type, AccessControlList> tempAcls = new HashMap<Type, AccessControlList>();
+    Map<Type, AccessControlList> tempBlacklist = new HashMap<Type, AccessControlList>();
     for (Type aclType : Type.values()) {
-      String aclStr = conf.get(aclType.getConfigKey(), ACL_DEFAULT);
+      String aclStr = conf.get(aclType.getAclConfigKey(), ACL_DEFAULT);
       tempAcls.put(aclType, new AccessControlList(aclStr));
+      String blacklistStr = conf.get(aclType.getBlacklistConfigKey());
+      if (blacklistStr != null) {
+        // Only add if blacklist is present
+        tempBlacklist.put(aclType, new AccessControlList(blacklistStr));
+        LOG.info("'{}' Blacklist '{}'", aclType, blacklistStr);
+      }
       LOG.info("'{}' ACL '{}'", aclType, aclStr);
     }
     acls = tempAcls;
+    blacklistedAcls = tempBlacklist;
   }
 
   @Override
@@ -109,12 +130,38 @@ public class KMSACLs implements Runnable {
     lastReload = System.currentTimeMillis();
     Configuration conf = KMSConfiguration.getACLsConf();
     // triggering the resource loading.
-    conf.get(Type.CREATE.getConfigKey());
+    conf.get(Type.CREATE.getAclConfigKey());
     return conf;
   }
 
+  /**
+   * First Check if user is in ACL for the KMS operation, if yes, then
+   * return true if user is not present in any configured blacklist for
+   * the operation
+   * @param type KMS Operation
+   * @param ugi UserGroupInformation of user
+   * @return true is user has access
+   */
   public boolean hasAccess(Type type, UserGroupInformation ugi) {
-    return acls.get(type).isUserAllowed(ugi);
+    boolean access = acls.get(type).isUserAllowed(ugi);
+    if (access) {
+      AccessControlList blacklist = blacklistedAcls.get(type);
+      access = (blacklist == null) || !blacklist.isUserInList(ugi);
+    }
+    return access;
+  }
+
+  public void assertAccess(KMSACLs.Type aclType,
+      UserGroupInformation ugi, KMSOp operation, String key)
+      throws AccessControlException {
+    if (!KMSWebApp.getACLs().hasAccess(aclType, ugi)) {
+      KMSWebApp.getUnauthorizedCallsMeter().mark();
+      KMSWebApp.getKMSAudit().unauthorized(ugi, operation, key);
+      throw new AuthorizationException(String.format(
+          (key != null) ? UNAUTHORIZED_MSG_WITH_KEY
+                        : UNAUTHORIZED_MSG_WITHOUT_KEY,
+          ugi.getShortUserName(), operation, key));
+    }
   }
 
 }

+ 2 - 10
hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSExceptionsProvider.java

@@ -21,22 +21,19 @@ import org.apache.hadoop.classification.InterfaceAudience;
 
 import com.sun.jersey.api.container.ContainerException;
 
-import org.apache.hadoop.crypto.key.kms.KMSRESTConstants;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authentication.client.AuthenticationException;
 import org.apache.hadoop.security.authorize.AuthorizationException;
+import org.apache.hadoop.util.HttpExceptionUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import javax.ws.rs.core.MediaType;
 import javax.ws.rs.core.Response;
 import javax.ws.rs.ext.ExceptionMapper;
 import javax.ws.rs.ext.Provider;
 
 import java.io.IOException;
-import java.util.LinkedHashMap;
-import java.util.Map;
 
 /**
  * Jersey provider that converts KMS exceptions into detailed HTTP errors.
@@ -50,12 +47,7 @@ public class KMSExceptionsProvider implements ExceptionMapper<Exception> {
   private static final String ENTER = System.getProperty("line.separator");
 
   protected Response createResponse(Response.Status status, Throwable ex) {
-    Map<String, Object> json = new LinkedHashMap<String, Object>();
-    json.put(KMSRESTConstants.ERROR_EXCEPTION_JSON, ex.getClass().getName());
-    json.put(KMSRESTConstants.ERROR_MESSAGE_JSON, getOneLineMessage(ex));
-    log(status, ex);
-    return Response.status(status).type(MediaType.APPLICATION_JSON).
-        entity(json).build();
+    return HttpExceptionUtils.createJerseyExceptionResponse(status, ex);
   }
 
   protected String getOneLineMessage(Throwable exception) {

+ 86 - 2
hadoop-common-project/hadoop-kms/src/site/apt/index.apt.vm

@@ -274,8 +274,13 @@ $ keytool -genkey -alias tomcat -keyalg RSA
   KMS ACLs configuration are defined in the KMS <<<etc/hadoop/kms-acls.xml>>>
   configuration file. This file is hot-reloaded when it changes.
 
-  KMS supports a fine grained access control via a set ACL
-  configuration properties:
+  KMS supports both fine grained access control as well as blacklist for kms
+  operations via a set ACL configuration properties.
+
+  A user accessing KMS is first checked for inclusion in the Access Control
+  List for the requested operation and then checked for exclusion in the
+  Black list for the operation before access is granted.
+
 
 +---+
   <property>
@@ -288,6 +293,16 @@ $ keytool -genkey -alias tomcat -keyalg RSA
     </description>
   </property>
 
+  <property>
+    <name>hadoop.kms.blacklist.CREATE</name>
+    <value>hdfs,foo</value>
+    <description>
+      Blacklist for create-key operations.
+      If the user does is in the Blacklist, the key material is not returned
+      as part of the response.
+    </description>
+  </property>
+
   <property>
     <name>hadoop.kms.acl.DELETE</name>
     <value>*</value>
@@ -296,6 +311,14 @@ $ keytool -genkey -alias tomcat -keyalg RSA
     </description>
   </property>
 
+  <property>
+    <name>hadoop.kms.blacklist.DELETE</name>
+    <value>hdfs,foo</value>
+    <description>
+      Blacklist for delete-key operations.
+    </description>
+  </property>
+
   <property>
     <name>hadoop.kms.acl.ROLLOVER</name>
     <value>*</value>
@@ -306,6 +329,14 @@ $ keytool -genkey -alias tomcat -keyalg RSA
     </description>
   </property>
 
+  <property>
+    <name>hadoop.kms.blacklist.ROLLOVER</name>
+    <value>hdfs,foo</value>
+    <description>
+      Blacklist for rollover-key operations.
+    </description>
+  </property>
+
   <property>
     <name>hadoop.kms.acl.GET</name>
     <value>*</value>
@@ -314,6 +345,14 @@ $ keytool -genkey -alias tomcat -keyalg RSA
     </description>
   </property>
 
+  <property>
+    <name>hadoop.kms.blacklist.GET</name>
+    <value>hdfs,foo</value>
+    <description>
+      ACL for get-key-version and get-current-key operations.
+    </description>
+  </property>
+
   <property>
     <name>hadoop.kms.acl.GET_KEYS</name>
     <value>*</value>
@@ -322,6 +361,14 @@ $ keytool -genkey -alias tomcat -keyalg RSA
     </description>
   </property>
 
+  <property>
+    <name>hadoop.kms.blacklist.GET_KEYS</name>
+    <value>hdfs,foo</value>
+    <description>
+      Blacklist for get-keys operation.
+    </description>
+  </property>
+
   <property>
     <name>hadoop.kms.acl.GET_METADATA</name>
     <value>*</value>
@@ -330,6 +377,14 @@ $ keytool -genkey -alias tomcat -keyalg RSA
     </description>
   </property>
 
+  <property>
+    <name>hadoop.kms.blacklist.GET_METADATA</name>
+    <value>hdfs,foo</value>
+    <description>
+      Blacklist for get-key-metadata and get-keys-metadata operations.
+    </description>
+  </property>
+
   <property>
     <name>hadoop.kms.acl.SET_KEY_MATERIAL</name>
     <value>*</value>
@@ -339,6 +394,15 @@ $ keytool -genkey -alias tomcat -keyalg RSA
     </description>
   </property>
 
+  <property>
+    <name>hadoop.kms.blacklist.SET_KEY_MATERIAL</name>
+    <value>hdfs,foo</value>
+    <description>
+        Complimentary Blacklist for CREATE and ROLLOVER operation to allow the client
+        to provide the key material when creating or rolling a key.
+    </description>
+  </property>
+
   <property>
     <name>hadoop.kms.acl.GENERATE_EEK</name>
     <value>*</value>
@@ -348,6 +412,15 @@ $ keytool -genkey -alias tomcat -keyalg RSA
     </description>
   </property>
 
+  <property>
+    <name>hadoop.kms.blacklist.GENERATE_EEK</name>
+    <value>hdfs,foo</value>
+    <description>
+      Blacklist for generateEncryptedKey
+      CryptoExtension operations
+    </description>
+  </property>
+
   <property>
     <name>hadoop.kms.acl.DECRYPT_EEK</name>
     <value>*</value>
@@ -357,6 +430,17 @@ $ keytool -genkey -alias tomcat -keyalg RSA
     </description>
   </property>
 </configuration>
+
+  <property>
+    <name>hadoop.kms.blacklist.DECRYPT_EEK</name>
+    <value>hdfs,foo</value>
+    <description>
+      Blacklist for decrypt EncryptedKey
+      CryptoExtension operations
+    </description>
+  </property>
+</configuration>
+
 +---+
 
 ** KMS Delegation Token Configuration

+ 94 - 6
hadoop-common-project/hadoop-kms/src/test/java/org/apache/hadoop/crypto/key/kms/server/TestKMS.java

@@ -268,6 +268,8 @@ public class TestKMS {
     List<String> principals = new ArrayList<String>();
     principals.add("HTTP/localhost");
     principals.add("client");
+    principals.add("hdfs");
+    principals.add("otheradmin");
     principals.add("client/host");
     principals.add("client1");
     for (KMSACLs.Type type : KMSACLs.Type.values()) {
@@ -621,12 +623,12 @@ public class TestKMS {
     conf.set("hadoop.kms.authentication.kerberos.name.rules", "DEFAULT");
 
     for (KMSACLs.Type type : KMSACLs.Type.values()) {
-      conf.set(type.getConfigKey(), type.toString());
+      conf.set(type.getAclConfigKey(), type.toString());
     }
-    conf.set(KMSACLs.Type.CREATE.getConfigKey(),
+    conf.set(KMSACLs.Type.CREATE.getAclConfigKey(),
         KMSACLs.Type.CREATE.toString() + ",SET_KEY_MATERIAL");
 
-    conf.set(KMSACLs.Type.ROLLOVER.getConfigKey(),
+    conf.set(KMSACLs.Type.ROLLOVER.getAclConfigKey(),
         KMSACLs.Type.ROLLOVER.toString() + ",SET_KEY_MATERIAL");
 
     writeConf(testDir, conf);
@@ -884,7 +886,7 @@ public class TestKMS {
 
         // test ACL reloading
         Thread.sleep(10); // to ensure the ACLs file modifiedTime is newer
-        conf.set(KMSACLs.Type.CREATE.getConfigKey(), "foo");
+        conf.set(KMSACLs.Type.CREATE.getAclConfigKey(), "foo");
         writeConf(testDir, conf);
         Thread.sleep(1000);
 
@@ -914,6 +916,92 @@ public class TestKMS {
     });
   }
 
+  @Test
+  public void testKMSBlackList() throws Exception {
+    Configuration conf = new Configuration();
+    conf.set("hadoop.security.authentication", "kerberos");
+    UserGroupInformation.setConfiguration(conf);
+    File testDir = getTestDir();
+    conf = createBaseKMSConf(testDir);
+    conf.set("hadoop.kms.authentication.type", "kerberos");
+    conf.set("hadoop.kms.authentication.kerberos.keytab",
+        keytab.getAbsolutePath());
+    conf.set("hadoop.kms.authentication.kerberos.principal", "HTTP/localhost");
+    conf.set("hadoop.kms.authentication.kerberos.name.rules", "DEFAULT");
+    for (KMSACLs.Type type : KMSACLs.Type.values()) {
+      conf.set(type.getAclConfigKey(), " ");
+    }
+    conf.set(KMSACLs.Type.CREATE.getAclConfigKey(), "client,hdfs,otheradmin");
+    conf.set(KMSACLs.Type.GENERATE_EEK.getAclConfigKey(), "client,hdfs,otheradmin");
+    conf.set(KMSACLs.Type.DECRYPT_EEK.getAclConfigKey(), "client,hdfs,otheradmin");
+    conf.set(KMSACLs.Type.DECRYPT_EEK.getBlacklistConfigKey(), "hdfs,otheradmin");
+
+    writeConf(testDir, conf);
+
+    runServer(null, null, testDir, new KMSCallable() {
+      @Override
+      public Void call() throws Exception {
+        final Configuration conf = new Configuration();
+        conf.setInt(KeyProvider.DEFAULT_BITLENGTH_NAME, 128);
+        final URI uri = createKMSUri(getKMSUrl());
+
+        doAs("client", new PrivilegedExceptionAction<Void>() {
+          @Override
+          public Void run() throws Exception {
+            try {
+              KMSClientProvider kp = new KMSClientProvider(uri, conf);
+              KeyProvider.KeyVersion kv = kp.createKey("ck0",
+                  new KeyProvider.Options(conf));
+              EncryptedKeyVersion eek =
+                  kp.generateEncryptedKey("ck0");
+              kp.decryptEncryptedKey(eek);
+              Assert.assertNull(kv.getMaterial());
+            } catch (Exception ex) {
+              Assert.fail(ex.getMessage());
+            }
+            return null;
+          }
+        });
+
+        doAs("hdfs", new PrivilegedExceptionAction<Void>() {
+          @Override
+          public Void run() throws Exception {
+            try {
+              KMSClientProvider kp = new KMSClientProvider(uri, conf);
+              KeyProvider.KeyVersion kv = kp.createKey("ck1",
+                  new KeyProvider.Options(conf));
+              EncryptedKeyVersion eek =
+                  kp.generateEncryptedKey("ck1");
+              kp.decryptEncryptedKey(eek);
+              Assert.fail("admin user must not be allowed to decrypt !!");
+            } catch (Exception ex) {
+            }
+            return null;
+          }
+        });
+
+        doAs("otheradmin", new PrivilegedExceptionAction<Void>() {
+          @Override
+          public Void run() throws Exception {
+            try {
+              KMSClientProvider kp = new KMSClientProvider(uri, conf);
+              KeyProvider.KeyVersion kv = kp.createKey("ck2",
+                  new KeyProvider.Options(conf));
+              EncryptedKeyVersion eek =
+                  kp.generateEncryptedKey("ck2");
+              kp.decryptEncryptedKey(eek);
+              Assert.fail("admin user must not be allowed to decrypt !!");
+            } catch (Exception ex) {
+            }
+            return null;
+          }
+        });
+
+        return null;
+      }
+    });
+  }
+
   @Test
   public void testServicePrincipalACLs() throws Exception {
     Configuration conf = new Configuration();
@@ -927,9 +1015,9 @@ public class TestKMS {
     conf.set("hadoop.kms.authentication.kerberos.principal", "HTTP/localhost");
     conf.set("hadoop.kms.authentication.kerberos.name.rules", "DEFAULT");
     for (KMSACLs.Type type : KMSACLs.Type.values()) {
-      conf.set(type.getConfigKey(), " ");
+      conf.set(type.getAclConfigKey(), " ");
     }
-    conf.set(KMSACLs.Type.CREATE.getConfigKey(), "client");
+    conf.set(KMSACLs.Type.CREATE.getAclConfigKey(), "client");
 
     writeConf(testDir, conf);
 

+ 1 - 1
hadoop-common-project/hadoop-kms/src/test/java/org/apache/hadoop/crypto/key/kms/server/TestKMSACLs.java

@@ -37,7 +37,7 @@ public class TestKMSACLs {
   public void testCustom() {
     Configuration conf = new Configuration(false);
     for (KMSACLs.Type type : KMSACLs.Type.values()) {
-      conf.set(type.getConfigKey(), type.toString() + " ");
+      conf.set(type.getAclConfigKey(), type.toString() + " ");
     }
     KMSACLs acls = new KMSACLs(conf);
     for (KMSACLs.Type type : KMSACLs.Type.values()) {

+ 3 - 0
hadoop-dist/pom.xml

@@ -114,6 +114,9 @@
                       run rm -rf hadoop-${project.version}
                       run mkdir hadoop-${project.version}
                       run cd hadoop-${project.version}
+                      run cp $ROOT/LICENSE.txt .
+                      run cp $ROOT/NOTICE.txt .
+                      run cp $ROOT/README.txt .
                       run cp -r $ROOT/hadoop-common-project/hadoop-common/target/hadoop-common-${project.version}/* .
                       run cp -r $ROOT/hadoop-common-project/hadoop-nfs/target/hadoop-nfs-${project.version}/* .
                       run cp -r $ROOT/hadoop-hdfs-project/hadoop-hdfs/target/hadoop-hdfs-${project.version}/* .

+ 31 - 39
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSFileSystem.java

@@ -40,13 +40,12 @@ import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.lib.wsrs.EnumSetParam;
 import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.security.authentication.client.AuthenticationException;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.TokenIdentifier;
-import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenIdentifier;
 import org.apache.hadoop.security.token.delegation.web.DelegationTokenAuthenticatedURL;
 import org.apache.hadoop.security.token.delegation.web.DelegationTokenAuthenticator;
 import org.apache.hadoop.security.token.delegation.web.KerberosDelegationTokenAuthenticator;
+import org.apache.hadoop.util.HttpExceptionUtils;
 import org.apache.hadoop.util.Progressable;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.StringUtils;
@@ -179,11 +178,6 @@ public class HttpFSFileSystem extends FileSystem
   public static final String ACL_ENTRIES_JSON = "entries";
   public static final String ACL_BIT_JSON = "aclBit";
 
-  public static final String ERROR_JSON = "RemoteException";
-  public static final String ERROR_EXCEPTION_JSON = "exception";
-  public static final String ERROR_CLASSNAME_JSON = "javaClassName";
-  public static final String ERROR_MESSAGE_JSON = "message";
-
   public static final int HTTP_TEMPORARY_REDIRECT = 307;
 
   private static final String HTTP_GET = "GET";
@@ -223,7 +217,6 @@ public class HttpFSFileSystem extends FileSystem
   private URI uri;
   private Path workingDir;
   private UserGroupInformation realUser;
-  private String doAs;
 
 
 
@@ -336,7 +329,6 @@ public class HttpFSFileSystem extends FileSystem
     if (realUser == null) {
       realUser = UserGroupInformation.getLoginUser();
     }
-    doAs = ugi.getShortUserName();
     super.initialize(name, conf);
     try {
       uri = new URI(name.getScheme() + "://" + name.getAuthority());
@@ -435,7 +427,7 @@ public class HttpFSFileSystem extends FileSystem
     params.put(OP_PARAM, Operation.OPEN.toString());
     HttpURLConnection conn = getConnection(Operation.OPEN.getMethod(), params,
                                            f, true);
-    HttpFSUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
+    HttpExceptionUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
     return new FSDataInputStream(
       new HttpFSDataInputStream(conn.getInputStream(), bufferSize));
   }
@@ -462,7 +454,7 @@ public class HttpFSFileSystem extends FileSystem
       try {
         super.close();
       } finally {
-        HttpFSUtils.validateResponse(conn, closeStatus);
+        HttpExceptionUtils.validateResponse(conn, closeStatus);
       }
     }
 
@@ -498,11 +490,11 @@ public class HttpFSFileSystem extends FileSystem
             OutputStream os = new BufferedOutputStream(conn.getOutputStream(), bufferSize);
             return new HttpFSDataOutputStream(conn, os, expectedStatus, statistics);
           } catch (IOException ex) {
-            HttpFSUtils.validateResponse(conn, expectedStatus);
+            HttpExceptionUtils.validateResponse(conn, expectedStatus);
             throw ex;
           }
         } else {
-          HttpFSUtils.validateResponse(conn, HTTP_TEMPORARY_REDIRECT);
+          HttpExceptionUtils.validateResponse(conn, HTTP_TEMPORARY_REDIRECT);
           throw new IOException("Missing HTTP 'Location' header for [" + conn.getURL() + "]");
         }
       } else {
@@ -514,7 +506,7 @@ public class HttpFSFileSystem extends FileSystem
       if (exceptionAlreadyHandled) {
         throw ex;
       } else {
-        HttpFSUtils.validateResponse(conn, HTTP_TEMPORARY_REDIRECT);
+        HttpExceptionUtils.validateResponse(conn, HTTP_TEMPORARY_REDIRECT);
         throw ex;
       }
     }
@@ -595,7 +587,7 @@ public class HttpFSFileSystem extends FileSystem
     params.put(SOURCES_PARAM, srcs);
     HttpURLConnection conn = getConnection(Operation.CONCAT.getMethod(),
         params, f, true);
-    HttpFSUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
+    HttpExceptionUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
   }
 
   /**
@@ -609,7 +601,7 @@ public class HttpFSFileSystem extends FileSystem
     params.put(DESTINATION_PARAM, dst.toString());
     HttpURLConnection conn = getConnection(Operation.RENAME.getMethod(),
                                            params, src, true);
-    HttpFSUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
+    HttpExceptionUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
     JSONObject json = (JSONObject) HttpFSUtils.jsonParse(conn);
     return (Boolean) json.get(RENAME_JSON);
   }
@@ -644,7 +636,7 @@ public class HttpFSFileSystem extends FileSystem
     params.put(RECURSIVE_PARAM, Boolean.toString(recursive));
     HttpURLConnection conn = getConnection(Operation.DELETE.getMethod(),
                                            params, f, true);
-    HttpFSUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
+    HttpExceptionUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
     JSONObject json = (JSONObject) HttpFSUtils.jsonParse(conn);
     return (Boolean) json.get(DELETE_JSON);
   }
@@ -665,7 +657,7 @@ public class HttpFSFileSystem extends FileSystem
     params.put(OP_PARAM, Operation.LISTSTATUS.toString());
     HttpURLConnection conn = getConnection(Operation.LISTSTATUS.getMethod(),
                                            params, f, true);
-    HttpFSUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
+    HttpExceptionUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
     JSONObject json = (JSONObject) HttpFSUtils.jsonParse(conn);
     json = (JSONObject) json.get(FILE_STATUSES_JSON);
     JSONArray jsonArray = (JSONArray) json.get(FILE_STATUS_JSON);
@@ -713,7 +705,7 @@ public class HttpFSFileSystem extends FileSystem
     params.put(PERMISSION_PARAM, permissionToString(permission));
     HttpURLConnection conn = getConnection(Operation.MKDIRS.getMethod(),
                                            params, f, true);
-    HttpFSUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
+    HttpExceptionUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
     JSONObject json = (JSONObject) HttpFSUtils.jsonParse(conn);
     return (Boolean) json.get(MKDIRS_JSON);
   }
@@ -734,7 +726,7 @@ public class HttpFSFileSystem extends FileSystem
     params.put(OP_PARAM, Operation.GETFILESTATUS.toString());
     HttpURLConnection conn = getConnection(Operation.GETFILESTATUS.getMethod(),
                                            params, f, true);
-    HttpFSUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
+    HttpExceptionUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
     JSONObject json = (JSONObject) HttpFSUtils.jsonParse(conn);
     json = (JSONObject) json.get(FILE_STATUS_JSON);
     f = makeQualified(f);
@@ -753,7 +745,7 @@ public class HttpFSFileSystem extends FileSystem
       HttpURLConnection conn =
         getConnection(Operation.GETHOMEDIRECTORY.getMethod(), params,
                       new Path(getUri().toString(), "/"), false);
-      HttpFSUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
+      HttpExceptionUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
       JSONObject json = (JSONObject) HttpFSUtils.jsonParse(conn);
       return new Path((String) json.get(HOME_DIR_JSON));
     } catch (IOException ex) {
@@ -778,7 +770,7 @@ public class HttpFSFileSystem extends FileSystem
     params.put(GROUP_PARAM, groupname);
     HttpURLConnection conn = getConnection(Operation.SETOWNER.getMethod(),
                                            params, p, true);
-    HttpFSUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
+    HttpExceptionUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
   }
 
   /**
@@ -793,7 +785,7 @@ public class HttpFSFileSystem extends FileSystem
     params.put(OP_PARAM, Operation.SETPERMISSION.toString());
     params.put(PERMISSION_PARAM, permissionToString(permission));
     HttpURLConnection conn = getConnection(Operation.SETPERMISSION.getMethod(), params, p, true);
-    HttpFSUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
+    HttpExceptionUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
   }
 
   /**
@@ -815,7 +807,7 @@ public class HttpFSFileSystem extends FileSystem
     params.put(ACCESS_TIME_PARAM, Long.toString(atime));
     HttpURLConnection conn = getConnection(Operation.SETTIMES.getMethod(),
                                            params, p, true);
-    HttpFSUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
+    HttpExceptionUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
   }
 
   /**
@@ -837,7 +829,7 @@ public class HttpFSFileSystem extends FileSystem
     params.put(REPLICATION_PARAM, Short.toString(replication));
     HttpURLConnection conn =
       getConnection(Operation.SETREPLICATION.getMethod(), params, src, true);
-    HttpFSUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
+    HttpExceptionUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
     JSONObject json = (JSONObject) HttpFSUtils.jsonParse(conn);
     return (Boolean) json.get(SET_REPLICATION_JSON);
   }
@@ -857,7 +849,7 @@ public class HttpFSFileSystem extends FileSystem
     params.put(ACLSPEC_PARAM, AclEntry.aclSpecToString(aclSpec));
     HttpURLConnection conn = getConnection(
             Operation.MODIFYACLENTRIES.getMethod(), params, path, true);
-    HttpFSUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
+    HttpExceptionUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
   }
 
   /**
@@ -874,7 +866,7 @@ public class HttpFSFileSystem extends FileSystem
     params.put(ACLSPEC_PARAM, AclEntry.aclSpecToString(aclSpec));
     HttpURLConnection conn = getConnection(
             Operation.REMOVEACLENTRIES.getMethod(), params, path, true);
-    HttpFSUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
+    HttpExceptionUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
   }
 
   /**
@@ -888,7 +880,7 @@ public class HttpFSFileSystem extends FileSystem
     params.put(OP_PARAM, Operation.REMOVEDEFAULTACL.toString());
     HttpURLConnection conn = getConnection(
             Operation.REMOVEDEFAULTACL.getMethod(), params, path, true);
-    HttpFSUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
+    HttpExceptionUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
   }
 
   /**
@@ -902,7 +894,7 @@ public class HttpFSFileSystem extends FileSystem
     params.put(OP_PARAM, Operation.REMOVEACL.toString());
     HttpURLConnection conn = getConnection(Operation.REMOVEACL.getMethod(),
             params, path, true);
-    HttpFSUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
+    HttpExceptionUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
   }
 
   /**
@@ -920,7 +912,7 @@ public class HttpFSFileSystem extends FileSystem
     params.put(ACLSPEC_PARAM, AclEntry.aclSpecToString(aclSpec));
     HttpURLConnection conn = getConnection(Operation.SETACL.getMethod(),
                                            params, path, true);
-    HttpFSUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
+    HttpExceptionUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
   }
 
   /**
@@ -935,7 +927,7 @@ public class HttpFSFileSystem extends FileSystem
     params.put(OP_PARAM, Operation.GETACLSTATUS.toString());
     HttpURLConnection conn = getConnection(Operation.GETACLSTATUS.getMethod(),
             params, path, true);
-    HttpFSUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
+    HttpExceptionUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
     JSONObject json = (JSONObject) HttpFSUtils.jsonParse(conn);
     json = (JSONObject) json.get(ACL_STATUS_JSON);
     return createAclStatus(json);
@@ -996,7 +988,7 @@ public class HttpFSFileSystem extends FileSystem
     params.put(OP_PARAM, Operation.GETCONTENTSUMMARY.toString());
     HttpURLConnection conn =
       getConnection(Operation.GETCONTENTSUMMARY.getMethod(), params, f, true);
-    HttpFSUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
+    HttpExceptionUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
     JSONObject json = (JSONObject) ((JSONObject)
       HttpFSUtils.jsonParse(conn)).get(CONTENT_SUMMARY_JSON);
     return new ContentSummary((Long) json.get(CONTENT_SUMMARY_LENGTH_JSON),
@@ -1014,7 +1006,7 @@ public class HttpFSFileSystem extends FileSystem
     params.put(OP_PARAM, Operation.GETFILECHECKSUM.toString());
     HttpURLConnection conn =
       getConnection(Operation.GETFILECHECKSUM.getMethod(), params, f, true);
-    HttpFSUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
+    HttpExceptionUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
     final JSONObject json = (JSONObject) ((JSONObject)
       HttpFSUtils.jsonParse(conn)).get(FILE_CHECKSUM_JSON);
     return new FileChecksum() {
@@ -1115,7 +1107,7 @@ public class HttpFSFileSystem extends FileSystem
     params.put(XATTR_SET_FLAG_PARAM, EnumSetParam.toString(flag));
     HttpURLConnection conn = getConnection(Operation.SETXATTR.getMethod(),
         params, f, true);
-    HttpFSUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
+    HttpExceptionUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
   }
 
   @Override
@@ -1125,7 +1117,7 @@ public class HttpFSFileSystem extends FileSystem
     params.put(XATTR_NAME_PARAM, name);
     HttpURLConnection conn = getConnection(Operation.GETXATTRS.getMethod(),
         params, f, true);
-    HttpFSUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
+    HttpExceptionUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
     JSONObject json = (JSONObject) HttpFSUtils.jsonParse(conn);
     Map<String, byte[]> xAttrs = createXAttrMap(
         (JSONArray) json.get(XATTRS_JSON));
@@ -1169,7 +1161,7 @@ public class HttpFSFileSystem extends FileSystem
     params.put(OP_PARAM, Operation.GETXATTRS.toString());
     HttpURLConnection conn = getConnection(Operation.GETXATTRS.getMethod(),
         params, f, true);
-    HttpFSUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
+    HttpExceptionUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
     JSONObject json = (JSONObject) HttpFSUtils.jsonParse(conn);
     return createXAttrMap((JSONArray) json.get(XATTRS_JSON));
   }
@@ -1185,7 +1177,7 @@ public class HttpFSFileSystem extends FileSystem
     multiValuedParams.put(XATTR_NAME_PARAM, names);
     HttpURLConnection conn = getConnection(Operation.GETXATTRS.getMethod(),
         params, multiValuedParams, f, true);
-    HttpFSUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
+    HttpExceptionUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
     JSONObject json = (JSONObject) HttpFSUtils.jsonParse(conn);
     return createXAttrMap((JSONArray) json.get(XATTRS_JSON));
   }
@@ -1196,7 +1188,7 @@ public class HttpFSFileSystem extends FileSystem
     params.put(OP_PARAM, Operation.LISTXATTRS.toString());
     HttpURLConnection conn = getConnection(Operation.LISTXATTRS.getMethod(),
         params, f, true);
-    HttpFSUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
+    HttpExceptionUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
     JSONObject json = (JSONObject) HttpFSUtils.jsonParse(conn);
     return createXAttrNames((String) json.get(XATTRNAMES_JSON));
   }
@@ -1208,6 +1200,6 @@ public class HttpFSFileSystem extends FileSystem
     params.put(XATTR_NAME_PARAM, name);
     HttpURLConnection conn = getConnection(Operation.REMOVEXATTR.getMethod(),
         params, f, true);
-    HttpFSUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
+    HttpExceptionUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
   }
 }

+ 0 - 50
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSUtils.java

@@ -19,13 +19,11 @@ package org.apache.hadoop.fs.http.client;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.fs.Path;
-import org.json.simple.JSONObject;
 import org.json.simple.parser.JSONParser;
 import org.json.simple.parser.ParseException;
 
 import java.io.IOException;
 import java.io.InputStreamReader;
-import java.lang.reflect.Constructor;
 import java.net.HttpURLConnection;
 import java.net.URI;
 import java.net.URL;
@@ -115,54 +113,6 @@ public class HttpFSUtils {
     return new URL(sb.toString());
   }
 
-  /**
-   * Validates the status of an <code>HttpURLConnection</code> against an
-   * expected HTTP status code. If the current status code is not the expected
-   * one it throws an exception with a detail message using Server side error
-   * messages if available.
-   *
-   * @param conn the <code>HttpURLConnection</code>.
-   * @param expected the expected HTTP status code.
-   *
-   * @throws IOException thrown if the current status code does not match the
-   * expected one.
-   */
-  @SuppressWarnings({"unchecked"})
-  static void validateResponse(HttpURLConnection conn, int expected)
-    throws IOException {
-    int status = conn.getResponseCode();
-    if (status != expected) {
-      try {
-        JSONObject json = (JSONObject) HttpFSUtils.jsonParse(conn);
-        json = (JSONObject) json.get(HttpFSFileSystem.ERROR_JSON);
-        String message = (String) json.get(HttpFSFileSystem.ERROR_MESSAGE_JSON);
-        String exception = (String)
-          json.get(HttpFSFileSystem.ERROR_EXCEPTION_JSON);
-        String className = (String)
-          json.get(HttpFSFileSystem.ERROR_CLASSNAME_JSON);
-
-        try {
-          ClassLoader cl = HttpFSFileSystem.class.getClassLoader();
-          Class klass = cl.loadClass(className);
-          Constructor constr = klass.getConstructor(String.class);
-          throw (IOException) constr.newInstance(message);
-        } catch (IOException ex) {
-          throw ex;
-        } catch (Exception ex) {
-          throw new IOException(MessageFormat.format("{0} - {1}", exception,
-                                                     message));
-        }
-      } catch (IOException ex) {
-        if (ex.getCause() instanceof IOException) {
-          throw (IOException) ex.getCause();
-        }
-        throw new IOException(
-          MessageFormat.format("HTTP status [{0}], {1}",
-                               status, conn.getResponseMessage()));
-      }
-    }
-  }
-
   /**
    * Convenience method that JSON Parses the <code>InputStream</code> of a
    * <code>HttpURLConnection</code>.

+ 2 - 12
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/wsrs/ExceptionProvider.java

@@ -19,15 +19,12 @@
 package org.apache.hadoop.lib.wsrs;
 
 import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.fs.http.client.HttpFSFileSystem;
+import org.apache.hadoop.util.HttpExceptionUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import javax.ws.rs.core.MediaType;
 import javax.ws.rs.core.Response;
 import javax.ws.rs.ext.ExceptionMapper;
-import java.util.LinkedHashMap;
-import java.util.Map;
 
 @InterfaceAudience.Private
 public class ExceptionProvider implements ExceptionMapper<Throwable> {
@@ -36,14 +33,7 @@ public class ExceptionProvider implements ExceptionMapper<Throwable> {
   private static final String ENTER = System.getProperty("line.separator");
 
   protected Response createResponse(Response.Status status, Throwable throwable) {
-    Map<String, Object> json = new LinkedHashMap<String, Object>();
-    json.put(HttpFSFileSystem.ERROR_MESSAGE_JSON, getOneLineMessage(throwable));
-    json.put(HttpFSFileSystem.ERROR_EXCEPTION_JSON, throwable.getClass().getSimpleName());
-    json.put(HttpFSFileSystem.ERROR_CLASSNAME_JSON, throwable.getClass().getName());
-    Map<String, Object> response = new LinkedHashMap<String, Object>();
-    response.put(HttpFSFileSystem.ERROR_JSON, json);
-    log(status, throwable);
-    return Response.status(status).type(MediaType.APPLICATION_JSON).entity(response).build();
+    return HttpExceptionUtils.createJerseyExceptionResponse(status, throwable);
   }
 
   protected String getOneLineMessage(Throwable throwable) {

+ 3 - 1
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/BaseTestHttpFSWith.java

@@ -183,6 +183,8 @@ public abstract class BaseTestHttpFSWith extends HFSTestCase {
 
   private void testCreate() throws Exception {
     Path path = new Path(getProxiedFSTestDir(), "foo.txt");
+    FileSystem fs = FileSystem.get(getProxiedFSConf());
+    fs.delete(path, true);
     testCreate(path, false);
     testCreate(path, true);
     try {
@@ -190,7 +192,7 @@ public abstract class BaseTestHttpFSWith extends HFSTestCase {
       Assert.fail("the create should have failed because the file exists " +
                   "and override is FALSE");
     } catch (IOException ex) {
-
+System.out.println("#");
     } catch (Exception ex) {
       Assert.fail(ex.toString());
     }

+ 4 - 6
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSServerNoACLs.java

@@ -189,9 +189,8 @@ public class TestHttpFSServerNoACLs extends HTestCase {
       Assert.assertEquals(HttpURLConnection.HTTP_INTERNAL_ERROR, resp);
       reader = new BufferedReader(new InputStreamReader(conn.getErrorStream()));
       String res = reader.readLine();
-      Assert.assertTrue(res.contains("RemoteException"));
-      Assert.assertTrue(res.contains("ACL"));
-      Assert.assertTrue(res.contains("rejected"));
+      Assert.assertTrue(res.contains("AclException"));
+      Assert.assertTrue(res.contains("Support for ACLs has been disabled"));
     }
   }
 
@@ -224,9 +223,8 @@ public class TestHttpFSServerNoACLs extends HTestCase {
       BufferedReader reader;
       reader = new BufferedReader(new InputStreamReader(conn.getErrorStream()));
       String err = reader.readLine();
-      Assert.assertTrue(err.contains("RemoteException"));
-      Assert.assertTrue(err.contains("ACL"));
-      Assert.assertTrue(err.contains("rejected"));
+      Assert.assertTrue(err.contains("AclException"));
+      Assert.assertTrue(err.contains("Support for ACLs has been disabled"));
     }
   }
 

+ 3 - 3
hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/OpenFileCtx.java

@@ -421,7 +421,7 @@ class OpenFileCtx {
       if (existantWriteCtx != null) {
         if (!existantWriteCtx.getReplied()) {
           if (LOG.isDebugEnabled()) {
-            LOG.debug("Repeated write request which hasn't be served: xid="
+            LOG.debug("Repeated write request which hasn't been served: xid="
                 + xid + ", drop it.");
           }
         } else {
@@ -579,7 +579,7 @@ class OpenFileCtx {
    * writing, and there is no other threads writing (i.e., asyncStatus is
    * false), start the writing and set asyncStatus to true.
    * 
-   * @return True if the new write is sequencial and we can start writing
+   * @return True if the new write is sequential and we can start writing
    *         (including the case that there is already a thread writing).
    */
   private synchronized boolean checkAndStartWrite(
@@ -898,7 +898,7 @@ class OpenFileCtx {
       long offset = nextOffset.get();
       if (range.getMin() > offset) {
         if (LOG.isDebugEnabled()) {
-          LOG.debug("The next sequencial write has not arrived yet");
+          LOG.debug("The next sequential write has not arrived yet");
         }
         processCommits(nextOffset.get()); // handle race
         this.asyncStatus = false;

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/RpcProgramNfs3.java

@@ -1423,7 +1423,7 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
         throw io;
       }
       // This happens when startAfter was just deleted
-      LOG.info("Cookie cound't be found: " + new String(startAfter)
+      LOG.info("Cookie couldn't be found: " + new String(startAfter)
           + ", do listing from beginning");
       dlisting = dfsClient
           .listPaths(dirFileIdPath, HdfsFileStatus.EMPTY_NAME);

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

@@ -435,6 +435,15 @@ Release 2.6.0 - UNRELEASED
     HDFS-4257. The ReplaceDatanodeOnFailure policies could have a forgiving
     option (szetszwo via cmccabe)
 
+    HDFS-6959. Make the HDFS home directory location customizable. (yzhang via
+    cmccabe)
+
+    HDFS-6886. Use single editlog record for creating file + overwrite. (Yi Liu
+    via jing9)
+
+    HDFS-6376. Distcp data between two HA clusters requires another configuration.
+    (Dave Marion and Haohui Mai via jing9)
+
   OPTIMIZATIONS
 
     HDFS-6690. Deduplicate xattr names in memory. (wang)
@@ -585,6 +594,14 @@ Release 2.6.0 - UNRELEASED
     HDFS-6972. TestRefreshUserMappings.testRefreshSuperUserGroupsConfiguration
     doesn't decode url correctly. (Yongjun Zhang via wang)
 
+    HDFS-6942. Fix typos in log messages. (Ray Chiang via wheat9)
+
+    HDFS-6848. Lack of synchronization on access to datanodeUuid in
+    DataStorage#format(). (Xiaoyu Yao via Arpit Agarwal)
+
+    HDFS-6996. SnapshotDiff report can hit IndexOutOfBoundsException when there
+    are nested renamed directory/file. (jing9)
+
     BREAKDOWN OF HDFS-6134 AND HADOOP-10150 SUBTASKS AND RELATED JIRAS
   
       HDFS-6387. HDFS CLI admin tool for creating & deleting an
@@ -678,6 +695,14 @@ Release 2.6.0 - UNRELEASED
 
       HDFS-6954. With crypto, no native lib systems are too verbose. (clamb via wang)
 
+      HDFS-2975. Rename with overwrite flag true can make NameNode to stuck in safemode 
+      on NN (crash + restart). (Yi Liu via umamahesh)
+
+      HDFS-6905. fs-encryption merge triggered release audit failures. (clamb via tucu)
+
+      HDFS-6714. TestBlocksScheduledCounter#testBlocksScheduledCounter should
+      shutdown cluster (vinayakumarb)
+
 Release 2.5.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

+ 0 - 271
hadoop-hdfs-project/hadoop-hdfs/LICENSE.txt

@@ -1,271 +0,0 @@
-
-                                 Apache License
-                           Version 2.0, January 2004
-                        http://www.apache.org/licenses/
-
-   TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
-
-   1. Definitions.
-
-      "License" shall mean the terms and conditions for use, reproduction,
-      and distribution as defined by Sections 1 through 9 of this document.
-
-      "Licensor" shall mean the copyright owner or entity authorized by
-      the copyright owner that is granting the License.
-
-      "Legal Entity" shall mean the union of the acting entity and all
-      other entities that control, are controlled by, or are under common
-      control with that entity. For the purposes of this definition,
-      "control" means (i) the power, direct or indirect, to cause the
-      direction or management of such entity, whether by contract or
-      otherwise, or (ii) ownership of fifty percent (50%) or more of the
-      outstanding shares, or (iii) beneficial ownership of such entity.
-
-      "You" (or "Your") shall mean an individual or Legal Entity
-      exercising permissions granted by this License.
-
-      "Source" form shall mean the preferred form for making modifications,
-      including but not limited to software source code, documentation
-      source, and configuration files.
-
-      "Object" form shall mean any form resulting from mechanical
-      transformation or translation of a Source form, including but
-      not limited to compiled object code, generated documentation,
-      and conversions to other media types.
-
-      "Work" shall mean the work of authorship, whether in Source or
-      Object form, made available under the License, as indicated by a
-      copyright notice that is included in or attached to the work
-      (an example is provided in the Appendix below).
-
-      "Derivative Works" shall mean any work, whether in Source or Object
-      form, that is based on (or derived from) the Work and for which the
-      editorial revisions, annotations, elaborations, or other modifications
-      represent, as a whole, an original work of authorship. For the purposes
-      of this License, Derivative Works shall not include works that remain
-      separable from, or merely link (or bind by name) to the interfaces of,
-      the Work and Derivative Works thereof.
-
-      "Contribution" shall mean any work of authorship, including
-      the original version of the Work and any modifications or additions
-      to that Work or Derivative Works thereof, that is intentionally
-      submitted to Licensor for inclusion in the Work by the copyright owner
-      or by an individual or Legal Entity authorized to submit on behalf of
-      the copyright owner. For the purposes of this definition, "submitted"
-      means any form of electronic, verbal, or written communication sent
-      to the Licensor or its representatives, including but not limited to
-      communication on electronic mailing lists, source code control systems,
-      and issue tracking systems that are managed by, or on behalf of, the
-      Licensor for the purpose of discussing and improving the Work, but
-      excluding communication that is conspicuously marked or otherwise
-      designated in writing by the copyright owner as "Not a Contribution."
-
-      "Contributor" shall mean Licensor and any individual or Legal Entity
-      on behalf of whom a Contribution has been received by Licensor and
-      subsequently incorporated within the Work.
-
-   2. Grant of Copyright License. Subject to the terms and conditions of
-      this License, each Contributor hereby grants to You a perpetual,
-      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
-      copyright license to reproduce, prepare Derivative Works of,
-      publicly display, publicly perform, sublicense, and distribute the
-      Work and such Derivative Works in Source or Object form.
-
-   3. Grant of Patent License. Subject to the terms and conditions of
-      this License, each Contributor hereby grants to You a perpetual,
-      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
-      (except as stated in this section) patent license to make, have made,
-      use, offer to sell, sell, import, and otherwise transfer the Work,
-      where such license applies only to those patent claims licensable
-      by such Contributor that are necessarily infringed by their
-      Contribution(s) alone or by combination of their Contribution(s)
-      with the Work to which such Contribution(s) was submitted. If You
-      institute patent litigation against any entity (including a
-      cross-claim or counterclaim in a lawsuit) alleging that the Work
-      or a Contribution incorporated within the Work constitutes direct
-      or contributory patent infringement, then any patent licenses
-      granted to You under this License for that Work shall terminate
-      as of the date such litigation is filed.
-
-   4. Redistribution. You may reproduce and distribute copies of the
-      Work or Derivative Works thereof in any medium, with or without
-      modifications, and in Source or Object form, provided that You
-      meet the following conditions:
-
-      (a) You must give any other recipients of the Work or
-          Derivative Works a copy of this License; and
-
-      (b) You must cause any modified files to carry prominent notices
-          stating that You changed the files; and
-
-      (c) You must retain, in the Source form of any Derivative Works
-          that You distribute, all copyright, patent, trademark, and
-          attribution notices from the Source form of the Work,
-          excluding those notices that do not pertain to any part of
-          the Derivative Works; and
-
-      (d) If the Work includes a "NOTICE" text file as part of its
-          distribution, then any Derivative Works that You distribute must
-          include a readable copy of the attribution notices contained
-          within such NOTICE file, excluding those notices that do not
-          pertain to any part of the Derivative Works, in at least one
-          of the following places: within a NOTICE text file distributed
-          as part of the Derivative Works; within the Source form or
-          documentation, if provided along with the Derivative Works; or,
-          within a display generated by the Derivative Works, if and
-          wherever such third-party notices normally appear. The contents
-          of the NOTICE file are for informational purposes only and
-          do not modify the License. You may add Your own attribution
-          notices within Derivative Works that You distribute, alongside
-          or as an addendum to the NOTICE text from the Work, provided
-          that such additional attribution notices cannot be construed
-          as modifying the License.
-
-      You may add Your own copyright statement to Your modifications and
-      may provide additional or different license terms and conditions
-      for use, reproduction, or distribution of Your modifications, or
-      for any such Derivative Works as a whole, provided Your use,
-      reproduction, and distribution of the Work otherwise complies with
-      the conditions stated in this License.
-
-   5. Submission of Contributions. Unless You explicitly state otherwise,
-      any Contribution intentionally submitted for inclusion in the Work
-      by You to the Licensor shall be under the terms and conditions of
-      this License, without any additional terms or conditions.
-      Notwithstanding the above, nothing herein shall supersede or modify
-      the terms of any separate license agreement you may have executed
-      with Licensor regarding such Contributions.
-
-   6. Trademarks. This License does not grant permission to use the trade
-      names, trademarks, service marks, or product names of the Licensor,
-      except as required for reasonable and customary use in describing the
-      origin of the Work and reproducing the content of the NOTICE file.
-
-   7. Disclaimer of Warranty. Unless required by applicable law or
-      agreed to in writing, Licensor provides the Work (and each
-      Contributor provides its Contributions) on an "AS IS" BASIS,
-      WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
-      implied, including, without limitation, any warranties or conditions
-      of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
-      PARTICULAR PURPOSE. You are solely responsible for determining the
-      appropriateness of using or redistributing the Work and assume any
-      risks associated with Your exercise of permissions under this License.
-
-   8. Limitation of Liability. In no event and under no legal theory,
-      whether in tort (including negligence), contract, or otherwise,
-      unless required by applicable law (such as deliberate and grossly
-      negligent acts) or agreed to in writing, shall any Contributor be
-      liable to You for damages, including any direct, indirect, special,
-      incidental, or consequential damages of any character arising as a
-      result of this License or out of the use or inability to use the
-      Work (including but not limited to damages for loss of goodwill,
-      work stoppage, computer failure or malfunction, or any and all
-      other commercial damages or losses), even if such Contributor
-      has been advised of the possibility of such damages.
-
-   9. Accepting Warranty or Additional Liability. While redistributing
-      the Work or Derivative Works thereof, You may choose to offer,
-      and charge a fee for, acceptance of support, warranty, indemnity,
-      or other liability obligations and/or rights consistent with this
-      License. However, in accepting such obligations, You may act only
-      on Your own behalf and on Your sole responsibility, not on behalf
-      of any other Contributor, and only if You agree to indemnify,
-      defend, and hold each Contributor harmless for any liability
-      incurred by, or claims asserted against, such Contributor by reason
-      of your accepting any such warranty or additional liability.
-
-   END OF TERMS AND CONDITIONS
-
-   APPENDIX: How to apply the Apache License to your work.
-
-      To apply the Apache License to your work, attach the following
-      boilerplate notice, with the fields enclosed by brackets "[]"
-      replaced with your own identifying information. (Don't include
-      the brackets!)  The text should be enclosed in the appropriate
-      comment syntax for the file format. We also recommend that a
-      file or class name and description of purpose be included on the
-      same "printed page" as the copyright notice for easier
-      identification within third-party archives.
-
-   Copyright [yyyy] [name of copyright owner]
-
-   Licensed under the Apache License, Version 2.0 (the "License");
-   you may not use this file except in compliance with the License.
-   You may obtain a copy of the License at
-
-       http://www.apache.org/licenses/LICENSE-2.0
-
-   Unless required by applicable law or agreed to in writing, software
-   distributed under the License is distributed on an "AS IS" BASIS,
-   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-   See the License for the specific language governing permissions and
-   limitations under the License.
-
-
-APACHE HADOOP SUBCOMPONENTS:
-
-The Apache Hadoop project contains subcomponents with separate copyright
-notices and license terms. Your use of the source code for the these
-subcomponents is subject to the terms and conditions of the following
-licenses. 
-
-For the org.apache.hadoop.util.bloom.* classes:
-
-/**
- *
- * Copyright (c) 2005, European Commission project OneLab under contract
- * 034819 (http://www.one-lab.org)
- * All rights reserved.
- * Redistribution and use in source and binary forms, with or 
- * without modification, are permitted provided that the following 
- * conditions are met:
- *  - Redistributions of source code must retain the above copyright 
- *    notice, this list of conditions and the following disclaimer.
- *  - Redistributions in binary form must reproduce the above copyright 
- *    notice, this list of conditions and the following disclaimer in 
- *    the documentation and/or other materials provided with the distribution.
- *  - Neither the name of the University Catholique de Louvain - UCL
- *    nor the names of its contributors may be used to endorse or 
- *    promote products derived from this software without specific prior 
- *    written permission.
- *    
- * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS 
- * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT 
- * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS 
- * FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE 
- * COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, 
- * INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, 
- * BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; 
- * LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER 
- * CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT 
- * LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN 
- * ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE 
- * POSSIBILITY OF SUCH DAMAGE.
- */
-
-For src/main/native/util/tree.h:
-
-/*-
- * Copyright 2002 Niels Provos <provos@citi.umich.edu>
- * All rights reserved.
- *
- * Redistribution and use in source and binary forms, with or without
- * modification, are permitted provided that the following conditions
- * are met:
- * 1. Redistributions of source code must retain the above copyright
- *    notice, this list of conditions and the following disclaimer.
- * 2. Redistributions in binary form must reproduce the above copyright
- *    notice, this list of conditions and the following disclaimer in the
- *    documentation and/or other materials provided with the distribution.
- *
- * THIS SOFTWARE IS PROVIDED BY THE AUTHOR ``AS IS'' AND ANY EXPRESS OR
- * IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES
- * OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED.
- * IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR ANY DIRECT, INDIRECT,
- * INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT
- * NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
- * DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
- * THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
- * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF
- * THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
- */

+ 0 - 2
hadoop-hdfs-project/hadoop-hdfs/NOTICE.txt

@@ -1,2 +0,0 @@
-This product includes software developed by The Apache Software
-Foundation (http://www.apache.org/).

+ 3 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java

@@ -41,6 +41,8 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final int     DFS_STREAM_BUFFER_SIZE_DEFAULT = 4096;
   public static final String  DFS_BYTES_PER_CHECKSUM_KEY = "dfs.bytes-per-checksum";
   public static final int     DFS_BYTES_PER_CHECKSUM_DEFAULT = 512;
+  public static final String  DFS_USER_HOME_DIR_PREFIX_KEY = "dfs.user.home.dir.prefix";
+  public static final String  DFS_USER_HOME_DIR_PREFIX_DEFAULT = "/user";
   public static final String  DFS_CLIENT_RETRY_POLICY_ENABLED_KEY = "dfs.client.retry.policy.enabled";
   public static final boolean DFS_CLIENT_RETRY_POLICY_ENABLED_DEFAULT = false; 
   public static final String  DFS_CLIENT_RETRY_POLICY_SPEC_KEY = "dfs.client.retry.policy.spec";
@@ -528,6 +530,7 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   
   public static final String  DFS_NAMESERVICES = "dfs.nameservices";
   public static final String  DFS_NAMESERVICE_ID = "dfs.nameservice.id";
+  public static final String  DFS_INTERNAL_NAMESERVICES_KEY = "dfs.internal.nameservices";
   public static final String  DFS_NAMENODE_RESOURCE_CHECK_INTERVAL_KEY = "dfs.namenode.resource.check.interval";
   public static final int     DFS_NAMENODE_RESOURCE_CHECK_INTERVAL_DEFAULT = 5000;
   public static final String  DFS_NAMENODE_DU_RESERVED_KEY = "dfs.namenode.resource.du.reserved";

+ 72 - 8
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java

@@ -60,6 +60,7 @@ import java.util.Set;
 
 import javax.net.SocketFactory;
 
+import com.google.common.collect.Sets;
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.CommandLineParser;
 import org.apache.commons.cli.Option;
@@ -612,7 +613,7 @@ public class DFSUtil {
     String keySuffix = concatSuffixes(suffixes);
     return addSuffix(key, keySuffix);
   }
-  
+
   /**
    * Returns the configured address for all NameNodes in the cluster.
    * @param conf configuration
@@ -621,14 +622,25 @@ public class DFSUtil {
    * @return a map(nameserviceId to map(namenodeId to InetSocketAddress))
    */
   private static Map<String, Map<String, InetSocketAddress>>
-    getAddresses(Configuration conf,
-      String defaultAddress, String... keys) {
+    getAddresses(Configuration conf, String defaultAddress, String... keys) {
     Collection<String> nameserviceIds = getNameServiceIds(conf);
-    
+    return getAddressesForNsIds(conf, nameserviceIds, defaultAddress, keys);
+  }
+
+  /**
+   * Returns the configured address for all NameNodes in the cluster.
+   * @param conf configuration
+   * @param nsIds
+   *@param defaultAddress default address to return in case key is not found.
+   * @param keys Set of keys to look for in the order of preference   @return a map(nameserviceId to map(namenodeId to InetSocketAddress))
+   */
+  private static Map<String, Map<String, InetSocketAddress>>
+    getAddressesForNsIds(Configuration conf, Collection<String> nsIds,
+                         String defaultAddress, String... keys) {
     // Look for configurations of the form <key>[.<nameserviceId>][.<namenodeId>]
     // across all of the configured nameservices and namenodes.
     Map<String, Map<String, InetSocketAddress>> ret = Maps.newLinkedHashMap();
-    for (String nsId : emptyAsSingletonNull(nameserviceIds)) {
+    for (String nsId : emptyAsSingletonNull(nsIds)) {
       Map<String, InetSocketAddress> isas =
         getAddressesForNameserviceId(conf, nsId, defaultAddress, keys);
       if (!isas.isEmpty()) {
@@ -773,8 +785,7 @@ public class DFSUtil {
 
   /**
    * Returns list of InetSocketAddresses corresponding to namenodes from the
-   * configuration. Note this is to be used by datanodes to get the list of
-   * namenode addresses to talk to.
+   * configuration.
    * 
    * Returns namenode address specifically configured for datanodes (using
    * service ports), if found. If not, regular RPC address configured for other
@@ -805,7 +816,60 @@ public class DFSUtil {
     }
     return addressList;
   }
-  
+
+  /**
+   * Returns list of InetSocketAddresses corresponding to the namenode
+   * that manages this cluster. Note this is to be used by datanodes to get
+   * the list of namenode addresses to talk to.
+   *
+   * Returns namenode address specifically configured for datanodes (using
+   * service ports), if found. If not, regular RPC address configured for other
+   * clients is returned.
+   *
+   * @param conf configuration
+   * @return list of InetSocketAddress
+   * @throws IOException on error
+   */
+  public static Map<String, Map<String, InetSocketAddress>>
+    getNNServiceRpcAddressesForCluster(Configuration conf) throws IOException {
+    // Use default address as fall back
+    String defaultAddress;
+    try {
+      defaultAddress = NetUtils.getHostPortString(NameNode.getAddress(conf));
+    } catch (IllegalArgumentException e) {
+      defaultAddress = null;
+    }
+
+    Collection<String> parentNameServices = conf.getTrimmedStringCollection
+            (DFSConfigKeys.DFS_INTERNAL_NAMESERVICES_KEY);
+
+    if (parentNameServices.isEmpty()) {
+      parentNameServices = conf.getTrimmedStringCollection
+              (DFSConfigKeys.DFS_NAMESERVICES);
+    } else {
+      // Ensure that the internal service is ineed in the list of all available
+      // nameservices.
+      Set<String> availableNameServices = Sets.newHashSet(conf
+              .getTrimmedStringCollection(DFSConfigKeys.DFS_NAMESERVICES));
+      for (String nsId : parentNameServices) {
+        if (!availableNameServices.contains(nsId)) {
+          throw new IOException("Unknown nameservice: " + nsId);
+        }
+      }
+    }
+
+    Map<String, Map<String, InetSocketAddress>> addressList =
+            getAddressesForNsIds(conf, parentNameServices, defaultAddress,
+                    DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY, DFS_NAMENODE_RPC_ADDRESS_KEY);
+    if (addressList.isEmpty()) {
+      throw new IOException("Incorrect configuration: namenode address "
+              + DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY + " or "
+              + DFS_NAMENODE_RPC_ADDRESS_KEY
+              + " is not configured.");
+    }
+    return addressList;
+  }
+
   /**
    * Flatten the given map, as returned by other functions in this class,
    * into a flat list of {@link ConfiguredNNAddress} instances.

+ 8 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java

@@ -102,6 +102,8 @@ import com.google.common.base.Preconditions;
 public class DistributedFileSystem extends FileSystem {
   private Path workingDir;
   private URI uri;
+  private String homeDirPrefix =
+      DFSConfigKeys.DFS_USER_HOME_DIR_PREFIX_DEFAULT;
 
   DFSClient dfs;
   private boolean verifyChecksum = true;
@@ -136,7 +138,10 @@ public class DistributedFileSystem extends FileSystem {
     if (host == null) {
       throw new IOException("Incomplete HDFS URI, no host: "+ uri);
     }
-
+    homeDirPrefix = conf.get(
+        DFSConfigKeys.DFS_USER_HOME_DIR_PREFIX_KEY,
+        DFSConfigKeys.DFS_USER_HOME_DIR_PREFIX_DEFAULT);
+    
     this.dfs = new DFSClient(uri, conf, statistics);
     this.uri = URI.create(uri.getScheme()+"://"+uri.getAuthority());
     this.workingDir = getHomeDirectory();
@@ -167,10 +172,10 @@ public class DistributedFileSystem extends FileSystem {
     workingDir = fixRelativePart(dir);
   }
 
-  
   @Override
   public Path getHomeDirectory() {
-    return makeQualified(new Path("/user/" + dfs.ugi.getShortUserName()));
+    return makeQualified(new Path(homeDirPrefix + "/"
+        + dfs.ugi.getShortUserName()));
   }
 
   /**

+ 12 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/inotify/Event.java

@@ -100,6 +100,7 @@ public abstract class Event {
     private String groupName;
     private FsPermission perms;
     private String symlinkTarget;
+    private boolean overwrite;
 
     public static class Builder {
       private INodeType iNodeType;
@@ -110,6 +111,7 @@ public abstract class Event {
       private String groupName;
       private FsPermission perms;
       private String symlinkTarget;
+      private boolean overwrite;
 
       public Builder iNodeType(INodeType type) {
         this.iNodeType = type;
@@ -150,6 +152,11 @@ public abstract class Event {
         this.symlinkTarget = symlinkTarget;
         return this;
       }
+      
+      public Builder overwrite(boolean overwrite) {
+        this.overwrite = overwrite;
+        return this;
+      }
 
       public CreateEvent build() {
         return new CreateEvent(this);
@@ -166,6 +173,7 @@ public abstract class Event {
       this.groupName = b.groupName;
       this.perms = b.perms;
       this.symlinkTarget = b.symlinkTarget;
+      this.overwrite = b.overwrite;
     }
 
     public INodeType getiNodeType() {
@@ -208,6 +216,10 @@ public abstract class Event {
     public String getSymlinkTarget() {
       return symlinkTarget;
     }
+    
+    public boolean getOverwrite() {
+      return overwrite;
+    }
   }
 
   /**

+ 4 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java

@@ -2430,7 +2430,8 @@ public class PBHelper {
             .perms(convert(create.getPerms()))
             .replication(create.getReplication())
             .symlinkTarget(create.getSymlinkTarget().isEmpty() ? null :
-            create.getSymlinkTarget()).build());
+            create.getSymlinkTarget())
+            .overwrite(create.getOverwrite()).build());
         break;
       case EVENT_METADATA:
         InotifyProtos.MetadataUpdateEventProto meta =
@@ -2508,7 +2509,8 @@ public class PBHelper {
                     .setPerms(convert(ce2.getPerms()))
                     .setReplication(ce2.getReplication())
                     .setSymlinkTarget(ce2.getSymlinkTarget() == null ?
-                        "" : ce2.getSymlinkTarget()).build().toByteString()
+                        "" : ce2.getSymlinkTarget())
+                    .setOverwrite(ce2.getOverwrite()).build().toByteString()
             ).build());
         break;
       case METADATA:

+ 6 - 6
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolManager.java

@@ -149,12 +149,12 @@ class BlockPoolManager {
   
   void refreshNamenodes(Configuration conf)
       throws IOException {
-    LOG.info("Refresh request received for nameservices: "
-        + conf.get(DFSConfigKeys.DFS_NAMESERVICES));
-    
-    Map<String, Map<String, InetSocketAddress>> newAddressMap = 
-      DFSUtil.getNNServiceRpcAddresses(conf);
-    
+    LOG.info("Refresh request received for nameservices: " + conf.get
+            (DFSConfigKeys.DFS_NAMESERVICES));
+
+    Map<String, Map<String, InetSocketAddress>> newAddressMap = DFSUtil
+            .getNNServiceRpcAddressesForCluster(conf);
+
     synchronized (refreshNamenodesLock) {
       doRefreshNamenodes(newAddressMap);
     }

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java

@@ -1744,7 +1744,7 @@ public class DataNode extends Configured
             + b + " (numBytes=" + b.getNumBytes() + ")"
             + ", stage=" + stage
             + ", clientname=" + clientname
-            + ", targests=" + Arrays.asList(targets));
+            + ", targets=" + Arrays.asList(targets));
       }
       this.targets = targets;
       this.targetStorageTypes = targetStorageTypes;

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java

@@ -463,7 +463,7 @@ public class DataStorage extends Storage {
     this.clusterID = nsInfo.getClusterID();
     this.namespaceID = nsInfo.getNamespaceID();
     this.cTime = 0;
-    this.datanodeUuid = datanodeUuid;
+    setDatanodeUuid(datanodeUuid);
 
     if (sd.getStorageUuid() == null) {
       // Assign a new Storage UUID.

+ 35 - 9
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java

@@ -428,7 +428,8 @@ public class FSDirectory implements Closeable {
   /**
    * @throws SnapshotAccessControlException 
    * @see #unprotectedRenameTo(String, String, long)
-   * @deprecated Use {@link #renameTo(String, String, boolean, Rename...)}
+   * @deprecated Use {@link #renameTo(String, String, long,
+   *                                  BlocksMapUpdateInfo, Rename...)}
    */
   @Deprecated
   boolean renameTo(String src, String dst, long mtime)
@@ -452,7 +453,7 @@ public class FSDirectory implements Closeable {
    * @see #unprotectedRenameTo(String, String, long, Options.Rename...)
    */
   void renameTo(String src, String dst, long mtime,
-      Options.Rename... options)
+      BlocksMapUpdateInfo collectedBlocks, Options.Rename... options)
       throws FileAlreadyExistsException, FileNotFoundException,
       ParentNotDirectoryException, QuotaExceededException,
       UnresolvedLinkException, IOException {
@@ -462,7 +463,7 @@ public class FSDirectory implements Closeable {
     }
     writeLock();
     try {
-      if (unprotectedRenameTo(src, dst, mtime, options)) {
+      if (unprotectedRenameTo(src, dst, mtime, collectedBlocks, options)) {
         namesystem.incrDeletedFileCount(1);
       }
     } finally {
@@ -479,7 +480,7 @@ public class FSDirectory implements Closeable {
    * @throws QuotaExceededException if the operation violates any quota limit
    * @throws FileAlreadyExistsException if the src is a symlink that points to dst
    * @throws SnapshotAccessControlException if path is in RO snapshot
-   * @deprecated See {@link #renameTo(String, String, boolean, Rename...)}
+   * @deprecated See {@link #renameTo(String, String, long, BlocksMapUpdateInfo, Rename...)}
    */
   @Deprecated
   boolean unprotectedRenameTo(String src, String dst, long timestamp)
@@ -567,6 +568,30 @@ public class FSDirectory implements Closeable {
     return false;
   }
 
+  /**
+   * Rename src to dst.
+   * <br>
+   * Note: This is to be used by {@link FSEditLog} only.
+   * <br>
+   * 
+   * @param src source path
+   * @param dst destination path
+   * @param timestamp modification time
+   * @param options Rename options
+   */
+  boolean unprotectedRenameTo(String src, String dst, long timestamp,
+      Options.Rename... options) throws FileAlreadyExistsException, 
+      FileNotFoundException, ParentNotDirectoryException, 
+      QuotaExceededException, UnresolvedLinkException, IOException {
+    BlocksMapUpdateInfo collectedBlocks = new BlocksMapUpdateInfo();
+    boolean ret = unprotectedRenameTo(src, dst, timestamp, 
+        collectedBlocks, options);
+    if (!collectedBlocks.getToDeleteList().isEmpty()) {
+      getFSNamesystem().removeBlocksAndUpdateSafemodeTotal(collectedBlocks);
+    }
+    return ret;
+  }
+  
   /**
    * Rename src to dst.
    * See {@link DistributedFileSystem#rename(Path, Path, Options.Rename...)}
@@ -575,12 +600,14 @@ public class FSDirectory implements Closeable {
    * @param src source path
    * @param dst destination path
    * @param timestamp modification time
+   * @param collectedBlocks blocks to be removed
    * @param options Rename options
    */
   boolean unprotectedRenameTo(String src, String dst, long timestamp,
-      Options.Rename... options) throws FileAlreadyExistsException,
-      FileNotFoundException, ParentNotDirectoryException,
-      QuotaExceededException, UnresolvedLinkException, IOException {
+      BlocksMapUpdateInfo collectedBlocks, Options.Rename... options) 
+      throws FileAlreadyExistsException, FileNotFoundException, 
+      ParentNotDirectoryException, QuotaExceededException, 
+      UnresolvedLinkException, IOException {
     assert hasWriteLock();
     boolean overwrite = options != null && Arrays.asList(options).contains
             (Rename.OVERWRITE);
@@ -670,7 +697,6 @@ public class FSDirectory implements Closeable {
         if (removedDst != null) {
           undoRemoveDst = false;
           if (removedNum > 0) {
-            BlocksMapUpdateInfo collectedBlocks = new BlocksMapUpdateInfo();
             List<INode> removedINodes = new ChunkedArrayList<INode>();
             if (!removedDst.isInLatestSnapshot(dstIIP.getLatestSnapshotId())) {
               removedDst.destroyAndCollectBlocks(collectedBlocks, removedINodes);
@@ -680,7 +706,7 @@ public class FSDirectory implements Closeable {
                   dstIIP.getLatestSnapshotId(), collectedBlocks, removedINodes,
                   true).get(Quota.NAMESPACE) >= 0;
             }
-            getFSNamesystem().removePathAndBlocks(src, collectedBlocks,
+            getFSNamesystem().removePathAndBlocks(src, null, 
                 removedINodes, false);
           }
         }

+ 4 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java

@@ -702,7 +702,8 @@ public class FSEditLog implements LogsPurgeable {
    * Add open lease record to edit log. 
    * Records the block locations of the last block.
    */
-  public void logOpenFile(String path, INodeFile newNode, boolean toLogRpcIds) {
+  public void logOpenFile(String path, INodeFile newNode, boolean overwrite,
+      boolean toLogRpcIds) {
     Preconditions.checkArgument(newNode.isUnderConstruction());
     PermissionStatus permissions = newNode.getPermissionStatus();
     AddOp op = AddOp.getInstance(cache.get())
@@ -716,7 +717,8 @@ public class FSEditLog implements LogsPurgeable {
       .setPermissionStatus(permissions)
       .setClientName(newNode.getFileUnderConstructionFeature().getClientName())
       .setClientMachine(
-          newNode.getFileUnderConstructionFeature().getClientMachine());
+          newNode.getFileUnderConstructionFeature().getClientMachine())
+      .setOverwrite(overwrite);
 
     AclFeature f = newNode.getAclFeature();
     if (f != null) {

+ 6 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java

@@ -340,8 +340,12 @@ public class FSEditLogLoader {
 
       // See if the file already exists (persistBlocks call)
       final INodesInPath iip = fsDir.getLastINodeInPath(path);
-      final INodeFile oldFile = INodeFile.valueOf(
-          iip.getINode(0), path, true);
+      INodeFile oldFile = INodeFile.valueOf(iip.getINode(0), path, true);
+      if (oldFile != null && addCloseOp.overwrite) {
+        // This is OP_ADD with overwrite
+        fsDir.unprotectedDelete(path, addCloseOp.mtime);
+        oldFile = null;
+      }
       INodeFile newFile = oldFile;
       if (oldFile == null) { // this is OP_ADD on a new file (case 1)
         // versions > 0 support per file replication

+ 18 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java

@@ -407,6 +407,7 @@ public abstract class FSEditLogOp {
     List<XAttr> xAttrs;
     String clientName;
     String clientMachine;
+    boolean overwrite;
     
     private AddCloseOp(FSEditLogOpCodes opCode) {
       super(opCode);
@@ -486,6 +487,11 @@ public abstract class FSEditLogOp {
       this.clientMachine = clientMachine;
       return (T)this;
     }
+    
+    <T extends AddCloseOp> T setOverwrite(boolean overwrite) {
+      this.overwrite = overwrite;
+      return (T)this;
+    }
 
     @Override
     public void writeFields(DataOutputStream out) throws IOException {
@@ -505,6 +511,7 @@ public abstract class FSEditLogOp {
         b.build().writeDelimitedTo(out);
         FSImageSerialization.writeString(clientName,out);
         FSImageSerialization.writeString(clientMachine,out);
+        FSImageSerialization.writeBoolean(overwrite, out);
         // write clientId and callId
         writeRpcIds(rpcClientId, rpcCallId, out);
       }
@@ -570,6 +577,12 @@ public abstract class FSEditLogOp {
         this.xAttrs = readXAttrsFromEditLog(in, logVersion);
         this.clientName = FSImageSerialization.readString(in);
         this.clientMachine = FSImageSerialization.readString(in);
+        if (NameNodeLayoutVersion.supports(
+            NameNodeLayoutVersion.Feature.CREATE_OVERWRITE, logVersion)) {
+          this.overwrite = FSImageSerialization.readBoolean(in);
+        } else {
+          this.overwrite = false;
+        }
         // read clientId and callId
         readRpcIds(in, logVersion);
       } else {
@@ -625,6 +638,8 @@ public abstract class FSEditLogOp {
       builder.append(clientName);
       builder.append(", clientMachine=");
       builder.append(clientMachine);
+      builder.append(", overwrite=");
+      builder.append(overwrite);
       if (this.opCode == OP_ADD) {
         appendRpcIdsToString(builder, rpcClientId, rpcCallId);
       }
@@ -653,6 +668,8 @@ public abstract class FSEditLogOp {
           Long.toString(blockSize));
       XMLUtils.addSaxString(contentHandler, "CLIENT_NAME", clientName);
       XMLUtils.addSaxString(contentHandler, "CLIENT_MACHINE", clientMachine);
+      XMLUtils.addSaxString(contentHandler, "OVERWRITE", 
+          Boolean.toString(overwrite));
       for (Block b : blocks) {
         FSEditLogOp.blockToXml(contentHandler, b);
       }
@@ -676,6 +693,7 @@ public abstract class FSEditLogOp {
       this.blockSize = Long.parseLong(st.getValue("BLOCKSIZE"));
       this.clientName = st.getValue("CLIENT_NAME");
       this.clientMachine = st.getValue("CLIENT_MACHINE");
+      this.overwrite = Boolean.parseBoolean(st.getValueOrNull("OVERWRITE"));
       if (st.hasChildren("BLOCK")) {
         List<Stanza> blocks = st.getChildren("BLOCK");
         this.blocks = new Block[blocks.size()];

+ 17 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageSerialization.java

@@ -41,6 +41,7 @@ import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotFSImageFormat.Ref
 import org.apache.hadoop.hdfs.util.XMLUtils;
 import org.apache.hadoop.hdfs.util.XMLUtils.InvalidXmlException;
 import org.apache.hadoop.hdfs.util.XMLUtils.Stanza;
+import org.apache.hadoop.io.BooleanWritable;
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.ShortWritable;
@@ -88,6 +89,7 @@ public class FSImageSerialization {
     final IntWritable U_INT = new IntWritable();
     final LongWritable U_LONG = new LongWritable();
     final FsPermission FILE_PERM = new FsPermission((short) 0);
+    final BooleanWritable U_BOOLEAN = new BooleanWritable();
   }
 
   private static void writePermissionStatus(INodeAttributes inode,
@@ -366,6 +368,21 @@ public class FSImageSerialization {
     uLong.write(out);
   }
   
+  /** read the boolean value */
+  static boolean readBoolean(DataInput in) throws IOException {
+    BooleanWritable uBoolean = TL_DATA.get().U_BOOLEAN;
+    uBoolean.readFields(in);
+    return uBoolean.get();
+  }
+  
+  /** write the boolean value */
+  static void writeBoolean(boolean value, DataOutputStream out) 
+      throws IOException {
+    BooleanWritable uBoolean = TL_DATA.get().U_BOOLEAN;
+    uBoolean.set(value);
+    uBoolean.write(out);
+  }
+  
   /** read the int value */
   static int readInt(DataInput in) throws IOException {
     IntWritable uInt = TL_DATA.get().U_INT;

+ 39 - 21
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -2448,6 +2448,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
      * A special RetryStartFileException is used to indicate that we should
      * retry creation of a FileEncryptionInfo.
      */
+    BlocksMapUpdateInfo toRemoveBlocks = null;
     try {
       boolean shouldContinue = true;
       int iters = 0;
@@ -2496,9 +2497,9 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
           checkOperation(OperationCategory.WRITE);
           checkNameNodeSafeMode("Cannot create file" + src);
           src = resolvePath(src, pathComponents);
-          startFileInternal(pc, src, permissions, holder, clientMachine, create,
-              overwrite, createParent, replication, blockSize, suite, edek,
-              logRetryCache);
+          toRemoveBlocks = startFileInternal(pc, src, permissions, holder, 
+              clientMachine, create, overwrite, createParent, replication, 
+              blockSize, suite, edek, logRetryCache);
           stat = dir.getFileInfo(src, false,
               FSDirectory.isReservedRawName(srcArg));
         } catch (StandbyException se) {
@@ -2519,6 +2520,10 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       // They need to be sync'ed even when an exception was thrown.
       if (!skipSync) {
         getEditLog().logSync();
+        if (toRemoveBlocks != null) {
+          removeBlocks(toRemoveBlocks);
+          toRemoveBlocks.clear();
+        }
       }
     }
 
@@ -2535,11 +2540,11 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
    * For description of parameters and exceptions thrown see
    * {@link ClientProtocol#create}
    */
-  private void startFileInternal(FSPermissionChecker pc, String src,
-      PermissionStatus permissions, String holder, String clientMachine,
-      boolean create, boolean overwrite, boolean createParent,
-      short replication, long blockSize, CipherSuite suite,
-      EncryptedKeyVersion edek, boolean logRetryEntry)
+  private BlocksMapUpdateInfo startFileInternal(FSPermissionChecker pc, 
+      String src, PermissionStatus permissions, String holder, 
+      String clientMachine, boolean create, boolean overwrite, 
+      boolean createParent, short replication, long blockSize, 
+      CipherSuite suite, EncryptedKeyVersion edek, boolean logRetryEntry)
       throws FileAlreadyExistsException, AccessControlException,
       UnresolvedLinkException, FileNotFoundException,
       ParentNotDirectoryException, RetryStartFileException, IOException {
@@ -2575,9 +2580,12 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     if (isPermissionEnabled) {
       if (overwrite && myFile != null) {
         checkPathAccess(pc, src, FsAction.WRITE);
-      } else {
-        checkAncestorAccess(pc, src, FsAction.WRITE);
       }
+      /*
+       * To overwrite existing file, need to check 'w' permission 
+       * of parent (equals to ancestor in this case)
+       */
+      checkAncestorAccess(pc, src, FsAction.WRITE);
     }
 
     if (!createParent) {
@@ -2585,6 +2593,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     }
 
     try {
+      BlocksMapUpdateInfo toRemoveBlocks = null;
       if (myFile == null) {
         if (!create) {
           throw new FileNotFoundException("Can't overwrite non-existent " +
@@ -2592,11 +2601,12 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
         }
       } else {
         if (overwrite) {
-          try {
-            deleteInt(src, true, false); // File exists - delete if overwrite
-          } catch (AccessControlException e) {
-            logAuditEvent(false, "delete", src);
-            throw e;
+          toRemoveBlocks = new BlocksMapUpdateInfo();
+          List<INode> toRemoveINodes = new ChunkedArrayList<INode>();
+          long ret = dir.delete(src, toRemoveBlocks, toRemoveINodes, now());
+          if (ret >= 0) {
+            incrDeletedFileCount(ret);
+            removePathAndBlocks(src, null, toRemoveINodes, true);
           }
         } else {
           // If lease soft limit time is expired, recover the lease
@@ -2630,11 +2640,12 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       }
 
       // record file record in log, record new generation stamp
-      getEditLog().logOpenFile(src, newNode, logRetryEntry);
+      getEditLog().logOpenFile(src, newNode, overwrite, logRetryEntry);
       if (NameNode.stateChangeLog.isDebugEnabled()) {
         NameNode.stateChangeLog.debug("DIR* NameSystem.startFile: added " +
             src + " inode " + newNode.getId() + " " + holder);
       }
+      return toRemoveBlocks;
     } catch (IOException ie) {
       NameNode.stateChangeLog.warn("DIR* NameSystem.startFile: " + src + " " +
           ie.getMessage());
@@ -2737,7 +2748,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     }
 
     if (writeToEditLog) {
-      getEditLog().logOpenFile(src, cons, logRetryCache);
+      getEditLog().logOpenFile(src, cons, false, logRetryCache);
     }
     return ret;
   }
@@ -3627,12 +3638,14 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     HdfsFileStatus resultingStat = null;
     boolean success = false;
     writeLock();
+    BlocksMapUpdateInfo collectedBlocks = new BlocksMapUpdateInfo();
     try {
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("Cannot rename " + src);
       src = resolvePath(src, srcComponents);
       dst = resolvePath(dst, dstComponents);
-      renameToInternal(pc, src, dst, cacheEntry != null, options);
+      renameToInternal(pc, src, dst, cacheEntry != null, 
+          collectedBlocks, options);
       resultingStat = getAuditFileInfo(dst, false);
       success = true;
     } finally {
@@ -3640,6 +3653,10 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       RetryCache.setState(cacheEntry, success);
     }
     getEditLog().logSync();
+    if (!collectedBlocks.getToDeleteList().isEmpty()) {
+      removeBlocks(collectedBlocks);
+      collectedBlocks.clear();
+    }
     if (resultingStat != null) {
       StringBuilder cmd = new StringBuilder("rename options=");
       for (Rename option : options) {
@@ -3649,8 +3666,9 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     }
   }
 
-  private void renameToInternal(FSPermissionChecker pc, String src, String dst,
-      boolean logRetryCache, Options.Rename... options) throws IOException {
+  private void renameToInternal(FSPermissionChecker pc, String src, 
+      String dst, boolean logRetryCache, BlocksMapUpdateInfo collectedBlocks, 
+      Options.Rename... options) throws IOException {
     assert hasWriteLock();
     if (isPermissionEnabled) {
       // Rename does not operates on link targets
@@ -3665,7 +3683,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
 
     waitForLoadingFSImage();
     long mtime = now();
-    dir.renameTo(src, dst, mtime, options);
+    dir.renameTo(src, dst, mtime, collectedBlocks, options);
     getEditLog().logRename(src, dst, mtime, logRetryCache, options);
   }
   

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeReference.java

@@ -28,7 +28,6 @@ import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.DirectoryWithSnapshotFeature;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
-import org.apache.hadoop.hdfs.server.namenode.XAttrFeature;
 
 import com.google.common.base.Preconditions;
 
@@ -450,7 +449,8 @@ public abstract class INodeReference extends INode {
           end = mid;
         }
       }
-      if (withNameList.get(start).lastSnapshotId >= snapshotId) {
+      if (start < withNameList.size() &&
+          withNameList.get(start).lastSnapshotId >= snapshotId) {
         return withNameList.get(start);
       } else {
         return this.getParentReference();

+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/InotifyFSEditLogOpTranslator.java

@@ -50,6 +50,7 @@ public class InotifyFSEditLogOpTranslator {
             .ownerName(addOp.permissions.getUserName())
             .groupName(addOp.permissions.getGroupName())
             .perms(addOp.permissions.getPermission())
+            .overwrite(addOp.overwrite)
             .iNodeType(Event.CreateEvent.INodeType.FILE).build() };
       } else {
         return new Event[] { new Event.AppendEvent(addOp.path) };

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

@@ -65,7 +65,9 @@ public class NameNodeLayoutVersion {
   public static enum Feature implements LayoutFeature {
     ROLLING_UPGRADE(-55, -53, "Support rolling upgrade", false),
     EDITLOG_LENGTH(-56, "Add length field to every edit log op"),
-    XATTRS(-57, "Extended attributes");
+    XATTRS(-57, "Extended attributes"),
+    CREATE_OVERWRITE(-58, "Use single editlog record for " +
+        "creating file with overwrite");
     
     private final FeatureInfo info;
 

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/DirectorySnapshottableFeature.java

@@ -61,7 +61,7 @@ public class DirectorySnapshottableFeature extends DirectoryWithSnapshotFeature
   /**
    * Snapshots of this directory in ascending order of snapshot names.
    * Note that snapshots in ascending order of snapshot id are stored in
-   * {@link INodeDirectoryWithSnapshot}.diffs (a private field).
+   * {@link DirectoryWithSnapshotFeature}.diffs (a private field).
    */
   private final List<Snapshot> snapshotsByNames = new ArrayList<Snapshot>();
   /** Number of snapshots allowed. */

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/GetConf.java

@@ -186,7 +186,7 @@ public class GetConf extends Configured implements Tool {
   static class NameNodesCommandHandler extends CommandHandler {
     @Override
     int doWorkInternal(GetConf tool, String []args) throws IOException {
-      tool.printMap(DFSUtil.getNNServiceRpcAddresses(tool.getConf()));
+      tool.printMap(DFSUtil.getNNServiceRpcAddressesForCluster(tool.getConf()));
       return 0;
     }
   }
@@ -223,7 +223,7 @@ public class GetConf extends Configured implements Tool {
     public int doWorkInternal(GetConf tool, String []args) throws IOException {
       Configuration config = tool.getConf();
       List<ConfiguredNNAddress> cnnlist = DFSUtil.flattenAddressMap(
-          DFSUtil.getNNServiceRpcAddresses(config));
+          DFSUtil.getNNServiceRpcAddressesForCluster(config));
       if (!cnnlist.isEmpty()) {
         for (ConfiguredNNAddress cnn : cnnlist) {
           InetSocketAddress rpc = cnn.getAddress();

+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/proto/inotify.proto

@@ -72,6 +72,7 @@ message CreateEventProto {
   required FsPermissionProto perms = 6;
   optional int32 replication = 7;
   optional string symlinkTarget = 8;
+  optional bool overwrite = 9;
 }
 
 message CloseEventProto {

+ 18 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml

@@ -1115,6 +1115,16 @@
   </description>
 </property>
 
+<property>
+  <name>dfs.internal.nameservices</name>
+  <value></value>
+  <description>
+    Comma-separated list of nameservices that belong to this cluster.
+    Datanode will report to all the nameservices in this list. By default
+    this is set to the value of dfs.nameservices.
+  </description>
+</property>
+
 <property>
   <name>dfs.ha.namenodes.EXAMPLENAMESERVICE</name>
   <value></value>
@@ -2098,4 +2108,12 @@
   </description>
 </property>
 
+<property>
+  <name>dfs.user.home.dir.prefix</name>
+  <value>/user</value>
+  <description>The directory to prepend to user name to get the user's
+    home direcotry.
+  </description>
+</property>
+
 </configuration>

+ 15 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlocksScheduledCounter.java

@@ -27,6 +27,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
+import org.junit.After;
 import org.junit.Test;
 
 /**
@@ -35,14 +36,25 @@ import org.junit.Test;
  * scheduled to a datanode.
  */
 public class TestBlocksScheduledCounter {
+  MiniDFSCluster cluster = null;
+  FileSystem fs = null;
+
+  @After
+  public void tearDown() throws IOException {
+    if (fs != null) {
+      fs.close();
+    }
+    if(cluster!=null){
+      cluster.shutdown();
+    }
+  }
 
   @Test
   public void testBlocksScheduledCounter() throws IOException {
-    MiniDFSCluster cluster = new MiniDFSCluster.Builder(new HdfsConfiguration())
-                                               .build();
+    cluster = new MiniDFSCluster.Builder(new HdfsConfiguration()).build();
 
     cluster.waitActive();
-    FileSystem fs = cluster.getFileSystem();
+    fs = cluster.getFileSystem();
     
     //open a file an write a few bytes:
     FSDataOutputStream out = fs.create(new Path("/testBlockScheduledCounter"));

+ 2 - 6
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSInotifyEventInputStream.java

@@ -147,12 +147,7 @@ public class TestDFSInotifyEventInputStream {
       Assert.assertTrue(re2.getSrcPath().equals("/file4"));
       Assert.assertTrue(re.getTimestamp() > 0);
 
-      // DeleteOp
-      next = waitForNextEvent(eis);
-      Assert.assertTrue(next.getEventType() == Event.EventType.UNLINK);
-      Assert.assertTrue(((Event.UnlinkEvent) next).getPath().equals("/file2"));
-
-      // AddOp
+      // AddOp with overwrite
       next = waitForNextEvent(eis);
       Assert.assertTrue(next.getEventType() == Event.EventType.CREATE);
       Event.CreateEvent ce = (Event.CreateEvent) next;
@@ -161,6 +156,7 @@ public class TestDFSInotifyEventInputStream {
       Assert.assertTrue(ce.getCtime() > 0);
       Assert.assertTrue(ce.getReplication() > 0);
       Assert.assertTrue(ce.getSymlinkTarget() == null);
+      Assert.assertTrue(ce.getOverwrite());
 
       // CloseOp
       next = waitForNextEvent(eis);

+ 6 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSRename.java

@@ -131,6 +131,7 @@ public class TestDFSRename {
   
   /**
    * Check the blocks of dst file are cleaned after rename with overwrite
+   * Restart NN to check the rename successfully
    */
   @Test(timeout = 120000)
   public void testRenameWithOverwrite() throws Exception {
@@ -160,6 +161,11 @@ public class TestDFSRename {
       dfs.rename(srcPath, dstPath, Rename.OVERWRITE);
       assertTrue(bm.getStoredBlock(lbs.getLocatedBlocks().get(0).getBlock().
           getLocalBlock()) == null);
+      
+      // Restart NN and check the rename successfully
+      cluster.restartNameNodes();
+      assertFalse(dfs.exists(srcPath));
+      assertTrue(dfs.exists(dstPath));
     } finally {
       if (dfs != null) {
         dfs.close();

+ 26 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java

@@ -21,6 +21,7 @@ package org.apache.hadoop.hdfs;
 import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_FAILOVER_PROXY_PROVIDER_KEY_PREFIX;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HA_NAMENODES_KEY_PREFIX;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_INTERNAL_NAMESERVICES_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_BACKUP_ADDRESS_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_HTTPS_PORT_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY;
@@ -865,4 +866,29 @@ public class TestDFSUtil {
     // let's make sure that a password that doesn't exist returns null
     Assert.assertEquals(null, DFSUtil.getPassword(conf,"invalid-alias"));
   }
+
+  @Test
+  public void testGetNNServiceRpcAddressesForNsIds() throws IOException {
+    Configuration conf = new HdfsConfiguration();
+    conf.set(DFS_NAMESERVICES, "nn1,nn2");
+    conf.set(DFS_INTERNAL_NAMESERVICES_KEY, "nn1");
+    // Test - configured list of namenodes are returned
+    final String NN1_ADDRESS = "localhost:9000";
+    final String NN2_ADDRESS = "localhost:9001";
+    conf.set(DFSUtil.addKeySuffixes(DFS_NAMENODE_RPC_ADDRESS_KEY, "nn1"),
+            NN1_ADDRESS);
+    conf.set(DFSUtil.addKeySuffixes(DFS_NAMENODE_RPC_ADDRESS_KEY, "nn2"),
+            NN2_ADDRESS);
+
+    Map<String, Map<String, InetSocketAddress>> nnMap = DFSUtil
+            .getNNServiceRpcAddressesForCluster(conf);
+    assertEquals(1, nnMap.size());
+    assertTrue(nnMap.containsKey("nn1"));
+    conf.set(DFS_INTERNAL_NAMESERVICES_KEY, "nn3");
+    try {
+      DFSUtil.getNNServiceRpcAddressesForCluster(conf);
+      fail("Should fail for misconfiguration");
+    } catch (IOException ignored) {
+    }
+  }
 }

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend4.java

@@ -125,7 +125,7 @@ public class TestFileAppend4 {
     while (!recovered && tries-- > 0) {
       try {
         out = fs.append(file1);
-        LOG.info("Successfully opened for appends");
+        LOG.info("Successfully opened for append");
         recovered = true;
       } catch (IOException e) {
         LOG.info("Failed open for append, waiting on lease recovery");

+ 119 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreation.java

@@ -38,6 +38,7 @@ import static org.junit.Assert.fail;
 import static org.junit.Assume.assumeTrue;
 
 import java.io.BufferedReader;
+import java.io.ByteArrayOutputStream;
 import java.io.File;
 import java.io.FileNotFoundException;
 import java.io.FileReader;
@@ -70,6 +71,7 @@ import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
 import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset;
@@ -78,6 +80,8 @@ import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.INodeId;
 import org.apache.hadoop.hdfs.server.namenode.LeaseExpiredException;
 import org.apache.hadoop.hdfs.server.namenode.LeaseManager;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
+import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
 import org.apache.hadoop.io.EnumSetWritable;
 import org.apache.hadoop.io.IOUtils;
@@ -86,6 +90,7 @@ import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.util.Time;
 import org.apache.log4j.Level;
+import org.junit.Assert;
 import org.junit.Test;
 
 /**
@@ -1210,4 +1215,118 @@ public class TestFileCreation {
     }
   }
 
+  /**
+   * 1. Check the blocks of old file are cleaned after creating with overwrite
+   * 2. Restart NN, check the file
+   * 3. Save new checkpoint and restart NN, check the file
+   */
+  @Test(timeout = 120000)
+  public void testFileCreationWithOverwrite() throws Exception {
+    Configuration conf = new Configuration();
+    conf.setInt("dfs.blocksize", blockSize);
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).
+        numDataNodes(3).build();
+    DistributedFileSystem dfs = cluster.getFileSystem();
+    try {
+      dfs.mkdirs(new Path("/foo/dir"));
+      String file = "/foo/dir/file";
+      Path filePath = new Path(file);
+      
+      // Case 1: Create file with overwrite, check the blocks of old file
+      // are cleaned after creating with overwrite
+      NameNode nn = cluster.getNameNode();
+      FSNamesystem fsn = NameNodeAdapter.getNamesystem(nn);
+      BlockManager bm = fsn.getBlockManager();
+      
+      FSDataOutputStream out = dfs.create(filePath);
+      byte[] oldData = AppendTestUtil.randomBytes(seed, fileSize);
+      try {
+        out.write(oldData);
+      } finally {
+        out.close();
+      }
+      
+      LocatedBlocks oldBlocks = NameNodeAdapter.getBlockLocations(
+          nn, file, 0, fileSize);
+      assertBlocks(bm, oldBlocks, true);
+      
+      out = dfs.create(filePath, true);
+      byte[] newData = AppendTestUtil.randomBytes(seed, fileSize);
+      try {
+        out.write(newData);
+      } finally {
+        out.close();
+      }
+      dfs.deleteOnExit(filePath);
+      
+      LocatedBlocks newBlocks = NameNodeAdapter.getBlockLocations(
+          nn, file, 0, fileSize);
+      assertBlocks(bm, newBlocks, true);
+      assertBlocks(bm, oldBlocks, false);
+      
+      FSDataInputStream in = dfs.open(filePath);
+      byte[] result = null;
+      try {
+        result = readAll(in);
+      } finally {
+        in.close();
+      }
+      Assert.assertArrayEquals(newData, result);
+      
+      // Case 2: Restart NN, check the file
+      cluster.restartNameNode();
+      nn = cluster.getNameNode();
+      in = dfs.open(filePath);
+      try {
+        result = readAll(in);
+      } finally {
+        in.close();
+      }
+      Assert.assertArrayEquals(newData, result);
+      
+      // Case 3: Save new checkpoint and restart NN, check the file
+      NameNodeAdapter.enterSafeMode(nn, false);
+      NameNodeAdapter.saveNamespace(nn);
+      cluster.restartNameNode();
+      nn = cluster.getNameNode();
+      
+      in = dfs.open(filePath);
+      try {
+        result = readAll(in);
+      } finally {
+        in.close();
+      }
+      Assert.assertArrayEquals(newData, result);
+    } finally {
+      if (dfs != null) {
+        dfs.close();
+      }
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
+  
+  private void assertBlocks(BlockManager bm, LocatedBlocks lbs, 
+      boolean exist) {
+    for (LocatedBlock locatedBlock : lbs.getLocatedBlocks()) {
+      if (exist) {
+        assertTrue(bm.getStoredBlock(locatedBlock.getBlock().
+            getLocalBlock()) != null);
+      } else {
+        assertTrue(bm.getStoredBlock(locatedBlock.getBlock().
+            getLocalBlock()) == null);
+      }
+    }
+  }
+  
+  private byte[] readAll(FSDataInputStream in) throws IOException {
+    ByteArrayOutputStream out = new ByteArrayOutputStream();
+    byte[] buffer = new byte[1024];
+    int n = 0;
+    while((n = in.read(buffer)) > -1) {
+      out.write(buffer, 0, n);
+    }
+    return out.toByteArray();
+  }
 }

+ 28 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLocalDFS.java

@@ -90,7 +90,9 @@ public class TestLocalDFS {
 
       // test home directory
       Path home = 
-        fileSys.makeQualified(new Path("/user/" + getUserName(fileSys))); 
+        fileSys.makeQualified(
+            new Path(DFSConfigKeys.DFS_USER_HOME_DIR_PREFIX_DEFAULT
+                + "/" + getUserName(fileSys))); 
       Path fsHome = fileSys.getHomeDirectory();
       assertEquals(home, fsHome);
 
@@ -99,4 +101,29 @@ public class TestLocalDFS {
       cluster.shutdown();
     }
   }
+  
+  /**
+   * Tests get/set working directory in DFS.
+   */
+  @Test(timeout=30000)
+  public void testHomeDirectory() throws IOException {
+    final String[] homeBases = new String[] {"/home", "/home/user"};
+    Configuration conf = new HdfsConfiguration();
+    for (final String homeBase : homeBases) {
+      conf.set(DFSConfigKeys.DFS_USER_HOME_DIR_PREFIX_KEY, homeBase);
+      MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();
+      FileSystem fileSys = cluster.getFileSystem();
+      try {    
+        // test home directory
+        Path home = 
+            fileSys.makeQualified(
+                new Path(homeBase + "/" + getUserName(fileSys))); 
+        Path fsHome = fileSys.getHomeDirectory();
+        assertEquals(home, fsHome);
+      } finally {
+        fileSys.close();
+        cluster.shutdown();
+      }
+    }
+  }
 }

+ 22 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockPoolManager.java

@@ -23,15 +23,18 @@ import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.util.ArrayList;
 import java.util.List;
+import java.util.Map;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
+import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 import org.mockito.Mockito;
+import org.mockito.internal.util.reflection.Whitebox;
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
 
@@ -130,6 +133,25 @@ public class TestBlockPoolManager {
         "refresh #2\n", log.toString());
   }
 
+  @Test
+  public void testInternalNameService() throws Exception {
+    Configuration conf = new Configuration();
+    conf.set(DFSConfigKeys.DFS_NAMESERVICES, "ns1,ns2,ns3");
+    addNN(conf, "ns1", "mock1:8020");
+    addNN(conf, "ns2", "mock1:8020");
+    addNN(conf, "ns3", "mock1:8020");
+    conf.set(DFSConfigKeys.DFS_INTERNAL_NAMESERVICES_KEY, "ns1");
+    bpm.refreshNamenodes(conf);
+    assertEquals("create #1\n", log.toString());
+    @SuppressWarnings("unchecked")
+    Map<String, BPOfferService> map = (Map<String, BPOfferService>) Whitebox
+            .getInternalState(bpm, "bpByNameserviceId");
+    Assert.assertFalse(map.containsKey("ns2"));
+    Assert.assertFalse(map.containsKey("ns3"));
+    Assert.assertTrue(map.containsKey("ns1"));
+    log.setLength(0);
+  }
+
   private static void addNN(Configuration conf, String ns, String addr) {
     String key = DFSUtil.addKeySuffixes(
         DFSConfigKeys.DFS_NAMENODE_RPC_ADDRESS_KEY, ns);

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/CreateEditsLog.java

@@ -99,7 +99,7 @@ public class CreateEditsLog {
       INodeFile fileUc = new INodeFile(inodeId.nextValue(), null,
           p, 0L, 0L, BlockInfo.EMPTY_ARRAY, replication, blockSize);
       fileUc.toUnderConstruction("", "");
-      editLog.logOpenFile(filePath, fileUc, false);
+      editLog.logOpenFile(filePath, fileUc, false, false);
       editLog.logCloseFile(filePath, inode);
 
       if (currentBlockId - bidAtSync >= 2000) { // sync every 2K blocks

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java

@@ -197,7 +197,7 @@ public class TestEditLog {
             p, 0L, 0L, BlockInfo.EMPTY_ARRAY, replication, blockSize);
         inode.toUnderConstruction("", "");
 
-        editLog.logOpenFile("/filename" + (startIndex + i), inode, false);
+        editLog.logOpenFile("/filename" + (startIndex + i), inode, false, false);
         editLog.logCloseFile("/filename" + (startIndex + i), inode);
         editLog.logSync();
       }

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStartup.java

@@ -441,7 +441,7 @@ public class TestStartup {
     checkNameSpace(conf);
 
     // read an image compressed in Gzip and store it uncompressed
-    LOG.info("Read an compressed iamge and store it as uncompressed.");
+    LOG.info("Read a compressed image and store it as uncompressed.");
     conf.setBoolean(DFSConfigKeys.DFS_IMAGE_COMPRESS_KEY, false);
     checkNameSpace(conf);
 

+ 36 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotDiffReport.java

@@ -492,4 +492,40 @@ public class TestSnapshotDiffReport {
         new DiffReportEntry(DiffType.RENAME, DFSUtil.string2Bytes("foo"),
             DFSUtil.string2Bytes("bar")));
   }
+
+  /**
+   * Nested renamed dir/file and the withNameList in the WithCount node of the
+   * parental directory is empty due to snapshot deletion. See HDFS-6996 for
+   * details.
+   */
+  @Test
+  public void testDiffReportWithRenameAndSnapshotDeletion() throws Exception {
+    final Path root = new Path("/");
+    final Path foo = new Path(root, "foo");
+    final Path bar = new Path(foo, "bar");
+    DFSTestUtil.createFile(hdfs, bar, BLOCKSIZE, REPLICATION, seed);
+
+    SnapshotTestHelper.createSnapshot(hdfs, root, "s0");
+    // rename /foo to /foo2
+    final Path foo2 = new Path(root, "foo2");
+    hdfs.rename(foo, foo2);
+    // now /foo/bar becomes /foo2/bar
+    final Path bar2 = new Path(foo2, "bar");
+
+    // delete snapshot s0 so that the withNameList inside of the WithCount node
+    // of foo becomes empty
+    hdfs.deleteSnapshot(root, "s0");
+
+    // create snapshot s1 and rename bar again
+    SnapshotTestHelper.createSnapshot(hdfs, root, "s1");
+    final Path bar3 = new Path(foo2, "bar-new");
+    hdfs.rename(bar2, bar3);
+
+    // we always put modification on the file before rename
+    verifyDiffReport(root, "s1", "",
+        new DiffReportEntry(DiffType.MODIFY, DFSUtil.string2Bytes("")),
+        new DiffReportEntry(DiffType.MODIFY, DFSUtil.string2Bytes("foo2")),
+        new DiffReportEntry(DiffType.RENAME, DFSUtil.string2Bytes("foo2/bar"),
+            DFSUtil.string2Bytes("foo2/bar-new")));
+  }
 }

+ 22 - 4
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestGetConf.java

@@ -18,11 +18,13 @@
 package org.apache.hadoop.hdfs.tools;
 
 import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_INTERNAL_NAMESERVICES_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_BACKUP_ADDRESS_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_RPC_ADDRESS_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_SECONDARY_HTTP_ADDRESS_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMESERVICES;
+import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
@@ -121,13 +123,13 @@ public class TestGetConf {
       TestType type, HdfsConfiguration conf) throws IOException {
     switch (type) {
     case NAMENODE:
-      return DFSUtil.getNNServiceRpcAddresses(conf);
+      return DFSUtil.getNNServiceRpcAddressesForCluster(conf);
     case BACKUP:
       return DFSUtil.getBackupNodeAddresses(conf);
     case SECONDARY:
       return DFSUtil.getSecondaryNameNodeAddresses(conf);
     case NNRPCADDRESSES:
-      return DFSUtil.getNNServiceRpcAddresses(conf);
+      return DFSUtil.getNNServiceRpcAddressesForCluster(conf);
     }
     return null;
   }
@@ -226,7 +228,7 @@ public class TestGetConf {
     String[] actual = toStringArray(list);
     Arrays.sort(actual);
     Arrays.sort(expected);
-    assertTrue(Arrays.equals(expected, actual));
+    assertArrayEquals(expected, actual);
 
     // Test GetConf returned addresses
     getAddressListFromTool(type, conf, checkPort, list);
@@ -425,7 +427,23 @@ public class TestGetConf {
     assertEquals(hostsFile.toUri().getPath(),ret.trim());
     cleanupFile(localFileSys, excludeFile.getParent());
   }
-  
+
+  @Test
+  public void testIncludeInternalNameServices() throws Exception {
+    final int nsCount = 10;
+    final int remoteNsCount = 4;
+    HdfsConfiguration conf = new HdfsConfiguration();
+    setupNameServices(conf, nsCount);
+    setupAddress(conf, DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY, nsCount, 1000);
+    setupAddress(conf, DFS_NAMENODE_RPC_ADDRESS_KEY, nsCount, 1500);
+    conf.set(DFS_INTERNAL_NAMESERVICES_KEY, "ns1");
+    setupStaticHostResolution(nsCount);
+
+    String[] includedNN = new String[] {"nn1:1001"};
+    verifyAddresses(conf, TestType.NAMENODE, false, includedNN);
+    verifyAddresses(conf, TestType.NNRPCADDRESSES, true, includedNN);
+  }
+
   private void writeConfigFile(Path name, ArrayList<String> nodes) 
       throws IOException {
       // delete if it already exists

BIN
hadoop-hdfs-project/hadoop-hdfs/src/test/resources/editsStored


+ 119 - 63
hadoop-hdfs-project/hadoop-hdfs/src/test/resources/editsStored.xml

@@ -1,6 +1,6 @@
 <?xml version="1.0" encoding="UTF-8"?>
 <EDITS>
-  <EDITS_VERSION>-57</EDITS_VERSION>
+  <EDITS_VERSION>-58</EDITS_VERSION>
   <RECORD>
     <OPCODE>OP_START_LOG_SEGMENT</OPCODE>
     <DATA>
@@ -42,6 +42,7 @@
       <BLOCKSIZE>512</BLOCKSIZE>
       <CLIENT_NAME>DFSClient_NONMAPREDUCE_1233039831_1</CLIENT_NAME>
       <CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
+      <OVERWRITE>false</OVERWRITE>
       <PERMISSION_STATUS>
         <USERNAME>andrew</USERNAME>
         <GROUPNAME>supergroup</GROUPNAME>
@@ -64,6 +65,7 @@
       <BLOCKSIZE>512</BLOCKSIZE>
       <CLIENT_NAME></CLIENT_NAME>
       <CLIENT_MACHINE></CLIENT_MACHINE>
+      <OVERWRITE>false</OVERWRITE>
       <PERMISSION_STATUS>
         <USERNAME>andrew</USERNAME>
         <GROUPNAME>supergroup</GROUPNAME>
@@ -174,6 +176,7 @@
       <BLOCKSIZE>512</BLOCKSIZE>
       <CLIENT_NAME>DFSClient_NONMAPREDUCE_1233039831_1</CLIENT_NAME>
       <CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
+      <OVERWRITE>false</OVERWRITE>
       <PERMISSION_STATUS>
         <USERNAME>andrew</USERNAME>
         <GROUPNAME>supergroup</GROUPNAME>
@@ -196,6 +199,7 @@
       <BLOCKSIZE>512</BLOCKSIZE>
       <CLIENT_NAME></CLIENT_NAME>
       <CLIENT_MACHINE></CLIENT_MACHINE>
+      <OVERWRITE>false</OVERWRITE>
       <PERMISSION_STATUS>
         <USERNAME>andrew</USERNAME>
         <GROUPNAME>supergroup</GROUPNAME>
@@ -204,17 +208,61 @@
     </DATA>
   </RECORD>
   <RECORD>
-    <OPCODE>OP_SET_REPLICATION</OPCODE>
+    <OPCODE>OP_ADD</OPCODE>
     <DATA>
       <TXID>17</TXID>
+      <LENGTH>0</LENGTH>
+      <INODEID>16388</INODEID>
       <PATH>/file_create</PATH>
       <REPLICATION>1</REPLICATION>
+      <MTIME>1402899229912</MTIME>
+      <ATIME>1402899229912</ATIME>
+      <BLOCKSIZE>512</BLOCKSIZE>
+      <CLIENT_NAME>DFSClient_NONMAPREDUCE_1233039831_1</CLIENT_NAME>
+      <CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
+      <OVERWRITE>true</OVERWRITE>
+      <PERMISSION_STATUS>
+        <USERNAME>andrew</USERNAME>
+        <GROUPNAME>supergroup</GROUPNAME>
+        <MODE>420</MODE>
+      </PERMISSION_STATUS>
+      <RPC_CLIENTID>e03f4a52-3d85-4e05-8942-286185e639bd</RPC_CLIENTID>
+      <RPC_CALLID>21</RPC_CALLID>
     </DATA>
   </RECORD>
   <RECORD>
-    <OPCODE>OP_SET_PERMISSIONS</OPCODE>
+    <OPCODE>OP_CLOSE</OPCODE>
     <DATA>
       <TXID>18</TXID>
+      <LENGTH>0</LENGTH>
+      <INODEID>0</INODEID>
+      <PATH>/file_create</PATH>
+      <REPLICATION>1</REPLICATION>
+      <MTIME>1402899229931</MTIME>
+      <ATIME>1402899229912</ATIME>
+      <BLOCKSIZE>512</BLOCKSIZE>
+      <CLIENT_NAME></CLIENT_NAME>
+      <CLIENT_MACHINE></CLIENT_MACHINE>
+      <OVERWRITE>false</OVERWRITE>
+      <PERMISSION_STATUS>
+        <USERNAME>andrew</USERNAME>
+        <GROUPNAME>supergroup</GROUPNAME>
+        <MODE>420</MODE>
+      </PERMISSION_STATUS>
+    </DATA>
+  </RECORD>
+  <RECORD>
+    <OPCODE>OP_SET_REPLICATION</OPCODE>
+    <DATA>
+      <TXID>19</TXID>
+      <PATH>/file_create</PATH>
+      <REPLICATION>1</REPLICATION>
+    </DATA>
+  </RECORD>
+  <RECORD>
+    <OPCODE>OP_SET_PERMISSIONS</OPCODE>
+    <DATA>
+      <TXID>20</TXID>
       <SRC>/file_create</SRC>
       <MODE>511</MODE>
     </DATA>
@@ -222,7 +270,7 @@
   <RECORD>
     <OPCODE>OP_SET_OWNER</OPCODE>
     <DATA>
-      <TXID>19</TXID>
+      <TXID>21</TXID>
       <SRC>/file_create</SRC>
       <USERNAME>newOwner</USERNAME>
     </DATA>
@@ -230,7 +278,7 @@
   <RECORD>
     <OPCODE>OP_TIMES</OPCODE>
     <DATA>
-      <TXID>20</TXID>
+      <TXID>22</TXID>
       <LENGTH>0</LENGTH>
       <PATH>/file_create</PATH>
       <MTIME>1285195527000</MTIME>
@@ -240,7 +288,7 @@
   <RECORD>
     <OPCODE>OP_SET_QUOTA</OPCODE>
     <DATA>
-      <TXID>21</TXID>
+      <TXID>23</TXID>
       <SRC>/directory_mkdir</SRC>
       <NSQUOTA>1000</NSQUOTA>
       <DSQUOTA>-1</DSQUOTA>
@@ -249,7 +297,7 @@
   <RECORD>
     <OPCODE>OP_RENAME</OPCODE>
     <DATA>
-      <TXID>22</TXID>
+      <TXID>24</TXID>
       <LENGTH>0</LENGTH>
       <SRC>/file_create</SRC>
       <DST>/file_moved</DST>
@@ -262,7 +310,7 @@
   <RECORD>
     <OPCODE>OP_ADD</OPCODE>
     <DATA>
-      <TXID>23</TXID>
+      <TXID>25</TXID>
       <LENGTH>0</LENGTH>
       <INODEID>16389</INODEID>
       <PATH>/file_concat_target</PATH>
@@ -272,6 +320,7 @@
       <BLOCKSIZE>512</BLOCKSIZE>
       <CLIENT_NAME>DFSClient_NONMAPREDUCE_1233039831_1</CLIENT_NAME>
       <CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
+      <OVERWRITE>false</OVERWRITE>
       <PERMISSION_STATUS>
         <USERNAME>andrew</USERNAME>
         <GROUPNAME>supergroup</GROUPNAME>
@@ -284,21 +333,21 @@
   <RECORD>
     <OPCODE>OP_ALLOCATE_BLOCK_ID</OPCODE>
     <DATA>
-      <TXID>24</TXID>
+      <TXID>26</TXID>
       <BLOCK_ID>1073741825</BLOCK_ID>
     </DATA>
   </RECORD>
   <RECORD>
     <OPCODE>OP_SET_GENSTAMP_V2</OPCODE>
     <DATA>
-      <TXID>25</TXID>
+      <TXID>27</TXID>
       <GENSTAMPV2>1001</GENSTAMPV2>
     </DATA>
   </RECORD>
   <RECORD>
     <OPCODE>OP_ADD_BLOCK</OPCODE>
     <DATA>
-      <TXID>26</TXID>
+      <TXID>28</TXID>
       <PATH>/file_concat_target</PATH>
       <BLOCK>
         <BLOCK_ID>1073741825</BLOCK_ID>
@@ -312,21 +361,21 @@
   <RECORD>
     <OPCODE>OP_ALLOCATE_BLOCK_ID</OPCODE>
     <DATA>
-      <TXID>27</TXID>
+      <TXID>29</TXID>
       <BLOCK_ID>1073741826</BLOCK_ID>
     </DATA>
   </RECORD>
   <RECORD>
     <OPCODE>OP_SET_GENSTAMP_V2</OPCODE>
     <DATA>
-      <TXID>28</TXID>
+      <TXID>30</TXID>
       <GENSTAMPV2>1002</GENSTAMPV2>
     </DATA>
   </RECORD>
   <RECORD>
     <OPCODE>OP_ADD_BLOCK</OPCODE>
     <DATA>
-      <TXID>29</TXID>
+      <TXID>31</TXID>
       <PATH>/file_concat_target</PATH>
       <BLOCK>
         <BLOCK_ID>1073741825</BLOCK_ID>
@@ -345,21 +394,21 @@
   <RECORD>
     <OPCODE>OP_ALLOCATE_BLOCK_ID</OPCODE>
     <DATA>
-      <TXID>30</TXID>
+      <TXID>32</TXID>
       <BLOCK_ID>1073741827</BLOCK_ID>
     </DATA>
   </RECORD>
   <RECORD>
     <OPCODE>OP_SET_GENSTAMP_V2</OPCODE>
     <DATA>
-      <TXID>31</TXID>
+      <TXID>33</TXID>
       <GENSTAMPV2>1003</GENSTAMPV2>
     </DATA>
   </RECORD>
   <RECORD>
     <OPCODE>OP_ADD_BLOCK</OPCODE>
     <DATA>
-      <TXID>32</TXID>
+      <TXID>34</TXID>
       <PATH>/file_concat_target</PATH>
       <BLOCK>
         <BLOCK_ID>1073741826</BLOCK_ID>
@@ -378,7 +427,7 @@
   <RECORD>
     <OPCODE>OP_CLOSE</OPCODE>
     <DATA>
-      <TXID>33</TXID>
+      <TXID>35</TXID>
       <LENGTH>0</LENGTH>
       <INODEID>0</INODEID>
       <PATH>/file_concat_target</PATH>
@@ -388,6 +437,7 @@
       <BLOCKSIZE>512</BLOCKSIZE>
       <CLIENT_NAME></CLIENT_NAME>
       <CLIENT_MACHINE></CLIENT_MACHINE>
+      <OVERWRITE>false</OVERWRITE>
       <BLOCK>
         <BLOCK_ID>1073741825</BLOCK_ID>
         <NUM_BYTES>512</NUM_BYTES>
@@ -413,7 +463,7 @@
   <RECORD>
     <OPCODE>OP_ADD</OPCODE>
     <DATA>
-      <TXID>34</TXID>
+      <TXID>36</TXID>
       <LENGTH>0</LENGTH>
       <INODEID>16390</INODEID>
       <PATH>/file_concat_0</PATH>
@@ -423,6 +473,7 @@
       <BLOCKSIZE>512</BLOCKSIZE>
       <CLIENT_NAME>DFSClient_NONMAPREDUCE_1233039831_1</CLIENT_NAME>
       <CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
+      <OVERWRITE>false</OVERWRITE>
       <PERMISSION_STATUS>
         <USERNAME>andrew</USERNAME>
         <GROUPNAME>supergroup</GROUPNAME>
@@ -435,21 +486,21 @@
   <RECORD>
     <OPCODE>OP_ALLOCATE_BLOCK_ID</OPCODE>
     <DATA>
-      <TXID>35</TXID>
+      <TXID>37</TXID>
       <BLOCK_ID>1073741828</BLOCK_ID>
     </DATA>
   </RECORD>
   <RECORD>
     <OPCODE>OP_SET_GENSTAMP_V2</OPCODE>
     <DATA>
-      <TXID>36</TXID>
+      <TXID>38</TXID>
       <GENSTAMPV2>1004</GENSTAMPV2>
     </DATA>
   </RECORD>
   <RECORD>
     <OPCODE>OP_ADD_BLOCK</OPCODE>
     <DATA>
-      <TXID>37</TXID>
+      <TXID>39</TXID>
       <PATH>/file_concat_0</PATH>
       <BLOCK>
         <BLOCK_ID>1073741828</BLOCK_ID>
@@ -463,21 +514,21 @@
   <RECORD>
     <OPCODE>OP_ALLOCATE_BLOCK_ID</OPCODE>
     <DATA>
-      <TXID>38</TXID>
+      <TXID>40</TXID>
       <BLOCK_ID>1073741829</BLOCK_ID>
     </DATA>
   </RECORD>
   <RECORD>
     <OPCODE>OP_SET_GENSTAMP_V2</OPCODE>
     <DATA>
-      <TXID>39</TXID>
+      <TXID>41</TXID>
       <GENSTAMPV2>1005</GENSTAMPV2>
     </DATA>
   </RECORD>
   <RECORD>
     <OPCODE>OP_ADD_BLOCK</OPCODE>
     <DATA>
-      <TXID>40</TXID>
+      <TXID>42</TXID>
       <PATH>/file_concat_0</PATH>
       <BLOCK>
         <BLOCK_ID>1073741828</BLOCK_ID>
@@ -496,21 +547,21 @@
   <RECORD>
     <OPCODE>OP_ALLOCATE_BLOCK_ID</OPCODE>
     <DATA>
-      <TXID>41</TXID>
+      <TXID>43</TXID>
       <BLOCK_ID>1073741830</BLOCK_ID>
     </DATA>
   </RECORD>
   <RECORD>
     <OPCODE>OP_SET_GENSTAMP_V2</OPCODE>
     <DATA>
-      <TXID>42</TXID>
+      <TXID>44</TXID>
       <GENSTAMPV2>1006</GENSTAMPV2>
     </DATA>
   </RECORD>
   <RECORD>
     <OPCODE>OP_ADD_BLOCK</OPCODE>
     <DATA>
-      <TXID>43</TXID>
+      <TXID>45</TXID>
       <PATH>/file_concat_0</PATH>
       <BLOCK>
         <BLOCK_ID>1073741829</BLOCK_ID>
@@ -529,7 +580,7 @@
   <RECORD>
     <OPCODE>OP_CLOSE</OPCODE>
     <DATA>
-      <TXID>44</TXID>
+      <TXID>46</TXID>
       <LENGTH>0</LENGTH>
       <INODEID>0</INODEID>
       <PATH>/file_concat_0</PATH>
@@ -539,6 +590,7 @@
       <BLOCKSIZE>512</BLOCKSIZE>
       <CLIENT_NAME></CLIENT_NAME>
       <CLIENT_MACHINE></CLIENT_MACHINE>
+      <OVERWRITE>false</OVERWRITE>
       <BLOCK>
         <BLOCK_ID>1073741828</BLOCK_ID>
         <NUM_BYTES>512</NUM_BYTES>
@@ -564,7 +616,7 @@
   <RECORD>
     <OPCODE>OP_ADD</OPCODE>
     <DATA>
-      <TXID>45</TXID>
+      <TXID>47</TXID>
       <LENGTH>0</LENGTH>
       <INODEID>16391</INODEID>
       <PATH>/file_concat_1</PATH>
@@ -574,6 +626,7 @@
       <BLOCKSIZE>512</BLOCKSIZE>
       <CLIENT_NAME>DFSClient_NONMAPREDUCE_1233039831_1</CLIENT_NAME>
       <CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
+      <OVERWRITE>false</OVERWRITE>
       <PERMISSION_STATUS>
         <USERNAME>andrew</USERNAME>
         <GROUPNAME>supergroup</GROUPNAME>
@@ -586,21 +639,21 @@
   <RECORD>
     <OPCODE>OP_ALLOCATE_BLOCK_ID</OPCODE>
     <DATA>
-      <TXID>46</TXID>
+      <TXID>48</TXID>
       <BLOCK_ID>1073741831</BLOCK_ID>
     </DATA>
   </RECORD>
   <RECORD>
     <OPCODE>OP_SET_GENSTAMP_V2</OPCODE>
     <DATA>
-      <TXID>47</TXID>
+      <TXID>49</TXID>
       <GENSTAMPV2>1007</GENSTAMPV2>
     </DATA>
   </RECORD>
   <RECORD>
     <OPCODE>OP_ADD_BLOCK</OPCODE>
     <DATA>
-      <TXID>48</TXID>
+      <TXID>50</TXID>
       <PATH>/file_concat_1</PATH>
       <BLOCK>
         <BLOCK_ID>1073741831</BLOCK_ID>
@@ -614,21 +667,21 @@
   <RECORD>
     <OPCODE>OP_ALLOCATE_BLOCK_ID</OPCODE>
     <DATA>
-      <TXID>49</TXID>
+      <TXID>51</TXID>
       <BLOCK_ID>1073741832</BLOCK_ID>
     </DATA>
   </RECORD>
   <RECORD>
     <OPCODE>OP_SET_GENSTAMP_V2</OPCODE>
     <DATA>
-      <TXID>50</TXID>
+      <TXID>52</TXID>
       <GENSTAMPV2>1008</GENSTAMPV2>
     </DATA>
   </RECORD>
   <RECORD>
     <OPCODE>OP_ADD_BLOCK</OPCODE>
     <DATA>
-      <TXID>51</TXID>
+      <TXID>53</TXID>
       <PATH>/file_concat_1</PATH>
       <BLOCK>
         <BLOCK_ID>1073741831</BLOCK_ID>
@@ -647,21 +700,21 @@
   <RECORD>
     <OPCODE>OP_ALLOCATE_BLOCK_ID</OPCODE>
     <DATA>
-      <TXID>52</TXID>
+      <TXID>54</TXID>
       <BLOCK_ID>1073741833</BLOCK_ID>
     </DATA>
   </RECORD>
   <RECORD>
     <OPCODE>OP_SET_GENSTAMP_V2</OPCODE>
     <DATA>
-      <TXID>53</TXID>
+      <TXID>55</TXID>
       <GENSTAMPV2>1009</GENSTAMPV2>
     </DATA>
   </RECORD>
   <RECORD>
     <OPCODE>OP_ADD_BLOCK</OPCODE>
     <DATA>
-      <TXID>54</TXID>
+      <TXID>56</TXID>
       <PATH>/file_concat_1</PATH>
       <BLOCK>
         <BLOCK_ID>1073741832</BLOCK_ID>
@@ -680,7 +733,7 @@
   <RECORD>
     <OPCODE>OP_CLOSE</OPCODE>
     <DATA>
-      <TXID>55</TXID>
+      <TXID>57</TXID>
       <LENGTH>0</LENGTH>
       <INODEID>0</INODEID>
       <PATH>/file_concat_1</PATH>
@@ -690,6 +743,7 @@
       <BLOCKSIZE>512</BLOCKSIZE>
       <CLIENT_NAME></CLIENT_NAME>
       <CLIENT_MACHINE></CLIENT_MACHINE>
+      <OVERWRITE>false</OVERWRITE>
       <BLOCK>
         <BLOCK_ID>1073741831</BLOCK_ID>
         <NUM_BYTES>512</NUM_BYTES>
@@ -715,7 +769,7 @@
   <RECORD>
     <OPCODE>OP_CONCAT_DELETE</OPCODE>
     <DATA>
-      <TXID>56</TXID>
+      <TXID>58</TXID>
       <LENGTH>0</LENGTH>
       <TRG>/file_concat_target</TRG>
       <TIMESTAMP>1402899230394</TIMESTAMP>
@@ -730,7 +784,7 @@
   <RECORD>
     <OPCODE>OP_SYMLINK</OPCODE>
     <DATA>
-      <TXID>57</TXID>
+      <TXID>59</TXID>
       <LENGTH>0</LENGTH>
       <INODEID>16392</INODEID>
       <PATH>/file_symlink</PATH>
@@ -749,7 +803,7 @@
   <RECORD>
     <OPCODE>OP_ADD</OPCODE>
     <DATA>
-      <TXID>58</TXID>
+      <TXID>60</TXID>
       <LENGTH>0</LENGTH>
       <INODEID>16393</INODEID>
       <PATH>/hard-lease-recovery-test</PATH>
@@ -759,6 +813,7 @@
       <BLOCKSIZE>512</BLOCKSIZE>
       <CLIENT_NAME>DFSClient_NONMAPREDUCE_1233039831_1</CLIENT_NAME>
       <CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
+      <OVERWRITE>false</OVERWRITE>
       <PERMISSION_STATUS>
         <USERNAME>andrew</USERNAME>
         <GROUPNAME>supergroup</GROUPNAME>
@@ -771,21 +826,21 @@
   <RECORD>
     <OPCODE>OP_ALLOCATE_BLOCK_ID</OPCODE>
     <DATA>
-      <TXID>59</TXID>
+      <TXID>61</TXID>
       <BLOCK_ID>1073741834</BLOCK_ID>
     </DATA>
   </RECORD>
   <RECORD>
     <OPCODE>OP_SET_GENSTAMP_V2</OPCODE>
     <DATA>
-      <TXID>60</TXID>
+      <TXID>62</TXID>
       <GENSTAMPV2>1010</GENSTAMPV2>
     </DATA>
   </RECORD>
   <RECORD>
     <OPCODE>OP_ADD_BLOCK</OPCODE>
     <DATA>
-      <TXID>61</TXID>
+      <TXID>63</TXID>
       <PATH>/hard-lease-recovery-test</PATH>
       <BLOCK>
         <BLOCK_ID>1073741834</BLOCK_ID>
@@ -799,7 +854,7 @@
   <RECORD>
     <OPCODE>OP_UPDATE_BLOCKS</OPCODE>
     <DATA>
-      <TXID>62</TXID>
+      <TXID>64</TXID>
       <PATH>/hard-lease-recovery-test</PATH>
       <BLOCK>
         <BLOCK_ID>1073741834</BLOCK_ID>
@@ -813,14 +868,14 @@
   <RECORD>
     <OPCODE>OP_SET_GENSTAMP_V2</OPCODE>
     <DATA>
-      <TXID>63</TXID>
+      <TXID>65</TXID>
       <GENSTAMPV2>1011</GENSTAMPV2>
     </DATA>
   </RECORD>
   <RECORD>
     <OPCODE>OP_REASSIGN_LEASE</OPCODE>
     <DATA>
-      <TXID>64</TXID>
+      <TXID>66</TXID>
       <LEASEHOLDER>DFSClient_NONMAPREDUCE_1233039831_1</LEASEHOLDER>
       <PATH>/hard-lease-recovery-test</PATH>
       <NEWHOLDER>HDFS_NameNode</NEWHOLDER>
@@ -829,7 +884,7 @@
   <RECORD>
     <OPCODE>OP_CLOSE</OPCODE>
     <DATA>
-      <TXID>65</TXID>
+      <TXID>67</TXID>
       <LENGTH>0</LENGTH>
       <INODEID>0</INODEID>
       <PATH>/hard-lease-recovery-test</PATH>
@@ -839,6 +894,7 @@
       <BLOCKSIZE>512</BLOCKSIZE>
       <CLIENT_NAME></CLIENT_NAME>
       <CLIENT_MACHINE></CLIENT_MACHINE>
+      <OVERWRITE>false</OVERWRITE>
       <BLOCK>
         <BLOCK_ID>1073741834</BLOCK_ID>
         <NUM_BYTES>11</NUM_BYTES>
@@ -854,7 +910,7 @@
   <RECORD>
     <OPCODE>OP_ADD_CACHE_POOL</OPCODE>
     <DATA>
-      <TXID>66</TXID>
+      <TXID>68</TXID>
       <POOLNAME>pool1</POOLNAME>
       <OWNERNAME>andrew</OWNERNAME>
       <GROUPNAME>andrew</GROUPNAME>
@@ -868,7 +924,7 @@
   <RECORD>
     <OPCODE>OP_MODIFY_CACHE_POOL</OPCODE>
     <DATA>
-      <TXID>67</TXID>
+      <TXID>69</TXID>
       <POOLNAME>pool1</POOLNAME>
       <LIMIT>99</LIMIT>
       <RPC_CLIENTID>e03f4a52-3d85-4e05-8942-286185e639bd</RPC_CLIENTID>
@@ -878,7 +934,7 @@
   <RECORD>
     <OPCODE>OP_ADD_CACHE_DIRECTIVE</OPCODE>
     <DATA>
-      <TXID>68</TXID>
+      <TXID>70</TXID>
       <ID>1</ID>
       <PATH>/path</PATH>
       <REPLICATION>1</REPLICATION>
@@ -891,7 +947,7 @@
   <RECORD>
     <OPCODE>OP_MODIFY_CACHE_DIRECTIVE</OPCODE>
     <DATA>
-      <TXID>69</TXID>
+      <TXID>71</TXID>
       <ID>1</ID>
       <REPLICATION>2</REPLICATION>
       <RPC_CLIENTID>e03f4a52-3d85-4e05-8942-286185e639bd</RPC_CLIENTID>
@@ -901,7 +957,7 @@
   <RECORD>
     <OPCODE>OP_REMOVE_CACHE_DIRECTIVE</OPCODE>
     <DATA>
-      <TXID>70</TXID>
+      <TXID>72</TXID>
       <ID>1</ID>
       <RPC_CLIENTID>e03f4a52-3d85-4e05-8942-286185e639bd</RPC_CLIENTID>
       <RPC_CALLID>77</RPC_CALLID>
@@ -910,7 +966,7 @@
   <RECORD>
     <OPCODE>OP_REMOVE_CACHE_POOL</OPCODE>
     <DATA>
-      <TXID>71</TXID>
+      <TXID>73</TXID>
       <POOLNAME>pool1</POOLNAME>
       <RPC_CLIENTID>e03f4a52-3d85-4e05-8942-286185e639bd</RPC_CLIENTID>
       <RPC_CALLID>78</RPC_CALLID>
@@ -919,7 +975,7 @@
   <RECORD>
     <OPCODE>OP_SET_ACL</OPCODE>
     <DATA>
-      <TXID>72</TXID>
+      <TXID>74</TXID>
       <SRC>/file_concat_target</SRC>
       <ENTRY>
         <SCOPE>ACCESS</SCOPE>
@@ -952,7 +1008,7 @@
   <RECORD>
     <OPCODE>OP_SET_XATTR</OPCODE>
     <DATA>
-      <TXID>73</TXID>
+      <TXID>75</TXID>
       <SRC>/file_concat_target</SRC>
       <XATTR>
         <NAMESPACE>USER</NAMESPACE>
@@ -966,7 +1022,7 @@
   <RECORD>
     <OPCODE>OP_SET_XATTR</OPCODE>
     <DATA>
-      <TXID>74</TXID>
+      <TXID>76</TXID>
       <SRC>/file_concat_target</SRC>
       <XATTR>
         <NAMESPACE>USER</NAMESPACE>
@@ -980,7 +1036,7 @@
   <RECORD>
     <OPCODE>OP_REMOVE_XATTR</OPCODE>
     <DATA>
-      <TXID>75</TXID>
+      <TXID>77</TXID>
       <SRC>/file_concat_target</SRC>
       <XATTR>
         <NAMESPACE>USER</NAMESPACE>
@@ -993,21 +1049,21 @@
   <RECORD>
     <OPCODE>OP_ROLLING_UPGRADE_START</OPCODE>
     <DATA>
-      <TXID>76</TXID>
+      <TXID>78</TXID>
       <STARTTIME>1402899233646</STARTTIME>
     </DATA>
   </RECORD>
   <RECORD>
     <OPCODE>OP_ROLLING_UPGRADE_FINALIZE</OPCODE>
     <DATA>
-      <TXID>77</TXID>
+      <TXID>79</TXID>
       <FINALIZETIME>1402899233647</FINALIZETIME>
     </DATA>
   </RECORD>
   <RECORD>
     <OPCODE>OP_END_LOG_SEGMENT</OPCODE>
     <DATA>
-      <TXID>78</TXID>
+      <TXID>80</TXID>
     </DATA>
   </RECORD>
 </EDITS>

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

@@ -267,6 +267,12 @@ Release 2.6.0 - UNRELEASED
     MAPREDUCE-5931. Validate SleepJob command line parameters (Gera Shegalov
     via jlowe)
 
+    MAPREDUCE-6063. Correct spill size calculation for spills wrapping the
+    circular buffer. (zhihai xu via cdouglas)
+
+    MAPREDUCE-6071. JobImpl#makeUberDecision doesn't log that Uber mode is
+    disabled because of too much CPUs (Tsuyoshi OZAWA via jlowe)
+
 Release 2.5.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

+ 0 - 341
hadoop-mapreduce-project/LICENSE.txt

@@ -1,341 +0,0 @@
-
-                                 Apache License
-                           Version 2.0, January 2004
-                        http://www.apache.org/licenses/
-
-   TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
-
-   1. Definitions.
-
-      "License" shall mean the terms and conditions for use, reproduction,
-      and distribution as defined by Sections 1 through 9 of this document.
-
-      "Licensor" shall mean the copyright owner or entity authorized by
-      the copyright owner that is granting the License.
-
-      "Legal Entity" shall mean the union of the acting entity and all
-      other entities that control, are controlled by, or are under common
-      control with that entity. For the purposes of this definition,
-      "control" means (i) the power, direct or indirect, to cause the
-      direction or management of such entity, whether by contract or
-      otherwise, or (ii) ownership of fifty percent (50%) or more of the
-      outstanding shares, or (iii) beneficial ownership of such entity.
-
-      "You" (or "Your") shall mean an individual or Legal Entity
-      exercising permissions granted by this License.
-
-      "Source" form shall mean the preferred form for making modifications,
-      including but not limited to software source code, documentation
-      source, and configuration files.
-
-      "Object" form shall mean any form resulting from mechanical
-      transformation or translation of a Source form, including but
-      not limited to compiled object code, generated documentation,
-      and conversions to other media types.
-
-      "Work" shall mean the work of authorship, whether in Source or
-      Object form, made available under the License, as indicated by a
-      copyright notice that is included in or attached to the work
-      (an example is provided in the Appendix below).
-
-      "Derivative Works" shall mean any work, whether in Source or Object
-      form, that is based on (or derived from) the Work and for which the
-      editorial revisions, annotations, elaborations, or other modifications
-      represent, as a whole, an original work of authorship. For the purposes
-      of this License, Derivative Works shall not include works that remain
-      separable from, or merely link (or bind by name) to the interfaces of,
-      the Work and Derivative Works thereof.
-
-      "Contribution" shall mean any work of authorship, including
-      the original version of the Work and any modifications or additions
-      to that Work or Derivative Works thereof, that is intentionally
-      submitted to Licensor for inclusion in the Work by the copyright owner
-      or by an individual or Legal Entity authorized to submit on behalf of
-      the copyright owner. For the purposes of this definition, "submitted"
-      means any form of electronic, verbal, or written communication sent
-      to the Licensor or its representatives, including but not limited to
-      communication on electronic mailing lists, source code control systems,
-      and issue tracking systems that are managed by, or on behalf of, the
-      Licensor for the purpose of discussing and improving the Work, but
-      excluding communication that is conspicuously marked or otherwise
-      designated in writing by the copyright owner as "Not a Contribution."
-
-      "Contributor" shall mean Licensor and any individual or Legal Entity
-      on behalf of whom a Contribution has been received by Licensor and
-      subsequently incorporated within the Work.
-
-   2. Grant of Copyright License. Subject to the terms and conditions of
-      this License, each Contributor hereby grants to You a perpetual,
-      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
-      copyright license to reproduce, prepare Derivative Works of,
-      publicly display, publicly perform, sublicense, and distribute the
-      Work and such Derivative Works in Source or Object form.
-
-   3. Grant of Patent License. Subject to the terms and conditions of
-      this License, each Contributor hereby grants to You a perpetual,
-      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
-      (except as stated in this section) patent license to make, have made,
-      use, offer to sell, sell, import, and otherwise transfer the Work,
-      where such license applies only to those patent claims licensable
-      by such Contributor that are necessarily infringed by their
-      Contribution(s) alone or by combination of their Contribution(s)
-      with the Work to which such Contribution(s) was submitted. If You
-      institute patent litigation against any entity (including a
-      cross-claim or counterclaim in a lawsuit) alleging that the Work
-      or a Contribution incorporated within the Work constitutes direct
-      or contributory patent infringement, then any patent licenses
-      granted to You under this License for that Work shall terminate
-      as of the date such litigation is filed.
-
-   4. Redistribution. You may reproduce and distribute copies of the
-      Work or Derivative Works thereof in any medium, with or without
-      modifications, and in Source or Object form, provided that You
-      meet the following conditions:
-
-      (a) You must give any other recipients of the Work or
-          Derivative Works a copy of this License; and
-
-      (b) You must cause any modified files to carry prominent notices
-          stating that You changed the files; and
-
-      (c) You must retain, in the Source form of any Derivative Works
-          that You distribute, all copyright, patent, trademark, and
-          attribution notices from the Source form of the Work,
-          excluding those notices that do not pertain to any part of
-          the Derivative Works; and
-
-      (d) If the Work includes a "NOTICE" text file as part of its
-          distribution, then any Derivative Works that You distribute must
-          include a readable copy of the attribution notices contained
-          within such NOTICE file, excluding those notices that do not
-          pertain to any part of the Derivative Works, in at least one
-          of the following places: within a NOTICE text file distributed
-          as part of the Derivative Works; within the Source form or
-          documentation, if provided along with the Derivative Works; or,
-          within a display generated by the Derivative Works, if and
-          wherever such third-party notices normally appear. The contents
-          of the NOTICE file are for informational purposes only and
-          do not modify the License. You may add Your own attribution
-          notices within Derivative Works that You distribute, alongside
-          or as an addendum to the NOTICE text from the Work, provided
-          that such additional attribution notices cannot be construed
-          as modifying the License.
-
-      You may add Your own copyright statement to Your modifications and
-      may provide additional or different license terms and conditions
-      for use, reproduction, or distribution of Your modifications, or
-      for any such Derivative Works as a whole, provided Your use,
-      reproduction, and distribution of the Work otherwise complies with
-      the conditions stated in this License.
-
-   5. Submission of Contributions. Unless You explicitly state otherwise,
-      any Contribution intentionally submitted for inclusion in the Work
-      by You to the Licensor shall be under the terms and conditions of
-      this License, without any additional terms or conditions.
-      Notwithstanding the above, nothing herein shall supersede or modify
-      the terms of any separate license agreement you may have executed
-      with Licensor regarding such Contributions.
-
-   6. Trademarks. This License does not grant permission to use the trade
-      names, trademarks, service marks, or product names of the Licensor,
-      except as required for reasonable and customary use in describing the
-      origin of the Work and reproducing the content of the NOTICE file.
-
-   7. Disclaimer of Warranty. Unless required by applicable law or
-      agreed to in writing, Licensor provides the Work (and each
-      Contributor provides its Contributions) on an "AS IS" BASIS,
-      WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
-      implied, including, without limitation, any warranties or conditions
-      of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
-      PARTICULAR PURPOSE. You are solely responsible for determining the
-      appropriateness of using or redistributing the Work and assume any
-      risks associated with Your exercise of permissions under this License.
-
-   8. Limitation of Liability. In no event and under no legal theory,
-      whether in tort (including negligence), contract, or otherwise,
-      unless required by applicable law (such as deliberate and grossly
-      negligent acts) or agreed to in writing, shall any Contributor be
-      liable to You for damages, including any direct, indirect, special,
-      incidental, or consequential damages of any character arising as a
-      result of this License or out of the use or inability to use the
-      Work (including but not limited to damages for loss of goodwill,
-      work stoppage, computer failure or malfunction, or any and all
-      other commercial damages or losses), even if such Contributor
-      has been advised of the possibility of such damages.
-
-   9. Accepting Warranty or Additional Liability. While redistributing
-      the Work or Derivative Works thereof, You may choose to offer,
-      and charge a fee for, acceptance of support, warranty, indemnity,
-      or other liability obligations and/or rights consistent with this
-      License. However, in accepting such obligations, You may act only
-      on Your own behalf and on Your sole responsibility, not on behalf
-      of any other Contributor, and only if You agree to indemnify,
-      defend, and hold each Contributor harmless for any liability
-      incurred by, or claims asserted against, such Contributor by reason
-      of your accepting any such warranty or additional liability.
-
-   END OF TERMS AND CONDITIONS
-
-   APPENDIX: How to apply the Apache License to your work.
-
-      To apply the Apache License to your work, attach the following
-      boilerplate notice, with the fields enclosed by brackets "[]"
-      replaced with your own identifying information. (Don't include
-      the brackets!)  The text should be enclosed in the appropriate
-      comment syntax for the file format. We also recommend that a
-      file or class name and description of purpose be included on the
-      same "printed page" as the copyright notice for easier
-      identification within third-party archives.
-
-   Copyright [yyyy] [name of copyright owner]
-
-   Licensed under the Apache License, Version 2.0 (the "License");
-   you may not use this file except in compliance with the License.
-   You may obtain a copy of the License at
-
-       http://www.apache.org/licenses/LICENSE-2.0
-
-   Unless required by applicable law or agreed to in writing, software
-   distributed under the License is distributed on an "AS IS" BASIS,
-   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-   See the License for the specific language governing permissions and
-   limitations under the License.
-
-
-APACHE HADOOP SUBCOMPONENTS:
-
-The Apache Hadoop project contains subcomponents with separate copyright
-notices and license terms. Your use of the source code for the these
-subcomponents is subject to the terms and conditions of the following
-licenses. 
-
-For the org.apache.hadoop.util.bloom.* classes:
-
-/**
- *
- * Copyright (c) 2005, European Commission project OneLab under contract
- * 034819 (http://www.one-lab.org)
- * All rights reserved.
- * Redistribution and use in source and binary forms, with or 
- * without modification, are permitted provided that the following 
- * conditions are met:
- *  - Redistributions of source code must retain the above copyright 
- *    notice, this list of conditions and the following disclaimer.
- *  - Redistributions in binary form must reproduce the above copyright 
- *    notice, this list of conditions and the following disclaimer in 
- *    the documentation and/or other materials provided with the distribution.
- *  - Neither the name of the University Catholique de Louvain - UCL
- *    nor the names of its contributors may be used to endorse or 
- *    promote products derived from this software without specific prior 
- *    written permission.
- *    
- * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS 
- * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT 
- * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS 
- * FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE 
- * COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, 
- * INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, 
- * BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; 
- * LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER 
- * CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT 
- * LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN 
- * ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE 
- * POSSIBILITY OF SUCH DAMAGE.
- */
-
-The binary distribution of this product bundles binaries of leveldbjni
-(https://github.com/fusesource/leveldbjni), which is available under the
-following license:
-
-Copyright (c) 2011 FuseSource Corp. All rights reserved.
-
-Redistribution and use in source and binary forms, with or without
-modification, are permitted provided that the following conditions are
-met:
-
-   * Redistributions of source code must retain the above copyright
-notice, this list of conditions and the following disclaimer.
-   * Redistributions in binary form must reproduce the above
-copyright notice, this list of conditions and the following disclaimer
-in the documentation and/or other materials provided with the
-distribution.
-   * Neither the name of FuseSource Corp. nor the names of its
-contributors may be used to endorse or promote products derived from
-this software without specific prior written permission.
-
-THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
-"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
-LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
-A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
-OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
-SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
-LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
-DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
-THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
-(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
-OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
-
-The binary distribution of this product bundles binaries of leveldb
-(http://code.google.com/p/leveldb/), which is available under the following
-license:
-
-Copyright (c) 2011 The LevelDB Authors. All rights reserved.
-
-Redistribution and use in source and binary forms, with or without
-modification, are permitted provided that the following conditions are
-met:
-
-   * Redistributions of source code must retain the above copyright
-notice, this list of conditions and the following disclaimer.
-   * Redistributions in binary form must reproduce the above
-copyright notice, this list of conditions and the following disclaimer
-in the documentation and/or other materials provided with the
-distribution.
-   * Neither the name of Google Inc. nor the names of its
-contributors may be used to endorse or promote products derived from
-this software without specific prior written permission.
-
-THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
-"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
-LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
-A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
-OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
-SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
-LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
-DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
-THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
-(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
-OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
-
-The binary distribution of this product bundles binaries of snappy
-(http://code.google.com/p/snappy/), which is available under the following
-license:
-
-Copyright 2011, Google Inc.
-All rights reserved.
-
-Redistribution and use in source and binary forms, with or without
-modification, are permitted provided that the following conditions are
-met:
-
-    * Redistributions of source code must retain the above copyright
-notice, this list of conditions and the following disclaimer.
-    * Redistributions in binary form must reproduce the above
-copyright notice, this list of conditions and the following disclaimer
-in the documentation and/or other materials provided with the
-distribution.
-    * Neither the name of Google Inc. nor the names of its
-contributors may be used to endorse or promote products derived from
-this software without specific prior written permission.
-
-THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
-"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
-LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
-A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
-OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
-SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
-LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
-DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
-THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
-(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
-OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.

+ 0 - 2
hadoop-mapreduce-project/NOTICE.txt

@@ -1,2 +0,0 @@
-This product includes software developed by The Apache Software
-Foundation (http://www.apache.org/).

+ 2 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/JobImpl.java

@@ -1285,6 +1285,8 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
         msg.append(" too many reduces;");
       if (!smallInput)
         msg.append(" too much input;");
+      if (!smallCpu)
+        msg.append(" too much CPU;");
       if (!smallMemory)
         msg.append(" too much RAM;");
       if (!notChainJob)

+ 1 - 3
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/MapTask.java

@@ -1575,9 +1575,7 @@ public class MapTask extends Task {
                                        InterruptedException {
       //approximate the length of the output file to be the length of the
       //buffer + header lengths for the partitions
-      final long size = (bufend >= bufstart
-          ? bufend - bufstart
-          : (bufvoid - bufend) + bufstart) +
+      final long size = distanceTo(bufstart, bufend, bufvoid) +
                   partitions * APPROX_HEADER_LENGTH;
       FSDataOutputStream out = null;
       try {

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

@@ -61,6 +61,9 @@ Release 2.6.0 - UNRELEASED
     YARN-2395. FairScheduler: Preemption timeout should be configurable per 
     queue. (Wei Yan via kasha)
 
+    YARN-2394. FairScheduler: Configure fairSharePreemptionThreshold per queue.
+    (Wei Yan via kasha)
+
   IMPROVEMENTS
 
     YARN-2197. Add a link to YARN CHANGES.txt in the left side of doc
@@ -172,6 +175,12 @@ Release 2.6.0 - UNRELEASED
     YARN-1506. Changed RMNode/SchedulerNode to update resource with event
     notification. (Junping Du via jianhe)
 
+    YARN-2509. Enable Cross Origin Filter for timeline server only and not all
+    Yarn servers (Mit Desai via jeagles)
+
+    YARN-2511. Allowed all origins by default when CrossOriginFilter is
+    enabled. (Jonathan Eagles via zjshen)
+
   OPTIMIZATIONS
 
   BUG FIXES
@@ -272,6 +281,9 @@ Release 2.6.0 - UNRELEASED
     YARN-2462. TestNodeManagerResync#testBlockNewContainerRequestsOnStartAndResync
     should have a test timeout (Eric Payne via jlowe)
 
+    YARN-2431. NM restart: cgroup is not removed for reacquired containers
+    (jlowe)
+
 Release 2.5.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

+ 0 - 341
hadoop-yarn-project/LICENSE.txt

@@ -1,341 +0,0 @@
-
-                                 Apache License
-                           Version 2.0, January 2004
-                        http://www.apache.org/licenses/
-
-   TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
-
-   1. Definitions.
-
-      "License" shall mean the terms and conditions for use, reproduction,
-      and distribution as defined by Sections 1 through 9 of this document.
-
-      "Licensor" shall mean the copyright owner or entity authorized by
-      the copyright owner that is granting the License.
-
-      "Legal Entity" shall mean the union of the acting entity and all
-      other entities that control, are controlled by, or are under common
-      control with that entity. For the purposes of this definition,
-      "control" means (i) the power, direct or indirect, to cause the
-      direction or management of such entity, whether by contract or
-      otherwise, or (ii) ownership of fifty percent (50%) or more of the
-      outstanding shares, or (iii) beneficial ownership of such entity.
-
-      "You" (or "Your") shall mean an individual or Legal Entity
-      exercising permissions granted by this License.
-
-      "Source" form shall mean the preferred form for making modifications,
-      including but not limited to software source code, documentation
-      source, and configuration files.
-
-      "Object" form shall mean any form resulting from mechanical
-      transformation or translation of a Source form, including but
-      not limited to compiled object code, generated documentation,
-      and conversions to other media types.
-
-      "Work" shall mean the work of authorship, whether in Source or
-      Object form, made available under the License, as indicated by a
-      copyright notice that is included in or attached to the work
-      (an example is provided in the Appendix below).
-
-      "Derivative Works" shall mean any work, whether in Source or Object
-      form, that is based on (or derived from) the Work and for which the
-      editorial revisions, annotations, elaborations, or other modifications
-      represent, as a whole, an original work of authorship. For the purposes
-      of this License, Derivative Works shall not include works that remain
-      separable from, or merely link (or bind by name) to the interfaces of,
-      the Work and Derivative Works thereof.
-
-      "Contribution" shall mean any work of authorship, including
-      the original version of the Work and any modifications or additions
-      to that Work or Derivative Works thereof, that is intentionally
-      submitted to Licensor for inclusion in the Work by the copyright owner
-      or by an individual or Legal Entity authorized to submit on behalf of
-      the copyright owner. For the purposes of this definition, "submitted"
-      means any form of electronic, verbal, or written communication sent
-      to the Licensor or its representatives, including but not limited to
-      communication on electronic mailing lists, source code control systems,
-      and issue tracking systems that are managed by, or on behalf of, the
-      Licensor for the purpose of discussing and improving the Work, but
-      excluding communication that is conspicuously marked or otherwise
-      designated in writing by the copyright owner as "Not a Contribution."
-
-      "Contributor" shall mean Licensor and any individual or Legal Entity
-      on behalf of whom a Contribution has been received by Licensor and
-      subsequently incorporated within the Work.
-
-   2. Grant of Copyright License. Subject to the terms and conditions of
-      this License, each Contributor hereby grants to You a perpetual,
-      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
-      copyright license to reproduce, prepare Derivative Works of,
-      publicly display, publicly perform, sublicense, and distribute the
-      Work and such Derivative Works in Source or Object form.
-
-   3. Grant of Patent License. Subject to the terms and conditions of
-      this License, each Contributor hereby grants to You a perpetual,
-      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
-      (except as stated in this section) patent license to make, have made,
-      use, offer to sell, sell, import, and otherwise transfer the Work,
-      where such license applies only to those patent claims licensable
-      by such Contributor that are necessarily infringed by their
-      Contribution(s) alone or by combination of their Contribution(s)
-      with the Work to which such Contribution(s) was submitted. If You
-      institute patent litigation against any entity (including a
-      cross-claim or counterclaim in a lawsuit) alleging that the Work
-      or a Contribution incorporated within the Work constitutes direct
-      or contributory patent infringement, then any patent licenses
-      granted to You under this License for that Work shall terminate
-      as of the date such litigation is filed.
-
-   4. Redistribution. You may reproduce and distribute copies of the
-      Work or Derivative Works thereof in any medium, with or without
-      modifications, and in Source or Object form, provided that You
-      meet the following conditions:
-
-      (a) You must give any other recipients of the Work or
-          Derivative Works a copy of this License; and
-
-      (b) You must cause any modified files to carry prominent notices
-          stating that You changed the files; and
-
-      (c) You must retain, in the Source form of any Derivative Works
-          that You distribute, all copyright, patent, trademark, and
-          attribution notices from the Source form of the Work,
-          excluding those notices that do not pertain to any part of
-          the Derivative Works; and
-
-      (d) If the Work includes a "NOTICE" text file as part of its
-          distribution, then any Derivative Works that You distribute must
-          include a readable copy of the attribution notices contained
-          within such NOTICE file, excluding those notices that do not
-          pertain to any part of the Derivative Works, in at least one
-          of the following places: within a NOTICE text file distributed
-          as part of the Derivative Works; within the Source form or
-          documentation, if provided along with the Derivative Works; or,
-          within a display generated by the Derivative Works, if and
-          wherever such third-party notices normally appear. The contents
-          of the NOTICE file are for informational purposes only and
-          do not modify the License. You may add Your own attribution
-          notices within Derivative Works that You distribute, alongside
-          or as an addendum to the NOTICE text from the Work, provided
-          that such additional attribution notices cannot be construed
-          as modifying the License.
-
-      You may add Your own copyright statement to Your modifications and
-      may provide additional or different license terms and conditions
-      for use, reproduction, or distribution of Your modifications, or
-      for any such Derivative Works as a whole, provided Your use,
-      reproduction, and distribution of the Work otherwise complies with
-      the conditions stated in this License.
-
-   5. Submission of Contributions. Unless You explicitly state otherwise,
-      any Contribution intentionally submitted for inclusion in the Work
-      by You to the Licensor shall be under the terms and conditions of
-      this License, without any additional terms or conditions.
-      Notwithstanding the above, nothing herein shall supersede or modify
-      the terms of any separate license agreement you may have executed
-      with Licensor regarding such Contributions.
-
-   6. Trademarks. This License does not grant permission to use the trade
-      names, trademarks, service marks, or product names of the Licensor,
-      except as required for reasonable and customary use in describing the
-      origin of the Work and reproducing the content of the NOTICE file.
-
-   7. Disclaimer of Warranty. Unless required by applicable law or
-      agreed to in writing, Licensor provides the Work (and each
-      Contributor provides its Contributions) on an "AS IS" BASIS,
-      WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
-      implied, including, without limitation, any warranties or conditions
-      of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
-      PARTICULAR PURPOSE. You are solely responsible for determining the
-      appropriateness of using or redistributing the Work and assume any
-      risks associated with Your exercise of permissions under this License.
-
-   8. Limitation of Liability. In no event and under no legal theory,
-      whether in tort (including negligence), contract, or otherwise,
-      unless required by applicable law (such as deliberate and grossly
-      negligent acts) or agreed to in writing, shall any Contributor be
-      liable to You for damages, including any direct, indirect, special,
-      incidental, or consequential damages of any character arising as a
-      result of this License or out of the use or inability to use the
-      Work (including but not limited to damages for loss of goodwill,
-      work stoppage, computer failure or malfunction, or any and all
-      other commercial damages or losses), even if such Contributor
-      has been advised of the possibility of such damages.
-
-   9. Accepting Warranty or Additional Liability. While redistributing
-      the Work or Derivative Works thereof, You may choose to offer,
-      and charge a fee for, acceptance of support, warranty, indemnity,
-      or other liability obligations and/or rights consistent with this
-      License. However, in accepting such obligations, You may act only
-      on Your own behalf and on Your sole responsibility, not on behalf
-      of any other Contributor, and only if You agree to indemnify,
-      defend, and hold each Contributor harmless for any liability
-      incurred by, or claims asserted against, such Contributor by reason
-      of your accepting any such warranty or additional liability.
-
-   END OF TERMS AND CONDITIONS
-
-   APPENDIX: How to apply the Apache License to your work.
-
-      To apply the Apache License to your work, attach the following
-      boilerplate notice, with the fields enclosed by brackets "[]"
-      replaced with your own identifying information. (Don't include
-      the brackets!)  The text should be enclosed in the appropriate
-      comment syntax for the file format. We also recommend that a
-      file or class name and description of purpose be included on the
-      same "printed page" as the copyright notice for easier
-      identification within third-party archives.
-
-   Copyright [yyyy] [name of copyright owner]
-
-   Licensed under the Apache License, Version 2.0 (the "License");
-   you may not use this file except in compliance with the License.
-   You may obtain a copy of the License at
-
-       http://www.apache.org/licenses/LICENSE-2.0
-
-   Unless required by applicable law or agreed to in writing, software
-   distributed under the License is distributed on an "AS IS" BASIS,
-   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-   See the License for the specific language governing permissions and
-   limitations under the License.
-
-
-APACHE HADOOP SUBCOMPONENTS:
-
-The Apache Hadoop project contains subcomponents with separate copyright
-notices and license terms. Your use of the source code for the these
-subcomponents is subject to the terms and conditions of the following
-licenses. 
-
-For the org.apache.hadoop.util.bloom.* classes:
-
-/**
- *
- * Copyright (c) 2005, European Commission project OneLab under contract
- * 034819 (http://www.one-lab.org)
- * All rights reserved.
- * Redistribution and use in source and binary forms, with or 
- * without modification, are permitted provided that the following 
- * conditions are met:
- *  - Redistributions of source code must retain the above copyright 
- *    notice, this list of conditions and the following disclaimer.
- *  - Redistributions in binary form must reproduce the above copyright 
- *    notice, this list of conditions and the following disclaimer in 
- *    the documentation and/or other materials provided with the distribution.
- *  - Neither the name of the University Catholique de Louvain - UCL
- *    nor the names of its contributors may be used to endorse or 
- *    promote products derived from this software without specific prior 
- *    written permission.
- *    
- * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS 
- * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT 
- * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS 
- * FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE 
- * COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, 
- * INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, 
- * BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; 
- * LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER 
- * CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT 
- * LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN 
- * ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE 
- * POSSIBILITY OF SUCH DAMAGE.
- */
-
-The binary distribution of this product bundles binaries of leveldbjni
-(https://github.com/fusesource/leveldbjni), which is available under the
-following license:
-
-Copyright (c) 2011 FuseSource Corp. All rights reserved.
-
-Redistribution and use in source and binary forms, with or without
-modification, are permitted provided that the following conditions are
-met:
-
-   * Redistributions of source code must retain the above copyright
-notice, this list of conditions and the following disclaimer.
-   * Redistributions in binary form must reproduce the above
-copyright notice, this list of conditions and the following disclaimer
-in the documentation and/or other materials provided with the
-distribution.
-   * Neither the name of FuseSource Corp. nor the names of its
-contributors may be used to endorse or promote products derived from
-this software without specific prior written permission.
-
-THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
-"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
-LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
-A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
-OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
-SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
-LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
-DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
-THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
-(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
-OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
-
-The binary distribution of this product bundles binaries of leveldb
-(http://code.google.com/p/leveldb/), which is available under the following
-license:
-
-Copyright (c) 2011 The LevelDB Authors. All rights reserved.
-
-Redistribution and use in source and binary forms, with or without
-modification, are permitted provided that the following conditions are
-met:
-
-   * Redistributions of source code must retain the above copyright
-notice, this list of conditions and the following disclaimer.
-   * Redistributions in binary form must reproduce the above
-copyright notice, this list of conditions and the following disclaimer
-in the documentation and/or other materials provided with the
-distribution.
-   * Neither the name of Google Inc. nor the names of its
-contributors may be used to endorse or promote products derived from
-this software without specific prior written permission.
-
-THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
-"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
-LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
-A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
-OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
-SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
-LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
-DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
-THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
-(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
-OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
-
-The binary distribution of this product bundles binaries of snappy
-(http://code.google.com/p/snappy/), which is available under the following
-license:
-
-Copyright 2011, Google Inc.
-All rights reserved.
-
-Redistribution and use in source and binary forms, with or without
-modification, are permitted provided that the following conditions are
-met:
-
-    * Redistributions of source code must retain the above copyright
-notice, this list of conditions and the following disclaimer.
-    * Redistributions in binary form must reproduce the above
-copyright notice, this list of conditions and the following disclaimer
-in the documentation and/or other materials provided with the
-distribution.
-    * Neither the name of Google Inc. nor the names of its
-contributors may be used to endorse or promote products derived from
-this software without specific prior written permission.
-
-THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
-"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
-LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
-A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
-OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
-SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
-LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
-DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
-THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
-(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
-OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.

+ 0 - 17
hadoop-yarn-project/NOTICE.txt

@@ -1,17 +0,0 @@
-This product includes software developed by The Apache Software
-Foundation (http://www.apache.org/).
-
-The binary distribution of this product bundles binaries of
-org.iq80.leveldb:leveldb-api (https://github.com/dain/leveldb), which has the
-following notices:
-* Copyright 2011 Dain Sundstrom <dain@iq80.com>
-* Copyright 2011 FuseSource Corp. http://fusesource.com
-
-The binary distribution of this product bundles binaries of
-org.fusesource.hawtjni:hawtjni-runtime (https://github.com/fusesource/hawtjni),
-which has the following notices:
-* This product includes software developed by FuseSource Corp.
-  http://fusesource.com
-* This product includes software developed at
-  Progress Software Corporation and/or its  subsidiaries or affiliates.
-* This product includes software developed by IBM Corporation and others.

+ 8 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java

@@ -1239,6 +1239,14 @@ public class YarnConfiguration extends Configuration {
   public static final String TIMELINE_SERVICE_KEYTAB =
       TIMELINE_SERVICE_PREFIX + "keytab";
 
+  /** Enables cross origin support for timeline server.*/
+  public static final String TIMELINE_SERVICE_HTTP_CROSS_ORIGIN_ENABLED =
+      TIMELINE_SERVICE_PREFIX + "http-cross-origin.enabled";
+
+  /** Default value for cross origin support for timeline server.*/
+  public static final boolean
+      TIMELINE_SERVICE_HTTP_CROSS_ORIGIN_ENABLED_DEFAULT = false;
+
   ////////////////////////////////
   // Other Configs
   ////////////////////////////////

+ 15 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryServer.java

@@ -46,6 +46,7 @@ import org.apache.hadoop.yarn.server.timeline.TimelineStore;
 import org.apache.hadoop.yarn.server.timeline.security.TimelineACLsManager;
 import org.apache.hadoop.yarn.server.timeline.security.TimelineAuthenticationFilterInitializer;
 import org.apache.hadoop.yarn.server.timeline.security.TimelineDelegationTokenSecretManagerService;
+import org.apache.hadoop.yarn.server.timeline.webapp.CrossOriginFilterInitializer;
 import org.apache.hadoop.yarn.webapp.WebApp;
 import org.apache.hadoop.yarn.webapp.WebApps;
 import org.apache.hadoop.yarn.webapp.util.WebAppUtils;
@@ -197,17 +198,27 @@ public class ApplicationHistoryServer extends CompositeService {
     // the customized filter will be loaded by the timeline server to do Kerberos
     // + DT authentication.
     String initializers = conf.get("hadoop.http.filter.initializers");
-    boolean modifiedInitialiers = false;
+    boolean modifiedInitializers = false;
 
     initializers =
         initializers == null || initializers.length() == 0 ? "" : initializers;
 
+    if (!initializers.contains(CrossOriginFilterInitializer.class.getName())) {
+      if(conf.getBoolean(YarnConfiguration
+          .TIMELINE_SERVICE_HTTP_CROSS_ORIGIN_ENABLED, YarnConfiguration
+              .TIMELINE_SERVICE_HTTP_CROSS_ORIGIN_ENABLED_DEFAULT)) {
+        initializers = CrossOriginFilterInitializer.class.getName() + ","
+            + initializers;
+        modifiedInitializers = true;
+      }
+    }
+
     if (!initializers.contains(TimelineAuthenticationFilterInitializer.class
       .getName())) {
       initializers =
           TimelineAuthenticationFilterInitializer.class.getName() + ","
               + initializers;
-      modifiedInitialiers = true;
+      modifiedInitializers = true;
     }
 
     String[] parts = initializers.split(",");
@@ -216,14 +227,14 @@ public class ApplicationHistoryServer extends CompositeService {
       filterInitializer = filterInitializer.trim();
       if (filterInitializer.equals(AuthenticationFilterInitializer.class
         .getName())) {
-        modifiedInitialiers = true;
+        modifiedInitializers = true;
         continue;
       }
       target.add(filterInitializer);
     }
     String actualInitializers =
         org.apache.commons.lang.StringUtils.join(target, ",");
-    if (modifiedInitialiers) {
+    if (modifiedInitializers) {
       conf.set("hadoop.http.filter.initializers", actualInitializers);
     }
     String bindAddress = WebAppUtils.getWebAppBindURL(conf,

+ 7 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/webapp/CrossOriginFilter.java

@@ -76,6 +76,7 @@ public class CrossOriginFilter implements Filter {
   private List<String> allowedMethods = new ArrayList<String>();
   private List<String> allowedHeaders = new ArrayList<String>();
   private List<String> allowedOrigins = new ArrayList<String>();
+  private boolean allowAllOrigins = true;
   private String maxAge;
 
   @Override
@@ -171,7 +172,9 @@ public class CrossOriginFilter implements Filter {
     }
     allowedOrigins =
         Arrays.asList(allowedOriginsConfig.trim().split("\\s*,\\s*"));
+    allowAllOrigins = allowedOrigins.contains("*");
     LOG.info("Allowed Origins: " + StringUtils.join(allowedOrigins, ','));
+    LOG.info("Allow All Origins: " + allowAllOrigins);
   }
 
   private void initializeMaxAge(FilterConfig filterConfig) {
@@ -199,8 +202,9 @@ public class CrossOriginFilter implements Filter {
     return origin != null;
   }
 
-  private boolean isOriginAllowed(String origin) {
-    return allowedOrigins.contains(origin);
+  @VisibleForTesting
+  boolean isOriginAllowed(String origin) {
+    return allowAllOrigins || allowedOrigins.contains(origin);
   }
 
   private boolean areHeadersAllowed(String accessControlRequestHeaders) {
@@ -213,7 +217,7 @@ public class CrossOriginFilter implements Filter {
 
   private boolean isMethodAllowed(String accessControlRequestMethod) {
     if (accessControlRequestMethod == null) {
-      return false;
+      return true;
     }
     return allowedMethods.contains(accessControlRequestMethod);
   }

+ 15 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/webapp/TestCrossOriginFilter.java

@@ -31,6 +31,7 @@ import javax.servlet.ServletException;
 import javax.servlet.http.HttpServletRequest;
 import javax.servlet.http.HttpServletResponse;
 
+import org.junit.Assert;
 import org.junit.Test;
 
 import static org.mockito.Mockito.when;
@@ -65,6 +66,20 @@ public class TestCrossOriginFilter {
     verify(mockChain).doFilter(mockReq, mockRes);
   }
 
+  @Test
+  public void testAllowAllOrigins() throws ServletException, IOException {
+
+    // Setup the configuration settings of the server
+    Map<String, String> conf = new HashMap<String, String>();
+    conf.put(CrossOriginFilter.ALLOWED_ORIGINS, "*");
+    FilterConfig filterConfig = new FilterConfigTest(conf);
+
+    // Object under test
+    CrossOriginFilter filter = new CrossOriginFilter();
+    filter.init(filterConfig);
+    Assert.assertTrue(filter.isOriginAllowed("example.org"));
+  }
+
   @Test
   public void testDisallowedOrigin() throws ServletException, IOException {
 

+ 11 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/LinuxContainerExecutor.java

@@ -19,6 +19,7 @@
 package org.apache.hadoop.yarn.server.nodemanager;
 
 import com.google.common.base.Optional;
+
 import java.io.File;
 import java.io.IOException;
 import java.net.InetSocketAddress;
@@ -341,6 +342,16 @@ public class LinuxContainerExecutor extends ContainerExecutor {
     return 0;
   }
 
+  @Override
+  public int reacquireContainer(String user, ContainerId containerId)
+      throws IOException {
+    try {
+      return super.reacquireContainer(user, containerId);
+    } finally {
+      resourcesHandler.postExecute(containerId);
+    }
+  }
+
   @Override
   public boolean signalContainer(String user, String pid, Signal signal)
       throws IOException {

+ 60 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestLinuxContainerExecutor.java

@@ -30,6 +30,8 @@ import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.PrintWriter;
 import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Set;
 
 import org.junit.Assert;
 import org.apache.commons.logging.Log;
@@ -42,11 +44,15 @@ import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
+import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor.Signal;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
+import org.apache.hadoop.yarn.server.nodemanager.util.LCEResourcesHandler;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -353,4 +359,58 @@ public class TestLinuxContainerExecutor {
     }
   }
 
+  @Test(timeout=10000)
+  public void testPostExecuteAfterReacquisition() throws Exception {
+    // make up some bogus container ID
+    ApplicationId appId = ApplicationId.newInstance(12345, 67890);
+    ApplicationAttemptId attemptId =
+        ApplicationAttemptId.newInstance(appId, 54321);
+    ContainerId cid = ContainerId.newInstance(attemptId, 9876);
+
+    Configuration conf = new YarnConfiguration();
+    conf.setClass(YarnConfiguration.NM_LINUX_CONTAINER_RESOURCES_HANDLER,
+        TestResourceHandler.class, LCEResourcesHandler.class);
+    LinuxContainerExecutor lce = new LinuxContainerExecutor();
+    lce.setConf(conf);
+    try {
+      lce.init();
+    } catch (IOException e) {
+      // expected if LCE isn't setup right, but not necessary for this test
+    }
+    lce.reacquireContainer("foouser", cid);
+    Assert.assertTrue("postExec not called after reacquisition",
+        TestResourceHandler.postExecContainers.contains(cid));
+  }
+
+  private static class TestResourceHandler implements LCEResourcesHandler {
+    static Set<ContainerId> postExecContainers = new HashSet<ContainerId>();
+
+    @Override
+    public void setConf(Configuration conf) {
+    }
+
+    @Override
+    public Configuration getConf() {
+      return null;
+    }
+
+    @Override
+    public void init(LinuxContainerExecutor lce) throws IOException {
+    }
+
+    @Override
+    public void preExecute(ContainerId containerId, Resource containerResource)
+        throws IOException {
+    }
+
+    @Override
+    public void postExecute(ContainerId containerId) {
+      postExecContainers.add(containerId);
+    }
+
+    @Override
+    public String getResourcesOption(ContainerId containerId) {
+      return null;
+    }
+  }
 }

+ 21 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/AllocationConfiguration.java

@@ -70,6 +70,12 @@ public class AllocationConfiguration {
   // allowed to preempt other jobs' tasks.
   private final Map<String, Long> fairSharePreemptionTimeouts;
 
+  // The fair share preemption threshold for each queue. If a queue waits
+  // fairSharePreemptionTimeout without receiving
+  // fairshare * fairSharePreemptionThreshold resources, it is allowed to
+  // preempt other queues' tasks.
+  private final Map<String, Float> fairSharePreemptionThresholds;
+
   private final Map<String, SchedulingPolicy> schedulingPolicies;
   
   private final SchedulingPolicy defaultSchedulingPolicy;
@@ -92,6 +98,7 @@ public class AllocationConfiguration {
       SchedulingPolicy defaultSchedulingPolicy,
       Map<String, Long> minSharePreemptionTimeouts,
       Map<String, Long> fairSharePreemptionTimeouts,
+      Map<String, Float> fairSharePreemptionThresholds,
       Map<String, Map<QueueACL, AccessControlList>> queueAcls,
       QueuePlacementPolicy placementPolicy,
       Map<FSQueueType, Set<String>> configuredQueues) {
@@ -108,6 +115,7 @@ public class AllocationConfiguration {
     this.schedulingPolicies = schedulingPolicies;
     this.minSharePreemptionTimeouts = minSharePreemptionTimeouts;
     this.fairSharePreemptionTimeouts = fairSharePreemptionTimeouts;
+    this.fairSharePreemptionThresholds = fairSharePreemptionThresholds;
     this.queueAcls = queueAcls;
     this.placementPolicy = placementPolicy;
     this.configuredQueues = configuredQueues;
@@ -126,6 +134,7 @@ public class AllocationConfiguration {
     queueAcls = new HashMap<String, Map<QueueACL, AccessControlList>>();
     minSharePreemptionTimeouts = new HashMap<String, Long>();
     fairSharePreemptionTimeouts = new HashMap<String, Long>();
+    fairSharePreemptionThresholds = new HashMap<String, Float>();
     schedulingPolicies = new HashMap<String, SchedulingPolicy>();
     defaultSchedulingPolicy = SchedulingPolicy.DEFAULT_POLICY;
     configuredQueues = new HashMap<FSQueueType, Set<String>>();
@@ -171,7 +180,18 @@ public class AllocationConfiguration {
     return (fairSharePreemptionTimeout == null) ?
         -1 : fairSharePreemptionTimeout;
   }
-  
+
+  /**
+   * Get a queue's fair share preemption threshold in the allocation file.
+   * Return -1f if not set.
+   */
+  public float getFairSharePreemptionThreshold(String queueName) {
+    Float fairSharePreemptionThreshold =
+        fairSharePreemptionThresholds.get(queueName);
+    return (fairSharePreemptionThreshold == null) ?
+        -1f : fairSharePreemptionThreshold;
+  }
+
   public ResourceWeights getQueueWeight(String queue) {
     ResourceWeights weight = queueWeights.get(queue);
     return (weight == null) ? ResourceWeights.NEUTRAL : weight;

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

@@ -218,6 +218,8 @@ public class AllocationFileLoaderService extends AbstractService {
     Map<String, SchedulingPolicy> queuePolicies = new HashMap<String, SchedulingPolicy>();
     Map<String, Long> minSharePreemptionTimeouts = new HashMap<String, Long>();
     Map<String, Long> fairSharePreemptionTimeouts = new HashMap<String, Long>();
+    Map<String, Float> fairSharePreemptionThresholds =
+        new HashMap<String, Float>();
     Map<String, Map<QueueACL, AccessControlList>> queueAcls =
         new HashMap<String, Map<QueueACL, AccessControlList>>();
     int userMaxAppsDefault = Integer.MAX_VALUE;
@@ -225,6 +227,7 @@ public class AllocationFileLoaderService extends AbstractService {
     float queueMaxAMShareDefault = -1.0f;
     long defaultFairSharePreemptionTimeout = Long.MAX_VALUE;
     long defaultMinSharePreemptionTimeout = Long.MAX_VALUE;
+    float defaultFairSharePreemptionThreshold = 0.5f;
     SchedulingPolicy defaultSchedPolicy = SchedulingPolicy.DEFAULT_POLICY;
 
     QueuePlacementPolicy newPlacementPolicy = null;
@@ -277,7 +280,8 @@ public class AllocationFileLoaderService extends AbstractService {
           String text = ((Text)element.getFirstChild()).getData().trim();
           int val = Integer.parseInt(text);
           userMaxAppsDefault = val;
-        } else if ("defaultFairSharePreemptionTimeout".equals(element.getTagName())) {
+        } else if ("defaultFairSharePreemptionTimeout"
+            .equals(element.getTagName())) {
           String text = ((Text)element.getFirstChild()).getData().trim();
           long val = Long.parseLong(text) * 1000L;
           defaultFairSharePreemptionTimeout = val;
@@ -287,10 +291,17 @@ public class AllocationFileLoaderService extends AbstractService {
             long val = Long.parseLong(text) * 1000L;
             defaultFairSharePreemptionTimeout = val;
           }
-        } else if ("defaultMinSharePreemptionTimeout".equals(element.getTagName())) {
+        } else if ("defaultMinSharePreemptionTimeout"
+            .equals(element.getTagName())) {
           String text = ((Text)element.getFirstChild()).getData().trim();
           long val = Long.parseLong(text) * 1000L;
           defaultMinSharePreemptionTimeout = val;
+        } else if ("defaultFairSharePreemptionThreshold"
+            .equals(element.getTagName())) {
+          String text = ((Text)element.getFirstChild()).getData().trim();
+          float val = Float.parseFloat(text);
+          val = Math.max(Math.min(val, 1.0f), 0.0f);
+          defaultFairSharePreemptionThreshold = val;
         } else if ("queueMaxAppsDefault".equals(element.getTagName())) {
           String text = ((Text)element.getFirstChild()).getData().trim();
           int val = Integer.parseInt(text);
@@ -326,7 +337,7 @@ public class AllocationFileLoaderService extends AbstractService {
       loadQueue(parent, element, minQueueResources, maxQueueResources,
           queueMaxApps, userMaxApps, queueMaxAMShares, queueWeights,
           queuePolicies, minSharePreemptionTimeouts, fairSharePreemptionTimeouts,
-          queueAcls, configuredQueues);
+          fairSharePreemptionThresholds, queueAcls, configuredQueues);
     }
 
     // Load placement policy and pass it configured queues
@@ -349,11 +360,18 @@ public class AllocationFileLoaderService extends AbstractService {
           defaultFairSharePreemptionTimeout);
     }
 
+    // Set the fair share preemption threshold for the root queue
+    if (!fairSharePreemptionThresholds.containsKey(QueueManager.ROOT_QUEUE)) {
+      fairSharePreemptionThresholds.put(QueueManager.ROOT_QUEUE,
+          defaultFairSharePreemptionThreshold);
+    }
+
     AllocationConfiguration info = new AllocationConfiguration(minQueueResources,
         maxQueueResources, queueMaxApps, userMaxApps, queueWeights,
         queueMaxAMShares, userMaxAppsDefault, queueMaxAppsDefault,
         queueMaxAMShareDefault, queuePolicies, defaultSchedPolicy,
-        minSharePreemptionTimeouts, fairSharePreemptionTimeouts, queueAcls,
+        minSharePreemptionTimeouts, fairSharePreemptionTimeouts,
+        fairSharePreemptionThresholds, queueAcls,
         newPlacementPolicy, configuredQueues);
     
     lastSuccessfulReload = clock.getTime();
@@ -365,13 +383,15 @@ public class AllocationFileLoaderService extends AbstractService {
   /**
    * Loads a queue from a queue element in the configuration file
    */
-  private void loadQueue(String parentName, Element element, Map<String, Resource> minQueueResources,
+  private void loadQueue(String parentName, Element element,
+      Map<String, Resource> minQueueResources,
       Map<String, Resource> maxQueueResources, Map<String, Integer> queueMaxApps,
       Map<String, Integer> userMaxApps, Map<String, Float> queueMaxAMShares,
       Map<String, ResourceWeights> queueWeights,
       Map<String, SchedulingPolicy> queuePolicies,
       Map<String, Long> minSharePreemptionTimeouts,
       Map<String, Long> fairSharePreemptionTimeouts,
+      Map<String, Float> fairSharePreemptionThresholds,
       Map<String, Map<QueueACL, AccessControlList>> queueAcls, 
       Map<FSQueueType, Set<String>> configuredQueues) 
       throws AllocationConfigurationException {
@@ -418,6 +438,11 @@ public class AllocationFileLoaderService extends AbstractService {
         String text = ((Text)field.getFirstChild()).getData().trim();
         long val = Long.parseLong(text) * 1000L;
         fairSharePreemptionTimeouts.put(queueName, val);
+      } else if ("fairSharePreemptionThreshold".equals(field.getTagName())) {
+        String text = ((Text)field.getFirstChild()).getData().trim();
+        float val = Float.parseFloat(text);
+        val = Math.max(Math.min(val, 1.0f), 0.0f);
+        fairSharePreemptionThresholds.put(queueName, val);
       } else if ("schedulingPolicy".equals(field.getTagName())
           || "schedulingMode".equals(field.getTagName())) {
         String text = ((Text)field.getFirstChild()).getData().trim();
@@ -434,7 +459,8 @@ public class AllocationFileLoaderService extends AbstractService {
         loadQueue(queueName, field, minQueueResources, maxQueueResources,
             queueMaxApps, userMaxApps, queueMaxAMShares, queueWeights,
             queuePolicies, minSharePreemptionTimeouts,
-            fairSharePreemptionTimeouts, queueAcls, configuredQueues);
+            fairSharePreemptionTimeouts, fairSharePreemptionThresholds,
+            queueAcls, configuredQueues);
         configuredQueues.get(FSQueueType.PARENT).add(queueName);
         isLeaf = false;
       }
@@ -449,11 +475,15 @@ public class AllocationFileLoaderService extends AbstractService {
       }
     }
     queueAcls.put(queueName, acls);
-    if (maxQueueResources.containsKey(queueName) && minQueueResources.containsKey(queueName)
+    if (maxQueueResources.containsKey(queueName) &&
+        minQueueResources.containsKey(queueName)
         && !Resources.fitsIn(minQueueResources.get(queueName),
             maxQueueResources.get(queueName))) {
-      LOG.warn(String.format("Queue %s has max resources %s less than min resources %s",
-          queueName, maxQueueResources.get(queueName), minQueueResources.get(queueName)));
+      LOG.warn(
+          String.format(
+              "Queue %s has max resources %s less than min resources %s",
+          queueName, maxQueueResources.get(queueName),
+              minQueueResources.get(queueName)));
     }
   }
   

+ 47 - 7
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSLeafQueue.java

@@ -24,6 +24,7 @@ import java.util.Collections;
 import java.util.Comparator;
 import java.util.List;
 
+import com.google.common.annotations.VisibleForTesting;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
@@ -54,7 +55,7 @@ public class FSLeafQueue extends FSQueue {
   
   // Variables used for preemption
   private long lastTimeAtMinShare;
-  private long lastTimeAtHalfFairShare;
+  private long lastTimeAtFairShareThreshold;
   
   // Track the AM resource usage for this queue
   private Resource amResourceUsage;
@@ -65,7 +66,7 @@ public class FSLeafQueue extends FSQueue {
       FSParentQueue parent) {
     super(name, scheduler, parent);
     this.lastTimeAtMinShare = scheduler.getClock().getTime();
-    this.lastTimeAtHalfFairShare = scheduler.getClock().getTime();
+    this.lastTimeAtFairShareThreshold = scheduler.getClock().getTime();
     activeUsersManager = new ActiveUsersManager(getMetrics());
     amResourceUsage = Resource.newInstance(0, 0);
   }
@@ -275,16 +276,17 @@ public class FSLeafQueue extends FSQueue {
     return lastTimeAtMinShare;
   }
 
-  public void setLastTimeAtMinShare(long lastTimeAtMinShare) {
+  private void setLastTimeAtMinShare(long lastTimeAtMinShare) {
     this.lastTimeAtMinShare = lastTimeAtMinShare;
   }
 
-  public long getLastTimeAtHalfFairShare() {
-    return lastTimeAtHalfFairShare;
+  public long getLastTimeAtFairShareThreshold() {
+    return lastTimeAtFairShareThreshold;
   }
 
-  public void setLastTimeAtHalfFairShare(long lastTimeAtHalfFairShare) {
-    this.lastTimeAtHalfFairShare = lastTimeAtHalfFairShare;
+  private void setLastTimeAtFairShareThreshold(
+      long lastTimeAtFairShareThreshold) {
+    this.lastTimeAtFairShareThreshold = lastTimeAtFairShareThreshold;
   }
 
   @Override
@@ -328,6 +330,20 @@ public class FSLeafQueue extends FSQueue {
     // TODO Auto-generated method stub
   }
 
+  /**
+   * Update the preemption fields for the queue, i.e. the times since last was
+   * at its guaranteed share and over its fair share threshold.
+   */
+  public void updateStarvationStats() {
+    long now = scheduler.getClock().getTime();
+    if (!isStarvedForMinShare()) {
+      setLastTimeAtMinShare(now);
+    }
+    if (!isStarvedForFairShare()) {
+      setLastTimeAtFairShareThreshold(now);
+    }
+  }
+
   /**
    * Helper method to check if the queue should preempt containers
    *
@@ -337,4 +353,28 @@ public class FSLeafQueue extends FSQueue {
     return parent.getPolicy().checkIfUsageOverFairShare(getResourceUsage(),
         getFairShare());
   }
+
+  /**
+   * Is a queue being starved for its min share.
+   */
+  @VisibleForTesting
+  boolean isStarvedForMinShare() {
+    return isStarved(getMinShare());
+  }
+
+  /**
+   * Is a queue being starved for its fair share threshold.
+   */
+  @VisibleForTesting
+  boolean isStarvedForFairShare() {
+    return isStarved(
+        Resources.multiply(getFairShare(), getFairSharePreemptionThreshold()));
+  }
+
+  private boolean isStarved(Resource share) {
+    Resource desiredShare = Resources.min(FairScheduler.getResourceCalculator(),
+        scheduler.getClusterResource(), share, getDemand());
+    return Resources.lessThan(FairScheduler.getResourceCalculator(),
+        scheduler.getClusterResource(), getResourceUsage(), desiredShare);
+  }
 }

+ 3 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSParentQueue.java

@@ -78,11 +78,11 @@ public class FSParentQueue extends FSQueue {
   }
 
   @Override
-  public void updatePreemptionTimeouts() {
-    super.updatePreemptionTimeouts();
+  public void updatePreemptionVariables() {
+    super.updatePreemptionVariables();
     // For child queues
     for (FSQueue childQueue : childQueues) {
-      childQueue.updatePreemptionTimeouts();
+      childQueue.updatePreemptionVariables();
     }
   }
 

+ 19 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSQueue.java

@@ -54,6 +54,7 @@ public abstract class FSQueue implements Queue, Schedulable {
 
   private long fairSharePreemptionTimeout = Long.MAX_VALUE;
   private long minSharePreemptionTimeout = Long.MAX_VALUE;
+  private float fairSharePreemptionThreshold = 0.5f;
 
   public FSQueue(String name, FairScheduler scheduler, FSParentQueue parent) {
     this.name = name;
@@ -186,6 +187,14 @@ public abstract class FSQueue implements Queue, Schedulable {
     this.minSharePreemptionTimeout = minSharePreemptionTimeout;
   }
 
+  public float getFairSharePreemptionThreshold() {
+    return fairSharePreemptionThreshold;
+  }
+
+  public void setFairSharePreemptionThreshold(float fairSharePreemptionThreshold) {
+    this.fairSharePreemptionThreshold = fairSharePreemptionThreshold;
+  }
+
   /**
    * Recomputes the shares for all child queues and applications based on this
    * queue's current share
@@ -193,21 +202,27 @@ public abstract class FSQueue implements Queue, Schedulable {
   public abstract void recomputeShares();
 
   /**
-   * Update the min/fair share preemption timeouts for this queue.
+   * Update the min/fair share preemption timeouts and threshold for this queue.
    */
-  public void updatePreemptionTimeouts() {
-    // For min share
+  public void updatePreemptionVariables() {
+    // For min share timeout
     minSharePreemptionTimeout = scheduler.getAllocationConfiguration()
         .getMinSharePreemptionTimeout(getName());
     if (minSharePreemptionTimeout == -1 && parent != null) {
       minSharePreemptionTimeout = parent.getMinSharePreemptionTimeout();
     }
-    // For fair share
+    // For fair share timeout
     fairSharePreemptionTimeout = scheduler.getAllocationConfiguration()
         .getFairSharePreemptionTimeout(getName());
     if (fairSharePreemptionTimeout == -1 && parent != null) {
       fairSharePreemptionTimeout = parent.getFairSharePreemptionTimeout();
     }
+    // For fair share preemption threshold
+    fairSharePreemptionThreshold = scheduler.getAllocationConfiguration()
+        .getFairSharePreemptionThreshold(getName());
+    if (fairSharePreemptionThreshold < 0 && parent != null) {
+      fairSharePreemptionThreshold = parent.getFairSharePreemptionThreshold();
+    }
   }
 
   /**

+ 18 - 42
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java

@@ -299,7 +299,7 @@ public class FairScheduler extends
    */
   protected synchronized void update() {
     long start = getClock().getTime();
-    updatePreemptionVariables(); // Determine if any queues merit preemption
+    updateStarvationStats(); // Determine if any queues merit preemption
 
     FSQueue rootQueue = queueMgr.getRootQueue();
 
@@ -329,48 +329,20 @@ public class FairScheduler extends
 
   /**
    * Update the preemption fields for all QueueScheduables, i.e. the times since
-   * each queue last was at its guaranteed share and at > 1/2 of its fair share
-   * for each type of task.
+   * each queue last was at its guaranteed share and over its fair share
+   * threshold for each type of task.
    */
-  private void updatePreemptionVariables() {
-    long now = getClock().getTime();
-    lastPreemptionUpdateTime = now;
+  private void updateStarvationStats() {
+    lastPreemptionUpdateTime = clock.getTime();
     for (FSLeafQueue sched : queueMgr.getLeafQueues()) {
-      if (!isStarvedForMinShare(sched)) {
-        sched.setLastTimeAtMinShare(now);
-      }
-      if (!isStarvedForFairShare(sched)) {
-        sched.setLastTimeAtHalfFairShare(now);
-      }
+      sched.updateStarvationStats();
     }
   }
 
-  /**
-   * Is a queue below its min share for the given task type?
-   */
-  boolean isStarvedForMinShare(FSLeafQueue sched) {
-    Resource desiredShare = Resources.min(RESOURCE_CALCULATOR, clusterResource,
-      sched.getMinShare(), sched.getDemand());
-    return Resources.lessThan(RESOURCE_CALCULATOR, clusterResource,
-        sched.getResourceUsage(), desiredShare);
-  }
-
-  /**
-   * Is a queue being starved for fair share for the given task type? This is
-   * defined as being below half its fair share.
-   */
-  boolean isStarvedForFairShare(FSLeafQueue sched) {
-    Resource desiredFairShare = Resources.min(RESOURCE_CALCULATOR,
-        clusterResource,
-        Resources.multiply(sched.getFairShare(), .5), sched.getDemand());
-    return Resources.lessThan(RESOURCE_CALCULATOR, clusterResource,
-        sched.getResourceUsage(), desiredFairShare);
-  }
-
   /**
    * Check for queues that need tasks preempted, either because they have been
    * below their guaranteed share for minSharePreemptionTimeout or they have
-   * been below half their fair share for the fairSharePreemptionTimeout. If
+   * been below their fair share threshold for the fairSharePreemptionTimeout. If
    * such queues exist, compute how many tasks of each type need to be preempted
    * and then select the right ones using preemptTasks.
    */
@@ -499,11 +471,11 @@ public class FairScheduler extends
    * Return the resource amount that this queue is allowed to preempt, if any.
    * If the queue has been below its min share for at least its preemption
    * timeout, it should preempt the difference between its current share and
-   * this min share. If it has been below half its fair share for at least the
-   * fairSharePreemptionTimeout, it should preempt enough tasks to get up to its
-   * full fair share. If both conditions hold, we preempt the max of the two
-   * amounts (this shouldn't happen unless someone sets the timeouts to be
-   * identical for some reason).
+   * this min share. If it has been below its fair share preemption threshold
+   * for at least the fairSharePreemptionTimeout, it should preempt enough tasks
+   * to get up to its full fair share. If both conditions hold, we preempt the
+   * max of the two amounts (this shouldn't happen unless someone sets the
+   * timeouts to be identical for some reason).
    */
   protected Resource resToPreempt(FSLeafQueue sched, long curTime) {
     long minShareTimeout = sched.getMinSharePreemptionTimeout();
@@ -516,7 +488,7 @@ public class FairScheduler extends
       resDueToMinShare = Resources.max(RESOURCE_CALCULATOR, clusterResource,
           Resources.none(), Resources.subtract(target, sched.getResourceUsage()));
     }
-    if (curTime - sched.getLastTimeAtHalfFairShare() > fairShareTimeout) {
+    if (curTime - sched.getLastTimeAtFairShareThreshold() > fairShareTimeout) {
       Resource target = Resources.min(RESOURCE_CALCULATOR, clusterResource,
           sched.getFairShare(), sched.getDemand());
       resDueToFairShare = Resources.max(RESOURCE_CALCULATOR, clusterResource,
@@ -1094,7 +1066,11 @@ public class FairScheduler extends
   public FSAppAttempt getSchedulerApp(ApplicationAttemptId appAttemptId) {
     return super.getApplicationAttempt(appAttemptId);
   }
-  
+
+  public static ResourceCalculator getResourceCalculator() {
+    return RESOURCE_CALCULATOR;
+  }
+
   /**
    * Subqueue metrics might be a little out of date because fair shares are
    * recalculated at the update interval, but the root queue metrics needs to

+ 5 - 27
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/QueueManager.java

@@ -181,7 +181,7 @@ public class QueueManager {
         parent.addChildQueue(leafQueue);
         queues.put(leafQueue.getName(), leafQueue);
         leafQueues.add(leafQueue);
-        setPreemptionTimeout(leafQueue, parent, queueConf);
+        leafQueue.updatePreemptionVariables();
         return leafQueue;
       } else {
         FSParentQueue newParent = new FSParentQueue(queueName, scheduler, parent);
@@ -193,7 +193,7 @@ public class QueueManager {
         }
         parent.addChildQueue(newParent);
         queues.put(newParent.getName(), newParent);
-        setPreemptionTimeout(newParent, parent, queueConf);
+        newParent.updatePreemptionVariables();
         parent = newParent;
       }
     }
@@ -201,29 +201,6 @@ public class QueueManager {
     return parent;
   }
 
-  /**
-   * Set the min/fair share preemption timeouts for the given queue.
-   * If the timeout is configured in the allocation file, the queue will use
-   * that value; otherwise, the queue inherits the value from its parent queue.
-   */
-  private void setPreemptionTimeout(FSQueue queue,
-      FSParentQueue parentQueue, AllocationConfiguration queueConf) {
-    // For min share
-    long minSharePreemptionTimeout =
-        queueConf.getMinSharePreemptionTimeout(queue.getQueueName());
-    if (minSharePreemptionTimeout == -1) {
-      minSharePreemptionTimeout = parentQueue.getMinSharePreemptionTimeout();
-    }
-    queue.setMinSharePreemptionTimeout(minSharePreemptionTimeout);
-    // For fair share
-    long fairSharePreemptionTimeout =
-        queueConf.getFairSharePreemptionTimeout(queue.getQueueName());
-    if (fairSharePreemptionTimeout == -1) {
-      fairSharePreemptionTimeout = parentQueue.getFairSharePreemptionTimeout();
-    }
-    queue.setFairSharePreemptionTimeout(fairSharePreemptionTimeout);
-  }
-
   /**
    * Make way for the given queue if possible, by removing incompatible
    * queues with no apps in them. Incompatibility could be due to
@@ -409,7 +386,8 @@ public class QueueManager {
 
     // Update steady fair shares for all queues
     rootQueue.recomputeSteadyShares();
-    // Update the fair share preemption timeouts for all queues recursively
-    rootQueue.updatePreemptionTimeouts();
+    // Update the fair share preemption timeouts and preemption for all queues
+    // recursively
+    rootQueue.updatePreemptionVariables();
   }
 }

+ 43 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestAllocationFileLoaderService.java

@@ -187,13 +187,15 @@ public class TestAllocationFileLoaderService {
     out.println("<queue name=\"queueF\" type=\"parent\" >");
     out.println("</queue>");
     // Create hierarchical queues G,H, with different min/fair share preemption
-    // timeouts
+    // timeouts and preemption thresholds
     out.println("<queue name=\"queueG\">");
     out.println("<fairSharePreemptionTimeout>120</fairSharePreemptionTimeout>");
     out.println("<minSharePreemptionTimeout>50</minSharePreemptionTimeout>");
+    out.println("<fairSharePreemptionThreshold>0.6</fairSharePreemptionThreshold>");
     out.println("   <queue name=\"queueH\">");
     out.println("   <fairSharePreemptionTimeout>180</fairSharePreemptionTimeout>");
     out.println("   <minSharePreemptionTimeout>40</minSharePreemptionTimeout>");
+    out.println("   <fairSharePreemptionThreshold>0.7</fairSharePreemptionThreshold>");
     out.println("   </queue>");
     out.println("</queue>");
     // Set default limit of apps per queue to 15
@@ -211,6 +213,8 @@ public class TestAllocationFileLoaderService {
         + "</defaultMinSharePreemptionTimeout>");
     // Set default fair share preemption timeout to 5 minutes
     out.println("<defaultFairSharePreemptionTimeout>300</defaultFairSharePreemptionTimeout>");
+    // Set default fair share preemption threshold to 0.4
+    out.println("<defaultFairSharePreemptionThreshold>0.4</defaultFairSharePreemptionThreshold>");
     // Set default scheduling policy to DRF
     out.println("<defaultQueueSchedulingPolicy>drf</defaultQueueSchedulingPolicy>");
     out.println("</allocations>");
@@ -299,6 +303,26 @@ public class TestAllocationFileLoaderService {
     assertEquals(120000, queueConf.getFairSharePreemptionTimeout("root.queueG"));
     assertEquals(180000, queueConf.getFairSharePreemptionTimeout("root.queueG.queueH"));
 
+    assertEquals(.4f, queueConf.getFairSharePreemptionThreshold("root"), 0.01);
+    assertEquals(-1, queueConf.getFairSharePreemptionThreshold("root." +
+        YarnConfiguration.DEFAULT_QUEUE_NAME), 0.01);
+    assertEquals(-1,
+        queueConf.getFairSharePreemptionThreshold("root.queueA"), 0.01);
+    assertEquals(-1,
+        queueConf.getFairSharePreemptionThreshold("root.queueB"), 0.01);
+    assertEquals(-1,
+        queueConf.getFairSharePreemptionThreshold("root.queueC"), 0.01);
+    assertEquals(-1,
+        queueConf.getFairSharePreemptionThreshold("root.queueD"), 0.01);
+    assertEquals(-1,
+        queueConf.getFairSharePreemptionThreshold("root.queueE"), 0.01);
+    assertEquals(-1,
+        queueConf.getFairSharePreemptionThreshold("root.queueF"), 0.01);
+    assertEquals(.6f,
+        queueConf.getFairSharePreemptionThreshold("root.queueG"), 0.01);
+    assertEquals(.7f,
+        queueConf.getFairSharePreemptionThreshold("root.queueG.queueH"), 0.01);
+
     assertTrue(queueConf.getConfiguredQueues()
         .get(FSQueueType.PARENT)
         .contains("root.queueF"));
@@ -346,9 +370,10 @@ public class TestAllocationFileLoaderService {
     out.println("<pool name=\"queueD\">");
     out.println("<maxRunningApps>3</maxRunningApps>");
     out.println("</pool>");
-    // Give queue E a preemption timeout of one minute
+    // Give queue E a preemption timeout of one minute and 0.3f threshold
     out.println("<pool name=\"queueE\">");
     out.println("<minSharePreemptionTimeout>60</minSharePreemptionTimeout>");
+    out.println("<fairSharePreemptionThreshold>0.3</fairSharePreemptionThreshold>");
     out.println("</pool>");
     // Set default limit of apps per queue to 15
     out.println("<queueMaxAppsDefault>15</queueMaxAppsDefault>");
@@ -363,6 +388,8 @@ public class TestAllocationFileLoaderService {
         + "</defaultMinSharePreemptionTimeout>");
     // Set fair share preemption timeout to 5 minutes
     out.println("<fairSharePreemptionTimeout>300</fairSharePreemptionTimeout>");
+    // Set default fair share preemption threshold to 0.6f
+    out.println("<defaultFairSharePreemptionThreshold>0.6</defaultFairSharePreemptionThreshold>");
     out.println("</allocations>");
     out.close();
     
@@ -429,6 +456,20 @@ public class TestAllocationFileLoaderService {
     assertEquals(-1, queueConf.getFairSharePreemptionTimeout("root.queueC"));
     assertEquals(-1, queueConf.getFairSharePreemptionTimeout("root.queueD"));
     assertEquals(-1, queueConf.getFairSharePreemptionTimeout("root.queueE"));
+
+    assertEquals(.6f, queueConf.getFairSharePreemptionThreshold("root"), 0.01);
+    assertEquals(-1, queueConf.getFairSharePreemptionThreshold("root."
+        + YarnConfiguration.DEFAULT_QUEUE_NAME), 0.01);
+    assertEquals(-1,
+        queueConf.getFairSharePreemptionThreshold("root.queueA"), 0.01);
+    assertEquals(-1,
+        queueConf.getFairSharePreemptionThreshold("root.queueB"), 0.01);
+    assertEquals(-1,
+        queueConf.getFairSharePreemptionThreshold("root.queueC"), 0.01);
+    assertEquals(-1,
+        queueConf.getFairSharePreemptionThreshold("root.queueD"), 0.01);
+    assertEquals(.3f,
+        queueConf.getFairSharePreemptionThreshold("root.queueE"), 0.01);
   }
   
   @Test

+ 170 - 28
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFSLeafQueue.java

@@ -18,50 +18,66 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
+import java.io.File;
+import java.io.FileWriter;
 import java.io.IOException;
+import java.io.PrintWriter;
+import java.util.Collection;
 
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.apache.hadoop.yarn.event.AsyncDispatcher;
-import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
+import org.apache.hadoop.yarn.server.resourcemanager.MockNodes;
+import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
+import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeAddedSchedulerEvent;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeUpdateSchedulerEvent;
 import org.apache.hadoop.yarn.util.resource.Resources;
+import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
 import org.mockito.Mockito;
 
-public class TestFSLeafQueue {
-  private FSLeafQueue schedulable = null;
-  private Resource maxResource = Resources.createResource(10);
+public class TestFSLeafQueue extends FairSchedulerTestBase {
+  private final static String ALLOC_FILE = new File(TEST_DIR,
+      TestFSLeafQueue.class.getName() + ".xml").getAbsolutePath();
+  private Resource maxResource = Resources.createResource(1024 * 8);
 
   @Before
   public void setup() throws IOException {
-    FairScheduler scheduler = new FairScheduler();
-    Configuration conf = createConfiguration();
-    // All tests assume only one assignment per node update
-    conf.set(FairSchedulerConfiguration.ASSIGN_MULTIPLE, "false");
-    ResourceManager resourceManager = new ResourceManager();
-    resourceManager.init(conf);
-    ((AsyncDispatcher)resourceManager.getRMContext().getDispatcher()).start();
-    scheduler.init(conf);
-    scheduler.start();
-    scheduler.reinitialize(conf, resourceManager.getRMContext());
-    
-    String queueName = "root.queue1";
-    scheduler.allocConf = mock(AllocationConfiguration.class);
-    when(scheduler.allocConf.getMaxResources(queueName)).thenReturn(maxResource);
-    when(scheduler.allocConf.getMinResources(queueName)).thenReturn(Resources.none());
+    conf = createConfiguration();
+    conf.setClass(YarnConfiguration.RM_SCHEDULER, FairScheduler.class,
+        ResourceScheduler.class);
+  }
 
-    schedulable = new FSLeafQueue(queueName, scheduler, null);
+  @After
+  public void teardown() {
+    if (resourceManager != null) {
+      resourceManager.stop();
+      resourceManager = null;
+    }
+    conf = null;
   }
 
   @Test
   public void testUpdateDemand() {
+    conf.set(FairSchedulerConfiguration.ASSIGN_MULTIPLE, "false");
+    resourceManager = new MockRM(conf);
+    resourceManager.start();
+    scheduler = (FairScheduler) resourceManager.getResourceScheduler();
+    scheduler.allocConf = mock(AllocationConfiguration.class);
+
+    String queueName = "root.queue1";
+    when(scheduler.allocConf.getMaxResources(queueName)).thenReturn(maxResource);
+    when(scheduler.allocConf.getMinResources(queueName)).thenReturn(Resources.none());
+    FSLeafQueue schedulable = new FSLeafQueue(queueName, scheduler, null);
+
     FSAppAttempt app = mock(FSAppAttempt.class);
     Mockito.when(app.getDemand()).thenReturn(maxResource);
 
@@ -73,11 +89,137 @@ public class TestFSLeafQueue {
     assertTrue("Demand is greater than max allowed ",
         Resources.equals(schedulable.getDemand(), maxResource));
   }
-  
-  private Configuration createConfiguration() {
-    Configuration conf = new YarnConfiguration();
-    conf.setClass(YarnConfiguration.RM_SCHEDULER, FairScheduler.class,
-        ResourceScheduler.class);
-    return conf;
+
+  @Test (timeout = 5000)
+  public void test() throws Exception {
+    conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE);
+    PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE));
+    out.println("<?xml version=\"1.0\"?>");
+    out.println("<allocations>");
+    out.println("<queue name=\"queueA\">");
+    out.println("<minResources>2048mb,0vcores</minResources>");
+    out.println("</queue>");
+    out.println("<queue name=\"queueB\">");
+    out.println("<minResources>2048mb,0vcores</minResources>");
+    out.println("</queue>");
+    out.println("</allocations>");
+    out.close();
+
+    resourceManager = new MockRM(conf);
+    resourceManager.start();
+    scheduler = (FairScheduler) resourceManager.getResourceScheduler();
+
+    // Add one big node (only care about aggregate capacity)
+    RMNode node1 =
+        MockNodes.newNodeInfo(1, Resources.createResource(4 * 1024, 4), 1,
+            "127.0.0.1");
+    NodeAddedSchedulerEvent nodeEvent1 = new NodeAddedSchedulerEvent(node1);
+    scheduler.handle(nodeEvent1);
+
+    scheduler.update();
+
+    // Queue A wants 3 * 1024. Node update gives this all to A
+    createSchedulingRequest(3 * 1024, "queueA", "user1");
+    scheduler.update();
+    NodeUpdateSchedulerEvent nodeEvent2 = new NodeUpdateSchedulerEvent(node1);
+    scheduler.handle(nodeEvent2);
+
+    // Queue B arrives and wants 1 * 1024
+    createSchedulingRequest(1 * 1024, "queueB", "user1");
+    scheduler.update();
+    Collection<FSLeafQueue> queues = scheduler.getQueueManager().getLeafQueues();
+    assertEquals(3, queues.size());
+
+    // Queue A should be above min share, B below.
+    FSLeafQueue queueA =
+        scheduler.getQueueManager().getLeafQueue("queueA", false);
+    FSLeafQueue queueB =
+        scheduler.getQueueManager().getLeafQueue("queueB", false);
+    assertFalse(queueA.isStarvedForMinShare());
+    assertTrue(queueB.isStarvedForMinShare());
+
+    // Node checks in again, should allocate for B
+    scheduler.handle(nodeEvent2);
+    // Now B should have min share ( = demand here)
+    assertFalse(queueB.isStarvedForMinShare());
+  }
+
+  @Test (timeout = 5000)
+  public void testIsStarvedForFairShare() throws Exception {
+    conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE);
+    PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE));
+    out.println("<?xml version=\"1.0\"?>");
+    out.println("<allocations>");
+    out.println("<queue name=\"queueA\">");
+    out.println("<weight>.2</weight>");
+    out.println("</queue>");
+    out.println("<queue name=\"queueB\">");
+    out.println("<weight>.8</weight>");
+    out.println("<fairSharePreemptionThreshold>.4</fairSharePreemptionThreshold>");
+    out.println("<queue name=\"queueB1\">");
+    out.println("</queue>");
+    out.println("<queue name=\"queueB2\">");
+    out.println("<fairSharePreemptionThreshold>.6</fairSharePreemptionThreshold>");
+    out.println("</queue>");
+    out.println("</queue>");
+    out.println("<defaultFairSharePreemptionThreshold>.5</defaultFairSharePreemptionThreshold>");
+    out.println("</allocations>");
+    out.close();
+
+    resourceManager = new MockRM(conf);
+    resourceManager.start();
+    scheduler = (FairScheduler) resourceManager.getResourceScheduler();
+
+    // Add one big node (only care about aggregate capacity)
+    RMNode node1 =
+        MockNodes.newNodeInfo(1, Resources.createResource(10 * 1024, 10), 1,
+            "127.0.0.1");
+    NodeAddedSchedulerEvent nodeEvent1 = new NodeAddedSchedulerEvent(node1);
+    scheduler.handle(nodeEvent1);
+
+    scheduler.update();
+
+    // Queue A wants 4 * 1024. Node update gives this all to A
+    createSchedulingRequest(1 * 1024, "queueA", "user1", 4);
+    scheduler.update();
+    NodeUpdateSchedulerEvent nodeEvent2 = new NodeUpdateSchedulerEvent(node1);
+    for (int i = 0; i < 4; i ++) {
+      scheduler.handle(nodeEvent2);
+    }
+
+    QueueManager queueMgr = scheduler.getQueueManager();
+    FSLeafQueue queueA = queueMgr.getLeafQueue("queueA", false);
+    assertEquals(4 * 1024, queueA.getResourceUsage().getMemory());
+
+    // Both queue B1 and queue B2 want 3 * 1024
+    createSchedulingRequest(1 * 1024, "queueB.queueB1", "user1", 3);
+    createSchedulingRequest(1 * 1024, "queueB.queueB2", "user1", 3);
+    scheduler.update();
+    for (int i = 0; i < 4; i ++) {
+      scheduler.handle(nodeEvent2);
+    }
+
+    FSLeafQueue queueB1 = queueMgr.getLeafQueue("queueB.queueB1", false);
+    FSLeafQueue queueB2 = queueMgr.getLeafQueue("queueB.queueB2", false);
+    assertEquals(2 * 1024, queueB1.getResourceUsage().getMemory());
+    assertEquals(2 * 1024, queueB2.getResourceUsage().getMemory());
+
+    // For queue B1, the fairSharePreemptionThreshold is 0.4, and the fair share
+    // threshold is 1.6 * 1024
+    assertFalse(queueB1.isStarvedForFairShare());
+
+    // For queue B2, the fairSharePreemptionThreshold is 0.6, and the fair share
+    // threshold is 2.4 * 1024
+    assertTrue(queueB2.isStarvedForFairShare());
+
+    // Node checks in again
+    scheduler.handle(nodeEvent2);
+    scheduler.handle(nodeEvent2);
+    assertEquals(3 * 1024, queueB1.getResourceUsage().getMemory());
+    assertEquals(3 * 1024, queueB2.getResourceUsage().getMemory());
+
+    // Both queue B1 and queue B2 usages go to 3 * 1024
+    assertFalse(queueB1.isStarvedForFairShare());
+    assertFalse(queueB2.isStarvedForFairShare());
   }
 }

+ 34 - 124
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java

@@ -1061,9 +1061,11 @@ public class TestFairScheduler extends FairSchedulerTestBase {
     out.println("  </queue>");
     out.println("  <fairSharePreemptionTimeout>100</fairSharePreemptionTimeout>");
     out.println("  <minSharePreemptionTimeout>120</minSharePreemptionTimeout>");
+    out.println("  <fairSharePreemptionThreshold>.5</fairSharePreemptionThreshold>");
     out.println("</queue>");
     out.println("<defaultFairSharePreemptionTimeout>300</defaultFairSharePreemptionTimeout>");
     out.println("<defaultMinSharePreemptionTimeout>200</defaultMinSharePreemptionTimeout>");
+    out.println("<defaultFairSharePreemptionThreshold>.6</defaultFairSharePreemptionThreshold>");
     out.println("</allocations>");
     out.close();
 
@@ -1080,125 +1082,7 @@ public class TestFairScheduler extends FairSchedulerTestBase {
 
     assertEquals(100000, root.getFairSharePreemptionTimeout());
     assertEquals(120000, root.getMinSharePreemptionTimeout());
-  }
-  
-  @Test (timeout = 5000)
-  public void testIsStarvedForMinShare() throws Exception {
-    conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE);
-
-    PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE));
-    out.println("<?xml version=\"1.0\"?>");
-    out.println("<allocations>");
-    out.println("<queue name=\"queueA\">");
-    out.println("<minResources>2048mb,0vcores</minResources>");
-    out.println("</queue>");
-    out.println("<queue name=\"queueB\">");
-    out.println("<minResources>2048mb,0vcores</minResources>");
-    out.println("</queue>");
-    out.println("</allocations>");
-    out.close();
-
-    scheduler.init(conf);
-    scheduler.start();
-    scheduler.reinitialize(conf, resourceManager.getRMContext());
-
-    // Add one big node (only care about aggregate capacity)
-    RMNode node1 =
-        MockNodes.newNodeInfo(1, Resources.createResource(4 * 1024, 4), 1,
-            "127.0.0.1");
-    NodeAddedSchedulerEvent nodeEvent1 = new NodeAddedSchedulerEvent(node1);
-    scheduler.handle(nodeEvent1);
-
-    // Queue A wants 3 * 1024. Node update gives this all to A
-    createSchedulingRequest(3 * 1024, "queueA", "user1");
-    scheduler.update();
-    NodeUpdateSchedulerEvent nodeEvent2 = new NodeUpdateSchedulerEvent(node1);
-    scheduler.handle(nodeEvent2);
-
-    // Queue B arrives and wants 1 * 1024
-    createSchedulingRequest(1 * 1024, "queueB", "user1");
-    scheduler.update();
-    Collection<FSLeafQueue> queues = scheduler.getQueueManager().getLeafQueues();
-    assertEquals(3, queues.size());
-
-    // Queue A should be above min share, B below.
-    for (FSLeafQueue p : queues) {
-      if (p.getName().equals("root.queueA")) {
-        assertEquals(false, scheduler.isStarvedForMinShare(p));
-      }
-      else if (p.getName().equals("root.queueB")) {
-        assertEquals(true, scheduler.isStarvedForMinShare(p));
-      }
-    }
-
-    // Node checks in again, should allocate for B
-    scheduler.handle(nodeEvent2);
-    // Now B should have min share ( = demand here)
-    for (FSLeafQueue p : queues) {
-      if (p.getName().equals("root.queueB")) {
-        assertEquals(false, scheduler.isStarvedForMinShare(p));
-      }
-    }
-  }
-
-  @Test (timeout = 5000)
-  public void testIsStarvedForFairShare() throws Exception {
-    conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE);
-
-    PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE));
-    out.println("<?xml version=\"1.0\"?>");
-    out.println("<allocations>");
-    out.println("<queue name=\"queueA\">");
-    out.println("<weight>.25</weight>");
-    out.println("</queue>");
-    out.println("<queue name=\"queueB\">");
-    out.println("<weight>.75</weight>");
-    out.println("</queue>");
-    out.println("</allocations>");
-    out.close();
-
-    scheduler.init(conf);
-    scheduler.start();
-    scheduler.reinitialize(conf, resourceManager.getRMContext());
-
-    // Add one big node (only care about aggregate capacity)
-    RMNode node1 =
-        MockNodes.newNodeInfo(1, Resources.createResource(4 * 1024, 4), 1,
-            "127.0.0.1");
-    NodeAddedSchedulerEvent nodeEvent1 = new NodeAddedSchedulerEvent(node1);
-    scheduler.handle(nodeEvent1);
-
-    // Queue A wants 3 * 1024. Node update gives this all to A
-    createSchedulingRequest(3 * 1024, "queueA", "user1");
-    scheduler.update();
-    NodeUpdateSchedulerEvent nodeEvent2 = new NodeUpdateSchedulerEvent(node1);
-    scheduler.handle(nodeEvent2);
-
-    // Queue B arrives and wants 1 * 1024
-    createSchedulingRequest(1 * 1024, "queueB", "user1");
-    scheduler.update();
-    Collection<FSLeafQueue> queues = scheduler.getQueueManager().getLeafQueues();
-    assertEquals(3, queues.size());
-
-    // Queue A should be above fair share, B below.
-    for (FSLeafQueue p : queues) {
-      if (p.getName().equals("root.queueA")) {
-        assertEquals(false, scheduler.isStarvedForFairShare(p));
-      }
-      else if (p.getName().equals("root.queueB")) {
-        assertEquals(true, scheduler.isStarvedForFairShare(p));
-      }
-    }
-
-    // Node checks in again, should allocate for B
-    scheduler.handle(nodeEvent2);
-    // B should not be starved for fair share, since entire demand is
-    // satisfied.
-    for (FSLeafQueue p : queues) {
-      if (p.getName().equals("root.queueB")) {
-        assertEquals(false, scheduler.isStarvedForFairShare(p));
-      }
-    }
+    assertEquals(0.5f, root.getFairSharePreemptionThreshold(), 0.01);
   }
 
   @Test (timeout = 5000)
@@ -1385,7 +1269,8 @@ public class TestFairScheduler extends FairSchedulerTestBase {
     out.println("<queue name=\"queueB\">");
     out.println("<weight>2</weight>");
     out.println("</queue>");
-    out.print("<defaultFairSharePreemptionTimeout>10</defaultFairSharePreemptionTimeout>");
+    out.println("<defaultFairSharePreemptionTimeout>10</defaultFairSharePreemptionTimeout>");
+    out.println("<defaultFairSharePreemptionThreshold>.5</defaultFairSharePreemptionThreshold>");
     out.println("</allocations>");
     out.close();
 
@@ -1468,8 +1353,9 @@ public class TestFairScheduler extends FairSchedulerTestBase {
     out.println("<weight>.25</weight>");
     out.println("<minResources>1024mb,0vcores</minResources>");
     out.println("</queue>");
-    out.print("<defaultMinSharePreemptionTimeout>5</defaultMinSharePreemptionTimeout>");
-    out.print("<defaultFairSharePreemptionTimeout>10</defaultFairSharePreemptionTimeout>");
+    out.println("<defaultMinSharePreemptionTimeout>5</defaultMinSharePreemptionTimeout>");
+    out.println("<defaultFairSharePreemptionTimeout>10</defaultFairSharePreemptionTimeout>");
+    out.println("<defaultFairSharePreemptionThreshold>.5</defaultFairSharePreemptionThreshold>");
     out.println("</allocations>");
     out.close();
 
@@ -1753,8 +1639,6 @@ public class TestFairScheduler extends FairSchedulerTestBase {
   @Test
   public void testBackwardsCompatiblePreemptionConfiguration() throws Exception {
     conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE);
-    MockClock clock = new MockClock();
-    scheduler.setClock(clock);
 
     PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE));
     out.println("<?xml version=\"1.0\"?>");
@@ -1842,6 +1726,32 @@ public class TestFairScheduler extends FairSchedulerTestBase {
         .getFairSharePreemptionTimeout());
   }
 
+  @Test
+  public void testPreemptionVariablesForQueueCreatedRuntime() throws Exception {
+    conf.set(FairSchedulerConfiguration.USER_AS_DEFAULT_QUEUE, "true");
+    scheduler.init(conf);
+    scheduler.start();
+    scheduler.reinitialize(conf, resourceManager.getRMContext());
+
+    // Set preemption variables for the root queue
+    FSParentQueue root = scheduler.getQueueManager().getRootQueue();
+    root.setMinSharePreemptionTimeout(10000);
+    root.setFairSharePreemptionTimeout(15000);
+    root.setFairSharePreemptionThreshold(.6f);
+
+    // User1 submits one application
+    ApplicationAttemptId appAttemptId = createAppAttemptId(1, 1);
+    createApplicationWithAMResource(appAttemptId, "default", "user1", null);
+
+    // The user1 queue should inherit the configurations from the root queue
+    FSLeafQueue userQueue =
+        scheduler.getQueueManager().getLeafQueue("user1", true);
+    assertEquals(1, userQueue.getRunnableAppSchedulables().size());
+    assertEquals(10000, userQueue.getMinSharePreemptionTimeout());
+    assertEquals(15000, userQueue.getFairSharePreemptionTimeout());
+    assertEquals(.6f, userQueue.getFairSharePreemptionThreshold(), 0.001);
+  }
+
   @Test (timeout = 5000)
   public void testMultipleContainersWaitingForReservation() throws IOException {
     scheduler.init(conf);

+ 10 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/FairScheduler.apt.vm

@@ -277,6 +277,12 @@ Allocation file format
      threshold before it will try to preempt containers to take resources from other
      queues. If not set, the queue will inherit the value from its parent queue.
 
+   * fairSharePreemptionThreshold: the fair share preemption threshold for the
+     queue. If the queue waits fairSharePreemptionTimeout without receiving
+     fairSharePreemptionThreshold*fairShare resources, it is allowed to preempt
+     containers to take resources from other queues. If not set, the queue will
+     inherit the value from its parent queue.
+
  * <<User elements>>, which represent settings governing the behavior of individual 
      users. They can contain a single property: maxRunningApps, a limit on the 
      number of running apps for a particular user.
@@ -292,6 +298,10 @@ Allocation file format
    preemption timeout for the root queue; overridden by minSharePreemptionTimeout
    element in root queue.
 
+ * <<A defaultFairSharePreemptionThreshold element>>, which sets the fair share
+   preemption threshold for the root queue; overridden by fairSharePreemptionThreshold
+   element in root queue.
+
  * <<A queueMaxAppsDefault element>>, which sets the default running app limit
    for queues; overriden by maxRunningApps element in each queue.