Selaa lähdekoodia

Merge trunk into HDFS-6581

Conflicts:
	hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
	hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeLayoutVersion.java
arp 10 vuotta sitten
vanhempi
commit
339d21f273
86 muutettua tiedostoa jossa 1786 lisäystä ja 676 poistoa
  1. 25 0
      hadoop-common-project/hadoop-common/CHANGES.txt
  2. 1 1
      hadoop-common-project/hadoop-common/dev-support/findbugsExcludeFile.xml
  3. 2 2
      hadoop-common-project/hadoop-common/src/main/bin/hadoop-functions.sh
  4. 26 10
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProviderFactory.java
  5. 2 55
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/KMSClientProvider.java
  6. 18 3
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/random/OsSecureRandom.java
  7. 11 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/authorize/AccessControlList.java
  8. 1 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/ssl/FileBasedKeyStoresFactory.java
  9. 3 12
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticationFilter.java
  10. 4 2
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticationHandler.java
  11. 2 18
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticator.java
  12. 185 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/HttpExceptionUtils.java
  13. 51 18
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/TestCryptoCodec.java
  14. 1 1
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/TestCryptoStreams.java
  15. 13 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/key/TestKeyProviderFactory.java
  16. 15 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/random/TestOsSecureRandom.java
  17. 14 21
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/delegation/web/TestDelegationTokenAuthenticationHandlerWithMocks.java
  18. 167 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestHttpExceptionUtils.java
  19. 1 0
      hadoop-common-project/hadoop-kms/pom.xml
  20. 6 21
      hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMS.java
  21. 51 4
      hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSACLs.java
  22. 2 10
      hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSExceptionsProvider.java
  23. 86 2
      hadoop-common-project/hadoop-kms/src/site/apt/index.apt.vm
  24. 94 6
      hadoop-common-project/hadoop-kms/src/test/java/org/apache/hadoop/crypto/key/kms/server/TestKMS.java
  25. 1 1
      hadoop-common-project/hadoop-kms/src/test/java/org/apache/hadoop/crypto/key/kms/server/TestKMSACLs.java
  26. 31 39
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSFileSystem.java
  27. 0 50
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSUtils.java
  28. 2 12
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/wsrs/ExceptionProvider.java
  29. 3 1
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/BaseTestHttpFSWith.java
  30. 4 6
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSServerNoACLs.java
  31. 28 0
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  32. 3 3
      hadoop-hdfs-project/hadoop-hdfs/pom.xml
  33. 7 8
      hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs
  34. 3 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
  35. 72 8
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
  36. 8 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
  37. 12 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/inotify/Event.java
  38. 4 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
  39. 6 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolManager.java
  40. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java
  41. 3 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
  42. 4 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
  43. 6 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
  44. 18 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java
  45. 17 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageSerialization.java
  46. 29 17
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
  47. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeReference.java
  48. 1 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/InotifyFSEditLogOpTranslator.java
  49. 4 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeLayoutVersion.java
  50. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/DirectorySnapshottableFeature.java
  51. 77 98
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java
  52. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/GetConf.java
  53. 1 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/proto/inotify.proto
  54. 18 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
  55. 15 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlocksScheduledCounter.java
  56. 2 6
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSInotifyEventInputStream.java
  57. 26 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java
  58. 119 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreation.java
  59. 12 12
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestHDFSServerPorts.java
  60. 28 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLocalDFS.java
  61. 22 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockPoolManager.java
  62. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/CreateEditsLog.java
  63. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java
  64. 8 8
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestValidateConfigurationSettings.java
  65. 26 37
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestDelegationTokensWithHA.java
  66. 5 5
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestHAMetrics.java
  67. 21 31
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestHAStateTransitions.java
  68. 22 32
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestStandbyCheckpoints.java
  69. 36 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotDiffReport.java
  70. 22 4
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestGetConf.java
  71. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tools/TestTools.java
  72. BIN
      hadoop-hdfs-project/hadoop-hdfs/src/test/resources/editsStored
  73. 119 63
      hadoop-hdfs-project/hadoop-hdfs/src/test/resources/editsStored.xml
  74. 6 0
      hadoop-mapreduce-project/CHANGES.txt
  75. 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
  76. 1 3
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/MapTask.java
  77. 12 0
      hadoop-yarn-project/CHANGES.txt
  78. 8 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
  79. 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
  80. 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
  81. 11 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/webapp/CrossOriginFilterInitializer.java
  82. 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
  83. 2 5
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/webapp/TestCrossOriginFilterInitializer.java
  84. 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
  85. 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
  86. 1 0
      pom.xml

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

@@ -330,6 +330,9 @@ Trunk (Unreleased)
 
     HADOOP-11033. shell scripts ignore JAVA_HOME on OS X. (aw)
 
+    HADOOP-11052. hadoop_verify_secure_prereq's results aren't checked 
+    in bin/hdfs (aw)
+
   OPTIMIZATIONS
 
     HADOOP-7761. Improve the performance of raw comparisons. (todd)
@@ -493,6 +496,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 +695,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 +757,12 @@ 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)
+
+    HADOOP-11067. warning message 'ssl.client.truststore.location has not
+    been set' gets printed for hftp command. (Xiaoyu Yao via Arpit Agarwal)
+
 Release 2.5.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES
@@ -757,6 +780,8 @@ Release 2.5.1 - UNRELEASED
 
     HADOOP-11001. Fix test-patch to work with the git repo. (kasha)
 
+    HADOOP-11065. Rat check should exclude "**/build/**". (kasha)
+
 Release 2.5.0 - 2014-08-11
 
   INCOMPATIBLE CHANGES

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

+ 2 - 2
hadoop-common-project/hadoop-common/src/main/bin/hadoop-functions.sh

@@ -644,9 +644,9 @@ function hadoop_verify_secure_prereq
   # this.
   
   # ${EUID} comes from the shell itself!
-  if [[ "${EUID}" -ne 0 ]] || [[ -n "${HADOOP_SECURE_COMMAND}" ]]; then
+  if [[ "${EUID}" -ne 0 ]] && [[ -z "${HADOOP_SECURE_COMMAND}" ]]; then
     hadoop_error "ERROR: You must be a privileged in order to run a secure serice."
-    return 1
+    exit 1
   else
     return 0
   fi

+ 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

+ 1 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/ssl/FileBasedKeyStoresFactory.java

