Browse Source

Merge branch 'trunk' into HADOOP-13345

Steve Loughran 7 years ago
parent
commit
11e5f54fee
100 changed files with 3358 additions and 712 deletions
  1. 2 2
      BUILDING.txt
  2. 14 3
      dev-support/docker/Dockerfile
  3. 148 36
      hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/client/AuthenticatedURL.java
  4. 13 17
      hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/client/KerberosAuthenticator.java
  5. 1 4
      hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/client/PseudoAuthenticator.java
  6. 16 10
      hadoop-common-project/hadoop-common/HadoopCommon.cmake
  7. 1 1
      hadoop-common-project/hadoop-common/src/CMakeLists.txt
  8. 11 0
      hadoop-common-project/hadoop-common/src/main/conf/hadoop-policy.xml
  9. 67 4
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java
  10. 126 21
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProviderCryptoExtension.java
  11. 54 104
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/KMSClientProvider.java
  12. 1 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/KMSRESTConstants.java
  13. 20 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/LoadBalancingKMSClientProvider.java
  14. 58 16
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSDataOutputStreamBuilder.java
  15. 9 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FsServerDefaults.java
  16. 4 2
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/LocatedFileStatus.java
  17. 1 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/permission/FsPermission.java
  18. 6 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/sftp/SFTPFileSystem.java
  19. 3 3
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/AclCommands.java
  20. 2 2
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/CommandWithDestination.java
  21. 2 2
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Ls.java
  22. 8 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpServer2.java
  23. 1 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/CodecRegistry.java
  24. 8 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ErasureCodeConstants.java
  25. 7 7
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java
  26. 134 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/KMSUtil.java
  27. 135 7
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/curator/ZKCuratorManager.java
  28. 2 0
      hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/compress/bzip2/org_apache_hadoop_io_compress_bzip2.h
  29. 9 1
      hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
  30. 12 12
      hadoop-common-project/hadoop-common/src/site/markdown/Metrics.md
  31. 31 2
      hadoop-common-project/hadoop-common/src/site/markdown/filesystem/filesystem.md
  32. 182 0
      hadoop-common-project/hadoop-common/src/site/markdown/filesystem/fsdataoutputstreambuilder.md
  33. 1 0
      hadoop-common-project/hadoop-common/src/site/markdown/filesystem/index.md
  34. 1 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestCommonConfigurationFields.java
  35. 183 114
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfiguration.java
  36. 6 2
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfigurationSubclass.java
  37. 4 4
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestDeprecatedKeys.java
  38. 5 3
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestGetInstances.java
  39. 88 25
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/key/TestKeyProviderCryptoExtension.java
  40. 4 2
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestAvroFSInput.java
  41. 13 6
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestDU.java
  42. 5 3
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFilterFs.java
  43. 13 12
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestGetFileBlockLocations.java
  44. 5 2
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestGlobExpander.java
  45. 52 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestLocatedFileStatus.java
  46. 13 11
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestTrash.java
  47. 4 2
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestTruncatedInputBug.java
  48. 12 2
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/permission/TestFsPermission.java
  49. 13 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpServer.java
  50. 241 1
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpServerWithSpengo.java
  51. 29 4
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestFairCallQueue.java
  52. 3 3
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/log/TestLog4Json.java
  53. 3 6
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestScriptBasedMapping.java
  54. 3 4
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestScriptBasedMappingWithDependency.java
  55. 4 2
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestAuthenticationFilter.java
  56. 4 3
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestAuthenticationWithProxyUserFilter.java
  57. 6 3
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestWhitelistBasedResolver.java
  58. 8 4
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/TestToken.java
  59. 2 4
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestAsyncDiskService.java
  60. 7 6
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestCacheableIPList.java
  61. 7 5
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestFileBasedIPList.java
  62. 1 1
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestFindClass.java
  63. 10 2
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestGenericsUtil.java
  64. 5 2
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestIndexedSort.java
  65. 4 4
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestNativeLibraryChecker.java
  66. 40 1
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/curator/TestZKCuratorManager.java
  67. 6 0
      hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/EagerKeyGeneratorKeyProviderCryptoExtension.java
  68. 90 23
      hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMS.java
  69. 5 3
      hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSJSONReader.java
  70. 2 32
      hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSServerJSONUtils.java
  71. 18 0
      hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSWebApp.java
  72. 19 0
      hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KeyAuthorizationKeyProvider.java
  73. 59 1
      hadoop-common-project/hadoop-kms/src/site/markdown/index.md.vm
  74. 45 1
      hadoop-common-project/hadoop-kms/src/test/java/org/apache/hadoop/crypto/key/kms/server/TestKMS.java
  75. 6 1
      hadoop-common-project/hadoop-kms/src/test/java/org/apache/hadoop/crypto/key/kms/server/TestKMSAudit.java
  76. 23 2
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
  77. 2 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClientFaultInjector.java
  78. 95 58
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
  79. 29 29
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java
  80. 2 1
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java
  81. 62 26
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
  82. 38 8
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsAdmin.java
  83. 1 1
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java
  84. 31 4
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
  85. 49 6
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ErasureCodingPolicy.java
  86. 73 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ErasureCodingPolicyState.java
  87. 7 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java
  88. 216 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ReencryptionStatus.java
  89. 58 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ReencryptionStatusIterator.java
  90. 16 2
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/SystemErasureCodingPolicies.java
  91. 257 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ZoneReencryptionStatus.java
  92. 42 2
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
  93. 145 9
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
  94. 4 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto
  95. 41 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/encryption.proto
  96. 24 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/hdfs.proto
  97. 1 0
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/BaseTestHttpFSWith.java
  98. 1 1
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/CMakeLists.txt
  99. 2 0
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/fuse-dfs/CMakeLists.txt
  100. 2 0
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs-tests/CMakeLists.txt

+ 2 - 2
BUILDING.txt

@@ -7,7 +7,7 @@ Requirements:
 * JDK 1.8+
 * JDK 1.8+
 * Maven 3.3 or later
 * Maven 3.3 or later
 * ProtocolBuffer 2.5.0
 * ProtocolBuffer 2.5.0
-* CMake 2.6 or newer (if compiling native code), must be 3.0 or newer on Mac
+* CMake 3.1 or newer (if compiling native code)
 * Zlib devel (if compiling native code)
 * Zlib devel (if compiling native code)
 * openssl devel (if compiling native hadoop-pipes and to get the best HDFS encryption performance)
 * openssl devel (if compiling native hadoop-pipes and to get the best HDFS encryption performance)
 * Linux FUSE (Filesystem in Userspace) version 2.6 or above (if compiling fuse_dfs)
 * Linux FUSE (Filesystem in Userspace) version 2.6 or above (if compiling fuse_dfs)
@@ -345,7 +345,7 @@ Requirements:
 * JDK 1.8+
 * JDK 1.8+
 * Maven 3.0 or later
 * Maven 3.0 or later
 * ProtocolBuffer 2.5.0
 * ProtocolBuffer 2.5.0
-* CMake 2.6 or newer
+* CMake 3.1 or newer
 * Windows SDK 7.1 or Visual Studio 2010 Professional
 * Windows SDK 7.1 or Visual Studio 2010 Professional
 * Windows SDK 8.1 (if building CPU rate control for the container executor)
 * Windows SDK 8.1 (if building CPU rate control for the container executor)
 * zlib headers (if building native code bindings for zlib)
 * zlib headers (if building native code bindings for zlib)

+ 14 - 3
dev-support/docker/Dockerfile

@@ -35,7 +35,6 @@ ENV DEBCONF_TERSE true
 RUN apt-get -q update && apt-get -q install --no-install-recommends -y \
 RUN apt-get -q update && apt-get -q install --no-install-recommends -y \
     build-essential \
     build-essential \
     bzip2 \
     bzip2 \
-    cmake \
     curl \
     curl \
     doxygen \
     doxygen \
     fuse \
     fuse \
@@ -89,11 +88,22 @@ RUN apt-get -q update && apt-get -q install --no-install-recommends -y ant
 ######
 ######
 RUN mkdir -p /opt/maven && \
 RUN mkdir -p /opt/maven && \
     curl -L -s -S \
     curl -L -s -S \
-         http://www-us.apache.org/dist/maven/maven-3/3.3.9/binaries/apache-maven-3.3.9-bin.tar.gz \
+         https://www-us.apache.org/dist/maven/maven-3/3.3.9/binaries/apache-maven-3.3.9-bin.tar.gz \
          -o /opt/maven.tar.gz && \
          -o /opt/maven.tar.gz && \
     tar xzf /opt/maven.tar.gz --strip-components 1 -C /opt/maven
     tar xzf /opt/maven.tar.gz --strip-components 1 -C /opt/maven
 ENV MAVEN_HOME /opt/maven
 ENV MAVEN_HOME /opt/maven
-ENV PATH "$PATH:/opt/maven/bin"
+ENV PATH "${PATH}:/opt/maven/bin"
+
+######
+# Install cmake
+######
+RUN mkdir -p /opt/cmake && \
+    curl -L -s -S \
+         https://cmake.org/files/v3.1/cmake-3.1.0-Linux-x86_64.tar.gz \
+         -o /opt/cmake.tar.gz && \
+    tar xzf /opt/cmake.tar.gz --strip-components 1 -C /opt/cmake
+ENV CMAKE_HOME /opt/cmake
+ENV PATH "${PATH}:/opt/cmake/bin"
 
 
 ######
 ######
 # Install findbugs
 # Install findbugs
@@ -104,6 +114,7 @@ RUN mkdir -p /opt/findbugs && \
          -o /opt/findbugs.tar.gz && \
          -o /opt/findbugs.tar.gz && \
     tar xzf /opt/findbugs.tar.gz --strip-components 1 -C /opt/findbugs
     tar xzf /opt/findbugs.tar.gz --strip-components 1 -C /opt/findbugs
 ENV FINDBUGS_HOME /opt/findbugs
 ENV FINDBUGS_HOME /opt/findbugs
+ENV PATH "${PATH}:/opt/findbugs/bin"
 
 
 ####
 ####
 # Install shellcheck
 # Install shellcheck

+ 148 - 36
hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/client/AuthenticatedURL.java

@@ -19,8 +19,14 @@ import org.slf4j.LoggerFactory;
 
 
 import java.io.FileNotFoundException;
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.IOException;
+import java.net.CookieHandler;
+import java.net.HttpCookie;
 import java.net.HttpURLConnection;
 import java.net.HttpURLConnection;
+import java.net.URI;
 import java.net.URL;
 import java.net.URL;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
 import java.util.List;
 import java.util.List;
 import java.util.Map;
 import java.util.Map;
 
 
@@ -69,14 +75,99 @@ public class AuthenticatedURL {
    */
    */
   public static final String AUTH_COOKIE = "hadoop.auth";
   public static final String AUTH_COOKIE = "hadoop.auth";
 
 
-  private static final String AUTH_COOKIE_EQ = AUTH_COOKIE + "=";
+  // a lightweight cookie handler that will be attached to url connections.
+  // client code is not required to extract or inject auth cookies.
+  private static class AuthCookieHandler extends CookieHandler {
+    private HttpCookie authCookie;
+    private Map<String, List<String>> cookieHeaders = Collections.emptyMap();
+
+    @Override
+    public synchronized Map<String, List<String>> get(URI uri,
+        Map<String, List<String>> requestHeaders) throws IOException {
+      // call getter so it will reset headers if token is expiring.
+      getAuthCookie();
+      return cookieHeaders;
+    }
+
+    @Override
+    public void put(URI uri, Map<String, List<String>> responseHeaders) {
+      List<String> headers = responseHeaders.get("Set-Cookie");
+      if (headers != null) {
+        for (String header : headers) {
+          List<HttpCookie> cookies;
+          try {
+            cookies = HttpCookie.parse(header);
+          } catch (IllegalArgumentException iae) {
+            // don't care. just skip malformed cookie headers.
+            LOG.debug("Cannot parse cookie header: " + header, iae);
+            continue;
+          }
+          for (HttpCookie cookie : cookies) {
+            if (AUTH_COOKIE.equals(cookie.getName())) {
+              setAuthCookie(cookie);
+            }
+          }
+        }
+      }
+    }
+
+    // return the auth cookie if still valid.
+    private synchronized HttpCookie getAuthCookie() {
+      if (authCookie != null && authCookie.hasExpired()) {
+        setAuthCookie(null);
+      }
+      return authCookie;
+    }
+
+    private synchronized void setAuthCookie(HttpCookie cookie) {
+      final HttpCookie oldCookie = authCookie;
+      // will redefine if new cookie is valid.
+      authCookie = null;
+      cookieHeaders = Collections.emptyMap();
+      boolean valid = cookie != null && !cookie.getValue().isEmpty() &&
+          !cookie.hasExpired();
+      if (valid) {
+        // decrease lifetime to avoid using a cookie soon to expire.
+        // allows authenticators to pre-emptively reauthenticate to
+        // prevent clients unnecessarily receiving a 401.
+        long maxAge = cookie.getMaxAge();
+        if (maxAge != -1) {
+          cookie.setMaxAge(maxAge * 9/10);
+          valid = !cookie.hasExpired();
+        }
+      }
+      if (valid) {
+        // v0 cookies value aren't quoted by default but tomcat demands
+        // quoting.
+        if (cookie.getVersion() == 0) {
+          String value = cookie.getValue();
+          if (!value.startsWith("\"")) {
+            value = "\"" + value + "\"";
+            cookie.setValue(value);
+          }
+        }
+        authCookie = cookie;
+        cookieHeaders = new HashMap<>();
+        cookieHeaders.put("Cookie", Arrays.asList(cookie.toString()));
+      }
+      LOG.trace("Setting token value to {} ({})", authCookie, oldCookie);
+    }
+
+    private void setAuthCookieValue(String value) {
+      HttpCookie c = null;
+      if (value != null) {
+        c = new HttpCookie(AUTH_COOKIE, value);
+      }
+      setAuthCookie(c);
+    }
+  }
 
 
   /**
   /**
    * Client side authentication token.
    * Client side authentication token.
    */
    */
   public static class Token {
   public static class Token {
 
 
-    private String token;
+    private final AuthCookieHandler cookieHandler = new AuthCookieHandler();
 
 
     /**
     /**
      * Creates a token.
      * Creates a token.
@@ -102,7 +193,7 @@ public class AuthenticatedURL {
      * @return if a token from the server has been set.
      * @return if a token from the server has been set.
      */
      */
     public boolean isSet() {
     public boolean isSet() {
-      return token != null;
+      return cookieHandler.getAuthCookie() != null;
     }
     }
 
 
     /**
     /**
@@ -111,7 +202,36 @@ public class AuthenticatedURL {
      * @param tokenStr string representation of the tokenStr.
      * @param tokenStr string representation of the tokenStr.
      */
      */
     void set(String tokenStr) {
     void set(String tokenStr) {
-      token = tokenStr;
+      cookieHandler.setAuthCookieValue(tokenStr);
+    }
+
+    /**
+     * Installs a cookie handler for the http request to manage session
+     * cookies.
+     * @param url
+     * @return HttpUrlConnection
+     * @throws IOException
+     */
+    HttpURLConnection openConnection(URL url,
+        ConnectionConfigurator connConfigurator) throws IOException {
+      // the cookie handler is unfortunately a global static.  it's a
+      // synchronized class method so we can safely swap the handler while
+      // instantiating the connection object to prevent it leaking into
+      // other connections.
+      final HttpURLConnection conn;
+      synchronized(CookieHandler.class) {
+        CookieHandler current = CookieHandler.getDefault();
+        CookieHandler.setDefault(cookieHandler);
+        try {
+          conn = (HttpURLConnection)url.openConnection();
+        } finally {
+          CookieHandler.setDefault(current);
+        }
+      }
+      if (connConfigurator != null) {
+        connConfigurator.configure(conn);
+      }
+      return conn;
     }
     }
 
 
     /**
     /**
@@ -121,7 +241,15 @@ public class AuthenticatedURL {
      */
      */
     @Override
     @Override
     public String toString() {
     public String toString() {
-      return token;
+      String value = "";
+      HttpCookie authCookie = cookieHandler.getAuthCookie();
+      if (authCookie != null) {
+        value = authCookie.getValue();
+        if (value.startsWith("\"")) { // tests don't want the quotes.
+          value = value.substring(1, value.length()-1);
+        }
+      }
+      return value;
     }
     }
 
 
   }
   }
@@ -218,27 +346,25 @@ public class AuthenticatedURL {
       throw new IllegalArgumentException("token cannot be NULL");
       throw new IllegalArgumentException("token cannot be NULL");
     }
     }
     authenticator.authenticate(url, token);
     authenticator.authenticate(url, token);
-    HttpURLConnection conn = (HttpURLConnection) url.openConnection();
-    if (connConfigurator != null) {
-      conn = connConfigurator.configure(conn);
-    }
-    injectToken(conn, token);
-    return conn;
+
+    // allow the token to create the connection with a cookie handler for
+    // managing session cookies.
+    return token.openConnection(url, connConfigurator);
   }
   }
 
 
   /**
   /**
-   * Helper method that injects an authentication token to send with a connection.
+   * Helper method that injects an authentication token to send with a
+   * connection. Callers should prefer using
+   * {@link Token#openConnection(URL, ConnectionConfigurator)} which
+   * automatically manages authentication tokens.
    *
    *
    * @param conn connection to inject the authentication token into.
    * @param conn connection to inject the authentication token into.
    * @param token authentication token to inject.
    * @param token authentication token to inject.
    */
    */
   public static void injectToken(HttpURLConnection conn, Token token) {
   public static void injectToken(HttpURLConnection conn, Token token) {
-    String t = token.token;
-    if (t != null) {
-      if (!t.startsWith("\"")) {
-        t = "\"" + t + "\"";
-      }
-      conn.addRequestProperty("Cookie", AUTH_COOKIE_EQ + t);
+    HttpCookie authCookie = token.cookieHandler.getAuthCookie();
+    if (authCookie != null) {
+      conn.addRequestProperty("Cookie", authCookie.toString());
     }
     }
   }
   }
 
 
@@ -258,24 +384,10 @@ public class AuthenticatedURL {
     if (respCode == HttpURLConnection.HTTP_OK
     if (respCode == HttpURLConnection.HTTP_OK
         || respCode == HttpURLConnection.HTTP_CREATED
         || respCode == HttpURLConnection.HTTP_CREATED
         || respCode == HttpURLConnection.HTTP_ACCEPTED) {
         || respCode == HttpURLConnection.HTTP_ACCEPTED) {
-      Map<String, List<String>> headers = conn.getHeaderFields();
-      List<String> cookies = headers.get("Set-Cookie");
-      if (cookies != null) {
-        for (String cookie : cookies) {
-          if (cookie.startsWith(AUTH_COOKIE_EQ)) {
-            String value = cookie.substring(AUTH_COOKIE_EQ.length());
-            int separator = value.indexOf(";");
-            if (separator > -1) {
-              value = value.substring(0, separator);
-            }
-            if (value.length() > 0) {
-              LOG.trace("Setting token value to {} ({}), resp={}", value,
-                  token, respCode);
-              token.set(value);
-            }
-          }
-        }
-      }
+      // cookie handler should have already extracted the token.  try again
+      // for backwards compatibility if this method is called on a connection
+      // not opened via this instance.
+      token.cookieHandler.put(null, conn.getHeaderFields());
     } else if (respCode == HttpURLConnection.HTTP_NOT_FOUND) {
     } else if (respCode == HttpURLConnection.HTTP_NOT_FOUND) {
       LOG.trace("Setting token value to null ({}), resp={}", token, respCode);
       LOG.trace("Setting token value to null ({}), resp={}", token, respCode);
       token.set(null);
       token.set(null);

+ 13 - 17
hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/client/KerberosAuthenticator.java

@@ -147,7 +147,6 @@ public class KerberosAuthenticator implements Authenticator {
   }
   }
   
   
   private URL url;
   private URL url;
-  private HttpURLConnection conn;
   private Base64 base64;
   private Base64 base64;
   private ConnectionConfigurator connConfigurator;
   private ConnectionConfigurator connConfigurator;
 
 
@@ -182,10 +181,7 @@ public class KerberosAuthenticator implements Authenticator {
     if (!token.isSet()) {
     if (!token.isSet()) {
       this.url = url;
       this.url = url;
       base64 = new Base64(0);
       base64 = new Base64(0);
-      conn = (HttpURLConnection) url.openConnection();
-      if (connConfigurator != null) {
-        conn = connConfigurator.configure(conn);
-      }
+      HttpURLConnection conn = token.openConnection(url, connConfigurator);
       conn.setRequestMethod(AUTH_HTTP_METHOD);
       conn.setRequestMethod(AUTH_HTTP_METHOD);
       conn.connect();
       conn.connect();
       
       
@@ -200,7 +196,7 @@ public class KerberosAuthenticator implements Authenticator {
         }
         }
         needFallback = true;
         needFallback = true;
       }
       }
-      if (!needFallback && isNegotiate()) {
+      if (!needFallback && isNegotiate(conn)) {
         LOG.debug("Performing our own SPNEGO sequence.");
         LOG.debug("Performing our own SPNEGO sequence.");
         doSpnegoSequence(token);
         doSpnegoSequence(token);
       } else {
       } else {
@@ -249,7 +245,7 @@ public class KerberosAuthenticator implements Authenticator {
   /*
   /*
   * Indicates if the response is starting a SPNEGO negotiation.
   * Indicates if the response is starting a SPNEGO negotiation.
   */
   */
-  private boolean isNegotiate() throws IOException {
+  private boolean isNegotiate(HttpURLConnection conn) throws IOException {
     boolean negotiate = false;
     boolean negotiate = false;
     if (conn.getResponseCode() == HttpURLConnection.HTTP_UNAUTHORIZED) {
     if (conn.getResponseCode() == HttpURLConnection.HTTP_UNAUTHORIZED) {
       String authHeader = conn.getHeaderField(WWW_AUTHENTICATE);
       String authHeader = conn.getHeaderField(WWW_AUTHENTICATE);
@@ -267,7 +263,8 @@ public class KerberosAuthenticator implements Authenticator {
    * @throws IOException if an IO error occurred.
    * @throws IOException if an IO error occurred.
    * @throws AuthenticationException if an authentication error occurred.
    * @throws AuthenticationException if an authentication error occurred.
    */
    */
-  private void doSpnegoSequence(AuthenticatedURL.Token token) throws IOException, AuthenticationException {
+  private void doSpnegoSequence(final AuthenticatedURL.Token token)
+      throws IOException, AuthenticationException {
     try {
     try {
       AccessControlContext context = AccessController.getContext();
       AccessControlContext context = AccessController.getContext();
       Subject subject = Subject.getSubject(context);
       Subject subject = Subject.getSubject(context);
@@ -308,13 +305,15 @@ public class KerberosAuthenticator implements Authenticator {
 
 
             // Loop while the context is still not established
             // Loop while the context is still not established
             while (!established) {
             while (!established) {
+              HttpURLConnection conn =
+                  token.openConnection(url, connConfigurator);
               outToken = gssContext.initSecContext(inToken, 0, inToken.length);
               outToken = gssContext.initSecContext(inToken, 0, inToken.length);
               if (outToken != null) {
               if (outToken != null) {
-                sendToken(outToken);
+                sendToken(conn, outToken);
               }
               }
 
 
               if (!gssContext.isEstablished()) {
               if (!gssContext.isEstablished()) {
-                inToken = readToken();
+                inToken = readToken(conn);
               } else {
               } else {
                 established = true;
                 established = true;
               }
               }
@@ -337,18 +336,14 @@ public class KerberosAuthenticator implements Authenticator {
     } catch (LoginException ex) {
     } catch (LoginException ex) {
       throw new AuthenticationException(ex);
       throw new AuthenticationException(ex);
     }
     }
-    AuthenticatedURL.extractToken(conn, token);
   }
   }
 
 
   /*
   /*
   * Sends the Kerberos token to the server.
   * Sends the Kerberos token to the server.
   */
   */
-  private void sendToken(byte[] outToken) throws IOException {
+  private void sendToken(HttpURLConnection conn, byte[] outToken)
+      throws IOException {
     String token = base64.encodeToString(outToken);
     String token = base64.encodeToString(outToken);
-    conn = (HttpURLConnection) url.openConnection();
-    if (connConfigurator != null) {
-      conn = connConfigurator.configure(conn);
-    }
     conn.setRequestMethod(AUTH_HTTP_METHOD);
     conn.setRequestMethod(AUTH_HTTP_METHOD);
     conn.setRequestProperty(AUTHORIZATION, NEGOTIATE + " " + token);
     conn.setRequestProperty(AUTHORIZATION, NEGOTIATE + " " + token);
     conn.connect();
     conn.connect();
@@ -357,7 +352,8 @@ public class KerberosAuthenticator implements Authenticator {
   /*
   /*
   * Retrieves the Kerberos token returned by the server.
   * Retrieves the Kerberos token returned by the server.
   */
   */
-  private byte[] readToken() throws IOException, AuthenticationException {
+  private byte[] readToken(HttpURLConnection conn)
+      throws IOException, AuthenticationException {
     int status = conn.getResponseCode();
     int status = conn.getResponseCode();
     if (status == HttpURLConnection.HTTP_OK || status == HttpURLConnection.HTTP_UNAUTHORIZED) {
     if (status == HttpURLConnection.HTTP_OK || status == HttpURLConnection.HTTP_UNAUTHORIZED) {
       String authHeader = conn.getHeaderField(WWW_AUTHENTICATE);
       String authHeader = conn.getHeaderField(WWW_AUTHENTICATE);

+ 1 - 4
hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/client/PseudoAuthenticator.java

@@ -68,10 +68,7 @@ public class PseudoAuthenticator implements Authenticator {
     String paramSeparator = (strUrl.contains("?")) ? "&" : "?";
     String paramSeparator = (strUrl.contains("?")) ? "&" : "?";
     strUrl += paramSeparator + USER_NAME_EQ + getUserName();
     strUrl += paramSeparator + USER_NAME_EQ + getUserName();
     url = new URL(strUrl);
     url = new URL(strUrl);
-    HttpURLConnection conn = (HttpURLConnection) url.openConnection();
-    if (connConfigurator != null) {
-      conn = connConfigurator.configure(conn);
-    }
+    HttpURLConnection conn = token.openConnection(url, connConfigurator);
     conn.setRequestMethod("OPTIONS");
     conn.setRequestMethod("OPTIONS");
     conn.connect();
     conn.connect();
     AuthenticatedURL.extractToken(conn, token);
     AuthenticatedURL.extractToken(conn, token);

+ 16 - 10
hadoop-common-project/hadoop-common/HadoopCommon.cmake

@@ -117,19 +117,25 @@ macro(hadoop_set_find_shared_library_without_version)
     endif()
     endif()
 endmacro()
 endmacro()
 
 
-#
-# Configuration.
-#
 
 
-# Initialise the shared gcc/g++ flags if they aren't already defined.
-if(NOT DEFINED GCC_SHARED_FLAGS)
-    set(GCC_SHARED_FLAGS "-g -O2 -Wall -pthread -D_FILE_OFFSET_BITS=64")
-endif()
+# set the shared compiler flags
+# support for GNU C/C++, add other compilers as necessary
 
 
-# Add in support other compilers here, if necessary,
-# the assumption is that GCC or a GCC-compatible compiler is being used.
+if (CMAKE_C_COMPILER_ID STREQUAL "GNU")
+  if(NOT DEFINED GCC_SHARED_FLAGS)
+    find_package(Threads REQUIRED)
+    if(CMAKE_USE_PTHREADS_INIT)
+      set(GCC_SHARED_FLAGS "-g -O2 -Wall -pthread -D_FILE_OFFSET_BITS=64")
+    else()
+      set(GCC_SHARED_FLAGS "-g -O2 -Wall -D_FILE_OFFSET_BITS=64")
+    endif()
+  endif()
+elseif (CMAKE_C_COMPILER_ID STREQUAL "Clang" OR
+        CMAKE_C_COMPILER_ID STREQUAL "AppleClang")
+  set(GCC_SHARED_FLAGS "-g -O2 -Wall -D_FILE_OFFSET_BITS=64")
+endif()
 
 
-# Set the shared GCC-compatible compiler and linker flags.
+# Set the shared linker flags.
 hadoop_add_compiler_flags("${GCC_SHARED_FLAGS}")
 hadoop_add_compiler_flags("${GCC_SHARED_FLAGS}")
 hadoop_add_linker_flags("${LINKER_SHARED_FLAGS}")
 hadoop_add_linker_flags("${LINKER_SHARED_FLAGS}")
 
 

+ 1 - 1
hadoop-common-project/hadoop-common/src/CMakeLists.txt

@@ -20,7 +20,7 @@
 # CMake configuration.
 # CMake configuration.
 #
 #
 
 
-cmake_minimum_required(VERSION 2.6 FATAL_ERROR)
+cmake_minimum_required(VERSION 3.1 FATAL_ERROR)
 
 
 list(APPEND CMAKE_MODULE_PATH ${CMAKE_SOURCE_DIR}/..)
 list(APPEND CMAKE_MODULE_PATH ${CMAKE_SOURCE_DIR}/..)
 include(HadoopCommon)
 include(HadoopCommon)

+ 11 - 0
hadoop-common-project/hadoop-common/src/main/conf/hadoop-policy.xml

@@ -223,4 +223,15 @@
     group list is separated by a blank. For e.g. "alice,bob users,wheel".
     group list is separated by a blank. For e.g. "alice,bob users,wheel".
     A special value of "*" means all users are allowed.</description>
     A special value of "*" means all users are allowed.</description>
   </property>
   </property>
+
+  <property>
+    <name>security.collector-nodemanager.protocol.acl</name>
+    <value>*</value>
+    <description>ACL for CollectorNodemanagerProtocol, used by nodemanager
+    if timeline service v2 is enabled, for the timeline collector and nodemanager
+    to communicate with each other.
+    The ACL is a comma-separated list of user and group names. The user and
+    group list is separated by a blank. For e.g. "alice,bob users,wheel".
+    A special value of "*" means all users are allowed.</description>
+  </property>
 </configuration>
 </configuration>

+ 67 - 4
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java

@@ -308,18 +308,25 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
       this.customMessage = customMessage;
       this.customMessage = customMessage;
     }
     }
 
 
+    private final String getWarningMessage(String key) {
+      return getWarningMessage(key, null);
+    }
+
     /**
     /**
      * Method to provide the warning message. It gives the custom message if
      * Method to provide the warning message. It gives the custom message if
      * non-null, and default message otherwise.
      * non-null, and default message otherwise.
      * @param key the associated deprecated key.
      * @param key the associated deprecated key.
+     * @param source the property source.
      * @return message that is to be logged when a deprecated key is used.
      * @return message that is to be logged when a deprecated key is used.
      */
      */
-    private final String getWarningMessage(String key) {
+    private String getWarningMessage(String key, String source) {
       String warningMessage;
       String warningMessage;
       if(customMessage == null) {
       if(customMessage == null) {
         StringBuilder message = new StringBuilder(key);
         StringBuilder message = new StringBuilder(key);
-        String deprecatedKeySuffix = " is deprecated. Instead, use ";
-        message.append(deprecatedKeySuffix);
+        if (source != null) {
+          message.append(" in " + source);
+        }
+        message.append(" is deprecated. Instead, use ");
         for (int i = 0; i < newKeys.length; i++) {
         for (int i = 0; i < newKeys.length; i++) {
           message.append(newKeys[i]);
           message.append(newKeys[i]);
           if(i != newKeys.length-1) {
           if(i != newKeys.length-1) {
@@ -593,6 +600,14 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
     return deprecationContext.get().getDeprecatedKeyMap().containsKey(key);
     return deprecationContext.get().getDeprecatedKeyMap().containsKey(key);
   }
   }
 
 
+  private static String getDeprecatedKey(String key) {
+    return deprecationContext.get().getReverseDeprecatedKeyMap().get(key);
+  }
+
+  private static DeprecatedKeyInfo getDeprecatedKeyInfo(String key) {
+    return deprecationContext.get().getDeprecatedKeyMap().get(key);
+  }
+
   /**
   /**
    * Sets all deprecated properties that are not currently set but have a
    * Sets all deprecated properties that are not currently set but have a
    * corresponding new property that is set. Useful for iterating the
    * corresponding new property that is set. Useful for iterating the
@@ -1270,6 +1285,13 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
     LOG_DEPRECATION.info(message);
     LOG_DEPRECATION.info(message);
   }
   }
 
 
+  void logDeprecationOnce(String name, String source) {
+    DeprecatedKeyInfo keyInfo = getDeprecatedKeyInfo(name);
+    if (keyInfo != null && !keyInfo.getAndSetAccessed()) {
+      LOG_DEPRECATION.info(keyInfo.getWarningMessage(name, source));
+    }
+  }
+
   /**
   /**
    * Unset a previously set property.
    * Unset a previously set property.
    */
    */
@@ -2079,6 +2101,47 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
     return pass;
     return pass;
   }
   }
 
 
+  /**
+   * Get the credential entry by name from a credential provider.
+   *
+   * Handle key deprecation.
+   *
+   * @param provider a credential provider
+   * @param name alias of the credential
+   * @return the credential entry or null if not found
+   */
+  private CredentialEntry getCredentialEntry(CredentialProvider provider,
+                                             String name) throws IOException {
+    CredentialEntry entry = provider.getCredentialEntry(name);
+    if (entry != null) {
+      return entry;
+    }
+
+    // The old name is stored in the credential provider.
+    String oldName = getDeprecatedKey(name);
+    if (oldName != null) {
+      entry = provider.getCredentialEntry(oldName);
+      if (entry != null) {
+        logDeprecationOnce(oldName, provider.toString());
+        return entry;
+      }
+    }
+
+    // The name is deprecated.
+    DeprecatedKeyInfo keyInfo = getDeprecatedKeyInfo(name);
+    if (keyInfo != null && keyInfo.newKeys != null) {
+      for (String newName : keyInfo.newKeys) {
+        entry = provider.getCredentialEntry(newName);
+        if (entry != null) {
+          logDeprecationOnce(name, null);
+          return entry;
+        }
+      }
+    }
+
+    return null;
+  }
+
   /**
   /**
    * Try and resolve the provided element name as a credential provider
    * Try and resolve the provided element name as a credential provider
    * alias.
    * alias.
@@ -2096,7 +2159,7 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
       if (providers != null) {
       if (providers != null) {
         for (CredentialProvider provider : providers) {
         for (CredentialProvider provider : providers) {
           try {
           try {
-            CredentialEntry entry = provider.getCredentialEntry(name);
+            CredentialEntry entry = getCredentialEntry(provider, name);
             if (entry != null) {
             if (entry != null) {
               pass = entry.getCredential();
               pass = entry.getCredential();
               break;
               break;

+ 126 - 21
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProviderCryptoExtension.java

@@ -22,6 +22,8 @@ import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.nio.ByteBuffer;
 import java.security.GeneralSecurityException;
 import java.security.GeneralSecurityException;
 import java.security.SecureRandom;
 import java.security.SecureRandom;
+import java.util.List;
+import java.util.ListIterator;
 
 
 import javax.crypto.Cipher;
 import javax.crypto.Cipher;
 import javax.crypto.spec.IvParameterSpec;
 import javax.crypto.spec.IvParameterSpec;
@@ -247,6 +249,25 @@ public class KeyProviderCryptoExtension extends
      */
      */
     EncryptedKeyVersion reencryptEncryptedKey(EncryptedKeyVersion ekv)
     EncryptedKeyVersion reencryptEncryptedKey(EncryptedKeyVersion ekv)
         throws IOException, GeneralSecurityException;
         throws IOException, GeneralSecurityException;
+
+    /**
+     * Batched version of {@link #reencryptEncryptedKey(EncryptedKeyVersion)}.
+     * <p>
+     * For each encrypted key version, re-encrypts an encrypted key version,
+     * using its initialization vector and key material, but with the latest
+     * key version name of its key name. If the latest key version name in the
+     * provider is the same as the one encrypted the passed-in encrypted key
+     * version, the same encrypted key version is returned.
+     * <p>
+     * NOTE: The generated key is not stored by the <code>KeyProvider</code>
+     *
+     * @param  ekvs List containing the EncryptedKeyVersion's
+     * @throws IOException If any EncryptedKeyVersion could not be re-encrypted
+     * @throws GeneralSecurityException If any EncryptedKeyVersion could not be
+     *                            re-encrypted because of a cryptographic issue.
+     */
+    void reencryptEncryptedKeys(List<EncryptedKeyVersion> ekvs)
+        throws IOException, GeneralSecurityException;
   }
   }
 
 
   private static class DefaultCryptoExtension implements CryptoExtension {
   private static class DefaultCryptoExtension implements CryptoExtension {
@@ -315,7 +336,7 @@ public class KeyProviderCryptoExtension extends
           .checkNotNull(ekNow, "KeyVersion name '%s' does not exist", ekName);
           .checkNotNull(ekNow, "KeyVersion name '%s' does not exist", ekName);
       Preconditions.checkArgument(ekv.getEncryptedKeyVersion().getVersionName()
       Preconditions.checkArgument(ekv.getEncryptedKeyVersion().getVersionName()
               .equals(KeyProviderCryptoExtension.EEK),
               .equals(KeyProviderCryptoExtension.EEK),
-          "encryptedKey version name must be '%s', is '%s'",
+          "encryptedKey version name must be '%s', but found '%s'",
           KeyProviderCryptoExtension.EEK,
           KeyProviderCryptoExtension.EEK,
           ekv.getEncryptedKeyVersion().getVersionName());
           ekv.getEncryptedKeyVersion().getVersionName());
 
 
@@ -336,30 +357,67 @@ public class KeyProviderCryptoExtension extends
     }
     }
 
 
     @Override
     @Override
-    public KeyVersion decryptEncryptedKey(
-        EncryptedKeyVersion encryptedKeyVersion) throws IOException,
-        GeneralSecurityException {
-      // Fetch the encryption key material
-      final String encryptionKeyVersionName =
-          encryptedKeyVersion.getEncryptionKeyVersionName();
-      final KeyVersion encryptionKey =
-          keyProvider.getKeyVersion(encryptionKeyVersionName);
-      Preconditions.checkNotNull(encryptionKey,
-          "KeyVersion name '%s' does not exist", encryptionKeyVersionName);
-      Preconditions.checkArgument(
-              encryptedKeyVersion.getEncryptedKeyVersion().getVersionName()
-                    .equals(KeyProviderCryptoExtension.EEK),
-                "encryptedKey version name must be '%s', is '%s'",
-                KeyProviderCryptoExtension.EEK,
-                encryptedKeyVersion.getEncryptedKeyVersion().getVersionName()
-            );
+    public void reencryptEncryptedKeys(List<EncryptedKeyVersion> ekvs)
+        throws IOException, GeneralSecurityException {
+      Preconditions.checkNotNull(ekvs, "Input list is null");
+      KeyVersion ekNow = null;
+      Decryptor decryptor = null;
+      Encryptor encryptor = null;
+      try (CryptoCodec cc = CryptoCodec.getInstance(keyProvider.getConf())) {
+        decryptor = cc.createDecryptor();
+        encryptor = cc.createEncryptor();
+        ListIterator<EncryptedKeyVersion> iter = ekvs.listIterator();
+        while (iter.hasNext()) {
+          final EncryptedKeyVersion ekv = iter.next();
+          Preconditions.checkNotNull(ekv, "EncryptedKeyVersion is null");
+          final String ekName = ekv.getEncryptionKeyName();
+          Preconditions.checkNotNull(ekName, "Key name is null");
+          Preconditions.checkNotNull(ekv.getEncryptedKeyVersion(),
+              "EncryptedKeyVersion is null");
+          Preconditions.checkArgument(
+              ekv.getEncryptedKeyVersion().getVersionName()
+                  .equals(KeyProviderCryptoExtension.EEK),
+              "encryptedKey version name must be '%s', but found '%s'",
+              KeyProviderCryptoExtension.EEK,
+              ekv.getEncryptedKeyVersion().getVersionName());
+
+          if (ekNow == null) {
+            ekNow = keyProvider.getCurrentKey(ekName);
+            Preconditions
+                .checkNotNull(ekNow, "Key name '%s' does not exist", ekName);
+          } else {
+            Preconditions.checkArgument(ekNow.getName().equals(ekName),
+                "All keys must have the same key name. Expected '%s' "
+                    + "but found '%s'", ekNow.getName(), ekName);
+          }
+
+          final String encryptionKeyVersionName =
+              ekv.getEncryptionKeyVersionName();
+          final KeyVersion encryptionKey =
+              keyProvider.getKeyVersion(encryptionKeyVersionName);
+          Preconditions.checkNotNull(encryptionKey,
+              "KeyVersion name '%s' does not exist", encryptionKeyVersionName);
+          if (encryptionKey.equals(ekNow)) {
+            // no-op if same key version
+            continue;
+          }
+
+          final KeyVersion ek =
+              decryptEncryptedKey(decryptor, encryptionKey, ekv);
+          iter.set(generateEncryptedKey(encryptor, ekNow, ek.getMaterial(),
+              ekv.getEncryptedKeyIv()));
+        }
+      }
+    }
 
 
+    private KeyVersion decryptEncryptedKey(final Decryptor decryptor,
+        final KeyVersion encryptionKey,
+        final EncryptedKeyVersion encryptedKeyVersion)
+        throws IOException, GeneralSecurityException {
       // Encryption key IV is determined from encrypted key's IV
       // Encryption key IV is determined from encrypted key's IV
       final byte[] encryptionIV =
       final byte[] encryptionIV =
           EncryptedKeyVersion.deriveIV(encryptedKeyVersion.getEncryptedKeyIv());
           EncryptedKeyVersion.deriveIV(encryptedKeyVersion.getEncryptedKeyIv());
 
 
-      CryptoCodec cc = CryptoCodec.getInstance(keyProvider.getConf());
-      Decryptor decryptor = cc.createDecryptor();
       decryptor.init(encryptionKey.getMaterial(), encryptionIV);
       decryptor.init(encryptionKey.getMaterial(), encryptionIV);
       final KeyVersion encryptedKV =
       final KeyVersion encryptedKV =
           encryptedKeyVersion.getEncryptedKeyVersion();
           encryptedKeyVersion.getEncryptedKeyVersion();
@@ -372,10 +430,35 @@ public class KeyProviderCryptoExtension extends
       bbOut.flip();
       bbOut.flip();
       byte[] decryptedKey = new byte[keyLen];
       byte[] decryptedKey = new byte[keyLen];
       bbOut.get(decryptedKey);
       bbOut.get(decryptedKey);
-      cc.close();
       return new KeyVersion(encryptionKey.getName(), EK, decryptedKey);
       return new KeyVersion(encryptionKey.getName(), EK, decryptedKey);
     }
     }
 
 
+    @Override
+    public KeyVersion decryptEncryptedKey(
+        EncryptedKeyVersion encryptedKeyVersion)
+        throws IOException, GeneralSecurityException {
+      // Fetch the encryption key material
+      final String encryptionKeyVersionName =
+          encryptedKeyVersion.getEncryptionKeyVersionName();
+      final KeyVersion encryptionKey =
+          keyProvider.getKeyVersion(encryptionKeyVersionName);
+      Preconditions
+          .checkNotNull(encryptionKey, "KeyVersion name '%s' does not exist",
+              encryptionKeyVersionName);
+      Preconditions.checkArgument(
+          encryptedKeyVersion.getEncryptedKeyVersion().getVersionName()
+              .equals(KeyProviderCryptoExtension.EEK),
+          "encryptedKey version name must be '%s', but found '%s'",
+          KeyProviderCryptoExtension.EEK,
+          encryptedKeyVersion.getEncryptedKeyVersion().getVersionName());
+
+      try (CryptoCodec cc = CryptoCodec.getInstance(keyProvider.getConf())) {
+        final Decryptor decryptor = cc.createDecryptor();
+        return decryptEncryptedKey(decryptor, encryptionKey,
+            encryptedKeyVersion);
+      }
+    }
+
     @Override
     @Override
     public void warmUpEncryptedKeys(String... keyNames)
     public void warmUpEncryptedKeys(String... keyNames)
         throws IOException {
         throws IOException {
@@ -470,6 +553,28 @@ public class KeyProviderCryptoExtension extends
     return getExtension().reencryptEncryptedKey(ekv);
     return getExtension().reencryptEncryptedKey(ekv);
   }
   }
 
 
+  /**
+   * Batched version of {@link #reencryptEncryptedKey(EncryptedKeyVersion)}.
+   * <p>
+   * For each encrypted key version, re-encrypts an encrypted key version,
+   * using its initialization vector and key material, but with the latest
+   * key version name of its key name. If the latest key version name in the
+   * provider is the same as the one encrypted the passed-in encrypted key
+   * version, the same encrypted key version is returned.
+   * <p>
+   * NOTE: The generated key is not stored by the <code>KeyProvider</code>
+   *
+   * @param  ekvs List containing the EncryptedKeyVersion's
+   * @return      The re-encrypted EncryptedKeyVersion's, in the same order.
+   * @throws IOException If any EncryptedKeyVersion could not be re-encrypted
+   * @throws GeneralSecurityException If any EncryptedKeyVersion could not be
+   *                            re-encrypted because of a cryptographic issue.
+   */
+  public void reencryptEncryptedKeys(List<EncryptedKeyVersion> ekvs)
+      throws IOException, GeneralSecurityException {
+    getExtension().reencryptEncryptedKeys(ekvs);
+  }
+
   /**
   /**
    * Creates a <code>KeyProviderCryptoExtension</code> using a given
    * Creates a <code>KeyProviderCryptoExtension</code> using a given
    * {@link KeyProvider}.
    * {@link KeyProvider}.

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

@@ -32,8 +32,6 @@ import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.security.ProviderUtils;
 import org.apache.hadoop.security.ProviderUtils;
 import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.security.authentication.client.AuthenticatedURL;
-import org.apache.hadoop.security.authentication.client.AuthenticationException;
 import org.apache.hadoop.security.authentication.client.ConnectionConfigurator;
 import org.apache.hadoop.security.authentication.client.ConnectionConfigurator;
 import org.apache.hadoop.security.ssl.SSLFactory;
 import org.apache.hadoop.security.ssl.SSLFactory;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.Token;
@@ -70,7 +68,6 @@ import java.security.PrivilegedExceptionAction;
 import java.util.ArrayList;
 import java.util.ArrayList;
 import java.util.Date;
 import java.util.Date;
 import java.util.HashMap;
 import java.util.HashMap;
-import java.util.LinkedList;
 import java.util.List;
 import java.util.List;
 import java.util.Map;
 import java.util.Map;
 import java.util.Queue;
 import java.util.Queue;
@@ -84,6 +81,13 @@ import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import com.google.common.base.Preconditions;
 import com.google.common.base.Strings;
 import com.google.common.base.Strings;
 
 
+import static org.apache.hadoop.util.KMSUtil.checkNotEmpty;
+import static org.apache.hadoop.util.KMSUtil.checkNotNull;
+import static org.apache.hadoop.util.KMSUtil.parseJSONEncKeyVersion;
+import static org.apache.hadoop.util.KMSUtil.parseJSONEncKeyVersions;
+import static org.apache.hadoop.util.KMSUtil.parseJSONKeyVersion;
+import static org.apache.hadoop.util.KMSUtil.parseJSONMetadata;
+
 /**
 /**
  * KMS client <code>KeyProvider</code> implementation.
  * KMS client <code>KeyProvider</code> implementation.
  */
  */
@@ -219,77 +223,11 @@ public class KMSClientProvider extends KeyProvider implements CryptoExtension,
     }
     }
   }
   }
 
 
-  @SuppressWarnings("rawtypes")
-  private static List<EncryptedKeyVersion>
-      parseJSONEncKeyVersions(String keyName, List valueList) {
-    List<EncryptedKeyVersion> ekvs = new LinkedList<EncryptedKeyVersion>();
-    if (!valueList.isEmpty()) {
-      for (Object values : valueList) {
-        Map valueMap = (Map) values;
-        ekvs.add(parseJSONEncKeyVersion(keyName, valueMap));
-      }
-    }
-    return ekvs;
-  }
-
-  private static EncryptedKeyVersion parseJSONEncKeyVersion(String keyName,
-      Map valueMap) {
-    String versionName = checkNotNull(
-        (String) valueMap.get(KMSRESTConstants.VERSION_NAME_FIELD),
-        KMSRESTConstants.VERSION_NAME_FIELD);
-
-    byte[] iv = Base64.decodeBase64(checkNotNull(
-        (String) valueMap.get(KMSRESTConstants.IV_FIELD),
-        KMSRESTConstants.IV_FIELD));
-
-    Map encValueMap = checkNotNull((Map)
-            valueMap.get(KMSRESTConstants.ENCRYPTED_KEY_VERSION_FIELD),
-        KMSRESTConstants.ENCRYPTED_KEY_VERSION_FIELD);
-
-    String encVersionName = checkNotNull((String)
-            encValueMap.get(KMSRESTConstants.VERSION_NAME_FIELD),
-        KMSRESTConstants.VERSION_NAME_FIELD);
-
-    byte[] encKeyMaterial = Base64.decodeBase64(checkNotNull((String)
-            encValueMap.get(KMSRESTConstants.MATERIAL_FIELD),
-        KMSRESTConstants.MATERIAL_FIELD));
-
-    return new KMSEncryptedKeyVersion(keyName, versionName, iv,
-        encVersionName, encKeyMaterial);
-  }
-
-  private static KeyVersion parseJSONKeyVersion(Map valueMap) {
-    KeyVersion keyVersion = null;
-    if (!valueMap.isEmpty()) {
-      byte[] material = (valueMap.containsKey(KMSRESTConstants.MATERIAL_FIELD))
-          ? Base64.decodeBase64((String) valueMap.get(KMSRESTConstants.MATERIAL_FIELD))
-          : null;
-      String versionName = (String)valueMap.get(KMSRESTConstants.VERSION_NAME_FIELD);
-      String keyName = (String)valueMap.get(KMSRESTConstants.NAME_FIELD);
-      keyVersion = new KMSKeyVersion(keyName, versionName, material);
-    }
-    return keyVersion;
-  }
-
-  @SuppressWarnings("unchecked")
-  private static Metadata parseJSONMetadata(Map valueMap) {
-    Metadata metadata = null;
-    if (!valueMap.isEmpty()) {
-      metadata = new KMSMetadata(
-          (String) valueMap.get(KMSRESTConstants.CIPHER_FIELD),
-          (Integer) valueMap.get(KMSRESTConstants.LENGTH_FIELD),
-          (String) valueMap.get(KMSRESTConstants.DESCRIPTION_FIELD),
-          (Map<String, String>) valueMap.get(KMSRESTConstants.ATTRIBUTES_FIELD),
-          new Date((Long) valueMap.get(KMSRESTConstants.CREATED_FIELD)),
-          (Integer) valueMap.get(KMSRESTConstants.VERSIONS_FIELD));
-    }
-    return metadata;
-  }
-
-  private static void writeJson(Map map, OutputStream os) throws IOException {
+  private static void writeJson(Object obj, OutputStream os)
+      throws IOException {
     Writer writer = new OutputStreamWriter(os, StandardCharsets.UTF_8);
     Writer writer = new OutputStreamWriter(os, StandardCharsets.UTF_8);
     ObjectMapper jsonMapper = new ObjectMapper();
     ObjectMapper jsonMapper = new ObjectMapper();
-    jsonMapper.writerWithDefaultPrettyPrinter().writeValue(writer, map);
+    jsonMapper.writerWithDefaultPrettyPrinter().writeValue(writer, obj);
   }
   }
 
 
   /**
   /**
@@ -360,25 +298,6 @@ public class KMSClientProvider extends KeyProvider implements CryptoExtension,
     }
     }
   }
   }
 
 
-  public static <T> T checkNotNull(T o, String name)
-      throws IllegalArgumentException {
-    if (o == null) {
-      throw new IllegalArgumentException("Parameter '" + name +
-          "' cannot be null");
-    }
-    return o;
-  }
-
-  public static String checkNotEmpty(String s, String name)
-      throws IllegalArgumentException {
-    checkNotNull(s, name);
-    if (s.isEmpty()) {
-      throw new IllegalArgumentException("Parameter '" + name +
-          "' cannot be empty");
-    }
-    return s;
-  }
-
   private String kmsUrl;
   private String kmsUrl;
   private SSLFactory sslFactory;
   private SSLFactory sslFactory;
   private ConnectionConfigurator configurator;
   private ConnectionConfigurator configurator;
@@ -560,12 +479,12 @@ public class KMSClientProvider extends KeyProvider implements CryptoExtension,
     return conn;
     return conn;
   }
   }
 
 
-  private <T> T call(HttpURLConnection conn, Map jsonOutput,
+  private <T> T call(HttpURLConnection conn, Object jsonOutput,
       int expectedResponse, Class<T> klass) throws IOException {
       int expectedResponse, Class<T> klass) throws IOException {
     return call(conn, jsonOutput, expectedResponse, klass, authRetry);
     return call(conn, jsonOutput, expectedResponse, klass, authRetry);
   }
   }
 
 
-  private <T> T call(HttpURLConnection conn, Map jsonOutput,
+  private <T> T call(HttpURLConnection conn, Object jsonOutput,
       int expectedResponse, Class<T> klass, int authRetryCount)
       int expectedResponse, Class<T> klass, int authRetryCount)
       throws IOException {
       throws IOException {
     T ret = null;
     T ret = null;
@@ -601,17 +520,6 @@ public class KMSClientProvider extends KeyProvider implements CryptoExtension,
             authRetryCount - 1);
             authRetryCount - 1);
       }
       }
     }
     }
-    try {
-      AuthenticatedURL.extractToken(conn, authToken);
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Extracted token, authToken={}, its dt={}", authToken,
-            authToken.getDelegationToken());
-      }
-    } catch (AuthenticationException e) {
-      // Ignore the AuthExceptions.. since we are just using the method to
-      // extract and set the authToken.. (Workaround till we actually fix
-      // AuthenticatedURL properly to set authToken post initialization)
-    }
     HttpExceptionUtils.validateResponse(conn, expectedResponse);
     HttpExceptionUtils.validateResponse(conn, expectedResponse);
     if (conn.getContentType() != null
     if (conn.getContentType() != null
         && conn.getContentType().trim().toLowerCase()
         && conn.getContentType().trim().toLowerCase()
@@ -884,6 +792,48 @@ public class KMSClientProvider extends KeyProvider implements CryptoExtension,
     return parseJSONEncKeyVersion(ekv.getEncryptionKeyName(), response);
     return parseJSONEncKeyVersion(ekv.getEncryptionKeyName(), response);
   }
   }
 
 
+  @Override
+  public void reencryptEncryptedKeys(List<EncryptedKeyVersion> ekvs)
+      throws IOException, GeneralSecurityException {
+    checkNotNull(ekvs, "ekvs");
+    if (ekvs.isEmpty()) {
+      return;
+    }
+    final List<Map> jsonPayload = new ArrayList<>();
+    String keyName = null;
+    for (EncryptedKeyVersion ekv : ekvs) {
+      checkNotNull(ekv.getEncryptionKeyName(), "keyName");
+      checkNotNull(ekv.getEncryptionKeyVersionName(), "versionName");
+      checkNotNull(ekv.getEncryptedKeyIv(), "iv");
+      checkNotNull(ekv.getEncryptedKeyVersion(), "encryptedKey");
+      Preconditions.checkArgument(ekv.getEncryptedKeyVersion().getVersionName()
+              .equals(KeyProviderCryptoExtension.EEK),
+          "encryptedKey version name must be '%s', is '%s'",
+          KeyProviderCryptoExtension.EEK,
+          ekv.getEncryptedKeyVersion().getVersionName());
+      if (keyName == null) {
+        keyName = ekv.getEncryptionKeyName();
+      } else {
+        Preconditions.checkArgument(keyName.equals(ekv.getEncryptionKeyName()),
+            "All EncryptedKey must have the same key name.");
+      }
+      jsonPayload.add(KMSUtil.toJSON(ekv));
+    }
+    final URL url = createURL(KMSRESTConstants.KEY_RESOURCE, keyName,
+        KMSRESTConstants.REENCRYPT_BATCH_SUB_RESOURCE, null);
+    final HttpURLConnection conn = createConnection(url, HTTP_POST);
+    conn.setRequestProperty(CONTENT_TYPE, APPLICATION_JSON_MIME);
+    final List<Map> response =
+        call(conn, jsonPayload, HttpURLConnection.HTTP_OK, List.class);
+    Preconditions.checkArgument(response.size() == ekvs.size(),
+        "Response size is different than input size.");
+    for (int i = 0; i < response.size(); ++i) {
+      final Map item = response.get(i);
+      final EncryptedKeyVersion ekv = parseJSONEncKeyVersion(keyName, item);
+      ekvs.set(i, ekv);
+    }
+  }
+
   @Override
   @Override
   public List<KeyVersion> getKeyVersions(String name) throws IOException {
   public List<KeyVersion> getKeyVersions(String name) throws IOException {
     checkNotEmpty(name, "name");
     checkNotEmpty(name, "name");

+ 1 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/KMSRESTConstants.java

@@ -37,6 +37,7 @@ public class KMSRESTConstants {
   public static final String EEK_SUB_RESOURCE = "_eek";
   public static final String EEK_SUB_RESOURCE = "_eek";
   public static final String CURRENT_VERSION_SUB_RESOURCE = "_currentversion";
   public static final String CURRENT_VERSION_SUB_RESOURCE = "_currentversion";
   public static final String INVALIDATECACHE_RESOURCE = "_invalidatecache";
   public static final String INVALIDATECACHE_RESOURCE = "_invalidatecache";
+  public static final String REENCRYPT_BATCH_SUB_RESOURCE = "_reencryptbatch";
 
 
   public static final String KEY = "key";
   public static final String KEY = "key";
   public static final String EEK_OP = "eek_op";
   public static final String EEK_OP = "eek_op";

+ 20 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/LoadBalancingKMSClientProvider.java

@@ -312,6 +312,26 @@ public class LoadBalancingKMSClientProvider extends KeyProvider implements
     }
     }
   }
   }
 
 
+  @Override
+  public void reencryptEncryptedKeys(final List<EncryptedKeyVersion> ekvs)
+      throws IOException, GeneralSecurityException {
+    try {
+      doOp(new ProviderCallable<Void>() {
+        @Override
+        public Void call(KMSClientProvider provider)
+            throws IOException, GeneralSecurityException {
+          provider.reencryptEncryptedKeys(ekvs);
+          return null;
+        }
+      }, nextIdx());
+    } catch (WrapperException we) {
+      if (we.getCause() instanceof GeneralSecurityException) {
+        throw (GeneralSecurityException) we.getCause();
+      }
+      throw new IOException(we.getCause());
+    }
+  }
+
   @Override
   @Override
   public KeyVersion getKeyVersion(final String versionName) throws IOException {
   public KeyVersion getKeyVersion(final String versionName) throws IOException {
     return doOp(new ProviderCallable<KeyVersion>() {
     return doOp(new ProviderCallable<KeyVersion>() {

+ 58 - 16
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSDataOutputStreamBuilder.java

@@ -54,16 +54,29 @@ import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_FILE_BUFFER_
  * options accordingly, for example:
  * options accordingly, for example:
  *
  *
  * <code>
  * <code>
- * FSDataOutputStreamBuilder builder = fs.createFile(path);
- * builder.permission(perm)
+ *
+ * // Don't
+ * if (fs instanceof FooFileSystem) {
+ *   FooFileSystem fs = (FooFileSystem) fs;
+ *   OutputStream out = dfs.createFile(path)
+ *     .optionA()
+ *     .optionB("value")
+ *     .cache()
+ *   .build()
+ * } else if (fs instanceof BarFileSystem) {
+ *   ...
+ * }
+ *
+ * // Do
+ * OutputStream out = fs.createFile(path)
+ *   .permission(perm)
  *   .bufferSize(bufSize)
  *   .bufferSize(bufSize)
- *   .opt("dfs.outputstream.builder.lazy-persist", true)
- *   .opt("dfs.outputstream.builder.ec.policy-name", "rs-3-2-64k")
- *   .opt("fs.local.o-direct", true)
- *   .must("fs.s3a.fast-upload", true)
- *   .must("fs.azure.buffer-size", 256 * 1024 * 1024);
- * FSDataOutputStream out = builder.build();
- * ...
+ *   .opt("foofs:option.a", true)
+ *   .opt("foofs:option.b", "value")
+ *   .opt("barfs:cache", true)
+ *   .must("foofs:cache", true)
+ *   .must("barfs:cache-size", 256 * 1024 * 1024)
+ *   .build();
  * </code>
  * </code>
  *
  *
  * If the option is not related to the file system, the option will be ignored.
  * If the option is not related to the file system, the option will be ignored.
@@ -263,6 +276,8 @@ public abstract class FSDataOutputStreamBuilder
 
 
   /**
   /**
    * Set optional boolean parameter for the Builder.
    * Set optional boolean parameter for the Builder.
+   *
+   * @see #opt(String, String)
    */
    */
   public B opt(@Nonnull final String key, boolean value) {
   public B opt(@Nonnull final String key, boolean value) {
     mandatoryKeys.remove(key);
     mandatoryKeys.remove(key);
@@ -272,6 +287,8 @@ public abstract class FSDataOutputStreamBuilder
 
 
   /**
   /**
    * Set optional int parameter for the Builder.
    * Set optional int parameter for the Builder.
+   *
+   * @see #opt(String, String)
    */
    */
   public B opt(@Nonnull final String key, int value) {
   public B opt(@Nonnull final String key, int value) {
     mandatoryKeys.remove(key);
     mandatoryKeys.remove(key);
@@ -281,6 +298,8 @@ public abstract class FSDataOutputStreamBuilder
 
 
   /**
   /**
    * Set optional float parameter for the Builder.
    * Set optional float parameter for the Builder.
+   *
+   * @see #opt(String, String)
    */
    */
   public B opt(@Nonnull final String key, float value) {
   public B opt(@Nonnull final String key, float value) {
     mandatoryKeys.remove(key);
     mandatoryKeys.remove(key);
@@ -290,6 +309,8 @@ public abstract class FSDataOutputStreamBuilder
 
 
   /**
   /**
    * Set optional double parameter for the Builder.
    * Set optional double parameter for the Builder.
+   *
+   * @see #opt(String, String)
    */
    */
   public B opt(@Nonnull final String key, double value) {
   public B opt(@Nonnull final String key, double value) {
     mandatoryKeys.remove(key);
     mandatoryKeys.remove(key);
@@ -299,6 +320,8 @@ public abstract class FSDataOutputStreamBuilder
 
 
   /**
   /**
    * Set an array of string values as optional parameter for the Builder.
    * Set an array of string values as optional parameter for the Builder.
+   *
+   * @see #opt(String, String)
    */
    */
   public B opt(@Nonnull final String key, @Nonnull final String... values) {
   public B opt(@Nonnull final String key, @Nonnull final String... values) {
     mandatoryKeys.remove(key);
     mandatoryKeys.remove(key);
@@ -310,8 +333,7 @@ public abstract class FSDataOutputStreamBuilder
    * Set mandatory option to the Builder.
    * Set mandatory option to the Builder.
    *
    *
    * If the option is not supported or unavailable on the {@link FileSystem},
    * If the option is not supported or unavailable on the {@link FileSystem},
-   * the client should expect {@link #build()} throws
-   * {@link IllegalArgumentException}.
+   * the client should expect {@link #build()} throws IllegalArgumentException.
    */
    */
   public B must(@Nonnull final String key, @Nonnull final String value) {
   public B must(@Nonnull final String key, @Nonnull final String value) {
     mandatoryKeys.add(key);
     mandatoryKeys.add(key);
@@ -319,35 +341,55 @@ public abstract class FSDataOutputStreamBuilder
     return getThisBuilder();
     return getThisBuilder();
   }
   }
 
 
-  /** Set mandatory boolean option. */
+  /**
+   * Set mandatory boolean option.
+   *
+   * @see #must(String, String)
+   */
   public B must(@Nonnull final String key, boolean value) {
   public B must(@Nonnull final String key, boolean value) {
     mandatoryKeys.add(key);
     mandatoryKeys.add(key);
     options.setBoolean(key, value);
     options.setBoolean(key, value);
     return getThisBuilder();
     return getThisBuilder();
   }
   }
 
 
-  /** Set mandatory int option. */
+  /**
+   * Set mandatory int option.
+   *
+   * @see #must(String, String)
+   */
   public B must(@Nonnull final String key, int value) {
   public B must(@Nonnull final String key, int value) {
     mandatoryKeys.add(key);
     mandatoryKeys.add(key);
     options.setInt(key, value);
     options.setInt(key, value);
     return getThisBuilder();
     return getThisBuilder();
   }
   }
 
 
-  /** Set mandatory float option. */
+  /**
+   * Set mandatory float option.
+   *
+   * @see #must(String, String)
+   */
   public B must(@Nonnull final String key, float value) {
   public B must(@Nonnull final String key, float value) {
     mandatoryKeys.add(key);
     mandatoryKeys.add(key);
     options.setFloat(key, value);
     options.setFloat(key, value);
     return getThisBuilder();
     return getThisBuilder();
   }
   }
 
 
-  /** Set mandatory double option. */
+  /**
+   * Set mandatory double option.
+   *
+   * @see #must(String, String)
+   */
   public B must(@Nonnull final String key, double value) {
   public B must(@Nonnull final String key, double value) {
     mandatoryKeys.add(key);
     mandatoryKeys.add(key);
     options.setDouble(key, value);
     options.setDouble(key, value);
     return getThisBuilder();
     return getThisBuilder();
   }
   }
 
 
-  /** Set a string array as mandatory option. */
+  /**
+   * Set a string array as mandatory option.
+   *
+   * @see #must(String, String)
+   */
   public B must(@Nonnull final String key, @Nonnull final String... values) {
   public B must(@Nonnull final String key, @Nonnull final String... values) {
     mandatoryKeys.add(key);
     mandatoryKeys.add(key);
     options.setStrings(key, values);
     options.setStrings(key, values);

+ 9 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FsServerDefaults.java

@@ -60,6 +60,15 @@ public class FsServerDefaults implements Writable {
   public FsServerDefaults() {
   public FsServerDefaults() {
   }
   }
 
 
+  public FsServerDefaults(long blockSize, int bytesPerChecksum,
+      int writePacketSize, short replication, int fileBufferSize,
+      boolean encryptDataTransfer, long trashInterval,
+      DataChecksum.Type checksumType) {
+    this(blockSize, bytesPerChecksum, writePacketSize, replication,
+        fileBufferSize, encryptDataTransfer, trashInterval, checksumType,
+        null, (byte) 0);
+  }
+
   public FsServerDefaults(long blockSize, int bytesPerChecksum,
   public FsServerDefaults(long blockSize, int bytesPerChecksum,
       int writePacketSize, short replication, int fileBufferSize,
       int writePacketSize, short replication, int fileBufferSize,
       boolean encryptDataTransfer, long trashInterval,
       boolean encryptDataTransfer, long trashInterval,

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

@@ -84,8 +84,10 @@ public class LocatedFileStatus extends FileStatus {
           Path symlink, Path path, BlockLocation[] locations) {
           Path symlink, Path path, BlockLocation[] locations) {
     this(length, isdir, block_replication, blocksize, modification_time,
     this(length, isdir, block_replication, blocksize, modification_time,
         access_time, permission, owner, group, symlink, path,
         access_time, permission, owner, group, symlink, path,
-        permission.getAclBit(), permission.getEncryptedBit(),
-        permission.getErasureCodedBit(), locations);
+        permission == null ? false : permission.getAclBit(),
+        permission == null ? false : permission.getEncryptedBit(),
+        permission == null ? false : permission.getErasureCodedBit(),
+        locations);
   }
   }
 
 
   /**
   /**

+ 1 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/permission/FsPermission.java

@@ -163,7 +163,7 @@ public class FsPermission implements Writable, Serializable,
    */
    */
   public static FsPermission read(DataInput in) throws IOException {
   public static FsPermission read(DataInput in) throws IOException {
     FsPermission p = new FsPermission();
     FsPermission p = new FsPermission();
-    p.readFields(in);
+    p.fromShort(in.readShort());
     return p;
     return p;
   }
   }
 
 

+ 6 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/sftp/SFTPFileSystem.java

@@ -326,8 +326,10 @@ public class SFTPFileSystem extends FileSystem {
         String parentDir = parent.toUri().getPath();
         String parentDir = parent.toUri().getPath();
         boolean succeeded = true;
         boolean succeeded = true;
         try {
         try {
+          final String previousCwd = client.pwd();
           client.cd(parentDir);
           client.cd(parentDir);
           client.mkdir(pathName);
           client.mkdir(pathName);
+          client.cd(previousCwd);
         } catch (SftpException e) {
         } catch (SftpException e) {
           throw new IOException(String.format(E_MAKE_DIR_FORPATH, pathName,
           throw new IOException(String.format(E_MAKE_DIR_FORPATH, pathName,
               parentDir));
               parentDir));
@@ -474,8 +476,10 @@ public class SFTPFileSystem extends FileSystem {
     }
     }
     boolean renamed = true;
     boolean renamed = true;
     try {
     try {
+      final String previousCwd = channel.pwd();
       channel.cd("/");
       channel.cd("/");
       channel.rename(src.toUri().getPath(), dst.toUri().getPath());
       channel.rename(src.toUri().getPath(), dst.toUri().getPath());
+      channel.cd(previousCwd);
     } catch (SftpException e) {
     } catch (SftpException e) {
       renamed = false;
       renamed = false;
     }
     }
@@ -558,8 +562,10 @@ public class SFTPFileSystem extends FileSystem {
     }
     }
     OutputStream os;
     OutputStream os;
     try {
     try {
+      final String previousCwd = client.pwd();
       client.cd(parent.toUri().getPath());
       client.cd(parent.toUri().getPath());
       os = client.put(f.getName());
       os = client.put(f.getName());
+      client.cd(previousCwd);
     } catch (SftpException e) {
     } catch (SftpException e) {
       throw new IOException(e);
       throw new IOException(e);
     }
     }

+ 3 - 3
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/AclCommands.java

@@ -86,9 +86,9 @@ class AclCommands extends FsCommand {
           (perm.getOtherAction().implies(FsAction.EXECUTE) ? "t" : "T"));
           (perm.getOtherAction().implies(FsAction.EXECUTE) ? "t" : "T"));
       }
       }
 
 
-      AclStatus aclStatus = null;
-      List<AclEntry> entries = null;
-      if (perm.getAclBit()) {
+      final AclStatus aclStatus;
+      final List<AclEntry> entries;
+      if (item.stat.hasAcl()) {
         aclStatus = item.fs.getAclStatus(item.path);
         aclStatus = item.fs.getAclStatus(item.path);
         entries = aclStatus.getEntries();
         entries = aclStatus.getEntries();
       } else {
       } else {

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

@@ -444,8 +444,8 @@ abstract class CommandWithDestination extends FsCommand {
           src.stat.getPermission());
           src.stat.getPermission());
     }
     }
     if (shouldPreserve(FileAttribute.ACL)) {
     if (shouldPreserve(FileAttribute.ACL)) {
-      FsPermission perm = src.stat.getPermission();
-      if (perm.getAclBit()) {
+      if (src.stat.hasAcl()) {
+        FsPermission perm = src.stat.getPermission();
         List<AclEntry> srcEntries =
         List<AclEntry> srcEntries =
             src.fs.getAclStatus(src.path).getEntries();
             src.fs.getAclStatus(src.path).getEntries();
         List<AclEntry> srcFullEntries =
         List<AclEntry> srcFullEntries =

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

@@ -255,7 +255,7 @@ class Ls extends FsCommand {
       ContentSummary contentSummary = item.fs.getContentSummary(item.path);
       ContentSummary contentSummary = item.fs.getContentSummary(item.path);
       String line = String.format(lineFormat,
       String line = String.format(lineFormat,
           (stat.isDirectory() ? "d" : "-"),
           (stat.isDirectory() ? "d" : "-"),
-          stat.getPermission() + (stat.getPermission().getAclBit() ? "+" : " "),
+          stat.getPermission() + (stat.hasAcl() ? "+" : " "),
           (stat.isFile() ? stat.getReplication() : "-"),
           (stat.isFile() ? stat.getReplication() : "-"),
           stat.getOwner(),
           stat.getOwner(),
           stat.getGroup(),
           stat.getGroup(),
@@ -269,7 +269,7 @@ class Ls extends FsCommand {
     } else {
     } else {
       String line = String.format(lineFormat,
       String line = String.format(lineFormat,
           (stat.isDirectory() ? "d" : "-"),
           (stat.isDirectory() ? "d" : "-"),
-          stat.getPermission() + (stat.getPermission().getAclBit() ? "+" : " "),
+          stat.getPermission() + (stat.hasAcl() ? "+" : " "),
           (stat.isFile() ? stat.getReplication() : "-"),
           (stat.isFile() ? stat.getReplication() : "-"),
           stat.getOwner(),
           stat.getOwner(),
           stat.getGroup(),
           stat.getGroup(),

+ 8 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpServer2.java

@@ -128,6 +128,10 @@ public final class HttpServer2 implements FilterContainer {
   public static final String HTTP_MAX_RESPONSE_HEADER_SIZE_KEY =
   public static final String HTTP_MAX_RESPONSE_HEADER_SIZE_KEY =
       "hadoop.http.max.response.header.size";
       "hadoop.http.max.response.header.size";
   public static final int HTTP_MAX_RESPONSE_HEADER_SIZE_DEFAULT = 65536;
   public static final int HTTP_MAX_RESPONSE_HEADER_SIZE_DEFAULT = 65536;
+
+  public static final String HTTP_SOCKET_BACKLOG_SIZE_KEY =
+      "hadoop.http.socket.backlog.size";
+  public static final int HTTP_SOCKET_BACKLOG_SIZE_DEFAULT = 128;
   public static final String HTTP_MAX_THREADS_KEY = "hadoop.http.max.threads";
   public static final String HTTP_MAX_THREADS_KEY = "hadoop.http.max.threads";
   public static final String HTTP_TEMP_DIR_KEY = "hadoop.http.temp.dir";
   public static final String HTTP_TEMP_DIR_KEY = "hadoop.http.temp.dir";
 
 
@@ -433,6 +437,9 @@ public final class HttpServer2 implements FilterContainer {
       httpConfig.setResponseHeaderSize(responseHeaderSize);
       httpConfig.setResponseHeaderSize(responseHeaderSize);
       httpConfig.setSendServerVersion(false);
       httpConfig.setSendServerVersion(false);
 
 
+      int backlogSize = conf.getInt(HTTP_SOCKET_BACKLOG_SIZE_KEY,
+          HTTP_SOCKET_BACKLOG_SIZE_DEFAULT);
+
       for (URI ep : endpoints) {
       for (URI ep : endpoints) {
         final ServerConnector connector;
         final ServerConnector connector;
         String scheme = ep.getScheme();
         String scheme = ep.getScheme();
@@ -448,6 +455,7 @@ public final class HttpServer2 implements FilterContainer {
         }
         }
         connector.setHost(ep.getHost());
         connector.setHost(ep.getHost());
         connector.setPort(ep.getPort() == -1 ? 0 : ep.getPort());
         connector.setPort(ep.getPort() == -1 ? 0 : ep.getPort());
+        connector.setAcceptQueueSize(backlogSize);
         server.addListener(connector);
         server.addListener(connector);
       }
       }
       server.loadListeners();
       server.loadListeners();
@@ -640,7 +648,6 @@ public final class HttpServer2 implements FilterContainer {
 
 
   private static void configureChannelConnector(ServerConnector c) {
   private static void configureChannelConnector(ServerConnector c) {
     c.setIdleTimeout(10000);
     c.setIdleTimeout(10000);
-    c.setAcceptQueueSize(128);
     if(Shell.WINDOWS) {
     if(Shell.WINDOWS) {
       // result of setting the SO_REUSEADDR flag is different on Windows
       // result of setting the SO_REUSEADDR flag is different on Windows
       // http://msdn.microsoft.com/en-us/library/ms740621(v=vs.85).aspx
       // http://msdn.microsoft.com/en-us/library/ms740621(v=vs.85).aspx

+ 1 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/CodecRegistry.java

@@ -176,7 +176,7 @@ public final class CodecRegistry {
    * @return a map of all codec names, and their corresponding code list
    * @return a map of all codec names, and their corresponding code list
    * separated by ','.
    * separated by ','.
    */
    */
-  public HashMap<String, String> getCodec2CoderCompactMap() {
+  public Map<String, String> getCodec2CoderCompactMap() {
     return coderNameCompactMap;
     return coderNameCompactMap;
   }
   }
 }
 }

+ 8 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ErasureCodeConstants.java

@@ -30,6 +30,7 @@ public final class ErasureCodeConstants {
   public static final String RS_LEGACY_CODEC_NAME = "rs-legacy";
   public static final String RS_LEGACY_CODEC_NAME = "rs-legacy";
   public static final String XOR_CODEC_NAME = "xor";
   public static final String XOR_CODEC_NAME = "xor";
   public static final String HHXOR_CODEC_NAME = "hhxor";
   public static final String HHXOR_CODEC_NAME = "hhxor";
+  public static final String REPLICATION_CODEC_NAME = "replication";
 
 
   public static final ECSchema RS_6_3_SCHEMA = new ECSchema(
   public static final ECSchema RS_6_3_SCHEMA = new ECSchema(
       RS_CODEC_NAME, 6, 3);
       RS_CODEC_NAME, 6, 3);
@@ -45,4 +46,11 @@ public final class ErasureCodeConstants {
 
 
   public static final ECSchema RS_10_4_SCHEMA = new ECSchema(
   public static final ECSchema RS_10_4_SCHEMA = new ECSchema(
       RS_CODEC_NAME, 10, 4);
       RS_CODEC_NAME, 10, 4);
+
+  public static final ECSchema REPLICATION_1_2_SCHEMA = new ECSchema(
+      REPLICATION_CODEC_NAME, 1, 2);
+
+  public static final byte USER_DEFINED_POLICY_START_ID = (byte) 64;
+  public static final byte REPLICATION_POLICY_ID = (byte) 63;
+  public static final String REPLICATION_POLICY_NAME = REPLICATION_CODEC_NAME;
 }
 }

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

@@ -190,7 +190,7 @@ public class ProtobufRpcEngine implements RpcEngine {
         throws ServiceException {
         throws ServiceException {
       long startTime = 0;
       long startTime = 0;
       if (LOG.isDebugEnabled()) {
       if (LOG.isDebugEnabled()) {
-        startTime = Time.monotonicNow();
+        startTime = Time.now();
       }
       }
       
       
       if (args.length != 2) { // RpcController + Message
       if (args.length != 2) { // RpcController + Message
@@ -245,7 +245,7 @@ public class ProtobufRpcEngine implements RpcEngine {
       }
       }
 
 
       if (LOG.isDebugEnabled()) {
       if (LOG.isDebugEnabled()) {
-        long callTime = Time.monotonicNow() - startTime;
+        long callTime = Time.now() - startTime;
         LOG.debug("Call: " + method.getName() + " took " + callTime + "ms");
         LOG.debug("Call: " + method.getName() + " took " + callTime + "ms");
       }
       }
       
       
@@ -373,19 +373,19 @@ public class ProtobufRpcEngine implements RpcEngine {
         this.server = currentCallInfo.get().server;
         this.server = currentCallInfo.get().server;
         this.call = Server.getCurCall().get();
         this.call = Server.getCurCall().get();
         this.methodName = currentCallInfo.get().methodName;
         this.methodName = currentCallInfo.get().methodName;
-        this.setupTime = Time.monotonicNow();
+        this.setupTime = Time.now();
       }
       }
 
 
       @Override
       @Override
       public void setResponse(Message message) {
       public void setResponse(Message message) {
-        long processingTime = Time.monotonicNow() - setupTime;
+        long processingTime = Time.now() - setupTime;
         call.setDeferredResponse(RpcWritable.wrap(message));
         call.setDeferredResponse(RpcWritable.wrap(message));
         server.updateDeferredMetrics(methodName, processingTime);
         server.updateDeferredMetrics(methodName, processingTime);
       }
       }
 
 
       @Override
       @Override
       public void error(Throwable t) {
       public void error(Throwable t) {
-        long processingTime = Time.monotonicNow() - setupTime;
+        long processingTime = Time.now() - setupTime;
         String detailedMetricsName = t.getClass().getSimpleName();
         String detailedMetricsName = t.getClass().getSimpleName();
         server.updateDeferredMetrics(detailedMetricsName, processingTime);
         server.updateDeferredMetrics(detailedMetricsName, processingTime);
         call.setDeferredError(t);
         call.setDeferredError(t);
@@ -513,7 +513,7 @@ public class ProtobufRpcEngine implements RpcEngine {
         Message param = request.getValue(prototype);
         Message param = request.getValue(prototype);
 
 
         Message result;
         Message result;
-        long startTime = Time.monotonicNow();
+        long startTime = Time.now();
         int qTime = (int) (startTime - receiveTime);
         int qTime = (int) (startTime - receiveTime);
         Exception exception = null;
         Exception exception = null;
         boolean isDeferred = false;
         boolean isDeferred = false;
@@ -537,7 +537,7 @@ public class ProtobufRpcEngine implements RpcEngine {
           throw e;
           throw e;
         } finally {
         } finally {
           currentCallInfo.set(null);
           currentCallInfo.set(null);
-          int processingTime = (int) (Time.monotonicNow() - startTime);
+          int processingTime = (int) (Time.now() - startTime);
           if (LOG.isDebugEnabled()) {
           if (LOG.isDebugEnabled()) {
             String msg =
             String msg =
                 "Served: " + methodName + (isDeferred ? ", deferred" : "") +
                 "Served: " + methodName + (isDeferred ? ", deferred" : "") +

+ 134 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/KMSUtil.java

@@ -17,15 +17,24 @@
  */
  */
 package org.apache.hadoop.util;
 package org.apache.hadoop.util;
 
 
+import org.apache.commons.codec.binary.Base64;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.crypto.key.KeyProvider;
 import org.apache.hadoop.crypto.key.KeyProvider;
+import org.apache.hadoop.crypto.key.KeyProviderCryptoExtension.EncryptedKeyVersion;
 import org.apache.hadoop.crypto.key.KeyProviderFactory;
 import org.apache.hadoop.crypto.key.KeyProviderFactory;
+import org.apache.hadoop.crypto.key.kms.KMSClientProvider;
+import org.apache.hadoop.crypto.key.kms.KMSRESTConstants;
 import org.slf4j.Logger;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.slf4j.LoggerFactory;
 
 
 import java.io.IOException;
 import java.io.IOException;
 import java.net.URI;
 import java.net.URI;
+import java.util.ArrayList;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
 
 
 /**
 /**
  * Utils for KMS.
  * Utils for KMS.
@@ -71,4 +80,129 @@ public final class KMSUtil {
     }
     }
     return keyProvider;
     return keyProvider;
   }
   }
+
+  @SuppressWarnings("unchecked")
+  public static Map toJSON(KeyProvider.KeyVersion keyVersion) {
+    Map json = new HashMap();
+    if (keyVersion != null) {
+      json.put(KMSRESTConstants.NAME_FIELD,
+          keyVersion.getName());
+      json.put(KMSRESTConstants.VERSION_NAME_FIELD,
+          keyVersion.getVersionName());
+      json.put(KMSRESTConstants.MATERIAL_FIELD,
+          Base64.encodeBase64URLSafeString(
+              keyVersion.getMaterial()));
+    }
+    return json;
+  }
+
+  @SuppressWarnings("unchecked")
+  public static Map toJSON(EncryptedKeyVersion encryptedKeyVersion) {
+    Map json = new HashMap();
+    if (encryptedKeyVersion != null) {
+      json.put(KMSRESTConstants.VERSION_NAME_FIELD,
+          encryptedKeyVersion.getEncryptionKeyVersionName());
+      json.put(KMSRESTConstants.IV_FIELD, Base64
+          .encodeBase64URLSafeString(encryptedKeyVersion.getEncryptedKeyIv()));
+      json.put(KMSRESTConstants.ENCRYPTED_KEY_VERSION_FIELD,
+          toJSON(encryptedKeyVersion.getEncryptedKeyVersion()));
+    }
+    return json;
+  }
+
+  public static <T> T checkNotNull(T o, String name)
+      throws IllegalArgumentException {
+    if (o == null) {
+      throw new IllegalArgumentException("Parameter '" + name +
+          "' cannot be null");
+    }
+    return o;
+  }
+
+  public static String checkNotEmpty(String s, String name)
+      throws IllegalArgumentException {
+    checkNotNull(s, name);
+    if (s.isEmpty()) {
+      throw new IllegalArgumentException("Parameter '" + name +
+          "' cannot be empty");
+    }
+    return s;
+  }
+
+  @SuppressWarnings("rawtypes")
+  public static List<EncryptedKeyVersion>
+      parseJSONEncKeyVersions(String keyName, List valueList) {
+    checkNotNull(valueList, "valueList");
+    List<EncryptedKeyVersion> ekvs = new ArrayList<>(valueList.size());
+    if (!valueList.isEmpty()) {
+      for (Object values : valueList) {
+        Map valueMap = (Map) values;
+        ekvs.add(parseJSONEncKeyVersion(keyName, valueMap));
+      }
+    }
+    return ekvs;
+  }
+
+  @SuppressWarnings("unchecked")
+  public static EncryptedKeyVersion parseJSONEncKeyVersion(String keyName,
+      Map valueMap) {
+    checkNotNull(valueMap, "valueMap");
+    String versionName = checkNotNull(
+        (String) valueMap.get(KMSRESTConstants.VERSION_NAME_FIELD),
+        KMSRESTConstants.VERSION_NAME_FIELD);
+
+    byte[] iv = Base64.decodeBase64(checkNotNull(
+        (String) valueMap.get(KMSRESTConstants.IV_FIELD),
+        KMSRESTConstants.IV_FIELD));
+
+    Map encValueMap = checkNotNull((Map)
+            valueMap.get(KMSRESTConstants.ENCRYPTED_KEY_VERSION_FIELD),
+        KMSRESTConstants.ENCRYPTED_KEY_VERSION_FIELD);
+
+    String encVersionName = checkNotNull((String)
+            encValueMap.get(KMSRESTConstants.VERSION_NAME_FIELD),
+        KMSRESTConstants.VERSION_NAME_FIELD);
+
+    byte[] encKeyMaterial = Base64.decodeBase64(checkNotNull((String)
+            encValueMap.get(KMSRESTConstants.MATERIAL_FIELD),
+        KMSRESTConstants.MATERIAL_FIELD));
+
+    return new KMSClientProvider.KMSEncryptedKeyVersion(keyName, versionName,
+        iv, encVersionName, encKeyMaterial);
+  }
+
+  @SuppressWarnings("unchecked")
+  public static KeyProvider.KeyVersion parseJSONKeyVersion(Map valueMap) {
+    checkNotNull(valueMap, "valueMap");
+    KeyProvider.KeyVersion keyVersion = null;
+    if (!valueMap.isEmpty()) {
+      byte[] material =
+          (valueMap.containsKey(KMSRESTConstants.MATERIAL_FIELD)) ?
+              Base64.decodeBase64(
+                  (String) valueMap.get(KMSRESTConstants.MATERIAL_FIELD)) :
+              null;
+      String versionName =
+          (String) valueMap.get(KMSRESTConstants.VERSION_NAME_FIELD);
+      String keyName = (String) valueMap.get(KMSRESTConstants.NAME_FIELD);
+      keyVersion =
+          new KMSClientProvider.KMSKeyVersion(keyName, versionName, material);
+    }
+    return keyVersion;
+  }
+
+  @SuppressWarnings("unchecked")
+  public static KeyProvider.Metadata parseJSONMetadata(Map valueMap) {
+    checkNotNull(valueMap, "valueMap");
+    KeyProvider.Metadata metadata = null;
+    if (!valueMap.isEmpty()) {
+      metadata = new KMSClientProvider.KMSMetadata(
+          (String) valueMap.get(KMSRESTConstants.CIPHER_FIELD),
+          (Integer) valueMap.get(KMSRESTConstants.LENGTH_FIELD),
+          (String) valueMap.get(KMSRESTConstants.DESCRIPTION_FIELD),
+          (Map<String, String>) valueMap.get(KMSRESTConstants.ATTRIBUTES_FIELD),
+          new Date((Long) valueMap.get(KMSRESTConstants.CREATED_FIELD)),
+          (Integer) valueMap.get(KMSRESTConstants.VERSIONS_FIELD));
+    }
+    return metadata;
+  }
 }
 }

+ 135 - 7
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/curator/ZKCuratorManager.java

@@ -26,6 +26,8 @@ import java.util.List;
 import org.apache.curator.framework.AuthInfo;
 import org.apache.curator.framework.AuthInfo;
 import org.apache.curator.framework.CuratorFramework;
 import org.apache.curator.framework.CuratorFramework;
 import org.apache.curator.framework.CuratorFrameworkFactory;
 import org.apache.curator.framework.CuratorFrameworkFactory;
+import org.apache.curator.framework.api.transaction.CuratorTransaction;
+import org.apache.curator.framework.api.transaction.CuratorTransactionFinal;
 import org.apache.curator.retry.RetryNTimes;
 import org.apache.curator.retry.RetryNTimes;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
@@ -33,9 +35,12 @@ import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.util.ZKUtil;
 import org.apache.hadoop.util.ZKUtil;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.data.ACL;
 import org.apache.zookeeper.data.ACL;
+import org.apache.zookeeper.data.Stat;
 import org.slf4j.Logger;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.slf4j.LoggerFactory;
 
 
+import com.google.common.base.Preconditions;
+
 /**
 /**
  * Helper class that provides utility methods specific to ZK operations.
  * Helper class that provides utility methods specific to ZK operations.
  */
  */
@@ -51,7 +56,6 @@ public final class ZKCuratorManager {
   /** Curator for ZooKeeper. */
   /** Curator for ZooKeeper. */
   private CuratorFramework curator;
   private CuratorFramework curator;
 
 
-
   public ZKCuratorManager(Configuration config) throws IOException {
   public ZKCuratorManager(Configuration config) throws IOException {
     this.conf = config;
     this.conf = config;
   }
   }
@@ -116,7 +120,6 @@ public final class ZKCuratorManager {
 
 
   /**
   /**
    * Start the connection to the ZooKeeper ensemble.
    * Start the connection to the ZooKeeper ensemble.
-   * @param conf Configuration for the connection.
    * @throws IOException If the connection cannot be started.
    * @throws IOException If the connection cannot be started.
    */
    */
   public void start() throws IOException {
   public void start() throws IOException {
@@ -125,7 +128,6 @@ public final class ZKCuratorManager {
 
 
   /**
   /**
    * Start the connection to the ZooKeeper ensemble.
    * Start the connection to the ZooKeeper ensemble.
-   * @param conf Configuration for the connection.
    * @param authInfos List of authentication keys.
    * @param authInfos List of authentication keys.
    * @throws IOException If the connection cannot be started.
    * @throws IOException If the connection cannot be started.
    */
    */
@@ -179,7 +181,6 @@ public final class ZKCuratorManager {
   /**
   /**
    * Get the data in a ZNode.
    * Get the data in a ZNode.
    * @param path Path of the ZNode.
    * @param path Path of the ZNode.
-   * @param stat Output statistics of the ZNode.
    * @return The data in the ZNode.
    * @return The data in the ZNode.
    * @throws Exception If it cannot contact Zookeeper.
    * @throws Exception If it cannot contact Zookeeper.
    */
    */
@@ -190,15 +191,37 @@ public final class ZKCuratorManager {
   /**
   /**
    * Get the data in a ZNode.
    * Get the data in a ZNode.
    * @param path Path of the ZNode.
    * @param path Path of the ZNode.
-   * @param stat Output statistics of the ZNode.
+   * @param stat
+   * @return The data in the ZNode.
+   * @throws Exception If it cannot contact Zookeeper.
+   */
+  public byte[] getData(final String path, Stat stat) throws Exception {
+    return curator.getData().storingStatIn(stat).forPath(path);
+  }
+
+  /**
+   * Get the data in a ZNode.
+   * @param path Path of the ZNode.
    * @return The data in the ZNode.
    * @return The data in the ZNode.
    * @throws Exception If it cannot contact Zookeeper.
    * @throws Exception If it cannot contact Zookeeper.
    */
    */
-  public String getSringData(final String path) throws Exception {
+  public String getStringData(final String path) throws Exception {
     byte[] bytes = getData(path);
     byte[] bytes = getData(path);
     return new String(bytes, Charset.forName("UTF-8"));
     return new String(bytes, Charset.forName("UTF-8"));
   }
   }
 
 
+  /**
+   * Get the data in a ZNode.
+   * @param path Path of the ZNode.
+   * @param stat Output statistics of the ZNode.
+   * @return The data in the ZNode.
+   * @throws Exception If it cannot contact Zookeeper.
+   */
+  public String getStringData(final String path, Stat stat) throws Exception {
+    byte[] bytes = getData(path, stat);
+    return new String(bytes, Charset.forName("UTF-8"));
+  }
+
   /**
   /**
    * Set data into a ZNode.
    * Set data into a ZNode.
    * @param path Path of the ZNode.
    * @param path Path of the ZNode.
@@ -271,15 +294,37 @@ public final class ZKCuratorManager {
     return created;
     return created;
   }
   }
 
 
+  /**
+   * Utility function to ensure that the configured base znode exists.
+   * This recursively creates the znode as well as all of its parents.
+   * @param path Path of the znode to create.
+   * @throws Exception If it cannot create the file.
+   */
+  public void createRootDirRecursively(String path) throws Exception {
+    String[] pathParts = path.split("/");
+    Preconditions.checkArgument(
+        pathParts.length >= 1 && pathParts[0].isEmpty(),
+        "Invalid path: %s", path);
+    StringBuilder sb = new StringBuilder();
+
+    for (int i = 1; i < pathParts.length; i++) {
+      sb.append("/").append(pathParts[i]);
+      create(sb.toString());
+    }
+  }
+
   /**
   /**
    * Delete a ZNode.
    * Delete a ZNode.
    * @param path Path of the ZNode.
    * @param path Path of the ZNode.
+   * @return If the znode was deleted.
    * @throws Exception If it cannot contact ZooKeeper.
    * @throws Exception If it cannot contact ZooKeeper.
    */
    */
-  public void delete(final String path) throws Exception {
+  public boolean delete(final String path) throws Exception {
     if (exists(path)) {
     if (exists(path)) {
       curator.delete().deletingChildrenIfNeeded().forPath(path);
       curator.delete().deletingChildrenIfNeeded().forPath(path);
+      return true;
     }
     }
+    return false;
   }
   }
 
 
   /**
   /**
@@ -291,4 +336,87 @@ public final class ZKCuratorManager {
   public static String getNodePath(String root, String nodeName) {
   public static String getNodePath(String root, String nodeName) {
     return root + "/" + nodeName;
     return root + "/" + nodeName;
   }
   }
+
+  public void safeCreate(String path, byte[] data, List<ACL> acl,
+      CreateMode mode, List<ACL> fencingACL, String fencingNodePath)
+      throws Exception {
+    if (!exists(path)) {
+      SafeTransaction transaction = createTransaction(fencingACL,
+          fencingNodePath);
+      transaction.create(path, data, acl, mode);
+      transaction.commit();
+    }
+  }
+
+  /**
+   * Deletes the path. Checks for existence of path as well.
+   * @param path Path to be deleted.
+   * @throws Exception if any problem occurs while performing deletion.
+   */
+  public void safeDelete(final String path, List<ACL> fencingACL,
+      String fencingNodePath) throws Exception {
+    if (exists(path)) {
+      SafeTransaction transaction = createTransaction(fencingACL,
+          fencingNodePath);
+      transaction.delete(path);
+      transaction.commit();
+    }
+  }
+
+  public void safeSetData(String path, byte[] data, int version,
+      List<ACL> fencingACL, String fencingNodePath)
+      throws Exception {
+    SafeTransaction transaction = createTransaction(fencingACL,
+        fencingNodePath);
+    transaction.setData(path, data, version);
+    transaction.commit();
+  }
+
+  public SafeTransaction createTransaction(List<ACL> fencingACL,
+      String fencingNodePath) throws Exception {
+    return new SafeTransaction(fencingACL, fencingNodePath);
+  }
+
+  /**
+   * Use curator transactions to ensure zk-operations are performed in an all
+   * or nothing fashion. This is equivalent to using ZooKeeper#multi.
+   *
+   * TODO (YARN-3774): Curator 3.0 introduces CuratorOp similar to Op. We ll
+   * have to rewrite this inner class when we adopt that.
+   */
+  public class SafeTransaction {
+    private CuratorTransactionFinal transactionFinal;
+    private String fencingNodePath;
+
+    SafeTransaction(List<ACL> fencingACL, String fencingNodePath)
+        throws Exception {
+      this.fencingNodePath = fencingNodePath;
+      CuratorTransaction transaction = curator.inTransaction();
+      transactionFinal = transaction.create()
+          .withMode(CreateMode.PERSISTENT).withACL(fencingACL)
+          .forPath(fencingNodePath, new byte[0]).and();
+    }
+
+    public void commit() throws Exception {
+      transactionFinal = transactionFinal.delete()
+          .forPath(fencingNodePath).and();
+      transactionFinal.commit();
+    }
+
+    public void create(String path, byte[] data, List<ACL> acl, CreateMode mode)
+        throws Exception {
+      transactionFinal = transactionFinal.create()
+          .withMode(mode).withACL(acl).forPath(path, data).and();
+    }
+
+    public void delete(String path) throws Exception {
+      transactionFinal = transactionFinal.delete().forPath(path).and();
+    }
+
+    public void setData(String path, byte[] data, int version)
+        throws Exception {
+      transactionFinal = transactionFinal.setData()
+          .withVersion(version).forPath(path, data).and();
+    }
+  }
 }
 }

+ 2 - 0
hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/compress/bzip2/org_apache_hadoop_io_compress_bzip2.h

@@ -27,7 +27,9 @@
 
 
 #include "org_apache_hadoop.h"
 #include "org_apache_hadoop.h"
 
 
+#ifndef HADOOP_BZIP2_LIBRARY
 #define HADOOP_BZIP2_LIBRARY "libbz2.so.1"
 #define HADOOP_BZIP2_LIBRARY "libbz2.so.1"
+#endif
 
 
 
 
 /* A helper macro to convert the java 'stream-handle' to a bz_stream pointer. */
 /* A helper macro to convert the java 'stream-handle' to a bz_stream pointer. */

+ 9 - 1
hadoop-common-project/hadoop-common/src/main/resources/core-default.xml

@@ -1488,7 +1488,15 @@
     This flag is relevant only when fs.azure.authorization is enabled.
     This flag is relevant only when fs.azure.authorization is enabled.
   </description>
   </description>
 </property>
 </property>
-
+<property>
+  <name>fs.azure.saskey.usecontainersaskeyforallaccess</name>
+  <value>true</value>
+  <description>
+    Use container saskey for access to all blobs within the container.
+    Blob-specific saskeys are not used when this setting is enabled.
+    This setting provides better performance compared to blob-specific saskeys.
+  </description>
+</property>
 <property>
 <property>
   <name>io.seqfile.compress.blocksize</name>
   <name>io.seqfile.compress.blocksize</name>
   <value>1000000</value>
   <value>1000000</value>

+ 12 - 12
hadoop-common-project/hadoop-common/src/site/markdown/Metrics.md

@@ -179,8 +179,8 @@ Each metrics record contains tags such as ProcessName, SessionId, and Hostname a
 | `GenerateEDEKTimeAvgTime` | Average time of generating EDEK in milliseconds |
 | `GenerateEDEKTimeAvgTime` | Average time of generating EDEK in milliseconds |
 | `WarmUpEDEKTimeNumOps` | Total number of warming up EDEK |
 | `WarmUpEDEKTimeNumOps` | Total number of warming up EDEK |
 | `WarmUpEDEKTimeAvgTime` | Average time of warming up EDEK in milliseconds |
 | `WarmUpEDEKTimeAvgTime` | Average time of warming up EDEK in milliseconds |
-| `ResourceCheckTime`*num*`s(50|75|90|95|99)thPercentileLatency` | The 50/75/90/95/99th percentile of NameNode resource check latency in milliseconds. Percentile measurement is off by default, by watching no intervals. The intervals are specified by `dfs.metrics.percentiles.intervals`. |
-| `StorageBlockReport`*num*`s(50|75|90|95|99)thPercentileLatency` | The 50/75/90/95/99th percentile of storage block report latency in milliseconds. Percentile measurement is off by default, by watching no intervals. The intervals are specified by `dfs.metrics.percentiles.intervals`. |
+| `ResourceCheckTime`*num*`s(50/75/90/95/99)thPercentileLatency` | The 50/75/90/95/99th percentile of NameNode resource check latency in milliseconds. Percentile measurement is off by default, by watching no intervals. The intervals are specified by `dfs.metrics.percentiles.intervals`. |
+| `StorageBlockReport`*num*`s(50/75/90/95/99)thPercentileLatency` | The 50/75/90/95/99th percentile of storage block report latency in milliseconds. Percentile measurement is off by default, by watching no intervals. The intervals are specified by `dfs.metrics.percentiles.intervals`. |
 
 
 FSNamesystem
 FSNamesystem
 ------------
 ------------
@@ -240,8 +240,8 @@ Each metrics record contains tags such as HAState and Hostname as additional inf
 | `NumInMaintenanceLiveDataNodes` | Number of live Datanodes which are in maintenance state |
 | `NumInMaintenanceLiveDataNodes` | Number of live Datanodes which are in maintenance state |
 | `NumInMaintenanceDeadDataNodes` | Number of dead Datanodes which are in maintenance state |
 | `NumInMaintenanceDeadDataNodes` | Number of dead Datanodes which are in maintenance state |
 | `NumEnteringMaintenanceDataNodes` | Number of Datanodes that are entering the maintenance state |
 | `NumEnteringMaintenanceDataNodes` | Number of Datanodes that are entering the maintenance state |
-| `FSN(Read|Write)Lock`*OperationName*`NumOps` | Total number of acquiring lock by operations |
-| `FSN(Read|Write)Lock`*OperationName*`AvgTime` | Average time of holding the lock by operations in milliseconds |
+| `FSN(Read/Write)Lock`*OperationName*`NumOps` | Total number of acquiring lock by operations |
+| `FSN(Read/Write)Lock`*OperationName*`AvgTime` | Average time of holding the lock by operations in milliseconds |
 
 
 JournalNode
 JournalNode
 -----------
 -----------
@@ -308,13 +308,13 @@ Each metrics record contains tags such as SessionId and Hostname as additional i
 | `RamDiskBlocksEvictedWithoutRead` | Total number of blocks evicted in memory without ever being read from memory |
 | `RamDiskBlocksEvictedWithoutRead` | Total number of blocks evicted in memory without ever being read from memory |
 | `RamDiskBlocksEvictionWindowMsNumOps` | Number of blocks evicted in memory|
 | `RamDiskBlocksEvictionWindowMsNumOps` | Number of blocks evicted in memory|
 | `RamDiskBlocksEvictionWindowMsAvgTime` | Average time of blocks in memory before being evicted in milliseconds |
 | `RamDiskBlocksEvictionWindowMsAvgTime` | Average time of blocks in memory before being evicted in milliseconds |
-| `RamDiskBlocksEvictionWindows`*num*`s(50|75|90|95|99)thPercentileLatency` | The 50/75/90/95/99th percentile of latency between memory write and eviction in milliseconds. Percentile measurement is off by default, by watching no intervals. The intervals are specified by `dfs.metrics.percentiles.intervals`. |
+| `RamDiskBlocksEvictionWindows`*num*`s(50/75/90/95/99)thPercentileLatency` | The 50/75/90/95/99th percentile of latency between memory write and eviction in milliseconds. Percentile measurement is off by default, by watching no intervals. The intervals are specified by `dfs.metrics.percentiles.intervals`. |
 | `RamDiskBlocksLazyPersisted` | Total number of blocks written to disk by lazy writer |
 | `RamDiskBlocksLazyPersisted` | Total number of blocks written to disk by lazy writer |
 | `RamDiskBlocksDeletedBeforeLazyPersisted` | Total number of blocks deleted by application before being persisted to disk |
 | `RamDiskBlocksDeletedBeforeLazyPersisted` | Total number of blocks deleted by application before being persisted to disk |
 | `RamDiskBytesLazyPersisted` | Total number of bytes written to disk by lazy writer |
 | `RamDiskBytesLazyPersisted` | Total number of bytes written to disk by lazy writer |
 | `RamDiskBlocksLazyPersistWindowMsNumOps` | Number of blocks written to disk by lazy writer |
 | `RamDiskBlocksLazyPersistWindowMsNumOps` | Number of blocks written to disk by lazy writer |
 | `RamDiskBlocksLazyPersistWindowMsAvgTime` | Average time of blocks written to disk by lazy writer in milliseconds |
 | `RamDiskBlocksLazyPersistWindowMsAvgTime` | Average time of blocks written to disk by lazy writer in milliseconds |
-| `RamDiskBlocksLazyPersistWindows`*num*`s(50|75|90|95|99)thPercentileLatency` | The 50/75/90/95/99th percentile of latency between memory write and disk persist in milliseconds. Percentile measurement is off by default, by watching no intervals. The intervals are specified by `dfs.metrics.percentiles.intervals`. |
+| `RamDiskBlocksLazyPersistWindows`*num*`s(50/75/90/95/99)thPercentileLatency` | The 50/75/90/95/99th percentile of latency between memory write and disk persist in milliseconds. Percentile measurement is off by default, by watching no intervals. The intervals are specified by `dfs.metrics.percentiles.intervals`. |
 | `FsyncCount` | Total number of fsync |
 | `FsyncCount` | Total number of fsync |
 | `VolumeFailures` | Total number of volume failures occurred |
 | `VolumeFailures` | Total number of volume failures occurred |
 | `ReadBlockOpNumOps` | Total number of read operations |
 | `ReadBlockOpNumOps` | Total number of read operations |
@@ -380,23 +380,23 @@ contains tags such as Hostname as additional information along with metrics.
 | `TotalMetadataOperations` | Total number (monotonically increasing) of metadata operations. Metadata operations include stat, list, mkdir, delete, move, open and posix_fadvise. |
 | `TotalMetadataOperations` | Total number (monotonically increasing) of metadata operations. Metadata operations include stat, list, mkdir, delete, move, open and posix_fadvise. |
 | `MetadataOperationRateNumOps` | The number of metadata operations within an interval time of metric |
 | `MetadataOperationRateNumOps` | The number of metadata operations within an interval time of metric |
 | `MetadataOperationRateAvgTime` | Mean time of metadata operations in milliseconds |
 | `MetadataOperationRateAvgTime` | Mean time of metadata operations in milliseconds |
-| `MetadataOperationLatency`*num*`s(50|75|90|95|99)thPercentileLatency` | The 50/75/90/95/99th percentile of metadata operations latency in milliseconds. Percentile measurement is off by default, by watching no intervals. The intervals are specified by `dfs.metrics.percentiles.intervals`. |
+| `MetadataOperationLatency`*num*`s(50/75/90/95/99)thPercentileLatency` | The 50/75/90/95/99th percentile of metadata operations latency in milliseconds. Percentile measurement is off by default, by watching no intervals. The intervals are specified by `dfs.metrics.percentiles.intervals`. |
 | `TotalDataFileIos` | Total number (monotonically increasing) of data file io operations |
 | `TotalDataFileIos` | Total number (monotonically increasing) of data file io operations |
 | `DataFileIoRateNumOps` | The number of data file io operations within an interval time of metric |
 | `DataFileIoRateNumOps` | The number of data file io operations within an interval time of metric |
 | `DataFileIoRateAvgTime` | Mean time of data file io operations in milliseconds |
 | `DataFileIoRateAvgTime` | Mean time of data file io operations in milliseconds |
-| `DataFileIoLatency`*num*`s(50|75|90|95|99)thPercentileLatency` | The 50/75/90/95/99th percentile of data file io operations latency in milliseconds. Percentile measurement is off by default, by watching no intervals. The intervals are specified by `dfs.metrics.percentiles.intervals`. |
+| `DataFileIoLatency`*num*`s(50/75/90/95/99)thPercentileLatency` | The 50/75/90/95/99th percentile of data file io operations latency in milliseconds. Percentile measurement is off by default, by watching no intervals. The intervals are specified by `dfs.metrics.percentiles.intervals`. |
 | `FlushIoRateNumOps` | The number of file flush io operations within an interval time of metric |
 | `FlushIoRateNumOps` | The number of file flush io operations within an interval time of metric |
 | `FlushIoRateAvgTime` | Mean time of file flush io operations in milliseconds |
 | `FlushIoRateAvgTime` | Mean time of file flush io operations in milliseconds |
-| `FlushIoLatency`*num*`s(50|75|90|95|99)thPercentileLatency` | The 50/75/90/95/99th percentile of file flush io operations latency in milliseconds. Percentile measurement is off by default, by watching no intervals. The intervals are specified by `dfs.metrics.percentiles.intervals`. |
+| `FlushIoLatency`*num*`s(50/75/90/95/99)thPercentileLatency` | The 50/75/90/95/99th percentile of file flush io operations latency in milliseconds. Percentile measurement is off by default, by watching no intervals. The intervals are specified by `dfs.metrics.percentiles.intervals`. |
 | `SyncIoRateNumOps` | The number of file sync io operations within an interval time of metric |
 | `SyncIoRateNumOps` | The number of file sync io operations within an interval time of metric |
 | `SyncIoRateAvgTime` | Mean time of file sync io operations in milliseconds |
 | `SyncIoRateAvgTime` | Mean time of file sync io operations in milliseconds |
-| `SyncIoLatency`*num*`s(50|75|90|95|99)thPercentileLatency` | The 50/75/90/95/99th percentile of file sync io operations latency in milliseconds. Percentile measurement is off by default, by watching no intervals. The intervals are specified by `dfs.metrics.percentiles.intervals`. |
+| `SyncIoLatency`*num*`s(50/75/90/95/99)thPercentileLatency` | The 50/75/90/95/99th percentile of file sync io operations latency in milliseconds. Percentile measurement is off by default, by watching no intervals. The intervals are specified by `dfs.metrics.percentiles.intervals`. |
 | `ReadIoRateNumOps` | The number of file read io operations within an interval time of metric |
 | `ReadIoRateNumOps` | The number of file read io operations within an interval time of metric |
 | `ReadIoRateAvgTime` | Mean time of file read io operations in milliseconds |
 | `ReadIoRateAvgTime` | Mean time of file read io operations in milliseconds |
-| `ReadIoLatency`*num*`s(50|75|90|95|99)thPercentileLatency` | The 50/75/90/95/99th percentile of file read io operations latency in milliseconds. Percentile measurement is off by default, by watching no intervals. The intervals are specified by `dfs.metrics.percentiles.intervals`. |
+| `ReadIoLatency`*num*`s(50/75/90/95/99)thPercentileLatency` | The 50/75/90/95/99th percentile of file read io operations latency in milliseconds. Percentile measurement is off by default, by watching no intervals. The intervals are specified by `dfs.metrics.percentiles.intervals`. |
 | `WriteIoRateNumOps` | The number of file write io operations within an interval time of metric |
 | `WriteIoRateNumOps` | The number of file write io operations within an interval time of metric |
 | `WriteIoRateAvgTime` | Mean time of file write io operations in milliseconds |
 | `WriteIoRateAvgTime` | Mean time of file write io operations in milliseconds |
-| `WriteIoLatency`*num*`s(50|75|90|95|99)thPercentileLatency` | The 50/75/90/95/99th percentile of file write io operations latency in milliseconds. Percentile measurement is off by default, by watching no intervals. The intervals are specified by `dfs.metrics.percentiles.intervals`. |
+| `WriteIoLatency`*num*`s(50/75/90/95/99)thPercentileLatency` | The 50/75/90/95/99th percentile of file write io operations latency in milliseconds. Percentile measurement is off by default, by watching no intervals. The intervals are specified by `dfs.metrics.percentiles.intervals`. |
 | `TotalFileIoErrors` | Total number (monotonically increasing) of file io error operations |
 | `TotalFileIoErrors` | Total number (monotonically increasing) of file io error operations |
 | `FileIoErrorRateNumOps` | The number of file io error operations within an interval time of metric |
 | `FileIoErrorRateNumOps` | The number of file io error operations within an interval time of metric |
 | `FileIoErrorRateAvgTime` | It measures the mean time in milliseconds from the start of an operation to hitting a failure |
 | `FileIoErrorRateAvgTime` | It measures the mean time in milliseconds from the start of an operation to hitting a failure |

+ 31 - 2
hadoop-common-project/hadoop-common/src/site/markdown/filesystem/filesystem.md

@@ -553,7 +553,7 @@ on a path that exists and is a file. Instead the operation returns false.
        FS' = FS
        FS' = FS
        result = False
        result = False
 
 
-### `FSDataOutputStream create(Path, ...)`
+### <a name='FileSystem.create'></a> `FSDataOutputStream create(Path, ...)`
 
 
 
 
     FSDataOutputStream create(Path p,
     FSDataOutputStream create(Path p,
@@ -616,7 +616,24 @@ this precondition fails.
 
 
 * Not covered: symlinks. The resolved path of the symlink is used as the final path argument to the `create()` operation
 * Not covered: symlinks. The resolved path of the symlink is used as the final path argument to the `create()` operation
 
 
-### `FSDataOutputStream append(Path p, int bufferSize, Progressable progress)`
+### `FSDataOutputStreamBuilder createFile(Path p)`
+
+Make a `FSDataOutputStreamBuilder` to specify the parameters to create a file.
+
+#### Implementation Notes
+
+`createFile(p)` returns a `FSDataOutputStreamBuilder` only and does not make
+change on filesystem immediately. When `build()` is invoked on the `FSDataOutputStreamBuilder`,
+the builder parameters are verified and [`create(Path p)`](#FileSystem.create)
+is invoked on the underlying filesystem. `build()` has the same preconditions
+and postconditions as [`create(Path p)`](#FileSystem.create).
+
+* Similar to [`create(Path p)`](#FileSystem.create), files are overwritten
+by default, unless specify `builder.overwrite(false)`.
+* Unlike [`create(Path p)`](#FileSystem.create), missing parent directories are
+not created by default, unless specify `builder.recursive()`.
+
+### <a name='FileSystem.append'></a> `FSDataOutputStream append(Path p, int bufferSize, Progressable progress)`
 
 
 Implementations without a compliant call SHOULD throw `UnsupportedOperationException`.
 Implementations without a compliant call SHOULD throw `UnsupportedOperationException`.
 
 
@@ -634,6 +651,18 @@ Implementations without a compliant call SHOULD throw `UnsupportedOperationExcep
 Return: `FSDataOutputStream`, which can update the entry `FS.Files[p]`
 Return: `FSDataOutputStream`, which can update the entry `FS.Files[p]`
 by appending data to the existing list.
 by appending data to the existing list.
 
 
+### `FSDataOutputStreamBuilder appendFile(Path p)`
+
+Make a `FSDataOutputStreamBuilder` to specify the parameters to append to an
+existing file.
+
+#### Implementation Notes
+
+`appendFile(p)` returns a `FSDataOutputStreamBuilder` only and does not make
+change on filesystem immediately. When `build()` is invoked on the `FSDataOutputStreamBuilder`,
+the builder parameters are verified and [`append()`](#FileSystem.append) is
+invoked on the underlying filesystem. `build()` has the same preconditions and
+postconditions as [`append()`](#FileSystem.append).
 
 
 ### `FSDataInputStream open(Path f, int bufferSize)`
 ### `FSDataInputStream open(Path f, int bufferSize)`
 
 

+ 182 - 0
hadoop-common-project/hadoop-common/src/site/markdown/filesystem/fsdataoutputstreambuilder.md

@@ -0,0 +1,182 @@
+<!---
+  Licensed under the Apache License, Version 2.0 (the "License");
+  you may not use this file except in compliance with the License.
+  You may obtain a copy of the License at
+
+   http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License. See accompanying LICENSE file.
+-->
+
+<!--  ============================================================= -->
+<!--  CLASS: FSDataOutputStreamBuilder -->
+<!--  ============================================================= -->
+
+# class `org.apache.hadoop.fs.FSDataOutputStreamBuilder`
+
+<!-- MACRO{toc|fromDepth=1|toDepth=2} -->
+
+Builder pattern for `FSDataOutputStream` and its subclasses. It is used to
+create a new file or open an existing file on `FileSystem` for write.
+
+## Invariants
+
+The `FSDataOutputStreamBuilder` interface does not validate parameters
+and modify the state of `FileSystem` until [`build()`](#Builder.build) is
+invoked.
+
+## Implementation-agnostic parameters.
+
+### <a name="Builder.create"></a> `FSDataOutputStreamBuilder create()`
+
+Specify `FSDataOutputStreamBuilder` to create a file on `FileSystem`, equivalent
+to `CreateFlag#CREATE`.
+
+### <a name="Builder.append"></a> `FSDataOutputStreamBuilder append()`
+
+Specify `FSDataOutputStreamBuilder` to append to an existing file on
+`FileSystem`, equivalent to `CreateFlag#APPEND`.
+
+### <a name="Builder.overwrite"></a> `FSDataOutputStreamBuilder overwrite(boolean overwrite)`
+
+Specify `FSDataOutputStreamBuilder` to overwrite an existing file or not. If
+giving `overwrite==true`, it truncates an existing file, equivalent to
+`CreateFlag#OVERWITE`.
+
+### <a name="Builder.permission"></a> `FSDataOutputStreamBuilder permission(FsPermission permission)`
+
+Set permission for the file.
+
+### <a name="Builder.bufferSize"></a> `FSDataOutputStreamBuilder bufferSize(int bufSize)`
+
+Set the size of the buffer to be used.
+
+### <a name="Builder.replication"></a> `FSDataOutputStreamBuilder replication(short replica)`
+
+Set the replication factor.
+
+### <a name="Builder.blockSize"></a> `FSDataOutputStreamBuilder blockSize(long size)`
+
+Set block size in bytes.
+
+### <a name="Builder.recursive"></a> `FSDataOutputStreamBuilder recursive()`
+
+Create parent directories if they do not exist.
+
+### <a name="Builder.progress"></a> `FSDataOutputStreamBuilder progress(Progresable prog)`
+
+Set the facility of reporting progress.
+
+### <a name="Builder.checksumOpt"></a> `FSDataOutputStreamBuilder checksumOpt(ChecksumOpt chksumOpt)`
+
+Set checksum opt.
+
+### Set optional or mandatory parameters
+
+    FSDataOutputStreamBuilder opt(String key, ...)
+    FSDataOutputStreamBuilder must(String key, ...)
+
+Set optional or mandatory parameters to the builder. Using `opt()` or `must()`,
+client can specify FS-specific parameters without inspecting the concrete type
+of `FileSystem`.
+
+    // Don't
+    if (fs instanceof FooFileSystem) {
+        FooFileSystem fs = (FooFileSystem) fs;
+        out = dfs.createFile(path)
+            .optionA()
+            .optionB("value")
+            .cache()
+            .build()
+    } else if (fs instanceof BarFileSystem) {
+        ...
+    }
+
+    // Do
+    out = fs.createFile(path)
+        .permission(perm)
+        .bufferSize(bufSize)
+        .opt("foofs:option.a", true)
+        .opt("foofs:option.b", "value")
+        .opt("barfs:cache", true)
+        .must("foofs:cache", true)
+        .must("barfs:cache-size", 256 * 1024 * 1024)
+        .build();
+
+#### Implementation Notes
+
+The concrete `FileSystem` and/or `FSDataOutputStreamBuilder` implementation
+MUST verify that implementation-agnostic parameters (i.e., "syncable") or
+implementation-specific parameters (i.e., "foofs:cache")
+are supported. `FileSystem` will satisfy optional parameters (via `opt(key, ...)`)
+on best effort. If the mandatory parameters (via `must(key, ...)`) can not be satisfied
+in the `FileSystem`, `IllegalArgumentException` should be thrown in `build()`.
+
+The behavior of resolving the conflicts between the parameters set by
+builder methods (i.e., `bufferSize()`) and `opt()`/`must()` is undefined.
+
+## HDFS-specific parameters.
+
+`HdfsDataOutputStreamBuilder extends FSDataOutputStreamBuilder` provides additional
+HDFS-specific parameters, for further customize file creation / append behavior.
+
+### `FSDataOutpuStreamBuilder favoredNodes(InetSocketAddress[] nodes)`
+
+Set favored DataNodes for new blocks.
+
+### `FSDataOutputStreamBuilder syncBlock()`
+
+Force closed blocks to the disk device. See `CreateFlag#SYNC_BLOCK`
+
+### `FSDataOutputStreamBuilder lazyPersist()`
+
+Create the block on transient storage if possible.
+
+### `FSDataOutputStreamBuilder newBlock()`
+
+Append data to a new block instead of the end of the last partial block.
+
+### `FSDataOutputStreamBuilder noLocalWrite()`
+
+Advise that a block replica NOT be written to the local DataNode.
+
+### `FSDataOutputStreamBuilder ecPolicyName()`
+
+Enforce the file to be a striped file with erasure coding policy 'policyName',
+no matter what its parent directory's replication or erasure coding policy is.
+
+### `FSDataOutputStreamBuilder replicate()`
+
+Enforce the file to be a replicated file, no matter what its parent directory's
+replication or erasure coding policy is.
+
+## Builder interface
+
+### <a name="Builder.build"></a> `FSDataOutputStream build()`
+
+Create a new file or append an existing file on the underlying `FileSystem`,
+and return `FSDataOutputStream` for write.
+
+#### Preconditions
+
+The following combinations of parameters are not supported:
+
+    if APPEND|OVERWRITE: raise HadoopIllegalArgumentException
+    if CREATE|APPEND|OVERWRITE: raise HadoopIllegalArgumentExdeption
+
+`FileSystem` may reject the request for other reasons and throw `IOException`,
+see `FileSystem#create(path, ...)` and `FileSystem#append()`.
+
+#### Postconditions
+
+    FS' where :
+       FS'.Files'[p] == []
+       ancestors(p) is-subset-of FS'.Directories'
+
+    result = FSDataOutputStream
+
+The result is `FSDataOutputStream` to be used to write data to filesystem.

+ 1 - 0
hadoop-common-project/hadoop-common/src/site/markdown/filesystem/index.md

@@ -33,5 +33,6 @@ HDFS as these are commonly expected by Hadoop client applications.
 1. [Model](model.html)
 1. [Model](model.html)
 1. [FileSystem class](filesystem.html)
 1. [FileSystem class](filesystem.html)
 1. [FSDataInputStream class](fsdatainputstream.html)
 1. [FSDataInputStream class](fsdatainputstream.html)
+1. [FSDataOutputStreamBuilder class](fsdataoutputstreambuilder.html)
 2. [Testing with the Filesystem specification](testing.html)
 2. [Testing with the Filesystem specification](testing.html)
 2. [Extending the specification and its tests](extending.html)
 2. [Extending the specification and its tests](extending.html)

+ 1 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestCommonConfigurationFields.java

@@ -123,6 +123,7 @@ public class TestCommonConfigurationFields extends TestConfigurationFieldsBase {
     xmlPropsToSkipCompare.add("fs.azure.secure.mode");
     xmlPropsToSkipCompare.add("fs.azure.secure.mode");
     xmlPropsToSkipCompare.add("fs.azure.authorization");
     xmlPropsToSkipCompare.add("fs.azure.authorization");
     xmlPropsToSkipCompare.add("fs.azure.authorization.caching.enable");
     xmlPropsToSkipCompare.add("fs.azure.authorization.caching.enable");
+    xmlPropsToSkipCompare.add("fs.azure.saskey.usecontainersaskeyforallaccess");
     xmlPropsToSkipCompare.add("fs.azure.user.agent.prefix");
     xmlPropsToSkipCompare.add("fs.azure.user.agent.prefix");
 
 
     // Deprecated properties.  These should eventually be removed from the
     // Deprecated properties.  These should eventually be removed from the

File diff suppressed because it is too large
+ 183 - 114
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfiguration.java


+ 6 - 2
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfigurationSubclass.java

@@ -17,7 +17,8 @@
  */
  */
 package org.apache.hadoop.conf;
 package org.apache.hadoop.conf;
 
 
-import junit.framework.TestCase;
+import org.junit.Test;
+import static org.junit.Assert.*;
 
 
 import java.util.Properties;
 import java.util.Properties;
 
 
@@ -25,11 +26,12 @@ import java.util.Properties;
  * Created 21-Jan-2009 13:42:36
  * Created 21-Jan-2009 13:42:36
  */
  */
 
 
-public class TestConfigurationSubclass extends TestCase {
+public class TestConfigurationSubclass {
   private static final String EMPTY_CONFIGURATION_XML
   private static final String EMPTY_CONFIGURATION_XML
           = "/org/apache/hadoop/conf/empty-configuration.xml";
           = "/org/apache/hadoop/conf/empty-configuration.xml";
 
 
 
 
+  @Test
   public void testGetProps() {
   public void testGetProps() {
     SubConf conf = new SubConf(true);
     SubConf conf = new SubConf(true);
     Properties properties = conf.getProperties();
     Properties properties = conf.getProperties();
@@ -37,6 +39,7 @@ public class TestConfigurationSubclass extends TestCase {
             properties.getProperty("hadoop.tmp.dir"));
             properties.getProperty("hadoop.tmp.dir"));
   }
   }
 
 
+  @Test
   public void testReload() throws Throwable {
   public void testReload() throws Throwable {
     SubConf conf = new SubConf(true);
     SubConf conf = new SubConf(true);
     assertFalse(conf.isReloaded());
     assertFalse(conf.isReloaded());
@@ -45,6 +48,7 @@ public class TestConfigurationSubclass extends TestCase {
     Properties properties = conf.getProperties();
     Properties properties = conf.getProperties();
   }
   }
 
 
+  @Test
   public void testReloadNotQuiet() throws Throwable {
   public void testReloadNotQuiet() throws Throwable {
     SubConf conf = new SubConf(true);
     SubConf conf = new SubConf(true);
     conf.setQuietMode(false);
     conf.setQuietMode(false);

+ 4 - 4
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestDeprecatedKeys.java

@@ -21,15 +21,14 @@ package org.apache.hadoop.conf;
 import java.io.ByteArrayOutputStream;
 import java.io.ByteArrayOutputStream;
 import java.util.Map;
 import java.util.Map;
 
 
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.junit.Test;
 import org.junit.Test;
+import static org.junit.Assert.*;
 
 
-import junit.framework.TestCase;
-
-public class TestDeprecatedKeys extends TestCase {
+public class TestDeprecatedKeys {
  
  
   //Tests a deprecated key
   //Tests a deprecated key
+  @Test
   public void testDeprecatedKeys() throws Exception {
   public void testDeprecatedKeys() throws Exception {
     Configuration conf = new Configuration();
     Configuration conf = new Configuration();
     conf.set("topology.script.file.name", "xyz");
     conf.set("topology.script.file.name", "xyz");
@@ -39,6 +38,7 @@ public class TestDeprecatedKeys extends TestCase {
   }
   }
   
   
   //Tests reading / writing a conf file with deprecation after setting
   //Tests reading / writing a conf file with deprecation after setting
+  @Test
   public void testReadWriteWithDeprecatedKeys() throws Exception {
   public void testReadWriteWithDeprecatedKeys() throws Exception {
     Configuration conf = new Configuration();
     Configuration conf = new Configuration();
     conf.setBoolean("old.config.yet.to.be.deprecated", true);
     conf.setBoolean("old.config.yet.to.be.deprecated", true);

+ 5 - 3
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestGetInstances.java

@@ -18,10 +18,11 @@
 package org.apache.hadoop.conf;
 package org.apache.hadoop.conf;
 
 
 import java.util.List;
 import java.util.List;
+import org.junit.Test;
 
 
-import junit.framework.TestCase;
+import static org.junit.Assert.*;
 
 
-public class TestGetInstances extends TestCase {
+public class TestGetInstances {
   
   
   interface SampleInterface {}
   interface SampleInterface {}
   
   
@@ -30,7 +31,7 @@ public class TestGetInstances extends TestCase {
   static class SampleClass implements SampleInterface {
   static class SampleClass implements SampleInterface {
     SampleClass() {}
     SampleClass() {}
   }
   }
-	
+
   static class AnotherClass implements ChildInterface {
   static class AnotherClass implements ChildInterface {
     AnotherClass() {}
     AnotherClass() {}
   }
   }
@@ -39,6 +40,7 @@ public class TestGetInstances extends TestCase {
    * Makes sure <code>Configuration.getInstances()</code> returns
    * Makes sure <code>Configuration.getInstances()</code> returns
    * instances of the required type.
    * instances of the required type.
    */
    */
+  @Test
   public void testGetInstances() throws Exception {
   public void testGetInstances() throws Exception {
     Configuration conf = new Configuration();
     Configuration conf = new Configuration();
     
     

+ 88 - 25
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/key/TestKeyProviderCryptoExtension.java

@@ -22,6 +22,7 @@ import java.net.URI;
 import java.net.URISyntaxException;
 import java.net.URISyntaxException;
 import java.security.GeneralSecurityException;
 import java.security.GeneralSecurityException;
 import java.security.SecureRandom;
 import java.security.SecureRandom;
+import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Arrays;
 import java.util.List;
 import java.util.List;
 
 
@@ -40,7 +41,9 @@ import org.junit.rules.Timeout;
 import static org.apache.hadoop.crypto.key.KeyProvider.KeyVersion;
 import static org.apache.hadoop.crypto.key.KeyProvider.KeyVersion;
 import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 import static org.junit.Assert.fail;
 
 
 public class TestKeyProviderCryptoExtension {
 public class TestKeyProviderCryptoExtension {
@@ -90,13 +93,7 @@ public class TestKeyProviderCryptoExtension {
     KeyVersion k1 = kpExt.decryptEncryptedKey(ek1);
     KeyVersion k1 = kpExt.decryptEncryptedKey(ek1);
     assertEquals(KeyProviderCryptoExtension.EK, k1.getVersionName());
     assertEquals(KeyProviderCryptoExtension.EK, k1.getVersionName());
     assertEquals(encryptionKey.getMaterial().length, k1.getMaterial().length);
     assertEquals(encryptionKey.getMaterial().length, k1.getMaterial().length);
-    if (Arrays.equals(k1.getMaterial(), encryptionKey.getMaterial())) {
-      fail("Encrypted key material should not equal encryption key material");
-    }
-    if (Arrays.equals(ek1.getEncryptedKeyVersion().getMaterial(),
-        encryptionKey.getMaterial())) {
-      fail("Encrypted key material should not equal decrypted key material");
-    }
+
     // Decrypt it again and it should be the same
     // Decrypt it again and it should be the same
     KeyVersion k1a = kpExt.decryptEncryptedKey(ek1);
     KeyVersion k1a = kpExt.decryptEncryptedKey(ek1);
     assertArrayEquals(k1.getMaterial(), k1a.getMaterial());
     assertArrayEquals(k1.getMaterial(), k1a.getMaterial());
@@ -153,9 +150,6 @@ public class TestKeyProviderCryptoExtension {
     final KeyVersion k1 = kpExt.decryptEncryptedKey(ek1);
     final KeyVersion k1 = kpExt.decryptEncryptedKey(ek1);
     assertEquals(KeyProviderCryptoExtension.EK, k1.getVersionName());
     assertEquals(KeyProviderCryptoExtension.EK, k1.getVersionName());
     assertEquals(encryptionKey.getMaterial().length, k1.getMaterial().length);
     assertEquals(encryptionKey.getMaterial().length, k1.getMaterial().length);
-    if (Arrays.equals(k1.getMaterial(), encryptionKey.getMaterial())) {
-      fail("Encrypted key material should not equal encryption key material");
-    }
 
 
     // Roll the EK
     // Roll the EK
     kpExt.rollNewVersion(ek1.getEncryptionKeyName());
     kpExt.rollNewVersion(ek1.getEncryptionKeyName());
@@ -173,10 +167,7 @@ public class TestKeyProviderCryptoExtension {
     assertEquals("Length of encryption key material and EEK material should "
     assertEquals("Length of encryption key material and EEK material should "
             + "be the same", encryptionKey.getMaterial().length,
             + "be the same", encryptionKey.getMaterial().length,
         ek2.getEncryptedKeyVersion().getMaterial().length);
         ek2.getEncryptedKeyVersion().getMaterial().length);
-    if (Arrays.equals(ek2.getEncryptedKeyVersion().getMaterial(),
-        encryptionKey.getMaterial())) {
-      fail("Encrypted key material should not equal decrypted key material");
-    }
+
     if (Arrays.equals(ek2.getEncryptedKeyVersion().getMaterial(),
     if (Arrays.equals(ek2.getEncryptedKeyVersion().getMaterial(),
         ek1.getEncryptedKeyVersion().getMaterial())) {
         ek1.getEncryptedKeyVersion().getMaterial())) {
       fail("Re-encrypted EEK should have different material");
       fail("Re-encrypted EEK should have different material");
@@ -186,9 +177,6 @@ public class TestKeyProviderCryptoExtension {
     final KeyVersion k2 = kpExt.decryptEncryptedKey(ek2);
     final KeyVersion k2 = kpExt.decryptEncryptedKey(ek2);
     assertEquals(KeyProviderCryptoExtension.EK, k2.getVersionName());
     assertEquals(KeyProviderCryptoExtension.EK, k2.getVersionName());
     assertEquals(encryptionKey.getMaterial().length, k2.getMaterial().length);
     assertEquals(encryptionKey.getMaterial().length, k2.getMaterial().length);
-    if (Arrays.equals(k2.getMaterial(), encryptionKey.getMaterial())) {
-      fail("Encrypted key material should not equal encryption key material");
-    }
 
 
     // Re-encrypting the same EEK with the same EK should be deterministic
     // Re-encrypting the same EEK with the same EK should be deterministic
     final KeyProviderCryptoExtension.EncryptedKeyVersion ek2a =
     final KeyProviderCryptoExtension.EncryptedKeyVersion ek2a =
@@ -203,10 +191,7 @@ public class TestKeyProviderCryptoExtension {
     assertEquals("Length of encryption key material and EEK material should "
     assertEquals("Length of encryption key material and EEK material should "
             + "be the same", encryptionKey.getMaterial().length,
             + "be the same", encryptionKey.getMaterial().length,
         ek2a.getEncryptedKeyVersion().getMaterial().length);
         ek2a.getEncryptedKeyVersion().getMaterial().length);
-    if (Arrays.equals(ek2a.getEncryptedKeyVersion().getMaterial(),
-        encryptionKey.getMaterial())) {
-      fail("Encrypted key material should not equal decrypted key material");
-    }
+
     if (Arrays.equals(ek2a.getEncryptedKeyVersion().getMaterial(),
     if (Arrays.equals(ek2a.getEncryptedKeyVersion().getMaterial(),
         ek1.getEncryptedKeyVersion().getMaterial())) {
         ek1.getEncryptedKeyVersion().getMaterial())) {
       fail("Re-encrypted EEK should have different material");
       fail("Re-encrypted EEK should have different material");
@@ -227,10 +212,6 @@ public class TestKeyProviderCryptoExtension {
     assertEquals("Length of encryption key material and EEK material should "
     assertEquals("Length of encryption key material and EEK material should "
             + "be the same", encryptionKey.getMaterial().length,
             + "be the same", encryptionKey.getMaterial().length,
         ek3.getEncryptedKeyVersion().getMaterial().length);
         ek3.getEncryptedKeyVersion().getMaterial().length);
-    if (Arrays.equals(ek3.getEncryptedKeyVersion().getMaterial(),
-        encryptionKey.getMaterial())) {
-      fail("Encrypted key material should not equal decrypted key material");
-    }
 
 
     if (Arrays.equals(ek3.getEncryptedKeyVersion().getMaterial(),
     if (Arrays.equals(ek3.getEncryptedKeyVersion().getMaterial(),
         ek1.getEncryptedKeyVersion().getMaterial())) {
         ek1.getEncryptedKeyVersion().getMaterial())) {
@@ -240,6 +221,78 @@ public class TestKeyProviderCryptoExtension {
         ek3.getEncryptedKeyVersion().getMaterial());
         ek3.getEncryptedKeyVersion().getMaterial());
   }
   }
 
 
+  @Test
+  public void testReencryptEncryptedKeys() throws Exception {
+    List<EncryptedKeyVersion> ekvs = new ArrayList<>(4);
+    // Generate 2 new EEKs @v0 and add to the list
+    ekvs.add(kpExt.generateEncryptedKey(encryptionKey.getName()));
+    ekvs.add(kpExt.generateEncryptedKey(encryptionKey.getName()));
+
+    // Roll the EK
+    kpExt.rollNewVersion(ekvs.get(0).getEncryptionKeyName());
+    // Generate 1 new EEK @v1 add to the list.
+    ekvs.add(kpExt.generateEncryptedKey(encryptionKey.getName()));
+
+    // Roll the EK again
+    kpExt.rollNewVersion(ekvs.get(0).getEncryptionKeyName());
+    // Generate 1 new EEK @v2 add to the list.
+    ekvs.add(kpExt.generateEncryptedKey(encryptionKey.getName()));
+
+    // leave a deep copy of the original, for verification purpose.
+    List<EncryptedKeyVersion> ekvsOrig = new ArrayList<>(ekvs.size());
+    for (EncryptedKeyVersion ekv : ekvs) {
+      ekvsOrig.add(new EncryptedKeyVersion(ekv.getEncryptionKeyName(),
+          ekv.getEncryptionKeyVersionName(), ekv.getEncryptedKeyIv(),
+          ekv.getEncryptedKeyVersion()));
+    }
+
+    // Reencrypt ekvs
+    kpExt.reencryptEncryptedKeys(ekvs);
+
+    // Verify each ekv
+    for (int i = 0; i < ekvs.size(); ++i) {
+      final EncryptedKeyVersion ekv = ekvs.get(i);
+      final EncryptedKeyVersion orig = ekvsOrig.get(i);
+      assertEquals("Version name should be EEK",
+          KeyProviderCryptoExtension.EEK,
+          ekv.getEncryptedKeyVersion().getVersionName());
+      assertEquals("Encryption key name should be " + ENCRYPTION_KEY_NAME,
+          ENCRYPTION_KEY_NAME, ekv.getEncryptionKeyName());
+      assertNotNull("Expected encrypted key material",
+          ekv.getEncryptedKeyVersion().getMaterial());
+      assertEquals("Length of encryption key material and EEK material should "
+              + "be the same", encryptionKey.getMaterial().length,
+          ekv.getEncryptedKeyVersion().getMaterial().length);
+      assertFalse(
+          "Encrypted key material should not equal encryption key material",
+          Arrays.equals(ekv.getEncryptedKeyVersion().getMaterial(),
+              encryptionKey.getMaterial()));
+
+      if (i < 3) {
+        assertFalse("Re-encrypted EEK should have different material",
+            Arrays.equals(ekv.getEncryptedKeyVersion().getMaterial(),
+                orig.getEncryptedKeyVersion().getMaterial()));
+      } else {
+        assertTrue("Re-encrypted EEK should have same material",
+            Arrays.equals(ekv.getEncryptedKeyVersion().getMaterial(),
+                orig.getEncryptedKeyVersion().getMaterial()));
+      }
+
+      // Decrypt the new EEK into an EK and check it
+      final KeyVersion kv = kpExt.decryptEncryptedKey(ekv);
+      assertEquals(KeyProviderCryptoExtension.EK, kv.getVersionName());
+
+      // Decrypt it again and it should be the same
+      KeyVersion kv1 = kpExt.decryptEncryptedKey(ekv);
+      assertArrayEquals(kv.getMaterial(), kv1.getMaterial());
+
+      // Verify decrypting the new EEK and orig EEK gives the same material.
+      final KeyVersion origKv = kpExt.decryptEncryptedKey(orig);
+      assertTrue("Returned EEK and original EEK should both decrypt to the "
+          + "same kv.", Arrays.equals(origKv.getMaterial(), kv.getMaterial()));
+    }
+  }
+
   @Test
   @Test
   public void testNonDefaultCryptoExtensionSelectionWithCachingKeyProvider()
   public void testNonDefaultCryptoExtensionSelectionWithCachingKeyProvider()
           throws Exception {
           throws Exception {
@@ -341,6 +394,11 @@ public class TestKeyProviderCryptoExtension {
       return ekv;
       return ekv;
     }
     }
 
 
+    @Override
+    public void reencryptEncryptedKeys(List<EncryptedKeyVersion> ekvs)
+        throws IOException, GeneralSecurityException {
+    }
+
     @Override
     @Override
     public KeyVersion decryptEncryptedKey(
     public KeyVersion decryptEncryptedKey(
             EncryptedKeyVersion encryptedKeyVersion)
             EncryptedKeyVersion encryptedKeyVersion)
@@ -453,5 +511,10 @@ public class TestKeyProviderCryptoExtension {
         throws IOException, GeneralSecurityException {
         throws IOException, GeneralSecurityException {
       return ekv;
       return ekv;
     }
     }
+
+    @Override
+    public void reencryptEncryptedKeys(List<EncryptedKeyVersion> ekvs)
+        throws IOException, GeneralSecurityException {
+    }
   }
   }
 }
 }

+ 4 - 2
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestAvroFSInput.java

@@ -24,9 +24,10 @@ import java.io.OutputStreamWriter;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.test.GenericTestUtils;
 
 
-import junit.framework.TestCase;
+import org.junit.Test;
+import static org.junit.Assert.*;
 
 
-public class TestAvroFSInput extends TestCase {
+public class TestAvroFSInput {
 
 
   private static final String INPUT_DIR = "AvroFSInput";
   private static final String INPUT_DIR = "AvroFSInput";
 
 
@@ -34,6 +35,7 @@ public class TestAvroFSInput extends TestCase {
     return new Path(GenericTestUtils.getTempPath(INPUT_DIR));
     return new Path(GenericTestUtils.getTempPath(INPUT_DIR));
   }
   }
 
 
+  @Test
   public void testAFSInput() throws Exception {
   public void testAFSInput() throws Exception {
     Configuration conf = new Configuration();
     Configuration conf = new Configuration();
     FileSystem fs = FileSystem.getLocal(conf);
     FileSystem fs = FileSystem.getLocal(conf);

+ 13 - 6
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestDU.java

@@ -17,7 +17,10 @@
  */
  */
 package org.apache.hadoop.fs;
 package org.apache.hadoop.fs;
 
 
-import junit.framework.TestCase;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import static org.junit.Assert.*;
 
 
 import java.io.File;
 import java.io.File;
 import java.io.IOException;
 import java.io.IOException;
@@ -29,16 +32,16 @@ import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.test.GenericTestUtils;
 
 
 /** This test makes sure that "DU" does not get to run on each call to getUsed */
 /** This test makes sure that "DU" does not get to run on each call to getUsed */
-public class TestDU extends TestCase {
+public class TestDU {
   final static private File DU_DIR = GenericTestUtils.getTestDir("dutmp");
   final static private File DU_DIR = GenericTestUtils.getTestDir("dutmp");
 
 
-  @Override
+  @Before
   public void setUp() {
   public void setUp() {
-      FileUtil.fullyDelete(DU_DIR);
-      assertTrue(DU_DIR.mkdirs());
+    FileUtil.fullyDelete(DU_DIR);
+    assertTrue(DU_DIR.mkdirs());
   }
   }
 
 
-  @Override
+  @After
   public void tearDown() throws IOException {
   public void tearDown() throws IOException {
       FileUtil.fullyDelete(DU_DIR);
       FileUtil.fullyDelete(DU_DIR);
   }
   }
@@ -69,6 +72,7 @@ public class TestDU extends TestCase {
    * @throws IOException
    * @throws IOException
    * @throws InterruptedException
    * @throws InterruptedException
    */
    */
+  @Test
   public void testDU() throws IOException, InterruptedException {
   public void testDU() throws IOException, InterruptedException {
     final int writtenSize = 32*1024;   // writing 32K
     final int writtenSize = 32*1024;   // writing 32K
     // Allow for extra 4K on-disk slack for local file systems
     // Allow for extra 4K on-disk slack for local file systems
@@ -107,6 +111,8 @@ public class TestDU extends TestCase {
         duSize >= writtenSize &&
         duSize >= writtenSize &&
         writtenSize <= (duSize + slack));
         writtenSize <= (duSize + slack));
   }
   }
+
+  @Test
   public void testDUGetUsedWillNotReturnNegative() throws IOException {
   public void testDUGetUsedWillNotReturnNegative() throws IOException {
     File file = new File(DU_DIR, "data");
     File file = new File(DU_DIR, "data");
     assertTrue(file.createNewFile());
     assertTrue(file.createNewFile());
@@ -118,6 +124,7 @@ public class TestDU extends TestCase {
     assertTrue(String.valueOf(duSize), duSize >= 0L);
     assertTrue(String.valueOf(duSize), duSize >= 0L);
   }
   }
 
 
+  @Test
   public void testDUSetInitialValue() throws IOException {
   public void testDUSetInitialValue() throws IOException {
     File file = new File(DU_DIR, "dataX");
     File file = new File(DU_DIR, "dataX");
     createFile(file, 8192);
     createFile(file, 8192);

+ 5 - 3
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFilterFs.java

@@ -23,12 +23,12 @@ import java.lang.reflect.Modifier;
 import java.net.URI;
 import java.net.URI;
 import java.util.Iterator;
 import java.util.Iterator;
 
 
-import junit.framework.TestCase;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.viewfs.ConfigUtil;
 import org.apache.hadoop.fs.viewfs.ConfigUtil;
+import org.junit.Test;
 
 
-public class TestFilterFs extends TestCase {
+public class TestFilterFs {
 
 
   private static final Log LOG = FileSystem.LOG;
   private static final Log LOG = FileSystem.LOG;
 
 
@@ -41,7 +41,8 @@ public class TestFilterFs extends TestCase {
       return null;
       return null;
     }
     }
   }
   }
-  
+
+  @Test
   public void testFilterFileSystem() throws Exception {
   public void testFilterFileSystem() throws Exception {
     for (Method m : AbstractFileSystem.class.getDeclaredMethods()) {
     for (Method m : AbstractFileSystem.class.getDeclaredMethods()) {
       if (Modifier.isStatic(m.getModifiers()))
       if (Modifier.isStatic(m.getModifiers()))
@@ -69,6 +70,7 @@ public class TestFilterFs extends TestCase {
   
   
   // Test that FilterFs will accept an AbstractFileSystem to be filtered which
   // Test that FilterFs will accept an AbstractFileSystem to be filtered which
   // has an optional authority, such as ViewFs
   // has an optional authority, such as ViewFs
+  @Test
   public void testFilteringWithNonrequiredAuthority() throws Exception {
   public void testFilteringWithNonrequiredAuthority() throws Exception {
     Configuration conf = new Configuration();
     Configuration conf = new Configuration();
     ConfigUtil.addLink(conf, "custom", "/mnt", URI.create("file:///"));
     ConfigUtil.addLink(conf, "custom", "/mnt", URI.create("file:///"));

+ 13 - 12
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestGetFileBlockLocations.java

@@ -22,7 +22,10 @@ import java.util.Arrays;
 import java.util.Comparator;
 import java.util.Comparator;
 import java.util.Random;
 import java.util.Random;
 
 
-import junit.framework.TestCase;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import static org.junit.Assert.*;
 
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.test.GenericTestUtils;
@@ -30,7 +33,7 @@ import org.apache.hadoop.test.GenericTestUtils;
 /**
 /**
  * Testing the correctness of FileSystem.getFileBlockLocations.
  * Testing the correctness of FileSystem.getFileBlockLocations.
  */
  */
-public class TestGetFileBlockLocations extends TestCase {
+public class TestGetFileBlockLocations {
   private static String TEST_ROOT_DIR = GenericTestUtils.getTempPath(
   private static String TEST_ROOT_DIR = GenericTestUtils.getTempPath(
       "testGetFileBlockLocations");
       "testGetFileBlockLocations");
   private static final int FileLength = 4 * 1024 * 1024; // 4MB
   private static final int FileLength = 4 * 1024 * 1024; // 4MB
@@ -39,11 +42,8 @@ public class TestGetFileBlockLocations extends TestCase {
   private FileSystem fs;
   private FileSystem fs;
   private Random random;
   private Random random;
 
 
-  /**
-   * @see TestCase#setUp()
-   */
-  @Override
-  protected void setUp() throws IOException {
+  @Before
+  public void setUp() throws IOException {
     conf = new Configuration();
     conf = new Configuration();
     Path rootPath = new Path(TEST_ROOT_DIR);
     Path rootPath = new Path(TEST_ROOT_DIR);
     path = new Path(rootPath, "TestGetFileBlockLocations");
     path = new Path(rootPath, "TestGetFileBlockLocations");
@@ -91,15 +91,14 @@ public class TestGetFileBlockLocations extends TestCase {
       assertTrue(locations.length == 0);
       assertTrue(locations.length == 0);
     }
     }
   }
   }
-  /**
-   * @see TestCase#tearDown()
-   */
-  @Override
-  protected void tearDown() throws IOException {
+
+  @After
+  public void tearDown() throws IOException {
     fs.delete(path, true);
     fs.delete(path, true);
     fs.close();
     fs.close();
   }
   }
 
 
+  @Test
   public void testFailureNegativeParameters() throws IOException {
   public void testFailureNegativeParameters() throws IOException {
     FileStatus status = fs.getFileStatus(path);
     FileStatus status = fs.getFileStatus(path);
     try {
     try {
@@ -117,6 +116,7 @@ public class TestGetFileBlockLocations extends TestCase {
     }
     }
   }
   }
 
 
+  @Test
   public void testGetFileBlockLocations1() throws IOException {
   public void testGetFileBlockLocations1() throws IOException {
     FileStatus status = fs.getFileStatus(path);
     FileStatus status = fs.getFileStatus(path);
     oneTest(0, (int) status.getLen(), status);
     oneTest(0, (int) status.getLen(), status);
@@ -130,6 +130,7 @@ public class TestGetFileBlockLocations extends TestCase {
     }
     }
   }
   }
 
 
+  @Test
   public void testGetFileBlockLocations2() throws IOException {
   public void testGetFileBlockLocations2() throws IOException {
     FileStatus status = fs.getFileStatus(path);
     FileStatus status = fs.getFileStatus(path);
     for (int i = 0; i < 1000; ++i) {
     for (int i = 0; i < 1000; ++i) {

+ 5 - 2
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestGlobExpander.java

@@ -20,10 +20,12 @@ package org.apache.hadoop.fs;
 import java.io.IOException;
 import java.io.IOException;
 import java.util.List;
 import java.util.List;
 
 
-import junit.framework.TestCase;
+import org.junit.Test;
+import static org.junit.Assert.*;
 
 
-public class TestGlobExpander extends TestCase {
+public class TestGlobExpander {
 
 
+  @Test
   public void testExpansionIsIdentical() throws IOException {
   public void testExpansionIsIdentical() throws IOException {
     checkExpansionIsIdentical("");
     checkExpansionIsIdentical("");
     checkExpansionIsIdentical("/}");
     checkExpansionIsIdentical("/}");
@@ -35,6 +37,7 @@ public class TestGlobExpander extends TestCase {
     checkExpansionIsIdentical("p{a\\/b,c\\/d}s");
     checkExpansionIsIdentical("p{a\\/b,c\\/d}s");
   }
   }
 
 
+  @Test
   public void testExpansion() throws IOException {
   public void testExpansion() throws IOException {
     checkExpansion("{a/b}", "a/b");
     checkExpansion("{a/b}", "a/b");
     checkExpansion("/}{a/b}", "/}a/b");
     checkExpansion("/}{a/b}", "/}a/b");

+ 52 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestLocatedFileStatus.java

@@ -0,0 +1,52 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs;
+
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.junit.Test;
+
+import java.io.IOException;
+
+import static org.mockito.Mockito.mock;
+
+/**
+ * This class tests the LocatedFileStatus class.
+ */
+public class TestLocatedFileStatus {
+  @Test
+  public void testDeprecatedConstruction() throws IOException {
+    BlockLocation[] locs = new BlockLocation[] {mock(BlockLocation.class)};
+    final boolean isDir = false;
+    final int repl = 3;
+    final long blocksize = 64 * 1024 * 1024;
+    final long modificationTime = 0;
+    final long accessTime = 0;
+
+    // We should be able to pass null for the permission.
+    LocatedFileStatus lfsNullPerm = new LocatedFileStatus(1, isDir,
+        repl, blocksize, modificationTime, accessTime, null, null, null,
+        null, new Path("/some-file.txt"), locs);
+    FsPermission permission = mock(FsPermission.class);
+
+    // We should also be able to pass a permission or the permission.
+    LocatedFileStatus lfsNonNullPerm = new LocatedFileStatus(1, isDir,
+        repl, blocksize, modificationTime, accessTime, permission, null,
+        null, null, new Path("/some-file.txt"), locs);
+  }
+}

+ 13 - 11
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestTrash.java

@@ -33,20 +33,21 @@ import java.util.Random;
 import java.util.Set;
 import java.util.Set;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicInteger;
 
 
-import junit.framework.TestCase;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import static org.junit.Assert.*;
 
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.TrashPolicyDefault.Emptier;
 import org.apache.hadoop.fs.TrashPolicyDefault.Emptier;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.util.Time;
 import org.apache.hadoop.util.Time;
-import org.junit.Before;
-import org.junit.Test;
 
 
 /**
 /**
  * This class tests commands from Trash.
  * This class tests commands from Trash.
  */
  */
-public class TestTrash extends TestCase {
+public class TestTrash {
 
 
   private final static Path TEST_DIR = new Path(GenericTestUtils.getTempPath(
   private final static Path TEST_DIR = new Path(GenericTestUtils.getTempPath(
       "testTrash"));
       "testTrash"));
@@ -507,19 +508,22 @@ public class TestTrash extends TestCase {
     }
     }
   }
   }
 
 
+  @Test
   public void testTrash() throws IOException {
   public void testTrash() throws IOException {
     Configuration conf = new Configuration();
     Configuration conf = new Configuration();
     conf.setClass("fs.file.impl", TestLFS.class, FileSystem.class);
     conf.setClass("fs.file.impl", TestLFS.class, FileSystem.class);
     trashShell(FileSystem.getLocal(conf), TEST_DIR);
     trashShell(FileSystem.getLocal(conf), TEST_DIR);
   }
   }
 
 
+  @Test
   public void testNonDefaultFS() throws IOException {
   public void testNonDefaultFS() throws IOException {
     Configuration conf = new Configuration();
     Configuration conf = new Configuration();
     conf.setClass("fs.file.impl", TestLFS.class, FileSystem.class);
     conf.setClass("fs.file.impl", TestLFS.class, FileSystem.class);
     conf.set("fs.defaultFS", "invalid://host/bar/foo");
     conf.set("fs.defaultFS", "invalid://host/bar/foo");
     trashNonDefaultFS(conf);
     trashNonDefaultFS(conf);
   }
   }
-  
+
+  @Test
   public void testPluggableTrash() throws IOException {
   public void testPluggableTrash() throws IOException {
     Configuration conf = new Configuration();
     Configuration conf = new Configuration();
 
 
@@ -604,6 +608,7 @@ public class TestTrash extends TestCase {
     verifyTrashPermission(FileSystem.getLocal(conf), conf);
     verifyTrashPermission(FileSystem.getLocal(conf), conf);
   }
   }
 
 
+  @Test
   public void testTrashEmptier() throws Exception {
   public void testTrashEmptier() throws Exception {
     Configuration conf = new Configuration();
     Configuration conf = new Configuration();
     // Trash with 12 second deletes and 6 seconds checkpoints
     // Trash with 12 second deletes and 6 seconds checkpoints
@@ -665,12 +670,9 @@ public class TestTrash extends TestCase {
     emptierThread.interrupt();
     emptierThread.interrupt();
     emptierThread.join();
     emptierThread.join();
   }
   }
-  
-  /**
-   * @see TestCase#tearDown()
-   */
-  @Override
-  protected void tearDown() throws IOException {
+
+  @After
+  public void tearDown() throws IOException {
     File trashDir = new File(TEST_DIR.toUri().getPath());
     File trashDir = new File(TEST_DIR.toUri().getPath());
     if (trashDir.exists() && !FileUtil.fullyDelete(trashDir)) {
     if (trashDir.exists() && !FileUtil.fullyDelete(trashDir)) {
       throw new IOException("Cannot remove data directory: " + trashDir);
       throw new IOException("Cannot remove data directory: " + trashDir);

+ 4 - 2
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestTruncatedInputBug.java

@@ -20,16 +20,17 @@ package org.apache.hadoop.fs;
 import java.io.DataOutputStream;
 import java.io.DataOutputStream;
 import java.io.IOException;
 import java.io.IOException;
 
 
-import junit.framework.TestCase;
+import static org.junit.Assert.*;
 
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.test.GenericTestUtils;
+import org.junit.Test;
 
 
 /**
 /**
  * test for the input truncation bug when mark/reset is used.
  * test for the input truncation bug when mark/reset is used.
  * HADOOP-1489
  * HADOOP-1489
  */
  */
-public class TestTruncatedInputBug extends TestCase {
+public class TestTruncatedInputBug {
   private static String TEST_ROOT_DIR =
   private static String TEST_ROOT_DIR =
       GenericTestUtils.getTestDir().getAbsolutePath();
       GenericTestUtils.getTestDir().getAbsolutePath();
   
   
@@ -49,6 +50,7 @@ public class TestTruncatedInputBug extends TestCase {
    * checksum file system currently depends on the request size
    * checksum file system currently depends on the request size
    * >= bytesPerSum to work properly.
    * >= bytesPerSum to work properly.
    */
    */
+  @Test
   public void testTruncatedInputBug() throws IOException {
   public void testTruncatedInputBug() throws IOException {
     final int ioBufSize = 512;
     final int ioBufSize = 512;
     final int fileSize = ioBufSize*4;
     final int fileSize = ioBufSize*4;

+ 12 - 2
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/permission/TestFsPermission.java

@@ -21,11 +21,14 @@ import java.io.IOException;
 
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 
 
-import junit.framework.TestCase;
+import org.junit.Test;
+import static org.junit.Assert.*;
 
 
 import static org.apache.hadoop.fs.permission.FsAction.*;
 import static org.apache.hadoop.fs.permission.FsAction.*;
 
 
-public class TestFsPermission extends TestCase {
+public class TestFsPermission {
+
+  @Test
   public void testFsAction() {
   public void testFsAction() {
     //implies
     //implies
     for(FsAction a : FsAction.values()) {
     for(FsAction a : FsAction.values()) {
@@ -53,6 +56,7 @@ public class TestFsPermission extends TestCase {
    * Ensure that when manually specifying permission modes we get
    * Ensure that when manually specifying permission modes we get
    * the expected values back out for all combinations
    * the expected values back out for all combinations
    */
    */
+  @Test
   public void testConvertingPermissions() {
   public void testConvertingPermissions() {
     for(short s = 0; s <= 01777; s++) {
     for(short s = 0; s <= 01777; s++) {
       assertEquals(s, new FsPermission(s).toShort());
       assertEquals(s, new FsPermission(s).toShort());
@@ -80,6 +84,7 @@ public class TestFsPermission extends TestCase {
     assertEquals(02000, s);
     assertEquals(02000, s);
   }
   }
 
 
+  @Test
   public void testSpecialBitsToString() {
   public void testSpecialBitsToString() {
     for (boolean sb : new boolean[] { false, true }) {
     for (boolean sb : new boolean[] { false, true }) {
       for (FsAction u : FsAction.values()) {
       for (FsAction u : FsAction.values()) {
@@ -106,6 +111,7 @@ public class TestFsPermission extends TestCase {
     }
     }
   }
   }
 
 
+  @Test
   public void testFsPermission() {
   public void testFsPermission() {
     String symbolic = "-rwxrwxrwx";
     String symbolic = "-rwxrwxrwx";
 
 
@@ -132,6 +138,7 @@ public class TestFsPermission extends TestCase {
     }
     }
   }
   }
 
 
+  @Test
   public void testSymbolicPermission() {
   public void testSymbolicPermission() {
     for (int i = 0; i < SYMBOLIC.length; ++i) {
     for (int i = 0; i < SYMBOLIC.length; ++i) {
       short val = 0777;
       short val = 0777;
@@ -146,6 +153,7 @@ public class TestFsPermission extends TestCase {
     }
     }
   }
   }
 
 
+  @Test
   public void testUMaskParser() throws IOException {
   public void testUMaskParser() throws IOException {
     Configuration conf = new Configuration();
     Configuration conf = new Configuration();
     
     
@@ -163,6 +171,7 @@ public class TestFsPermission extends TestCase {
     }
     }
   }
   }
 
 
+  @Test
   public void testSymbolicUmasks() {
   public void testSymbolicUmasks() {
     Configuration conf = new Configuration();
     Configuration conf = new Configuration();
     
     
@@ -176,6 +185,7 @@ public class TestFsPermission extends TestCase {
     assertEquals(0111, FsPermission.getUMask(conf).toShort());
     assertEquals(0111, FsPermission.getUMask(conf).toShort());
   }
   }
 
 
+  @Test
   public void testBadUmasks() {
   public void testBadUmasks() {
     Configuration conf = new Configuration();
     Configuration conf = new Configuration();
     
     

+ 13 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpServer.java

@@ -682,4 +682,17 @@ public class TestHttpServer extends HttpServerFunctionalTest {
       stopHttpServer(myServer2);
       stopHttpServer(myServer2);
     }
     }
   }
   }
+
+  @Test
+  public void testBacklogSize() throws Exception
+  {
+    final int backlogSize = 2048;
+    Configuration conf = new Configuration();
+    conf.setInt(HttpServer2.HTTP_SOCKET_BACKLOG_SIZE_KEY, backlogSize);
+    HttpServer2 srv = createServer("test", conf);
+    List<?> listeners = (List<?>) Whitebox.getInternalState(srv,
+            "listeners");
+    ServerConnector listener = (ServerConnector)listeners.get(0);
+    assertEquals(backlogSize, listener.getAcceptQueueSize());
+  }
 }
 }

+ 241 - 1
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpServerWithSpengo.java

@@ -22,9 +22,12 @@ import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.minikdc.MiniKdc;
 import org.apache.hadoop.minikdc.MiniKdc;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.AuthenticationFilterInitializer;
 import org.apache.hadoop.security.AuthenticationFilterInitializer;
+import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod;
 import org.apache.hadoop.security.authentication.KerberosTestUtils;
 import org.apache.hadoop.security.authentication.KerberosTestUtils;
 import org.apache.hadoop.security.authentication.client.AuthenticatedURL;
 import org.apache.hadoop.security.authentication.client.AuthenticatedURL;
+import org.apache.hadoop.security.authentication.client.AuthenticationException;
 import org.apache.hadoop.security.authentication.server.AuthenticationFilter;
 import org.apache.hadoop.security.authentication.server.AuthenticationFilter;
 import org.apache.hadoop.security.authentication.server.AuthenticationToken;
 import org.apache.hadoop.security.authentication.server.AuthenticationToken;
 import org.apache.hadoop.security.authentication.util.Signer;
 import org.apache.hadoop.security.authentication.util.Signer;
@@ -32,6 +35,7 @@ import org.apache.hadoop.security.authentication.util.SignerSecretProvider;
 import org.apache.hadoop.security.authentication.util.StringSignerSecretProviderCreator;
 import org.apache.hadoop.security.authentication.util.StringSignerSecretProviderCreator;
 import org.apache.hadoop.security.authorize.AccessControlList;
 import org.apache.hadoop.security.authorize.AccessControlList;
 import org.apache.hadoop.security.authorize.ProxyUsers;
 import org.apache.hadoop.security.authorize.ProxyUsers;
+import org.ietf.jgss.GSSException;
 import org.junit.AfterClass;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.Test;
@@ -45,7 +49,14 @@ import java.io.Writer;
 import java.net.HttpURLConnection;
 import java.net.HttpURLConnection;
 import java.net.URI;
 import java.net.URI;
 import java.net.URL;
 import java.net.URL;
+import java.security.AccessController;
+import java.security.PrivilegedExceptionAction;
+import java.util.HashSet;
 import java.util.Properties;
 import java.util.Properties;
+import java.util.Set;
+import javax.security.auth.Subject;
+import javax.servlet.ServletContext;
+
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.assertTrue;
 
 
 /**
 /**
@@ -72,16 +83,25 @@ public class TestHttpServerWithSpengo {
   private static MiniKdc testMiniKDC;
   private static MiniKdc testMiniKDC;
   private static File secretFile = new File(testRootDir, SECRET_STR);
   private static File secretFile = new File(testRootDir, SECRET_STR);
 
 
+  private static UserGroupInformation authUgi;
+
   @BeforeClass
   @BeforeClass
   public static void setUp() throws Exception {
   public static void setUp() throws Exception {
     try {
     try {
       testMiniKDC = new MiniKdc(MiniKdc.createConf(), testRootDir);
       testMiniKDC = new MiniKdc(MiniKdc.createConf(), testRootDir);
       testMiniKDC.start();
       testMiniKDC.start();
       testMiniKDC.createPrincipal(
       testMiniKDC.createPrincipal(
-          httpSpnegoKeytabFile, HTTP_USER + "/localhost");
+          httpSpnegoKeytabFile, HTTP_USER + "/localhost", "keytab-user");
     } catch (Exception e) {
     } catch (Exception e) {
       assertTrue("Couldn't setup MiniKDC", false);
       assertTrue("Couldn't setup MiniKDC", false);
     }
     }
+
+    System.setProperty("sun.security.krb5.debug", "true");
+    Configuration conf = new Configuration();
+    SecurityUtil.setAuthenticationMethod(AuthenticationMethod.KERBEROS, conf);
+    UserGroupInformation.setConfiguration(conf);
+    authUgi = UserGroupInformation.loginUserFromKeytabAndReturnUGI(
+        "keytab-user", httpSpnegoKeytabFile.toString());
     Writer w = new FileWriter(secretFile);
     Writer w = new FileWriter(secretFile);
     w.write("secret");
     w.write("secret");
     w.close();
     w.close();
@@ -209,6 +229,226 @@ public class TestHttpServerWithSpengo {
     }
     }
   }
   }
 
 
+  @Test
+  public void testSessionCookie() throws Exception {
+    Configuration conf = new Configuration();
+    conf.set(HttpServer2.FILTER_INITIALIZER_PROPERTY,
+        AuthenticationFilterInitializer.class.getName());
+    conf.set(PREFIX + "type", "kerberos");
+    conf.setBoolean(PREFIX + "simple.anonymous.allowed", false);
+    conf.set(PREFIX + "signer.secret.provider",
+        TestSignerSecretProvider.class.getName());
+
+    conf.set(PREFIX + "kerberos.keytab",
+        httpSpnegoKeytabFile.getAbsolutePath());
+    conf.set(PREFIX + "kerberos.principal", httpSpnegoPrincipal);
+    conf.set(PREFIX + "cookie.domain", realm);
+    conf.setBoolean(CommonConfigurationKeys.HADOOP_SECURITY_AUTHORIZATION,
+        true);
+
+    //setup logs dir
+    System.setProperty("hadoop.log.dir", testRootDir.getAbsolutePath());
+
+    HttpServer2 httpServer = null;
+    // Create http server to test.
+    httpServer = getCommonBuilder()
+        .setConf(conf)
+        .build();
+    httpServer.start();
+
+    // Get signer to encrypt token
+    final Signer signer = new Signer(new TestSignerSecretProvider());
+    final AuthenticatedURL authUrl = new AuthenticatedURL();
+
+    final URL url = new URL("http://" + NetUtils.getHostPortString(
+        httpServer.getConnectorAddress(0)) + "/conf");
+
+    // this illustrates an inconsistency with AuthenticatedURL.  the
+    // authenticator is only called when the token is not set.  if the
+    // authenticator fails then it must throw an AuthenticationException to
+    // the caller, yet the caller may see 401 for subsequent requests
+    // that require re-authentication like token expiration.
+    final UserGroupInformation simpleUgi =
+        UserGroupInformation.createRemoteUser("simple-user");
+
+    authUgi.doAs(new PrivilegedExceptionAction<Void>() {
+      @Override
+      public Void run() throws Exception {
+        TestSignerSecretProvider.rollSecret();
+        HttpURLConnection conn = null;
+        AuthenticatedURL.Token token = new AuthenticatedURL.Token();
+
+        // initial request should trigger authentication and set the token.
+        conn = authUrl.openConnection(url, token);
+        Assert.assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode());
+        Assert.assertTrue(token.isSet());
+        String cookie = token.toString();
+
+        // token should not change.
+        conn = authUrl.openConnection(url, token);
+        Assert.assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode());
+        Assert.assertTrue(token.isSet());
+        Assert.assertEquals(cookie, token.toString());
+
+        // roll secret to invalidate token.
+        TestSignerSecretProvider.rollSecret();
+        conn = authUrl.openConnection(url, token);
+        // this may or may not happen.  under normal circumstances the
+        // jdk will silently renegotiate and the client never sees a 401.
+        // however in some cases the jdk will give up doing spnego.  since
+        // the token is already set, the authenticator isn't invoked (which
+        // would do the spnego if the jdk doesn't), which causes the client
+        // to see a 401.
+        if (conn.getResponseCode() == HttpURLConnection.HTTP_UNAUTHORIZED) {
+          // if this happens, the token should be cleared which means the
+          // next request should succeed and receive a new token.
+          Assert.assertFalse(token.isSet());
+          conn = authUrl.openConnection(url, token);
+        }
+
+        // token should change.
+        Assert.assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode());
+        Assert.assertTrue(token.isSet());
+        Assert.assertNotEquals(cookie, token.toString());
+        cookie = token.toString();
+
+        // token should not change.
+        for (int i=0; i < 3; i++) {
+          conn = authUrl.openConnection(url, token);
+          Assert.assertEquals("attempt"+i,
+              HttpURLConnection.HTTP_OK, conn.getResponseCode());
+          Assert.assertTrue(token.isSet());
+          Assert.assertEquals(cookie, token.toString());
+        }
+
+        // blow out the kerberos creds test only auth token is used.
+        Subject s = Subject.getSubject(AccessController.getContext());
+        Set<Object> oldCreds = new HashSet<>(s.getPrivateCredentials());
+        s.getPrivateCredentials().clear();
+
+        // token should not change.
+        for (int i=0; i < 3; i++) {
+          try {
+            conn = authUrl.openConnection(url, token);
+            Assert.assertEquals("attempt"+i,
+                HttpURLConnection.HTTP_OK, conn.getResponseCode());
+          } catch (AuthenticationException ae) {
+            Assert.fail("attempt"+i+" "+ae);
+          }
+          Assert.assertTrue(token.isSet());
+          Assert.assertEquals(cookie, token.toString());
+        }
+
+        // invalidate token.  connections should fail now and token should be
+        // unset.
+        TestSignerSecretProvider.rollSecret();
+        conn = authUrl.openConnection(url, token);
+        Assert.assertEquals(
+            HttpURLConnection.HTTP_UNAUTHORIZED, conn.getResponseCode());
+        Assert.assertFalse(token.isSet());
+        Assert.assertEquals("", token.toString());
+
+        // restore the kerberos creds, should work again.
+        s.getPrivateCredentials().addAll(oldCreds);
+        conn = authUrl.openConnection(url, token);
+        Assert.assertEquals(
+            HttpURLConnection.HTTP_OK, conn.getResponseCode());
+        Assert.assertTrue(token.isSet());
+        cookie = token.toString();
+
+        // token should not change.
+        for (int i=0; i < 3; i++) {
+          conn = authUrl.openConnection(url, token);
+          Assert.assertEquals("attempt"+i,
+              HttpURLConnection.HTTP_OK, conn.getResponseCode());
+          Assert.assertTrue(token.isSet());
+          Assert.assertEquals(cookie, token.toString());
+        }
+        return null;
+      }
+    });
+
+    simpleUgi.doAs(new PrivilegedExceptionAction<Void>() {
+      @Override
+      public Void run() throws Exception {
+        TestSignerSecretProvider.rollSecret();
+        AuthenticatedURL authUrl = new AuthenticatedURL();
+        AuthenticatedURL.Token token = new AuthenticatedURL.Token();
+        HttpURLConnection conn = null;
+
+        // initial connect with unset token will trigger authenticator which
+        // should fail since we have no creds and leave token unset.
+        try {
+          authUrl.openConnection(url, token);
+          Assert.fail("should fail with no credentials");
+        } catch (AuthenticationException ae) {
+          Assert.assertNotNull(ae.getCause());
+          Assert.assertEquals(GSSException.class, ae.getCause().getClass());
+          GSSException gsse = (GSSException)ae.getCause();
+          Assert.assertEquals(GSSException.NO_CRED, gsse.getMajor());
+        } catch (Throwable t) {
+          Assert.fail("Unexpected exception" + t);
+        }
+        Assert.assertFalse(token.isSet());
+
+        // create a valid token and save its value.
+        token = getEncryptedAuthToken(signer, "valid");
+        String cookie = token.toString();
+
+        // server should accept token.  after the request the token should
+        // be set to the same value (ie. server didn't reissue cookie)
+        conn = authUrl.openConnection(url, token);
+        Assert.assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode());
+        Assert.assertTrue(token.isSet());
+        Assert.assertEquals(cookie, token.toString());
+
+        conn = authUrl.openConnection(url, token);
+        Assert.assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode());
+        Assert.assertTrue(token.isSet());
+        Assert.assertEquals(cookie, token.toString());
+
+        // change the secret to effectively invalidate the cookie.  see above
+        // regarding inconsistency.  the authenticator has no way to know the
+        // token is bad, so the client will encounter a 401 instead of
+        // AuthenticationException.
+        TestSignerSecretProvider.rollSecret();
+        conn = authUrl.openConnection(url, token);
+        Assert.assertEquals(
+            HttpURLConnection.HTTP_UNAUTHORIZED, conn.getResponseCode());
+        Assert.assertFalse(token.isSet());
+        Assert.assertEquals("", token.toString());
+        return null;
+      }
+    });
+  }
+
+  public static class TestSignerSecretProvider extends SignerSecretProvider {
+    static int n = 0;
+    static byte[] secret;
+
+    static void rollSecret() {
+      secret = ("secret[" + (n++) + "]").getBytes();
+    }
+
+    public TestSignerSecretProvider() {
+    }
+
+    @Override
+    public void init(Properties config, ServletContext servletContext,
+            long tokenValidity) throws Exception {
+      rollSecret();
+    }
+
+    @Override
+    public byte[] getCurrentSecret() {
+      return secret;
+    }
+
+    @Override
+    public byte[][] getAllSecrets() {
+      return new byte[][]{secret};
+    }
+  }
 
 
   private AuthenticatedURL.Token getEncryptedAuthToken(Signer signer,
   private AuthenticatedURL.Token getEncryptedAuthToken(Signer signer,
       String user) throws Exception {
       String user) throws Exception {

+ 29 - 4
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestFairCallQueue.java

@@ -25,7 +25,9 @@ import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 import static org.mockito.Mockito.when;
 import static org.mockito.Mockito.times;
 import static org.mockito.Mockito.times;
 
 
-import junit.framework.TestCase;
+import org.junit.Before;
+import org.junit.Test;
+import static org.junit.Assert.*;
 
 
 import javax.management.MBeanServer;
 import javax.management.MBeanServer;
 import javax.management.ObjectName;
 import javax.management.ObjectName;
@@ -39,13 +41,12 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.List;
 import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.BlockingQueue;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
-import org.junit.Test;
 import org.mockito.Mockito;
 import org.mockito.Mockito;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.ipc.CallQueueManager.CallQueueOverflowException;
 import org.apache.hadoop.ipc.CallQueueManager.CallQueueOverflowException;
 import org.apache.hadoop.ipc.protobuf.RpcHeaderProtos.RpcResponseHeaderProto.RpcStatusProto;
 import org.apache.hadoop.ipc.protobuf.RpcHeaderProtos.RpcResponseHeaderProto.RpcStatusProto;
 
 
-public class TestFairCallQueue extends TestCase {
+public class TestFairCallQueue {
   private FairCallQueue<Schedulable> fcq;
   private FairCallQueue<Schedulable> fcq;
 
 
   private Schedulable mockCall(String id, int priority) {
   private Schedulable mockCall(String id, int priority) {
@@ -65,6 +66,7 @@ public class TestFairCallQueue extends TestCase {
   }
   }
 
 
   @SuppressWarnings("deprecation")
   @SuppressWarnings("deprecation")
+  @Before
   public void setUp() {
   public void setUp() {
     Configuration conf = new Configuration();
     Configuration conf = new Configuration();
     conf.setInt("ns." + FairCallQueue.IPC_CALLQUEUE_PRIORITY_LEVELS_KEY, 2);
     conf.setInt("ns." + FairCallQueue.IPC_CALLQUEUE_PRIORITY_LEVELS_KEY, 2);
@@ -74,6 +76,7 @@ public class TestFairCallQueue extends TestCase {
 
 
   // Validate that the total capacity of all subqueues equals
   // Validate that the total capacity of all subqueues equals
   // the maxQueueSize for different values of maxQueueSize
   // the maxQueueSize for different values of maxQueueSize
+  @Test
   public void testTotalCapacityOfSubQueues() {
   public void testTotalCapacityOfSubQueues() {
     Configuration conf = new Configuration();
     Configuration conf = new Configuration();
     FairCallQueue<Schedulable> fairCallQueue;
     FairCallQueue<Schedulable> fairCallQueue;
@@ -291,11 +294,12 @@ public class TestFairCallQueue extends TestCase {
 
 
   //
   //
   // Ensure that FairCallQueue properly implements BlockingQueue
   // Ensure that FairCallQueue properly implements BlockingQueue
-  //
+  @Test
   public void testPollReturnsNullWhenEmpty() {
   public void testPollReturnsNullWhenEmpty() {
     assertNull(fcq.poll());
     assertNull(fcq.poll());
   }
   }
 
 
+  @Test
   public void testPollReturnsTopCallWhenNotEmpty() {
   public void testPollReturnsTopCallWhenNotEmpty() {
     Schedulable call = mockCall("c");
     Schedulable call = mockCall("c");
     assertTrue(fcq.offer(call));
     assertTrue(fcq.offer(call));
@@ -306,6 +310,7 @@ public class TestFairCallQueue extends TestCase {
     assertEquals(0, fcq.size());
     assertEquals(0, fcq.size());
   }
   }
 
 
+  @Test
   public void testOfferSucceeds() {
   public void testOfferSucceeds() {
 
 
     for (int i = 0; i < 5; i++) {
     for (int i = 0; i < 5; i++) {
@@ -316,6 +321,7 @@ public class TestFairCallQueue extends TestCase {
     assertEquals(5, fcq.size());
     assertEquals(5, fcq.size());
   }
   }
 
 
+  @Test
   public void testOfferFailsWhenFull() {
   public void testOfferFailsWhenFull() {
     for (int i = 0; i < 5; i++) { assertTrue(fcq.offer(mockCall("c"))); }
     for (int i = 0; i < 5; i++) { assertTrue(fcq.offer(mockCall("c"))); }
 
 
@@ -324,6 +330,7 @@ public class TestFairCallQueue extends TestCase {
     assertEquals(5, fcq.size());
     assertEquals(5, fcq.size());
   }
   }
 
 
+  @Test
   public void testOfferSucceedsWhenScheduledLowPriority() {
   public void testOfferSucceedsWhenScheduledLowPriority() {
     // Scheduler will schedule into queue 0 x 5, then queue 1
     // Scheduler will schedule into queue 0 x 5, then queue 1
     int mockedPriorities[] = {0, 0, 0, 0, 0, 1, 0};
     int mockedPriorities[] = {0, 0, 0, 0, 0, 1, 0};
@@ -334,10 +341,12 @@ public class TestFairCallQueue extends TestCase {
     assertEquals(6, fcq.size());
     assertEquals(6, fcq.size());
   }
   }
 
 
+  @Test
   public void testPeekNullWhenEmpty() {
   public void testPeekNullWhenEmpty() {
     assertNull(fcq.peek());
     assertNull(fcq.peek());
   }
   }
 
 
+  @Test
   public void testPeekNonDestructive() {
   public void testPeekNonDestructive() {
     Schedulable call = mockCall("c", 0);
     Schedulable call = mockCall("c", 0);
     assertTrue(fcq.offer(call));
     assertTrue(fcq.offer(call));
@@ -347,6 +356,7 @@ public class TestFairCallQueue extends TestCase {
     assertEquals(1, fcq.size());
     assertEquals(1, fcq.size());
   }
   }
 
 
+  @Test
   public void testPeekPointsAtHead() {
   public void testPeekPointsAtHead() {
     Schedulable call = mockCall("c", 0);
     Schedulable call = mockCall("c", 0);
     Schedulable next = mockCall("b", 0);
     Schedulable next = mockCall("b", 0);
@@ -356,10 +366,12 @@ public class TestFairCallQueue extends TestCase {
     assertEquals(call, fcq.peek()); // Peek points at the head
     assertEquals(call, fcq.peek()); // Peek points at the head
   }
   }
 
 
+  @Test
   public void testPollTimeout() throws InterruptedException {
   public void testPollTimeout() throws InterruptedException {
     assertNull(fcq.poll(10, TimeUnit.MILLISECONDS));
     assertNull(fcq.poll(10, TimeUnit.MILLISECONDS));
   }
   }
 
 
+  @Test
   public void testPollSuccess() throws InterruptedException {
   public void testPollSuccess() throws InterruptedException {
     Schedulable call = mockCall("c", 0);
     Schedulable call = mockCall("c", 0);
     assertTrue(fcq.offer(call));
     assertTrue(fcq.offer(call));
@@ -369,6 +381,7 @@ public class TestFairCallQueue extends TestCase {
     assertEquals(0, fcq.size());
     assertEquals(0, fcq.size());
   }
   }
 
 
+  @Test
   public void testOfferTimeout() throws InterruptedException {
   public void testOfferTimeout() throws InterruptedException {
     for (int i = 0; i < 5; i++) {
     for (int i = 0; i < 5; i++) {
       assertTrue(fcq.offer(mockCall("c"), 10, TimeUnit.MILLISECONDS));
       assertTrue(fcq.offer(mockCall("c"), 10, TimeUnit.MILLISECONDS));
@@ -380,6 +393,7 @@ public class TestFairCallQueue extends TestCase {
   }
   }
 
 
   @SuppressWarnings("deprecation")
   @SuppressWarnings("deprecation")
+  @Test
   public void testDrainTo() {
   public void testDrainTo() {
     Configuration conf = new Configuration();
     Configuration conf = new Configuration();
     conf.setInt("ns." + FairCallQueue.IPC_CALLQUEUE_PRIORITY_LEVELS_KEY, 2);
     conf.setInt("ns." + FairCallQueue.IPC_CALLQUEUE_PRIORITY_LEVELS_KEY, 2);
@@ -397,6 +411,7 @@ public class TestFairCallQueue extends TestCase {
   }
   }
 
 
   @SuppressWarnings("deprecation")
   @SuppressWarnings("deprecation")
+  @Test
   public void testDrainToWithLimit() {
   public void testDrainToWithLimit() {
     Configuration conf = new Configuration();
     Configuration conf = new Configuration();
     conf.setInt("ns." + FairCallQueue.IPC_CALLQUEUE_PRIORITY_LEVELS_KEY, 2);
     conf.setInt("ns." + FairCallQueue.IPC_CALLQUEUE_PRIORITY_LEVELS_KEY, 2);
@@ -413,16 +428,19 @@ public class TestFairCallQueue extends TestCase {
     assertEquals(2, fcq2.size());
     assertEquals(2, fcq2.size());
   }
   }
 
 
+  @Test
   public void testInitialRemainingCapacity() {
   public void testInitialRemainingCapacity() {
     assertEquals(10, fcq.remainingCapacity());
     assertEquals(10, fcq.remainingCapacity());
   }
   }
 
 
+  @Test
   public void testFirstQueueFullRemainingCapacity() {
   public void testFirstQueueFullRemainingCapacity() {
     while (fcq.offer(mockCall("c"))) ; // Queue 0 will fill up first, then queue 1
     while (fcq.offer(mockCall("c"))) ; // Queue 0 will fill up first, then queue 1
 
 
     assertEquals(5, fcq.remainingCapacity());
     assertEquals(5, fcq.remainingCapacity());
   }
   }
 
 
+  @Test
   public void testAllQueuesFullRemainingCapacity() {
   public void testAllQueuesFullRemainingCapacity() {
     int[] mockedPriorities = {0, 0, 0, 0, 0, 1, 1, 1, 1, 1, 0};
     int[] mockedPriorities = {0, 0, 0, 0, 0, 1, 1, 1, 1, 1, 0};
     int i = 0;
     int i = 0;
@@ -432,6 +450,7 @@ public class TestFairCallQueue extends TestCase {
     assertEquals(10, fcq.size());
     assertEquals(10, fcq.size());
   }
   }
 
 
+  @Test
   public void testQueuesPartialFilledRemainingCapacity() {
   public void testQueuesPartialFilledRemainingCapacity() {
     int[] mockedPriorities = {0, 1, 0, 1, 0};
     int[] mockedPriorities = {0, 1, 0, 1, 0};
     for (int i = 0; i < 5; i++) { fcq.offer(mockCall("c", mockedPriorities[i])); }
     for (int i = 0; i < 5; i++) { fcq.offer(mockCall("c", mockedPriorities[i])); }
@@ -555,12 +574,14 @@ public class TestFairCallQueue extends TestCase {
   }
   }
 
 
   // Make sure put will overflow into lower queues when the top is full
   // Make sure put will overflow into lower queues when the top is full
+  @Test
   public void testPutOverflows() throws InterruptedException {
   public void testPutOverflows() throws InterruptedException {
     // We can fit more than 5, even though the scheduler suggests the top queue
     // We can fit more than 5, even though the scheduler suggests the top queue
     assertCanPut(fcq, 8, 8);
     assertCanPut(fcq, 8, 8);
     assertEquals(8, fcq.size());
     assertEquals(8, fcq.size());
   }
   }
 
 
+  @Test
   public void testPutBlocksWhenAllFull() throws InterruptedException {
   public void testPutBlocksWhenAllFull() throws InterruptedException {
     assertCanPut(fcq, 10, 10); // Fill up
     assertCanPut(fcq, 10, 10); // Fill up
     assertEquals(10, fcq.size());
     assertEquals(10, fcq.size());
@@ -569,10 +590,12 @@ public class TestFairCallQueue extends TestCase {
     assertCanPut(fcq, 0, 1); // Will block
     assertCanPut(fcq, 0, 1); // Will block
   }
   }
 
 
+  @Test
   public void testTakeBlocksWhenEmpty() throws InterruptedException {
   public void testTakeBlocksWhenEmpty() throws InterruptedException {
     assertCanTake(fcq, 0, 1);
     assertCanTake(fcq, 0, 1);
   }
   }
 
 
+  @Test
   public void testTakeRemovesCall() throws InterruptedException {
   public void testTakeRemovesCall() throws InterruptedException {
     Schedulable call = mockCall("c");
     Schedulable call = mockCall("c");
     fcq.offer(call);
     fcq.offer(call);
@@ -581,6 +604,7 @@ public class TestFairCallQueue extends TestCase {
     assertEquals(0, fcq.size());
     assertEquals(0, fcq.size());
   }
   }
 
 
+  @Test
   public void testTakeTriesNextQueue() throws InterruptedException {
   public void testTakeTriesNextQueue() throws InterruptedException {
 
 
     // A mux which only draws from q 0
     // A mux which only draws from q 0
@@ -597,6 +621,7 @@ public class TestFairCallQueue extends TestCase {
     assertEquals(0, fcq.size());
     assertEquals(0, fcq.size());
   }
   }
 
 
+  @Test
   public void testFairCallQueueMXBean() throws Exception {
   public void testFairCallQueueMXBean() throws Exception {
     MBeanServer mbs = ManagementFactory.getPlatformMBeanServer();
     MBeanServer mbs = ManagementFactory.getPlatformMBeanServer();
     ObjectName mxbeanName = new ObjectName(
     ObjectName mxbeanName = new ObjectName(

+ 3 - 3
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/log/TestLog4Json.java

@@ -20,7 +20,8 @@ package org.apache.hadoop.log;
 
 
 import com.fasterxml.jackson.databind.JsonNode;
 import com.fasterxml.jackson.databind.JsonNode;
 import com.fasterxml.jackson.databind.node.ContainerNode;
 import com.fasterxml.jackson.databind.node.ContainerNode;
-import junit.framework.TestCase;
+import org.junit.Test;
+import static org.junit.Assert.*;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.util.Time;
 import org.apache.hadoop.util.Time;
@@ -33,7 +34,6 @@ import org.apache.log4j.spi.HierarchyEventListener;
 import org.apache.log4j.spi.LoggerFactory;
 import org.apache.log4j.spi.LoggerFactory;
 import org.apache.log4j.spi.LoggerRepository;
 import org.apache.log4j.spi.LoggerRepository;
 import org.apache.log4j.spi.ThrowableInformation;
 import org.apache.log4j.spi.ThrowableInformation;
-import org.junit.Test;
 
 
 import java.io.IOException;
 import java.io.IOException;
 import java.io.StringWriter;
 import java.io.StringWriter;
@@ -42,7 +42,7 @@ import java.net.NoRouteToHostException;
 import java.util.Enumeration;
 import java.util.Enumeration;
 import java.util.Vector;
 import java.util.Vector;
 
 
-public class TestLog4Json extends TestCase {
+public class TestLog4Json {
 
 
   private static final Log LOG = LogFactory.getLog(TestLog4Json.class);
   private static final Log LOG = LogFactory.getLog(TestLog4Json.class);
 
 

+ 3 - 6
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestScriptBasedMapping.java

@@ -19,15 +19,12 @@ package org.apache.hadoop.net;
 
 
 import java.util.ArrayList;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.List;
-
-import org.apache.hadoop.conf.Configuration;
-
-import junit.framework.TestCase;
 import org.junit.Test;
 import org.junit.Test;
+import static org.junit.Assert.*;
 
 
-public class TestScriptBasedMapping extends TestCase {
-
+import org.apache.hadoop.conf.Configuration;
 
 
+public class TestScriptBasedMapping {
   
   
   public TestScriptBasedMapping() {
   public TestScriptBasedMapping() {
 
 

+ 3 - 4
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestScriptBasedMappingWithDependency.java

@@ -19,13 +19,12 @@ package org.apache.hadoop.net;
 
 
 import java.util.ArrayList;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.List;
+import org.junit.Test;
+import static org.junit.Assert.*;
 
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 
 
-import junit.framework.TestCase;
-import org.junit.Test;
-
-public class TestScriptBasedMappingWithDependency extends TestCase {
+public class TestScriptBasedMappingWithDependency {
 
 
   
   
   public TestScriptBasedMappingWithDependency() {
   public TestScriptBasedMappingWithDependency() {

+ 4 - 2
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestAuthenticationFilter.java

@@ -17,19 +17,21 @@
 package org.apache.hadoop.security;
 package org.apache.hadoop.security;
 
 
 
 
-import junit.framework.TestCase;
+import static org.junit.Assert.*;
 import org.apache.hadoop.http.HttpServer2;
 import org.apache.hadoop.http.HttpServer2;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.http.FilterContainer;
 import org.apache.hadoop.http.FilterContainer;
+import org.junit.Test;
 import org.mockito.Mockito;
 import org.mockito.Mockito;
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
 import org.mockito.stubbing.Answer;
 
 
 import java.util.Map;
 import java.util.Map;
 
 
-public class TestAuthenticationFilter extends TestCase {
+public class TestAuthenticationFilter {
 
 
   @SuppressWarnings("unchecked")
   @SuppressWarnings("unchecked")
+  @Test
   public void testConfiguration() throws Exception {
   public void testConfiguration() throws Exception {
     Configuration conf = new Configuration();
     Configuration conf = new Configuration();
     conf.set("hadoop.http.authentication.foo", "bar");
     conf.set("hadoop.http.authentication.foo", "bar");

+ 4 - 3
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestAuthenticationWithProxyUserFilter.java

@@ -16,8 +16,8 @@
  */
  */
 package org.apache.hadoop.security;
 package org.apache.hadoop.security;
 
 
-
-import junit.framework.TestCase;
+import org.junit.Test;
+import static org.junit.Assert.*;
 import org.apache.hadoop.http.HttpServer2;
 import org.apache.hadoop.http.HttpServer2;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.http.FilterContainer;
 import org.apache.hadoop.http.FilterContainer;
@@ -30,9 +30,10 @@ import java.util.Map;
  * This class is tested for {@link AuthenticationWithProxyUserFilter}
  * This class is tested for {@link AuthenticationWithProxyUserFilter}
  * to verify configurations of this filter.
  * to verify configurations of this filter.
  */
  */
-public class TestAuthenticationWithProxyUserFilter extends TestCase {
+public class TestAuthenticationWithProxyUserFilter {
 
 
   @SuppressWarnings("unchecked")
   @SuppressWarnings("unchecked")
+  @Test
   public void testConfiguration() throws Exception {
   public void testConfiguration() throws Exception {
     Configuration conf = new Configuration();
     Configuration conf = new Configuration();
     conf.set("hadoop.http.authentication.foo", "bar");
     conf.set("hadoop.http.authentication.foo", "bar");

+ 6 - 3
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestWhitelistBasedResolver.java

@@ -21,17 +21,18 @@ import java.io.IOException;
 import java.net.InetAddress;
 import java.net.InetAddress;
 import java.util.Map;
 import java.util.Map;
 
 
-import junit.framework.TestCase;
+import org.junit.Test;
+import static org.junit.Assert.*;
 
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.security.WhitelistBasedResolver;
 import org.apache.hadoop.util.TestFileBasedIPList;
 import org.apache.hadoop.util.TestFileBasedIPList;
 
 
-public class TestWhitelistBasedResolver extends TestCase {
+public class TestWhitelistBasedResolver {
 
 
   public static final Map<String, String> SASL_PRIVACY_PROPS =
   public static final Map<String, String> SASL_PRIVACY_PROPS =
     WhitelistBasedResolver.getSaslProperties(new Configuration());
     WhitelistBasedResolver.getSaslProperties(new Configuration());
 
 
+  @Test
   public void testFixedVariableAndLocalWhiteList() throws IOException {
   public void testFixedVariableAndLocalWhiteList() throws IOException {
 
 
     String[] fixedIps = {"10.119.103.112", "10.221.102.0/23"};
     String[] fixedIps = {"10.119.103.112", "10.221.102.0/23"};
@@ -79,6 +80,7 @@ public class TestWhitelistBasedResolver extends TestCase {
    * Check  for inclusion in whitelist
    * Check  for inclusion in whitelist
    * Check for exclusion from whitelist
    * Check for exclusion from whitelist
    */
    */
+  @Test
   public void testFixedAndLocalWhiteList() throws IOException {
   public void testFixedAndLocalWhiteList() throws IOException {
 
 
     String[] fixedIps = {"10.119.103.112", "10.221.102.0/23"};
     String[] fixedIps = {"10.119.103.112", "10.221.102.0/23"};
@@ -128,6 +130,7 @@ public class TestWhitelistBasedResolver extends TestCase {
    * Add a bunch of subnets and IPSs to the whitelist
    * Add a bunch of subnets and IPSs to the whitelist
    * Check  for inclusion in whitelist with a null value
    * Check  for inclusion in whitelist with a null value
    */
    */
+  @Test
   public void testNullIPAddress() throws IOException {
   public void testNullIPAddress() throws IOException {
 
 
     String[] fixedIps = {"10.119.103.112", "10.221.102.0/23"};
     String[] fixedIps = {"10.119.103.112", "10.221.102.0/23"};

+ 8 - 4
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/TestToken.java

@@ -25,11 +25,12 @@ import org.apache.hadoop.io.*;
 import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenIdentifier;
 import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenIdentifier;
 import org.apache.hadoop.security.token.delegation.TestDelegationToken.TestDelegationTokenIdentifier;
 import org.apache.hadoop.security.token.delegation.TestDelegationToken.TestDelegationTokenIdentifier;
 import org.apache.hadoop.security.token.delegation.TestDelegationToken.TestDelegationTokenSecretManager;
 import org.apache.hadoop.security.token.delegation.TestDelegationToken.TestDelegationTokenSecretManager;
+import org.junit.Test;
 
 
-import junit.framework.TestCase;
+import static org.junit.Assert.*;
 
 
 /** Unit tests for Token */
 /** Unit tests for Token */
-public class TestToken extends TestCase {
+public class TestToken {
 
 
   static boolean isEqual(Object a, Object b) {
   static boolean isEqual(Object a, Object b) {
     return a == null ? b == null : a.equals(b);
     return a == null ? b == null : a.equals(b);
@@ -45,6 +46,7 @@ public class TestToken extends TestCase {
   /**
   /**
    * Test token serialization
    * Test token serialization
    */
    */
+  @Test
   public void testTokenSerialization() throws IOException {
   public void testTokenSerialization() throws IOException {
     // Get a token
     // Get a token
     Token<TokenIdentifier> sourceToken = new Token<TokenIdentifier>();
     Token<TokenIdentifier> sourceToken = new Token<TokenIdentifier>();
@@ -76,7 +78,8 @@ public class TestToken extends TestCase {
     }
     }
   }
   }
 
 
-  public static void testEncodeWritable() throws Exception {
+  @Test
+  public void testEncodeWritable() throws Exception {
     String[] values = new String[]{"", "a", "bb", "ccc", "dddd", "eeeee",
     String[] values = new String[]{"", "a", "bb", "ccc", "dddd", "eeeee",
         "ffffff", "ggggggg", "hhhhhhhh", "iiiiiiiii",
         "ffffff", "ggggggg", "hhhhhhhh", "iiiiiiiii",
         "abcdefghijklmnopqrstuvwxyzABCDEFGHIJKLM" +
         "abcdefghijklmnopqrstuvwxyzABCDEFGHIJKLM" +
@@ -96,7 +99,8 @@ public class TestToken extends TestCase {
       checkUrlSafe(encode);
       checkUrlSafe(encode);
     }
     }
   }
   }
-  
+
+  @Test
   public void testDecodeIdentifier() throws IOException {
   public void testDecodeIdentifier() throws IOException {
     TestDelegationTokenSecretManager secretManager =
     TestDelegationTokenSecretManager secretManager =
       new TestDelegationTokenSecretManager(0, 0, 0, 0);
       new TestDelegationTokenSecretManager(0, 0, 0, 0);

+ 2 - 4
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestAsyncDiskService.java

@@ -17,17 +17,15 @@
  */
  */
 package org.apache.hadoop.util;
 package org.apache.hadoop.util;
 
 
-import junit.framework.TestCase;
-
-import org.apache.hadoop.util.AsyncDiskService;
 import org.junit.Test;
 import org.junit.Test;
+import static org.junit.Assert.*;
 import org.slf4j.Logger;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.slf4j.LoggerFactory;
 
 
 /**
 /**
  * A test for AsyncDiskService.
  * A test for AsyncDiskService.
  */
  */
-public class TestAsyncDiskService extends TestCase {
+public class TestAsyncDiskService {
   
   
   public static final Logger LOG =
   public static final Logger LOG =
       LoggerFactory.getLogger(TestAsyncDiskService.class);
       LoggerFactory.getLogger(TestAsyncDiskService.class);

+ 7 - 6
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestCacheableIPList.java

@@ -18,14 +18,11 @@
 package org.apache.hadoop.util;
 package org.apache.hadoop.util;
 
 
 import java.io.IOException;
 import java.io.IOException;
+import org.junit.Test;
 
 
-import org.apache.hadoop.util.CacheableIPList;
-import org.apache.hadoop.util.FileBasedIPList;
+import static org.junit.Assert.*;
 
 
-
-import junit.framework.TestCase;
-
-public class TestCacheableIPList extends TestCase {
+public class TestCacheableIPList {
 
 
   /**
   /**
    * Add a bunch of subnets and IPSs to the file
    * Add a bunch of subnets and IPSs to the file
@@ -37,6 +34,7 @@ public class TestCacheableIPList extends TestCase {
    * test for inclusion
    * test for inclusion
    * Check for exclusion
    * Check for exclusion
    */
    */
+  @Test
   public void testAddWithSleepForCacheTimeout() throws IOException, InterruptedException {
   public void testAddWithSleepForCacheTimeout() throws IOException, InterruptedException {
 
 
     String[] ips = {"10.119.103.112", "10.221.102.0/23", "10.113.221.221"};
     String[] ips = {"10.119.103.112", "10.221.102.0/23", "10.113.221.221"};
@@ -76,6 +74,7 @@ public class TestCacheableIPList extends TestCase {
    * test for inclusion
    * test for inclusion
    * Check for exclusion
    * Check for exclusion
    */
    */
+  @Test
   public void testRemovalWithSleepForCacheTimeout() throws IOException, InterruptedException {
   public void testRemovalWithSleepForCacheTimeout() throws IOException, InterruptedException {
 
 
     String[] ips = {"10.119.103.112", "10.221.102.0/23",
     String[] ips = {"10.119.103.112", "10.221.102.0/23",
@@ -115,6 +114,7 @@ public class TestCacheableIPList extends TestCase {
    * test for inclusion
    * test for inclusion
    * Check for exclusion
    * Check for exclusion
    */
    */
+  @Test
   public void testAddWithRefresh() throws IOException, InterruptedException {
   public void testAddWithRefresh() throws IOException, InterruptedException {
 
 
     String[] ips = {"10.119.103.112", "10.221.102.0/23", "10.113.221.221"};
     String[] ips = {"10.119.103.112", "10.221.102.0/23", "10.113.221.221"};
@@ -154,6 +154,7 @@ public class TestCacheableIPList extends TestCase {
    * test for inclusion
    * test for inclusion
    * Check for exclusion
    * Check for exclusion
    */
    */
+  @Test
   public void testRemovalWithRefresh() throws IOException, InterruptedException {
   public void testRemovalWithRefresh() throws IOException, InterruptedException {
 
 
     String[] ips = {"10.119.103.112", "10.221.102.0/23",
     String[] ips = {"10.119.103.112", "10.221.102.0/23",

+ 7 - 5
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestFileBasedIPList.java

@@ -22,14 +22,11 @@ import java.io.IOException;
 import java.util.Arrays;
 import java.util.Arrays;
 
 
 import org.apache.commons.io.FileUtils;
 import org.apache.commons.io.FileUtils;
-import org.apache.hadoop.util.FileBasedIPList;
-import org.apache.hadoop.util.IPList;
 import org.junit.After;
 import org.junit.After;
 import org.junit.Test;
 import org.junit.Test;
+import static org.junit.Assert.*;
 
 
-import junit.framework.TestCase;
-
-public class TestFileBasedIPList extends TestCase {
+public class TestFileBasedIPList {
 
 
   @After
   @After
   public void tearDown() {
   public void tearDown() {
@@ -127,6 +124,7 @@ public class TestFileBasedIPList extends TestCase {
    * test for inclusion
    * test for inclusion
    * should be true as if the feature is turned off
    * should be true as if the feature is turned off
    */
    */
+  @Test
   public void testFileNotSpecified() {
   public void testFileNotSpecified() {
 
 
     IPList ipl = new FileBasedIPList(null);
     IPList ipl = new FileBasedIPList(null);
@@ -140,6 +138,7 @@ public class TestFileBasedIPList extends TestCase {
    * test for inclusion
    * test for inclusion
    * should be true as if the feature is turned off
    * should be true as if the feature is turned off
    */
    */
+  @Test
   public void testFileMissing() {
   public void testFileMissing() {
 
 
     IPList ipl = new FileBasedIPList("missingips.txt");
     IPList ipl = new FileBasedIPList("missingips.txt");
@@ -153,6 +152,7 @@ public class TestFileBasedIPList extends TestCase {
    * test for inclusion
    * test for inclusion
    * should be true as if the feature is turned off
    * should be true as if the feature is turned off
    */
    */
+  @Test
   public void testWithEmptyList() throws IOException {
   public void testWithEmptyList() throws IOException {
     String[] ips = {};
     String[] ips = {};
 
 
@@ -168,6 +168,7 @@ public class TestFileBasedIPList extends TestCase {
    * test for inclusion
    * test for inclusion
    * should be true as if the feature is turned off
    * should be true as if the feature is turned off
    */
    */
+  @Test
   public void testForBadFIle() throws IOException {
   public void testForBadFIle() throws IOException {
     String[] ips = { "10.221.102/23"};
     String[] ips = { "10.221.102/23"};
 
 
@@ -187,6 +188,7 @@ public class TestFileBasedIPList extends TestCase {
    * Check  for inclusion with good entries
    * Check  for inclusion with good entries
    * Check for exclusion
    * Check for exclusion
    */
    */
+  @Test
   public void testWithAWrongEntry() throws IOException {
   public void testWithAWrongEntry() throws IOException {
 
 
     String[] ips = {"10.119.103.112", "10.221.102/23", "10.221.204.1/23"};
     String[] ips = {"10.119.103.112", "10.221.102/23", "10.221.204.1/23"};

+ 1 - 1
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestFindClass.java

@@ -19,7 +19,7 @@ package org.apache.hadoop.util;
 
 
 import java.io.ByteArrayOutputStream;
 import java.io.ByteArrayOutputStream;
 import java.io.PrintStream;
 import java.io.PrintStream;
-import junit.framework.Assert;
+import org.junit.Assert;
 import org.apache.hadoop.util.FindClass;
 import org.apache.hadoop.util.FindClass;
 import org.apache.hadoop.util.ToolRunner;
 import org.apache.hadoop.util.ToolRunner;
 import org.junit.Test;
 import org.junit.Test;

+ 10 - 2
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestGenericsUtil.java

@@ -21,12 +21,14 @@ package org.apache.hadoop.util;
 import java.util.ArrayList;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.List;
 
 
-import junit.framework.TestCase;
+import org.junit.Test;
+import static org.junit.Assert.*;
 
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 
 
-public class TestGenericsUtil extends TestCase {
+public class TestGenericsUtil {
 
 
+  @Test
   public void testToArray() {
   public void testToArray() {
 
 
     //test a list of size 10
     //test a list of size 10
@@ -45,6 +47,7 @@ public class TestGenericsUtil extends TestCase {
     }
     }
   }
   }
 
 
+  @Test
   public void testWithEmptyList() {
   public void testWithEmptyList() {
     try {
     try {
       List<String> list = new ArrayList<String>();
       List<String> list = new ArrayList<String>();
@@ -57,6 +60,7 @@ public class TestGenericsUtil extends TestCase {
     }
     }
   }
   }
 
 
+  @Test
   public void testWithEmptyList2() {
   public void testWithEmptyList2() {
     List<String> list = new ArrayList<String>();
     List<String> list = new ArrayList<String>();
     //this method should not throw IndexOutOfBoundsException
     //this method should not throw IndexOutOfBoundsException
@@ -81,6 +85,7 @@ public class TestGenericsUtil extends TestCase {
     }
     }
   }
   }
 
 
+  @Test
   public void testWithGenericClass() {
   public void testWithGenericClass() {
 
 
     GenericClass<String> testSubject = new GenericClass<String>();
     GenericClass<String> testSubject = new GenericClass<String>();
@@ -102,6 +107,7 @@ public class TestGenericsUtil extends TestCase {
 
 
   }
   }
 
 
+  @Test
   public void testGenericOptionsParser() throws Exception {
   public void testGenericOptionsParser() throws Exception {
      GenericOptionsParser parser = new GenericOptionsParser(
      GenericOptionsParser parser = new GenericOptionsParser(
         new Configuration(), new String[] {"-jt"});
         new Configuration(), new String[] {"-jt"});
@@ -116,6 +122,7 @@ public class TestGenericsUtil extends TestCase {
             "y=z", parser.getConfiguration().get("x"));
             "y=z", parser.getConfiguration().get("x"));
   }
   }
 
 
+  @Test
   public void testGetClass() {
   public void testGetClass() {
 
 
     //test with Integer
     //test with Integer
@@ -131,6 +138,7 @@ public class TestGenericsUtil extends TestCase {
             GenericClass.class, c2);
             GenericClass.class, c2);
   }
   }
 
 
+  @Test
   public void testIsLog4jLogger() throws Exception {
   public void testIsLog4jLogger() throws Exception {
     assertFalse("False if clazz is null", GenericsUtil.isLog4jLogger(null));
     assertFalse("False if clazz is null", GenericsUtil.isLog4jLogger(null));
     assertTrue("The implementation is Log4j",
     assertTrue("The implementation is Log4j",

+ 5 - 2
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestIndexedSort.java

@@ -21,14 +21,15 @@ import java.io.IOException;
 import java.util.Arrays;
 import java.util.Arrays;
 import java.util.Random;
 import java.util.Random;
 
 
-import junit.framework.TestCase;
+import org.junit.Test;
+import static org.junit.Assert.*;
 
 
 import org.apache.hadoop.io.DataInputBuffer;
 import org.apache.hadoop.io.DataInputBuffer;
 import org.apache.hadoop.io.DataOutputBuffer;
 import org.apache.hadoop.io.DataOutputBuffer;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.WritableComparator;
 import org.apache.hadoop.io.WritableComparator;
 
 
-public class TestIndexedSort extends TestCase {
+public class TestIndexedSort {
 
 
   public void sortAllEqual(IndexedSorter sorter) throws Exception {
   public void sortAllEqual(IndexedSorter sorter) throws Exception {
     final int SAMPLE = 500;
     final int SAMPLE = 500;
@@ -128,6 +129,7 @@ public class TestIndexedSort extends TestCase {
   }
   }
 
 
 
 
+  @Test
   public void testQuickSort() throws Exception {
   public void testQuickSort() throws Exception {
     QuickSort sorter = new QuickSort();
     QuickSort sorter = new QuickSort();
     sortRandom(sorter);
     sortRandom(sorter);
@@ -158,6 +160,7 @@ public class TestIndexedSort extends TestCase {
     assertTrue(Arrays.equals(values, check));
     assertTrue(Arrays.equals(values, check));
   }
   }
 
 
+  @Test
   public void testHeapSort() throws Exception {
   public void testHeapSort() throws Exception {
     HeapSort sorter = new HeapSort();
     HeapSort sorter = new HeapSort();
     sortRandom(sorter);
     sortRandom(sorter);

+ 4 - 4
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestNativeLibraryChecker.java

@@ -19,13 +19,13 @@ package org.apache.hadoop.util;
 
 
 import java.io.ByteArrayOutputStream;
 import java.io.ByteArrayOutputStream;
 import java.io.PrintStream;
 import java.io.PrintStream;
-
-import junit.framework.TestCase;
+import org.junit.Test;
+import static org.junit.Assert.*;
 
 
 import org.apache.hadoop.util.ExitUtil.ExitException;
 import org.apache.hadoop.util.ExitUtil.ExitException;
-import org.junit.Test;
 
 
-public class TestNativeLibraryChecker extends TestCase {
+
+public class TestNativeLibraryChecker {
   private void expectExit(String [] args) {
   private void expectExit(String [] args) {
     try {
     try {
       // should throw exit exception
       // should throw exit exception

+ 40 - 1
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/curator/TestZKCuratorManager.java

@@ -21,11 +21,15 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.assertTrue;
 
 
+import java.util.Arrays;
 import java.util.List;
 import java.util.List;
 
 
 import org.apache.curator.test.TestingServer;
 import org.apache.curator.test.TestingServer;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
+import org.apache.hadoop.util.ZKUtil;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.data.ACL;
 import org.junit.After;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.Test;
@@ -67,7 +71,7 @@ public class TestZKCuratorManager {
     curator.create(testZNode);
     curator.create(testZNode);
     assertTrue(curator.exists(testZNode));
     assertTrue(curator.exists(testZNode));
     curator.setData(testZNode, expectedString, -1);
     curator.setData(testZNode, expectedString, -1);
-    String testString = curator.getSringData("/test");
+    String testString = curator.getStringData("/test");
     assertEquals(expectedString, testString);
     assertEquals(expectedString, testString);
   }
   }
 
 
@@ -92,4 +96,39 @@ public class TestZKCuratorManager {
     children = curator.getChildren("/");
     children = curator.getChildren("/");
     assertEquals(2, children.size());
     assertEquals(2, children.size());
   }
   }
+
+  @Test
+  public void testTransaction() throws Exception {
+    List<ACL> zkAcl = ZKUtil.parseACLs(CommonConfigurationKeys.ZK_ACL_DEFAULT);
+    String fencingNodePath = "/fencing";
+    String node1 = "/node1";
+    String node2 = "/node2";
+    byte[] testData = "testData".getBytes("UTF-8");
+    assertFalse(curator.exists(fencingNodePath));
+    assertFalse(curator.exists(node1));
+    assertFalse(curator.exists(node2));
+    ZKCuratorManager.SafeTransaction txn = curator.createTransaction(
+        zkAcl, fencingNodePath);
+    txn.create(node1, testData, zkAcl, CreateMode.PERSISTENT);
+    txn.create(node2, testData, zkAcl, CreateMode.PERSISTENT);
+    assertFalse(curator.exists(fencingNodePath));
+    assertFalse(curator.exists(node1));
+    assertFalse(curator.exists(node2));
+    txn.commit();
+    assertFalse(curator.exists(fencingNodePath));
+    assertTrue(curator.exists(node1));
+    assertTrue(curator.exists(node2));
+    assertTrue(Arrays.equals(testData, curator.getData(node1)));
+    assertTrue(Arrays.equals(testData, curator.getData(node2)));
+
+    byte[] setData = "setData".getBytes("UTF-8");
+    txn = curator.createTransaction(zkAcl, fencingNodePath);
+    txn.setData(node1, setData, -1);
+    txn.delete(node2);
+    assertTrue(curator.exists(node2));
+    assertTrue(Arrays.equals(testData, curator.getData(node1)));
+    txn.commit();
+    assertFalse(curator.exists(node2));
+    assertTrue(Arrays.equals(setData, curator.getData(node1)));
+  }
 }
 }

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

@@ -141,6 +141,12 @@ public class EagerKeyGeneratorKeyProviderCryptoExtension
         throws IOException, GeneralSecurityException {
         throws IOException, GeneralSecurityException {
       return keyProviderCryptoExtension.reencryptEncryptedKey(ekv);
       return keyProviderCryptoExtension.reencryptEncryptedKey(ekv);
     }
     }
+
+    @Override
+    public void reencryptEncryptedKeys(List<EncryptedKeyVersion> ekvs)
+        throws IOException, GeneralSecurityException {
+      keyProviderCryptoExtension.reencryptEncryptedKeys(ekvs);
+    }
   }
   }
 
 
   /**
   /**

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

@@ -17,6 +17,9 @@
  */
  */
 package org.apache.hadoop.crypto.key.kms.server;
 package org.apache.hadoop.crypto.key.kms.server;
 
 
+import com.google.common.base.Preconditions;
+import com.google.common.base.Stopwatch;
+import org.apache.hadoop.util.KMSUtil;
 import org.apache.commons.codec.binary.Base64;
 import org.apache.commons.codec.binary.Base64;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.crypto.key.KeyProvider;
 import org.apache.hadoop.crypto.key.KeyProvider;
@@ -53,6 +56,9 @@ import java.util.LinkedList;
 import java.util.List;
 import java.util.List;
 import java.util.Map;
 import java.util.Map;
 
 
+import static org.apache.hadoop.util.KMSUtil.checkNotEmpty;
+import static org.apache.hadoop.util.KMSUtil.checkNotNull;
+
 /**
 /**
  * Class providing the REST bindings, via Jersey, for the KMS.
  * Class providing the REST bindings, via Jersey, for the KMS.
  */
  */
@@ -64,7 +70,7 @@ public class KMS {
     CREATE_KEY, DELETE_KEY, ROLL_NEW_VERSION, INVALIDATE_CACHE,
     CREATE_KEY, DELETE_KEY, ROLL_NEW_VERSION, INVALIDATE_CACHE,
     GET_KEYS, GET_KEYS_METADATA,
     GET_KEYS, GET_KEYS_METADATA,
     GET_KEY_VERSIONS, GET_METADATA, GET_KEY_VERSION, GET_CURRENT_KEY,
     GET_KEY_VERSIONS, GET_METADATA, GET_KEY_VERSION, GET_CURRENT_KEY,
-    GENERATE_EEK, DECRYPT_EEK, REENCRYPT_EEK
+    GENERATE_EEK, DECRYPT_EEK, REENCRYPT_EEK, REENCRYPT_EEK_BATCH
   }
   }
 
 
   private KeyProviderCryptoExtension provider;
   private KeyProviderCryptoExtension provider;
@@ -72,6 +78,8 @@ public class KMS {
 
 
   static final Logger LOG = LoggerFactory.getLogger(KMS.class);
   static final Logger LOG = LoggerFactory.getLogger(KMS.class);
 
 
+  private static final int MAX_NUM_PER_BATCH = 10000;
+
   public KMS() throws Exception {
   public KMS() throws Exception {
     provider = KMSWebApp.getKeyProvider();
     provider = KMSWebApp.getKeyProvider();
     kmsAudit= KMSWebApp.getKMSAudit();
     kmsAudit= KMSWebApp.getKMSAudit();
@@ -109,7 +117,7 @@ public class KMS {
       KMSWebApp.getAdminCallsMeter().mark();
       KMSWebApp.getAdminCallsMeter().mark();
       UserGroupInformation user = HttpUserGroupInformation.get();
       UserGroupInformation user = HttpUserGroupInformation.get();
       final String name = (String) jsonKey.get(KMSRESTConstants.NAME_FIELD);
       final String name = (String) jsonKey.get(KMSRESTConstants.NAME_FIELD);
-      KMSClientProvider.checkNotEmpty(name, KMSRESTConstants.NAME_FIELD);
+      checkNotEmpty(name, KMSRESTConstants.NAME_FIELD);
       assertAccess(KMSACLs.Type.CREATE, user, KMSOp.CREATE_KEY, name);
       assertAccess(KMSACLs.Type.CREATE, user, KMSOp.CREATE_KEY, name);
       String cipher = (String) jsonKey.get(KMSRESTConstants.CIPHER_FIELD);
       String cipher = (String) jsonKey.get(KMSRESTConstants.CIPHER_FIELD);
       final String material;
       final String material;
@@ -158,7 +166,7 @@ public class KMS {
       if (!KMSWebApp.getACLs().hasAccess(KMSACLs.Type.GET, user)) {
       if (!KMSWebApp.getACLs().hasAccess(KMSACLs.Type.GET, user)) {
         keyVersion = removeKeyMaterial(keyVersion);
         keyVersion = removeKeyMaterial(keyVersion);
       }
       }
-      Map json = KMSServerJSONUtils.toJSON(keyVersion);
+      Map json = KMSUtil.toJSON(keyVersion);
       String requestURL = KMSMDCFilter.getURL();
       String requestURL = KMSMDCFilter.getURL();
       int idx = requestURL.lastIndexOf(KMSRESTConstants.KEYS_RESOURCE);
       int idx = requestURL.lastIndexOf(KMSRESTConstants.KEYS_RESOURCE);
       requestURL = requestURL.substring(0, idx);
       requestURL = requestURL.substring(0, idx);
@@ -181,7 +189,7 @@ public class KMS {
       KMSWebApp.getAdminCallsMeter().mark();
       KMSWebApp.getAdminCallsMeter().mark();
       UserGroupInformation user = HttpUserGroupInformation.get();
       UserGroupInformation user = HttpUserGroupInformation.get();
       assertAccess(KMSACLs.Type.DELETE, user, KMSOp.DELETE_KEY, name);
       assertAccess(KMSACLs.Type.DELETE, user, KMSOp.DELETE_KEY, name);
-      KMSClientProvider.checkNotEmpty(name, "name");
+      checkNotEmpty(name, "name");
       LOG.debug("Deleting key with name {}.", name);
       LOG.debug("Deleting key with name {}.", name);
       user.doAs(new PrivilegedExceptionAction<Void>() {
       user.doAs(new PrivilegedExceptionAction<Void>() {
         @Override
         @Override
@@ -212,7 +220,7 @@ public class KMS {
       KMSWebApp.getAdminCallsMeter().mark();
       KMSWebApp.getAdminCallsMeter().mark();
       UserGroupInformation user = HttpUserGroupInformation.get();
       UserGroupInformation user = HttpUserGroupInformation.get();
       assertAccess(KMSACLs.Type.ROLLOVER, user, KMSOp.ROLL_NEW_VERSION, name);
       assertAccess(KMSACLs.Type.ROLLOVER, user, KMSOp.ROLL_NEW_VERSION, name);
-      KMSClientProvider.checkNotEmpty(name, "name");
+      checkNotEmpty(name, "name");
       LOG.debug("Rolling key with name {}.", name);
       LOG.debug("Rolling key with name {}.", name);
       final String material = (String)
       final String material = (String)
               jsonMaterial.get(KMSRESTConstants.MATERIAL_FIELD);
               jsonMaterial.get(KMSRESTConstants.MATERIAL_FIELD);
@@ -242,7 +250,7 @@ public class KMS {
       if (!KMSWebApp.getACLs().hasAccess(KMSACLs.Type.GET, user)) {
       if (!KMSWebApp.getACLs().hasAccess(KMSACLs.Type.GET, user)) {
         keyVersion = removeKeyMaterial(keyVersion);
         keyVersion = removeKeyMaterial(keyVersion);
       }
       }
-      Map json = KMSServerJSONUtils.toJSON(keyVersion);
+      Map json = KMSUtil.toJSON(keyVersion);
       LOG.trace("Exiting rolloverKey Method.");
       LOG.trace("Exiting rolloverKey Method.");
       return Response.ok().type(MediaType.APPLICATION_JSON).entity(json)
       return Response.ok().type(MediaType.APPLICATION_JSON).entity(json)
               .build();
               .build();
@@ -260,7 +268,7 @@ public class KMS {
     try {
     try {
       LOG.trace("Entering invalidateCache Method.");
       LOG.trace("Entering invalidateCache Method.");
       KMSWebApp.getAdminCallsMeter().mark();
       KMSWebApp.getAdminCallsMeter().mark();
-      KMSClientProvider.checkNotEmpty(name, "name");
+      checkNotEmpty(name, "name");
       UserGroupInformation user = HttpUserGroupInformation.get();
       UserGroupInformation user = HttpUserGroupInformation.get();
       assertAccess(KMSACLs.Type.ROLLOVER, user, KMSOp.INVALIDATE_CACHE, name);
       assertAccess(KMSACLs.Type.ROLLOVER, user, KMSOp.INVALIDATE_CACHE, name);
       LOG.debug("Invalidating cache with key name {}.", name);
       LOG.debug("Invalidating cache with key name {}.", name);
@@ -369,7 +377,7 @@ public class KMS {
     try {
     try {
       LOG.trace("Entering getMetadata method.");
       LOG.trace("Entering getMetadata method.");
       UserGroupInformation user = HttpUserGroupInformation.get();
       UserGroupInformation user = HttpUserGroupInformation.get();
-      KMSClientProvider.checkNotEmpty(name, "name");
+      checkNotEmpty(name, "name");
       KMSWebApp.getAdminCallsMeter().mark();
       KMSWebApp.getAdminCallsMeter().mark();
       assertAccess(KMSACLs.Type.GET_METADATA, user, KMSOp.GET_METADATA, name);
       assertAccess(KMSACLs.Type.GET_METADATA, user, KMSOp.GET_METADATA, name);
       LOG.debug("Getting metadata for key with name {}.", name);
       LOG.debug("Getting metadata for key with name {}.", name);
@@ -403,7 +411,7 @@ public class KMS {
     try {
     try {
       LOG.trace("Entering getCurrentVersion method.");
       LOG.trace("Entering getCurrentVersion method.");
       UserGroupInformation user = HttpUserGroupInformation.get();
       UserGroupInformation user = HttpUserGroupInformation.get();
-      KMSClientProvider.checkNotEmpty(name, "name");
+      checkNotEmpty(name, "name");
       KMSWebApp.getKeyCallsMeter().mark();
       KMSWebApp.getKeyCallsMeter().mark();
       assertAccess(KMSACLs.Type.GET, user, KMSOp.GET_CURRENT_KEY, name);
       assertAccess(KMSACLs.Type.GET, user, KMSOp.GET_CURRENT_KEY, name);
       LOG.debug("Getting key version for key with name {}.", name);
       LOG.debug("Getting key version for key with name {}.", name);
@@ -417,7 +425,7 @@ public class KMS {
             }
             }
       );
       );
 
 
-      Object json = KMSServerJSONUtils.toJSON(keyVersion);
+      Object json = KMSUtil.toJSON(keyVersion);
       kmsAudit.ok(user, KMSOp.GET_CURRENT_KEY, name, "");
       kmsAudit.ok(user, KMSOp.GET_CURRENT_KEY, name, "");
       LOG.trace("Exiting getCurrentVersion method.");
       LOG.trace("Exiting getCurrentVersion method.");
       return Response.ok().type(MediaType.APPLICATION_JSON).entity(json)
       return Response.ok().type(MediaType.APPLICATION_JSON).entity(json)
@@ -436,7 +444,7 @@ public class KMS {
     try {
     try {
       LOG.trace("Entering getKeyVersion method.");
       LOG.trace("Entering getKeyVersion method.");
       UserGroupInformation user = HttpUserGroupInformation.get();
       UserGroupInformation user = HttpUserGroupInformation.get();
-      KMSClientProvider.checkNotEmpty(versionName, "versionName");
+      checkNotEmpty(versionName, "versionName");
       KMSWebApp.getKeyCallsMeter().mark();
       KMSWebApp.getKeyCallsMeter().mark();
       assertAccess(KMSACLs.Type.GET, user, KMSOp.GET_KEY_VERSION);
       assertAccess(KMSACLs.Type.GET, user, KMSOp.GET_KEY_VERSION);
       LOG.debug("Getting key with version name {}.", versionName);
       LOG.debug("Getting key with version name {}.", versionName);
@@ -453,7 +461,7 @@ public class KMS {
       if (keyVersion != null) {
       if (keyVersion != null) {
         kmsAudit.ok(user, KMSOp.GET_KEY_VERSION, keyVersion.getName(), "");
         kmsAudit.ok(user, KMSOp.GET_KEY_VERSION, keyVersion.getName(), "");
       }
       }
-      Object json = KMSServerJSONUtils.toJSON(keyVersion);
+      Object json = KMSUtil.toJSON(keyVersion);
       LOG.trace("Exiting getKeyVersion method.");
       LOG.trace("Exiting getKeyVersion method.");
       return Response.ok().type(MediaType.APPLICATION_JSON).entity(json)
       return Response.ok().type(MediaType.APPLICATION_JSON).entity(json)
               .build();
               .build();
@@ -477,8 +485,8 @@ public class KMS {
     try {
     try {
       LOG.trace("Entering generateEncryptedKeys method.");
       LOG.trace("Entering generateEncryptedKeys method.");
       UserGroupInformation user = HttpUserGroupInformation.get();
       UserGroupInformation user = HttpUserGroupInformation.get();
-      KMSClientProvider.checkNotEmpty(name, "name");
-      KMSClientProvider.checkNotNull(edekOp, "eekOp");
+      checkNotEmpty(name, "name");
+      checkNotNull(edekOp, "eekOp");
       LOG.debug("Generating encrypted key with name {}," +
       LOG.debug("Generating encrypted key with name {}," +
               " the edek Operation is {}.", name, edekOp);
               " the edek Operation is {}.", name, edekOp);
 
 
@@ -512,7 +520,7 @@ public class KMS {
         kmsAudit.ok(user, KMSOp.GENERATE_EEK, name, "");
         kmsAudit.ok(user, KMSOp.GENERATE_EEK, name, "");
         retJSON = new ArrayList();
         retJSON = new ArrayList();
         for (EncryptedKeyVersion edek : retEdeks) {
         for (EncryptedKeyVersion edek : retEdeks) {
-          ((ArrayList) retJSON).add(KMSServerJSONUtils.toJSON(edek));
+          ((ArrayList) retJSON).add(KMSUtil.toJSON(edek));
         }
         }
       } else {
       } else {
         StringBuilder error;
         StringBuilder error;
@@ -535,6 +543,64 @@ public class KMS {
     }
     }
   }
   }
 
 
+  @SuppressWarnings("rawtypes")
+  @POST
+  @Path(KMSRESTConstants.KEY_RESOURCE + "/{name:.*}/" +
+      KMSRESTConstants.REENCRYPT_BATCH_SUB_RESOURCE)
+  @Consumes(MediaType.APPLICATION_JSON)
+  @Produces(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8)
+  public Response reencryptEncryptedKeys(
+      @PathParam("name") final String name,
+      final List<Map> jsonPayload)
+      throws Exception {
+    LOG.trace("Entering reencryptEncryptedKeys method.");
+    try {
+      final Stopwatch sw = new Stopwatch().start();
+      checkNotEmpty(name, "name");
+      checkNotNull(jsonPayload, "jsonPayload");
+      final UserGroupInformation user = HttpUserGroupInformation.get();
+      KMSWebApp.getReencryptEEKBatchCallsMeter().mark();
+      if (jsonPayload.size() > MAX_NUM_PER_BATCH) {
+        LOG.warn("Payload size {} too big for reencryptEncryptedKeys from"
+            + " user {}.", jsonPayload.size(), user);
+      }
+      assertAccess(KMSACLs.Type.GENERATE_EEK, user, KMSOp.REENCRYPT_EEK_BATCH,
+          name);
+      LOG.debug("Batch reencrypting {} Encrypted Keys for key name {}",
+          jsonPayload.size(), name);
+      final List<EncryptedKeyVersion> ekvs =
+          KMSUtil.parseJSONEncKeyVersions(name, jsonPayload);
+      Preconditions.checkArgument(ekvs.size() == jsonPayload.size(),
+          "EncryptedKey size mismatch after parsing from json");
+      for (EncryptedKeyVersion ekv : ekvs) {
+        Preconditions.checkArgument(name.equals(ekv.getEncryptionKeyName()),
+            "All EncryptedKeys must be under the given key name " + name);
+      }
+
+      user.doAs(new PrivilegedExceptionAction<Void>() {
+        @Override
+        public Void run() throws Exception {
+          provider.reencryptEncryptedKeys(ekvs);
+          return null;
+        }
+      });
+      List retJSON = new ArrayList<>(ekvs.size());
+      for (EncryptedKeyVersion ekv: ekvs) {
+        retJSON.add(KMSUtil.toJSON(ekv));
+      }
+      kmsAudit.ok(user, KMSOp.REENCRYPT_EEK_BATCH, name,
+          "reencrypted " + ekvs.size() + " keys");
+      LOG.info("reencryptEncryptedKeys {} keys for key {} took {}",
+          jsonPayload.size(), name, sw.stop());
+      LOG.trace("Exiting reencryptEncryptedKeys method.");
+      return Response.ok().type(MediaType.APPLICATION_JSON).entity(retJSON)
+          .build();
+    } catch (Exception e) {
+      LOG.debug("Exception in reencryptEncryptedKeys.", e);
+      throw e;
+    }
+  }
+
   @SuppressWarnings("rawtypes")
   @SuppressWarnings("rawtypes")
   @POST
   @POST
   @Path(KMSRESTConstants.KEY_VERSION_RESOURCE + "/{versionName:.*}/" +
   @Path(KMSRESTConstants.KEY_VERSION_RESOURCE + "/{versionName:.*}/" +
@@ -548,8 +614,8 @@ public class KMS {
     try {
     try {
       LOG.trace("Entering decryptEncryptedKey method.");
       LOG.trace("Entering decryptEncryptedKey method.");
       UserGroupInformation user = HttpUserGroupInformation.get();
       UserGroupInformation user = HttpUserGroupInformation.get();
-      KMSClientProvider.checkNotEmpty(versionName, "versionName");
-      KMSClientProvider.checkNotNull(eekOp, "eekOp");
+      checkNotEmpty(versionName, "versionName");
+      checkNotNull(eekOp, "eekOp");
       LOG.debug("Decrypting key for {}, the edek Operation is {}.",
       LOG.debug("Decrypting key for {}, the edek Operation is {}.",
               versionName, eekOp);
               versionName, eekOp);
 
 
@@ -558,13 +624,14 @@ public class KMS {
       String ivStr = (String) jsonPayload.get(KMSRESTConstants.IV_FIELD);
       String ivStr = (String) jsonPayload.get(KMSRESTConstants.IV_FIELD);
       String encMaterialStr =
       String encMaterialStr =
               (String) jsonPayload.get(KMSRESTConstants.MATERIAL_FIELD);
               (String) jsonPayload.get(KMSRESTConstants.MATERIAL_FIELD);
-      KMSClientProvider.checkNotNull(ivStr, KMSRESTConstants.IV_FIELD);
+      checkNotNull(ivStr, KMSRESTConstants.IV_FIELD);
       final byte[] iv = Base64.decodeBase64(ivStr);
       final byte[] iv = Base64.decodeBase64(ivStr);
-      KMSClientProvider.checkNotNull(encMaterialStr,
+      checkNotNull(encMaterialStr,
           KMSRESTConstants.MATERIAL_FIELD);
           KMSRESTConstants.MATERIAL_FIELD);
       final byte[] encMaterial = Base64.decodeBase64(encMaterialStr);
       final byte[] encMaterial = Base64.decodeBase64(encMaterialStr);
       Object retJSON;
       Object retJSON;
       if (eekOp.equals(KMSRESTConstants.EEK_DECRYPT)) {
       if (eekOp.equals(KMSRESTConstants.EEK_DECRYPT)) {
+        KMSWebApp.getDecryptEEKCallsMeter().mark();
         assertAccess(KMSACLs.Type.DECRYPT_EEK, user, KMSOp.DECRYPT_EEK,
         assertAccess(KMSACLs.Type.DECRYPT_EEK, user, KMSOp.DECRYPT_EEK,
                 keyName);
                 keyName);
 
 
@@ -582,9 +649,10 @@ public class KMS {
               }
               }
         );
         );
 
 
-        retJSON = KMSServerJSONUtils.toJSON(retKeyVersion);
+        retJSON = KMSUtil.toJSON(retKeyVersion);
         kmsAudit.ok(user, KMSOp.DECRYPT_EEK, keyName, "");
         kmsAudit.ok(user, KMSOp.DECRYPT_EEK, keyName, "");
       } else if (eekOp.equals(KMSRESTConstants.EEK_REENCRYPT)) {
       } else if (eekOp.equals(KMSRESTConstants.EEK_REENCRYPT)) {
+        KMSWebApp.getReencryptEEKCallsMeter().mark();
         assertAccess(KMSACLs.Type.GENERATE_EEK, user, KMSOp.REENCRYPT_EEK,
         assertAccess(KMSACLs.Type.GENERATE_EEK, user, KMSOp.REENCRYPT_EEK,
             keyName);
             keyName);
 
 
@@ -599,7 +667,7 @@ public class KMS {
               }
               }
             });
             });
 
 
-        retJSON = KMSServerJSONUtils.toJSON(retEncryptedKeyVersion);
+        retJSON = KMSUtil.toJSON(retEncryptedKeyVersion);
         kmsAudit.ok(user, KMSOp.REENCRYPT_EEK, keyName, "");
         kmsAudit.ok(user, KMSOp.REENCRYPT_EEK, keyName, "");
       } else {
       } else {
         StringBuilder error;
         StringBuilder error;
@@ -612,7 +680,6 @@ public class KMS {
         LOG.error(error.toString());
         LOG.error(error.toString());
         throw new IllegalArgumentException(error.toString());
         throw new IllegalArgumentException(error.toString());
       }
       }
-      KMSWebApp.getDecryptEEKCallsMeter().mark();
       LOG.trace("Exiting handleEncryptedKeyOp method.");
       LOG.trace("Exiting handleEncryptedKeyOp method.");
       return Response.ok().type(MediaType.APPLICATION_JSON).entity(retJSON)
       return Response.ok().type(MediaType.APPLICATION_JSON).entity(retJSON)
           .build();
           .build();
@@ -631,7 +698,7 @@ public class KMS {
     try {
     try {
       LOG.trace("Entering getKeyVersions method.");
       LOG.trace("Entering getKeyVersions method.");
       UserGroupInformation user = HttpUserGroupInformation.get();
       UserGroupInformation user = HttpUserGroupInformation.get();
-      KMSClientProvider.checkNotEmpty(name, "name");
+      checkNotEmpty(name, "name");
       KMSWebApp.getKeyCallsMeter().mark();
       KMSWebApp.getKeyCallsMeter().mark();
       assertAccess(KMSACLs.Type.GET, user, KMSOp.GET_KEY_VERSIONS, name);
       assertAccess(KMSACLs.Type.GET, user, KMSOp.GET_KEY_VERSIONS, name);
       LOG.debug("Getting key versions for key {}", name);
       LOG.debug("Getting key versions for key {}", name);

+ 5 - 3
hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSJSONReader.java

@@ -31,21 +31,23 @@ import java.io.IOException;
 import java.io.InputStream;
 import java.io.InputStream;
 import java.lang.annotation.Annotation;
 import java.lang.annotation.Annotation;
 import java.lang.reflect.Type;
 import java.lang.reflect.Type;
+import java.util.List;
 import java.util.Map;
 import java.util.Map;
 
 
 @Provider
 @Provider
 @Consumes(MediaType.APPLICATION_JSON)
 @Consumes(MediaType.APPLICATION_JSON)
 @InterfaceAudience.Private
 @InterfaceAudience.Private
-public class KMSJSONReader implements MessageBodyReader<Map> {
+public class KMSJSONReader implements MessageBodyReader<Object> {
 
 
   @Override
   @Override
   public boolean isReadable(Class<?> type, Type genericType,
   public boolean isReadable(Class<?> type, Type genericType,
       Annotation[] annotations, MediaType mediaType) {
       Annotation[] annotations, MediaType mediaType) {
-    return type.isAssignableFrom(Map.class);
+    return type.isAssignableFrom(Map.class) || type
+        .isAssignableFrom(List.class);
   }
   }
 
 
   @Override
   @Override
-  public Map readFrom(Class<Map> type, Type genericType,
+  public Object readFrom(Class<Object> type, Type genericType,
       Annotation[] annotations, MediaType mediaType,
       Annotation[] annotations, MediaType mediaType,
       MultivaluedMap<String, String> httpHeaders, InputStream entityStream)
       MultivaluedMap<String, String> httpHeaders, InputStream entityStream)
       throws IOException, WebApplicationException {
       throws IOException, WebApplicationException {

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

@@ -17,11 +17,10 @@
  */
  */
 package org.apache.hadoop.crypto.key.kms.server;
 package org.apache.hadoop.crypto.key.kms.server;
 
 
-import org.apache.commons.codec.binary.Base64;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.crypto.key.KeyProvider;
 import org.apache.hadoop.crypto.key.KeyProvider;
-import org.apache.hadoop.crypto.key.KeyProviderCryptoExtension.EncryptedKeyVersion;
 import org.apache.hadoop.crypto.key.kms.KMSRESTConstants;
 import org.apache.hadoop.crypto.key.kms.KMSRESTConstants;
+import org.apache.hadoop.util.KMSUtil;
 
 
 import java.util.ArrayList;
 import java.util.ArrayList;
 import java.util.LinkedHashMap;
 import java.util.LinkedHashMap;
@@ -33,47 +32,18 @@ import java.util.Map;
  */
  */
 @InterfaceAudience.Private
 @InterfaceAudience.Private
 public class KMSServerJSONUtils {
 public class KMSServerJSONUtils {
-  @SuppressWarnings("unchecked")
-  public static Map toJSON(KeyProvider.KeyVersion keyVersion) {
-    Map json = new LinkedHashMap();
-    if (keyVersion != null) {
-      json.put(KMSRESTConstants.NAME_FIELD,
-          keyVersion.getName());
-      json.put(KMSRESTConstants.VERSION_NAME_FIELD,
-          keyVersion.getVersionName());
-      json.put(KMSRESTConstants.MATERIAL_FIELD,
-          Base64.encodeBase64URLSafeString(
-              keyVersion.getMaterial()));
-    }
-    return json;
-  }
 
 
   @SuppressWarnings("unchecked")
   @SuppressWarnings("unchecked")
   public static List toJSON(List<KeyProvider.KeyVersion> keyVersions) {
   public static List toJSON(List<KeyProvider.KeyVersion> keyVersions) {
     List json = new ArrayList();
     List json = new ArrayList();
     if (keyVersions != null) {
     if (keyVersions != null) {
       for (KeyProvider.KeyVersion version : keyVersions) {
       for (KeyProvider.KeyVersion version : keyVersions) {
-        json.add(toJSON(version));
+        json.add(KMSUtil.toJSON(version));
       }
       }
     }
     }
     return json;
     return json;
   }
   }
 
 
-  @SuppressWarnings("unchecked")
-  public static Map toJSON(EncryptedKeyVersion encryptedKeyVersion) {
-    Map json = new LinkedHashMap();
-    if (encryptedKeyVersion != null) {
-      json.put(KMSRESTConstants.VERSION_NAME_FIELD,
-          encryptedKeyVersion.getEncryptionKeyVersionName());
-      json.put(KMSRESTConstants.IV_FIELD,
-          Base64.encodeBase64URLSafeString(
-              encryptedKeyVersion.getEncryptedKeyIv()));
-      json.put(KMSRESTConstants.ENCRYPTED_KEY_VERSION_FIELD,
-          toJSON(encryptedKeyVersion.getEncryptedKeyVersion()));
-    }
-    return json;
-  }
-
   @SuppressWarnings("unchecked")
   @SuppressWarnings("unchecked")
   public static Map toJSON(String keyName, KeyProvider.Metadata meta) {
   public static Map toJSON(String keyName, KeyProvider.Metadata meta) {
     Map json = new LinkedHashMap();
     Map json = new LinkedHashMap();

+ 18 - 0
hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSWebApp.java

@@ -60,6 +60,10 @@ public class KMSWebApp implements ServletContextListener {
       "generate_eek.calls.meter";
       "generate_eek.calls.meter";
   private static final String DECRYPT_EEK_METER = METRICS_PREFIX +
   private static final String DECRYPT_EEK_METER = METRICS_PREFIX +
       "decrypt_eek.calls.meter";
       "decrypt_eek.calls.meter";
+  private static final String REENCRYPT_EEK_METER = METRICS_PREFIX +
+      "reencrypt_eek.calls.meter";
+  private static final String REENCRYPT_EEK_BATCH_METER = METRICS_PREFIX +
+      "reencrypt_eek_batch.calls.meter";
 
 
   private static Logger LOG;
   private static Logger LOG;
   private static MetricRegistry metricRegistry;
   private static MetricRegistry metricRegistry;
@@ -72,6 +76,8 @@ public class KMSWebApp implements ServletContextListener {
   private static Meter unauthorizedCallsMeter;
   private static Meter unauthorizedCallsMeter;
   private static Meter unauthenticatedCallsMeter;
   private static Meter unauthenticatedCallsMeter;
   private static Meter decryptEEKCallsMeter;
   private static Meter decryptEEKCallsMeter;
+  private static Meter reencryptEEKCallsMeter;
+  private static Meter reencryptEEKBatchCallsMeter;
   private static Meter generateEEKCallsMeter;
   private static Meter generateEEKCallsMeter;
   private static Meter invalidCallsMeter;
   private static Meter invalidCallsMeter;
   private static KMSAudit kmsAudit;
   private static KMSAudit kmsAudit;
@@ -131,6 +137,10 @@ public class KMSWebApp implements ServletContextListener {
           new Meter());
           new Meter());
       decryptEEKCallsMeter = metricRegistry.register(DECRYPT_EEK_METER,
       decryptEEKCallsMeter = metricRegistry.register(DECRYPT_EEK_METER,
           new Meter());
           new Meter());
+      reencryptEEKCallsMeter = metricRegistry.register(REENCRYPT_EEK_METER,
+          new Meter());
+      reencryptEEKBatchCallsMeter = metricRegistry.register(
+          REENCRYPT_EEK_BATCH_METER, new Meter());
       adminCallsMeter = metricRegistry.register(ADMIN_CALLS_METER, new Meter());
       adminCallsMeter = metricRegistry.register(ADMIN_CALLS_METER, new Meter());
       keyCallsMeter = metricRegistry.register(KEY_CALLS_METER, new Meter());
       keyCallsMeter = metricRegistry.register(KEY_CALLS_METER, new Meter());
       invalidCallsMeter = metricRegistry.register(INVALID_CALLS_METER,
       invalidCallsMeter = metricRegistry.register(INVALID_CALLS_METER,
@@ -239,6 +249,14 @@ public class KMSWebApp implements ServletContextListener {
     return decryptEEKCallsMeter;
     return decryptEEKCallsMeter;
   }
   }
 
 
+  public static Meter getReencryptEEKCallsMeter() {
+    return reencryptEEKCallsMeter;
+  }
+
+  public static Meter getReencryptEEKBatchCallsMeter() {
+    return reencryptEEKBatchCallsMeter;
+  }
+
   public static Meter getUnauthorizedCallsMeter() {
   public static Meter getUnauthorizedCallsMeter() {
     return unauthorizedCallsMeter;
     return unauthorizedCallsMeter;
   }
   }

+ 19 - 0
hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KeyAuthorizationKeyProvider.java

@@ -288,6 +288,25 @@ public class KeyAuthorizationKeyProvider extends KeyProviderCryptoExtension {
     }
     }
   }
   }
 
 
+  @Override
+  public void reencryptEncryptedKeys(List<EncryptedKeyVersion> ekvs)
+      throws IOException, GeneralSecurityException {
+    if (ekvs.isEmpty()) {
+      return;
+    }
+    readLock.lock();
+    try {
+      for (EncryptedKeyVersion ekv : ekvs) {
+        verifyKeyVersionBelongsToKey(ekv);
+      }
+      final String keyName = ekvs.get(0).getEncryptionKeyName();
+      doAccessCheck(keyName, KeyOpType.GENERATE_EEK);
+      provider.reencryptEncryptedKeys(ekvs);
+    } finally {
+      readLock.unlock();
+    }
+  }
+
   @Override
   @Override
   public KeyVersion getKeyVersion(String versionName) throws IOException {
   public KeyVersion getKeyVersion(String versionName) throws IOException {
     readLock.lock();
     readLock.lock();

+ 59 - 1
hadoop-common-project/hadoop-kms/src/site/markdown/index.md.vm

@@ -506,7 +506,7 @@ $H5 Key ACLs
 KMS supports access control for all non-read operations at the Key level. All Key Access operations are classified as :
 KMS supports access control for all non-read operations at the Key level. All Key Access operations are classified as :
 
 
 * MANAGEMENT - createKey, deleteKey, rolloverNewVersion
 * MANAGEMENT - createKey, deleteKey, rolloverNewVersion
-* GENERATE_EEK - generateEncryptedKey, reencryptEncryptedKey, warmUpEncryptedKeys
+* GENERATE_EEK - generateEncryptedKey, reencryptEncryptedKey, reencryptEncryptedKeys, warmUpEncryptedKeys
 * DECRYPT_EEK - decryptEncryptedKey
 * DECRYPT_EEK - decryptEncryptedKey
 * READ - getKeyVersion, getKeyVersions, getMetadata, getKeysMetadata, getCurrentKey
 * READ - getKeyVersion, getKeyVersions, getMetadata, getKeysMetadata, getCurrentKey
 * ALL - all of the above
 * ALL - all of the above
@@ -983,6 +983,64 @@ This is usually useful after a [Rollover](#Rollover_Key) of an encryption key. R
       }
       }
     }
     }
 
 
+$H4 Batch Re-encrypt Encrypted Keys With The Latest KeyVersion
+
+Batched version of the above re-encrypt Encrypted Key. This command takes a list of previously generated encrypted key, and re-encrypts them using the latest KeyVersion encryption key in the KeyProvider, and return the re-encrypted encrypted keys in the same sequence. For each encrypted key, if the latest KeyVersion is the same as the one used to generate the encrypted key, no action is taken and the same encrypted key is returned.
+
+This is usually useful after a [Rollover](#Rollover_Key) of an encryption key. Re-encrypting the encrypted key will allow it to be encrypted using the latest version of the encryption key, but still with the same key material and initialization vector.
+
+All Encrypted keys for a batch request must be under the same encryption key name, but could be potentially under different versions of the encryption key.
+
+*REQUEST:*
+
+    POST http://HOST:PORT/kms/v1/key/<key-name>/_reencryptbatch
+    Content-Type: application/json
+
+    [
+      {
+        "versionName"         : "<encryptionVersionName>",
+        "iv"                  : "<iv>",            //base64
+        "encryptedKeyVersion" : {
+            "versionName"       : "EEK",
+            "material"          : "<material>",    //base64
+        }
+      },
+      {
+        "versionName"         : "<encryptionVersionName>",
+        "iv"                  : "<iv>",            //base64
+        "encryptedKeyVersion" : {
+            "versionName"       : "EEK",
+            "material"          : "<material>",    //base64
+        }
+      },
+      ...
+    ]
+
+*RESPONSE:*
+
+    200 OK
+    Content-Type: application/json
+
+    [
+      {
+        "versionName"         : "<encryptionVersionName>",
+        "iv"                  : "<iv>",            //base64
+        "encryptedKeyVersion" : {
+            "versionName"       : "EEK",
+            "material"          : "<material>",    //base64
+        }
+      },
+      {
+        "versionName"         : "<encryptionVersionName>",
+        "iv"                  : "<iv>",            //base64
+        "encryptedKeyVersion" : {
+            "versionName"       : "EEK",
+            "material"          : "<material>",    //base64
+        }
+      },
+      ...
+    ]
+
 $H4 Get Key Version
 $H4 Get Key Version
 
 
 *REQUEST:*
 *REQUEST:*

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

@@ -97,6 +97,7 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotEquals;
 import static org.junit.Assert.assertNotEquals;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.fail;
 import static org.mockito.Mockito.when;
 import static org.mockito.Mockito.when;
 
 
 public class TestKMS {
 public class TestKMS {
@@ -722,6 +723,22 @@ public class TestKMS {
         assertArrayEquals(k1.getMaterial(), k1r.getMaterial());
         assertArrayEquals(k1.getMaterial(), k1r.getMaterial());
         assertEquals(kv.getMaterial().length, k1r.getMaterial().length);
         assertEquals(kv.getMaterial().length, k1r.getMaterial().length);
 
 
+        // test re-encrypt batch
+        EncryptedKeyVersion ek3 = kpExt.generateEncryptedKey(kv.getName());
+        KeyVersion latest = kpExt.rollNewVersion(kv.getName());
+        List<EncryptedKeyVersion> ekvs = new ArrayList<>(3);
+        ekvs.add(ek1);
+        ekvs.add(ek2);
+        ekvs.add(ek3);
+        ekvs.add(ek1);
+        ekvs.add(ek2);
+        ekvs.add(ek3);
+        kpExt.reencryptEncryptedKeys(ekvs);
+        for (EncryptedKeyVersion ekv: ekvs) {
+          assertEquals(latest.getVersionName(),
+              ekv.getEncryptionKeyVersionName());
+        }
+
         // deleteKey()
         // deleteKey()
         kp.deleteKey("k1");
         kp.deleteKey("k1");
 
 
@@ -1134,6 +1151,10 @@ public class TestKMS {
                 KeyProviderCryptoExtension.createKeyProviderCryptoExtension(kp);
                 KeyProviderCryptoExtension.createKeyProviderCryptoExtension(kp);
             EncryptedKeyVersion ekv = kpce.generateEncryptedKey("k1");
             EncryptedKeyVersion ekv = kpce.generateEncryptedKey("k1");
             kpce.reencryptEncryptedKey(ekv);
             kpce.reencryptEncryptedKey(ekv);
+            List<EncryptedKeyVersion> ekvs = new ArrayList<>(2);
+            ekvs.add(ekv);
+            ekvs.add(ekv);
+            kpce.reencryptEncryptedKeys(ekvs);
             return null;
             return null;
           }
           }
         });
         });
@@ -1563,6 +1584,10 @@ public class TestKMS {
             KeyProviderCryptoExtension kpCE = KeyProviderCryptoExtension.
             KeyProviderCryptoExtension kpCE = KeyProviderCryptoExtension.
                 createKeyProviderCryptoExtension(kp);
                 createKeyProviderCryptoExtension(kp);
             kpCE.reencryptEncryptedKey(encKv);
             kpCE.reencryptEncryptedKey(encKv);
+            List<EncryptedKeyVersion> ekvs = new ArrayList<>(2);
+            ekvs.add(encKv);
+            ekvs.add(encKv);
+            kpCE.reencryptEncryptedKeys(ekvs);
             return null;
             return null;
           }
           }
         });
         });
@@ -1669,8 +1694,27 @@ public class TestKMS {
               KeyProviderCryptoExtension kpCE = KeyProviderCryptoExtension.
               KeyProviderCryptoExtension kpCE = KeyProviderCryptoExtension.
                   createKeyProviderCryptoExtension(kp);
                   createKeyProviderCryptoExtension(kp);
               kpCE.reencryptEncryptedKey(encKv);
               kpCE.reencryptEncryptedKey(encKv);
+              fail("Should not have been able to reencryptEncryptedKey");
+            } catch (AuthorizationException ex) {
+              LOG.info("reencryptEncryptedKey caught expected exception.", ex);
+            }
+            return null;
+          }
+        });
+        doAs("GENERATE_EEK", new PrivilegedExceptionAction<Void>() {
+          @Override
+          public Void run() throws Exception {
+            KeyProvider kp = createProvider(uri, conf);
+            try {
+              KeyProviderCryptoExtension kpCE = KeyProviderCryptoExtension.
+                  createKeyProviderCryptoExtension(kp);
+              List<EncryptedKeyVersion> ekvs = new ArrayList<>(2);
+              ekvs.add(encKv);
+              ekvs.add(encKv);
+              kpCE.reencryptEncryptedKeys(ekvs);
+              fail("Should not have been able to reencryptEncryptedKeys");
             } catch (AuthorizationException ex) {
             } catch (AuthorizationException ex) {
-              LOG.info("Caught expected exception.", ex);
+              LOG.info("reencryptEncryptedKeys caught expected exception.", ex);
             }
             }
             return null;
             return null;
           }
           }

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

@@ -115,6 +115,9 @@ public class TestKMSAudit {
     kmsAudit.ok(luser, KMSOp.REENCRYPT_EEK, "k1", "testmsg");
     kmsAudit.ok(luser, KMSOp.REENCRYPT_EEK, "k1", "testmsg");
     kmsAudit.ok(luser, KMSOp.REENCRYPT_EEK, "k1", "testmsg");
     kmsAudit.ok(luser, KMSOp.REENCRYPT_EEK, "k1", "testmsg");
     kmsAudit.evictCacheForTesting();
     kmsAudit.evictCacheForTesting();
+    kmsAudit.ok(luser, KMSOp.REENCRYPT_EEK_BATCH, "k1", "testmsg");
+    kmsAudit.ok(luser, KMSOp.REENCRYPT_EEK_BATCH, "k1", "testmsg");
+    kmsAudit.evictCacheForTesting();
     String out = getAndResetLogOutput();
     String out = getAndResetLogOutput();
     System.out.println(out);
     System.out.println(out);
     Assert.assertTrue(
     Assert.assertTrue(
@@ -128,7 +131,9 @@ public class TestKMSAudit {
             + "OK\\[op=DECRYPT_EEK, key=k1, user=luser, accessCount=6, interval=[^m]{1,4}ms\\] testmsg"
             + "OK\\[op=DECRYPT_EEK, key=k1, user=luser, accessCount=6, interval=[^m]{1,4}ms\\] testmsg"
             + "OK\\[op=DECRYPT_EEK, key=k1, user=luser, accessCount=1, interval=[^m]{1,4}ms\\] testmsg"
             + "OK\\[op=DECRYPT_EEK, key=k1, user=luser, accessCount=1, interval=[^m]{1,4}ms\\] testmsg"
             + "OK\\[op=REENCRYPT_EEK, key=k1, user=luser, accessCount=1, interval=[^m]{1,4}ms\\] testmsg"
             + "OK\\[op=REENCRYPT_EEK, key=k1, user=luser, accessCount=1, interval=[^m]{1,4}ms\\] testmsg"
-            + "OK\\[op=REENCRYPT_EEK, key=k1, user=luser, accessCount=3, interval=[^m]{1,4}ms\\] testmsg"));
+            + "OK\\[op=REENCRYPT_EEK, key=k1, user=luser, accessCount=3, interval=[^m]{1,4}ms\\] testmsg"
+            + "OK\\[op=REENCRYPT_EEK_BATCH, key=k1, user=luser\\] testmsg"
+            + "OK\\[op=REENCRYPT_EEK_BATCH, key=k1, user=luser\\] testmsg"));
   }
   }
 
 
   @Test
   @Test

+ 23 - 2
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java

@@ -121,6 +121,7 @@ import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants.ReencryptAction;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.RollingUpgradeAction;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.RollingUpgradeAction;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
@@ -131,11 +132,13 @@ import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.OpenFileEntry;
 import org.apache.hadoop.hdfs.protocol.OpenFileEntry;
 import org.apache.hadoop.hdfs.protocol.OpenFilesIterator;
 import org.apache.hadoop.hdfs.protocol.OpenFilesIterator;
 import org.apache.hadoop.hdfs.protocol.QuotaByStorageTypeExceededException;
 import org.apache.hadoop.hdfs.protocol.QuotaByStorageTypeExceededException;
+import org.apache.hadoop.hdfs.protocol.ReencryptionStatusIterator;
 import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
 import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
 import org.apache.hadoop.hdfs.protocol.SnapshotAccessControlException;
 import org.apache.hadoop.hdfs.protocol.SnapshotAccessControlException;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
 import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
 import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
 import org.apache.hadoop.hdfs.protocol.UnresolvedPathException;
 import org.apache.hadoop.hdfs.protocol.UnresolvedPathException;
+import org.apache.hadoop.hdfs.protocol.ZoneReencryptionStatus;
 import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtoUtil;
 import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtoUtil;
 import org.apache.hadoop.hdfs.protocol.datatransfer.IOStreamPair;
 import org.apache.hadoop.hdfs.protocol.datatransfer.IOStreamPair;
 import org.apache.hadoop.hdfs.protocol.datatransfer.ReplaceDatanodeOnFailure;
 import org.apache.hadoop.hdfs.protocol.datatransfer.ReplaceDatanodeOnFailure;
@@ -2639,6 +2642,23 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
     return new EncryptionZoneIterator(namenode, tracer);
     return new EncryptionZoneIterator(namenode, tracer);
   }
   }
 
 
+  public void reencryptEncryptionZone(String zone, ReencryptAction action)
+      throws IOException {
+    checkOpen();
+    try (TraceScope ignored = newPathTraceScope("reencryptEncryptionZone",
+        zone)) {
+      namenode.reencryptEncryptionZone(zone, action);
+    } catch (RemoteException re) {
+      throw re.unwrapRemoteException(AccessControlException.class,
+          SafeModeException.class, UnresolvedPathException.class);
+    }
+  }
+
+  public RemoteIterator<ZoneReencryptionStatus> listReencryptionStatus()
+      throws IOException {
+    checkOpen();
+    return new ReencryptionStatusIterator(namenode, tracer);
+  }
 
 
   public void setErasureCodingPolicy(String src, String ecPolicyName)
   public void setErasureCodingPolicy(String src, String ecPolicyName)
       throws IOException {
       throws IOException {
@@ -2764,7 +2784,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
     }
     }
   }
   }
 
 
-  public HashMap<String, String> getErasureCodingCodecs() throws IOException {
+  public Map<String, String> getErasureCodingCodecs() throws IOException {
     checkOpen();
     checkOpen();
     try (TraceScope ignored = tracer.newScope("getErasureCodingCodecs")) {
     try (TraceScope ignored = tracer.newScope("getErasureCodingCodecs")) {
       return namenode.getErasureCodingCodecs();
       return namenode.getErasureCodingCodecs();
@@ -3044,7 +3064,8 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
    *
    *
    * @param src path to get the information for
    * @param src path to get the information for
    * @return Returns the policy information if file or directory on the path is
    * @return Returns the policy information if file or directory on the path is
-   * erasure coded, null otherwise
+   * erasure coded, null otherwise. Null will be returned if directory or file
+   * has REPLICATION policy.
    * @throws IOException
    * @throws IOException
    */
    */
 
 

+ 2 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClientFaultInjector.java

@@ -61,4 +61,6 @@ public class DFSClientFaultInjector {
   public boolean skipRollingRestartWait() {
   public boolean skipRollingRestartWait() {
     return false;
     return false;
   }
   }
+
+  public void sleepBeforeHedgedGet() {}
 }
 }

+ 95 - 58
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java

@@ -182,6 +182,11 @@ public class DFSInputStream extends FSInputStream
     openInfo(false);
     openInfo(false);
   }
   }
 
 
+  @VisibleForTesting
+  public long getlastBlockBeingWrittenLengthForTesting() {
+    return lastBlockBeingWrittenLength;
+  }
+
   /**
   /**
    * Grab the open-file info from namenode
    * Grab the open-file info from namenode
    * @param refreshLocatedBlocks whether to re-fetch locatedblocks
    * @param refreshLocatedBlocks whether to re-fetch locatedblocks
@@ -209,7 +214,8 @@ public class DFSInputStream extends FSInputStream
         }
         }
         retriesForLastBlockLength--;
         retriesForLastBlockLength--;
       }
       }
-      if (retriesForLastBlockLength == 0) {
+      if (lastBlockBeingWrittenLength == -1
+          && retriesForLastBlockLength == 0) {
         throw new IOException("Could not obtain the last block locations.");
         throw new IOException("Could not obtain the last block locations.");
       }
       }
     }
     }
@@ -830,60 +836,85 @@ public class DFSInputStream extends FSInputStream
 
 
   private DNAddrPair chooseDataNode(LocatedBlock block,
   private DNAddrPair chooseDataNode(LocatedBlock block,
       Collection<DatanodeInfo> ignoredNodes) throws IOException {
       Collection<DatanodeInfo> ignoredNodes) throws IOException {
+    return chooseDataNode(block, ignoredNodes, true);
+  }
+
+  /**
+   * Choose datanode to read from.
+   *
+   * @param block             Block to choose datanode addr from
+   * @param ignoredNodes      Ignored nodes inside.
+   * @param refetchIfRequired Whether to refetch if no nodes to chose
+   *                          from.
+   * @return Returns chosen DNAddrPair; Can be null if refetchIfRequired is
+   * false.
+   */
+  private DNAddrPair chooseDataNode(LocatedBlock block,
+      Collection<DatanodeInfo> ignoredNodes, boolean refetchIfRequired)
+      throws IOException {
     while (true) {
     while (true) {
       DNAddrPair result = getBestNodeDNAddrPair(block, ignoredNodes);
       DNAddrPair result = getBestNodeDNAddrPair(block, ignoredNodes);
       if (result != null) {
       if (result != null) {
         return result;
         return result;
+      } else if (refetchIfRequired) {
+        block = refetchLocations(block, ignoredNodes);
       } else {
       } else {
-        String errMsg = getBestNodeDNAddrPairErrorString(block.getLocations(),
-            deadNodes, ignoredNodes);
-        String blockInfo = block.getBlock() + " file=" + src;
-        if (failures >= dfsClient.getConf().getMaxBlockAcquireFailures()) {
-          String description = "Could not obtain block: " + blockInfo;
-          DFSClient.LOG.warn(description + errMsg
-              + ". Throwing a BlockMissingException");
-          throw new BlockMissingException(src, description,
-              block.getStartOffset());
-        }
-
-        DatanodeInfo[] nodes = block.getLocations();
-        if (nodes == null || nodes.length == 0) {
-          DFSClient.LOG.info("No node available for " + blockInfo);
-        }
-        DFSClient.LOG.info("Could not obtain " + block.getBlock()
-            + " from any node: " + errMsg
-            + ". Will get new block locations from namenode and retry...");
-        try {
-          // Introducing a random factor to the wait time before another retry.
-          // The wait time is dependent on # of failures and a random factor.
-          // At the first time of getting a BlockMissingException, the wait time
-          // is a random number between 0..3000 ms. If the first retry
-          // still fails, we will wait 3000 ms grace period before the 2nd retry.
-          // Also at the second retry, the waiting window is expanded to 6000 ms
-          // alleviating the request rate from the server. Similarly the 3rd retry
-          // will wait 6000ms grace period before retry and the waiting window is
-          // expanded to 9000ms.
-          final int timeWindow = dfsClient.getConf().getTimeWindow();
-          double waitTime = timeWindow * failures +       // grace period for the last round of attempt
-              // expanding time window for each failure
-              timeWindow * (failures + 1) *
-              ThreadLocalRandom.current().nextDouble();
-          DFSClient.LOG.warn("DFS chooseDataNode: got # " + (failures + 1) +
-              " IOException, will wait for " + waitTime + " msec.");
-          Thread.sleep((long)waitTime);
-        } catch (InterruptedException e) {
-          Thread.currentThread().interrupt();
-          throw new InterruptedIOException(
-              "Interrupted while choosing DataNode for read.");
-        }
-        deadNodes.clear(); //2nd option is to remove only nodes[blockId]
-        openInfo(true);
-        block = refreshLocatedBlock(block);
-        failures++;
+        return null;
       }
       }
     }
     }
   }
   }
 
 
+  private LocatedBlock refetchLocations(LocatedBlock block,
+      Collection<DatanodeInfo> ignoredNodes) throws IOException {
+    String errMsg = getBestNodeDNAddrPairErrorString(block.getLocations(),
+        deadNodes, ignoredNodes);
+    String blockInfo = block.getBlock() + " file=" + src;
+    if (failures >= dfsClient.getConf().getMaxBlockAcquireFailures()) {
+      String description = "Could not obtain block: " + blockInfo;
+      DFSClient.LOG.warn(description + errMsg
+          + ". Throwing a BlockMissingException");
+      throw new BlockMissingException(src, description,
+          block.getStartOffset());
+    }
+
+    DatanodeInfo[] nodes = block.getLocations();
+    if (nodes == null || nodes.length == 0) {
+      DFSClient.LOG.info("No node available for " + blockInfo);
+    }
+    DFSClient.LOG.info("Could not obtain " + block.getBlock()
+        + " from any node: " + errMsg
+        + ". Will get new block locations from namenode and retry...");
+    try {
+      // Introducing a random factor to the wait time before another retry.
+      // The wait time is dependent on # of failures and a random factor.
+      // At the first time of getting a BlockMissingException, the wait time
+      // is a random number between 0..3000 ms. If the first retry
+      // still fails, we will wait 3000 ms grace period before the 2nd retry.
+      // Also at the second retry, the waiting window is expanded to 6000 ms
+      // alleviating the request rate from the server. Similarly the 3rd retry
+      // will wait 6000ms grace period before retry and the waiting window is
+      // expanded to 9000ms.
+      final int timeWindow = dfsClient.getConf().getTimeWindow();
+      // grace period for the last round of attempt
+      double waitTime = timeWindow * failures +
+          // expanding time window for each failure
+          timeWindow * (failures + 1) *
+          ThreadLocalRandom.current().nextDouble();
+      DFSClient.LOG.warn("DFS chooseDataNode: got # " + (failures + 1) +
+          " IOException, will wait for " + waitTime + " msec.");
+      Thread.sleep((long)waitTime);
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new InterruptedIOException(
+          "Interrupted while choosing DataNode for read.");
+    }
+    deadNodes.clear(); //2nd option is to remove only nodes[blockId]
+    openInfo(true);
+    block = refreshLocatedBlock(block);
+    failures++;
+    return block;
+  }
+
   /**
   /**
    * Get the best node from which to stream the data.
    * Get the best node from which to stream the data.
    * @param block LocatedBlock, containing nodes in priority order.
    * @param block LocatedBlock, containing nodes in priority order.
@@ -985,6 +1016,7 @@ public class DFSInputStream extends FSInputStream
     return new Callable<ByteBuffer>() {
     return new Callable<ByteBuffer>() {
       @Override
       @Override
       public ByteBuffer call() throws Exception {
       public ByteBuffer call() throws Exception {
+        DFSClientFaultInjector.get().sleepBeforeHedgedGet();
         try (TraceScope ignored = dfsClient.getTracer().
         try (TraceScope ignored = dfsClient.getTracer().
             newScope("hedgedRead" + hedgedReadId, parentSpanId)) {
             newScope("hedgedRead" + hedgedReadId, parentSpanId)) {
           actualGetFromOneDataNode(datanode, start, end, bb, corruptedBlocks);
           actualGetFromOneDataNode(datanode, start, end, bb, corruptedBlocks);
@@ -1159,20 +1191,22 @@ public class DFSInputStream extends FSInputStream
         // We are starting up a 'hedged' read. We have a read already
         // We are starting up a 'hedged' read. We have a read already
         // ongoing. Call getBestNodeDNAddrPair instead of chooseDataNode.
         // ongoing. Call getBestNodeDNAddrPair instead of chooseDataNode.
         // If no nodes to do hedged reads against, pass.
         // If no nodes to do hedged reads against, pass.
+        boolean refetch = false;
         try {
         try {
-          chosenNode = getBestNodeDNAddrPair(block, ignored);
-          if (chosenNode == null) {
-            chosenNode = chooseDataNode(block, ignored);
+          chosenNode = chooseDataNode(block, ignored, false);
+          if (chosenNode != null) {
+            // Latest block, if refreshed internally
+            block = chosenNode.block;
+            bb = ByteBuffer.allocate(len);
+            Callable<ByteBuffer> getFromDataNodeCallable =
+                getFromOneDataNode(chosenNode, block, start, end, bb,
+                    corruptedBlocks, hedgedReadId++);
+            Future<ByteBuffer> oneMoreRequest =
+                hedgedService.submit(getFromDataNodeCallable);
+            futures.add(oneMoreRequest);
+          } else {
+            refetch = true;
           }
           }
-          // Latest block, if refreshed internally
-          block = chosenNode.block;
-          bb = ByteBuffer.allocate(len);
-          Callable<ByteBuffer> getFromDataNodeCallable = getFromOneDataNode(
-              chosenNode, block, start, end, bb,
-              corruptedBlocks, hedgedReadId++);
-          Future<ByteBuffer> oneMoreRequest = hedgedService
-              .submit(getFromDataNodeCallable);
-          futures.add(oneMoreRequest);
         } catch (IOException ioe) {
         } catch (IOException ioe) {
           DFSClient.LOG.debug("Failed getting node for hedged read: {}",
           DFSClient.LOG.debug("Failed getting node for hedged read: {}",
               ioe.getMessage());
               ioe.getMessage());
@@ -1190,6 +1224,9 @@ public class DFSInputStream extends FSInputStream
         } catch (InterruptedException ie) {
         } catch (InterruptedException ie) {
           // Ignore and retry
           // Ignore and retry
         }
         }
+        if (refetch) {
+          refetchLocations(block, ignored);
+        }
         // We got here if exception. Ignore this node on next go around IFF
         // We got here if exception. Ignore this node on next go around IFF
         // we found a chosenNode to hedge read against.
         // we found a chosenNode to hedge read against.
         if (chosenNode != null && chosenNode.info != null) {
         if (chosenNode != null && chosenNode.info != null) {

+ 29 - 29
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java

@@ -17,32 +17,8 @@
  */
  */
 package org.apache.hadoop.hdfs;
 package org.apache.hadoop.hdfs;
 
 
-import java.io.IOException;
-import java.io.InterruptedIOException;
-import java.nio.ByteBuffer;
-import java.nio.channels.ClosedChannelException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.EnumSet;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.Callable;
-import java.util.concurrent.CompletionService;
-import java.util.concurrent.ExecutorCompletionService;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.Future;
-
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
 import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.CreateFlag;
@@ -52,6 +28,7 @@ import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo.DatanodeInfoBuilder;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo.DatanodeInfoBuilder;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
@@ -62,16 +39,38 @@ import org.apache.hadoop.io.ByteBufferPool;
 import org.apache.hadoop.io.ElasticByteBufferPool;
 import org.apache.hadoop.io.ElasticByteBufferPool;
 import org.apache.hadoop.io.MultipleIOException;
 import org.apache.hadoop.io.MultipleIOException;
 import org.apache.hadoop.io.erasurecode.CodecUtil;
 import org.apache.hadoop.io.erasurecode.CodecUtil;
-import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.io.erasurecode.ErasureCoderOptions;
 import org.apache.hadoop.io.erasurecode.ErasureCoderOptions;
 import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureEncoder;
 import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureEncoder;
 import org.apache.hadoop.util.DataChecksum;
 import org.apache.hadoop.util.DataChecksum;
 import org.apache.hadoop.util.Progressable;
 import org.apache.hadoop.util.Progressable;
 import org.apache.hadoop.util.Time;
 import org.apache.hadoop.util.Time;
-
-import com.google.common.base.Preconditions;
 import org.apache.htrace.core.TraceScope;
 import org.apache.htrace.core.TraceScope;
 
 
+import java.io.IOException;
+import java.io.InterruptedIOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.ClosedChannelException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.EnumSet;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CompletionService;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorCompletionService;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.TimeUnit;
+
 
 
 /**
 /**
  * This class supports writing files in striped layout and erasure coded format.
  * This class supports writing files in striped layout and erasure coded format.
@@ -777,7 +776,8 @@ public class DFSStripedOutputStream extends DFSOutputStream
     // should update the block group length based on the acked length
     // should update the block group length based on the acked length
     final long sentBytes = currentBlockGroup.getNumBytes();
     final long sentBytes = currentBlockGroup.getNumBytes();
     final long ackedBytes = getNumAckedStripes() * cellSize * numDataBlocks;
     final long ackedBytes = getNumAckedStripes() * cellSize * numDataBlocks;
-    Preconditions.checkState(ackedBytes <= sentBytes);
+    Preconditions.checkState(ackedBytes <= sentBytes,
+        "Acked:" + ackedBytes + ", Sent:" + sentBytes);
     currentBlockGroup.setNumBytes(ackedBytes);
     currentBlockGroup.setNumBytes(ackedBytes);
     newBG.setNumBytes(ackedBytes);
     newBG.setNumBytes(ackedBytes);
     dfsClient.namenode.updatePipeline(dfsClient.clientName, currentBlockGroup,
     dfsClient.namenode.updatePipeline(dfsClient.clientName, currentBlockGroup,

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java

@@ -1620,7 +1620,8 @@ class DataStreamer extends Daemon {
   }
   }
 
 
   /** update pipeline at the namenode */
   /** update pipeline at the namenode */
-  private void updatePipeline(long newGS) throws IOException {
+  @VisibleForTesting
+  public void updatePipeline(long newGS) throws IOException {
     final ExtendedBlock oldBlock = block.getCurrentBlock();
     final ExtendedBlock oldBlock = block.getCurrentBlock();
     // the new GS has been propagated to all DN, it should be ok to update the
     // the new GS has been propagated to all DN, it should be ok to update the
     // local block state
     // local block state

+ 62 - 26
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java

@@ -18,22 +18,14 @@
 
 
 package org.apache.hadoop.hdfs;
 package org.apache.hadoop.hdfs;
 
 
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.net.InetSocketAddress;
-import java.net.URI;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.EnumSet;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
 
 
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
 import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.crypto.key.KeyProviderDelegationTokenExtension;
 import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.BlockStoragePolicySpi;
 import org.apache.hadoop.fs.BlockStoragePolicySpi;
 import org.apache.hadoop.fs.CacheFlag;
 import org.apache.hadoop.fs.CacheFlag;
@@ -54,24 +46,24 @@ import org.apache.hadoop.fs.GlobalStorageStatistics;
 import org.apache.hadoop.fs.GlobalStorageStatistics.StorageStatisticsProvider;
 import org.apache.hadoop.fs.GlobalStorageStatistics.StorageStatisticsProvider;
 import org.apache.hadoop.fs.LocatedFileStatus;
 import org.apache.hadoop.fs.LocatedFileStatus;
 import org.apache.hadoop.fs.Options;
 import org.apache.hadoop.fs.Options;
-import org.apache.hadoop.fs.StorageStatistics;
-import org.apache.hadoop.fs.XAttrSetFlag;
 import org.apache.hadoop.fs.Options.ChecksumOpt;
 import org.apache.hadoop.fs.Options.ChecksumOpt;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.PathFilter;
 import org.apache.hadoop.fs.PathFilter;
 import org.apache.hadoop.fs.QuotaUsage;
 import org.apache.hadoop.fs.QuotaUsage;
 import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.fs.StorageStatistics;
+import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.fs.UnresolvedLinkException;
 import org.apache.hadoop.fs.UnresolvedLinkException;
 import org.apache.hadoop.fs.UnsupportedFileSystemException;
 import org.apache.hadoop.fs.UnsupportedFileSystemException;
+import org.apache.hadoop.fs.XAttrSetFlag;
 import org.apache.hadoop.fs.permission.AclEntry;
 import org.apache.hadoop.fs.permission.AclEntry;
 import org.apache.hadoop.fs.permission.AclStatus;
 import org.apache.hadoop.fs.permission.AclStatus;
-import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsAction;
-import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.DFSOpsCountStatistics.OpType;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
 import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
 import org.apache.hadoop.hdfs.client.impl.CorruptFileBlockIterator;
 import org.apache.hadoop.hdfs.client.impl.CorruptFileBlockIterator;
-import org.apache.hadoop.hdfs.DFSOpsCountStatistics.OpType;
 import org.apache.hadoop.hdfs.protocol.AddECPolicyResponse;
 import org.apache.hadoop.hdfs.protocol.AddECPolicyResponse;
 import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry;
@@ -82,29 +74,37 @@ import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 import org.apache.hadoop.hdfs.protocol.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.EncryptionZone;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants.ReencryptAction;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.RollingUpgradeAction;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.RollingUpgradeAction;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus;
 import org.apache.hadoop.hdfs.protocol.OpenFileEntry;
 import org.apache.hadoop.hdfs.protocol.OpenFileEntry;
+import org.apache.hadoop.hdfs.protocol.ZoneReencryptionStatus;
 import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
 import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
 import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
 import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Text;
-import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.util.Progressable;
 import org.apache.hadoop.util.Progressable;
-import org.apache.hadoop.crypto.key.KeyProviderDelegationTokenExtension;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Preconditions;
 
 
 import javax.annotation.Nonnull;
 import javax.annotation.Nonnull;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.EnumSet;
+import java.util.List;
+import java.util.Map;
 
 
 /****************************************************************
 /****************************************************************
  * Implementation of the abstract FileSystem for the DFS system.
  * Implementation of the abstract FileSystem for the DFS system.
@@ -2314,6 +2314,38 @@ public class DistributedFileSystem extends FileSystem {
     return dfs.listEncryptionZones();
     return dfs.listEncryptionZones();
   }
   }
 
 
+  /* HDFS only */
+  public void reencryptEncryptionZone(final Path zone,
+      final ReencryptAction action) throws IOException {
+    final Path absF = fixRelativePart(zone);
+    new FileSystemLinkResolver<Void>() {
+      @Override
+      public Void doCall(final Path p) throws IOException {
+        dfs.reencryptEncryptionZone(getPathName(p), action);
+        return null;
+      }
+
+      @Override
+      public Void next(final FileSystem fs, final Path p) throws IOException {
+        if (fs instanceof DistributedFileSystem) {
+          DistributedFileSystem myDfs = (DistributedFileSystem) fs;
+          myDfs.reencryptEncryptionZone(p, action);
+          return null;
+        }
+        throw new UnsupportedOperationException(
+            "Cannot call reencryptEncryptionZone"
+                + " on a symlink to a non-DistributedFileSystem: " + zone
+                + " -> " + p);
+      }
+    }.resolve(this, absF);
+  }
+
+  /* HDFS only */
+  public RemoteIterator<ZoneReencryptionStatus> listReencryptionStatus()
+      throws IOException {
+    return dfs.listReencryptionStatus();
+  }
+
   /* HDFS only */
   /* HDFS only */
   public FileEncryptionInfo getFileEncryptionInfo(final Path path)
   public FileEncryptionInfo getFileEncryptionInfo(final Path path)
       throws IOException {
       throws IOException {
@@ -2541,7 +2573,8 @@ public class DistributedFileSystem extends FileSystem {
    *
    *
    * @param path The path of the file or directory
    * @param path The path of the file or directory
    * @return Returns the policy information if file or directory on the path
    * @return Returns the policy information if file or directory on the path
-   * is erasure coded, null otherwise
+   * is erasure coded, null otherwise. Null will be returned if directory or
+   * file has REPLICATION policy.
    * @throws IOException
    * @throws IOException
    */
    */
   public ErasureCodingPolicy getErasureCodingPolicy(final Path path)
   public ErasureCodingPolicy getErasureCodingPolicy(final Path path)
@@ -2568,7 +2601,9 @@ public class DistributedFileSystem extends FileSystem {
   }
   }
 
 
   /**
   /**
-   * Retrieve all the erasure coding policies supported by this file system.
+   * Retrieve all the erasure coding policies supported by this file system,
+   * including enabled, disabled and removed policies, but excluding
+   * REPLICATION policy.
    *
    *
    * @return all erasure coding policies supported by this file system.
    * @return all erasure coding policies supported by this file system.
    * @throws IOException
    * @throws IOException
@@ -2585,7 +2620,7 @@ public class DistributedFileSystem extends FileSystem {
    * @return all erasure coding codecs and coders supported by this file system.
    * @return all erasure coding codecs and coders supported by this file system.
    * @throws IOException
    * @throws IOException
    */
    */
-  public HashMap<String, String> getAllErasureCodingCodecs()
+  public Map<String, String> getAllErasureCodingCodecs()
       throws IOException {
       throws IOException {
     return dfs.getErasureCodingCodecs();
     return dfs.getErasureCodingCodecs();
   }
   }
@@ -2593,8 +2628,9 @@ public class DistributedFileSystem extends FileSystem {
   /**
   /**
    * Add Erasure coding policies to HDFS. For each policy input, schema and
    * Add Erasure coding policies to HDFS. For each policy input, schema and
    * cellSize are musts, name and id are ignored. They will be automatically
    * cellSize are musts, name and id are ignored. They will be automatically
-   * created and assigned by Namenode once the policy is successfully added, and
-   * will be returned in the response.
+   * created and assigned by Namenode once the policy is successfully added,
+   * and will be returned in the response; policy states will be set to
+   * DISABLED automatically.
    *
    *
    * @param policies The user defined ec policy list to add.
    * @param policies The user defined ec policy list to add.
    * @return Return the response list of adding operations.
    * @return Return the response list of adding operations.

+ 38 - 8
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsAdmin.java

@@ -17,12 +17,6 @@
  */
  */
 package org.apache.hadoop.hdfs.client;
 package org.apache.hadoop.hdfs.client;
 
 
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.net.URI;
-import java.util.Collection;
-import java.util.EnumSet;
-
 import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.classification.InterfaceStability;
@@ -49,9 +43,17 @@ import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
 import org.apache.hadoop.hdfs.protocol.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants.ReencryptAction;
 import org.apache.hadoop.hdfs.protocol.OpenFileEntry;
 import org.apache.hadoop.hdfs.protocol.OpenFileEntry;
+import org.apache.hadoop.hdfs.protocol.ZoneReencryptionStatus;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.AccessControlException;
 
 
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.net.URI;
+import java.util.Collection;
+import java.util.EnumSet;
+
 /**
 /**
  * The public API for performing administrative functions on HDFS. Those writing
  * The public API for performing administrative functions on HDFS. Those writing
  * applications against HDFS should prefer this interface to directly accessing
  * applications against HDFS should prefer this interface to directly accessing
@@ -369,6 +371,33 @@ public class HdfsAdmin {
     return dfs.listEncryptionZones();
     return dfs.listEncryptionZones();
   }
   }
 
 
+  /**
+   * Performs re-encryption action for a given encryption zone.
+   *
+   * @param zone the root of the encryption zone
+   * @param action the re-encrypt action
+   * @throws IOException If any error occurs when handling re-encrypt action.
+   */
+  public void reencryptEncryptionZone(final Path zone,
+      final ReencryptAction action) throws IOException {
+    dfs.reencryptEncryptionZone(zone, action);
+  }
+
+  /**
+   * Returns a RemoteIterator which can be used to list all re-encryption
+   * information. For large numbers of re-encryptions, the iterator will fetch
+   * the list in a number of small batches.
+   * <p>
+   * Since the list is fetched in batches, it does not represent a
+   * consistent snapshot of the entire list of encryption zones.
+   * <p>
+   * This method can only be called by HDFS superusers.
+   */
+  public RemoteIterator<ZoneReencryptionStatus> listReencryptionStatus()
+      throws IOException {
+    return dfs.listReencryptionStatus();
+  }
+
   /**
   /**
    * Returns the FileEncryptionInfo on the HdfsFileStatus for the given path.
    * Returns the FileEncryptionInfo on the HdfsFileStatus for the given path.
    * The return value can be null if the path points to a directory, or a file
    * The return value can be null if the path points to a directory, or a file
@@ -525,8 +554,9 @@ public class HdfsAdmin {
   /**
   /**
    * Add Erasure coding policies to HDFS. For each policy input, schema and
    * Add Erasure coding policies to HDFS. For each policy input, schema and
    * cellSize are musts, name and id are ignored. They will be automatically
    * cellSize are musts, name and id are ignored. They will be automatically
-   * created and assigned by Namenode once the policy is successfully added, and
-   * will be returned in the response.
+   * created and assigned by Namenode once the policy is successfully added,
+   * and will be returned in the response; policy states will be set to
+   * DISABLED automatically.
    *
    *
    * @param policies The user defined ec policy list to add.
    * @param policies The user defined ec policy list to add.
    * @return Return the response list of adding operations.
    * @return Return the response list of adding operations.

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java

@@ -389,7 +389,7 @@ public interface HdfsClientConfigKeys {
 
 
     String  THREADPOOL_SIZE_KEY = PREFIX + "threadpool.size";
     String  THREADPOOL_SIZE_KEY = PREFIX + "threadpool.size";
     /**
     /**
-     * With default RS-6-3-64k erasure coding policy, each normal read could
+     * With default RS-6-3-1024k erasure coding policy, each normal read could
      * span 6 DNs, so this default value accommodates 3 read streams
      * span 6 DNs, so this default value accommodates 3 read streams
      */
      */
     int     THREADPOOL_SIZE_DEFAULT = 18;
     int     THREADPOOL_SIZE_DEFAULT = 18;

+ 31 - 4
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java

@@ -19,8 +19,8 @@ package org.apache.hadoop.hdfs.protocol;
 
 
 import java.io.IOException;
 import java.io.IOException;
 import java.util.EnumSet;
 import java.util.EnumSet;
-import java.util.HashMap;
 import java.util.List;
 import java.util.List;
+import java.util.Map;
 
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.classification.InterfaceStability;
@@ -41,6 +41,7 @@ import org.apache.hadoop.fs.permission.AclStatus;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.inotify.EventBatchList;
 import org.apache.hadoop.hdfs.inotify.EventBatchList;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants.ReencryptAction;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.RollingUpgradeAction;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.RollingUpgradeAction;
 import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
 import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
@@ -1443,6 +1444,30 @@ public interface ClientProtocol {
   BatchedEntries<EncryptionZone> listEncryptionZones(
   BatchedEntries<EncryptionZone> listEncryptionZones(
       long prevId) throws IOException;
       long prevId) throws IOException;
 
 
+  /**
+   * Used to implement re-encryption of encryption zones.
+   *
+   * @param zone the encryption zone to re-encrypt.
+   * @param action the action for the re-encryption.
+   * @throws IOException
+   */
+  @AtMostOnce
+  void reencryptEncryptionZone(String zone, ReencryptAction action)
+      throws IOException;
+
+  /**
+   * Used to implement cursor-based batched listing of
+   * {@ZoneReencryptionStatus}s.
+   *
+   * @param prevId ID of the last item in the previous batch. If there is no
+   *               previous batch, a negative value can be used.
+   * @return Batch of encryption zones.
+   * @throws IOException
+   */
+  @Idempotent
+  BatchedEntries<ZoneReencryptionStatus> listReencryptionStatus(long prevId)
+      throws IOException;
+
   /**
   /**
    * Set xattr of a file or directory.
    * Set xattr of a file or directory.
    * The name must be prefixed with the namespace followed by ".". For example,
    * The name must be prefixed with the namespace followed by ".". For example,
@@ -1588,7 +1613,8 @@ public interface ClientProtocol {
 
 
 
 
   /**
   /**
-   * Get the erasure coding policies loaded in Namenode.
+   * Get the erasure coding policies loaded in Namenode, excluding REPLICATION
+   * policy.
    *
    *
    * @throws IOException
    * @throws IOException
    */
    */
@@ -1601,10 +1627,11 @@ public interface ClientProtocol {
    * @throws IOException
    * @throws IOException
    */
    */
   @Idempotent
   @Idempotent
-  HashMap<String, String> getErasureCodingCodecs() throws IOException;
+  Map<String, String> getErasureCodingCodecs() throws IOException;
 
 
   /**
   /**
-   * Get the information about the EC policy for the path.
+   * Get the information about the EC policy for the path. Null will be returned
+   * if directory or file has REPLICATION policy.
    *
    *
    * @param src path to get the info for
    * @param src path to get the info for
    * @throws IOException
    * @throws IOException

+ 49 - 6
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ErasureCodingPolicy.java

@@ -17,14 +17,15 @@
  */
  */
 package org.apache.hadoop.hdfs.protocol;
 package org.apache.hadoop.hdfs.protocol;
 
 
-import java.io.Serializable;
-
 import com.google.common.base.Preconditions;
 import com.google.common.base.Preconditions;
 import org.apache.commons.lang.builder.EqualsBuilder;
 import org.apache.commons.lang.builder.EqualsBuilder;
 import org.apache.commons.lang.builder.HashCodeBuilder;
 import org.apache.commons.lang.builder.HashCodeBuilder;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.io.erasurecode.ECSchema;
 import org.apache.hadoop.io.erasurecode.ECSchema;
+import org.apache.hadoop.io.erasurecode.ErasureCodeConstants;
+
+import java.io.Serializable;
 
 
 /**
 /**
  * A policy about how to write/read/code an erasure coding file.
  * A policy about how to write/read/code an erasure coding file.
@@ -39,9 +40,11 @@ public final class ErasureCodingPolicy implements Serializable {
   private final ECSchema schema;
   private final ECSchema schema;
   private final int cellSize;
   private final int cellSize;
   private byte id;
   private byte id;
+  private ErasureCodingPolicyState state;
+
 
 
   public ErasureCodingPolicy(String name, ECSchema schema,
   public ErasureCodingPolicy(String name, ECSchema schema,
-      int cellSize, byte id) {
+      int cellSize, byte id, ErasureCodingPolicyState state) {
     Preconditions.checkNotNull(name);
     Preconditions.checkNotNull(name);
     Preconditions.checkNotNull(schema);
     Preconditions.checkNotNull(schema);
     Preconditions.checkArgument(cellSize > 0, "cellSize must be positive");
     Preconditions.checkArgument(cellSize > 0, "cellSize must be positive");
@@ -51,14 +54,22 @@ public final class ErasureCodingPolicy implements Serializable {
     this.schema = schema;
     this.schema = schema;
     this.cellSize = cellSize;
     this.cellSize = cellSize;
     this.id = id;
     this.id = id;
+    this.state = state;
+  }
+
+  public ErasureCodingPolicy(String name, ECSchema schema, int cellSize,
+      byte id) {
+    this(name, schema, cellSize, id, ErasureCodingPolicyState.DISABLED);
   }
   }
 
 
   public ErasureCodingPolicy(ECSchema schema, int cellSize, byte id) {
   public ErasureCodingPolicy(ECSchema schema, int cellSize, byte id) {
-    this(composePolicyName(schema, cellSize), schema, cellSize, id);
+    this(composePolicyName(schema, cellSize), schema, cellSize, id,
+        ErasureCodingPolicyState.DISABLED);
   }
   }
 
 
   public ErasureCodingPolicy(ECSchema schema, int cellSize) {
   public ErasureCodingPolicy(ECSchema schema, int cellSize) {
-    this(composePolicyName(schema, cellSize), schema, cellSize, (byte) -1);
+    this(composePolicyName(schema, cellSize), schema, cellSize, (byte) -1,
+        ErasureCodingPolicyState.DISABLED);
   }
   }
 
 
   public static String composePolicyName(ECSchema schema, int cellSize) {
   public static String composePolicyName(ECSchema schema, int cellSize) {
@@ -107,6 +118,35 @@ public final class ErasureCodingPolicy implements Serializable {
     this.id = id;
     this.id = id;
   }
   }
 
 
+
+  public boolean isReplicationPolicy() {
+    return (id == ErasureCodeConstants.REPLICATION_POLICY_ID);
+  }
+
+  public ErasureCodingPolicyState getState() {
+    return state;
+  }
+
+  public void setState(ErasureCodingPolicyState state) {
+    this.state = state;
+  }
+
+  public boolean isSystemPolicy() {
+    return (this.id < ErasureCodeConstants.USER_DEFINED_POLICY_START_ID);
+  }
+
+  public boolean isEnabled() {
+    return (this.state == ErasureCodingPolicyState.ENABLED);
+  }
+
+  public boolean isDisabled() {
+    return (this.state == ErasureCodingPolicyState.DISABLED);
+  }
+
+  public boolean isRemoved() {
+    return (this.state == ErasureCodingPolicyState.REMOVED);
+  }
+
   @Override
   @Override
   public boolean equals(Object o) {
   public boolean equals(Object o) {
     if (o == null) {
     if (o == null) {
@@ -124,6 +164,7 @@ public final class ErasureCodingPolicy implements Serializable {
         .append(schema, rhs.schema)
         .append(schema, rhs.schema)
         .append(cellSize, rhs.cellSize)
         .append(cellSize, rhs.cellSize)
         .append(id, rhs.id)
         .append(id, rhs.id)
+        .append(state, rhs.state)
         .isEquals();
         .isEquals();
   }
   }
 
 
@@ -134,6 +175,7 @@ public final class ErasureCodingPolicy implements Serializable {
         .append(schema)
         .append(schema)
         .append(cellSize)
         .append(cellSize)
         .append(id)
         .append(id)
+        .append(state)
         .toHashCode();
         .toHashCode();
   }
   }
 
 
@@ -142,7 +184,8 @@ public final class ErasureCodingPolicy implements Serializable {
     return "ErasureCodingPolicy=[" + "Name=" + name + ", "
     return "ErasureCodingPolicy=[" + "Name=" + name + ", "
         + "Schema=[" + schema.toString() + "], "
         + "Schema=[" + schema.toString() + "], "
         + "CellSize=" + cellSize + ", "
         + "CellSize=" + cellSize + ", "
-        + "Id=" + id
+        + "Id=" + id + ", "
+        + "State=" + state.toString()
         + "]";
         + "]";
   }
   }
 }
 }

+ 73 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ErasureCodingPolicyState.java

@@ -0,0 +1,73 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.protocol;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+/**
+ * Value denotes the possible states of an ErasureCodingPolicy.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public enum ErasureCodingPolicyState {
+
+  /** Policy is disabled. It's policy default state. */
+  DISABLED(1),
+  /** Policy is enabled. It can be applied to directory and file. */
+  ENABLED(2),
+  /**
+   * Policy is removed from the system. Due to there are potential files
+   * use this policy, it cannot be deleted from system immediately. A removed
+   * policy can be re-enabled later.*/
+  REMOVED(3);
+
+  private static final ErasureCodingPolicyState[] CACHED_VALUES =
+      ErasureCodingPolicyState.values();
+
+  private final int value;
+
+  ErasureCodingPolicyState(int v) {
+    value = v;
+  }
+
+  public int getValue() {
+    return value;
+  }
+
+  public static ErasureCodingPolicyState fromValue(int v) {
+    if (v > 0 && v <= CACHED_VALUES.length) {
+      return CACHED_VALUES[v - 1];
+    }
+    return null;
+  }
+
+  /** Read from in. */
+  public static ErasureCodingPolicyState read(DataInput in) throws IOException {
+    return fromValue(in.readByte());
+  }
+
+  /** Write to out. */
+  public void write(DataOutput out) throws IOException {
+    out.writeByte(ordinal());
+  }
+}

+ 7 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java

@@ -144,6 +144,13 @@ public final class HdfsConstants {
     ALL, LIVE, DEAD, DECOMMISSIONING, ENTERING_MAINTENANCE, IN_MAINTENANCE
     ALL, LIVE, DEAD, DECOMMISSIONING, ENTERING_MAINTENANCE, IN_MAINTENANCE
   }
   }
 
 
+  /**
+   * Re-encrypt encryption zone actions.
+   */
+  public enum ReencryptAction {
+    CANCEL, START
+  }
+
   /* Hidden constructor */
   /* Hidden constructor */
   protected HdfsConstants() {
   protected HdfsConstants() {
   }
   }

+ 216 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ReencryptionStatus.java

@@ -0,0 +1,216 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.protocol;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.fs.BatchedRemoteIterator.BatchedListEntries;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ReencryptionInfoProto;
+import org.apache.hadoop.hdfs.protocol.ZoneReencryptionStatus.State;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Map;
+import java.util.NavigableMap;
+import java.util.TreeMap;
+
+/**
+ * A class representing information about re-encrypting encryption zones. It
+ * contains a collection of @{code ZoneReencryptionStatus} for each EZ.
+ * <p>
+ * FSDirectory lock is used for synchronization (except test-only methods, which
+ * are not protected).
+ */
+@InterfaceAudience.Private
+public final class ReencryptionStatus {
+
+  public static final Logger LOG =
+      LoggerFactory.getLogger(ReencryptionStatus.class);
+
+  public static final BatchedListEntries<ZoneReencryptionStatus> EMPTY_LIST =
+      new BatchedListEntries<>(Lists.newArrayList(), false);
+
+  /**
+   * The zones that were submitted for re-encryption. This should preserve
+   * the order of submission.
+   */
+  private final TreeMap<Long, ZoneReencryptionStatus> zoneStatuses;
+  // Metrics
+  private long zonesReencrypted;
+
+  public ReencryptionStatus() {
+    zoneStatuses = new TreeMap<>();
+  }
+
+  @VisibleForTesting
+  public ReencryptionStatus(ReencryptionStatus rhs) {
+    if (rhs != null) {
+      this.zoneStatuses = new TreeMap<>(rhs.zoneStatuses);
+      this.zonesReencrypted = rhs.zonesReencrypted;
+    } else {
+      zoneStatuses = new TreeMap<>();
+    }
+  }
+
+  @VisibleForTesting
+  public void resetMetrics() {
+    zonesReencrypted = 0;
+    for (Map.Entry<Long, ZoneReencryptionStatus> entry : zoneStatuses
+        .entrySet()) {
+      entry.getValue().resetMetrics();
+    }
+  }
+
+  public ZoneReencryptionStatus getZoneStatus(final Long zondId) {
+    return zoneStatuses.get(zondId);
+  }
+
+  public void markZoneForRetry(final Long zoneId) {
+    final ZoneReencryptionStatus zs = zoneStatuses.get(zoneId);
+    Preconditions.checkNotNull(zs, "Cannot find zone " + zoneId);
+    LOG.info("Zone {} will retry re-encryption", zoneId);
+    zs.setState(State.Submitted);
+  }
+
+  public void markZoneStarted(final Long zoneId) {
+    final ZoneReencryptionStatus zs = zoneStatuses.get(zoneId);
+    Preconditions.checkNotNull(zs, "Cannot find zone " + zoneId);
+    LOG.info("Zone {} starts re-encryption processing", zoneId);
+    zs.setState(State.Processing);
+  }
+
+  public void markZoneCompleted(final Long zoneId) {
+    final ZoneReencryptionStatus zs = zoneStatuses.get(zoneId);
+    Preconditions.checkNotNull(zs, "Cannot find zone " + zoneId);
+    LOG.info("Zone {} completed re-encryption.", zoneId);
+    zs.setState(State.Completed);
+    zonesReencrypted++;
+  }
+
+  public Long getNextUnprocessedZone() {
+    for (Map.Entry<Long, ZoneReencryptionStatus> entry : zoneStatuses
+        .entrySet()) {
+      if (entry.getValue().getState() == State.Submitted) {
+        return entry.getKey();
+      }
+    }
+    return null;
+  }
+
+  public boolean hasRunningZone(final Long zoneId) {
+    return zoneStatuses.containsKey(zoneId)
+        && zoneStatuses.get(zoneId).getState() != State.Completed;
+  }
+
+  /**
+   * @param zoneId
+   * @return true if this is a zone is added.
+   */
+  private boolean addZoneIfNecessary(final Long zoneId, final String name,
+      final ReencryptionInfoProto reProto) {
+    if (!zoneStatuses.containsKey(zoneId)) {
+      LOG.debug("Adding zone {} for re-encryption status", zoneId);
+      Preconditions.checkNotNull(reProto);
+      final ZoneReencryptionStatus.Builder builder =
+          new ZoneReencryptionStatus.Builder();
+      builder.id(zoneId).zoneName(name)
+          .ezKeyVersionName(reProto.getEzKeyVersionName())
+          .submissionTime(reProto.getSubmissionTime())
+          .canceled(reProto.getCanceled())
+          .filesReencrypted(reProto.getNumReencrypted())
+          .fileReencryptionFailures(reProto.getNumFailures());
+      if (reProto.hasCompletionTime()) {
+        builder.completionTime(reProto.getCompletionTime());
+        builder.state(State.Completed);
+        zonesReencrypted++;
+      } else {
+        builder.state(State.Submitted);
+      }
+      if (reProto.hasLastFile()) {
+        builder.lastCheckpointFile(reProto.getLastFile());
+      }
+      return zoneStatuses.put(zoneId, builder.build()) == null;
+    }
+    return false;
+  }
+
+  public void updateZoneStatus(final Long zoneId, final String zonePath,
+      final ReencryptionInfoProto reProto) {
+    Preconditions.checkArgument(zoneId != null, "zoneId can't be null");
+    if (addZoneIfNecessary(zoneId, zonePath, reProto)) {
+      return;
+    }
+    final ZoneReencryptionStatus zs = getZoneStatus(zoneId);
+    assert zs != null;
+    if (reProto.hasCompletionTime()) {
+      zs.markZoneCompleted(reProto);
+    } else if (!reProto.hasLastFile() && !reProto.hasCompletionTime()) {
+      zs.markZoneSubmitted(reProto);
+    } else {
+      zs.updateZoneProcess(reProto);
+    }
+  }
+
+  public boolean removeZone(final Long zoneId) {
+    LOG.debug("Removing re-encryption status of zone {} ", zoneId);
+    return zoneStatuses.remove(zoneId) != null;
+  }
+
+  @VisibleForTesting
+  public int zonesQueued() {
+    int ret = 0;
+    for (Map.Entry<Long, ZoneReencryptionStatus> entry : zoneStatuses
+        .entrySet()) {
+      if (entry.getValue().getState() == State.Submitted) {
+        ret++;
+      }
+    }
+    return ret;
+  }
+
+  @VisibleForTesting
+  public int zonesTotal() {
+    return zoneStatuses.size();
+  }
+
+  @VisibleForTesting
+  public long getNumZonesReencrypted() {
+    return zonesReencrypted;
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder();
+    for (Map.Entry<Long, ZoneReencryptionStatus> entry : zoneStatuses
+        .entrySet()) {
+      sb.append("[zone:" + entry.getKey());
+      sb.append(" state:" + entry.getValue().getState());
+      sb.append(" lastProcessed:" + entry.getValue().getLastCheckpointFile());
+      sb.append(" filesReencrypted:" + entry.getValue().getFilesReencrypted());
+      sb.append(" fileReencryptionFailures:" + entry.getValue()
+          .getNumReencryptionFailures() + "]");
+    }
+    return sb.toString();
+  }
+
+  public NavigableMap<Long, ZoneReencryptionStatus> getZoneStatuses() {
+    return zoneStatuses;
+  }
+}

+ 58 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ReencryptionStatusIterator.java

@@ -0,0 +1,58 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdfs.protocol;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.fs.BatchedRemoteIterator;
+import org.apache.htrace.core.TraceScope;
+import org.apache.htrace.core.Tracer;
+
+import java.io.IOException;
+
+/**
+ * ReencryptionStatusIterator is a remote iterator that iterates over the
+ * reencryption status of encryption zones.
+ * It supports retrying in case of namenode failover.
+ */
+@InterfaceAudience.Private
+public class ReencryptionStatusIterator
+    extends BatchedRemoteIterator<Long, ZoneReencryptionStatus> {
+
+  private final ClientProtocol namenode;
+  private final Tracer tracer;
+
+  public ReencryptionStatusIterator(ClientProtocol namenode, Tracer tracer) {
+    super((long) 0);
+    this.namenode = namenode;
+    this.tracer = tracer;
+  }
+
+  @Override
+  public BatchedEntries<ZoneReencryptionStatus> makeRequest(Long prevId)
+      throws IOException {
+    try (TraceScope ignored = tracer.newScope("listReencryptionStatus")) {
+      return namenode.listReencryptionStatus(prevId);
+    }
+  }
+
+  @Override
+  public Long elementToPrevKey(ZoneReencryptionStatus entry) {
+    return entry.getId();
+  }
+}

+ 16 - 2
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/SystemErasureCodingPolicies.java

@@ -40,8 +40,8 @@ public final class SystemErasureCodingPolicies {
   // Private constructor, this is a utility class.
   // Private constructor, this is a utility class.
   private SystemErasureCodingPolicies() {}
   private SystemErasureCodingPolicies() {}
 
 
-  // 64 KB
-  private static final int DEFAULT_CELLSIZE = 64 * 1024;
+  // 1 MB
+  private static final int DEFAULT_CELLSIZE = 1024 * 1024;
 
 
   public static final byte RS_6_3_POLICY_ID = 1;
   public static final byte RS_6_3_POLICY_ID = 1;
   private static final ErasureCodingPolicy SYS_POLICY1 =
   private static final ErasureCodingPolicy SYS_POLICY1 =
@@ -68,6 +68,13 @@ public final class SystemErasureCodingPolicies {
       new ErasureCodingPolicy(ErasureCodeConstants.RS_10_4_SCHEMA,
       new ErasureCodingPolicy(ErasureCodeConstants.RS_10_4_SCHEMA,
           DEFAULT_CELLSIZE, RS_10_4_POLICY_ID);
           DEFAULT_CELLSIZE, RS_10_4_POLICY_ID);
 
 
+  // REPLICATION policy is always enabled.
+  private static final ErasureCodingPolicy REPLICATION_POLICY =
+      new ErasureCodingPolicy(ErasureCodeConstants.REPLICATION_POLICY_NAME,
+          ErasureCodeConstants.REPLICATION_1_2_SCHEMA,
+          DEFAULT_CELLSIZE,
+          ErasureCodeConstants.REPLICATION_POLICY_ID);
+
   private static final List<ErasureCodingPolicy> SYS_POLICIES =
   private static final List<ErasureCodingPolicy> SYS_POLICIES =
       Collections.unmodifiableList(Arrays.asList(
       Collections.unmodifiableList(Arrays.asList(
           SYS_POLICY1, SYS_POLICY2, SYS_POLICY3, SYS_POLICY4,
           SYS_POLICY1, SYS_POLICY2, SYS_POLICY3, SYS_POLICY4,
@@ -118,4 +125,11 @@ public final class SystemErasureCodingPolicies {
   public static ErasureCodingPolicy getByName(String name) {
   public static ErasureCodingPolicy getByName(String name) {
     return SYSTEM_POLICIES_BY_NAME.get(name);
     return SYSTEM_POLICIES_BY_NAME.get(name);
   }
   }
+
+  /**
+   * Get the special REPLICATION policy.
+   */
+  public static ErasureCodingPolicy  getReplicationPolicy() {
+    return REPLICATION_POLICY;
+  }
 }
 }

+ 257 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ZoneReencryptionStatus.java

@@ -0,0 +1,257 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.protocol;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ReencryptionInfoProto;
+
+/**
+ * A class representing information about re-encryption of an encryption zone.
+ * <p>
+ * FSDirectory lock is used for synchronization (except test-only methods, which
+ * are not protected).
+ */
+public class ZoneReencryptionStatus {
+  /**
+   * State of re-encryption.
+   */
+  public enum State {
+    /**
+     * Submitted for re-encryption but hasn't been picked up.
+     * This is the initial state.
+     */
+    Submitted,
+    /**
+     * Currently re-encrypting.
+     */
+    Processing,
+    /**
+     * Re-encryption completed.
+     */
+    Completed
+  }
+
+  private long id;
+  private String zoneName;
+  /**
+   * The re-encryption status of the zone. Note this is a in-memory only
+   * variable. On failover it will always be submitted, or completed if
+   * completionTime != 0;
+   */
+  private State state;
+  private String ezKeyVersionName;
+  private long submissionTime;
+  private long completionTime;
+  private boolean canceled;
+  /**
+   * Name of last file processed. It's important to record name (not inode)
+   * because we want to restore to the position even if the inode is removed.
+   */
+  private String lastCheckpointFile;
+  private long filesReencrypted;
+  private long numReencryptionFailures;
+
+  /**
+   * Builder of {@link ZoneReencryptionStatus}.
+   */
+  public static final class Builder {
+    private long id;
+    private String zoneName;
+    private State state;
+    private String ezKeyVersionName;
+    private long submissionTime;
+    private long completionTime;
+    private boolean canceled;
+    private String lastCheckpointFile;
+    private long filesReencrypted;
+    private long fileReencryptionFailures;
+
+    public Builder() {
+    }
+
+    public Builder id(final long inodeid) {
+      id = inodeid;
+      return this;
+    }
+
+    public Builder zoneName(final String ezName) {
+      zoneName = ezName;
+      return this;
+    }
+
+    public Builder state(final State st) {
+      state = st;
+      return this;
+    }
+
+    public Builder ezKeyVersionName(final String ezkvn) {
+      ezKeyVersionName = ezkvn;
+      return this;
+    }
+
+    public Builder submissionTime(final long submission) {
+      submissionTime = submission;
+      return this;
+    }
+
+    public Builder completionTime(final long completion) {
+      completionTime = completion;
+      return this;
+    }
+
+    public Builder canceled(final boolean isCanceled) {
+      canceled = isCanceled;
+      return this;
+    }
+
+    public Builder lastCheckpointFile(final String lastFile) {
+      lastCheckpointFile = lastFile;
+      return this;
+    }
+
+    public Builder filesReencrypted(final long numReencrypted) {
+      filesReencrypted = numReencrypted;
+      return this;
+    }
+
+    public Builder fileReencryptionFailures(final long numFailures) {
+      fileReencryptionFailures = numFailures;
+      return this;
+    }
+
+    public ZoneReencryptionStatus build() {
+      ZoneReencryptionStatus ret = new ZoneReencryptionStatus();
+      Preconditions.checkArgument(id != 0, "no inode id set.");
+      Preconditions.checkNotNull(state, "no state id set.");
+      Preconditions.checkNotNull(ezKeyVersionName, "no keyVersionName set.");
+      Preconditions
+          .checkArgument(submissionTime != 0, "no submission time set.");
+      ret.id = this.id;
+      ret.zoneName = this.zoneName;
+      ret.state = this.state;
+      ret.ezKeyVersionName = this.ezKeyVersionName;
+      ret.submissionTime = this.submissionTime;
+      ret.completionTime = this.completionTime;
+      ret.canceled = this.canceled;
+      ret.lastCheckpointFile = this.lastCheckpointFile;
+      ret.filesReencrypted = this.filesReencrypted;
+      ret.numReencryptionFailures = this.fileReencryptionFailures;
+      return ret;
+    }
+  }
+
+  public ZoneReencryptionStatus() {
+    reset();
+  }
+
+  void resetMetrics() {
+    filesReencrypted = 0;
+    numReencryptionFailures = 0;
+  }
+
+  public long getId() {
+    return id;
+  }
+
+  public String getZoneName() {
+    return zoneName;
+  }
+
+  void setState(final State s) {
+    state = s;
+  }
+
+  public State getState() {
+    return state;
+  }
+
+  public String getEzKeyVersionName() {
+    return ezKeyVersionName;
+  }
+
+  public long getSubmissionTime() {
+    return submissionTime;
+  }
+
+  public long getCompletionTime() {
+    return completionTime;
+  }
+
+  public boolean isCanceled() {
+    return canceled;
+  }
+
+  public String getLastCheckpointFile() {
+    return lastCheckpointFile;
+  }
+
+  public long getFilesReencrypted() {
+    return filesReencrypted;
+  }
+
+  public long getNumReencryptionFailures() {
+    return numReencryptionFailures;
+  }
+
+  public void reset() {
+    state = State.Submitted;
+    ezKeyVersionName = null;
+    submissionTime = 0;
+    completionTime = 0;
+    canceled = false;
+    lastCheckpointFile = null;
+    resetMetrics();
+  }
+
+  /**
+   * Set the zone name. The zone name is resolved from inode id and set during
+   * a listReencryptionStatus call, for the crypto admin to consume.
+   */
+  public void setZoneName(final String name) {
+    Preconditions.checkNotNull(name == null);
+    zoneName = name;
+  }
+
+  public void cancel() {
+    canceled = true;
+  }
+
+  void markZoneCompleted(final ReencryptionInfoProto proto) {
+    state = ZoneReencryptionStatus.State.Completed;
+    completionTime = proto.getCompletionTime();
+    lastCheckpointFile = null;
+    canceled = proto.getCanceled();
+    filesReencrypted = proto.getNumReencrypted();
+    numReencryptionFailures = proto.getNumFailures();
+  }
+
+  void markZoneSubmitted(final ReencryptionInfoProto proto) {
+    reset();
+    state = ZoneReencryptionStatus.State.Submitted;
+    ezKeyVersionName = proto.getEzKeyVersionName();
+    submissionTime = proto.getSubmissionTime();
+    filesReencrypted = proto.getNumReencrypted();
+    numReencryptionFailures = proto.getNumFailures();
+  }
+
+  void updateZoneProcess(final ReencryptionInfoProto proto) {
+    lastCheckpointFile = proto.getLastFile();
+    filesReencrypted = proto.getNumReencrypted();
+    numReencryptionFailures = proto.getNumFailures();
+  }
+}

+ 42 - 2
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java

@@ -26,6 +26,7 @@ import java.util.List;
 
 
 import com.google.common.collect.Lists;
 import com.google.common.collect.Lists;
 
 
+import java.util.Map;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeUnit;
 import java.util.stream.Collectors;
 import java.util.stream.Collectors;
 
 
@@ -65,6 +66,7 @@ import org.apache.hadoop.hdfs.protocol.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants.ReencryptAction;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.RollingUpgradeAction;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.RollingUpgradeAction;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
@@ -73,6 +75,7 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.BlocksStats;
 import org.apache.hadoop.hdfs.protocol.BlocksStats;
 import org.apache.hadoop.hdfs.protocol.OpenFileEntry;
 import org.apache.hadoop.hdfs.protocol.OpenFileEntry;
+import org.apache.hadoop.hdfs.protocol.ZoneReencryptionStatus;
 import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
 import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
 import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
 import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
@@ -179,6 +182,10 @@ import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.CreateEncrypt
 import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.EncryptionZoneProto;
 import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.EncryptionZoneProto;
 import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.GetEZForPathRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.GetEZForPathRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.ListEncryptionZonesRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.ListEncryptionZonesRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.ListReencryptionStatusRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.ListReencryptionStatusResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.ZoneReencryptionStatusProto;
+import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.ReencryptEncryptionZoneRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.AddErasureCodingPoliciesRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.AddErasureCodingPoliciesRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.AddErasureCodingPoliciesResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.AddErasureCodingPoliciesResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingPoliciesRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingPoliciesRequestProto;
@@ -1543,6 +1550,39 @@ public class ClientNamenodeProtocolTranslatorPB implements
     }
     }
   }
   }
 
 
+  @Override
+  public void reencryptEncryptionZone(String zone, ReencryptAction action)
+      throws IOException {
+    final ReencryptEncryptionZoneRequestProto.Builder builder =
+        ReencryptEncryptionZoneRequestProto.newBuilder();
+    builder.setZone(zone).setAction(PBHelperClient.convert(action));
+    ReencryptEncryptionZoneRequestProto req = builder.build();
+    try {
+      rpcProxy.reencryptEncryptionZone(null, req);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public BatchedEntries<ZoneReencryptionStatus> listReencryptionStatus(long id)
+      throws IOException {
+    final ListReencryptionStatusRequestProto req =
+        ListReencryptionStatusRequestProto.newBuilder().setId(id).build();
+    try {
+      ListReencryptionStatusResponseProto response =
+          rpcProxy.listReencryptionStatus(null, req);
+      List<ZoneReencryptionStatus> elements =
+          Lists.newArrayListWithCapacity(response.getStatusesCount());
+      for (ZoneReencryptionStatusProto p : response.getStatusesList()) {
+        elements.add(PBHelperClient.convert(p));
+      }
+      return new BatchedListEntries<>(elements, response.getHasMore());
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
   @Override
   @Override
   public void setXAttr(String src, XAttr xAttr, EnumSet<XAttrSetFlag> flag)
   public void setXAttr(String src, XAttr xAttr, EnumSet<XAttrSetFlag> flag)
       throws IOException {
       throws IOException {
@@ -1760,11 +1800,11 @@ public class ClientNamenodeProtocolTranslatorPB implements
   }
   }
 
 
   @Override
   @Override
-  public HashMap<String, String> getErasureCodingCodecs() throws IOException {
+  public Map<String, String> getErasureCodingCodecs() throws IOException {
     try {
     try {
       GetErasureCodingCodecsResponseProto response = rpcProxy
       GetErasureCodingCodecsResponseProto response = rpcProxy
           .getErasureCodingCodecs(null, VOID_GET_EC_CODEC_REQUEST);
           .getErasureCodingCodecs(null, VOID_GET_EC_CODEC_REQUEST);
-      HashMap<String, String> ecCodecs = new HashMap<String, String>();
+      Map<String, String> ecCodecs = new HashMap<>();
       for (CodecProto codec : response.getCodecList()) {
       for (CodecProto codec : response.getCodecList()) {
         ecCodecs.put(codec.getCodec(), codec.getCoders());
         ecCodecs.put(codec.getCodec(), codec.getCoders());
       }
       }

+ 145 - 9
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java

@@ -79,10 +79,12 @@ import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 import org.apache.hadoop.hdfs.protocol.ECBlockGroupsStats;
 import org.apache.hadoop.hdfs.protocol.ECBlockGroupsStats;
 import org.apache.hadoop.hdfs.protocol.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicyState;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.FsPermissionExtension;
 import org.apache.hadoop.hdfs.protocol.FsPermissionExtension;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants.ReencryptAction;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.RollingUpgradeAction;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.RollingUpgradeAction;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
@@ -99,6 +101,7 @@ import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffReportEntry;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffType;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffType;
 import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
 import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
 import org.apache.hadoop.hdfs.protocol.SystemErasureCodingPolicies;
 import org.apache.hadoop.hdfs.protocol.SystemErasureCodingPolicies;
+import org.apache.hadoop.hdfs.protocol.ZoneReencryptionStatus;
 import org.apache.hadoop.hdfs.protocol.proto.AclProtos.AclEntryProto;
 import org.apache.hadoop.hdfs.protocol.proto.AclProtos.AclEntryProto;
 import org.apache.hadoop.hdfs.protocol.proto.AclProtos.AclEntryProto.AclEntryScopeProto;
 import org.apache.hadoop.hdfs.protocol.proto.AclProtos.AclEntryProto.AclEntryScopeProto;
 import org.apache.hadoop.hdfs.protocol.proto.AclProtos.AclEntryProto.AclEntryTypeProto;
 import org.apache.hadoop.hdfs.protocol.proto.AclProtos.AclEntryProto.AclEntryTypeProto;
@@ -129,6 +132,9 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SafeMo
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ShortCircuitShmIdProto;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ShortCircuitShmIdProto;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ShortCircuitShmSlotProto;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ShortCircuitShmSlotProto;
 import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.EncryptionZoneProto;
 import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.EncryptionZoneProto;
+import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.ReencryptActionProto;
+import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.ReencryptionStateProto;
+import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.ZoneReencryptionStatusProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.AccessModeProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.AccessModeProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.AddECPolicyResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.AddECPolicyResponseProto;
@@ -157,6 +163,7 @@ import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.LocatedBlockProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.LocatedBlockProto.Builder;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.LocatedBlockProto.Builder;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.LocatedBlocksProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.LocatedBlocksProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.QuotaUsageProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.QuotaUsageProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ReencryptionInfoProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RollingUpgradeStatusProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RollingUpgradeStatusProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.SnapshotDiffReportEntryProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.SnapshotDiffReportEntryProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.SnapshotDiffReportProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.SnapshotDiffReportProto;
@@ -165,6 +172,7 @@ import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.SnapshottableDirectorySt
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageReportProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageReportProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageTypeProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageTypeProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageTypesProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageTypesProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ZoneEncryptionInfoProto;
 import org.apache.hadoop.hdfs.protocol.proto.InotifyProtos;
 import org.apache.hadoop.hdfs.protocol.proto.InotifyProtos;
 import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.GetXAttrsResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.GetXAttrsResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.ListXAttrsResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.ListXAttrsResponseProto;
@@ -1678,6 +1686,17 @@ public class PBHelperClient {
     return builder.build();
     return builder.build();
   }
   }
 
 
+  public static ReencryptActionProto convert(ReencryptAction a) {
+    switch (a) {
+    case CANCEL:
+      return ReencryptActionProto.CANCEL_REENCRYPT;
+    case START:
+      return ReencryptActionProto.START_REENCRYPT;
+    default:
+      throw new IllegalArgumentException("Unexpected value: " + a);
+    }
+  }
+
   public static RollingUpgradeActionProto convert(RollingUpgradeAction a) {
   public static RollingUpgradeActionProto convert(RollingUpgradeAction a) {
     switch (a) {
     switch (a) {
     case QUERY:
     case QUERY:
@@ -2282,6 +2301,17 @@ public class PBHelperClient {
     }
     }
   }
   }
 
 
+  public static ReencryptAction convert(ReencryptActionProto a) {
+    switch (a) {
+    case CANCEL_REENCRYPT:
+      return ReencryptAction.CANCEL;
+    case START_REENCRYPT:
+      return ReencryptAction.START;
+    default:
+      throw new IllegalArgumentException("Unexpected value: " + a);
+    }
+  }
+
   public static RollingUpgradeAction convert(RollingUpgradeActionProto a) {
   public static RollingUpgradeAction convert(RollingUpgradeActionProto a) {
     switch (a) {
     switch (a) {
     case QUERY:
     case QUERY:
@@ -2733,16 +2763,24 @@ public class PBHelperClient {
         .build();
         .build();
   }
   }
 
 
-  public static HdfsProtos.ZoneEncryptionInfoProto convert(
-      CipherSuite suite, CryptoProtocolVersion version, String keyName) {
+  public static ZoneEncryptionInfoProto convert(CipherSuite suite,
+      CryptoProtocolVersion version, String keyName) {
+    return convert(suite, version, keyName, null);
+  }
+
+  public static ZoneEncryptionInfoProto convert(CipherSuite suite,
+      CryptoProtocolVersion version, String keyName,
+      ReencryptionInfoProto proto) {
     if (suite == null || version == null || keyName == null) {
     if (suite == null || version == null || keyName == null) {
       return null;
       return null;
     }
     }
-    return HdfsProtos.ZoneEncryptionInfoProto.newBuilder()
-        .setSuite(convert(suite))
-        .setCryptoProtocolVersion(convert(version))
-        .setKeyName(keyName)
-        .build();
+    ZoneEncryptionInfoProto.Builder builder =
+        ZoneEncryptionInfoProto.newBuilder().setSuite(convert(suite))
+            .setCryptoProtocolVersion(convert(version)).setKeyName(keyName);
+    if (proto != null) {
+      builder.setReencryptionProto(proto);
+    }
+    return builder.build();
   }
   }
 
 
   public static FileEncryptionInfo convert(
   public static FileEncryptionInfo convert(
@@ -2759,6 +2797,91 @@ public class PBHelperClient {
         ezKeyVersionName);
         ezKeyVersionName);
   }
   }
 
 
+  public static ReencryptionInfoProto convert(String ezkvn, Long submissionTime,
+      boolean isCanceled, long numReencrypted, long numFailures,
+      Long completionTime, String lastFile) {
+    if (ezkvn == null || submissionTime == null) {
+      return null;
+    }
+    ReencryptionInfoProto.Builder builder =
+        ReencryptionInfoProto.newBuilder().setEzKeyVersionName(ezkvn)
+            .setSubmissionTime(submissionTime).setCanceled(isCanceled)
+            .setNumReencrypted(numReencrypted).setNumFailures(numFailures);
+    if (completionTime != null) {
+      builder.setCompletionTime(completionTime);
+    }
+    if (lastFile != null) {
+      builder.setLastFile(lastFile);
+    }
+    return builder.build();
+  }
+
+  public static ZoneReencryptionStatusProto convert(ZoneReencryptionStatus zs) {
+    ZoneReencryptionStatusProto.Builder builder =
+        ZoneReencryptionStatusProto.newBuilder()
+        .setId(zs.getId())
+        .setPath(zs.getZoneName())
+        .setEzKeyVersionName(zs.getEzKeyVersionName())
+        .setSubmissionTime(zs.getSubmissionTime())
+        .setCanceled(zs.isCanceled())
+        .setNumReencrypted(zs.getFilesReencrypted())
+        .setNumFailures(zs.getNumReencryptionFailures());
+    switch (zs.getState()) {
+    case Submitted:
+      builder.setState(ReencryptionStateProto.SUBMITTED);
+      break;
+    case Processing:
+      builder.setState(ReencryptionStateProto.PROCESSING);
+      break;
+    case Completed:
+      builder.setState(ReencryptionStateProto.COMPLETED);
+      break;
+    default:
+      throw new IllegalArgumentException("Unknown state " + zs.getState());
+    }
+    final long completion = zs.getCompletionTime();
+    if (completion != 0) {
+      builder.setCompletionTime(completion);
+    }
+    final String file = zs.getLastCheckpointFile();
+    if (file != null) {
+      builder.setLastFile(file);
+    }
+    return builder.build();
+  }
+
+  public static ZoneReencryptionStatus convert(
+      ZoneReencryptionStatusProto proto) {
+    ZoneReencryptionStatus.State state;
+    switch (proto.getState()) {
+    case SUBMITTED:
+      state = ZoneReencryptionStatus.State.Submitted;
+      break;
+    case PROCESSING:
+      state = ZoneReencryptionStatus.State.Processing;
+      break;
+    case COMPLETED:
+      state = ZoneReencryptionStatus.State.Completed;
+      break;
+    default:
+      throw new IllegalArgumentException("Unknown state " + proto.getState());
+    }
+    ZoneReencryptionStatus.Builder builder = new ZoneReencryptionStatus.
+        Builder().
+        id(proto.getId()).zoneName(proto.getPath()).state(state)
+        .ezKeyVersionName(proto.getEzKeyVersionName())
+        .submissionTime(proto.getSubmissionTime()).canceled(proto.getCanceled())
+        .filesReencrypted(proto.getNumReencrypted())
+        .fileReencryptionFailures(proto.getNumFailures());
+    if (proto.hasCompletionTime()) {
+      builder.completionTime(proto.getCompletionTime());
+    }
+    if (proto.hasLastFile()) {
+      builder.lastCheckpointFile(proto.getLastFile());
+    }
+    return builder.build();
+  }
+
   public static DatanodeInfo[] convert(DatanodeInfosProto datanodeInfosProto) {
   public static DatanodeInfo[] convert(DatanodeInfosProto datanodeInfosProto) {
     List<DatanodeInfoProto> proto = datanodeInfosProto.getDatanodesList();
     List<DatanodeInfoProto> proto = datanodeInfosProto.getDatanodesList();
     DatanodeInfo[] infos = new DatanodeInfo[proto.size()];
     DatanodeInfo[] infos = new DatanodeInfo[proto.size()];
@@ -2803,6 +2926,16 @@ public class PBHelperClient {
     return builder.build();
     return builder.build();
   }
   }
 
 
+  public static ErasureCodingPolicyState convertECState(
+      HdfsProtos.ErasureCodingPolicyState state) {
+    return ErasureCodingPolicyState.fromValue(state.getNumber());
+  }
+
+  public static HdfsProtos.ErasureCodingPolicyState convertECState(
+      ErasureCodingPolicyState state) {
+    return HdfsProtos.ErasureCodingPolicyState.valueOf(state.getValue());
+  }
+
   public static ErasureCodingPolicy convertErasureCodingPolicy(
   public static ErasureCodingPolicy convertErasureCodingPolicy(
       ErasureCodingPolicyProto proto) {
       ErasureCodingPolicyProto proto) {
     final byte id = (byte) (proto.getId() & 0xFF);
     final byte id = (byte) (proto.getId() & 0xFF);
@@ -2816,10 +2949,12 @@ public class PBHelperClient {
           "Missing schema field in ErasureCodingPolicy proto");
           "Missing schema field in ErasureCodingPolicy proto");
       Preconditions.checkArgument(proto.hasCellSize(),
       Preconditions.checkArgument(proto.hasCellSize(),
           "Missing cellsize field in ErasureCodingPolicy proto");
           "Missing cellsize field in ErasureCodingPolicy proto");
+      Preconditions.checkArgument(proto.hasState(),
+          "Missing state field in ErasureCodingPolicy proto");
 
 
       return new ErasureCodingPolicy(proto.getName(),
       return new ErasureCodingPolicy(proto.getName(),
           convertECSchema(proto.getSchema()),
           convertECSchema(proto.getSchema()),
-          proto.getCellSize(), id);
+          proto.getCellSize(), id, convertECState(proto.getState()));
     }
     }
     return policy;
     return policy;
   }
   }
@@ -2833,7 +2968,8 @@ public class PBHelperClient {
     if (SystemErasureCodingPolicies.getByID(policy.getId()) == null) {
     if (SystemErasureCodingPolicies.getByID(policy.getId()) == null) {
       builder.setName(policy.getName())
       builder.setName(policy.getName())
           .setSchema(convertECSchema(policy.getSchema()))
           .setSchema(convertECSchema(policy.getSchema()))
-          .setCellSize(policy.getCellSize());
+          .setCellSize(policy.getCellSize())
+          .setState(convertECState(policy.getState()));
     }
     }
     return builder.build();
     return builder.build();
   }
   }

+ 4 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto

@@ -941,6 +941,10 @@ service ClientNamenodeProtocol {
       returns(CreateEncryptionZoneResponseProto);
       returns(CreateEncryptionZoneResponseProto);
   rpc listEncryptionZones(ListEncryptionZonesRequestProto)
   rpc listEncryptionZones(ListEncryptionZonesRequestProto)
       returns(ListEncryptionZonesResponseProto);
       returns(ListEncryptionZonesResponseProto);
+  rpc reencryptEncryptionZone(ReencryptEncryptionZoneRequestProto)
+      returns(ReencryptEncryptionZoneResponseProto);
+  rpc listReencryptionStatus(ListReencryptionStatusRequestProto)
+      returns(ListReencryptionStatusResponseProto);
   rpc getEZForPath(GetEZForPathRequestProto)
   rpc getEZForPath(GetEZForPathRequestProto)
       returns(GetEZForPathResponseProto);
       returns(GetEZForPathResponseProto);
   rpc setErasureCodingPolicy(SetErasureCodingPolicyRequestProto)
   rpc setErasureCodingPolicy(SetErasureCodingPolicyRequestProto)

+ 41 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/encryption.proto

@@ -58,6 +58,47 @@ message ListEncryptionZonesResponseProto {
   required bool hasMore = 2;
   required bool hasMore = 2;
 }
 }
 
 
+enum ReencryptActionProto {
+  CANCEL_REENCRYPT = 1;
+  START_REENCRYPT = 2;
+}
+
+message ReencryptEncryptionZoneRequestProto {
+  required ReencryptActionProto action = 1;
+  required string zone = 2;
+}
+
+message ReencryptEncryptionZoneResponseProto {
+}
+
+message ListReencryptionStatusRequestProto {
+  required int64 id = 1;
+}
+
+enum ReencryptionStateProto {
+  SUBMITTED = 1;
+  PROCESSING = 2;
+  COMPLETED = 3;
+}
+
+message ZoneReencryptionStatusProto {
+  required int64 id = 1;
+  required string path = 2;
+  required ReencryptionStateProto state = 3;
+  required string ezKeyVersionName = 4;
+  required int64 submissionTime = 5;
+  required bool canceled = 6;
+  required int64 numReencrypted = 7;
+  required int64 numFailures = 8;
+  optional int64 completionTime = 9;
+  optional string lastFile = 10;
+}
+
+message ListReencryptionStatusResponseProto {
+  repeated ZoneReencryptionStatusProto statuses = 1;
+  required bool hasMore = 2;
+}
+
 message GetEZForPathRequestProto {
 message GetEZForPathRequestProto {
     required string src = 1;
     required string src = 1;
 }
 }

+ 24 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/hdfs.proto

@@ -313,6 +313,20 @@ message ZoneEncryptionInfoProto {
   required CipherSuiteProto suite = 1;
   required CipherSuiteProto suite = 1;
   required CryptoProtocolVersionProto cryptoProtocolVersion = 2;
   required CryptoProtocolVersionProto cryptoProtocolVersion = 2;
   required string keyName = 3;
   required string keyName = 3;
+  optional ReencryptionInfoProto reencryptionProto = 4;
+}
+
+/**
+ * Re-encryption information for an encryption zone
+ */
+message ReencryptionInfoProto {
+  required string ezKeyVersionName = 1;
+  required uint64 submissionTime = 2;
+  required bool canceled = 3;
+  required int64 numReencrypted = 4;
+  required int64 numFailures = 5;
+  optional uint64 completionTime = 6;
+  optional string lastFile = 7;
 }
 }
 
 
 /**
 /**
@@ -359,11 +373,21 @@ message ECSchemaProto {
   repeated ECSchemaOptionEntryProto options = 4;
   repeated ECSchemaOptionEntryProto options = 4;
 }
 }
 
 
+/**
+ * EC policy state.
+ */
+enum ErasureCodingPolicyState {
+  DISABLED = 1;
+  ENABLED = 2;
+  REMOVED = 3;
+}
+
 message ErasureCodingPolicyProto {
 message ErasureCodingPolicyProto {
   optional string name = 1;
   optional string name = 1;
   optional ECSchemaProto schema = 2;
   optional ECSchemaProto schema = 2;
   optional uint32 cellSize = 3;
   optional uint32 cellSize = 3;
   required uint32 id = 4; // Actually a byte - only 8 bits used
   required uint32 id = 4; // Actually a byte - only 8 bits used
+  optional ErasureCodingPolicyState state = 5 [default = ENABLED];
 }
 }
 
 
 message AddECPolicyResponseProto {
 message AddECPolicyResponseProto {

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

@@ -859,6 +859,7 @@ public abstract class BaseTestHttpFSWith extends HFSTestCase {
     FileStatus expectedFileStatus = expected.getFileStatus(path);
     FileStatus expectedFileStatus = expected.getFileStatus(path);
     FileStatus actualFileStatus = actual.getFileStatus(path);
     FileStatus actualFileStatus = actual.getFileStatus(path);
     assertEquals(actualFileStatus.hasAcl(), expectedFileStatus.hasAcl());
     assertEquals(actualFileStatus.hasAcl(), expectedFileStatus.hasAcl());
+    // backwards compat
     assertEquals(actualFileStatus.getPermission().getAclBit(),
     assertEquals(actualFileStatus.getPermission().getAclBit(),
         expectedFileStatus.getPermission().getAclBit());
         expectedFileStatus.getPermission().getAclBit());
   }
   }

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs-native-client/src/CMakeLists.txt

@@ -16,7 +16,7 @@
 # limitations under the License.
 # limitations under the License.
 #
 #
 
 
-cmake_minimum_required(VERSION 2.6 FATAL_ERROR)
+cmake_minimum_required(VERSION 3.1 FATAL_ERROR)
 
 
 enable_testing()
 enable_testing()
 
 

+ 2 - 0
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/fuse-dfs/CMakeLists.txt

@@ -16,6 +16,8 @@
 # limitations under the License.
 # limitations under the License.
 #
 #
 
 
+cmake_minimum_required(VERSION 3.1 FATAL_ERROR)
+
 set(CMAKE_SKIP_RPATH TRUE)
 set(CMAKE_SKIP_RPATH TRUE)
 
 
 # Flatten a list into a string.
 # Flatten a list into a string.

+ 2 - 0
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs-tests/CMakeLists.txt

@@ -16,6 +16,8 @@
 # limitations under the License.
 # limitations under the License.
 #
 #
 
 
+cmake_minimum_required(VERSION 3.1 FATAL_ERROR)
+
 include_directories(
 include_directories(
     ${CMAKE_CURRENT_SOURCE_DIR}/../libhdfs/include
     ${CMAKE_CURRENT_SOURCE_DIR}/../libhdfs/include
     ${GENERATED_JAVAH}
     ${GENERATED_JAVAH}

Some files were not shown because too many files changed in this diff