@@ -212,7 +212,7 @@ public class FileBasedKeyStoresFactory implements KeyStoresFactory {
       LOG.debug(mode.toString() + " Loaded TrustStore: " + truststoreLocation);
       trustManagers = new TrustManager[]{trustManager};
     } else {
-      LOG.warn("The property '" + locationProperty + "' has not been set, " +
+      LOG.debug("The property '" + locationProperty + "' has not been set, " +
           "no TrustStore will be loaded");
       trustManagers = null;
     }

+ 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()) {

+ 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;
@@ -180,11 +179,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";
@@ -224,7 +218,6 @@ public class HttpFSFileSystem extends FileSystem
   private URI uri;
   private Path workingDir;
   private UserGroupInformation realUser;
-  private String doAs;
 
 
 
@@ -337,7 +330,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());
@@ -436,7 +428,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));
   }
@@ -463,7 +455,7 @@ public class HttpFSFileSystem extends FileSystem
       try {
         super.close();
       } finally {
-        HttpFSUtils.validateResponse(conn, closeStatus);
+        HttpExceptionUtils.validateResponse(conn, closeStatus);
       }
     }
 
@@ -499,11 +491,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 {
@@ -515,7 +507,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;
       }
     }
@@ -596,7 +588,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);
   }
 
   /**
@@ -610,7 +602,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);
   }
@@ -645,7 +637,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);
   }
@@ -666,7 +658,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);
@@ -714,7 +706,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);
   }
@@ -735,7 +727,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);
@@ -754,7 +746,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) {
@@ -779,7 +771,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);
   }
 
   /**
@@ -794,7 +786,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);
   }
 
   /**
@@ -816,7 +808,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);
   }
 
   /**
@@ -838,7 +830,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);
   }
@@ -858,7 +850,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);
   }
 
   /**
@@ -875,7 +867,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);
   }
 
   /**
@@ -889,7 +881,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);
   }
 
   /**
@@ -903,7 +895,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);
   }
 
   /**
@@ -921,7 +913,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);
   }
 
   /**
@@ -936,7 +928,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);
@@ -998,7 +990,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),
@@ -1016,7 +1008,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() {
@@ -1117,7 +1109,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
@@ -1127,7 +1119,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));
@@ -1171,7 +1163,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));
   }
@@ -1187,7 +1179,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));
   }
@@ -1198,7 +1190,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));
   }
@@ -1210,6 +1202,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"));
     }
   }
 

+ 28 - 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)
@@ -587,6 +596,20 @@ Release 2.6.0 - UNRELEASED
 
     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)
+
+    HDFS-6831. Inconsistency between 'hdfs dfsadmin' and 'hdfs dfsadmin -help'.
+    (Xiaoyu Yao via Arpit Agarwal)
+
+    HDFS-6979. hdfs.dll does not produce .pdb files. (cnauroth)
+
+    HDFS-6862. Add missing timeout annotations to tests. (Xiaoyu Yao via
+    Arpit Agarwal)
+
     BREAKDOWN OF HDFS-6134 AND HADOOP-10150 SUBTASKS AND RELATED JIRAS
   
       HDFS-6387. HDFS CLI admin tool for creating & deleting an
@@ -683,6 +706,11 @@ Release 2.6.0 - UNRELEASED
       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

+ 3 - 3
hadoop-hdfs-project/hadoop-hdfs/pom.xml

@@ -415,11 +415,11 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
                     </exec>
                     <exec executable="msbuild" dir="${project.build.directory}/native"
                         failonerror="true">
-                      <arg line="ALL_BUILD.vcxproj /nologo /p:Configuration=Release"/>
+                      <arg line="ALL_BUILD.vcxproj /nologo /p:Configuration=RelWithDebInfo /p:LinkIncremental=false"/>
                     </exec>
                     <!-- Copy for inclusion in distribution. -->
                     <copy todir="${project.build.directory}/bin">
-                      <fileset dir="${project.build.directory}/native/target/bin/Release"/>
+                      <fileset dir="${project.build.directory}/native/target/bin/RelWithDebInfo"/>
                     </copy>
                   </target>
                 </configuration>
@@ -437,7 +437,7 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
                       <attribute name="test"/>
                       <sequential>
                         <echo message="Running @{test}"/>
-                        <exec executable="${project.build.directory}/native/Release/@{test}" failonerror="true" dir="${project.build.directory}/native/">
+                        <exec executable="${project.build.directory}/native/RelWithDebInfo/@{test}" failonerror="true" dir="${project.build.directory}/native/">
                           <env key="CLASSPATH" value="${test_classpath}:${compile_classpath}"/>
                           <!-- HADOOP_HOME required to find winutils. -->
                           <env key="HADOOP_HOME" value="${hadoop.common.build.dir}"/>

+ 7 - 8
hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs

@@ -225,14 +225,13 @@ esac
 
 if [[ -n "${secure_service}" ]]; then
   HADOOP_SECURE_USER="${secure_user}"
-  if hadoop_verify_secure_prereq; then
-    hadoop_setup_secure_service
-    priv_outfile="${HADOOP_LOG_DIR}/privileged-${HADOOP_IDENT_STRING}-${COMMAND-$HOSTNAME}.out"
-    priv_errfile="${HADOOP_LOG_DIR}/privileged-${HADOOP_IDENT_STRING}-${COMMAND-$HOSTNAME}.err"
-    priv_pidfile="${HADOOP_PID_DIR}/privileged-${HADOOP_IDENT_STRING}-${COMMAND-$HOSTNAME}.pid"
-    daemon_outfile="${HADOOP_LOG_DIR}/hadoop-${HADOOP_SECURE_USER}-${HADOOP_IDENT_STRING}-${COMMAND}-${HOSTNAME}.out"
-    daemon_pidfile="${HADOOP_PID_DIR}/hadoop-${HADOOP_SECURE_USER}-${HADOOP_IDENT_STRING}-${COMMAND}.pid"
-  fi
+  hadoop_verify_secure_prereq
+  hadoop_setup_secure_service
+  priv_outfile="${HADOOP_LOG_DIR}/privileged-${HADOOP_IDENT_STRING}-${COMMAND-$HOSTNAME}.out"
+  priv_errfile="${HADOOP_LOG_DIR}/privileged-${HADOOP_IDENT_STRING}-${COMMAND-$HOSTNAME}.err"
+  priv_pidfile="${HADOOP_PID_DIR}/privileged-${HADOOP_IDENT_STRING}-${COMMAND-$HOSTNAME}.pid"
+  daemon_outfile="${HADOOP_LOG_DIR}/hadoop-${HADOOP_SECURE_USER}-${HADOOP_IDENT_STRING}-${COMMAND}-${HOSTNAME}.out"
+  daemon_pidfile="${HADOOP_PID_DIR}/hadoop-${HADOOP_SECURE_USER}-${HADOOP_IDENT_STRING}-${COMMAND}.pid"
 else
   daemon_outfile="${HADOOP_LOG_DIR}/hadoop-${HADOOP_IDENT_STRING}-${COMMAND}-${HOSTNAME}.out"
   daemon_pidfile="${HADOOP_PID_DIR}/hadoop-${HADOOP_IDENT_STRING}-${COMMAND}.pid"

+ 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";
@@ -533,6 +535,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

@@ -2443,7 +2443,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 =
@@ -2521,7 +2522,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/DataStorage.java

@@ -464,7 +464,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.

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

@@ -430,7 +430,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)
@@ -481,7 +482,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)

+ 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())
@@ -717,7 +718,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

@@ -408,6 +408,7 @@ public abstract class FSEditLogOp {
     List<XAttr> xAttrs;
     String clientName;
     String clientMachine;
+    boolean overwrite;
     
     private AddCloseOp(FSEditLogOpCodes opCode) {
       super(opCode);
@@ -492,6 +493,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 {
@@ -512,6 +518,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);
       }
@@ -584,6 +591,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 {
@@ -641,6 +654,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);
       }
@@ -671,6 +686,8 @@ public abstract class FSEditLogOp {
           Boolean.toString(isLazyPersist));
       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);
       }
@@ -699,6 +716,7 @@ public abstract class FSEditLogOp {
 
       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,
@@ -368,6 +370,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;

+ 29 - 17
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -2481,6 +2481,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;
@@ -2529,9 +2530,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, isLazyPersist,
-              suite, edek, logRetryCache);
+          toRemoveBlocks = startFileInternal(pc, src, permissions, holder, 
+              clientMachine, create, overwrite, createParent, replication, 
+              blockSize, isLazyPersist, suite, edek, logRetryCache);
           stat = dir.getFileInfo(src, false,
               FSDirectory.isReservedRawName(srcArg));
         } catch (StandbyException se) {
@@ -2552,6 +2553,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();
+        }
       }
     }
 
@@ -2568,11 +2573,12 @@ 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, boolean isLazyPersist,
-      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, 
+       boolean isLazyPersist, CipherSuite suite,
+       EncryptedKeyVersion edek, boolean logRetryEntry)
       throws FileAlreadyExistsException, AccessControlException,
       UnresolvedLinkException, FileNotFoundException,
       ParentNotDirectoryException, RetryStartFileException, IOException {
@@ -2608,9 +2614,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) {
@@ -2618,6 +2627,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     }
 
     try {
+      BlocksMapUpdateInfo toRemoveBlocks = null;
       if (myFile == null) {
         if (!create) {
           throw new FileNotFoundException("Can't overwrite non-existent " +
@@ -2625,11 +2635,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
@@ -2663,11 +2674,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());
@@ -2775,7 +2787,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     }
 
     if (writeToEditLog) {
-      getEditLog().logOpenFile(src, cons, logRetryCache);
+      getEditLog().logOpenFile(src, cons, false, logRetryCache);
     }
     return ret;
   }

+ 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) };

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

@@ -66,8 +66,11 @@ public class NameNodeLayoutVersion {
     ROLLING_UPGRADE(-55, -53, "Support rolling upgrade", false),
     EDITLOG_LENGTH(-56, "Add length field to every edit log op"),
     XATTRS(-57, "Extended attributes"),
-    LAZY_PERSIST_FILES(-58, "Support for optional lazy persistence of "
+    CREATE_OVERWRITE(-58, "Use single editlog record for " +
+        "creating file with overwrite"),
+    LAZY_PERSIST_FILES(-59, "Support for optional lazy persistence of "
         + " files with reduced durability guarantees");
+
     
     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. */

+ 77 - 98
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java

@@ -352,6 +352,40 @@ public class DFSAdmin extends FsShell {
     }
   }
 
+  /**
+   * Common usage summary shared between "hdfs dfsadmin -help" and
+   * "hdfs dfsadmin"
+   */
+  private static final String commonUsageSummary =
+    "\t[-report [-live] [-dead] [-decommissioning]]\n" +
+    "\t[-safemode <enter | leave | get | wait>]\n" +
+    "\t[-saveNamespace]\n" +
+    "\t[-rollEdits]\n" +
+    "\t[-restoreFailedStorage true|false|check]\n" +
+    "\t[-refreshNodes]\n" +
+    "\t[" + SetQuotaCommand.USAGE + "]\n" +
+    "\t[" + ClearQuotaCommand.USAGE +"]\n" +
+    "\t[" + SetSpaceQuotaCommand.USAGE + "]\n" +
+    "\t[" + ClearSpaceQuotaCommand.USAGE +"]\n" +
+    "\t[-finalizeUpgrade]\n" +
+    "\t[" + RollingUpgradeCommand.USAGE +"]\n" +
+    "\t[-refreshServiceAcl]\n" +
+    "\t[-refreshUserToGroupsMappings]\n" +
+    "\t[-refreshSuperUserGroupsConfiguration]\n" +
+    "\t[-refreshCallQueue]\n" +
+    "\t[-refresh <host:ipc_port> <key> [arg1..argn]\n" +
+    "\t[-printTopology]\n" +
+    "\t[-refreshNamenodes datanode_host:ipc_port]\n"+
+    "\t[-deleteBlockPool datanode_host:ipc_port blockpoolId [force]]\n"+
+    "\t[-setBalancerBandwidth <bandwidth in bytes per second>]\n" +
+    "\t[-fetchImage <local directory>]\n" +
+    "\t[-allowSnapshot <snapshotDir>]\n" +
+    "\t[-disallowSnapshot <snapshotDir>]\n" +
+    "\t[-shutdownDatanode <datanode_host:ipc_port> [upgrade]]\n" +
+    "\t[-getDatanodeInfo <datanode_host:ipc_port>]\n" +
+    "\t[-metasave filename]\n" +
+    "\t[-help [cmd]]\n";
+
   /**
    * Construct a DFSAdmin object.
    */
@@ -589,7 +623,7 @@ public class DFSAdmin extends FsShell {
   
   /**
    * Command to ask the namenode to save the namespace.
-   * Usage: java DFSAdmin -saveNamespace
+   * Usage: hdfs dfsadmin -saveNamespace
    * @exception IOException 
    * @see org.apache.hadoop.hdfs.protocol.ClientProtocol#saveNamespace()
    */
@@ -630,7 +664,7 @@ public class DFSAdmin extends FsShell {
   
   /**
    * Command to enable/disable/check restoring of failed storage replicas in the namenode.
-   * Usage: java DFSAdmin -restoreFailedStorage true|false|check
+   * Usage: hdfs dfsadmin -restoreFailedStorage true|false|check
    * @exception IOException 
    * @see org.apache.hadoop.hdfs.protocol.ClientProtocol#restoreFailedStorage(String arg)
    */
@@ -668,7 +702,7 @@ public class DFSAdmin extends FsShell {
   /**
    * Command to ask the namenode to reread the hosts and excluded hosts 
    * file.
-   * Usage: java DFSAdmin -refreshNodes
+   * Usage: hdfs dfsadmin -refreshNodes
    * @exception IOException 
    */
   public int refreshNodes() throws IOException {
@@ -701,7 +735,7 @@ public class DFSAdmin extends FsShell {
   /**
    * Command to ask the namenode to set the balancer bandwidth for all of the
    * datanodes.
-   * Usage: java DFSAdmin -setBalancerBandwidth bandwidth
+   * Usage: hdfs dfsadmin -setBalancerBandwidth bandwidth
    * @param argv List of of command line parameters.
    * @param idx The index of the command that is being processed.
    * @exception IOException 
@@ -714,7 +748,7 @@ public class DFSAdmin extends FsShell {
       bandwidth = Long.parseLong(argv[idx]);
     } catch (NumberFormatException nfe) {
       System.err.println("NumberFormatException: " + nfe.getMessage());
-      System.err.println("Usage: java DFSAdmin"
+      System.err.println("Usage: hdfs dfsadmin"
                   + " [-setBalancerBandwidth <bandwidth in bytes per second>]");
       return exitCode;
     }
@@ -777,36 +811,11 @@ public class DFSAdmin extends FsShell {
   }
 
   private void printHelp(String cmd) {
-    String summary = "hadoop dfsadmin performs DFS administrative commands.\n" +
+    String summary = "hdfs dfsadmin performs DFS administrative commands.\n" +
+      "Note: Administrative commands can only be run with superuser permission.\n" +
       "The full syntax is: \n\n" +
-      "hadoop dfsadmin\n" +
-      "\t[-report [-live] [-dead] [-decommissioning]]\n" +
-      "\t[-safemode <enter | leave | get | wait>]\n" +
-      "\t[-saveNamespace]\n" +
-      "\t[-rollEdits]\n" +
-      "\t[-restoreFailedStorage true|false|check]\n" +
-      "\t[-refreshNodes]\n" +
-      "\t[" + SetQuotaCommand.USAGE + "]\n" +
-      "\t[" + ClearQuotaCommand.USAGE +"]\n" +
-      "\t[" + SetSpaceQuotaCommand.USAGE + "]\n" +
-      "\t[" + ClearSpaceQuotaCommand.USAGE +"]\n" +
-      "\t[-finalizeUpgrade]\n" +
-      "\t[" + RollingUpgradeCommand.USAGE +"]\n" +
-      "\t[-refreshServiceAcl]\n" +
-      "\t[-refreshUserToGroupsMappings]\n" +
-      "\t[-refreshSuperUserGroupsConfiguration]\n" +
-      "\t[-refreshCallQueue]\n" +
-      "\t[-refresh <host:ipc_port> <key> [arg1..argn]\n" +
-      "\t[-printTopology]\n" +
-      "\t[-refreshNamenodes datanodehost:port]\n"+
-      "\t[-deleteBlockPool datanodehost:port blockpoolId [force]]\n"+
-      "\t[-setBalancerBandwidth <bandwidth>]\n" +
-      "\t[-fetchImage <local directory>]\n" +
-      "\t[-allowSnapshot <snapshotDir>]\n" +
-      "\t[-disallowSnapshot <snapshotDir>]\n" +
-      "\t[-shutdownDatanode <datanode_host:ipc_port> [upgrade]]\n" +
-      "\t[-getDatanodeInfo <datanode_host:ipc_port>\n" +
-      "\t[-help [cmd]]\n";
+      "hdfs dfsadmin\n" +
+      commonUsageSummary;
 
     String report ="-report [-live] [-dead] [-decommissioning]:\n" +
       "\tReports basic filesystem information and statistics.\n" +
@@ -825,15 +834,13 @@ public class DFSAdmin extends FsShell {
 
     String saveNamespace = "-saveNamespace:\t" +
     "Save current namespace into storage directories and reset edits log.\n" +
-    "\t\tRequires superuser permissions and safe mode.\n";
+    "\t\tRequires safe mode.\n";
 
     String rollEdits = "-rollEdits:\t" +
-    "Rolls the edit log.\n" +
-    "\t\tRequires superuser permissions.\n";
+    "Rolls the edit log.\n";
     
     String restoreFailedStorage = "-restoreFailedStorage:\t" +
-    "Set/Unset/Check flag to attempt restore of failed storage replicas if they become available.\n" +
-    "\t\tRequires superuser permissions.\n";
+    "Set/Unset/Check flag to attempt restore of failed storage replicas if they become available.\n";
     
     String refreshNodes = "-refreshNodes: \tUpdates the namenode with the " +
       "set of datanodes allowed to connect to the namenode.\n\n" +
@@ -1021,7 +1028,7 @@ public class DFSAdmin extends FsShell {
 
   /**
    * Command to ask the namenode to finalize previously performed upgrade.
-   * Usage: java DFSAdmin -finalizeUpgrade
+   * Usage: hdfs dfsadmin -finalizeUpgrade
    * @exception IOException 
    */
   public int finalizeUpgrade() throws IOException {
@@ -1058,7 +1065,7 @@ public class DFSAdmin extends FsShell {
 
   /**
    * Dumps DFS data structures into specified file.
-   * Usage: java DFSAdmin -metasave filename
+   * Usage: hdfs dfsadmin -metasave filename
    * @param argv List of of command line parameters.
    * @param idx The index of the command that is being processed.
    * @exception IOException if an error occurred while accessing
@@ -1366,118 +1373,90 @@ public class DFSAdmin extends FsShell {
    */
   private static void printUsage(String cmd) {
     if ("-report".equals(cmd)) {
-      System.err.println("Usage: java DFSAdmin"
+      System.err.println("Usage: hdfs dfsadmin"
           + " [-report] [-live] [-dead] [-decommissioning]");
     } else if ("-safemode".equals(cmd)) {
-      System.err.println("Usage: java DFSAdmin"
+      System.err.println("Usage: hdfs dfsadmin"
           + " [-safemode enter | leave | get | wait]");
     } else if ("-allowSnapshot".equalsIgnoreCase(cmd)) {
-      System.err.println("Usage: java DFSAdmin"
+      System.err.println("Usage: hdfs dfsadmin"
           + " [-allowSnapshot <snapshotDir>]");
     } else if ("-disallowSnapshot".equalsIgnoreCase(cmd)) {
-      System.err.println("Usage: java DFSAdmin"
+      System.err.println("Usage: hdfs dfsadmin"
           + " [-disallowSnapshot <snapshotDir>]");
     } else if ("-saveNamespace".equals(cmd)) {
-      System.err.println("Usage: java DFSAdmin"
+      System.err.println("Usage: hdfs dfsadmin"
                          + " [-saveNamespace]");
     } else if ("-rollEdits".equals(cmd)) {
-      System.err.println("Usage: java DFSAdmin"
+      System.err.println("Usage: hdfs dfsadmin"
                          + " [-rollEdits]");
     } else if ("-restoreFailedStorage".equals(cmd)) {
-      System.err.println("Usage: java DFSAdmin"
+      System.err.println("Usage: hdfs dfsadmin"
           + " [-restoreFailedStorage true|false|check ]");
     } else if ("-refreshNodes".equals(cmd)) {
-      System.err.println("Usage: java DFSAdmin"
+      System.err.println("Usage: hdfs dfsadmin"
                          + " [-refreshNodes]");
     } else if ("-finalizeUpgrade".equals(cmd)) {
-      System.err.println("Usage: java DFSAdmin"
+      System.err.println("Usage: hdfs dfsadmin"
                          + " [-finalizeUpgrade]");
     } else if (RollingUpgradeCommand.matches(cmd)) {
-      System.err.println("Usage: java DFSAdmin"
+      System.err.println("Usage: hdfs dfsadmin"
           + " [" + RollingUpgradeCommand.USAGE+"]");
     } else if ("-metasave".equals(cmd)) {
-      System.err.println("Usage: java DFSAdmin"
+      System.err.println("Usage: hdfs dfsadmin"
           + " [-metasave filename]");
     } else if (SetQuotaCommand.matches(cmd)) {
-      System.err.println("Usage: java DFSAdmin"
+      System.err.println("Usage: hdfs dfsadmin"
                          + " [" + SetQuotaCommand.USAGE+"]");
     } else if (ClearQuotaCommand.matches(cmd)) {
-      System.err.println("Usage: java DFSAdmin"
+      System.err.println("Usage: hdfs dfsadmin"
                          + " ["+ClearQuotaCommand.USAGE+"]");
     } else if (SetSpaceQuotaCommand.matches(cmd)) {
-      System.err.println("Usage: java DFSAdmin"
+      System.err.println("Usage: hdfs dfsadmin"
                          + " [" + SetSpaceQuotaCommand.USAGE+"]");
     } else if (ClearSpaceQuotaCommand.matches(cmd)) {
-      System.err.println("Usage: java DFSAdmin"
+      System.err.println("Usage: hdfs dfsadmin"
                          + " ["+ClearSpaceQuotaCommand.USAGE+"]");
     } else if ("-refreshServiceAcl".equals(cmd)) {
-      System.err.println("Usage: java DFSAdmin"
+      System.err.println("Usage: hdfs dfsadmin"
                          + " [-refreshServiceAcl]");
     } else if ("-refreshUserToGroupsMappings".equals(cmd)) {
-      System.err.println("Usage: java DFSAdmin"
+      System.err.println("Usage: hdfs dfsadmin"
                          + " [-refreshUserToGroupsMappings]");
     } else if ("-refreshSuperUserGroupsConfiguration".equals(cmd)) {
-      System.err.println("Usage: java DFSAdmin"
+      System.err.println("Usage: hdfs dfsadmin"
                          + " [-refreshSuperUserGroupsConfiguration]");
     } else if ("-refreshCallQueue".equals(cmd)) {
-      System.err.println("Usage: java DFSAdmin"
+      System.err.println("Usage: hdfs dfsadmin"
                          + " [-refreshCallQueue]");
     } else if ("-refresh".equals(cmd)) {
-      System.err.println("Usage: java DFSAdmin"
+      System.err.println("Usage: hdfs dfsadmin"
                          + " [-refresh <hostname:port> <resource_identifier> [arg1..argn]");
     } else if ("-printTopology".equals(cmd)) {
-      System.err.println("Usage: java DFSAdmin"
+      System.err.println("Usage: hdfs dfsadmin"
                          + " [-printTopology]");
     } else if ("-refreshNamenodes".equals(cmd)) {
-      System.err.println("Usage: java DFSAdmin"
+      System.err.println("Usage: hdfs dfsadmin"
                          + " [-refreshNamenodes datanode-host:port]");
     } else if ("-deleteBlockPool".equals(cmd)) {
-      System.err.println("Usage: java DFSAdmin"
+      System.err.println("Usage: hdfs dfsadmin"
           + " [-deleteBlockPool datanode-host:port blockpoolId [force]]");
     } else if ("-setBalancerBandwidth".equals(cmd)) {
-      System.err.println("Usage: java DFSAdmin"
+      System.err.println("Usage: hdfs dfsadmin"
                   + " [-setBalancerBandwidth <bandwidth in bytes per second>]");
     } else if ("-fetchImage".equals(cmd)) {
-      System.err.println("Usage: java DFSAdmin"
+      System.err.println("Usage: hdfs dfsadmin"
           + " [-fetchImage <local directory>]");
     } else if ("-shutdownDatanode".equals(cmd)) {
-      System.err.println("Usage: java DFSAdmin"
+      System.err.println("Usage: hdfs dfsadmin"
           + " [-shutdownDatanode <datanode_host:ipc_port> [upgrade]]");
     } else if ("-getDatanodeInfo".equals(cmd)) {
-      System.err.println("Usage: java DFSAdmin"
+      System.err.println("Usage: hdfs dfsadmin"
           + " [-getDatanodeInfo <datanode_host:ipc_port>]");
     } else {
-      System.err.println("Usage: java DFSAdmin");
+      System.err.println("Usage: hdfs dfsadmin");
       System.err.println("Note: Administrative commands can only be run as the HDFS superuser.");
-      System.err.println("           [-report]");
-      System.err.println("           [-safemode enter | leave | get | wait]"); 
-      System.err.println("           [-allowSnapshot <snapshotDir>]");
-      System.err.println("           [-disallowSnapshot <snapshotDir>]");
-      System.err.println("           [-saveNamespace]");
-      System.err.println("           [-rollEdits]");
-      System.err.println("           [-restoreFailedStorage true|false|check]");
-      System.err.println("           [-refreshNodes]");
-      System.err.println("           [-finalizeUpgrade]");
-      System.err.println("           ["+RollingUpgradeCommand.USAGE+"]");
-      System.err.println("           [-metasave filename]");
-      System.err.println("           [-refreshServiceAcl]");
-      System.err.println("           [-refreshUserToGroupsMappings]");
-      System.err.println("           [-refreshSuperUserGroupsConfiguration]");
-      System.err.println("           [-refreshCallQueue]");
-      System.err.println("           [-refresh]");
-      System.err.println("           [-printTopology]");
-      System.err.println("           [-refreshNamenodes datanodehost:port]");
-      System.err.println("           [-deleteBlockPool datanode-host:port blockpoolId [force]]");
-      System.err.println("           ["+SetQuotaCommand.USAGE+"]");
-      System.err.println("           ["+ClearQuotaCommand.USAGE+"]");
-      System.err.println("           ["+SetSpaceQuotaCommand.USAGE+"]");
-      System.err.println("           ["+ClearSpaceQuotaCommand.USAGE+"]");      
-      System.err.println("           [-setBalancerBandwidth <bandwidth in bytes per second>]");
-      System.err.println("           [-fetchImage <local directory>]");
-      System.err.println("           [-shutdownDatanode <datanode_host:ipc_port> [upgrade]]");
-      System.err.println("           [-getDatanodeInfo <datanode_host:ipc_port>]");
-      System.err.println("           [-help [cmd]]");
-      System.err.println();
+      System.err.println(commonUsageSummary);
       ToolRunner.printGenericCommandUsage(System.err);
     }
   }

+ 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

@@ -1125,6 +1125,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>
@@ -2108,4 +2118,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);

+ 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) {
+    }
+  }
 }

+ 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();
+  }
 }

+ 12 - 12
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestHDFSServerPorts.java

@@ -17,14 +17,6 @@
  */
 package org.apache.hadoop.hdfs;
 
-import static org.apache.hadoop.hdfs.server.common.Util.fileAsURI;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-import java.io.File;
-import java.io.IOException;
-import java.net.UnknownHostException;
-
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -39,6 +31,14 @@ import org.apache.hadoop.net.DNS;
 import org.apache.hadoop.test.PathUtils;
 import org.junit.Test;
 
+import java.io.File;
+import java.io.IOException;
+import java.net.UnknownHostException;
+
+import static org.apache.hadoop.hdfs.server.common.Util.fileAsURI;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
 /**
  * This test checks correctness of port usage by hdfs components:
  * NameNode, DataNode, SecondaryNamenode and BackupNode.
@@ -245,7 +245,7 @@ public class TestHDFSServerPorts {
     return true;
   }
 
-  @Test
+  @Test(timeout = 300000)
   public void testNameNodePorts() throws Exception {
     runTestNameNodePorts(false);
     runTestNameNodePorts(true);
@@ -296,7 +296,7 @@ public class TestHDFSServerPorts {
   /**
    * Verify datanode port usage.
    */
-  @Test
+  @Test(timeout = 300000)
   public void testDataNodePorts() throws Exception {
     NameNode nn = null;
     try {
@@ -332,7 +332,7 @@ public class TestHDFSServerPorts {
   /**
    * Verify secondary namenode port usage.
    */
-  @Test
+  @Test(timeout = 300000)
   public void testSecondaryNodePorts() throws Exception {
     NameNode nn = null;
     try {
@@ -361,7 +361,7 @@ public class TestHDFSServerPorts {
     /**
      * Verify BackupNode port usage.
      */
-  @Test
+    @Test(timeout = 300000)
     public void testBackupNodePorts() throws Exception {
       NameNode nn = null;
       try {

+ 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, false);
       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, false);
         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();
       }

+ 8 - 8
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestValidateConfigurationSettings.java

@@ -17,11 +17,6 @@
  */
 package org.apache.hadoop.hdfs.server.namenode;
 
-import java.io.File;
-import java.io.IOException;
-import java.net.BindException;
-import java.util.Random;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
@@ -33,6 +28,11 @@ import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.After;
 import org.junit.Test;
 
+import java.io.File;
+import java.io.IOException;
+import java.net.BindException;
+import java.util.Random;
+
 /**
  * This class tests the validation of the configuration object when passed 
  * to the NameNode
@@ -49,7 +49,7 @@ public class TestValidateConfigurationSettings {
    * an exception
    * is thrown when trying to re-use the same port
    */
-  @Test(expected = BindException.class)
+  @Test(expected = BindException.class, timeout = 300000)
   public void testThatMatchingRPCandHttpPortsThrowException() 
       throws IOException {
 
@@ -79,7 +79,7 @@ public class TestValidateConfigurationSettings {
    * Tests setting the rpc port to a different as the web port that an 
    * exception is NOT thrown 
    */
-  @Test
+  @Test(timeout = 300000)
   public void testThatDifferentRPCandHttpPortsAreOK() 
       throws IOException {
 
@@ -117,7 +117,7 @@ public class TestValidateConfigurationSettings {
    * HDFS-3013: NameNode format command doesn't pick up
    * dfs.namenode.name.dir.NameServiceId configuration.
    */
-  @Test
+  @Test(timeout = 300000)
   public void testGenericKeysForNameNodeFormat()
       throws IOException {
     Configuration conf = new HdfsConfiguration();

+ 26 - 37
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestDelegationTokensWithHA.java

@@ -17,27 +17,7 @@
  */
 package org.apache.hadoop.hdfs.server.namenode.ha;
 
-import static org.junit.Assert.assertArrayEquals;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-import static org.mockito.Mockito.mock;
-
-import java.io.ByteArrayInputStream;
-import java.io.DataInputStream;
-import java.io.IOException;
-import java.net.InetSocketAddress;
-import java.net.URI;
-import java.security.PrivilegedExceptionAction;
-import java.util.Collection;
-import java.util.HashSet;
-import java.util.Map;
-
-import javax.servlet.http.HttpServletResponse;
-import javax.ws.rs.core.Response;
-
+import com.google.common.base.Joiner;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -45,11 +25,7 @@ import org.apache.hadoop.fs.AbstractFileSystem;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
-import org.apache.hadoop.hdfs.DistributedFileSystem;
-import org.apache.hadoop.hdfs.HAUtil;
-import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.hadoop.hdfs.MiniDFSNNTopology;
+import org.apache.hadoop.hdfs.*;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSecretManager;
@@ -75,7 +51,20 @@ import org.junit.Test;
 import org.mockito.internal.util.reflection.Whitebox;
 import org.mortbay.util.ajax.JSON;
 
-import com.google.common.base.Joiner;
+import javax.servlet.http.HttpServletResponse;
+import javax.ws.rs.core.Response;
+import java.io.ByteArrayInputStream;
+import java.io.DataInputStream;
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.net.URI;
+import java.security.PrivilegedExceptionAction;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Map;
+
+import static org.junit.Assert.*;
+import static org.mockito.Mockito.mock;
 
 /**
  * Test case for client support of delegation tokens in an HA cluster.
@@ -128,8 +117,8 @@ public class TestDelegationTokensWithHA {
       cluster.shutdown();
     }
   }
-  
-  @Test
+
+  @Test(timeout = 300000)
   public void testDelegationTokenDFSApi() throws Exception {
     final Token<DelegationTokenIdentifier> token =
         getDelegationToken(fs, "JobTracker");
@@ -192,7 +181,7 @@ public class TestDelegationTokensWithHA {
    * Test if correct exception (StandbyException or RetriableException) can be
    * thrown during the NN failover. 
    */
-  @Test
+  @Test(timeout = 300000)
   public void testDelegationTokenDuringNNFailover() throws Exception {
     EditLogTailer editLogTailer = nn1.getNamesystem().getEditLogTailer();
     // stop the editLogTailer of nn1
@@ -260,7 +249,7 @@ public class TestDelegationTokensWithHA {
     doRenewOrCancel(token, clientConf, TokenTestAction.CANCEL);
   }
 
-  @Test
+  @Test(timeout = 300000)
   public void testDelegationTokenWithDoAs() throws Exception {
     final Token<DelegationTokenIdentifier> token =
         getDelegationToken(fs, "JobTracker");
@@ -291,8 +280,8 @@ public class TestDelegationTokensWithHA {
       }
     });
   }
-  
-  @Test
+
+  @Test(timeout = 300000)
   public void testHAUtilClonesDelegationTokens() throws Exception {
     final Token<DelegationTokenIdentifier> token =
         getDelegationToken(fs, "JobTracker");
@@ -354,7 +343,7 @@ public class TestDelegationTokensWithHA {
    * exception if the URI is a logical URI. This bug fails the combination of
    * ha + mapred + security.
    */
-  @Test
+  @Test(timeout = 300000)
   public void testDFSGetCanonicalServiceName() throws Exception {
     URI hAUri = HATestUtil.getLogicalUri(cluster);
     String haService = HAUtil.buildTokenServiceForLogicalUri(hAUri,
@@ -368,8 +357,8 @@ public class TestDelegationTokensWithHA {
     token.renew(dfs.getConf());
     token.cancel(dfs.getConf());
   }
-  
-  @Test
+
+  @Test(timeout = 300000)
   public void testHdfsGetCanonicalServiceName() throws Exception {
     Configuration conf = dfs.getConf();
     URI haUri = HATestUtil.getLogicalUri(cluster);
@@ -390,7 +379,7 @@ public class TestDelegationTokensWithHA {
    * password. (HDFS-6475). With StandbyException, the client can failover to try
    * activeNN.
    */
-  @Test
+  @Test(timeout = 300000)
   public void testDelegationTokenStandbyNNAppearFirst() throws Exception {
     // make nn0 the standby NN, and nn1 the active NN
     cluster.transitionToStandby(0);

+ 5 - 5
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestHAMetrics.java

@@ -17,9 +17,6 @@
  */
 package org.apache.hadoop.hdfs.server.namenode.ha;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -33,14 +30,17 @@ import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.io.IOUtils;
 import org.junit.Test;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
 /**
  * Make sure HA-related metrics are updated and reported appropriately.
  */
 public class TestHAMetrics {
   
   private static final Log LOG = LogFactory.getLog(TestHAMetrics.class);
-  
-  @Test
+
+  @Test(timeout = 300000)
   public void testHAMetrics() throws Exception {
     Configuration conf = new Configuration();
     conf.setInt(DFSConfigKeys.DFS_HA_TAILEDITS_PERIOD_KEY, 1);

+ 21 - 31
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestHAStateTransitions.java

@@ -17,20 +17,7 @@
  */
 package org.apache.hadoop.hdfs.server.namenode.ha;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-import java.io.DataOutputStream;
-import java.io.File;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.net.URI;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
-
+import com.google.common.util.concurrent.Uninterruptibles;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.impl.Log4JLogger;
@@ -40,13 +27,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.ha.HAServiceProtocol.RequestSource;
 import org.apache.hadoop.ha.HAServiceProtocol.StateChangeRequestInfo;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
-import org.apache.hadoop.hdfs.DFSTestUtil;
-import org.apache.hadoop.hdfs.DFSUtil;
-import org.apache.hadoop.hdfs.HAUtil;
-import org.apache.hadoop.hdfs.HdfsConfiguration;
-import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.hadoop.hdfs.MiniDFSNNTopology;
+import org.apache.hadoop.hdfs.*;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
@@ -66,7 +47,16 @@ import org.junit.Assert;
 import org.junit.Test;
 import org.mockito.Mockito;
 
-import com.google.common.util.concurrent.Uninterruptibles;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.net.URI;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+import static org.junit.Assert.*;
 
 /**
  * Tests state transition from active->standby, and manual failover
@@ -92,7 +82,7 @@ public class TestHAStateTransitions {
    * active and standby mode, making sure it doesn't
    * double-play any edits.
    */
-  @Test
+  @Test(timeout = 300000)
   public void testTransitionActiveToStandby() throws Exception {
     Configuration conf = new Configuration();
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
@@ -148,7 +138,7 @@ public class TestHAStateTransitions {
    * Test that transitioning a service to the state that it is already
    * in is a nop, specifically, an exception is not thrown.
    */
-  @Test
+  @Test(timeout = 300000)
   public void testTransitionToCurrentStateIsANop() throws Exception {
     Configuration conf = new Configuration();
     conf.setLong(DFSConfigKeys.DFS_NAMENODE_PATH_BASED_CACHE_REFRESH_INTERVAL_MS, 1L);
@@ -220,7 +210,7 @@ public class TestHAStateTransitions {
   /**
    * Tests manual failover back and forth between two NameNodes.
    */
-  @Test
+  @Test(timeout = 300000)
   public void testManualFailoverAndFailback() throws Exception {
     Configuration conf = new Configuration();
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
@@ -346,7 +336,7 @@ public class TestHAStateTransitions {
   /**
    * Test that delegation tokens continue to work after the failover.
    */
-  @Test
+  @Test(timeout = 300000)
   public void testDelegationTokensAfterFailover() throws IOException {
     Configuration conf = new Configuration();
     conf.setBoolean(
@@ -383,7 +373,7 @@ public class TestHAStateTransitions {
    * Tests manual failover back and forth between two NameNodes
    * for federation cluster with two namespaces.
    */
-  @Test
+  @Test(timeout = 300000)
   public void testManualFailoverFailbackFederationHA() throws Exception {
     Configuration conf = new Configuration();
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
@@ -403,12 +393,12 @@ public class TestHAStateTransitions {
     }
   }
 
-  @Test
+  @Test(timeout = 300000)
   public void testFailoverWithEmptyInProgressEditLog() throws Exception {
     testFailoverAfterCrashDuringLogRoll(false);
   }
-  
-  @Test
+
+  @Test(timeout = 300000)
   public void testFailoverWithEmptyInProgressEditLogWithHeader()
       throws Exception {
     testFailoverAfterCrashDuringLogRoll(true);
@@ -570,7 +560,7 @@ public class TestHAStateTransitions {
    * by virtue of the fact that it wouldn't work properly if the proxies
    * returned were not for the correct NNs.
    */
-  @Test
+  @Test(timeout = 300000)
   public void testIsAtLeastOneActive() throws Exception {
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(new HdfsConfiguration())
         .nnTopology(MiniDFSNNTopology.simpleHATopology())

+ 22 - 32
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestStandbyCheckpoints.java

@@ -17,23 +17,11 @@
  */
 package org.apache.hadoop.hdfs.server.namenode.ha;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-import java.io.File;
-import java.io.IOException;
-import java.io.OutputStream;
-import java.net.BindException;
-import java.lang.management.ManagementFactory;
-import java.lang.management.ThreadInfo;
-import java.lang.management.ThreadMXBean;
-import java.net.URI;
-import java.net.URL;
-import java.util.List;
-import java.util.Random;
-
+import com.google.common.base.Supplier;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Lists;
+import com.google.common.io.Files;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -43,14 +31,8 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSNNTopology;
-import org.apache.hadoop.hdfs.server.namenode.FSImage;
-import org.apache.hadoop.hdfs.server.namenode.FSImageTestUtil;
-import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
-import org.apache.hadoop.hdfs.server.namenode.JournalSet;
-import org.apache.hadoop.hdfs.server.namenode.NNStorage;
+import org.apache.hadoop.hdfs.server.namenode.*;
 import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeFile;
-import org.apache.hadoop.hdfs.server.namenode.NameNode;
-import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
 import org.apache.hadoop.hdfs.util.Canceler;
 import org.apache.hadoop.io.compress.CompressionCodecFactory;
 import org.apache.hadoop.io.compress.CompressionOutputStream;
@@ -64,11 +46,19 @@ import org.junit.Before;
 import org.junit.Test;
 import org.mockito.Mockito;
 
-import com.google.common.base.Supplier;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.Lists;
-import com.google.common.io.Files;
+import java.io.File;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.lang.management.ManagementFactory;
+import java.lang.management.ThreadInfo;
+import java.lang.management.ThreadMXBean;
+import java.net.BindException;
+import java.net.URI;
+import java.net.URL;
+import java.util.List;
+import java.util.Random;
+
+import static org.junit.Assert.*;
 
 public class TestStandbyCheckpoints {
   private static final int NUM_DIRS_IN_LOG = 200000;
@@ -143,7 +133,7 @@ public class TestStandbyCheckpoints {
     }
   }
 
-  @Test
+  @Test(timeout = 300000)
   public void testSBNCheckpoints() throws Exception {
     JournalSet standbyJournalSet = NameNodeAdapter.spyOnJournalSet(nn1);
     
@@ -185,7 +175,7 @@ public class TestStandbyCheckpoints {
    * checkpoint for the given txid, but this should not cause
    * an abort, etc.
    */
-  @Test
+  @Test(timeout = 300000)
   public void testBothNodesInStandbyState() throws Exception {
     doEdits(0, 10);
     
@@ -216,7 +206,7 @@ public class TestStandbyCheckpoints {
    * same txid, which is a no-op. This test makes sure this doesn't
    * cause any problem.
    */
-  @Test
+  @Test(timeout = 300000)
   public void testCheckpointWhenNoNewTransactionsHappened()
       throws Exception {
     // Checkpoint as fast as we can, in a tight loop.

+ 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

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

@@ -90,7 +90,7 @@ public class TestTools {
       fail("testDFSAdminHelp error" + e);
     }
 
-    String pattern = "Usage: java DFSAdmin";
+    String pattern = "Usage: hdfs dfsadmin";
     checkOutput(new String[] { "-cancel", "-renew" }, pattern, System.err,
         DFSAdmin.class);
   }

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

+ 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

@@ -175,6 +175,15 @@ 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)
+
+    YARN-2508. Cross Origin configuration parameters prefix are not honored
+    (Mit Desai via jeagles)
+
   OPTIMIZATIONS
 
   BUG FIXES
@@ -275,6 +284,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

+ 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);
   }

+ 11 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/webapp/CrossOriginFilterInitializer.java

@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.yarn.server.timeline.webapp;
 
+import java.util.HashMap;
 import java.util.Map;
 
 import org.apache.hadoop.conf.Configuration;
@@ -37,6 +38,15 @@ public class CrossOriginFilterInitializer extends FilterInitializer {
   }
 
   static Map<String, String> getFilterParameters(Configuration conf) {
-    return conf.getValByRegex(PREFIX);
+    Map<String, String> filterParams =
+        new HashMap<String, String>();
+    for (Map.Entry<String, String> entry : conf.getValByRegex(PREFIX)
+        .entrySet()) {
+      String name = entry.getKey();
+      String value = entry.getValue();
+      name = name.substring(PREFIX.length());
+      filterParams.put(name, value);
+    }
+    return filterParams;
   }
 }

+ 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 {
 

+ 2 - 5
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/webapp/TestCrossOriginFilterInitializer.java

@@ -42,11 +42,8 @@ public class TestCrossOriginFilterInitializer {
         CrossOriginFilterInitializer.getFilterParameters(conf);
 
     // retrieve values
-    String rootvalue =
-        filterParameters.get(CrossOriginFilterInitializer.PREFIX + "rootparam");
-    String nestedvalue =
-        filterParameters.get(CrossOriginFilterInitializer.PREFIX
-            + "nested.param");
+    String rootvalue = filterParameters.get("rootparam");
+    String nestedvalue = filterParameters.get("nested.param");
     String outofscopeparam = filterParameters.get("outofscopeparam");
 
     // verify expected values are in place

+ 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;
+    }
+  }
 }

+ 1 - 0
pom.xml

@@ -324,6 +324,7 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xs
             <exclude>.gitignore</exclude>
             <exclude>.git/**</exclude>
             <exclude>.idea/**</exclude>
+	    <exclude>**/build/**</exclude>
          </excludes>
        </configuration>
       </plugin>