Selaa lähdekoodia

Merge branch 'trunk' into HDFS-7240

Xiaoyu Yao 7 vuotta sitten
vanhempi
commit
72a3743cc4
100 muutettua tiedostoa jossa 3585 lisäystä ja 791 poistoa
  1. 4 7
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java
  2. 117 95
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/KMSClientProvider.java
  3. 21 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/KMSDelegationToken.java
  4. 56 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/KMSLegacyTokenRenewer.java
  5. 103 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/KMSTokenRenewer.java
  6. 18 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/package-info.java
  7. 8 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ChecksumFileSystem.java
  8. 10 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeysPublic.java
  9. 82 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CompositeCrcFileChecksum.java
  10. 1 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java
  11. 11 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Options.java
  12. 62 7
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Command.java
  13. 6 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/CopyCommands.java
  14. 24 2
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Ls.java
  15. 27 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/PathData.java
  16. 16 5
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticatedURL.java
  17. 5 3
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticationHandler.java
  18. 1 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticator.java
  19. 1 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/service/launcher/IrqHandler.java
  20. 187 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/CrcComposer.java
  21. 220 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/CrcUtil.java
  22. 18 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/DataChecksum.java
  23. 43 2
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/KMSUtil.java
  24. 49 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/KMSUtilFaultInjector.java
  25. 1 0
      hadoop-common-project/hadoop-common/src/main/resources/META-INF/services/org.apache.hadoop.security.token.TokenIdentifier
  26. 2 1
      hadoop-common-project/hadoop-common/src/main/resources/META-INF/services/org.apache.hadoop.security.token.TokenRenewer
  27. 20 0
      hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
  28. 20 6
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfiguration.java
  29. 162 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/key/kms/TestKMSClientProvider.java
  30. 51 16
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/key/kms/TestLoadBalancingKMSClientProvider.java
  31. 33 1
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/shell/find/TestFind.java
  32. 242 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestCrcComposer.java
  33. 232 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestCrcUtil.java
  34. 65 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestKMSUtil.java
  35. 440 79
      hadoop-common-project/hadoop-kms/src/test/java/org/apache/hadoop/crypto/key/kms/server/TestKMS.java
  36. 2 2
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/fs/Hdfs.java
  37. 42 14
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
  38. 3 2
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
  39. 248 117
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/FileChecksumHelper.java
  40. 2 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java
  41. 27 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/impl/DfsClientConf.java
  42. 54 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/BlockChecksumOptions.java
  43. 30 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/BlockChecksumType.java
  44. 9 3
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/DataTransferProtocol.java
  45. 8 3
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Sender.java
  46. 44 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
  47. 0 3
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/RequestHedgingProxyProvider.java
  48. 7 4
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/shortcircuit/ShortCircuitCache.java
  49. 5 2
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/datatransfer.proto
  50. 21 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/hdfs.proto
  51. 0 34
      hadoop-hdfs-project/hadoop-hdfs-client/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestRequestHedgingProxyProvider.java
  52. 1 52
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/include/hdfspp/hdfspp.h
  53. 140 0
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/include/hdfspp/ioservice.h
  54. 3 4
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/bindings/c/hdfs.cc
  55. 1 1
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/CMakeLists.txt
  56. 8 5
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/async_stream.h
  57. 0 5
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/continuation/asio.h
  58. 18 5
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/ioservice_impl.cc
  59. 12 15
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/ioservice_impl.h
  60. 0 3
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/logging.h
  61. 7 8
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/namenode_info.cc
  62. 6 2
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/namenode_info.h
  63. 13 1
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/util.cc
  64. 9 16
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/util.h
  65. 22 5
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/connection/datanodeconnection.cc
  66. 7 19
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/connection/datanodeconnection.h
  67. 9 9
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filehandle.cc
  68. 5 7
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filehandle.h
  69. 58 9
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.cc
  70. 12 54
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h
  71. 2 2
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/namenode_operations.h
  72. 9 9
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/block_reader.cc
  73. 5 5
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/block_reader.h
  74. 2 2
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/datatransfer.h
  75. 1 1
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/rpc/namenode_tracker.cc
  76. 2 2
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/rpc/namenode_tracker.h
  77. 3 2
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/rpc/request.cc
  78. 1 1
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/rpc/rpc_connection.h
  79. 19 13
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/rpc/rpc_connection_impl.cc
  80. 5 4
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/rpc/rpc_connection_impl.h
  81. 7 7
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/rpc/rpc_engine.cc
  82. 5 4
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/rpc/rpc_engine.h
  83. 16 15
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/bad_datanode_test.cc
  84. 6 4
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/hdfs_ioservice_test.cc
  85. 2 2
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/mock_connection.h
  86. 2 2
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/remote_block_reader_test.cc
  87. 58 54
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/rpc_engine_test.cc
  88. 5 0
      hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/FederationNamespaceInfo.java
  89. 2 2
      hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/MountTableResolver.java
  90. 23 12
      hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/RemoteLocation.java
  91. 7 0
      hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RemoteLocationContext.java
  92. 65 2
      hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcClient.java
  93. 18 1
      hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java
  94. 7 4
      hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/impl/StateStoreFileImpl.java
  95. 1 1
      hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/MountTable.java
  96. 1 1
      hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/impl/pb/MountTablePBImpl.java
  97. 9 3
      hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/MockResolver.java
  98. 2 2
      hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterAdmin.java
  99. 61 1
      hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterMountTable.java
  100. 48 0
      hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterRpc.java

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

@@ -2869,15 +2869,12 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
    */
   public Map<String, String> getPropsWithPrefix(String confPrefix) {
     Properties props = getProps();
-    Enumeration e = props.propertyNames();
     Map<String, String> configMap = new HashMap<>();
-    String name = null;
-    while (e.hasMoreElements()) {
-      name = (String) e.nextElement();
+    for (String name : props.stringPropertyNames()) {
       if (name.startsWith(confPrefix)) {
-        String value = props.getProperty(name);
-        name = name.substring(confPrefix.length());
-        configMap.put(name, value);
+        String value = get(name);
+        String keyName = name.substring(confPrefix.length());
+        configMap.put(keyName, value);
       }
     }
     return configMap;

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

@@ -36,8 +36,9 @@ import org.apache.hadoop.security.authentication.client.ConnectionConfigurator;
 import org.apache.hadoop.security.ssl.SSLFactory;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.TokenIdentifier;
-import org.apache.hadoop.security.token.TokenRenewer;
+import org.apache.hadoop.security.token.TokenSelector;
 import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenIdentifier;
+import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenSelector;
 import org.apache.hadoop.security.token.delegation.web.DelegationTokenAuthenticatedURL;
 import org.apache.hadoop.util.HttpExceptionUtils;
 import org.apache.hadoop.util.KMSUtil;
@@ -82,6 +83,8 @@ import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import com.google.common.base.Strings;
 
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.KMS_CLIENT_COPY_LEGACY_TOKEN_KEY;
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.KMS_CLIENT_COPY_LEGACY_TOKEN_DEFAULT;
 import static org.apache.hadoop.util.KMSUtil.checkNotEmpty;
 import static org.apache.hadoop.util.KMSUtil.checkNotNull;
 import static org.apache.hadoop.util.KMSUtil.parseJSONEncKeyVersion;
@@ -96,16 +99,13 @@ import static org.apache.hadoop.util.KMSUtil.parseJSONMetadata;
 public class KMSClientProvider extends KeyProvider implements CryptoExtension,
     KeyProviderDelegationTokenExtension.DelegationTokenExtension {
 
-  private static final Logger LOG =
+  public static final Logger LOG =
       LoggerFactory.getLogger(KMSClientProvider.class);
 
   private static final String INVALID_SIGNATURE = "Invalid signature";
 
   private static final String ANONYMOUS_REQUESTS_DISALLOWED = "Anonymous requests are disallowed";
 
-  public static final String TOKEN_KIND_STR = KMSDelegationToken.TOKEN_KIND_STR;
-  public static final Text TOKEN_KIND = KMSDelegationToken.TOKEN_KIND;
-
   public static final String SCHEME_NAME = "kms";
 
   private static final String UTF8 = "UTF-8";
@@ -133,12 +133,17 @@ public class KMSClientProvider extends KeyProvider implements CryptoExtension,
   private static final ObjectWriter WRITER =
       new ObjectMapper().writerWithDefaultPrettyPrinter();
 
+  /* dtService defines the token service value for the kms token.
+   * The value can be legacy format which is ip:port format or it can be uri.
+   * If it's uri format, then the value is read from
+   * CommonConfigurationKeysPublic.HADOOP_SECURITY_KEY_PROVIDER_PATH at key
+   * provider creation time, and set to token's Service field.
+   * When a token is renewed / canceled, its Service field will be used to
+   * instantiate a KeyProvider, eliminating the need to read configs
+    * at that time.
+   */
   private final Text dtService;
-
-  // Allow fallback to default kms server port 9600 for certain tests that do
-  // not specify the port explicitly in the kms provider url.
-  @VisibleForTesting
-  public static volatile boolean fallbackDefaultPortForTesting = false;
+  private final boolean copyLegacyToken;
 
   private class EncryptedQueueRefiller implements
     ValueQueue.QueueRefiller<EncryptedKeyVersion> {
@@ -162,66 +167,6 @@ public class KMSClientProvider extends KeyProvider implements CryptoExtension,
     }
   }
 
-  /**
-   * The KMS implementation of {@link TokenRenewer}.
-   */
-  public static class KMSTokenRenewer extends TokenRenewer {
-    private static final Logger LOG =
-        LoggerFactory.getLogger(KMSTokenRenewer.class);
-
-    @Override
-    public boolean handleKind(Text kind) {
-      return kind.equals(TOKEN_KIND);
-    }
-
-    @Override
-    public boolean isManaged(Token<?> token) throws IOException {
-      return true;
-    }
-
-    @Override
-    public long renew(Token<?> token, Configuration conf) throws IOException {
-      LOG.debug("Renewing delegation token {}", token);
-      KeyProvider keyProvider = KMSUtil.createKeyProvider(conf,
-          KeyProviderFactory.KEY_PROVIDER_PATH);
-      try {
-        if (!(keyProvider instanceof
-            KeyProviderDelegationTokenExtension.DelegationTokenExtension)) {
-          LOG.warn("keyProvider {} cannot renew dt.", keyProvider == null ?
-              "null" : keyProvider.getClass());
-          return 0;
-        }
-        return ((KeyProviderDelegationTokenExtension.DelegationTokenExtension)
-            keyProvider).renewDelegationToken(token);
-      } finally {
-        if (keyProvider != null) {
-          keyProvider.close();
-        }
-      }
-    }
-
-    @Override
-    public void cancel(Token<?> token, Configuration conf) throws IOException {
-      LOG.debug("Canceling delegation token {}", token);
-      KeyProvider keyProvider = KMSUtil.createKeyProvider(conf,
-          KeyProviderFactory.KEY_PROVIDER_PATH);
-      try {
-        if (!(keyProvider instanceof
-            KeyProviderDelegationTokenExtension.DelegationTokenExtension)) {
-          LOG.warn("keyProvider {} cannot cancel dt.", keyProvider == null ?
-              "null" : keyProvider.getClass());
-          return;
-        }
-        ((KeyProviderDelegationTokenExtension.DelegationTokenExtension)
-            keyProvider).cancelDelegationToken(token);
-      } finally {
-        if (keyProvider != null) {
-          keyProvider.close();
-        }
-      }
-    }
-  }
-
   public static class KMSEncryptedKeyVersion extends EncryptedKeyVersion {
     public KMSEncryptedKeyVersion(String keyName, String keyVersionName,
         byte[] iv, String encryptedVersionName, byte[] keyMaterial) {
@@ -281,13 +226,13 @@ public class KMSClientProvider extends KeyProvider implements CryptoExtension,
           }
           hostsPart = t[0];
         }
-        return createProvider(conf, origUrl, port, hostsPart);
+        return createProvider(conf, origUrl, port, hostsPart, providerUri);
       }
       return null;
     }
 
-    private KeyProvider createProvider(Configuration conf,
-        URL origUrl, int port, String hostsPart) throws IOException {
+    private KeyProvider createProvider(Configuration conf, URL origUrl,
+        int port, String hostsPart, URI providerUri) throws IOException {
       String[] hosts = hostsPart.split(";");
       KMSClientProvider[] providers = new KMSClientProvider[hosts.length];
       for (int i = 0; i < hosts.length; i++) {
@@ -295,7 +240,7 @@ public class KMSClientProvider extends KeyProvider implements CryptoExtension,
           providers[i] =
               new KMSClientProvider(
                   new URI("kms", origUrl.getProtocol(), hosts[i], port,
-                      origUrl.getPath(), null, null), conf);
+                      origUrl.getPath(), null, null), conf, providerUri);
         } catch (URISyntaxException e) {
           throw new IOException("Could not instantiate KMSProvider.", e);
         }
@@ -353,17 +298,10 @@ public class KMSClientProvider extends KeyProvider implements CryptoExtension,
     }
   }
 
-  public KMSClientProvider(URI uri, Configuration conf) throws IOException {
+  public KMSClientProvider(URI uri, Configuration conf, URI providerUri) throws
+      IOException {
     super(conf);
     kmsUrl = createServiceURL(extractKMSPath(uri));
-    int kmsPort = kmsUrl.getPort();
-    if ((kmsPort == -1) && fallbackDefaultPortForTesting) {
-      kmsPort = 9600;
-    }
-
-    InetSocketAddress addr = new InetSocketAddress(kmsUrl.getHost(), kmsPort);
-    dtService = SecurityUtil.buildTokenService(addr);
-
     if ("https".equalsIgnoreCase(kmsUrl.getProtocol())) {
       sslFactory = new SSLFactory(SSLFactory.Mode.CLIENT, conf);
       try {
@@ -376,6 +314,9 @@ public class KMSClientProvider extends KeyProvider implements CryptoExtension,
             CommonConfigurationKeysPublic.KMS_CLIENT_TIMEOUT_SECONDS,
             CommonConfigurationKeysPublic.KMS_CLIENT_TIMEOUT_DEFAULT);
     authRetry = conf.getInt(AUTH_RETRY, DEFAULT_AUTH_RETRY);
+    copyLegacyToken = conf.getBoolean(KMS_CLIENT_COPY_LEGACY_TOKEN_KEY,
+        KMS_CLIENT_COPY_LEGACY_TOKEN_DEFAULT);
+
     configurator = new TimeoutConnConfigurator(timeout, sslFactory);
     encKeyVersionQueue =
         new ValueQueue<KeyProviderCryptoExtension.EncryptedKeyVersion>(
@@ -400,6 +341,7 @@ public class KMSClientProvider extends KeyProvider implements CryptoExtension,
                     KMS_CLIENT_ENC_KEY_CACHE_NUM_REFILL_THREADS_DEFAULT),
             new EncryptedQueueRefiller());
     authToken = new DelegationTokenAuthenticatedURL.Token();
+    dtService = new Text(providerUri.toString());
     LOG.info("KMSClientProvider for KMS url: {} delegation token service: {}" +
         " created.", kmsUrl, dtService);
   }
@@ -473,7 +415,7 @@ public class KMSClientProvider extends KeyProvider implements CryptoExtension,
         @Override
         public HttpURLConnection run() throws Exception {
           DelegationTokenAuthenticatedURL authUrl =
-              new DelegationTokenAuthenticatedURL(configurator);
+              createKMSAuthenticatedURL();
           return authUrl.openConnection(url, authToken, doAsUser);
         }
       });
@@ -924,7 +866,7 @@ public class KMSClientProvider extends KeyProvider implements CryptoExtension,
       LOG.debug("Renewing delegation token {} with url:{}, as:{}",
           token, url, doAsUser);
       final DelegationTokenAuthenticatedURL authUrl =
-          new DelegationTokenAuthenticatedURL(configurator);
+          createKMSAuthenticatedURL();
       return getActualUgi().doAs(
           new PrivilegedExceptionAction<Long>() {
             @Override
@@ -956,7 +898,7 @@ public class KMSClientProvider extends KeyProvider implements CryptoExtension,
               LOG.debug("Cancelling delegation token {} with url:{}, as:{}",
                   dToken, url, doAsUser);
               final DelegationTokenAuthenticatedURL authUrl =
-                  new DelegationTokenAuthenticatedURL(configurator);
+                  createKMSAuthenticatedURL();
               authUrl.cancelDelegationToken(url, token, doAsUser);
               return null;
             }
@@ -1008,6 +950,17 @@ public class KMSClientProvider extends KeyProvider implements CryptoExtension,
     return token;
   }
 
+  @VisibleForTesting
+  DelegationTokenAuthenticatedURL createKMSAuthenticatedURL() {
+    return new DelegationTokenAuthenticatedURL(configurator) {
+      @Override
+      public org.apache.hadoop.security.token.Token<? extends TokenIdentifier>
+          getDelegationToken(URL url, Credentials creds) {
+        return selectKMSDelegationToken(creds);
+      }
+    };
+  }
+
   @Override
   public Token<?>[] addDelegationTokens(final String renewer,
       Credentials credentials) throws IOException {
@@ -1016,7 +969,7 @@ public class KMSClientProvider extends KeyProvider implements CryptoExtension,
     if (token == null) {
       final URL url = createURL(null, null, null, null);
       final DelegationTokenAuthenticatedURL authUrl =
-          new DelegationTokenAuthenticatedURL(configurator);
+          createKMSAuthenticatedURL();
       try {
         final String doAsUser = getDoAsUser();
         token = getActualUgi().doAs(new PrivilegedExceptionAction<Token<?>>() {
@@ -1030,9 +983,16 @@ public class KMSClientProvider extends KeyProvider implements CryptoExtension,
           }
         });
         if (token != null) {
-          LOG.debug("New token received: ({})", token);
+          if (KMSDelegationToken.TOKEN_KIND.equals(token.getKind())) {
+            // do not set service for legacy kind, for compatibility.
+            token.setService(dtService);
+          }
+          LOG.info("New token created: ({})", token);
           credentials.addToken(token.getService(), token);
-          tokens = new Token<?>[] { token };
+          Token<?> legacyToken = createAndAddLegacyToken(credentials, token);
+          tokens = legacyToken == null ?
+              new Token<?>[] {token} :
+              new Token<?>[] {token, legacyToken};
         } else {
           throw new IOException("Got NULL as delegation token");
         }
@@ -1049,13 +1009,75 @@ public class KMSClientProvider extends KeyProvider implements CryptoExtension,
     return tokens;
   }
 
-  private boolean containsKmsDt(UserGroupInformation ugi) throws IOException {
-    // Add existing credentials from the UGI, since provider is cached.
-    Credentials creds = ugi.getCredentials();
+  /**
+   * If {@link CommonConfigurationKeysPublic#KMS_CLIENT_COPY_LEGACY_TOKEN_KEY}
+   * is true when creating the provider, then copy the passed-in token of
+   * {@link KMSDelegationToken#TOKEN_KIND} and create a new token of
+   * {@link KMSDelegationToken#TOKEN_LEGACY_KIND}, and add it to credentials.
+   *
+   * @return The legacy token, or null if one should not be created.
+   */
+  private Token<?> createAndAddLegacyToken(Credentials credentials,
+      Token<?> token) {
+    if (!copyLegacyToken || !KMSDelegationToken.TOKEN_KIND
+        .equals(token.getKind())) {
+      LOG.debug("Not creating legacy token because copyLegacyToken={}, "
+          + "token={}", copyLegacyToken, token);
+      return null;
+    }
+    // copy a KMS_DELEGATION_TOKEN and create a new kms-dt with the same
+    // underlying token for backwards-compatibility. Old clients/renewers
+    // does not parse the new token and can only work with kms-dt.
+    final Token<?> legacyToken = token.copyToken();
+    legacyToken.setKind(KMSDelegationToken.TOKEN_LEGACY_KIND);
+    final InetSocketAddress addr =
+        new InetSocketAddress(kmsUrl.getHost(), kmsUrl.getPort());
+    final Text fallBackServiceText = SecurityUtil.buildTokenService(addr);
+    legacyToken.setService(fallBackServiceText);
+    LOG.info("Copied token to legacy kind: {}", legacyToken);
+    credentials.addToken(legacyToken.getService(), legacyToken);
+    return legacyToken;
+  }
+
+  @VisibleForTesting
+  public Text getDelegationTokenService() {
+    return dtService;
+  }
+
+  /**
+   * Given a list of tokens, return the token that should be used for KMS
+   * authentication.
+   */
+  @VisibleForTesting
+  Token selectKMSDelegationToken(Credentials creds) {
+    // always look for TOKEN_KIND first
+    final TokenSelector<AbstractDelegationTokenIdentifier> tokenSelector =
+        new AbstractDelegationTokenSelector<AbstractDelegationTokenIdentifier>(
+            KMSDelegationToken.TOKEN_KIND) {
+        };
+    Token token = tokenSelector.selectToken(dtService, creds.getAllTokens());
+    LOG.debug("Searching service {} found token {}", dtService, token);
+    if (token != null) {
+      return token;
+    }
+
+    // fall back to look for token by service, regardless of kind.
+    // this is old behavior, keeping for compatibility reasons (for example,
+    // even if KMS server is new, if the job is submitted with an old kms
+    // client, job runners on new version should be able to find the token).
+    final InetSocketAddress addr =
+        new InetSocketAddress(kmsUrl.getHost(), kmsUrl.getPort());
+    final Text fallBackServiceText = SecurityUtil.buildTokenService(addr);
+    token = creds.getToken(fallBackServiceText);
+    LOG.debug("Selected delegation token {} using service:{}", token,
+        fallBackServiceText);
+    return token;
+  }
+
+  private boolean containsKmsDt(UserGroupInformation ugi) {
+    final Credentials creds = ugi.getCredentials();
     if (!creds.getAllTokens().isEmpty()) {
-      LOG.debug("Searching for token that matches service: {}", dtService);
-      org.apache.hadoop.security.token.Token<? extends TokenIdentifier>
-          dToken = creds.getToken(dtService);
+      final Token dToken = selectKMSDelegationToken(creds);
       if (dToken != null) {
         return true;
       }

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

@@ -27,7 +27,10 @@ import org.apache.hadoop.security.token.delegation.web.DelegationTokenIdentifier
 @InterfaceAudience.Private
 public final class KMSDelegationToken {
 
-  public static final String TOKEN_KIND_STR = "kms-dt";
+  public static final String TOKEN_LEGACY_KIND_STR = "kms-dt";
+  public static final Text TOKEN_LEGACY_KIND = new Text(TOKEN_LEGACY_KIND_STR);
+
+  public static final String TOKEN_KIND_STR = "KMS_DELEGATION_TOKEN";
   public static final Text TOKEN_KIND = new Text(TOKEN_KIND_STR);
 
   // Utility class is not supposed to be instantiated.
@@ -49,4 +52,21 @@ public final class KMSDelegationToken {
       return TOKEN_KIND;
     }
   }
+
+  /**
+   * DelegationTokenIdentifier used for the KMS for legacy tokens.
+   */
+  @Deprecated
+  public static class KMSLegacyDelegationTokenIdentifier
+      extends DelegationTokenIdentifier {
+
+    public KMSLegacyDelegationTokenIdentifier() {
+      super(TOKEN_LEGACY_KIND);
+    }
+
+    @Override
+    public Text getKind() {
+      return TOKEN_LEGACY_KIND;
+    }
+  }
 }

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

@@ -0,0 +1,56 @@
+/**
+ * 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.crypto.key.kms;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.crypto.key.KeyProvider;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.util.KMSUtil;
+
+import java.io.IOException;
+
+import static org.apache.hadoop.crypto.key.kms.KMSDelegationToken.TOKEN_LEGACY_KIND;
+
+/**
+ * The {@link KMSTokenRenewer} that supports legacy tokens.
+ */
+@InterfaceAudience.Private
+@Deprecated
+public class KMSLegacyTokenRenewer extends KMSTokenRenewer {
+
+  @Override
+  public boolean handleKind(Text kind) {
+    return kind.equals(TOKEN_LEGACY_KIND);
+  }
+
+  /**
+   * Create a key provider for token renewal / cancellation.
+   * Caller is responsible for closing the key provider.
+   */
+  @Override
+  protected KeyProvider createKeyProvider(Token<?> token,
+      Configuration conf) throws IOException {
+    assert token.getKind().equals(TOKEN_LEGACY_KIND);
+    // Legacy tokens get service from configuration.
+    return KMSUtil.createKeyProvider(conf,
+        CommonConfigurationKeysPublic.HADOOP_SECURITY_KEY_PROVIDER_PATH);
+  }
+}

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

@@ -0,0 +1,103 @@
+/**
+ * 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.crypto.key.kms;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.crypto.key.KeyProvider;
+import org.apache.hadoop.crypto.key.KeyProviderDelegationTokenExtension;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenRenewer;
+import org.apache.hadoop.util.KMSUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+
+import static org.apache.hadoop.crypto.key.kms.KMSDelegationToken.TOKEN_KIND;
+
+/**
+ * The KMS implementation of {@link TokenRenewer}.
+ */
+@InterfaceAudience.Private
+public class KMSTokenRenewer extends TokenRenewer {
+
+  public static final Logger LOG = LoggerFactory
+      .getLogger(org.apache.hadoop.crypto.key.kms.KMSTokenRenewer.class);
+
+  @Override
+  public boolean handleKind(Text kind) {
+    return kind.equals(TOKEN_KIND);
+  }
+
+  @Override
+  public boolean isManaged(Token<?> token) throws IOException {
+    return true;
+  }
+
+  @Override
+  public long renew(Token<?> token, Configuration conf) throws IOException {
+    LOG.debug("Renewing delegation token {}", token);
+    final KeyProvider keyProvider = createKeyProvider(token, conf);
+    try {
+      if (!(keyProvider instanceof
+          KeyProviderDelegationTokenExtension.DelegationTokenExtension)) {
+        LOG.warn("keyProvider {} cannot renew token {}.",
+            keyProvider == null ? "null" : keyProvider.getClass(), token);
+        return 0;
+      }
+      return ((KeyProviderDelegationTokenExtension.DelegationTokenExtension)
+          keyProvider).renewDelegationToken(token);
+    } finally {
+      if (keyProvider != null) {
+        keyProvider.close();
+      }
+    }
+  }
+
+  @Override
+  public void cancel(Token<?> token, Configuration conf) throws IOException {
+    LOG.debug("Canceling delegation token {}", token);
+    final KeyProvider keyProvider = createKeyProvider(token, conf);
+    try {
+      if (!(keyProvider instanceof
+          KeyProviderDelegationTokenExtension.DelegationTokenExtension)) {
+        LOG.warn("keyProvider {} cannot cancel token {}.",
+            keyProvider == null ? "null" : keyProvider.getClass(), token);
+        return;
+      }
+      ((KeyProviderDelegationTokenExtension.DelegationTokenExtension)
+          keyProvider).cancelDelegationToken(token);
+    } finally {
+      if (keyProvider != null) {
+        keyProvider.close();
+      }
+    }
+  }
+
+  /**
+   * Create a key provider for token renewal / cancellation.
+   * Caller is responsible for closing the key provider.
+   */
+  protected KeyProvider createKeyProvider(Token<?> token,
+      Configuration conf) throws IOException {
+    return KMSUtil
+        .createKeyProviderFromTokenService(conf, token.getService().toString());
+  }
+}

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

@@ -0,0 +1,18 @@
+/**
+ * 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.crypto.key.kms;

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

@@ -677,7 +677,14 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
   public FileStatus[] listStatus(Path f) throws IOException {
     return fs.listStatus(f, DEFAULT_FILTER);
   }
-  
+
+  @Override
+  public RemoteIterator<FileStatus> listStatusIterator(final Path p)
+      throws IOException {
+    // Not-using fs#listStatusIterator() since it includes crc files as well
+    return new DirListingIterator<>(p);
+  }
+
   /**
    * List the statuses of the files/directories in the given path if the path is
    * a directory.

+ 10 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeysPublic.java

@@ -770,6 +770,16 @@ public class CommonConfigurationKeysPublic {
   /**  Default value is 100 ms. */
   public static final int KMS_CLIENT_FAILOVER_SLEEP_BASE_MILLIS_DEFAULT  = 100;
 
+  /**
+   * @see
+   * <a href="{@docRoot}/../hadoop-project-dist/hadoop-common/core-default.xml">
+   * core-default.xml</a>
+   */
+  public static final String KMS_CLIENT_COPY_LEGACY_TOKEN_KEY =
+      "hadoop.security.kms.client.copy.legacy.token";
+  /**  Default value is true. */
+  public static final boolean KMS_CLIENT_COPY_LEGACY_TOKEN_DEFAULT = true;
+
   /**
    * @see
    * <a href="{@docRoot}/../hadoop-project-dist/hadoop-common/core-default.xml">

+ 82 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CompositeCrcFileChecksum.java

@@ -0,0 +1,82 @@
+/**
+ * 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 java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.Options.ChecksumOpt;
+import org.apache.hadoop.util.CrcUtil;
+import org.apache.hadoop.util.DataChecksum;
+
+/** Composite CRC. */
+@InterfaceAudience.LimitedPrivate({"HDFS"})
+@InterfaceStability.Unstable
+public class CompositeCrcFileChecksum extends FileChecksum {
+  public static final int LENGTH = Integer.SIZE / Byte.SIZE;
+
+  private int crc;
+  private DataChecksum.Type crcType;
+  private int bytesPerCrc;
+
+  /** Create a CompositeCrcFileChecksum. */
+  public CompositeCrcFileChecksum(
+      int crc, DataChecksum.Type crcType, int bytesPerCrc) {
+    this.crc = crc;
+    this.crcType = crcType;
+    this.bytesPerCrc = bytesPerCrc;
+  }
+
+  @Override
+  public String getAlgorithmName() {
+    return "COMPOSITE-" + crcType.name();
+  }
+
+  @Override
+  public int getLength() {
+    return LENGTH;
+  }
+
+  @Override
+  public byte[] getBytes() {
+    return CrcUtil.intToBytes(crc);
+  }
+
+  @Override
+  public ChecksumOpt getChecksumOpt() {
+    return new ChecksumOpt(crcType, bytesPerCrc);
+  }
+
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    crc = in.readInt();
+  }
+
+  @Override
+  public void write(DataOutput out) throws IOException {
+    out.writeInt(crc);
+  }
+
+  @Override
+  public String toString() {
+    return getAlgorithmName() + ":" + String.format("0x%08x", crc);
+  }
+}

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

@@ -2147,7 +2147,7 @@ public abstract class FileSystem extends Configured implements Closeable {
   /**
    * Generic iterator for implementing {@link #listStatusIterator(Path)}.
    */
-  private class DirListingIterator<T extends FileStatus> implements
+  protected class DirListingIterator<T extends FileStatus> implements
       RemoteIterator<T> {
 
     private final Path path;

+ 11 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Options.java

@@ -504,4 +504,15 @@ public final class Options {
 
   }
 
+  /**
+   * Enum for indicating what mode to use when combining chunk and block
+   * checksums to define an aggregate FileChecksum. This should be considered
+   * a client-side runtime option rather than a persistent property of any
+   * stored metadata, which is why this is not part of ChecksumOpt, which
+   * deals with properties of files at rest.
+   */
+  public enum ChecksumCombineMode {
+    MD5MD5CRC,  // MD5 of block checksums, which are MD5 over chunk CRCs
+    COMPOSITE_CRC  // Block/chunk-independent composite CRC
+  }
 }

+ 62 - 7
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Command.java

@@ -33,6 +33,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.PathNotFoundException;
+import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.util.StringUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -325,20 +326,68 @@ abstract public class Command extends Configured {
    */
   protected void processPaths(PathData parent, PathData ... items)
   throws IOException {
-    // TODO: this really should be iterative
     for (PathData item : items) {
       try {
-        processPath(item);
-        if (recursive && isPathRecursable(item)) {
-          recursePath(item);
-        }
-        postProcessPath(item);
+        processPathInternal(item);
       } catch (IOException e) {
         displayError(e);
       }
     }
   }
 
+  /**
+   * Iterates over the given expanded paths and invokes
+   * {@link #processPath(PathData)} on each element. If "recursive" is true,
+   * will do a post-visit DFS on directories.
+   * @param parent if called via a recurse, will be the parent dir, else null
+   * @param itemsIterator a iterator of {@link PathData} objects to process
+   * @throws IOException if anything goes wrong...
+   */
+  protected void processPaths(PathData parent,
+      RemoteIterator<PathData> itemsIterator) throws IOException {
+    int groupSize = getListingGroupSize();
+    if (groupSize == 0) {
+      // No grouping of contents required.
+      while (itemsIterator.hasNext()) {
+        processPaths(parent, itemsIterator.next());
+      }
+    } else {
+      List<PathData> items = new ArrayList<PathData>(groupSize);
+      while (itemsIterator.hasNext()) {
+        items.add(itemsIterator.next());
+        if (!itemsIterator.hasNext() || items.size() == groupSize) {
+          processPaths(parent, items.toArray(new PathData[items.size()]));
+          items.clear();
+        }
+      }
+    }
+  }
+
+  private void processPathInternal(PathData item) throws IOException {
+    processPath(item);
+    if (recursive && isPathRecursable(item)) {
+      recursePath(item);
+    }
+    postProcessPath(item);
+  }
+
+  /**
+   * Whether the directory listing for a path should be sorted.?
+   * @return true/false.
+   */
+  protected boolean isSorted() {
+    return false;
+  }
+
+  /**
+   * While using iterator method for listing for a path, whether to group items
+   * and process as array? If so what is the size of array?
+   * @return size of the grouping array.
+   */
+  protected int getListingGroupSize() {
+    return 0;
+  }
+
   /**
    * Determines whether a {@link PathData} item is recursable. Default
    * implementation is to recurse directories but can be overridden to recurse
@@ -384,7 +433,13 @@ abstract public class Command extends Configured {
   protected void recursePath(PathData item) throws IOException {
     try {
       depth++;
-      processPaths(item, item.getDirectoryContents());
+      if (isSorted()) {
+        // use the non-iterative method for listing because explicit sorting is
+        // required. Iterators not guaranteed to return sorted elements
+        processPaths(item, item.getDirectoryContents());
+      } else {
+        processPaths(item, item.getDirectoryContentsIterator());
+      }
     } finally {
       depth--;
     }

+ 6 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/CopyCommands.java

@@ -142,6 +142,12 @@ class CopyCommands {
         srcs.add(src);
       }
     }
+
+    @Override
+    protected boolean isSorted() {
+      //Sort the children for merge
+      return true;
+    }
   }
 
   static class Cp extends CommandWithDestination {

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

@@ -230,8 +230,30 @@ class Ls extends FsCommand {
   }
 
   @Override
-  protected void processPaths(PathData parent, PathData ... items)
-  throws IOException {
+  protected boolean isSorted() {
+    // use the non-iterative method for listing because explicit sorting is
+    // required based on time/size/reverse or Total number of entries
+    // required to print summary first when non-recursive.
+    return !isRecursive() || isOrderTime() || isOrderSize() || isOrderReverse();
+  }
+
+  @Override
+  protected int getListingGroupSize() {
+    if (pathOnly) {
+      // If there is a need of printing only paths, then no grouping required
+      return 0;
+    }
+    /*
+     * LS output should be formatted properly. Grouping 100 items and formatting
+     * the output to reduce the creation of huge sized arrays. This method will
+     * be called only when recursive is set.
+     */
+    return 100;
+  }
+
+  @Override
+  protected void processPaths(PathData parent, PathData... items)
+      throws IOException {
     if (parent != null && !isRecursive() && items.length != 0) {
       if (!pathOnly) {
         out.println("Found " + items.length + " items");

+ 27 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/PathData.java

@@ -37,6 +37,7 @@ import org.apache.hadoop.fs.PathIOException;
 import org.apache.hadoop.fs.PathIsDirectoryException;
 import org.apache.hadoop.fs.PathIsNotDirectoryException;
 import org.apache.hadoop.fs.PathNotFoundException;
+import org.apache.hadoop.fs.RemoteIterator;
 
 /**
  * Encapsulates a Path (path), its FileStatus (stat), and its FileSystem (fs).
@@ -276,6 +277,32 @@ public class PathData implements Comparable<PathData> {
     return items;
   }
 
+  /**
+   * Returns a RemoteIterator for PathData objects of the items contained in the
+   * given directory.
+   * @return remote iterator of PathData objects for its children
+   * @throws IOException if anything else goes wrong...
+   */
+  public RemoteIterator<PathData> getDirectoryContentsIterator()
+      throws IOException {
+    checkIfExists(FileTypeRequirement.SHOULD_BE_DIRECTORY);
+    final RemoteIterator<FileStatus> stats = this.fs.listStatusIterator(path);
+    return new RemoteIterator<PathData>() {
+
+      @Override
+      public boolean hasNext() throws IOException {
+        return stats.hasNext();
+      }
+
+      @Override
+      public PathData next() throws IOException {
+        FileStatus file = stats.next();
+        String child = getStringForChildPath(file.getPath());
+        return new PathData(fs, child, file);
+      }
+    };
+  }
+
   /**
    * Creates a new object for a child entry in this directory
    * @param child the basename will be appended to this object's path

+ 16 - 5
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticatedURL.java

@@ -300,11 +300,7 @@ public class DelegationTokenAuthenticatedURL extends AuthenticatedURL {
             creds.getAllTokens());
       }
       if (!creds.getAllTokens().isEmpty()) {
-        InetSocketAddress serviceAddr = new InetSocketAddress(url.getHost(),
-            url.getPort());
-        Text service = SecurityUtil.buildTokenService(serviceAddr);
-        dToken = creds.getToken(service);
-        LOG.debug("Using delegation token {} from service:{}", dToken, service);
+        dToken = getDelegationToken(url, creds);
         if (dToken != null) {
           if (useQueryStringForDelegationToken()) {
             // delegation token will go in the query string, injecting it
@@ -340,6 +336,21 @@ public class DelegationTokenAuthenticatedURL extends AuthenticatedURL {
     return conn;
   }
 
+  /**
+   * Select a delegation token from all tokens in credentials, based on url.
+   */
+  @InterfaceAudience.Private
+  public org.apache.hadoop.security.token.Token<? extends TokenIdentifier>
+      getDelegationToken(URL url, Credentials creds) {
+    final InetSocketAddress serviceAddr =
+        new InetSocketAddress(url.getHost(), url.getPort());
+    final Text service = SecurityUtil.buildTokenService(serviceAddr);
+    org.apache.hadoop.security.token.Token<? extends TokenIdentifier> dToken =
+        creds.getToken(service);
+    LOG.debug("Selected delegation token {} using service:{}", dToken, service);
+    return dToken;
+  }
+
   /**
    * Requests a delegation token using the configured <code>Authenticator</code>
    * for authentication.

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

@@ -81,7 +81,7 @@ import com.google.common.annotations.VisibleForTesting;
 @InterfaceStability.Evolving
 public abstract class DelegationTokenAuthenticationHandler
     implements AuthenticationHandler {
-  private static final Logger LOG =
+  public static final Logger LOG =
       LoggerFactory.getLogger(DelegationTokenAuthenticationHandler.class);
 
   protected static final String TYPE_POSTFIX = "-dt";
@@ -224,7 +224,8 @@ public abstract class DelegationTokenAuthenticationHandler
       HttpServletRequest request, HttpServletResponse response)
       throws IOException, AuthenticationException {
     boolean requestContinues = true;
-    LOG.trace("Processing operation for req=({}), token: {}", request, token);
+    LOG.trace("Processing operation for req=({}), token: {}",
+        request.getRequestURL(), token);
     String op = ServletUtils.getParameter(request,
         KerberosDelegationTokenAuthenticator.OP_PARAM);
     op = (op != null) ? StringUtils.toUpperCase(op) : null;
@@ -407,7 +408,8 @@ public abstract class DelegationTokenAuthenticationHandler
             HttpServletResponse.SC_FORBIDDEN, new AuthenticationException(ex));
       }
     } else {
-      LOG.debug("Falling back to {} (req={})", authHandler.getClass(), request);
+      LOG.debug("Falling back to {} (req={})", authHandler.getClass(),
+          request.getRequestURL());
       token = authHandler.authenticate(request, response);
     }
     return token;

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

@@ -50,7 +50,7 @@ import java.util.Map;
 @InterfaceAudience.Public
 @InterfaceStability.Evolving
 public abstract class DelegationTokenAuthenticator implements Authenticator {
-  private static Logger LOG = 
+  public static final Logger LOG =
       LoggerFactory.getLogger(DelegationTokenAuthenticator.class);
   
   private static final String CONTENT_TYPE = "Content-Type";

+ 1 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/service/launcher/IrqHandler.java

@@ -86,7 +86,7 @@ public final class IrqHandler implements SignalHandler {
    * Bind to the interrupt handler.
    * @throws IllegalArgumentException if the exception could not be set
    */
-  void bind() {
+  public void bind() {
     Preconditions.checkState(signal == null, "Handler already bound");
     try {
       signal = new Signal(name);

+ 187 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/CrcComposer.java

@@ -0,0 +1,187 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.util;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.IOException;
+
+/**
+ * Encapsulates logic for composing multiple CRCs into one or more combined CRCs
+ * corresponding to concatenated underlying data ranges. Optimized for composing
+ * a large number of CRCs that correspond to underlying chunks of data all of
+ * same size.
+ */
+@InterfaceAudience.LimitedPrivate({"Common", "HDFS", "MapReduce", "Yarn"})
+@InterfaceStability.Unstable
+public class CrcComposer {
+  private static final int CRC_SIZE_BYTES = 4;
+  private static final Logger LOG = LoggerFactory.getLogger(CrcComposer.class);
+
+  private final int crcPolynomial;
+  private final int precomputedMonomialForHint;
+  private final long bytesPerCrcHint;
+  private final long stripeLength;
+
+  private int curCompositeCrc = 0;
+  private long curPositionInStripe = 0;
+  private ByteArrayOutputStream digestOut = new ByteArrayOutputStream();
+
+  /**
+   * Returns a CrcComposer which will collapse all ingested CRCs into a single
+   * value.
+   */
+  public static CrcComposer newCrcComposer(
+      DataChecksum.Type type, long bytesPerCrcHint)
+      throws IOException {
+    return newStripedCrcComposer(type, bytesPerCrcHint, Long.MAX_VALUE);
+  }
+
+  /**
+   * Returns a CrcComposer which will collapse CRCs for every combined
+   * underlying data size which aligns with the specified stripe boundary. For
+   * example, if "update" is called with 20 CRCs and bytesPerCrc == 5, and
+   * stripeLength == 10, then every two (10 / 5) consecutive CRCs will be
+   * combined with each other, yielding a list of 10 CRC "stripes" in the
+   * final digest, each corresponding to 10 underlying data bytes. Using
+   * a stripeLength greater than the total underlying data size is equivalent
+   * to using a non-striped CrcComposer.
+   */
+  public static CrcComposer newStripedCrcComposer(
+      DataChecksum.Type type, long bytesPerCrcHint, long stripeLength)
+      throws IOException {
+    int polynomial = DataChecksum.getCrcPolynomialForType(type);
+    return new CrcComposer(
+        polynomial,
+        CrcUtil.getMonomial(bytesPerCrcHint, polynomial),
+        bytesPerCrcHint,
+        stripeLength);
+  }
+
+  CrcComposer(
+      int crcPolynomial,
+      int precomputedMonomialForHint,
+      long bytesPerCrcHint,
+      long stripeLength) {
+    LOG.debug(
+        "crcPolynomial=0x{}, precomputedMonomialForHint=0x{}, "
+        + "bytesPerCrcHint={}, stripeLength={}",
+        Integer.toString(crcPolynomial, 16),
+        Integer.toString(precomputedMonomialForHint, 16),
+        bytesPerCrcHint,
+        stripeLength);
+    this.crcPolynomial = crcPolynomial;
+    this.precomputedMonomialForHint = precomputedMonomialForHint;
+    this.bytesPerCrcHint = bytesPerCrcHint;
+    this.stripeLength = stripeLength;
+  }
+
+  /**
+   * Composes length / CRC_SIZE_IN_BYTES more CRCs from crcBuffer, with
+   * each CRC expected to correspond to exactly {@code bytesPerCrc} underlying
+   * data bytes.
+   *
+   * @param length must be a multiple of the expected byte-size of a CRC.
+   */
+  public void update(
+      byte[] crcBuffer, int offset, int length, long bytesPerCrc)
+      throws IOException {
+    if (length % CRC_SIZE_BYTES != 0) {
+      throw new IOException(String.format(
+          "Trying to update CRC from byte array with length '%d' at offset "
+          + "'%d' which is not a multiple of %d!",
+          length, offset, CRC_SIZE_BYTES));
+    }
+    int limit = offset + length;
+    while (offset < limit) {
+      int crcB = CrcUtil.readInt(crcBuffer, offset);
+      update(crcB, bytesPerCrc);
+      offset += CRC_SIZE_BYTES;
+    }
+  }
+
+  /**
+   * Composes {@code numChecksumsToRead} additional CRCs into the current digest
+   * out of {@code checksumIn}, with each CRC expected to correspond to exactly
+   * {@code bytesPerCrc} underlying data bytes.
+   */
+  public void update(
+      DataInputStream checksumIn, long numChecksumsToRead, long bytesPerCrc)
+      throws IOException {
+    for (long i = 0; i < numChecksumsToRead; ++i) {
+      int crcB = checksumIn.readInt();
+      update(crcB, bytesPerCrc);
+    }
+  }
+
+  /**
+   * Updates with a single additional CRC which corresponds to an underlying
+   * data size of {@code bytesPerCrc}.
+   */
+  public void update(int crcB, long bytesPerCrc) throws IOException {
+    if (curCompositeCrc == 0) {
+      curCompositeCrc = crcB;
+    } else if (bytesPerCrc == bytesPerCrcHint) {
+      curCompositeCrc = CrcUtil.composeWithMonomial(
+          curCompositeCrc, crcB, precomputedMonomialForHint, crcPolynomial);
+    } else {
+      curCompositeCrc = CrcUtil.compose(
+          curCompositeCrc, crcB, bytesPerCrc, crcPolynomial);
+    }
+
+    curPositionInStripe += bytesPerCrc;
+
+    if (curPositionInStripe > stripeLength) {
+      throw new IOException(String.format(
+          "Current position in stripe '%d' after advancing by bytesPerCrc '%d' "
+          + "exceeds stripeLength '%d' without stripe alignment.",
+          curPositionInStripe, bytesPerCrc, stripeLength));
+    } else if (curPositionInStripe == stripeLength) {
+      // Hit a stripe boundary; flush the curCompositeCrc and reset for next
+      // stripe.
+      digestOut.write(CrcUtil.intToBytes(curCompositeCrc), 0, CRC_SIZE_BYTES);
+      curCompositeCrc = 0;
+      curPositionInStripe = 0;
+    }
+  }
+
+  /**
+   * Returns byte representation of composed CRCs; if no stripeLength was
+   * specified, the digest should be of length equal to exactly one CRC.
+   * Otherwise, the number of CRCs in the returned array is equal to the
+   * total sum bytesPerCrc divided by stripeLength. If the sum of bytesPerCrc
+   * is not a multiple of stripeLength, then the last CRC in the array
+   * corresponds to totalLength % stripeLength underlying data bytes.
+   */
+  public byte[] digest() {
+    if (curPositionInStripe > 0) {
+      digestOut.write(CrcUtil.intToBytes(curCompositeCrc), 0, CRC_SIZE_BYTES);
+      curCompositeCrc = 0;
+      curPositionInStripe = 0;
+    }
+    byte[] digestValue = digestOut.toByteArray();
+    digestOut.reset();
+    return digestValue;
+  }
+}

+ 220 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/CrcUtil.java

@@ -0,0 +1,220 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.util;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+import java.io.IOException;
+import java.util.Arrays;
+
+/**
+ * This class provides utilities for working with CRCs.
+ */
+@InterfaceAudience.LimitedPrivate({"Common", "HDFS", "MapReduce", "Yarn"})
+@InterfaceStability.Unstable
+public final class CrcUtil {
+  public static final int MULTIPLICATIVE_IDENTITY = 0x80000000;
+  public static final int GZIP_POLYNOMIAL = 0xEDB88320;
+  public static final int CASTAGNOLI_POLYNOMIAL = 0x82F63B78;
+
+  /**
+   * Hide default constructor for a static utils class.
+   */
+  private CrcUtil() {
+  }
+
+  /**
+   * Compute x^({@code lengthBytes} * 8) mod {@code mod}, where {@code mod} is
+   * in "reversed" (little-endian) format such that {@code mod & 1} represents
+   * x^31 and has an implicit term x^32.
+   */
+  public static int getMonomial(long lengthBytes, int mod) {
+    if (lengthBytes == 0) {
+      return MULTIPLICATIVE_IDENTITY;
+    } else if (lengthBytes < 0) {
+      throw new IllegalArgumentException(
+          "lengthBytes must be positive, got " + lengthBytes);
+    }
+
+    // Decompose into
+    // x^degree == x ^ SUM(bit[i] * 2^i) == PRODUCT(x ^ (bit[i] * 2^i))
+    // Generate each x^(2^i) by squaring.
+    // Since 'degree' is in 'bits', but we only need to support byte
+    // granularity we can begin with x^8.
+    int multiplier = MULTIPLICATIVE_IDENTITY >>> 8;
+    int product = MULTIPLICATIVE_IDENTITY;
+    long degree = lengthBytes;
+    while (degree > 0) {
+      if ((degree & 1) != 0) {
+        product = (product == MULTIPLICATIVE_IDENTITY) ? multiplier :
+            galoisFieldMultiply(product, multiplier, mod);
+      }
+      multiplier = galoisFieldMultiply(multiplier, multiplier, mod);
+      degree >>= 1;
+    }
+    return product;
+  }
+
+  /**
+   * @param monomial Precomputed x^(lengthBInBytes * 8) mod {@code mod}
+   */
+  public static int composeWithMonomial(
+      int crcA, int crcB, int monomial, int mod) {
+    return galoisFieldMultiply(crcA, monomial, mod) ^ crcB;
+  }
+
+  /**
+   * @param lengthB length of content corresponding to {@code crcB}, in bytes.
+   */
+  public static int compose(int crcA, int crcB, long lengthB, int mod) {
+    int monomial = getMonomial(lengthB, mod);
+    return composeWithMonomial(crcA, crcB, monomial, mod);
+  }
+
+  /**
+   * @return 4-byte array holding the big-endian representation of
+   *     {@code value}.
+   */
+  public static byte[] intToBytes(int value) {
+    byte[] buf = new byte[4];
+    try {
+      writeInt(buf, 0, value);
+    } catch (IOException ioe) {
+      // Since this should only be able to occur from code bugs within this
+      // class rather than user input, we throw as a RuntimeException
+      // rather than requiring this method to declare throwing IOException
+      // for something the caller can't control.
+      throw new RuntimeException(ioe);
+    }
+    return buf;
+  }
+
+  /**
+   * Writes big-endian representation of {@code value} into {@code buf}
+   * starting at {@code offset}. buf.length must be greater than or
+   * equal to offset + 4.
+   */
+  public static void writeInt(byte[] buf, int offset, int value)
+      throws IOException {
+    if (offset + 4  > buf.length) {
+      throw new IOException(String.format(
+          "writeInt out of bounds: buf.length=%d, offset=%d",
+          buf.length, offset));
+    }
+    buf[offset + 0] = (byte)((value >>> 24) & 0xff);
+    buf[offset + 1] = (byte)((value >>> 16) & 0xff);
+    buf[offset + 2] = (byte)((value >>> 8) & 0xff);
+    buf[offset + 3] = (byte)(value & 0xff);
+  }
+
+  /**
+   * Reads 4-byte big-endian int value from {@code buf} starting at
+   * {@code offset}. buf.length must be greater than or equal to offset + 4.
+   */
+  public static int readInt(byte[] buf, int offset)
+      throws IOException {
+    if (offset + 4  > buf.length) {
+      throw new IOException(String.format(
+          "readInt out of bounds: buf.length=%d, offset=%d",
+          buf.length, offset));
+    }
+    int value = ((buf[offset + 0] & 0xff) << 24) |
+                ((buf[offset + 1] & 0xff) << 16) |
+                ((buf[offset + 2] & 0xff) << 8)  |
+                ((buf[offset + 3] & 0xff));
+    return value;
+  }
+
+  /**
+   * For use with debug statements; verifies bytes.length on creation,
+   * expecting it to represent exactly one CRC, and returns a hex
+   * formatted value.
+   */
+  public static String toSingleCrcString(final byte[] bytes)
+      throws IOException {
+    if (bytes.length != 4) {
+      throw new IOException((String.format(
+          "Unexpected byte[] length '%d' for single CRC. Contents: %s",
+          bytes.length, Arrays.toString(bytes))));
+    }
+    return String.format("0x%08x", readInt(bytes, 0));
+  }
+
+  /**
+   * For use with debug statements; verifies bytes.length on creation,
+   * expecting it to be divisible by CRC byte size, and returns a list of
+   * hex formatted values.
+   */
+  public static String toMultiCrcString(final byte[] bytes)
+      throws IOException {
+    if (bytes.length % 4 != 0) {
+      throw new IOException((String.format(
+          "Unexpected byte[] length '%d' not divisible by 4. Contents: %s",
+          bytes.length, Arrays.toString(bytes))));
+    }
+    StringBuilder sb = new StringBuilder();
+    sb.append('[');
+    for (int i = 0; i < bytes.length; i += 4) {
+      sb.append(String.format("0x%08x", readInt(bytes, i)));
+      if (i != bytes.length - 4) {
+        sb.append(", ");
+      }
+    }
+    sb.append(']');
+    return sb.toString();
+  }
+
+  /**
+   * Galois field multiplication of {@code p} and {@code q} with the
+   * generator polynomial {@code m} as the modulus.
+   *
+   * @param m The little-endian polynomial to use as the modulus when
+   *     multiplying p and q, with implicit "1" bit beyond the bottom bit.
+   */
+  private static int galoisFieldMultiply(int p, int q, int m) {
+    int summation = 0;
+
+    // Top bit is the x^0 place; each right-shift increments the degree of the
+    // current term.
+    int curTerm = MULTIPLICATIVE_IDENTITY;
+
+    // Iteratively multiply p by x mod m as we go to represent the q[i] term
+    // (of degree x^i) times p.
+    int px = p;
+
+    while (curTerm != 0) {
+      if ((q & curTerm) != 0) {
+        summation ^= px;
+      }
+
+      // Bottom bit represents highest degree since we're little-endian; before
+      // we multiply by "x" for the next term, check bottom bit to know whether
+      // the resulting px will thus have a term matching the implicit "1" term
+      // of "m" and thus will need to subtract "m" after mutiplying by "x".
+      boolean hasMaxDegree = ((px & 1) != 0);
+      px >>>= 1;
+      if (hasMaxDegree) {
+        px ^= m;
+      }
+      curTerm >>>= 1;
+    }
+    return summation;
+  }
+}

+ 18 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/DataChecksum.java

@@ -104,6 +104,24 @@ public class DataChecksum implements Checksum {
     }
   }
 
+  /**
+   * @return the int representation of the polynomial associated with the
+   *     CRC {@code type}, suitable for use with further CRC arithmetic.
+   * @throws IOException if there is no CRC polynomial applicable
+   *     to the given {@code type}.
+   */
+  public static int getCrcPolynomialForType(Type type) throws IOException {
+    switch (type) {
+    case CRC32:
+      return CrcUtil.GZIP_POLYNOMIAL;
+    case CRC32C:
+      return CrcUtil.CASTAGNOLI_POLYNOMIAL;
+    default:
+      throw new IOException(
+          "No CRC polynomial could be associated with type: " + type);
+    }
+  }
+
   public static DataChecksum newDataChecksum(Type type, int bytesPerChecksum ) {
     if ( bytesPerChecksum <= 0 ) {
       return null;

+ 43 - 2
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/KMSUtil.java

@@ -30,6 +30,7 @@ import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
 import java.net.URI;
+import java.net.URISyntaxException;
 import java.util.ArrayList;
 import java.util.Date;
 import java.util.HashMap;
@@ -41,8 +42,7 @@ import java.util.Map;
  */
 @InterfaceAudience.Private
 public final class KMSUtil {
-  public static final Logger LOG =
-      LoggerFactory.getLogger(KMSUtil.class);
+  public static final Logger LOG = LoggerFactory.getLogger(KMSUtil.class);
 
   private KMSUtil() { /* Hidden constructor */ }
 
@@ -64,6 +64,13 @@ public final class KMSUtil {
     if (providerUriStr == null || providerUriStr.isEmpty()) {
       return null;
     }
+    KeyProvider kp = KMSUtilFaultInjector.get().createKeyProviderForTests(
+        providerUriStr, conf);
+    if (kp != null) {
+      LOG.info("KeyProvider is created with uri: {}. This should happen only " +
+              "in tests.", providerUriStr);
+      return kp;
+    }
     return createKeyProviderFromUri(conf, URI.create(providerUriStr));
   }
 
@@ -205,4 +212,38 @@ public final class KMSUtil {
     }
     return metadata;
   }
+
+  /**
+   * Creates a key provider from token service field, which must be URI format.
+   *
+   * @param conf
+   * @param tokenServiceValue
+   * @return new KeyProvider or null
+   * @throws IOException
+   */
+  public static KeyProvider createKeyProviderFromTokenService(
+      final Configuration conf, final String tokenServiceValue)
+      throws IOException {
+    LOG.debug("Creating key provider from token service value {}. ",
+        tokenServiceValue);
+    final KeyProvider kp = KMSUtilFaultInjector.get()
+        .createKeyProviderForTests(tokenServiceValue, conf);
+    if (kp != null) {
+      LOG.info("KeyProvider is created with uri: {}. This should happen only "
+          + "in tests.", tokenServiceValue);
+      return kp;
+    }
+    if (!tokenServiceValue.contains("://")) {
+      throw new IllegalArgumentException(
+          "Invalid token service " + tokenServiceValue);
+    }
+    final URI tokenServiceUri;
+    try {
+      tokenServiceUri = new URI(tokenServiceValue);
+    } catch (URISyntaxException e) {
+      throw new IllegalArgumentException(
+          "Invalid token service " + tokenServiceValue, e);
+    }
+    return createKeyProviderFromUri(conf, tokenServiceUri);
+  }
 }

+ 49 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/KMSUtilFaultInjector.java

@@ -0,0 +1,49 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.util;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.crypto.key.KeyProvider;
+
+import java.io.IOException;
+
+/**
+ * Used for returning custom KeyProvider from test methods.
+ */
+@VisibleForTesting
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class KMSUtilFaultInjector {
+  private static KMSUtilFaultInjector instance = new KMSUtilFaultInjector();
+
+  public static KMSUtilFaultInjector get() {
+    return instance;
+  }
+
+  public static void set(KMSUtilFaultInjector injector) {
+    instance = injector;
+  }
+
+  public KeyProvider createKeyProviderForTests(String value, Configuration conf)
+      throws IOException {
+    return null;
+  }
+}

+ 1 - 0
hadoop-common-project/hadoop-common/src/main/resources/META-INF/services/org.apache.hadoop.security.token.TokenIdentifier

@@ -12,3 +12,4 @@
 #   limitations under the License.
 #
 org.apache.hadoop.crypto.key.kms.KMSDelegationToken$KMSDelegationTokenIdentifier
+org.apache.hadoop.crypto.key.kms.KMSDelegationToken$KMSLegacyDelegationTokenIdentifier

+ 2 - 1
hadoop-common-project/hadoop-common/src/main/resources/META-INF/services/org.apache.hadoop.security.token.TokenRenewer

@@ -11,4 +11,5 @@
 #   See the License for the specific language governing permissions and
 #   limitations under the License.
 #
-org.apache.hadoop.crypto.key.kms.KMSClientProvider$KMSTokenRenewer
+org.apache.hadoop.crypto.key.kms.KMSTokenRenewer
+org.apache.hadoop.crypto.key.kms.KMSLegacyTokenRenewer

+ 20 - 0
hadoop-common-project/hadoop-common/src/main/resources/core-default.xml

@@ -2615,6 +2615,26 @@
   </description>
 </property>
 
+<property>
+  <name>hadoop.security.kms.client.copy.legacy.token</name>
+  <value>true</value>
+  <description>
+    Expert only. Whether the KMS client provider should copy a token to legacy
+    kind. This is for KMS_DELEGATION_TOKEN to be backwards compatible. With the
+    default value set to true, the client will locally duplicate the
+    KMS_DELEGATION_TOKEN token and create a kms-dt token, with the service field
+    conforming to kms-dt. All other parts of the token remain the same.
+    Then the new clients will use KMS_DELEGATION_TOKEN and old clients will
+    use kms-dt to authenticate. Default value is true.
+    You should only change this to false if you know all the KMS servers
+    , clients (including both job submitters and job runners) and the
+    token renewers (usually Yarn RM) are on a version that supports
+    KMS_DELEGATION_TOKEN.
+    Turning this off prematurely may result in old clients failing to
+    authenticate with new servers.
+  </description>
+</property>
+
 <property>
   <name>hadoop.security.kms.client.failover.sleep.max.millis</name>
   <value>2000</value>

+ 20 - 6
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfiguration.java

@@ -2320,19 +2320,33 @@ public class TestConfiguration {
     FileUtil.fullyDelete(tmpDir);
   }
 
+  @Test
   public void testGettingPropertiesWithPrefix() throws Exception {
     Configuration conf = new Configuration();
     for (int i = 0; i < 10; i++) {
-      conf.set("prefix" + ".name" + i, "value");
+      conf.set("prefix." + "name" + i, "value" + i);
     }
     conf.set("different.prefix" + ".name", "value");
-    Map<String, String> props = conf.getPropsWithPrefix("prefix");
-    assertEquals(props.size(), 10);
+    Map<String, String> prefixedProps = conf.getPropsWithPrefix("prefix.");
+    assertEquals(prefixedProps.size(), 10);
+    for (int i = 0; i < 10; i++) {
+      assertEquals("value" + i, prefixedProps.get("name" + i));
+    }
 
+    // Repeat test with variable substitution
+    conf.set("foo", "bar");
+    for (int i = 0; i < 10; i++) {
+      conf.set("subprefix." + "subname" + i, "value_${foo}" + i);
+    }
+    prefixedProps = conf.getPropsWithPrefix("subprefix.");
+    assertEquals(prefixedProps.size(), 10);
+    for (int i = 0; i < 10; i++) {
+      assertEquals("value_bar" + i, prefixedProps.get("subname" + i));
+    }
     // test call with no properties for a given prefix
-    props = conf.getPropsWithPrefix("none");
-    assertNotNull(props.isEmpty());
-    assertTrue(props.isEmpty());
+    prefixedProps = conf.getPropsWithPrefix("none");
+    assertNotNull(prefixedProps.isEmpty());
+    assertTrue(prefixedProps.isEmpty());
   }
 
   public static void main(String[] argv) throws Exception {

+ 162 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/key/kms/TestKMSClientProvider.java

@@ -0,0 +1,162 @@
+/**
+ * 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.crypto.key.kms;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.security.Credentials;
+import org.apache.hadoop.security.SecurityUtil;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.delegation.web.DelegationTokenAuthenticatedURL;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.Timeout;
+import org.slf4j.event.Level;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.net.URI;
+import java.net.URL;
+
+import static org.apache.hadoop.crypto.key.kms.KMSDelegationToken.TOKEN_KIND;
+import static org.apache.hadoop.crypto.key.kms.KMSDelegationToken.TOKEN_LEGACY_KIND;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.when;
+
+/**
+ * Unit test for {@link KMSClientProvider} class.
+ */
+public class TestKMSClientProvider {
+
+  public static final Logger LOG =
+      LoggerFactory.getLogger(TestKMSClientProvider.class);
+
+  private final Token token = new Token();
+  private final Token legacyToken = new Token();
+  private final String uriString = "kms://https@host:16000/kms";
+  private final String legacyTokenService = "host:16000";
+
+  @Rule
+  public Timeout globalTimeout = new Timeout(30000);
+
+  {
+    GenericTestUtils.setLogLevel(KMSClientProvider.LOG, Level.TRACE);
+  }
+
+  @Before
+  public void setup() {
+    SecurityUtil.setTokenServiceUseIp(false);
+    token.setKind(TOKEN_KIND);
+    token.setService(new Text(uriString));
+    legacyToken.setKind(TOKEN_LEGACY_KIND);
+    legacyToken.setService(new Text(legacyTokenService));
+  }
+
+  @Test
+  public void testNotCopyFromLegacyToken() throws Exception {
+    final DelegationTokenAuthenticatedURL url =
+        mock(DelegationTokenAuthenticatedURL.class);
+    final Configuration conf = new Configuration();
+    final URI uri = new URI(uriString);
+    final KMSClientProvider kp = new KMSClientProvider(uri, conf, uri);
+    try {
+      final KMSClientProvider spyKp = spy(kp);
+      when(spyKp.createKMSAuthenticatedURL()).thenReturn(url);
+      when(url.getDelegationToken(any(URL.class),
+          any(DelegationTokenAuthenticatedURL.Token.class), any(String.class),
+          any(String.class))).thenReturn(legacyToken);
+
+      final Credentials creds = new Credentials();
+      final Token<?>[] tokens = spyKp.addDelegationTokens("yarn", creds);
+      LOG.info("Got tokens: {}", tokens);
+      assertEquals(1, tokens.length);
+      LOG.info("uri:" + uriString);
+      // if KMS server returned a legacy token, new client should leave the
+      // service being legacy and not set uri string
+      assertEquals(legacyTokenService, tokens[0].getService().toString());
+    } finally {
+      kp.close();
+    }
+  }
+
+  @Test
+  public void testCopyFromToken() throws Exception {
+    final DelegationTokenAuthenticatedURL url =
+        mock(DelegationTokenAuthenticatedURL.class);
+    final Configuration conf = new Configuration();
+    final URI uri = new URI(uriString);
+    final KMSClientProvider kp = new KMSClientProvider(uri, conf, uri);
+    try {
+      final KMSClientProvider spyKp = spy(kp);
+      when(spyKp.createKMSAuthenticatedURL()).thenReturn(url);
+      when(url.getDelegationToken(any(URL.class),
+          any(DelegationTokenAuthenticatedURL.Token.class), any(String.class),
+          any(String.class))).thenReturn(token);
+
+      final Credentials creds = new Credentials();
+      final Token<?>[] tokens = spyKp.addDelegationTokens("yarn", creds);
+      LOG.info("Got tokens: {}", tokens);
+      assertEquals(2, tokens.length);
+      assertTrue(creds.getAllTokens().contains(token));
+      assertNotNull(creds.getToken(legacyToken.getService()));
+    } finally {
+      kp.close();
+    }
+  }
+
+  @Test
+  public void testSelectTokenWhenBothExist() throws Exception {
+    final Credentials creds = new Credentials();
+    final Configuration conf = new Configuration();
+    final URI uri = new URI(uriString);
+    final KMSClientProvider kp = new KMSClientProvider(uri, conf, uri);
+    try {
+      creds.addToken(token.getService(), token);
+      creds.addToken(legacyToken.getService(), legacyToken);
+      Token t = kp.selectKMSDelegationToken(creds);
+      assertEquals(token, t);
+    } finally {
+      kp.close();
+    }
+  }
+
+  @Test
+  public void testSelectTokenLegacyService() throws Exception {
+    final Configuration conf = new Configuration();
+    final URI uri = new URI(uriString);
+    final KMSClientProvider kp = new KMSClientProvider(uri, conf, uri);
+    try {
+      Text legacyService = new Text(legacyTokenService);
+      token.setService(legacyService);
+      final Credentials creds = new Credentials();
+      creds.addToken(legacyService, token);
+      Token t = kp.selectKMSDelegationToken(creds);
+      assertEquals(token, t);
+    } finally {
+      kp.close();
+    }
+  }
+}

+ 51 - 16
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/key/kms/TestLoadBalancingKMSClientProvider.java

@@ -42,7 +42,8 @@ import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.authentication.client.AuthenticationException;
 import org.apache.hadoop.security.authorize.AuthorizationException;
-import org.junit.After;
+import org.apache.hadoop.util.KMSUtil;
+import org.apache.hadoop.util.KMSUtilFaultInjector;
 import org.junit.BeforeClass;
 import org.junit.Test;
 import org.mockito.Mockito;
@@ -56,33 +57,68 @@ public class TestLoadBalancingKMSClientProvider {
     SecurityUtil.setTokenServiceUseIp(false);
   }
 
-  @After
-  public void teardown() throws IOException {
-    KMSClientProvider.fallbackDefaultPortForTesting = false;
+  private void setKMSUtilFaultInjector() {
+    KMSUtilFaultInjector injector = new KMSUtilFaultInjector() {
+      @Override
+      public KeyProvider createKeyProviderForTests(
+          String value, Configuration conf) throws IOException {
+        return TestLoadBalancingKMSClientProvider
+            .createKeyProviderForTests(value, conf);
+      }
+    };
+    KMSUtilFaultInjector.set(injector);
+  }
+
+  public static KeyProvider createKeyProviderForTests(
+      String value, Configuration conf) throws IOException {
+    // The syntax for kms servers will be
+    // kms://http@localhost:port1/kms,kms://http@localhost:port2/kms
+    if (!value.contains(",")) {
+      return null;
+    }
+    String[] keyProviderUrisStr = value.split(",");
+    KMSClientProvider[] keyProviderArr =
+        new KMSClientProvider[keyProviderUrisStr.length];
+
+    int i = 0;
+    for (String keyProviderUri: keyProviderUrisStr) {
+      KMSClientProvider kmcp =
+          new KMSClientProvider(URI.create(keyProviderUri), conf, URI
+              .create(value));
+      keyProviderArr[i] = kmcp;
+      i++;
+    }
+    LoadBalancingKMSClientProvider lbkcp =
+        new LoadBalancingKMSClientProvider(keyProviderArr, conf);
+    return lbkcp;
   }
 
   @Test
   public void testCreation() throws Exception {
     Configuration conf = new Configuration();
-    KMSClientProvider.fallbackDefaultPortForTesting = true;
     KeyProvider kp = new KMSClientProvider.Factory().createProvider(new URI(
-        "kms://http@host1/kms/foo"), conf);
+        "kms://http@host1:9600/kms/foo"), conf);
     assertTrue(kp instanceof LoadBalancingKMSClientProvider);
     KMSClientProvider[] providers =
         ((LoadBalancingKMSClientProvider) kp).getProviders();
     assertEquals(1, providers.length);
-    assertEquals(Sets.newHashSet("http://host1/kms/foo/v1/"),
+    assertEquals(Sets.newHashSet("http://host1:9600/kms/foo/v1/"),
         Sets.newHashSet(providers[0].getKMSUrl()));
-
-    kp = new KMSClientProvider.Factory().createProvider(new URI(
-        "kms://http@host1;host2;host3/kms/foo"), conf);
+    setKMSUtilFaultInjector();
+    String uriStr = "kms://http@host1:9600/kms/foo," +
+        "kms://http@host2:9600/kms/foo," +
+        "kms://http@host3:9600/kms/foo";
+    conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_KEY_PROVIDER_PATH,
+        uriStr);
+    kp = KMSUtil.createKeyProvider(conf, CommonConfigurationKeysPublic
+        .HADOOP_SECURITY_KEY_PROVIDER_PATH);
     assertTrue(kp instanceof LoadBalancingKMSClientProvider);
     providers =
         ((LoadBalancingKMSClientProvider) kp).getProviders();
     assertEquals(3, providers.length);
-    assertEquals(Sets.newHashSet("http://host1/kms/foo/v1/",
-        "http://host2/kms/foo/v1/",
-        "http://host3/kms/foo/v1/"),
+    assertEquals(Sets.newHashSet("http://host1:9600/kms/foo/v1/",
+        "http://host2:9600/kms/foo/v1/",
+        "http://host3:9600/kms/foo/v1/"),
         Sets.newHashSet(providers[0].getKMSUrl(),
             providers[1].getKMSUrl(),
             providers[2].getKMSUrl()));
@@ -208,7 +244,7 @@ public class TestLoadBalancingKMSClientProvider {
 
   private class MyKMSClientProvider extends KMSClientProvider {
     public MyKMSClientProvider(URI uri, Configuration conf) throws IOException {
-      super(uri, conf);
+      super(uri, conf, uri);
     }
 
     @Override
@@ -245,9 +281,8 @@ public class TestLoadBalancingKMSClientProvider {
   @Test
   public void testClassCastException() throws Exception {
     Configuration conf = new Configuration();
-    KMSClientProvider.fallbackDefaultPortForTesting = true;
     KMSClientProvider p1 = new MyKMSClientProvider(
-        new URI("kms://http@host1/kms/foo"), conf);
+        new URI("kms://http@host1:9600/kms/foo"), conf);
     LoadBalancingKMSClientProvider kp = new LoadBalancingKMSClientProvider(
         new KMSClientProvider[] {p1}, 0, conf);
     try {

+ 33 - 1
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/shell/find/TestFind.java

@@ -19,18 +19,19 @@ package org.apache.hadoop.fs.shell.find;
 
 import static org.junit.Assert.*;
 import static org.mockito.Mockito.*;
-import static org.mockito.Matchers.*;
 
 import java.io.IOException;
 import java.io.PrintStream;
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.LinkedList;
+import java.util.NoSuchElementException;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.fs.shell.PathData;
 import org.apache.hadoop.fs.shell.find.BaseExpression;
 import org.apache.hadoop.fs.shell.find.Expression;
@@ -42,6 +43,9 @@ import org.junit.Rule;
 import org.junit.rules.Timeout;
 import org.junit.Test;
 import org.mockito.InOrder;
+import org.mockito.Mockito;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
 
 public class TestFind {
 
@@ -861,6 +865,34 @@ public class TestFind {
     when(mockFs.listStatus(eq(item5c.path))).thenReturn(
         new FileStatus[] { item5ca.stat });
 
+    when(mockFs.listStatusIterator(Mockito.any(Path.class)))
+        .thenAnswer(new Answer<RemoteIterator<FileStatus>>() {
+
+          @Override
+          public RemoteIterator<FileStatus> answer(InvocationOnMock invocation)
+              throws Throwable {
+            final Path p = (Path) invocation.getArguments()[0];
+            final FileStatus[] stats = mockFs.listStatus(p);
+
+            return new RemoteIterator<FileStatus>() {
+              private int i = 0;
+
+              @Override
+              public boolean hasNext() throws IOException {
+                return i < stats.length;
+              }
+
+              @Override
+              public FileStatus next() throws IOException {
+                if (!hasNext()) {
+                  throw new NoSuchElementException("No more entry in " + p);
+                }
+                return stats[i++];
+              }
+            };
+          }
+        });
+
     when(item1.stat.isSymlink()).thenReturn(false);
     when(item1a.stat.isSymlink()).thenReturn(false);
     when(item1aa.stat.isSymlink()).thenReturn(false);

+ 242 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestCrcComposer.java

@@ -0,0 +1,242 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.util;
+
+import java.io.ByteArrayInputStream;
+import java.io.DataInputStream;
+import java.io.IOException;
+import java.util.Random;
+
+import org.apache.hadoop.test.LambdaTestUtils;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.Timeout;
+
+import static org.junit.Assert.*;
+
+/**
+ * Unittests for CrcComposer.
+ */
+public class TestCrcComposer {
+  @Rule
+  public Timeout globalTimeout = new Timeout(10000);
+
+  private Random rand = new Random(1234);
+
+  private DataChecksum.Type type = DataChecksum.Type.CRC32C;
+  private DataChecksum checksum = DataChecksum.newDataChecksum(
+      type, Integer.MAX_VALUE);
+  private int dataSize = 75;
+  private byte[] data = new byte[dataSize];
+  private int chunkSize = 10;
+  private int cellSize = 20;
+
+  private int fullCrc;
+  private int[] crcsByChunk;
+  private int[] crcsByCell;
+
+  private byte[] crcBytesByChunk;
+  private byte[] crcBytesByCell;
+
+  @Before
+  public void setup() throws IOException {
+    rand.nextBytes(data);
+    fullCrc = getRangeChecksum(data, 0, dataSize);
+
+    // 7 chunks of size chunkSize, 1 chunk of size (dataSize % chunkSize).
+    crcsByChunk = new int[8];
+    for (int i = 0; i < 7; ++i) {
+      crcsByChunk[i] = getRangeChecksum(data, i * chunkSize, chunkSize);
+    }
+    crcsByChunk[7] = getRangeChecksum(
+        data, (crcsByChunk.length - 1) * chunkSize, dataSize % chunkSize);
+
+    // 3 cells of size cellSize, 1 cell of size (dataSize % cellSize).
+    crcsByCell = new int[4];
+    for (int i = 0; i < 3; ++i) {
+      crcsByCell[i] = getRangeChecksum(data, i * cellSize, cellSize);
+    }
+    crcsByCell[3] = getRangeChecksum(
+        data, (crcsByCell.length - 1) * cellSize, dataSize % cellSize);
+
+    crcBytesByChunk = intArrayToByteArray(crcsByChunk);
+    crcBytesByCell = intArrayToByteArray(crcsByCell);
+  }
+
+  private int getRangeChecksum(byte[] buf, int offset, int length) {
+    checksum.reset();
+    checksum.update(buf, offset, length);
+    return (int) checksum.getValue();
+  }
+
+  private byte[] intArrayToByteArray(int[] values) throws IOException {
+    byte[] bytes = new byte[values.length * 4];
+    for (int i = 0; i < values.length; ++i) {
+      CrcUtil.writeInt(bytes, i * 4, values[i]);
+    }
+    return bytes;
+  }
+
+  @Test
+  public void testUnstripedIncorrectChunkSize() throws IOException {
+    CrcComposer digester = CrcComposer.newCrcComposer(type, chunkSize);
+
+    // If we incorrectly specify that all CRCs ingested correspond to chunkSize
+    // when the last CRC in the array actually corresponds to
+    // dataSize % chunkSize then we expect the resulting CRC to not be equal to
+    // the fullCrc.
+    digester.update(crcBytesByChunk, 0, crcBytesByChunk.length, chunkSize);
+    byte[] digest = digester.digest();
+    assertEquals(4, digest.length);
+    int calculatedCrc = CrcUtil.readInt(digest, 0);
+    assertNotEquals(fullCrc, calculatedCrc);
+  }
+
+  @Test
+  public void testUnstripedByteArray() throws IOException {
+    CrcComposer digester = CrcComposer.newCrcComposer(type, chunkSize);
+    digester.update(crcBytesByChunk, 0, crcBytesByChunk.length - 4, chunkSize);
+    digester.update(
+        crcBytesByChunk, crcBytesByChunk.length - 4, 4, dataSize % chunkSize);
+
+    byte[] digest = digester.digest();
+    assertEquals(4, digest.length);
+    int calculatedCrc = CrcUtil.readInt(digest, 0);
+    assertEquals(fullCrc, calculatedCrc);
+  }
+
+  @Test
+  public void testUnstripedDataInputStream() throws IOException {
+    CrcComposer digester = CrcComposer.newCrcComposer(type, chunkSize);
+    DataInputStream input =
+        new DataInputStream(new ByteArrayInputStream(crcBytesByChunk));
+    digester.update(input, crcsByChunk.length - 1, chunkSize);
+    digester.update(input, 1, dataSize % chunkSize);
+
+    byte[] digest = digester.digest();
+    assertEquals(4, digest.length);
+    int calculatedCrc = CrcUtil.readInt(digest, 0);
+    assertEquals(fullCrc, calculatedCrc);
+  }
+
+  @Test
+  public void testUnstripedSingleCrcs() throws IOException {
+    CrcComposer digester = CrcComposer.newCrcComposer(type, chunkSize);
+    for (int i = 0; i < crcsByChunk.length - 1; ++i) {
+      digester.update(crcsByChunk[i], chunkSize);
+    }
+    digester.update(crcsByChunk[crcsByChunk.length - 1], dataSize % chunkSize);
+
+    byte[] digest = digester.digest();
+    assertEquals(4, digest.length);
+    int calculatedCrc = CrcUtil.readInt(digest, 0);
+    assertEquals(fullCrc, calculatedCrc);
+  }
+
+  @Test
+  public void testStripedByteArray() throws IOException {
+    CrcComposer digester =
+        CrcComposer.newStripedCrcComposer(type, chunkSize, cellSize);
+    digester.update(crcBytesByChunk, 0, crcBytesByChunk.length - 4, chunkSize);
+    digester.update(
+        crcBytesByChunk, crcBytesByChunk.length - 4, 4, dataSize % chunkSize);
+
+    byte[] digest = digester.digest();
+    assertArrayEquals(crcBytesByCell, digest);
+  }
+
+  @Test
+  public void testStripedDataInputStream() throws IOException {
+    CrcComposer digester =
+        CrcComposer.newStripedCrcComposer(type, chunkSize, cellSize);
+    DataInputStream input =
+        new DataInputStream(new ByteArrayInputStream(crcBytesByChunk));
+    digester.update(input, crcsByChunk.length - 1, chunkSize);
+    digester.update(input, 1, dataSize % chunkSize);
+
+    byte[] digest = digester.digest();
+    assertArrayEquals(crcBytesByCell, digest);
+  }
+
+  @Test
+  public void testStripedSingleCrcs() throws IOException {
+    CrcComposer digester =
+        CrcComposer.newStripedCrcComposer(type, chunkSize, cellSize);
+    for (int i = 0; i < crcsByChunk.length - 1; ++i) {
+      digester.update(crcsByChunk[i], chunkSize);
+    }
+    digester.update(crcsByChunk[crcsByChunk.length - 1], dataSize % chunkSize);
+
+    byte[] digest = digester.digest();
+    assertArrayEquals(crcBytesByCell, digest);
+  }
+
+  @Test
+  public void testMultiStageMixed() throws IOException {
+    CrcComposer digester =
+        CrcComposer.newStripedCrcComposer(type, chunkSize, cellSize);
+
+    // First combine chunks into cells.
+    DataInputStream input =
+        new DataInputStream(new ByteArrayInputStream(crcBytesByChunk));
+    digester.update(input, crcsByChunk.length - 1, chunkSize);
+    digester.update(input, 1, dataSize % chunkSize);
+    byte[] digest = digester.digest();
+
+    // Second, individually combine cells into full crc.
+    digester =
+        CrcComposer.newCrcComposer(type, cellSize);
+    for (int i = 0; i < digest.length - 4; i += 4) {
+      int cellCrc = CrcUtil.readInt(digest, i);
+      digester.update(cellCrc, cellSize);
+    }
+    digester.update(digest, digest.length - 4, 4, dataSize % cellSize);
+    digest = digester.digest();
+    assertEquals(4, digest.length);
+    int calculatedCrc = CrcUtil.readInt(digest, 0);
+    assertEquals(fullCrc, calculatedCrc);
+  }
+
+  @Test
+  public void testUpdateMismatchesStripe() throws Exception {
+    CrcComposer digester =
+        CrcComposer.newStripedCrcComposer(type, chunkSize, cellSize);
+
+    digester.update(crcsByChunk[0], chunkSize);
+
+    // Going from chunkSize to chunkSize + cellSize will cross a cellSize
+    // boundary in a single CRC, which is not allowed, since we'd lack a
+    // CRC corresponding to the actual cellSize boundary.
+    LambdaTestUtils.intercept(
+        IOException.class,
+        "stripe",
+        () -> digester.update(crcsByChunk[1], cellSize));
+  }
+
+  @Test
+  public void testUpdateByteArrayLengthUnalignedWithCrcSize()
+      throws Exception {
+    CrcComposer digester = CrcComposer.newCrcComposer(type, chunkSize);
+
+    LambdaTestUtils.intercept(
+        IOException.class,
+        "length",
+        () -> digester.update(crcBytesByChunk, 0, 6, chunkSize));
+  }
+}

+ 232 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestCrcUtil.java

@@ -0,0 +1,232 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.util;
+
+import java.io.IOException;
+import java.util.Random;
+
+import org.apache.hadoop.test.LambdaTestUtils;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.Timeout;
+
+import static org.junit.Assert.*;
+
+/**
+ * Unittests for CrcUtil.
+ */
+public class TestCrcUtil {
+  @Rule
+  public Timeout globalTimeout = new Timeout(10000);
+
+  private Random rand = new Random(1234);
+
+  @Test
+  public void testComposeCrc32() throws IOException {
+    byte[] data = new byte[64 * 1024];
+    rand.nextBytes(data);
+    doTestComposeCrc(data, DataChecksum.Type.CRC32, 512, false);
+    doTestComposeCrc(data, DataChecksum.Type.CRC32, 511, false);
+    doTestComposeCrc(data, DataChecksum.Type.CRC32, 32 * 1024, false);
+    doTestComposeCrc(data, DataChecksum.Type.CRC32, 32 * 1024 - 1, false);
+  }
+
+  @Test
+  public void testComposeCrc32c() throws IOException {
+    byte[] data = new byte[64 * 1024];
+    rand.nextBytes(data);
+    doTestComposeCrc(data, DataChecksum.Type.CRC32C, 512, false);
+    doTestComposeCrc(data, DataChecksum.Type.CRC32C, 511, false);
+    doTestComposeCrc(data, DataChecksum.Type.CRC32C, 32 * 1024, false);
+    doTestComposeCrc(data, DataChecksum.Type.CRC32C, 32 * 1024 - 1, false);
+  }
+
+  @Test
+  public void testComposeCrc32WithMonomial() throws IOException {
+    byte[] data = new byte[64 * 1024];
+    rand.nextBytes(data);
+    doTestComposeCrc(data, DataChecksum.Type.CRC32, 512, true);
+    doTestComposeCrc(data, DataChecksum.Type.CRC32, 511, true);
+    doTestComposeCrc(data, DataChecksum.Type.CRC32, 32 * 1024, true);
+    doTestComposeCrc(data, DataChecksum.Type.CRC32, 32 * 1024 - 1, true);
+  }
+
+  @Test
+  public void testComposeCrc32cWithMonomial() throws IOException {
+    byte[] data = new byte[64 * 1024];
+    rand.nextBytes(data);
+    doTestComposeCrc(data, DataChecksum.Type.CRC32C, 512, true);
+    doTestComposeCrc(data, DataChecksum.Type.CRC32C, 511, true);
+    doTestComposeCrc(data, DataChecksum.Type.CRC32C, 32 * 1024, true);
+    doTestComposeCrc(data, DataChecksum.Type.CRC32C, 32 * 1024 - 1, true);
+  }
+
+  @Test
+  public void testComposeCrc32ZeroLength() throws IOException {
+    doTestComposeCrcZerolength(DataChecksum.Type.CRC32);
+  }
+
+  @Test
+  public void testComposeCrc32CZeroLength() throws IOException {
+    doTestComposeCrcZerolength(DataChecksum.Type.CRC32C);
+  }
+
+  /**
+   * Helper method to compare a DataChecksum-computed end-to-end CRC against
+   * a piecewise-computed CRC that uses CrcUtil.compose on "chunk CRCs"
+   * corresponding to ever {@code chunkSize} bytes.
+   */
+  private static void doTestComposeCrc(
+      byte[] data, DataChecksum.Type type, int chunkSize, boolean useMonomial)
+      throws IOException {
+    int crcPolynomial = DataChecksum.getCrcPolynomialForType(type);
+
+    // Get full end-to-end CRC in a single shot first.
+    DataChecksum checksum = DataChecksum.newDataChecksum(
+        type, Integer.MAX_VALUE);
+    checksum.update(data, 0, data.length);
+    int fullCrc = (int) checksum.getValue();
+
+    // Now compute CRCs of each chunk individually first, and compose them in a
+    // second pass to compare to the end-to-end CRC.
+    int compositeCrc = 0;
+    int crcMonomial =
+        useMonomial ? CrcUtil.getMonomial(chunkSize, crcPolynomial) : 0;
+    for (int offset = 0;
+        offset + chunkSize <= data.length;
+        offset += chunkSize) {
+      checksum.reset();
+      checksum.update(data, offset, chunkSize);
+      int partialCrc = (int) checksum.getValue();
+      if (useMonomial) {
+        compositeCrc = CrcUtil.composeWithMonomial(
+            compositeCrc, partialCrc, crcMonomial, crcPolynomial);
+      } else {
+        compositeCrc = CrcUtil.compose(
+            compositeCrc, partialCrc, chunkSize, crcPolynomial);
+      }
+    }
+
+    // There may be a final partial chunk smaller than chunkSize.
+    int partialChunkSize = data.length % chunkSize;
+    if (partialChunkSize > 0) {
+      checksum.reset();
+      checksum.update(data, data.length - partialChunkSize, partialChunkSize);
+      int partialCrc = (int) checksum.getValue();
+      compositeCrc = CrcUtil.compose(
+          compositeCrc, partialCrc, partialChunkSize, crcPolynomial);
+    }
+    assertEquals(
+        String.format(
+            "Using CRC type '%s' with crcPolynomial '0x%08x' and chunkSize '%d'"
+            + ", expected '0x%08x', got '0x%08x'",
+            type, crcPolynomial, chunkSize, fullCrc, compositeCrc),
+        fullCrc,
+        compositeCrc);
+  }
+
+  /**
+   * Helper method for testing the behavior of composing a CRC with a
+   * zero-length second CRC.
+   */
+  private static void doTestComposeCrcZerolength(DataChecksum.Type type)
+      throws IOException {
+    // Without loss of generality, we can pick any integer as our fake crcA
+    // even if we don't happen to know the preimage.
+    int crcA = 0xCAFEBEEF;
+    int crcPolynomial = DataChecksum.getCrcPolynomialForType(type);
+    DataChecksum checksum = DataChecksum.newDataChecksum(
+        type, Integer.MAX_VALUE);
+    int crcB = (int) checksum.getValue();
+    assertEquals(crcA, CrcUtil.compose(crcA, crcB, 0, crcPolynomial));
+
+    int monomial = CrcUtil.getMonomial(0, crcPolynomial);
+    assertEquals(
+        crcA, CrcUtil.composeWithMonomial(crcA, crcB, monomial, crcPolynomial));
+  }
+
+  @Test
+  public void testIntSerialization() throws IOException {
+    byte[] bytes = CrcUtil.intToBytes(0xCAFEBEEF);
+    assertEquals(0xCAFEBEEF, CrcUtil.readInt(bytes, 0));
+
+    bytes = new byte[8];
+    CrcUtil.writeInt(bytes, 0, 0xCAFEBEEF);
+    assertEquals(0xCAFEBEEF, CrcUtil.readInt(bytes, 0));
+    CrcUtil.writeInt(bytes, 4, 0xABCDABCD);
+    assertEquals(0xABCDABCD, CrcUtil.readInt(bytes, 4));
+
+    // Assert big-endian format for general Java consistency.
+    assertEquals(0xBEEFABCD, CrcUtil.readInt(bytes, 2));
+  }
+
+  @Test
+  public void testToSingleCrcStringBadLength()
+      throws Exception {
+    LambdaTestUtils.intercept(
+        IOException.class,
+        "length",
+        () -> CrcUtil.toSingleCrcString(new byte[8]));
+  }
+
+  @Test
+  public void testToSingleCrcString() throws IOException {
+    byte[] buf = CrcUtil.intToBytes(0xcafebeef);
+    assertEquals(
+        "0xcafebeef", CrcUtil.toSingleCrcString(buf));
+  }
+
+  @Test
+  public void testToMultiCrcStringBadLength()
+      throws Exception {
+    LambdaTestUtils.intercept(
+        IOException.class,
+        "length",
+        () -> CrcUtil.toMultiCrcString(new byte[6]));
+  }
+
+  @Test
+  public void testToMultiCrcStringMultipleElements()
+      throws IOException {
+    byte[] buf = new byte[12];
+    CrcUtil.writeInt(buf, 0, 0xcafebeef);
+    CrcUtil.writeInt(buf, 4, 0xababcccc);
+    CrcUtil.writeInt(buf, 8, 0xddddefef);
+    assertEquals(
+        "[0xcafebeef, 0xababcccc, 0xddddefef]",
+        CrcUtil.toMultiCrcString(buf));
+  }
+
+  @Test
+  public void testToMultiCrcStringSingleElement()
+      throws IOException {
+    byte[] buf = new byte[4];
+    CrcUtil.writeInt(buf, 0, 0xcafebeef);
+    assertEquals(
+        "[0xcafebeef]",
+        CrcUtil.toMultiCrcString(buf));
+  }
+
+  @Test
+  public void testToMultiCrcStringNoElements()
+      throws IOException {
+    assertEquals(
+        "[]",
+        CrcUtil.toMultiCrcString(new byte[0]));
+  }
+}

+ 65 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestKMSUtil.java

@@ -0,0 +1,65 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.util;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.crypto.key.KeyProvider;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.Timeout;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Test {@link KMSUtil}.
+ */
+public class TestKMSUtil {
+
+  public static final Logger LOG = LoggerFactory.getLogger(TestKMSUtil.class);
+
+  @Rule
+  public Timeout globalTimeout = new Timeout(90000);
+
+  @Test
+  public void testCreateKeyProviderFromTokenService() throws Exception {
+    final Configuration conf = new Configuration();
+    KeyProvider kp = KMSUtil.createKeyProviderFromTokenService(conf,
+        "kms://https@localhost:9600/kms");
+    assertNotNull(kp);
+    kp.close();
+
+    kp = KMSUtil.createKeyProviderFromTokenService(conf,
+        "kms://https@localhost:9600/kms,kms://localhost1:9600/kms");
+    assertNotNull(kp);
+    kp.close();
+
+    String invalidService = "whatever:9600";
+    try {
+      KMSUtil.createKeyProviderFromTokenService(conf, invalidService);
+    } catch (Exception ex) {
+      LOG.info("Expected exception:", ex);
+      assertTrue(ex instanceof IllegalArgumentException);
+      GenericTestUtils.assertExceptionContains(
+          "Invalid token service " + invalidService, ex);
+    }
+  }
+}

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

@@ -1,3 +1,4 @@
+
 /**
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
@@ -31,26 +32,35 @@ import org.apache.hadoop.crypto.key.KeyProviderCryptoExtension.EncryptedKeyVersi
 import org.apache.hadoop.crypto.key.KeyProviderDelegationTokenExtension;
 import org.apache.hadoop.crypto.key.kms.KMSClientProvider;
 import org.apache.hadoop.crypto.key.kms.KMSDelegationToken;
+import org.apache.hadoop.crypto.key.kms.KMSTokenRenewer;
 import org.apache.hadoop.crypto.key.kms.LoadBalancingKMSClientProvider;
+import org.apache.hadoop.crypto.key.kms.TestLoadBalancingKMSClientProvider;
 import org.apache.hadoop.crypto.key.kms.ValueQueue;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.MultipleIOException;
+import org.apache.hadoop.io.Text;
 import org.apache.hadoop.minikdc.MiniKdc;
 import org.apache.hadoop.security.Credentials;
-import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authorize.AuthorizationException;
 import org.apache.hadoop.security.ssl.KeyStoreTestUtil;
 import org.apache.hadoop.security.ssl.SSLFactory;
 import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
+import org.apache.hadoop.security.token.delegation.web.DelegationTokenAuthenticationHandler;
+import org.apache.hadoop.security.token.delegation.web.DelegationTokenAuthenticator;
 import org.apache.hadoop.security.token.delegation.web.DelegationTokenIdentifier;
 import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.hadoop.util.KMSUtil;
+import org.apache.hadoop.util.KMSUtilFaultInjector;
 import org.apache.hadoop.util.Time;
 import org.apache.http.client.utils.URIBuilder;
 import org.junit.After;
+import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.Before;
+import org.junit.BeforeClass;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.Timeout;
@@ -71,7 +81,6 @@ import java.io.IOException;
 import java.io.InputStream;
 import java.io.Writer;
 import java.net.InetAddress;
-import java.net.InetSocketAddress;
 import java.net.ServerSocket;
 import java.net.SocketTimeoutException;
 import java.net.URI;
@@ -96,6 +105,10 @@ import java.util.concurrent.LinkedBlockingQueue;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.KMS_CLIENT_COPY_LEGACY_TOKEN_KEY;
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_KEY_PROVIDER_PATH;
+import static org.apache.hadoop.crypto.key.kms.KMSDelegationToken.TOKEN_KIND;
+import static org.apache.hadoop.crypto.key.kms.KMSDelegationToken.TOKEN_LEGACY_KIND;
 import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
@@ -113,6 +126,20 @@ public class TestKMS {
 
   private SSLFactory sslFactory;
 
+  private final KMSUtilFaultInjector oldInjector =
+      KMSUtilFaultInjector.get();
+
+  // Injector to create providers with different ports. Can only happen in tests
+  private final KMSUtilFaultInjector testInjector =
+      new KMSUtilFaultInjector() {
+        @Override
+        public KeyProvider createKeyProviderForTests(String value,
+            Configuration conf) throws IOException {
+          return TestLoadBalancingKMSClientProvider
+              .createKeyProviderForTests(value, conf);
+        }
+      };
+
   // Keep track of all key providers created during a test case, so they can be
   // closed at test tearDown.
   private List<KeyProvider> providersCreated = new LinkedList<>();
@@ -122,7 +149,12 @@ public class TestKMS {
 
   @Before
   public void setUp() throws Exception {
-    setUpMiniKdc();
+    GenericTestUtils.setLogLevel(KMSClientProvider.LOG, Level.TRACE);
+    GenericTestUtils
+        .setLogLevel(DelegationTokenAuthenticationHandler.LOG, Level.TRACE);
+    GenericTestUtils
+        .setLogLevel(DelegationTokenAuthenticator.LOG, Level.TRACE);
+    GenericTestUtils.setLogLevel(KMSUtil.LOG, Level.TRACE);
     // resetting kerberos security
     Configuration conf = new Configuration();
     UserGroupInformation.setConfiguration(conf);
@@ -141,24 +173,78 @@ public class TestKMS {
   }
 
   public static abstract class KMSCallable<T> implements Callable<T> {
-    private URL kmsUrl;
+    private List<URL> kmsUrl;
 
     protected URL getKMSUrl() {
-      return kmsUrl;
+      return kmsUrl.get(0);
+    }
+
+    protected URL[] getKMSHAUrl() {
+      URL[] urls = new URL[kmsUrl.size()];
+      return kmsUrl.toArray(urls);
+    }
+
+    protected void addKMSUrl(URL url) {
+      if (kmsUrl == null) {
+        kmsUrl = new ArrayList<URL>();
+      }
+      kmsUrl.add(url);
+    }
+
+    /*
+     * The format of the returned value will be
+     * kms://http:kms1.example1.com:port1,kms://http:kms2.example2.com:port2
+     */
+    protected String generateLoadBalancingKeyProviderUriString() {
+      if (kmsUrl == null || kmsUrl.size() == 0) {
+        return null;
+      }
+      StringBuffer sb = new StringBuffer();
+
+      for (int i = 0; i < kmsUrl.size(); i++) {
+        sb.append(KMSClientProvider.SCHEME_NAME + "://" +
+            kmsUrl.get(0).getProtocol() + "@");
+        URL url = kmsUrl.get(i);
+        sb.append(url.getAuthority());
+        if (url.getPath() != null) {
+          sb.append(url.getPath());
+        }
+        if (i < kmsUrl.size() - 1) {
+          sb.append(",");
+        }
+      }
+      return sb.toString();
     }
   }
 
   protected KeyProvider createProvider(URI uri, Configuration conf)
       throws IOException {
     final KeyProvider ret = new LoadBalancingKMSClientProvider(
-        new KMSClientProvider[] {new KMSClientProvider(uri, conf)}, conf);
+        new KMSClientProvider[] {new KMSClientProvider(uri, conf, uri)}, conf);
     providersCreated.add(ret);
     return ret;
   }
 
+  /**
+   * create a LoadBalancingKMSClientProvider from an array of URIs.
+   * @param uris an array of KMS URIs
+   * @param conf configuration object
+   * @return a LoadBalancingKMSClientProvider object
+   * @throws IOException
+   */
+  protected LoadBalancingKMSClientProvider createHAProvider(URI[] uris,
+      Configuration conf, String originalUri) throws IOException {
+    KMSClientProvider[] providers = new KMSClientProvider[uris.length];
+    for (int i = 0; i < providers.length; i++) {
+      providers[i] =
+          new KMSClientProvider(uris[i], conf, URI.create(originalUri));
+    }
+    return new LoadBalancingKMSClientProvider(providers, conf);
+  }
+
   private KMSClientProvider createKMSClientProvider(URI uri, Configuration conf)
       throws IOException {
-    final KMSClientProvider ret = new KMSClientProvider(uri, conf);
+    final KMSClientProvider ret = new KMSClientProvider(uri, conf, uri);
     providersCreated.add(ret);
     return ret;
   }
@@ -170,22 +256,33 @@ public class TestKMS {
 
   protected <T> T runServer(int port, String keystore, String password, File confDir,
       KMSCallable<T> callable) throws Exception {
+    return runServer(new int[] {port}, keystore, password, confDir, callable);
+  }
+
+  protected <T> T runServer(int[] ports, String keystore, String password,
+      File confDir, KMSCallable<T> callable) throws Exception {
     MiniKMS.Builder miniKMSBuilder = new MiniKMS.Builder().setKmsConfDir(confDir)
         .setLog4jConfFile("log4j.properties");
     if (keystore != null) {
       miniKMSBuilder.setSslConf(new File(keystore), password);
     }
-    if (port > 0) {
-      miniKMSBuilder.setPort(port);
+    final List<MiniKMS> kmsList = new ArrayList<>();
+    for (int i=0; i< ports.length; i++) {
+      if (ports[i] > 0) {
+        miniKMSBuilder.setPort(ports[i]);
+      }
+      MiniKMS miniKMS = miniKMSBuilder.build();
+      kmsList.add(miniKMS);
+      miniKMS.start();
+      LOG.info("Test KMS running at: " + miniKMS.getKMSUrl());
+      callable.addKMSUrl(miniKMS.getKMSUrl());
     }
-    MiniKMS miniKMS = miniKMSBuilder.build();
-    miniKMS.start();
     try {
-      System.out.println("Test KMS running at: " + miniKMS.getKMSUrl());
-      callable.kmsUrl = miniKMS.getKMSUrl();
       return callable.call();
     } finally {
-      miniKMS.stop();
+      for (MiniKMS miniKMS: kmsList) {
+        miniKMS.stop();
+      }
     }
   }
 
@@ -240,6 +337,13 @@ public class TestKMS {
     return new URI("kms://" + str);
   }
 
+  public static URI[] createKMSHAUri(URL[] kmsUrls) throws Exception {
+    URI[] uris = new URI[kmsUrls.length];
+    for (int i = 0; i < kmsUrls.length; i++) {
+      uris[i] = createKMSUri(kmsUrls[i]);
+    }
+    return uris;
+  }
 
   private static class KerberosConfiguration
       extends javax.security.auth.login.Configuration {
@@ -315,19 +419,17 @@ public class TestKMS {
         principals.toArray(new String[principals.size()]));
   }
 
-  private void setUpMiniKdc() throws Exception {
+  @BeforeClass
+  public static void setUpMiniKdc() throws Exception {
     Properties kdcConf = MiniKdc.createConf();
     setUpMiniKdc(kdcConf);
   }
 
   @After
   public void tearDown() throws Exception {
-    if (kdc != null) {
-      kdc.stop();
-      kdc = null;
-    }
     UserGroupInformation.setShouldRenewImmediatelyForTests(false);
     UserGroupInformation.reset();
+    KMSUtilFaultInjector.set(oldInjector);
     if (!providersCreated.isEmpty()) {
       final MultipleIOException.Builder b = new MultipleIOException.Builder();
       for (KeyProvider kp : providersCreated) {
@@ -345,6 +447,14 @@ public class TestKMS {
     }
   }
 
+  @AfterClass
+  public static void shutdownMiniKdc() {
+    if (kdc != null) {
+      kdc.stop();
+      kdc = null;
+    }
+  }
+
   private <T> T doAs(String user, final PrivilegedExceptionAction<T> action)
       throws Exception {
     UserGroupInformation.loginUserFromKeytab(user, keytab.getAbsolutePath());
@@ -501,8 +611,10 @@ public class TestKMS {
                 Token<?>[] tokens =
                     ((KeyProviderDelegationTokenExtension.DelegationTokenExtension)kp)
                     .addDelegationTokens("myuser", new Credentials());
-                Assert.assertEquals(1, tokens.length);
-                Assert.assertEquals("kms-dt", tokens[0].getKind().toString());
+                assertEquals(2, tokens.length);
+                assertEquals(KMSDelegationToken.TOKEN_KIND,
+                    tokens[0].getKind());
+                kp.close();
                 return null;
               }
             });
@@ -518,8 +630,9 @@ public class TestKMS {
           Token<?>[] tokens =
               ((KeyProviderDelegationTokenExtension.DelegationTokenExtension)kp)
               .addDelegationTokens("myuser", new Credentials());
-          Assert.assertEquals(1, tokens.length);
-          Assert.assertEquals("kms-dt", tokens[0].getKind().toString());
+          assertEquals(2, tokens.length);
+          assertEquals(KMSDelegationToken.TOKEN_KIND, tokens[0].getKind());
+          kp.close();
         }
         return null;
       }
@@ -2011,7 +2124,6 @@ public class TestKMS {
             return null;
           }
         });
-
         nonKerberosUgi.addCredentials(credentials);
 
         try {
@@ -2067,6 +2179,17 @@ public class TestKMS {
     testDelegationTokensOps(true, true);
   }
 
+  private Text getTokenService(KeyProvider provider) {
+    assertTrue("KeyProvider should be an instance of KMSClientProvider",
+        (provider instanceof LoadBalancingKMSClientProvider));
+    assertEquals("Num client providers should be 1", 1,
+        ((LoadBalancingKMSClientProvider)provider).getProviders().length);
+    Text tokenService =
+        (((LoadBalancingKMSClientProvider)provider).getProviders()[0])
+        .getDelegationTokenService();
+    return tokenService;
+  }
+
   private void testDelegationTokensOps(final boolean ssl, final boolean kerb)
       throws Exception {
     final File confDir = getTestDir();
@@ -2098,11 +2221,16 @@ public class TestKMS {
         final URI uri = createKMSUri(getKMSUrl());
         clientConf.set(KeyProviderFactory.KEY_PROVIDER_PATH,
             createKMSUri(getKMSUrl()).toString());
+        clientConf.setBoolean(KMS_CLIENT_COPY_LEGACY_TOKEN_KEY, false);
 
         doAs("client", new PrivilegedExceptionAction<Void>() {
           @Override
           public Void run() throws Exception {
             KeyProvider kp = createProvider(uri, clientConf);
+            // Unset the conf value for key provider path just to be sure that
+            // the key provider created for renew and cancel token is from
+            // token service field.
+            clientConf.unset(HADOOP_SECURITY_KEY_PROVIDER_PATH);
             // test delegation token retrieval
             KeyProviderDelegationTokenExtension kpdte =
                 KeyProviderDelegationTokenExtension.
@@ -2110,13 +2238,10 @@ public class TestKMS {
             final Credentials credentials = new Credentials();
             final Token<?>[] tokens =
                 kpdte.addDelegationTokens("client1", credentials);
-            Assert.assertEquals(1, credentials.getAllTokens().size());
-            InetSocketAddress kmsAddr =
-                new InetSocketAddress(getKMSUrl().getHost(),
-                    getKMSUrl().getPort());
-            Assert.assertEquals(KMSDelegationToken.TOKEN_KIND,
-                credentials.getToken(SecurityUtil.buildTokenService(kmsAddr)).
-                    getKind());
+            Text tokenService = getTokenService(kp);
+            assertEquals(1, credentials.getAllTokens().size());
+            assertEquals(TOKEN_KIND,
+                credentials.getToken(tokenService).getKind());
 
             // Test non-renewer user cannot renew.
             for (Token<?> token : tokens) {
@@ -2243,12 +2368,11 @@ public class TestKMS {
         final URI uri = createKMSUri(getKMSUrl());
         clientConf.set(KeyProviderFactory.KEY_PROVIDER_PATH,
             createKMSUri(getKMSUrl()).toString());
+        clientConf.setBoolean(KMS_CLIENT_COPY_LEGACY_TOKEN_KEY, false);
         final KeyProvider kp = createProvider(uri, clientConf);
         final KeyProviderDelegationTokenExtension kpdte =
             KeyProviderDelegationTokenExtension.
                 createKeyProviderDelegationTokenExtension(kp);
-        final InetSocketAddress kmsAddr =
-            new InetSocketAddress(getKMSUrl().getHost(), getKMSUrl().getPort());
 
         // Job 1 (e.g. YARN log aggregation job), with user DT.
         final Collection<Token<?>> job1Token = new HashSet<>();
@@ -2258,16 +2382,17 @@ public class TestKMS {
             // Get a DT and use it.
             final Credentials credentials = new Credentials();
             kpdte.addDelegationTokens("client", credentials);
+            Text tokenService = getTokenService(kp);
             Assert.assertEquals(1, credentials.getAllTokens().size());
-            Assert.assertEquals(KMSDelegationToken.TOKEN_KIND, credentials.
-                getToken(SecurityUtil.buildTokenService(kmsAddr)).getKind());
+
             UserGroupInformation.getCurrentUser().addCredentials(credentials);
             LOG.info("Added kms dt to credentials: {}", UserGroupInformation.
                 getCurrentUser().getCredentials().getAllTokens());
-            Token<?> token =
+            final Token<?> token =
                 UserGroupInformation.getCurrentUser().getCredentials()
-                    .getToken(SecurityUtil.buildTokenService(kmsAddr));
-            Assert.assertNotNull(token);
+                    .getToken(tokenService);
+            assertNotNull(token);
+            assertEquals(TOKEN_KIND, token.getKind());
             job1Token.add(token);
 
             // Decode the token to get max time.
@@ -2302,17 +2427,16 @@ public class TestKMS {
             // Get a new DT, but don't use it yet.
             final Credentials newCreds = new Credentials();
             kpdte.addDelegationTokens("client", newCreds);
-            Assert.assertEquals(1, newCreds.getAllTokens().size());
-            Assert.assertEquals(KMSDelegationToken.TOKEN_KIND,
-                newCreds.getToken(SecurityUtil.buildTokenService(kmsAddr)).
-                    getKind());
+            assertEquals(1, newCreds.getAllTokens().size());
+            final Text tokenService = getTokenService(kp);
+            assertEquals(TOKEN_KIND,
+                newCreds.getToken(tokenService).getKind());
 
             // Using job 1's DT should fail.
             final Credentials oldCreds = new Credentials();
             for (Token<?> token : job1Token) {
-              if (token.getKind().equals(KMSDelegationToken.TOKEN_KIND)) {
-                oldCreds
-                    .addToken(SecurityUtil.buildTokenService(kmsAddr), token);
+              if (token.getKind().equals(TOKEN_KIND)) {
+                oldCreds.addToken(tokenService, token);
               }
             }
             UserGroupInformation.getCurrentUser().addCredentials(oldCreds);
@@ -2326,12 +2450,11 @@ public class TestKMS {
             }
 
             // Using the new DT should succeed.
-            Assert.assertEquals(1, newCreds.getAllTokens().size());
-            Assert.assertEquals(KMSDelegationToken.TOKEN_KIND,
-                newCreds.getToken(SecurityUtil.buildTokenService(kmsAddr)).
-                    getKind());
+            assertEquals(1, newCreds.getAllTokens().size());
+            assertEquals(TOKEN_KIND,
+                newCreds.getToken(tokenService).getKind());
             UserGroupInformation.getCurrentUser().addCredentials(newCreds);
-            LOG.info("Credetials now are: {}", UserGroupInformation
+            LOG.info("Credentials now are: {}", UserGroupInformation
                 .getCurrentUser().getCredentials().getAllTokens());
             kp.getKeys();
             return null;
@@ -2357,7 +2480,13 @@ public class TestKMS {
     doKMSWithZK(true, true);
   }
 
-  public void doKMSWithZK(boolean zkDTSM, boolean zkSigner) throws Exception {
+  private <T> T runServerWithZooKeeper(boolean zkDTSM, boolean zkSigner,
+      KMSCallable<T> callable) throws Exception {
+    return runServerWithZooKeeper(zkDTSM, zkSigner, callable, 1);
+  }
+
+  private <T> T runServerWithZooKeeper(boolean zkDTSM, boolean zkSigner,
+      KMSCallable<T> callable, int kmsSize) throws Exception {
     TestingServer zkServer = null;
     try {
       zkServer = new TestingServer();
@@ -2403,43 +2532,265 @@ public class TestKMS {
 
       writeConf(testDir, conf);
 
-      KMSCallable<KeyProvider> c =
-          new KMSCallable<KeyProvider>() {
-        @Override
-        public KeyProvider call() throws Exception {
-          final Configuration conf = new Configuration();
-          conf.setInt(KeyProvider.DEFAULT_BITLENGTH_NAME, 128);
-          final URI uri = createKMSUri(getKMSUrl());
-
-          final KeyProvider kp =
-              doAs("SET_KEY_MATERIAL",
-                  new PrivilegedExceptionAction<KeyProvider>() {
-                    @Override
-                    public KeyProvider run() throws Exception {
-                      KeyProvider kp = createProvider(uri, conf);
-                          kp.createKey("k1", new byte[16],
-                              new KeyProvider.Options(conf));
-                          kp.createKey("k2", new byte[16],
-                              new KeyProvider.Options(conf));
-                          kp.createKey("k3", new byte[16],
-                              new KeyProvider.Options(conf));
-                      return kp;
-                    }
-                  });
-          return kp;
-        }
-      };
-
-      runServer(null, null, testDir, c);
+      int[] ports = new int[kmsSize];
+      for (int i = 0; i < ports.length; i++) {
+        ports[i] = -1;
+      }
+      return runServer(ports, null, null, testDir, callable);
     } finally {
       if (zkServer != null) {
         zkServer.stop();
         zkServer.close();
       }
     }
+  }
+
+  public void doKMSWithZK(boolean zkDTSM, boolean zkSigner) throws Exception {
+    KMSCallable<KeyProvider> c =
+        new KMSCallable<KeyProvider>() {
+          @Override
+          public KeyProvider call() throws Exception {
+            final Configuration conf = new Configuration();
+            conf.setInt(KeyProvider.DEFAULT_BITLENGTH_NAME, 128);
+            final URI uri = createKMSUri(getKMSUrl());
+
+            final KeyProvider kp =
+                doAs("SET_KEY_MATERIAL",
+                    new PrivilegedExceptionAction<KeyProvider>() {
+                      @Override
+                      public KeyProvider run() throws Exception {
+                        KeyProvider kp = createProvider(uri, conf);
+                        kp.createKey("k1", new byte[16],
+                            new KeyProvider.Options(conf));
+                        kp.createKey("k2", new byte[16],
+                            new KeyProvider.Options(conf));
+                        kp.createKey("k3", new byte[16],
+                            new KeyProvider.Options(conf));
+                        return kp;
+                      }
+                    });
+            return kp;
+          }
+        };
+
+    runServerWithZooKeeper(zkDTSM, zkSigner, c);
+  }
+
+  @Test
+  public void doKMSHAZKWithDelegationTokenAccess() throws Exception {
+    KMSCallable<Void> c = new KMSCallable<Void>() {
+      @Override
+      public Void call() throws Exception {
+        final Configuration conf = new Configuration();
+        conf.setInt(KeyProvider.DEFAULT_BITLENGTH_NAME, 128);
+        final URI[] uris = createKMSHAUri(getKMSHAUrl());
+        final Credentials credentials = new Credentials();
+        final String lbUri = generateLoadBalancingKeyProviderUriString();
+        final LoadBalancingKMSClientProvider lbkp =
+            createHAProvider(uris, conf, lbUri);
+        conf.unset(HADOOP_SECURITY_KEY_PROVIDER_PATH);
+        // Login as a Kerberos user principal using keytab.
+        // Connect to KMS to create a delegation token and add it to credentials
+        final String keyName = "k0";
+        doAs("SET_KEY_MATERIAL",
+            new PrivilegedExceptionAction<Void>() {
+              @Override
+              public Void run() throws Exception {
+                KeyProviderDelegationTokenExtension kpdte =
+                    KeyProviderDelegationTokenExtension.
+                        createKeyProviderDelegationTokenExtension(lbkp);
+                kpdte.addDelegationTokens("SET_KEY_MATERIAL", credentials);
+                kpdte.createKey(keyName, new KeyProvider.Options(conf));
+                return null;
+              }
+            });
+
+        assertTokenIdentifierEquals(credentials);
+
+        final LoadBalancingKMSClientProvider lbkp1 =
+            createHAProvider(uris, conf, lbUri);
+        // verify both tokens can be used to authenticate
+        for (Token t : credentials.getAllTokens()) {
+          assertTokenAccess(lbkp1, keyName, t);
+        }
+        return null;
+      }
+    };
+    runServerWithZooKeeper(true, true, c, 2);
+  }
+
+  /**
+   * Assert that the passed in credentials have 2 tokens, of kind
+   * {@link KMSDelegationToken#TOKEN_KIND} and
+   * {@link KMSDelegationToken#TOKEN_LEGACY_KIND}. Assert that the 2 tokens have
+   * the same identifier.
+   */
+  private void assertTokenIdentifierEquals(Credentials credentials)
+      throws IOException {
+    // verify the 2 tokens have the same identifier
+    assertEquals(2, credentials.getAllTokens().size());
+    Token token = null;
+    Token legacyToken = null;
+    for (Token t : credentials.getAllTokens()) {
+      if (KMSDelegationToken.TOKEN_KIND.equals(t.getKind())) {
+        token = t;
+      } else if (KMSDelegationToken.TOKEN_LEGACY_KIND.equals(t.getKind())) {
+        legacyToken = t;
+      }
+    }
+    assertNotNull(token);
+    assertNotNull(legacyToken);
+    final DelegationTokenIdentifier tokenId =
+        (DelegationTokenIdentifier) token.decodeIdentifier();
+    final DelegationTokenIdentifier legacyTokenId =
+        (DelegationTokenIdentifier) legacyToken.decodeIdentifier();
+    assertEquals("KMS DT and legacy dt should have identical identifier",
+        tokenId, legacyTokenId);
+  }
 
+  /**
+   * Tests token access with each providers in the
+   * {@link LoadBalancingKMSClientProvider}. This is to make sure the 2 token
+   * kinds are compatible and can both be used to authenticate.
+   */
+  private void assertTokenAccess(final LoadBalancingKMSClientProvider lbkp,
+      final String keyName, final Token token) throws Exception {
+    UserGroupInformation tokenUgi =
+        UserGroupInformation.createUserForTesting("test", new String[] {});
+    // Verify the tokens can authenticate to any KMS
+    tokenUgi.addToken(token);
+    tokenUgi.doAs(new PrivilegedExceptionAction<Void>() {
+      @Override
+      public Void run() throws Exception {
+        // Create a kms client with one provider at a time. Must use one
+        // provider so that if it fails to authenticate, it does not fall
+        // back to the next KMS instance.
+        // It should succeed because its delegation token can access any
+        // KMS instances.
+        for (KMSClientProvider provider : lbkp.getProviders()) {
+          if (token.getKind().equals(TOKEN_LEGACY_KIND) && !token.getService()
+              .equals(provider.getDelegationTokenService())) {
+            // Historically known issue: Legacy token can only work with the
+            // key provider specified in the token's Service
+            continue;
+          }
+          LOG.info("Rolling key {} via provider {} with token {}.", keyName,
+              provider, token);
+          provider.rollNewVersion(keyName);
+        }
+        return null;
+      }
+    });
   }
 
+  @Test
+  public void testKMSHAZKDelegationTokenRenewCancel() throws Exception {
+    testKMSHAZKDelegationTokenRenewCancel(TOKEN_KIND);
+  }
+
+  @Test
+  public void testKMSHAZKDelegationTokenRenewCancelLegacy() throws Exception {
+    testKMSHAZKDelegationTokenRenewCancel(TOKEN_LEGACY_KIND);
+  }
+
+  private void testKMSHAZKDelegationTokenRenewCancel(final Text tokenKind)
+      throws Exception {
+    GenericTestUtils.setLogLevel(KMSTokenRenewer.LOG, Level.TRACE);
+    assertTrue(tokenKind == TOKEN_KIND || tokenKind == TOKEN_LEGACY_KIND);
+    KMSCallable<Void> c = new KMSCallable<Void>() {
+      @Override
+      public Void call() throws Exception {
+        final Configuration conf = new Configuration();
+        final URI[] uris = createKMSHAUri(getKMSHAUrl());
+        final Credentials credentials = new Credentials();
+        // Create a UGI without Kerberos auth. It will be authenticated with
+        // delegation token.
+        final UserGroupInformation nonKerberosUgi =
+            UserGroupInformation.getCurrentUser();
+        final String lbUri = generateLoadBalancingKeyProviderUriString();
+        final LoadBalancingKMSClientProvider lbkp =
+            createHAProvider(uris, conf, lbUri);
+        conf.unset(HADOOP_SECURITY_KEY_PROVIDER_PATH);
+        // Login as a Kerberos user principal using keytab.
+        // Connect to KMS to create a delegation token and add it to credentials
+        doAs("SET_KEY_MATERIAL",
+            new PrivilegedExceptionAction<Void>() {
+              @Override
+              public Void run() throws Exception {
+                KeyProviderDelegationTokenExtension kpdte =
+                    KeyProviderDelegationTokenExtension.
+                        createKeyProviderDelegationTokenExtension(lbkp);
+                kpdte.addDelegationTokens("SET_KEY_MATERIAL", credentials);
+                return null;
+              }
+            });
+
+        // Test token renewal and cancellation
+        final Collection<Token<? extends TokenIdentifier>> tokens =
+            credentials.getAllTokens();
+        doAs("SET_KEY_MATERIAL", new PrivilegedExceptionAction<Void>() {
+          @Override
+          public Void run() throws Exception {
+            Assert.assertEquals(2, tokens.size());
+            boolean tokenFound = false;
+            for (Token token : tokens) {
+              if (!tokenKind.equals(token.getKind())) {
+                continue;
+              } else {
+                tokenFound = true;
+              }
+              KMSUtilFaultInjector.set(testInjector);
+              setupConfForToken(token.getKind(), conf, lbUri);
+
+              LOG.info("Testing token: {}", token);
+              long tokenLife = token.renew(conf);
+              LOG.info("Renewed token {}, new lifetime:{}", token, tokenLife);
+              Thread.sleep(10);
+              long newTokenLife = token.renew(conf);
+              LOG.info("Renewed token {}, new lifetime:{}", token,
+                  newTokenLife);
+              assertTrue(newTokenLife > tokenLife);
+
+              boolean canceled = false;
+              // test delegation token cancellation
+              if (!canceled) {
+                token.cancel(conf);
+                LOG.info("Cancelled token {}", token);
+                canceled = true;
+              }
+              assertTrue("token should have been canceled", canceled);
+              try {
+                token.renew(conf);
+                fail("should not be able to renew a canceled token " + token);
+              } catch (Exception e) {
+                LOG.info("Expected exception when renewing token", e);
+              }
+            }
+            assertTrue("Should have found token kind " + tokenKind + " from "
+                + tokens, tokenFound);
+            return null;
+          }
+        });
+        return null;
+      }
+    };
+    runServerWithZooKeeper(true, true, c, 2);
+  }
+
+  /**
+   * Set or unset the key provider configuration based on token kind.
+   */
+  private void setupConfForToken(Text tokenKind, Configuration conf,
+      String lbUri) {
+    if (tokenKind.equals(TOKEN_KIND)) {
+      conf.unset(HADOOP_SECURITY_KEY_PROVIDER_PATH);
+    } else {
+      // conf is only required for legacy tokens to create provider,
+      // new tokens create provider by parsing its own Service field
+      assertEquals(TOKEN_LEGACY_KIND, tokenKind);
+      conf.set(HADOOP_SECURITY_KEY_PROVIDER_PATH, lbUri);
+    }
+  }
 
   @Test
   public void testProxyUserKerb() throws Exception {
@@ -2558,6 +2909,16 @@ public class TestKMS {
 
   @Test
   public void testTGTRenewal() throws Exception {
+    shutdownMiniKdc();
+    try {
+      testTgtRenewalInt();
+    } finally {
+      shutdownMiniKdc();
+      setUpMiniKdc();
+    }
+  }
+
+  private void testTgtRenewalInt() throws Exception {
     Properties kdcConf = MiniKdc.createConf();
     kdcConf.setProperty(MiniKdc.MAX_TICKET_LIFETIME, "3");
     kdcConf.setProperty(MiniKdc.MIN_TICKET_LIFETIME, "3");

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/fs/Hdfs.java

@@ -130,9 +130,9 @@ public class Hdfs extends AbstractFileSystem {
   }
 
   @Override
-  public FileChecksum getFileChecksum(Path f) 
+  public FileChecksum getFileChecksum(Path f)
       throws IOException, UnresolvedLinkException {
-    return dfs.getFileChecksum(getUriPath(f), Long.MAX_VALUE);
+    return dfs.getFileChecksumWithCombineMode(getUriPath(f), Long.MAX_VALUE);
   }
 
   @Override

+ 42 - 14
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java

@@ -66,6 +66,7 @@ import org.apache.hadoop.fs.CacheFlag;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.FileAlreadyExistsException;
+import org.apache.hadoop.fs.FileChecksum;
 import org.apache.hadoop.fs.FileEncryptionInfo;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
@@ -76,6 +77,7 @@ import org.apache.hadoop.fs.HdfsBlockLocation;
 import org.apache.hadoop.fs.InvalidPathException;
 import org.apache.hadoop.fs.MD5MD5CRC32FileChecksum;
 import org.apache.hadoop.fs.Options;
+import org.apache.hadoop.fs.Options.ChecksumCombineMode;
 import org.apache.hadoop.fs.Options.ChecksumOpt;
 import org.apache.hadoop.fs.ParentNotDirectoryException;
 import org.apache.hadoop.fs.Path;
@@ -1753,18 +1755,8 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
     return encryptionKey;
   }
 
-  /**
-   * Get the checksum of the whole file or a range of the file. Note that the
-   * range always starts from the beginning of the file. The file can be
-   * in replicated form, or striped mode. It can be used to checksum and compare
-   * two replicated files, or two striped files, but not applicable for two
-   * files of different block layout forms.
-   * @param src The file path
-   * @param length the length of the range, i.e., the range is [0, length]
-   * @return The checksum
-   * @see DistributedFileSystem#getFileChecksum(Path)
-   */
-  public MD5MD5CRC32FileChecksum getFileChecksum(String src, long length)
+  private FileChecksum getFileChecksumInternal(
+      String src, long length, ChecksumCombineMode combineMode)
       throws IOException {
     checkOpen();
     Preconditions.checkArgument(length >= 0);
@@ -1779,15 +1771,51 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
 
     maker = ecPolicy != null ?
         new FileChecksumHelper.StripedFileNonStripedChecksumComputer(src,
-            length, blockLocations, namenode, this, ecPolicy) :
+            length, blockLocations, namenode, this, ecPolicy, combineMode) :
         new FileChecksumHelper.ReplicatedFileChecksumComputer(src, length,
-            blockLocations, namenode, this);
+            blockLocations, namenode, this, combineMode);
 
     maker.compute();
 
     return maker.getFileChecksum();
   }
 
+  /**
+   * Get the checksum of the whole file or a range of the file. Note that the
+   * range always starts from the beginning of the file. The file can be
+   * in replicated form, or striped mode. Depending on the
+   * dfs.checksum.combine.mode, checksums may or may not be comparable between
+   * different block layout forms.
+   *
+   * @param src The file path
+   * @param length the length of the range, i.e., the range is [0, length]
+   * @return The checksum
+   * @see DistributedFileSystem#getFileChecksum(Path)
+   */
+  public FileChecksum getFileChecksumWithCombineMode(String src, long length)
+      throws IOException {
+    ChecksumCombineMode combineMode = getConf().getChecksumCombineMode();
+    return getFileChecksumInternal(src, length, combineMode);
+  }
+
+  /**
+   * Get the checksum of the whole file or a range of the file. Note that the
+   * range always starts from the beginning of the file. The file can be
+   * in replicated form, or striped mode. It can be used to checksum and compare
+   * two replicated files, or two striped files, but not applicable for two
+   * files of different block layout forms.
+   *
+   * @param src The file path
+   * @param length the length of the range, i.e., the range is [0, length]
+   * @return The checksum
+   * @see DistributedFileSystem#getFileChecksum(Path)
+   */
+  public MD5MD5CRC32FileChecksum getFileChecksum(String src, long length)
+      throws IOException {
+    return (MD5MD5CRC32FileChecksum) getFileChecksumInternal(
+        src, length, ChecksumCombineMode.MD5MD5CRC);
+  }
+
   protected LocatedBlocks getBlockLocations(String src,
                                             long length) throws IOException {
     //get block locations for the file range

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

@@ -1681,7 +1681,8 @@ public class DistributedFileSystem extends FileSystem
     return new FileSystemLinkResolver<FileChecksum>() {
       @Override
       public FileChecksum doCall(final Path p) throws IOException {
-        return dfs.getFileChecksum(getPathName(p), Long.MAX_VALUE);
+        return dfs.getFileChecksumWithCombineMode(
+            getPathName(p), Long.MAX_VALUE);
       }
 
       @Override
@@ -1701,7 +1702,7 @@ public class DistributedFileSystem extends FileSystem
     return new FileSystemLinkResolver<FileChecksum>() {
       @Override
       public FileChecksum doCall(final Path p) throws IOException {
-        return dfs.getFileChecksum(getPathName(p), length);
+        return dfs.getFileChecksumWithCombineMode(getPathName(p), length);
       }
 
       @Override

+ 248 - 117
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/FileChecksumHelper.java

@@ -17,9 +17,14 @@
  */
 package org.apache.hadoop.hdfs;
 
+import org.apache.hadoop.fs.CompositeCrcFileChecksum;
+import org.apache.hadoop.fs.FileChecksum;
 import org.apache.hadoop.fs.MD5MD5CRC32CastagnoliFileChecksum;
-import org.apache.hadoop.fs.MD5MD5CRC32FileChecksum;
 import org.apache.hadoop.fs.MD5MD5CRC32GzipFileChecksum;
+import org.apache.hadoop.fs.Options.ChecksumCombineMode;
+import org.apache.hadoop.fs.PathIOException;
+import org.apache.hadoop.hdfs.protocol.BlockChecksumOptions;
+import org.apache.hadoop.hdfs.protocol.BlockChecksumType;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
@@ -41,6 +46,8 @@ import org.apache.hadoop.hdfs.security.token.block.InvalidBlockTokenException;
 import org.apache.hadoop.io.DataOutputBuffer;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.MD5Hash;
+import org.apache.hadoop.util.CrcComposer;
+import org.apache.hadoop.util.CrcUtil;
 import org.apache.hadoop.util.DataChecksum;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -67,9 +74,11 @@ final class FileChecksumHelper {
     private final long length;
     private final DFSClient client;
     private final ClientProtocol namenode;
-    private final DataOutputBuffer md5out = new DataOutputBuffer();
+    private final ChecksumCombineMode combineMode;
+    private final BlockChecksumType blockChecksumType;
+    private final DataOutputBuffer blockChecksumBuf = new DataOutputBuffer();
 
-    private MD5MD5CRC32FileChecksum fileChecksum;
+    private FileChecksum fileChecksum;
     private LocatedBlocks blockLocations;
 
     private int timeout;
@@ -88,12 +97,24 @@ final class FileChecksumHelper {
     FileChecksumComputer(String src, long length,
                          LocatedBlocks blockLocations,
                          ClientProtocol namenode,
-                         DFSClient client) throws IOException {
+                         DFSClient client,
+                         ChecksumCombineMode combineMode) throws IOException {
       this.src = src;
       this.length = length;
       this.blockLocations = blockLocations;
       this.namenode = namenode;
       this.client = client;
+      this.combineMode = combineMode;
+      switch (combineMode) {
+      case MD5MD5CRC:
+        this.blockChecksumType = BlockChecksumType.MD5CRC;
+        break;
+      case COMPOSITE_CRC:
+        this.blockChecksumType = BlockChecksumType.COMPOSITE_CRC;
+        break;
+      default:
+        throw new IOException("Unknown ChecksumCombineMode: " + combineMode);
+      }
 
       this.remaining = length;
 
@@ -121,11 +142,19 @@ final class FileChecksumHelper {
       return namenode;
     }
 
-    DataOutputBuffer getMd5out() {
-      return md5out;
+    ChecksumCombineMode getCombineMode() {
+      return combineMode;
+    }
+
+    BlockChecksumType getBlockChecksumType() {
+      return blockChecksumType;
+    }
+
+    DataOutputBuffer getBlockChecksumBuf() {
+      return blockChecksumBuf;
     }
 
-    MD5MD5CRC32FileChecksum getFileChecksum() {
+    FileChecksum getFileChecksum() {
       return fileChecksum;
     }
 
@@ -226,17 +255,31 @@ final class FileChecksumHelper {
     }
 
     /**
-     * Compute and aggregate block checksums block by block.
+     * Compute block checksums block by block and append the raw bytes of the
+     * block checksums into getBlockChecksumBuf().
+     *
      * @throws IOException
      */
     abstract void checksumBlocks() throws IOException;
 
     /**
-     * Make final file checksum result given the computing process done.
+     * Make final file checksum result given the per-block or per-block-group
+     * checksums collected into getBlockChecksumBuf().
      */
-    MD5MD5CRC32FileChecksum makeFinalResult() {
+    FileChecksum makeFinalResult() throws IOException {
+      switch (combineMode) {
+      case MD5MD5CRC:
+        return makeMd5CrcResult();
+      case COMPOSITE_CRC:
+        return makeCompositeCrcResult();
+      default:
+        throw new IOException("Unknown ChecksumCombineMode: " + combineMode);
+      }
+    }
+
+    FileChecksum makeMd5CrcResult() {
       //compute file MD5
-      final MD5Hash fileMD5 = MD5Hash.digest(md5out.getData());
+      final MD5Hash fileMD5 = MD5Hash.digest(blockChecksumBuf.getData());
       switch (crcType) {
       case CRC32:
         return new MD5MD5CRC32GzipFileChecksum(bytesPerCRC,
@@ -250,6 +293,58 @@ final class FileChecksumHelper {
       }
     }
 
+    FileChecksum makeCompositeCrcResult() throws IOException {
+      long blockSizeHint = 0;
+      if (locatedBlocks.size() > 0) {
+        blockSizeHint = locatedBlocks.get(0).getBlockSize();
+      }
+      CrcComposer crcComposer =
+          CrcComposer.newCrcComposer(getCrcType(), blockSizeHint);
+      byte[] blockChecksumBytes = blockChecksumBuf.getData();
+
+      long sumBlockLengths = 0;
+      for (int i = 0; i < locatedBlocks.size() - 1; ++i) {
+        LocatedBlock block = locatedBlocks.get(i);
+        // For everything except the last LocatedBlock, we expect getBlockSize()
+        // to accurately reflect the number of file bytes digested in the block
+        // checksum.
+        sumBlockLengths += block.getBlockSize();
+        int blockCrc = CrcUtil.readInt(blockChecksumBytes, i * 4);
+
+        crcComposer.update(blockCrc, block.getBlockSize());
+        LOG.debug(
+            "Added blockCrc 0x{} for block index {} of size {}",
+            Integer.toString(blockCrc, 16), i, block.getBlockSize());
+      }
+
+      // NB: In some cases the located blocks have their block size adjusted
+      // explicitly based on the requested length, but not all cases;
+      // these numbers may or may not reflect actual sizes on disk.
+      long reportedLastBlockSize =
+          blockLocations.getLastLocatedBlock().getBlockSize();
+      long consumedLastBlockLength = reportedLastBlockSize;
+      if (length - sumBlockLengths < reportedLastBlockSize) {
+        LOG.warn(
+            "Last block length {} is less than reportedLastBlockSize {}",
+            length - sumBlockLengths, reportedLastBlockSize);
+        consumedLastBlockLength = length - sumBlockLengths;
+      }
+      // NB: blockChecksumBytes.length may be much longer than actual bytes
+      // written into the DataOutput.
+      int lastBlockCrc = CrcUtil.readInt(
+          blockChecksumBytes, 4 * (locatedBlocks.size() - 1));
+      crcComposer.update(lastBlockCrc, consumedLastBlockLength);
+      LOG.debug(
+          "Added lastBlockCrc 0x{} for block index {} of size {}",
+          Integer.toString(lastBlockCrc, 16),
+          locatedBlocks.size() - 1,
+          consumedLastBlockLength);
+
+      int compositeCrc = CrcUtil.readInt(crcComposer.digest(), 0);
+      return new CompositeCrcFileChecksum(
+          compositeCrc, getCrcType(), bytesPerCRC);
+    }
+
     /**
      * Create and return a sender given an IO stream pair.
      */
@@ -267,6 +362,117 @@ final class FileChecksumHelper {
         IOUtils.closeStream(pair.out);
       }
     }
+
+    /**
+     * Parses out various checksum properties like bytesPerCrc, crcPerBlock,
+     * and crcType from {@code checksumData} and either stores them as the
+     * authoritative value or compares them to a previously extracted value
+     * to check comppatibility.
+     *
+     * @param checksumData response from the datanode
+     * @param locatedBlock the block corresponding to the response
+     * @param datanode the datanode which produced the response
+     * @param blockIdx the block or block-group index of the response
+     */
+    void extractChecksumProperties(
+        OpBlockChecksumResponseProto checksumData,
+        LocatedBlock locatedBlock,
+        DatanodeInfo datanode,
+        int blockIdx)
+        throws IOException {
+      //read byte-per-checksum
+      final int bpc = checksumData.getBytesPerCrc();
+      if (blockIdx == 0) { //first block
+        setBytesPerCRC(bpc);
+      } else if (bpc != getBytesPerCRC()) {
+        if (getBlockChecksumType() == BlockChecksumType.COMPOSITE_CRC) {
+          LOG.warn(
+              "Current bytesPerCRC={} doesn't match next bpc={}, but "
+              + "continuing anyway because we're using COMPOSITE_CRC. "
+              + "If trying to preserve CHECKSUMTYPE, only the current "
+              + "bytesPerCRC will be preserved.", getBytesPerCRC(), bpc);
+        } else {
+          throw new IOException("Byte-per-checksum not matched: bpc=" + bpc
+              + " but bytesPerCRC=" + getBytesPerCRC());
+        }
+      }
+
+      //read crc-per-block
+      final long cpb = checksumData.getCrcPerBlock();
+      if (getLocatedBlocks().size() > 1 && blockIdx == 0) {
+        setCrcPerBlock(cpb);
+      }
+
+      // read crc-type
+      final DataChecksum.Type ct;
+      if (checksumData.hasCrcType()) {
+        ct = PBHelperClient.convert(checksumData.getCrcType());
+      } else {
+        LOG.debug("Retrieving checksum from an earlier-version DataNode: " +
+            "inferring checksum by reading first byte");
+        ct = getClient().inferChecksumTypeByReading(locatedBlock, datanode);
+      }
+
+      if (blockIdx == 0) {
+        setCrcType(ct);
+      } else if (getCrcType() != DataChecksum.Type.MIXED &&
+          getCrcType() != ct) {
+        if (getBlockChecksumType() == BlockChecksumType.COMPOSITE_CRC) {
+          throw new IOException(
+              "DataChecksum.Type.MIXED is not supported for COMPOSITE_CRC");
+        } else {
+          // if crc types are mixed in a file
+          setCrcType(DataChecksum.Type.MIXED);
+        }
+      }
+
+      if (blockIdx == 0) {
+        LOG.debug("set bytesPerCRC={}, crcPerBlock={}",
+            getBytesPerCRC(), getCrcPerBlock());
+      }
+    }
+
+    /**
+     * Parses out the raw blockChecksum bytes from {@code checksumData}
+     * according to the blockChecksumType and populates the cumulative
+     * blockChecksumBuf with it.
+     *
+     * @return a debug-string representation of the parsed checksum if
+     *     debug is enabled, otherwise null.
+     */
+    String populateBlockChecksumBuf(OpBlockChecksumResponseProto checksumData)
+        throws IOException {
+      String blockChecksumForDebug = null;
+      switch (getBlockChecksumType()) {
+      case MD5CRC:
+        //read md5
+        final MD5Hash md5 = new MD5Hash(
+            checksumData.getBlockChecksum().toByteArray());
+        md5.write(getBlockChecksumBuf());
+        if (LOG.isDebugEnabled()) {
+          blockChecksumForDebug = md5.toString();
+        }
+        break;
+      case COMPOSITE_CRC:
+        BlockChecksumType returnedType = PBHelperClient.convert(
+            checksumData.getBlockChecksumOptions().getBlockChecksumType());
+        if (returnedType != BlockChecksumType.COMPOSITE_CRC) {
+          throw new IOException(String.format(
+              "Unexpected blockChecksumType '%s', expecting COMPOSITE_CRC",
+              returnedType));
+        }
+        byte[] crcBytes = checksumData.getBlockChecksum().toByteArray();
+        if (LOG.isDebugEnabled()) {
+          blockChecksumForDebug = CrcUtil.toSingleCrcString(crcBytes);
+        }
+        getBlockChecksumBuf().write(crcBytes);
+        break;
+      default:
+        throw new IOException(
+            "Unknown BlockChecksumType: " + getBlockChecksumType());
+      }
+      return blockChecksumForDebug;
+    }
   }
 
   /**
@@ -278,8 +484,10 @@ final class FileChecksumHelper {
     ReplicatedFileChecksumComputer(String src, long length,
                                    LocatedBlocks blockLocations,
                                    ClientProtocol namenode,
-                                   DFSClient client) throws IOException {
-      super(src, length, blockLocations, namenode, client);
+                                   DFSClient client,
+                                   ChecksumCombineMode combineMode)
+        throws IOException {
+      super(src, length, blockLocations, namenode, client, combineMode);
     }
 
     @Override
@@ -295,7 +503,8 @@ final class FileChecksumHelper {
         LocatedBlock locatedBlock = getLocatedBlocks().get(blockIdx);
 
         if (!checksumBlock(locatedBlock)) {
-          throw new IOException("Fail to get block MD5 for " + locatedBlock);
+          throw new PathIOException(
+              getSrc(), "Fail to get block MD5 for " + locatedBlock);
         }
       }
     }
@@ -368,9 +577,11 @@ final class FileChecksumHelper {
         LOG.debug("write to {}: {}, block={}", datanode,
             Op.BLOCK_CHECKSUM, block);
 
-        // get block MD5
-        createSender(pair).blockChecksum(block,
-            locatedBlock.getBlockToken());
+        // get block checksum
+        createSender(pair).blockChecksum(
+            block,
+            locatedBlock.getBlockToken(),
+            new BlockChecksumOptions(getBlockChecksumType()));
 
         final BlockOpResponseProto reply = BlockOpResponseProto.parseFrom(
             PBHelperClient.vintPrefixed(pair.in));
@@ -381,51 +592,11 @@ final class FileChecksumHelper {
 
         OpBlockChecksumResponseProto checksumData =
             reply.getChecksumResponse();
-
-        //read byte-per-checksum
-        final int bpc = checksumData.getBytesPerCrc();
-        if (blockIdx == 0) { //first block
-          setBytesPerCRC(bpc);
-        } else if (bpc != getBytesPerCRC()) {
-          throw new IOException("Byte-per-checksum not matched: bpc=" + bpc
-              + " but bytesPerCRC=" + getBytesPerCRC());
-        }
-
-        //read crc-per-block
-        final long cpb = checksumData.getCrcPerBlock();
-        if (getLocatedBlocks().size() > 1 && blockIdx == 0) {
-          setCrcPerBlock(cpb);
-        }
-
-        //read md5
-        final MD5Hash md5 = new MD5Hash(checksumData.getMd5().toByteArray());
-        md5.write(getMd5out());
-
-        // read crc-type
-        final DataChecksum.Type ct;
-        if (checksumData.hasCrcType()) {
-          ct = PBHelperClient.convert(checksumData.getCrcType());
-        } else {
-          LOG.debug("Retrieving checksum from an earlier-version DataNode: " +
-              "inferring checksum by reading first byte");
-          ct = getClient().inferChecksumTypeByReading(locatedBlock, datanode);
-        }
-
-        if (blockIdx == 0) { // first block
-          setCrcType(ct);
-        } else if (getCrcType() != DataChecksum.Type.MIXED
-            && getCrcType() != ct) {
-          // if crc types are mixed in a file
-          setCrcType(DataChecksum.Type.MIXED);
-        }
-
-        if (LOG.isDebugEnabled()) {
-          if (blockIdx == 0) {
-            LOG.debug("set bytesPerCRC=" + getBytesPerCRC()
-                + ", crcPerBlock=" + getCrcPerBlock());
-          }
-          LOG.debug("got reply from " + datanode + ": md5=" + md5);
-        }
+        extractChecksumProperties(
+            checksumData, locatedBlock, datanode, blockIdx);
+        String blockChecksumForDebug = populateBlockChecksumBuf(checksumData);
+        LOG.debug("got reply from {}: blockChecksum={}, blockChecksumType={}",
+            datanode, blockChecksumForDebug, getBlockChecksumType());
       }
     }
   }
@@ -442,9 +613,10 @@ final class FileChecksumHelper {
                                           LocatedBlocks blockLocations,
                                           ClientProtocol namenode,
                                           DFSClient client,
-                                          ErasureCodingPolicy ecPolicy)
+                                          ErasureCodingPolicy ecPolicy,
+                                          ChecksumCombineMode combineMode)
         throws IOException {
-      super(src, length, blockLocations, namenode, client);
+      super(src, length, blockLocations, namenode, client, combineMode);
 
       this.ecPolicy = ecPolicy;
     }
@@ -464,7 +636,8 @@ final class FileChecksumHelper {
         LocatedStripedBlock blockGroup = (LocatedStripedBlock) locatedBlock;
 
         if (!checksumBlockGroup(blockGroup)) {
-          throw new IOException("Fail to get block MD5 for " + locatedBlock);
+          throw new PathIOException(
+              getSrc(), "Fail to get block checksum for " + locatedBlock);
         }
       }
     }
@@ -519,16 +692,18 @@ final class FileChecksumHelper {
                              StripedBlockInfo stripedBlockInfo,
                              DatanodeInfo datanode,
                              long requestedNumBytes) throws IOException {
-
       try (IOStreamPair pair = getClient().connectToDN(datanode,
           getTimeout(), blockGroup.getBlockToken())) {
 
         LOG.debug("write to {}: {}, blockGroup={}",
             datanode, Op.BLOCK_GROUP_CHECKSUM, blockGroup);
 
-        // get block MD5
-        createSender(pair).blockGroupChecksum(stripedBlockInfo,
-            blockGroup.getBlockToken(), requestedNumBytes);
+        // get block group checksum
+        createSender(pair).blockGroupChecksum(
+            stripedBlockInfo,
+            blockGroup.getBlockToken(),
+            requestedNumBytes,
+            new BlockChecksumOptions(getBlockChecksumType()));
 
         BlockOpResponseProto reply = BlockOpResponseProto.parseFrom(
             PBHelperClient.vintPrefixed(pair.in));
@@ -538,54 +713,10 @@ final class FileChecksumHelper {
         DataTransferProtoUtil.checkBlockOpStatus(reply, logInfo);
 
         OpBlockChecksumResponseProto checksumData = reply.getChecksumResponse();
-
-        //read byte-per-checksum
-        final int bpc = checksumData.getBytesPerCrc();
-        if (bgIdx == 0) { //first block
-          setBytesPerCRC(bpc);
-        } else {
-          if (bpc != getBytesPerCRC()) {
-            throw new IOException("Byte-per-checksum not matched: bpc=" + bpc
-                + " but bytesPerCRC=" + getBytesPerCRC());
-          }
-        }
-
-        //read crc-per-block
-        final long cpb = checksumData.getCrcPerBlock();
-        if (getLocatedBlocks().size() > 1 && bgIdx == 0) { // first block
-          setCrcPerBlock(cpb);
-        }
-
-        //read md5
-        final MD5Hash md5 = new MD5Hash(
-            checksumData.getMd5().toByteArray());
-        md5.write(getMd5out());
-
-        // read crc-type
-        final DataChecksum.Type ct;
-        if (checksumData.hasCrcType()) {
-          ct = PBHelperClient.convert(checksumData.getCrcType());
-        } else {
-          LOG.debug("Retrieving checksum from an earlier-version DataNode: " +
-              "inferring checksum by reading first byte");
-          ct = getClient().inferChecksumTypeByReading(blockGroup, datanode);
-        }
-
-        if (bgIdx == 0) {
-          setCrcType(ct);
-        } else if (getCrcType() != DataChecksum.Type.MIXED &&
-            getCrcType() != ct) {
-          // if crc types are mixed in a file
-          setCrcType(DataChecksum.Type.MIXED);
-        }
-
-        if (LOG.isDebugEnabled()) {
-          if (bgIdx == 0) {
-            LOG.debug("set bytesPerCRC=" + getBytesPerCRC()
-                + ", crcPerBlock=" + getCrcPerBlock());
-          }
-          LOG.debug("got reply from " + datanode + ": md5=" + md5);
-        }
+        extractChecksumProperties(checksumData, blockGroup, datanode, bgIdx);
+        String blockChecksumForDebug = populateBlockChecksumBuf(checksumData);
+        LOG.debug("got reply from {}: blockChecksum={}, blockChecksumType={}",
+            datanode, blockChecksumForDebug, getBlockChecksumType());
       }
     }
   }

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

@@ -120,6 +120,8 @@ public interface HdfsClientConfigKeys {
   String  DFS_CHECKSUM_TYPE_DEFAULT = "CRC32C";
   String  DFS_BYTES_PER_CHECKSUM_KEY = "dfs.bytes-per-checksum";
   int     DFS_BYTES_PER_CHECKSUM_DEFAULT = 512;
+  String  DFS_CHECKSUM_COMBINE_MODE_KEY = "dfs.checksum.combine.mode";
+  String  DFS_CHECKSUM_COMBINE_MODE_DEFAULT = "MD5MD5CRC";
   String  DFS_DATANODE_SOCKET_WRITE_TIMEOUT_KEY =
       "dfs.datanode.socket.write.timeout";
   String  DFS_CLIENT_DOMAIN_SOCKET_DATA_TRAFFIC =

+ 27 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/impl/DfsClientConf.java

@@ -22,6 +22,7 @@ import com.google.common.base.Preconditions;
 import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.apache.hadoop.fs.Options.ChecksumCombineMode;
 import org.apache.hadoop.fs.Options.ChecksumOpt;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.ReplicaAccessorBuilder;
@@ -38,6 +39,8 @@ import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_BLOCK_SIZE_
 import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_BLOCK_SIZE_KEY;
 import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_BYTES_PER_CHECKSUM_DEFAULT;
 import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY;
+import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CHECKSUM_COMBINE_MODE_DEFAULT;
+import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CHECKSUM_COMBINE_MODE_KEY;
 import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CHECKSUM_TYPE_DEFAULT;
 import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CHECKSUM_TYPE_KEY;
 import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_CACHED_CONN_RETRY_DEFAULT;
@@ -106,6 +109,7 @@ public class DfsClientConf {
   private final int datanodeSocketWriteTimeout;
   private final int ioBufferSize;
   private final ChecksumOpt defaultChecksumOpt;
+  private final ChecksumCombineMode checksumCombineMode;
   private final int writePacketSize;
   private final int writeMaxPackets;
   private final ByteArrayManager.Conf writeByteArrayManagerConf;
@@ -177,6 +181,7 @@ public class DfsClientConf {
         CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_KEY,
         CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_DEFAULT);
     defaultChecksumOpt = getChecksumOptFromConf(conf);
+    checksumCombineMode = getChecksumCombineModeFromConf(conf);
     dataTransferTcpNoDelay = conf.getBoolean(
         DFS_DATA_TRANSFER_CLIENT_TCPNODELAY_KEY,
         DFS_DATA_TRANSFER_CLIENT_TCPNODELAY_DEFAULT);
@@ -300,6 +305,21 @@ public class DfsClientConf {
     }
   }
 
+  private static ChecksumCombineMode getChecksumCombineModeFromConf(
+      Configuration conf) {
+    final String mode = conf.get(
+        DFS_CHECKSUM_COMBINE_MODE_KEY,
+        DFS_CHECKSUM_COMBINE_MODE_DEFAULT);
+    try {
+      return ChecksumCombineMode.valueOf(mode);
+    } catch(IllegalArgumentException iae) {
+      LOG.warn("Bad checksum combine mode: {}. Using default {}", mode,
+               DFS_CHECKSUM_COMBINE_MODE_DEFAULT);
+      return ChecksumCombineMode.valueOf(
+          DFS_CHECKSUM_COMBINE_MODE_DEFAULT);
+    }
+  }
+
   // Construct a checksum option from conf
   public static ChecksumOpt getChecksumOptFromConf(Configuration conf) {
     DataChecksum.Type type = getChecksumType(conf);
@@ -392,6 +412,13 @@ public class DfsClientConf {
     return defaultChecksumOpt;
   }
 
+  /**
+   * @return the checksumCombineMode
+   */
+  public ChecksumCombineMode getChecksumCombineMode() {
+    return checksumCombineMode;
+  }
+
   /**
    * @return the writePacketSize
    */

+ 54 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/BlockChecksumOptions.java

@@ -0,0 +1,54 @@
+/**
+ * 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;
+
+/**
+ * Encapsulates various options related to how fine-grained data checksums are
+ * combined into block-level checksums.
+ */
+@InterfaceAudience.Private
+public class BlockChecksumOptions {
+  private final BlockChecksumType blockChecksumType;
+  private final long stripeLength;
+
+  public BlockChecksumOptions(
+      BlockChecksumType blockChecksumType, long stripeLength) {
+    this.blockChecksumType = blockChecksumType;
+    this.stripeLength = stripeLength;
+  }
+
+  public BlockChecksumOptions(BlockChecksumType blockChecksumType) {
+    this(blockChecksumType, 0);
+  }
+
+  public BlockChecksumType getBlockChecksumType() {
+    return blockChecksumType;
+  }
+
+  public long getStripeLength() {
+    return stripeLength;
+  }
+
+  @Override
+  public String toString() {
+    return String.format("blockChecksumType=%s, stripedLength=%d",
+        blockChecksumType, stripeLength);
+  }
+}

+ 30 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/BlockChecksumType.java

@@ -0,0 +1,30 @@
+/**
+ * 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;
+
+/**
+ * Algorithms/types denoting how block-level checksums are computed using
+ * lower-level chunk checksums/CRCs.
+ */
+@InterfaceAudience.Private
+public enum BlockChecksumType {
+  MD5CRC,  // BlockChecksum obtained by taking the MD5 digest of chunk CRCs
+  COMPOSITE_CRC  // Chunk-independent CRC, optionally striped
+}

+ 9 - 3
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/DataTransferProtocol.java

@@ -22,6 +22,7 @@ import java.io.IOException;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.hdfs.protocol.BlockChecksumOptions;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.StripedBlockInfo;
@@ -214,11 +215,13 @@ public interface DataTransferProtocol {
    *
    * @param blk a block.
    * @param blockToken security token for accessing the block.
+   * @param blockChecksumOptions determines how the block-level checksum is
+   *     computed from underlying block metadata.
    * @throws IOException
    */
   void blockChecksum(ExtendedBlock blk,
-      Token<BlockTokenIdentifier> blockToken) throws IOException;
-
+      Token<BlockTokenIdentifier> blockToken,
+      BlockChecksumOptions blockChecksumOptions) throws IOException;
 
   /**
    * Get striped block group checksum (MD5 of CRC32).
@@ -227,9 +230,12 @@ public interface DataTransferProtocol {
    * @param blockToken security token for accessing the block.
    * @param requestedNumBytes requested number of bytes in the block group
    *                          to compute the checksum.
+   * @param blockChecksumOptions determines how the block-level checksum is
+   *     computed from underlying block metadata.
    * @throws IOException
    */
   void blockGroupChecksum(StripedBlockInfo stripedBlockInfo,
           Token<BlockTokenIdentifier> blockToken,
-          long requestedNumBytes) throws IOException;
+          long requestedNumBytes,
+          BlockChecksumOptions blockChecksumOptions) throws IOException;
 }

+ 8 - 3
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Sender.java

@@ -27,6 +27,7 @@ import java.util.Arrays;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.hdfs.protocol.BlockChecksumOptions;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.StripedBlockInfo;
@@ -267,9 +268,11 @@ public class Sender implements DataTransferProtocol {
 
   @Override
   public void blockChecksum(final ExtendedBlock blk,
-      final Token<BlockTokenIdentifier> blockToken) throws IOException {
+      final Token<BlockTokenIdentifier> blockToken,
+      BlockChecksumOptions blockChecksumOptions) throws IOException {
     OpBlockChecksumProto proto = OpBlockChecksumProto.newBuilder()
         .setHeader(DataTransferProtoUtil.buildBaseHeader(blk, blockToken))
+        .setBlockChecksumOptions(PBHelperClient.convert(blockChecksumOptions))
         .build();
 
     send(out, Op.BLOCK_CHECKSUM, proto);
@@ -277,8 +280,9 @@ public class Sender implements DataTransferProtocol {
 
   @Override
   public void blockGroupChecksum(StripedBlockInfo stripedBlockInfo,
-      Token<BlockTokenIdentifier> blockToken, long requestedNumBytes)
-          throws IOException {
+      Token<BlockTokenIdentifier> blockToken,
+      long requestedNumBytes,
+      BlockChecksumOptions blockChecksumOptions) throws IOException {
     OpBlockGroupChecksumProto proto = OpBlockGroupChecksumProto.newBuilder()
         .setHeader(DataTransferProtoUtil.buildBaseHeader(
             stripedBlockInfo.getBlock(), blockToken))
@@ -291,6 +295,7 @@ public class Sender implements DataTransferProtocol {
         .setEcPolicy(PBHelperClient.convertErasureCodingPolicy(
             stripedBlockInfo.getErasureCodingPolicy()))
         .setRequestedNumBytes(requestedNumBytes)
+        .setBlockChecksumOptions(PBHelperClient.convert(blockChecksumOptions))
         .build();
 
     send(out, Op.BLOCK_GROUP_CHECKSUM, proto);

+ 44 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java

@@ -61,6 +61,8 @@ import org.apache.hadoop.hdfs.inotify.EventBatch;
 import org.apache.hadoop.hdfs.inotify.EventBatchList;
 import org.apache.hadoop.hdfs.protocol.AddErasureCodingPolicyResponse;
 import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.BlockChecksumOptions;
+import org.apache.hadoop.hdfs.protocol.BlockChecksumType;
 import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.protocol.BlockType;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry;
@@ -247,6 +249,48 @@ public class PBHelperClient {
     return HdfsProtos.ChecksumTypeProto.valueOf(type.id);
   }
 
+  public static HdfsProtos.BlockChecksumTypeProto convert(
+      BlockChecksumType type) {
+    switch(type) {
+    case MD5CRC:
+      return HdfsProtos.BlockChecksumTypeProto.MD5CRC;
+    case COMPOSITE_CRC:
+      return HdfsProtos.BlockChecksumTypeProto.COMPOSITE_CRC;
+    default:
+      throw new IllegalStateException(
+          "BUG: BlockChecksumType not found, type=" + type);
+    }
+  }
+
+  public static BlockChecksumType convert(
+      HdfsProtos.BlockChecksumTypeProto blockChecksumTypeProto) {
+    switch(blockChecksumTypeProto) {
+    case MD5CRC:
+      return BlockChecksumType.MD5CRC;
+    case COMPOSITE_CRC:
+      return BlockChecksumType.COMPOSITE_CRC;
+    default:
+      throw new IllegalStateException(
+          "BUG: BlockChecksumTypeProto not found, type="
+          + blockChecksumTypeProto);
+    }
+  }
+
+  public static HdfsProtos.BlockChecksumOptionsProto convert(
+      BlockChecksumOptions options) {
+    return HdfsProtos.BlockChecksumOptionsProto.newBuilder()
+        .setBlockChecksumType(convert(options.getBlockChecksumType()))
+        .setStripeLength(options.getStripeLength())
+        .build();
+  }
+
+  public static BlockChecksumOptions convert(
+      HdfsProtos.BlockChecksumOptionsProto options) {
+    return new BlockChecksumOptions(
+        convert(options.getBlockChecksumType()),
+        options.getStripeLength());
+  }
+
   public static ExtendedBlockProto convert(final ExtendedBlock b) {
     if (b == null) return null;
     return ExtendedBlockProto.newBuilder().

+ 0 - 3
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/RequestHedgingProxyProvider.java

@@ -79,9 +79,6 @@ public class RequestHedgingProxyProvider<T> extends
     public Object
     invoke(Object proxy, final Method method, final Object[] args)
             throws Throwable {
-      if (currentUsedProxy != null) {
-        return method.invoke(currentUsedProxy.proxy, args);
-      }
       Map<Future<Object>, ProxyInfo<T>> proxyMap = new HashMap<>();
       int numAttempts = 0;
 

+ 7 - 4
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/shortcircuit/ShortCircuitCache.java

@@ -664,6 +664,7 @@ public class ShortCircuitCache implements Closeable {
     unref(replica);
   }
 
+  static final int FETCH_OR_CREATE_RETRY_TIMES = 3;
   /**
    * Fetch or create a replica.
    *
@@ -678,11 +679,11 @@ public class ShortCircuitCache implements Closeable {
    */
   public ShortCircuitReplicaInfo fetchOrCreate(ExtendedBlockId key,
       ShortCircuitReplicaCreator creator) {
-    Waitable<ShortCircuitReplicaInfo> newWaitable = null;
+    Waitable<ShortCircuitReplicaInfo> newWaitable;
     lock.lock();
     try {
       ShortCircuitReplicaInfo info = null;
-      do {
+      for (int i = 0; i < FETCH_OR_CREATE_RETRY_TIMES; i++){
         if (closed) {
           LOG.trace("{}: can't fethchOrCreate {} because the cache is closed.",
               this, key);
@@ -692,11 +693,12 @@ public class ShortCircuitCache implements Closeable {
         if (waitable != null) {
           try {
             info = fetch(key, waitable);
+            break;
           } catch (RetriableException e) {
             LOG.debug("{}: retrying {}", this, e.getMessage());
           }
         }
-      } while (false);
+      }
       if (info != null) return info;
       // We need to load the replica ourselves.
       newWaitable = new Waitable<>(lock.newCondition());
@@ -717,7 +719,8 @@ public class ShortCircuitCache implements Closeable {
    *
    * @throws RetriableException   If the caller needs to retry.
    */
-  private ShortCircuitReplicaInfo fetch(ExtendedBlockId key,
+  @VisibleForTesting // ONLY for testing
+  protected ShortCircuitReplicaInfo fetch(ExtendedBlockId key,
       Waitable<ShortCircuitReplicaInfo> waitable) throws RetriableException {
     // Another thread is already in the process of loading this
     // ShortCircuitReplica.  So we simply wait for it to complete.

+ 5 - 2
hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/datatransfer.proto

@@ -148,8 +148,9 @@ message OpCopyBlockProto {
   required BaseHeaderProto header = 1;
 }
 
-message OpBlockChecksumProto { 
+message OpBlockChecksumProto {
   required BaseHeaderProto header = 1;
+  optional BlockChecksumOptionsProto blockChecksumOptions = 2;
 }
 
 message OpBlockGroupChecksumProto {
@@ -160,6 +161,7 @@ message OpBlockGroupChecksumProto {
   required ErasureCodingPolicyProto ecPolicy = 4;
   repeated uint32 blockIndices = 5;
   required uint64 requestedNumBytes = 6;
+  optional BlockChecksumOptionsProto blockChecksumOptions = 7;
 }
 
 /**
@@ -313,8 +315,9 @@ message DNTransferAckProto {
 message OpBlockChecksumResponseProto {
   required uint32 bytesPerCrc = 1;
   required uint64 crcPerBlock = 2;
-  required bytes md5 = 3;
+  required bytes blockChecksum = 3;
   optional ChecksumTypeProto crcType = 4;
+  optional BlockChecksumOptionsProto blockChecksumOptions = 5;
 }
 
 message OpCustomProto {

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

@@ -480,6 +480,27 @@ enum ChecksumTypeProto {
   CHECKSUM_CRC32C = 2;
 }
 
+enum BlockChecksumTypeProto {
+  MD5CRC = 1;  // BlockChecksum obtained by taking the MD5 digest of chunk CRCs
+  COMPOSITE_CRC = 2;  // Chunk-independent CRC, optionally striped
+}
+
+/**
+ * Algorithms/types denoting how block-level checksums are computed using
+ * lower-level chunk checksums/CRCs.
+ * These options should be kept in sync with
+ * org.apache.hadoop.hdfs.protocol.BlockChecksumOptions.
+ */
+message BlockChecksumOptionsProto {
+  optional BlockChecksumTypeProto blockChecksumType = 1 [default = MD5CRC];
+
+  // Only used if blockChecksumType specifies a striped format, such as
+  // COMPOSITE_CRC. If so, then the blockChecksum in the response is expected
+  // to be the concatenation of N crcs, where
+  // N == ((requestedLength - 1) / stripedLength) + 1
+  optional uint64 stripeLength = 2;
+}
+
 /**
  * HDFS Server Defaults
  */

+ 0 - 34
hadoop-hdfs-project/hadoop-hdfs-client/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestRequestHedgingProxyProvider.java

@@ -43,13 +43,10 @@ import org.junit.Assert;
 import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
-import static org.junit.Assert.assertEquals;
 import org.mockito.Matchers;
 import org.mockito.Mockito;
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
-import static org.mockito.Mockito.when;
-import static org.mockito.Mockito.mock;
 
 import com.google.common.collect.Lists;
 
@@ -102,37 +99,6 @@ public class TestRequestHedgingProxyProvider {
     Mockito.verify(goodMock).getStats();
   }
 
-  @Test
-  public void testRequestNNAfterOneSuccess() throws Exception {
-    final AtomicInteger count = new AtomicInteger(0);
-    final ClientProtocol goodMock = mock(ClientProtocol.class);
-    when(goodMock.getStats()).thenAnswer(new Answer<long[]>() {
-      @Override
-      public long[] answer(InvocationOnMock invocation) throws Throwable {
-        count.incrementAndGet();
-        Thread.sleep(1000);
-        return new long[]{1};
-      }
-    });
-    final ClientProtocol badMock = mock(ClientProtocol.class);
-    when(badMock.getStats()).thenAnswer(new Answer<long[]>() {
-      @Override
-      public long[] answer(InvocationOnMock invocation) throws Throwable {
-        count.incrementAndGet();
-        throw new IOException("Bad mock !!");
-      }
-    });
-
-    RequestHedgingProxyProvider<ClientProtocol> provider =
-        new RequestHedgingProxyProvider<>(conf, nnUri, ClientProtocol.class,
-            createFactory(badMock, goodMock, goodMock, badMock));
-    ClientProtocol proxy = provider.getProxy().proxy;
-    proxy.getStats();
-    assertEquals(2, count.get());
-    proxy.getStats();
-    assertEquals(3, count.get());
-  }
-
   @Test
   public void testHedgingWhenOneIsSlow() throws Exception {
     final ClientProtocol goodMock = Mockito.mock(ClientProtocol.class);

+ 1 - 52
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/include/hdfspp/hdfspp.h

@@ -19,6 +19,7 @@
 #define LIBHDFSPP_HDFSPP_H_
 
 #include "hdfspp/options.h"
+#include "hdfspp/ioservice.h"
 #include "hdfspp/status.h"
 #include "hdfspp/events.h"
 #include "hdfspp/block_location.h"
@@ -31,61 +32,9 @@
 
 #include <functional>
 #include <memory>
-#include <set>
-#include <iostream>
 
 namespace hdfs {
 
-/**
- * An IoService manages a queue of asynchronous tasks. All libhdfs++
- * operations are filed against a particular IoService.
- *
- * When an operation is queued into an IoService, the IoService will
- * run the callback handler associated with the operation. Note that
- * the IoService must be stopped before destructing the objects that
- * post the operations.
- *
- * From an implementation point of view the hdfs::IoService provides
- * a thin wrapper over an asio::io_service object so that additional
- * instrumentation and functionality can be added.
- **/
-
-class IoService : public std::enable_shared_from_this<IoService>
-{
- public:
-  static IoService *New();
-  static std::shared_ptr<IoService> MakeShared();
-  virtual ~IoService();
-
-  /**
-   * Start up as many threads as there are logical processors.
-   * Return number of threads created.
-   **/
-  virtual unsigned int InitDefaultWorkers() = 0;
-
-  /**
-   * Initialize with thread_count handler threads.
-   * If thread count is less than one print a log message and default to one thread.
-   * Return number of threads created.
-   **/
-  virtual unsigned int InitWorkers(unsigned int thread_count) = 0;
-
-  /**
-   * Place an item on the execution queue.  Will be invoked from outside of the calling context.
-   **/
-  virtual void PostTask(std::function<void(void)>& asyncTask) = 0;
-
-  /**
-   * Run the asynchronous tasks associated with this IoService.
-   **/
-  virtual void Run() = 0;
-  /**
-   * Stop running asynchronous tasks associated with this IoService.
-   * All worker threads will return as soon as they finish executing their current task.
-   **/
-  virtual void Stop() = 0;
-};
-
 /**
  * A node exclusion rule provides a simple way of testing if the
  * client should attempt to connect to a node based on the node's

+ 140 - 0
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/include/hdfspp/ioservice.h

@@ -0,0 +1,140 @@
+/**
+ * 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.
+ */
+
+/**
+ * An asio::io_service maintains a queue of asynchronous tasks and invokes them
+ * when they are ready to run.  Async network IO handlers become runnable when
+ * the associated IO operation has completed.  The hdfs::IoService is a thin
+ * wrapper over that object to make it easier to add logging and instrumentation
+ * to tasks that have been queued.
+ *
+ * Lifecycle management:
+ *   -The IoService *shall* outlive any tasks it owns.  Deleting a task
+ *    before it has been run **will** result in dangling reference issues.
+ *   -Dependencies (including transitive dependencies) of pending tasks
+ *    *shall* outlive the task.  Failure to ensure this **will** result in
+ *    danging reference issues.
+ *     -libhdfs++ uses shared_ptr/weak_ptr heavily as a mechanism to ensure
+ *      liveness of dependencies.
+ *     -refcounted pointers in lambda capture lists have a poor track record
+ *      for ensuring liveness in this library; it's easy to omit them because
+ *      the capture list isn't context aware.  Developers are encouraged to
+ *      write callable classes that explicitly list dependencies.
+ *
+ * Constraints on tasks:
+ *   -Tasks and async callbacks *shall* never do blocking IO or sleep().
+ *    At best this hurts performance by preventing worker threads from doing
+ *    useful work.  It may also cause situations that look like deadlocks
+ *    if the worker thread is stalled for long enough.
+ *   -Tasks and async callbacks *shall* not acquire locks that guard resources
+ *    that might be unavailable for an unknown amount of time.  Lock acquisition
+ *    when accessing shared data structures is acceptable and is often required.
+ *   -Tasks and async callbacks *should* not allow exceptions to escape their
+ *    scope since tasks will be executed on a different stack then where they
+ *    were created.  The exception will be caught by the IoService rather than
+ *    being forwarded to the next task.
+ *   -Tasks and async callbacks *should* not rely on thread local storage for
+ *    ancillary context.  The IoService does not support any sort of thread
+ *    affinity that would guarantee tasks Post()ed from one thread will always
+ *    be executed on the same thread.  Applications that only use a single
+ *    worker thread may use TLS but developers should be mindful that throughput
+ *    can no longer be scaled by adding threads.
+ **/
+#ifndef INCLUDE_HDFSPP_IOSERVICE_H_
+#define INCLUDE_HDFSPP_IOSERVICE_H_
+
+#include <memory>
+
+// forward decl
+namespace asio {
+  class io_service;
+}
+
+namespace hdfs {
+
+// (Un)comment this to determine if issues are due to concurrency or logic faults
+// If tests still fail with concurrency disabled it's most likely a logic bug
+#define DISABLE_CONCURRENT_WORKERS
+
+class IoService : public std::enable_shared_from_this<IoService>
+{
+ public:
+  static IoService *New();
+  static std::shared_ptr<IoService> MakeShared();
+  virtual ~IoService();
+
+  /**
+   * Start up as many threads as there are logical processors.
+   * Return number of threads created.
+   **/
+  virtual unsigned int InitDefaultWorkers() = 0;
+
+  /**
+   * Initialize with thread_count handler threads.
+   * If thread count is less than one print a log message and default to one thread.
+   * Return number of threads created.
+   **/
+  virtual unsigned int InitWorkers(unsigned int thread_count) = 0;
+
+  /**
+   * Add a worker thread to existing pool.
+   * Return true on success, false otherwise.
+   **/
+  virtual bool AddWorkerThread() = 0;
+
+  /**
+   * Return the number of worker threads in use.
+   **/
+  virtual unsigned int GetWorkerThreadCount() = 0;
+
+  /**
+   * Enqueue an item for deferred execution.  Non-blocking.
+   * Task will be invoked from outside of the calling context.
+   **/
+  virtual void PostTask(std::function<void(void)> asyncTask) = 0;
+
+  /**
+   * Provide type erasure for lambdas defined inside the argument list.
+   **/
+  template <typename LambdaInstance>
+  inline void PostLambda(LambdaInstance&& func)
+  {
+    std::function<void(void)> typeEraser = func;
+    this->PostTask(func);
+  }
+
+  /**
+   * Run the asynchronous tasks associated with this IoService.
+   **/
+  virtual void Run() = 0;
+  /**
+   * Stop running asynchronous tasks associated with this IoService.
+   * All worker threads will return as soon as they finish executing their current task.
+   **/
+  virtual void Stop() = 0;
+
+  /**
+   * Access underlying io_service object.  Only to be used in asio library calls.
+   * After HDFS-11884 is complete only tests should need direct access to the asio::io_service.
+   **/
+  virtual asio::io_service& GetRaw() = 0;
+};
+
+
+} // namespace hdfs
+#endif // include guard

+ 3 - 4
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/bindings/c/hdfs.cc

@@ -17,18 +17,17 @@
  */
 
 #include "hdfspp/hdfspp.h"
+#include "hdfspp/hdfs_ext.h"
 
-#include "fs/filesystem.h"
 #include "common/hdfs_configuration.h"
 #include "common/configuration_loader.h"
 #include "common/logging.h"
+#include "fs/filesystem.h"
+#include "fs/filehandle.h"
 
-#include <hdfs/hdfs.h>
-#include <hdfspp/hdfs_ext.h>
 
 #include <libgen.h>
 #include "limits.h"
-
 #include <string>
 #include <cstring>
 #include <iostream>

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/CMakeLists.txt

@@ -19,6 +19,6 @@ if(NEED_LINK_DL)
    set(LIB_DL dl)
 endif()
 
-add_library(common_obj OBJECT status.cc sasl_digest_md5.cc hdfs_ioservice.cc options.cc configuration.cc configuration_loader.cc hdfs_configuration.cc uri.cc util.cc retry_policy.cc cancel_tracker.cc logging.cc libhdfs_events_impl.cc auth_info.cc namenode_info.cc statinfo.cc fsinfo.cc content_summary.cc locks.cc config_parser.cc)
+add_library(common_obj OBJECT status.cc sasl_digest_md5.cc ioservice_impl.cc options.cc configuration.cc configuration_loader.cc hdfs_configuration.cc uri.cc util.cc retry_policy.cc cancel_tracker.cc logging.cc libhdfs_events_impl.cc auth_info.cc namenode_info.cc statinfo.cc fsinfo.cc content_summary.cc locks.cc config_parser.cc)
 add_library(common $<TARGET_OBJECTS:common_obj> $<TARGET_OBJECTS:uriparser2_obj>)
 target_link_libraries(common ${LIB_DL})

+ 8 - 5
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/async_stream.h

@@ -19,12 +19,15 @@
 #ifndef LIB_COMMON_ASYNC_STREAM_H_
 #define LIB_COMMON_ASYNC_STREAM_H_
 
-#include <asio.hpp>
+#include <asio/buffer.hpp>
+#include <asio/error_code.hpp>
+#include <functional>
 
 namespace hdfs {
 
-typedef asio::mutable_buffers_1 MutableBuffers;
-typedef asio::const_buffers_1   ConstBuffers;
+// Contiguous buffer types
+typedef asio::mutable_buffers_1 MutableBuffer;
+typedef asio::const_buffers_1   ConstBuffer;
 
 /*
  * asio-compatible stream implementation.
@@ -35,11 +38,11 @@ typedef asio::const_buffers_1   ConstBuffers;
  */
 class AsyncStream  {
 public:
-  virtual void async_read_some(const MutableBuffers &buf,
+  virtual void async_read_some(const MutableBuffer &buf,
           std::function<void (const asio::error_code & error,
                                  std::size_t bytes_transferred) > handler) = 0;
 
-  virtual void async_write_some(const ConstBuffers &buf,
+  virtual void async_write_some(const ConstBuffer &buf,
             std::function<void (const asio::error_code & error,
                                  std::size_t bytes_transferred) > handler) = 0;
 };

+ 0 - 5
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/continuation/asio.h

@@ -20,13 +20,8 @@
 
 #include "continuation.h"
 #include "common/util.h"
-
 #include "hdfspp/status.h"
-
-#include <asio/connect.hpp>
-#include <asio/read.hpp>
 #include <asio/write.hpp>
-#include <asio/ip/tcp.hpp>
 #include <memory>
 
 namespace hdfs {

+ 18 - 5
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/hdfs_ioservice.cc → hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/ioservice_impl.cc

@@ -16,14 +16,16 @@
  * limitations under the License.
  */
 
-#include "hdfs_ioservice.h"
+#include "ioservice_impl.h"
 
 #include <thread>
 #include <mutex>
 #include <vector>
 
+#include "common/util.h"
 #include "common/logging.h"
 
+
 namespace hdfs {
 
 IoService::~IoService() {}
@@ -99,7 +101,7 @@ void IoServiceImpl::ThreadExitHook() {
   LOG_DEBUG(kAsyncRuntime, << "Worker thread #" << std::this_thread::get_id() << " for IoServiceImpl@" << this << " exiting");
 }
 
-void IoServiceImpl::PostTask(std::function<void(void)>& asyncTask) {
+void IoServiceImpl::PostTask(std::function<void(void)> asyncTask) {
   io_service_.post(asyncTask);
 }
 
@@ -133,14 +135,25 @@ void IoServiceImpl::Run() {
     } catch (const std::exception & e) {
       LOG_WARN(kFileSystem, << "Unexpected exception in libhdfspp worker thread: " << e.what());
     } catch (...) {
-      LOG_WARN(kFileSystem, << "Unexpected value not derived from std::exception in libhdfspp worker thread");
+      LOG_WARN(kFileSystem, << "Caught unexpected value not derived from std::exception in libhdfspp worker thread");
     }
   }
 }
 
-unsigned int IoServiceImpl::get_worker_thread_count() {
+void IoServiceImpl::Stop() {
+  // Note: This doesn't wait for running operations to stop.
+  io_service_.stop();
+}
+
+asio::io_service& IoServiceImpl::GetRaw() {
+  return io_service_;
+}
+
+unsigned int IoServiceImpl::GetWorkerThreadCount() {
   mutex_guard state_lock(state_lock_);
   return worker_threads_.size();
-}
 
 }
+
+
+} // namespace hdfs

+ 12 - 15
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/hdfs_ioservice.h → hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/ioservice_impl.h

@@ -19,20 +19,16 @@
 #ifndef COMMON_HDFS_IOSERVICE_H_
 #define COMMON_HDFS_IOSERVICE_H_
 
-#include "hdfspp/hdfspp.h"
+#include "hdfspp/ioservice.h"
 
 #include <asio/io_service.hpp>
-#include "common/util.h"
+#include "common/new_delete.h"
 
 #include <mutex>
 #include <thread>
 
 namespace hdfs {
 
-// Uncomment this to determine if issues are due to concurrency or logic faults
-// If tests still fail with concurrency disabled it's most likely a logic bug
-#define DISABLE_CONCURRENT_WORKERS
-
 /*
  *  A thin wrapper over the asio::io_service with a few extras
  *    -manages it's own worker threads
@@ -41,23 +37,24 @@ namespace hdfs {
 
 class IoServiceImpl : public IoService {
  public:
+  MEMCHECKED_CLASS(IoServiceImpl)
   IoServiceImpl() {}
 
-  virtual unsigned int InitDefaultWorkers() override;
-  virtual unsigned int InitWorkers(unsigned int thread_count) override;
-  virtual void PostTask(std::function<void(void)>& asyncTask) override;
-  virtual void Run() override;
-  virtual void Stop() override { io_service_.stop(); }
+  unsigned int InitDefaultWorkers() override;
+  unsigned int InitWorkers(unsigned int thread_count) override;
+  void PostTask(std::function<void(void)> asyncTask) override;
+  void Run() override;
+  void Stop() override;
+  asio::io_service& GetRaw() override;
 
   // Add a single worker thread, in the common case try to avoid this in favor
   // of Init[Default]Workers. Public for use by tests and rare cases where a
   // client wants very explicit control of threading for performance reasons
   // e.g. pinning threads to NUMA nodes.
-  bool AddWorkerThread();
+  bool AddWorkerThread() override;
+
+  unsigned int GetWorkerThreadCount() override;
 
-  // Be very careful about using this: HDFS-10241
-  ::asio::io_service &io_service() { return io_service_; }
-  unsigned int get_worker_thread_count();
  private:
   std::mutex state_lock_;
   ::asio::io_service io_service_;

+ 0 - 3
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/logging.h

@@ -23,14 +23,11 @@
 
 #include "hdfspp/log.h"
 
-#include <iostream>
 #include <sstream>
 #include <mutex>
 #include <memory>
 #include <thread>
 
-#include <asio/ip/tcp.hpp>
-
 namespace hdfs {
 
 /**

+ 7 - 8
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/namenode_info.cc

@@ -20,11 +20,12 @@
 
 #include "common/util.h"
 #include "common/logging.h"
+#include "hdfspp/ioservice.h"
 
 #include <sstream>
 #include <utility>
 #include <future>
-#include <memory>
+
 
 namespace hdfs {
 
@@ -35,8 +36,6 @@ ResolvedNamenodeInfo& ResolvedNamenodeInfo::operator=(const NamenodeInfo &info)
   return *this;
 }
 
-
-
 std::string ResolvedNamenodeInfo::str() const {
   std::stringstream ss;
   ss << "ResolvedNamenodeInfo {nameservice: " << nameservice << ", name: " << name << ", uri: " << uri.str();
@@ -58,7 +57,7 @@ std::string ResolvedNamenodeInfo::str() const {
 }
 
 
-bool ResolveInPlace(::asio::io_service *ioservice, ResolvedNamenodeInfo &info) {
+bool ResolveInPlace(std::shared_ptr<IoService> ioservice, ResolvedNamenodeInfo &info) {
   // this isn't very memory friendly, but if it needs to be called often there are bigger issues at hand
   info.endpoints.clear();
   std::vector<ResolvedNamenodeInfo> resolved = BulkResolve(ioservice, {info});
@@ -76,7 +75,7 @@ typedef std::vector<asio::ip::tcp::endpoint> endpoint_vector;
 // RAII wrapper
 class ScopedResolver {
  private:
-  ::asio::io_service *io_service_;
+  std::shared_ptr<IoService> io_service_;
   std::string host_;
   std::string port_;
   ::asio::ip::tcp::resolver::query query_;
@@ -86,8 +85,8 @@ class ScopedResolver {
   // Caller blocks on access if resolution isn't finished
   std::shared_ptr<std::promise<Status>> result_status_;
  public:
-  ScopedResolver(::asio::io_service *service, const std::string &host, const std::string &port) :
-        io_service_(service), host_(host), port_(port), query_(host, port), resolver_(*io_service_)
+  ScopedResolver(std::shared_ptr<IoService> service, const std::string &host, const std::string &port) :
+        io_service_(service), host_(host), port_(port), query_(host, port), resolver_(io_service_->GetRaw())
   {
     if(!io_service_)
       LOG_ERROR(kAsyncRuntime, << "ScopedResolver@" << this << " passed nullptr to io_service");
@@ -140,7 +139,7 @@ class ScopedResolver {
   }
 };
 
-std::vector<ResolvedNamenodeInfo> BulkResolve(::asio::io_service *ioservice, const std::vector<NamenodeInfo> &nodes) {
+std::vector<ResolvedNamenodeInfo> BulkResolve(std::shared_ptr<IoService> ioservice, const std::vector<NamenodeInfo> &nodes) {
   std::vector< std::unique_ptr<ScopedResolver> > resolvers;
   resolvers.reserve(nodes.size());
 

+ 6 - 2
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/namenode_info.h

@@ -20,6 +20,7 @@
 #define COMMON_HDFS_NAMENODE_INFO_H_
 
 #include <asio.hpp>
+
 #include <hdfspp/options.h>
 
 #include <string>
@@ -27,6 +28,9 @@
 
 namespace hdfs {
 
+// Forward decl
+class IoService;
+
 // Internal representation of namenode info that keeps track
 // of its endpoints.
 struct ResolvedNamenodeInfo : public NamenodeInfo {
@@ -38,11 +42,11 @@ struct ResolvedNamenodeInfo : public NamenodeInfo {
 
 // Clear endpoints if set and resolve all of them in parallel.
 // Only successful lookups will be placed in the result set.
-std::vector<ResolvedNamenodeInfo> BulkResolve(::asio::io_service *ioservice, const std::vector<NamenodeInfo> &nodes);
+std::vector<ResolvedNamenodeInfo> BulkResolve(std::shared_ptr<IoService> ioservice, const std::vector<NamenodeInfo> &nodes);
 
 // Clear endpoints, if any, and resolve them again
 // Return true if endpoints were resolved
-bool ResolveInPlace(::asio::io_service *ioservice, ResolvedNamenodeInfo &info);
+bool ResolveInPlace(std::shared_ptr<IoService> ioservice, ResolvedNamenodeInfo &info);
 
 }
 

+ 13 - 1
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/util.cc

@@ -19,17 +19,25 @@
 #include "common/util.h"
 #include "common/util_c.h"
 
+#include <google/protobuf/message_lite.h>
 #include <google/protobuf/io/zero_copy_stream_impl_lite.h>
 
 #include <exception>
 #include <sstream>
-#include <iostream>
 #include <iomanip>
 #include <thread>
 
 
 namespace hdfs {
 
+Status ToStatus(const ::asio::error_code &ec) {
+  if (ec) {
+    return Status(ec.value(), ec.message().c_str());
+  } else {
+    return Status::OK();
+  }
+}
+
 bool ReadDelimitedPBMessage(::google::protobuf::io::CodedInputStream *in,
                             ::google::protobuf::MessageLite *msg) {
   uint32_t size = 0;
@@ -60,6 +68,10 @@ std::string SerializeDelimitedProtobufMessage(const ::google::protobuf::MessageL
   return buf;
 }
 
+int DelimitedPBMessageSize(const ::google::protobuf::MessageLite *msg) {
+  size_t size = msg->ByteSize();
+  return ::google::protobuf::io::CodedOutputStream::VarintSize32(size) + size;
+}
 
 std::string GetRandomClientName() {
   std::vector<unsigned char>buf(8);

+ 9 - 16
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/util.h

@@ -21,16 +21,19 @@
 #include "hdfspp/status.h"
 #include "common/logging.h"
 
-#include <sstream>
 #include <mutex>
 #include <string>
 
 #include <asio/error_code.hpp>
 #include <openssl/rand.h>
-
-#include <google/protobuf/message_lite.h>
 #include <google/protobuf/io/coded_stream.h>
-#include <asio.hpp>
+
+
+namespace google {
+  namespace protobuf {
+    class MessageLite;
+  }
+}
 
 namespace hdfs {
 
@@ -38,20 +41,11 @@ namespace hdfs {
 typedef std::lock_guard<std::mutex> mutex_guard;
 
 
-static inline Status ToStatus(const ::asio::error_code &ec) {
-  if (ec) {
-    return Status(ec.value(), ec.message().c_str());
-  } else {
-    return Status::OK();
-  }
-}
+Status ToStatus(const ::asio::error_code &ec);
 
 // Determine size of buffer that needs to be allocated in order to serialize msg
 // in delimited format
-static inline int DelimitedPBMessageSize(const ::google::protobuf::MessageLite *msg) {
-  size_t size = msg->ByteSize();
-  return ::google::protobuf::io::CodedOutputStream::VarintSize32(size) + size;
-}
+int DelimitedPBMessageSize(const ::google::protobuf::MessageLite *msg);
 
 // Construct msg from the input held in the CodedInputStream
 // return false on failure, otherwise return true
@@ -84,7 +78,6 @@ bool lock_held(T & mutex) {
 std::string SafeDisconnect(asio::ip::tcp::socket *sock);
 
 
-
 // The following helper function is used for classes that look like the following:
 //
 // template <typename socket_like_object>

+ 22 - 5
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/connection/datanodeconnection.cc

@@ -24,14 +24,14 @@ namespace hdfs {
 DataNodeConnection::~DataNodeConnection(){}
 DataNodeConnectionImpl::~DataNodeConnectionImpl(){}
 
-DataNodeConnectionImpl::DataNodeConnectionImpl(asio::io_service * io_service,
-                                                const ::hadoop::hdfs::DatanodeInfoProto &dn_proto,
-                                                const hadoop::common::TokenProto *token,
-                                                LibhdfsEvents *event_handlers) : event_handlers_(event_handlers)
+DataNodeConnectionImpl::DataNodeConnectionImpl(std::shared_ptr<IoService> io_service,
+                                               const ::hadoop::hdfs::DatanodeInfoProto &dn_proto,
+                                               const hadoop::common::TokenProto *token,
+                                               LibhdfsEvents *event_handlers) : event_handlers_(event_handlers)
 {
   using namespace ::asio::ip;
 
-  conn_.reset(new tcp::socket(*io_service));
+  conn_.reset(new tcp::socket(io_service->GetRaw()));
   auto datanode_addr = dn_proto.id();
   endpoints_[0] = tcp::endpoint(address::from_string(datanode_addr.ipaddr()),
                                   datanode_addr.xferport());
@@ -68,5 +68,22 @@ void DataNodeConnectionImpl::Cancel() {
   }
 }
 
+void DataNodeConnectionImpl::async_read_some(const MutableBuffer &buf,
+             std::function<void (const asio::error_code & error, std::size_t bytes_transferred) > handler)
+{
+  event_handlers_->call("DN_read_req", "", "", buf.end() - buf.begin());
+
+  mutex_guard state_lock(state_lock_);
+  conn_->async_read_some(buf, handler);
+}
+
+void DataNodeConnectionImpl::async_write_some(const ConstBuffer &buf,
+             std::function<void (const asio::error_code & error, std::size_t bytes_transferred) > handler)
+{
+  event_handlers_->call("DN_write_req", "", "", buf.end() - buf.begin());
+
+  mutex_guard state_lock(state_lock_);
+  conn_->async_write_some(buf, handler);
+}
 
 }

+ 7 - 19
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/connection/datanodeconnection.h

@@ -18,7 +18,7 @@
 #ifndef LIBHDFSPP_LIB_CONNECTION_DATANODECONNECTION_H_
 #define LIBHDFSPP_LIB_CONNECTION_DATANODECONNECTION_H_
 
-#include "common/hdfs_ioservice.h"
+#include "hdfspp/ioservice.h"
 #include "common/async_stream.h"
 #include "ClientNamenodeProtocol.pb.h"
 #include "common/libhdfs_events_impl.h"
@@ -58,13 +58,14 @@ private:
   // held (briefly) while posting async ops to the asio task queue
   std::mutex state_lock_;
 public:
+  MEMCHECKED_CLASS(DataNodeConnectionImpl)
   std::unique_ptr<asio::ip::tcp::socket, SocketDeleter> conn_;
   std::array<asio::ip::tcp::endpoint, 1> endpoints_;
   std::string uuid_;
   LibhdfsEvents *event_handlers_;
 
   virtual ~DataNodeConnectionImpl();
-  DataNodeConnectionImpl(asio::io_service * io_service, const ::hadoop::hdfs::DatanodeInfoProto &dn_proto,
+  DataNodeConnectionImpl(std::shared_ptr<IoService> io_service, const ::hadoop::hdfs::DatanodeInfoProto &dn_proto,
                           const hadoop::common::TokenProto *token,
                           LibhdfsEvents *event_handlers);
 
@@ -72,24 +73,11 @@ public:
 
   void Cancel() override;
 
-  void async_read_some(const MutableBuffers &buf,
-                         std::function<void (const asio::error_code & error, std::size_t bytes_transferred) > handler)
-                       override {
-    event_handlers_->call("DN_read_req", "", "", buf.end() - buf.begin());
+  void async_read_some(const MutableBuffer &buf,
+                       std::function<void (const asio::error_code & error, std::size_t bytes_transferred) > handler) override;
 
-
-    mutex_guard state_lock(state_lock_);
-    conn_->async_read_some(buf, handler);
-  };
-
-  void async_write_some(const ConstBuffers &buf,
-                          std::function<void (const asio::error_code & error, std::size_t bytes_transferred) > handler)
-                        override {
-    event_handlers_->call("DN_write_req", "", "", buf.end() - buf.begin());
-
-    mutex_guard state_lock(state_lock_);
-    conn_->async_write_some(buf, handler);
-  }
+  void async_write_some(const ConstBuffer &buf,
+                        std::function<void (const asio::error_code & error, std::size_t bytes_transferred) > handler) override;
 };
 
 }

+ 9 - 9
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filehandle.cc

@@ -36,10 +36,10 @@ FileHandle::~FileHandle() {}
 
 FileHandleImpl::FileHandleImpl(const std::string & cluster_name,
                                const std::string & path,
-                               ::asio::io_service *io_service, const std::string &client_name,
-                                 const std::shared_ptr<const struct FileInfo> file_info,
-                                 std::shared_ptr<BadDataNodeTracker> bad_data_nodes,
-                                 std::shared_ptr<LibhdfsEvents> event_handlers)
+                               std::shared_ptr<IoService> io_service, const std::string &client_name,
+                               const std::shared_ptr<const struct FileInfo> file_info,
+                               std::shared_ptr<BadDataNodeTracker> bad_data_nodes,
+                               std::shared_ptr<LibhdfsEvents> event_handlers)
     : cluster_name_(cluster_name), path_(path), io_service_(io_service), client_name_(client_name), file_info_(file_info),
       bad_node_tracker_(bad_data_nodes), offset_(0), cancel_state_(CancelTracker::New()), event_handlers_(event_handlers), bytes_read_(0) {
   LOG_TRACE(kFileHandle, << "FileHandleImpl::FileHandleImpl("
@@ -167,7 +167,7 @@ bool FileHandleImpl::CheckSeekBounds(ssize_t desired_position) {
  * on the FileHandle
  */
 void FileHandleImpl::AsyncPreadSome(
-    size_t offset, const MutableBuffers &buffers,
+    size_t offset, const MutableBuffer &buffer,
     std::shared_ptr<NodeExclusionRule> excluded_nodes,
     const std::function<void(const Status &, const std::string &, size_t)> handler) {
   using ::hadoop::hdfs::DatanodeInfoProto;
@@ -233,7 +233,7 @@ void FileHandleImpl::AsyncPreadSome(
 
   uint64_t offset_within_block = offset - block->offset();
   uint64_t size_within_block = std::min<uint64_t>(
-      block->b().numbytes() - offset_within_block, asio::buffer_size(buffers));
+      block->b().numbytes() - offset_within_block, asio::buffer_size(buffer));
 
   LOG_DEBUG(kFileHandle, << "FileHandleImpl::AsyncPreadSome("
             << FMT_THIS_ADDR << "), ...) Datanode hostname=" << dnHostName << ", IP Address=" << dnIpAddr
@@ -268,7 +268,7 @@ void FileHandleImpl::AsyncPreadSome(
     handler(status, dn_id, transferred);
   };
 
-  auto connect_handler = [handler,event_handlers,cluster_name,path,read_handler,block,offset_within_block,size_within_block, buffers, reader, dn_id, client_name]
+  auto connect_handler = [handler,event_handlers,cluster_name,path,read_handler,block,offset_within_block,size_within_block, buffer, reader, dn_id, client_name]
           (Status status, std::shared_ptr<DataNodeConnection> dn) {
     (void)dn;
     event_response event_resp = event_handlers->call(FILE_DN_CONNECT_EVENT, cluster_name.c_str(), path.c_str(), 0);
@@ -281,7 +281,7 @@ void FileHandleImpl::AsyncPreadSome(
     if (status.ok()) {
       reader->AsyncReadBlock(
           client_name, *block, offset_within_block,
-          asio::buffer(buffers, size_within_block), read_handler);
+          asio::buffer(buffer, size_within_block), read_handler);
     } else {
       handler(status, dn_id, 0);
     }
@@ -307,7 +307,7 @@ std::shared_ptr<BlockReader> FileHandleImpl::CreateBlockReader(const BlockReader
 }
 
 std::shared_ptr<DataNodeConnection> FileHandleImpl::CreateDataNodeConnection(
-    ::asio::io_service * io_service,
+    std::shared_ptr<IoService> io_service,
     const ::hadoop::hdfs::DatanodeInfoProto & dn,
     const hadoop::common::TokenProto * token) {
   LOG_TRACE(kFileHandle, << "FileHandleImpl::CreateDataNodeConnection("

+ 5 - 7
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filehandle.h

@@ -18,7 +18,7 @@
 #ifndef LIBHDFSPP_LIB_FS_FILEHANDLE_H_
 #define LIBHDFSPP_LIB_FS_FILEHANDLE_H_
 
-#include "common/hdfs_ioservice.h"
+#include "hdfspp/ioservice.h"
 #include "common/async_stream.h"
 #include "common/cancel_tracker.h"
 #include "common/libhdfs_events_impl.h"
@@ -26,12 +26,10 @@
 #include "reader/fileinfo.h"
 #include "reader/readergroup.h"
 
-#include "asio.hpp"
 #include "bad_datanode_tracker.h"
 #include "ClientNamenodeProtocol.pb.h"
 
 #include <mutex>
-#include <iostream>
 
 namespace hdfs {
 
@@ -53,7 +51,7 @@ public:
   MEMCHECKED_CLASS(FileHandleImpl)
   FileHandleImpl(const std::string & cluster_name,
                  const std::string & path,
-                 ::asio::io_service *io_service, const std::string &client_name,
+                 std::shared_ptr<IoService> io_service, const std::string &client_name,
                   const std::shared_ptr<const struct FileInfo> file_info,
                   std::shared_ptr<BadDataNodeTracker> bad_data_nodes,
                   std::shared_ptr<LibhdfsEvents> event_handlers);
@@ -93,7 +91,7 @@ public:
    * If trying to begin a read past the EOF, status will be Status::InvalidOffset.
    *
    */
-  void AsyncPreadSome(size_t offset, const MutableBuffers &buffers,
+  void AsyncPreadSome(size_t offset, const MutableBuffer &buffer,
                       std::shared_ptr<NodeExclusionRule> excluded_nodes,
                       const std::function<void(const Status &status,
                       const std::string &dn_id, size_t bytes_read)> handler);
@@ -124,13 +122,13 @@ protected:
                                                          std::shared_ptr<DataNodeConnection> dn,
                                                          std::shared_ptr<hdfs::LibhdfsEvents> event_handlers);
   virtual std::shared_ptr<DataNodeConnection> CreateDataNodeConnection(
-      ::asio::io_service *io_service,
+      std::shared_ptr<IoService> io_service,
       const ::hadoop::hdfs::DatanodeInfoProto & dn,
       const hadoop::common::TokenProto * token);
 private:
   const std::string cluster_name_;
   const std::string path_;
-  ::asio::io_service * const io_service_;
+  std::shared_ptr<IoService> io_service_;
   const std::string client_name_;
   const std::shared_ptr<const struct FileInfo> file_info_;
   std::shared_ptr<BadDataNodeTracker> bad_node_tracker_;

+ 58 - 9
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.cc

@@ -18,6 +18,7 @@
 
 #include "filesystem.h"
 
+#include "filehandle.h"
 #include "common/namenode_info.h"
 
 #include <functional>
@@ -104,6 +105,54 @@ FileSystem *FileSystem::New() {
  *                    FILESYSTEM IMPLEMENTATION
  ****************************************************************************/
 
+struct FileSystemImpl::FindSharedState {
+  //Name pattern (can have wild-cards) to find
+  const std::string name;
+  //Maximum depth to recurse after the end of path is reached.
+  //Can be set to 0 for pure path globbing and ignoring name pattern entirely.
+  const uint32_t maxdepth;
+  //Vector of all sub-directories from the path argument (each can have wild-cards)
+  std::vector<std::string> dirs;
+  //Callback from Find
+  const std::function<bool(const Status &, const std::vector<StatInfo> &, bool)> handler;
+  //outstanding_requests is incremented once for every GetListing call.
+  std::atomic<uint64_t> outstanding_requests;
+  //Boolean needed to abort all recursion on error or on user command
+  std::atomic<bool> aborted;
+  //Shared variables will need protection with a lock
+  std::mutex lock;
+  FindSharedState(const std::string path_, const std::string name_, const uint32_t maxdepth_,
+              const std::function<bool(const Status &, const std::vector<StatInfo> &, bool)> handler_,
+              uint64_t outstanding_recuests_, bool aborted_)
+      : name(name_),
+        maxdepth(maxdepth_),
+        handler(handler_),
+        outstanding_requests(outstanding_recuests_),
+        aborted(aborted_),
+        lock() {
+    //Constructing the list of sub-directories
+    std::stringstream ss(path_);
+    if(path_.back() != '/'){
+      ss << "/";
+    }
+    for (std::string token; std::getline(ss, token, '/'); ) {
+      dirs.push_back(token);
+    }
+  }
+};
+
+struct FileSystemImpl::FindOperationalState {
+  const std::string path;
+  const uint32_t depth;
+  const bool search_path;
+  FindOperationalState(const std::string path_, const uint32_t depth_, const bool search_path_)
+      : path(path_),
+        depth(depth_),
+        search_path(search_path_) {
+  }
+};
+
+
 const std::string get_effective_user_name(const std::string &user_name) {
   if (!user_name.empty())
     return user_name;
@@ -134,10 +183,10 @@ const std::string get_effective_user_name(const std::string &user_name) {
 }
 
 FileSystemImpl::FileSystemImpl(IoService *&io_service, const std::string &user_name, const Options &options) :
-     io_service_(static_cast<IoServiceImpl *>(io_service)), options_(options),
+     io_service_(io_service), options_(options),
      client_name_(GetRandomClientName()),
      nn_(
-       &io_service_->io_service(), options, client_name_,
+       io_service_, options, client_name_,
        get_effective_user_name(user_name), kNamenodeProtocol,
        kNamenodeProtocolVersion
      ),
@@ -166,10 +215,10 @@ FileSystemImpl::FileSystemImpl(IoService *&io_service, const std::string &user_n
 }
 
 FileSystemImpl::FileSystemImpl(std::shared_ptr<IoService> io_service, const std::string& user_name, const Options &options) :
-     io_service_(std::static_pointer_cast<IoServiceImpl>(io_service)), options_(options),
+     io_service_(io_service), options_(options),
      client_name_(GetRandomClientName()),
      nn_(
-       &io_service_->io_service(), options, client_name_,
+       io_service_, options, client_name_,
        get_effective_user_name(user_name), kNamenodeProtocol,
        kNamenodeProtocolVersion
      ),
@@ -178,7 +227,7 @@ FileSystemImpl::FileSystemImpl(std::shared_ptr<IoService> io_service, const std:
 {
   LOG_DEBUG(kFileSystem, << "FileSystemImpl::FileSystemImpl("
                          << FMT_THIS_ADDR << ", shared IoService@" << io_service_.get() << ") called");
-  int worker_thread_count = io_service_->get_worker_thread_count();
+  int worker_thread_count = io_service_->GetWorkerThreadCount();
   if(worker_thread_count < 1) {
     LOG_WARN(kFileSystem, << "FileSystemImpl::FileSystemImpl IoService provided doesn't have any worker threads. "
                           << "It needs at least 1 worker to connect to an HDFS cluster.")
@@ -217,7 +266,7 @@ void FileSystemImpl::Connect(const std::string &server,
   auto name_service = options_.services.find(server);
   if(name_service != options_.services.end()) {
     cluster_name_ = name_service->first;
-    resolved_namenodes = BulkResolve(&io_service_->io_service(), name_service->second);
+    resolved_namenodes = BulkResolve(io_service_, name_service->second);
   } else {
     cluster_name_ = server + ":" + service;
 
@@ -230,7 +279,7 @@ void FileSystemImpl::Connect(const std::string &server,
       handler(Status::Error(("Invalid namenode " + cluster_name_ + " in config").c_str()), this);
     }
 
-    resolved_namenodes = BulkResolve(&io_service_->io_service(), {tmp_info});
+    resolved_namenodes = BulkResolve(io_service_, {tmp_info});
   }
 
   for(unsigned int i=0;i<resolved_namenodes.size();i++) {
@@ -282,7 +331,7 @@ int FileSystemImpl::WorkerThreadCount() {
   if(!io_service_) {
     return -1;
   } else {
-    return io_service_->get_worker_thread_count();
+    return io_service_->GetWorkerThreadCount();
   }
 }
 
@@ -339,7 +388,7 @@ void FileSystemImpl::Open(
         LOG_DEBUG(kFileSystem, << "Operation not allowed on standby datanode");
       }
     }
-    handler(stat, stat.ok() ? new FileHandleImpl(cluster_name_, path, &io_service_->io_service(), client_name_, file_info, bad_node_tracker_, event_handlers_)
+    handler(stat, stat.ok() ? new FileHandleImpl(cluster_name_, path, io_service_, client_name_, file_info, bad_node_tracker_, event_handlers_)
                             : nullptr);
   });
 }

+ 12 - 54
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h

@@ -18,19 +18,18 @@
 #ifndef LIBHDFSPP_LIB_FS_FILESYSTEM_H_
 #define LIBHDFSPP_LIB_FS_FILESYSTEM_H_
 
-#include "filehandle.h"
-#include "hdfspp/hdfspp.h"
+#include "namenode_operations.h"
 #include "fs/bad_datanode_tracker.h"
-#include "reader/block_reader.h"
+#include "hdfspp/hdfspp.h"
 #include "reader/fileinfo.h"
 
-#include "asio.hpp"
-
 #include <thread>
-#include "namenode_operations.h"
 
 namespace hdfs {
 
+class FileHandle;
+
+
 /*
  * FileSystem: The consumer's main point of interaction with the cluster as
  * a whole.
@@ -48,6 +47,7 @@ public:
   MEMCHECKED_CLASS(FileSystemImpl)
   typedef std::function<void(const Status &, FileSystem *)> ConnectCallback;
 
+  // Note: Longer term it'd be cleaner to take a rvalue reference to a shared_ptr to get ownership
   explicit FileSystemImpl(IoService *&io_service, const std::string& user_name, const Options &options);
   explicit FileSystemImpl(std::shared_ptr<IoService>, const std::string& user_name, const Options &options);
   ~FileSystemImpl() override;
@@ -215,7 +215,7 @@ private:
    *  A side effect of this is that requests may outlive the RpcEngine they
    *  reference.
    **/
-  std::shared_ptr<IoServiceImpl> io_service_;
+  std::shared_ptr<IoService> io_service_;
   const Options options_;
   const std::string client_name_;
   std::string cluster_name_;
@@ -234,53 +234,11 @@ private:
 
   void GetListingShim(const Status &stat, const std::vector<StatInfo> & stat_infos, bool has_more,
               std::string path, const std::function<bool(const Status &, const std::vector<StatInfo> &, bool)> &handler);
-
-  struct FindSharedState {
-    //Name pattern (can have wild-cards) to find
-    const std::string name;
-    //Maximum depth to recurse after the end of path is reached.
-    //Can be set to 0 for pure path globbing and ignoring name pattern entirely.
-    const uint32_t maxdepth;
-    //Vector of all sub-directories from the path argument (each can have wild-cards)
-    std::vector<std::string> dirs;
-    //Callback from Find
-    const std::function<bool(const Status &, const std::vector<StatInfo> &, bool)> handler;
-    //outstanding_requests is incremented once for every GetListing call.
-    std::atomic<uint64_t> outstanding_requests;
-    //Boolean needed to abort all recursion on error or on user command
-    std::atomic<bool> aborted;
-    //Shared variables will need protection with a lock
-    std::mutex lock;
-    FindSharedState(const std::string path_, const std::string name_, const uint32_t maxdepth_,
-                const std::function<bool(const Status &, const std::vector<StatInfo> &, bool)> handler_,
-                uint64_t outstanding_recuests_, bool aborted_)
-        : name(name_),
-          maxdepth(maxdepth_),
-          handler(handler_),
-          outstanding_requests(outstanding_recuests_),
-          aborted(aborted_),
-          lock() {
-      //Constructing the list of sub-directories
-      std::stringstream ss(path_);
-      if(path_.back() != '/'){
-        ss << "/";
-      }
-      for (std::string token; std::getline(ss, token, '/'); ) {
-        dirs.push_back(token);
-      }
-    }
-  };
-
-  struct FindOperationalState {
-    const std::string path;
-    const uint32_t depth;
-    const bool search_path;
-    FindOperationalState(const std::string path_, const uint32_t depth_, const bool search_path_)
-        : path(path_),
-          depth(depth_),
-          search_path(search_path_) {
-    }
-  };
+  /**
+   * Helper struct to store state for recursive find
+   */
+  struct FindSharedState;
+  struct FindOperationalState;
 
   void FindShim(const Status &stat, const std::vector<StatInfo> & stat_infos,
                 bool directory_has_more, std::shared_ptr<FindOperationalState> current_state, std::shared_ptr<FindSharedState> shared_state);

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/namenode_operations.h

@@ -42,7 +42,7 @@ namespace hdfs {
 class NameNodeOperations {
 public:
   MEMCHECKED_CLASS(NameNodeOperations)
-  NameNodeOperations(::asio::io_service *io_service, const Options &options,
+  NameNodeOperations(std::shared_ptr<IoService> io_service, const Options &options,
             const std::string &client_name, const std::string &user_name,
             const char *protocol_name, int protocol_version) :
   io_service_(io_service),
@@ -119,7 +119,7 @@ private:
   static void DirectoryListingProtoToStatInfo(std::shared_ptr<std::vector<StatInfo>> stat_infos, const ::hadoop::hdfs::DirectoryListingProto & dl);
   static void GetFsStatsResponseProtoToFsInfo(hdfs::FsInfo & fs_info, const std::shared_ptr<::hadoop::hdfs::GetFsStatsResponseProto> & fs);
 
-  ::asio::io_service * io_service_;
+  std::shared_ptr<IoService> io_service_;
 
   // This is the only permanent owner of the RpcEngine, however the RPC layer
   // needs to reference count it prevent races during FileSystem destruction.

+ 9 - 9
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/block_reader.cc

@@ -431,7 +431,7 @@ private:
   std::shared_ptr<DataNodeConnection> shared_conn_;
 };
 
-void BlockReaderImpl::AsyncReadPacket(const MutableBuffers &buffers,
+void BlockReaderImpl::AsyncReadPacket(const MutableBuffer &buffer,
     const std::function<void(const Status &, size_t bytes_transferred)> &handler)
 {
   assert(state_ != kOpen && "Not connected");
@@ -450,7 +450,7 @@ void BlockReaderImpl::AsyncReadPacket(const MutableBuffers &buffers,
       .Push(new ReadChecksum(this))
       .Push(new ReadPadding(this))
       .Push(new ReadData(
-          this, m->state().bytes_transferred, buffers))
+          this, m->state().bytes_transferred, buffer))
       .Push(new AckRead(this));
 
   auto self = this->shared_from_this();
@@ -460,14 +460,14 @@ void BlockReaderImpl::AsyncReadPacket(const MutableBuffers &buffers,
 }
 
 
-size_t BlockReaderImpl::ReadPacket(const MutableBuffers &buffers, Status *status)
+size_t BlockReaderImpl::ReadPacket(const MutableBuffer &buffer, Status *status)
 {
   LOG_TRACE(kBlockReader, << "BlockReaderImpl::ReadPacket called");
 
   size_t transferred = 0;
   auto done = std::make_shared<std::promise<void>>();
   auto future = done->get_future();
-  AsyncReadPacket(buffers,
+  AsyncReadPacket(buffer,
                   [status, &transferred, done](const Status &stat, size_t t) {
                     *status = stat;
                     transferred = t;
@@ -504,7 +504,7 @@ private:
 
 struct BlockReaderImpl::ReadBlockContinuation : continuation::Continuation
 {
-  ReadBlockContinuation(BlockReader *reader, MutableBuffers buffer, size_t *transferred)
+  ReadBlockContinuation(BlockReader *reader, MutableBuffer buffer, size_t *transferred)
       : reader_(reader), buffer_(buffer), buffer_size_(asio::buffer_size(buffer)), transferred_(transferred) {}
 
   virtual void Run(const Next &next) override {
@@ -517,7 +517,7 @@ struct BlockReaderImpl::ReadBlockContinuation : continuation::Continuation
 
 private:
   BlockReader *reader_;
-  const MutableBuffers buffer_;
+  const MutableBuffer buffer_;
   const size_t buffer_size_;
   size_t *transferred_;
   std::function<void(const Status &)> next_;
@@ -542,7 +542,7 @@ void BlockReaderImpl::AsyncReadBlock(
     const std::string & client_name,
     const hadoop::hdfs::LocatedBlockProto &block,
     size_t offset,
-    const MutableBuffers &buffers,
+    const MutableBuffer &buffer,
     const std::function<void(const Status &, size_t)> handler)
 {
   LOG_TRACE(kBlockReader, << "BlockReaderImpl::AsyncReadBlock("
@@ -551,10 +551,10 @@ void BlockReaderImpl::AsyncReadBlock(
   auto m = continuation::Pipeline<size_t>::Create(cancel_state_);
   size_t * bytesTransferred = &m->state();
 
-  size_t size = asio::buffer_size(buffers);
+  size_t size = asio::buffer_size(buffer);
 
   m->Push(new RequestBlockContinuation(this, client_name, &block.b(), size, offset))
-    .Push(new ReadBlockContinuation(this, buffers, bytesTransferred));
+    .Push(new ReadBlockContinuation(this, buffer, bytesTransferred));
 
   m->Run([handler] (const Status &status, const size_t totalBytesTransferred) {
     handler(status, totalBytesTransferred);

+ 5 - 5
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/block_reader.h

@@ -72,11 +72,11 @@ public:
   virtual void AsyncReadBlock(
     const std::string & client_name,
     const hadoop::hdfs::LocatedBlockProto &block, size_t offset,
-    const MutableBuffers &buffers,
+    const MutableBuffer &buffer,
     const std::function<void(const Status &, size_t)> handler) = 0;
 
   virtual void AsyncReadPacket(
-    const MutableBuffers &buffers,
+    const MutableBuffer &buffer,
     const std::function<void(const Status &, size_t bytes_transferred)> &handler) = 0;
 
   virtual void AsyncRequestBlock(
@@ -98,7 +98,7 @@ public:
         chunk_padding_bytes_(0), cancel_state_(cancel_state), event_handlers_(event_handlers.get()) {}
 
   virtual void AsyncReadPacket(
-    const MutableBuffers &buffers,
+    const MutableBuffer &buffer,
     const std::function<void(const Status &, size_t bytes_transferred)> &handler) override;
 
   virtual void AsyncRequestBlock(
@@ -111,12 +111,12 @@ public:
   virtual void AsyncReadBlock(
     const std::string & client_name,
     const hadoop::hdfs::LocatedBlockProto &block, size_t offset,
-    const MutableBuffers &buffers,
+    const MutableBuffer &buffer,
     const std::function<void(const Status &, size_t)> handler) override;
 
   virtual void CancelOperation() override;
 
-  size_t ReadPacket(const MutableBuffers &buffers, Status *status);
+  size_t ReadPacket(const MutableBuffer &buffer, Status *status);
 
   Status RequestBlock(
     const std::string &client_name,

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/datatransfer.h

@@ -44,13 +44,13 @@ public:
 
   template <class Handler> void Handshake(const Handler &next);
 
-  void async_read_some(const MutableBuffers &buf,
+  void async_read_some(const MutableBuffer &buf,
           std::function<void (const asio::error_code & error,
                                  std::size_t bytes_transferred) > handler) override {
     stream_->async_read_some(buf, handler);
   }
 
-  void async_write_some(const ConstBuffers &buf,
+  void async_write_some(const ConstBuffer &buf,
             std::function<void (const asio::error_code & error,
                                  std::size_t bytes_transferred) > handler) override {
     stream_->async_write_some(buf, handler);

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/rpc/namenode_tracker.cc

@@ -35,7 +35,7 @@ static std::string format_endpoints(const std::vector<::asio::ip::tcp::endpoint>
 }
 
 HANamenodeTracker::HANamenodeTracker(const std::vector<ResolvedNamenodeInfo> &servers,
-                                     ::asio::io_service *ioservice,
+                                     std::shared_ptr<IoService> ioservice,
                                      std::shared_ptr<LibhdfsEvents> event_handlers)
                   : enabled_(false), resolved_(false),
                     ioservice_(ioservice), event_handlers_(event_handlers)

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/rpc/namenode_tracker.h

@@ -40,7 +40,7 @@ namespace hdfs {
 class HANamenodeTracker {
  public:
   HANamenodeTracker(const std::vector<ResolvedNamenodeInfo> &servers,
-                    ::asio::io_service *ioservice,
+                    std::shared_ptr<IoService> ioservice,
                     std::shared_ptr<LibhdfsEvents> event_handlers_);
 
   virtual ~HANamenodeTracker();
@@ -66,7 +66,7 @@ class HANamenodeTracker {
   bool resolved_;
 
   // Keep service in case a second round of DNS lookup is required
-  ::asio::io_service *ioservice_;
+  std::shared_ptr<IoService> ioservice_;
 
   // Event handlers, for now this is the simplest place to catch all failover events
   // and push info out to client application.  Possibly move into RPCEngine.

+ 3 - 2
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/rpc/request.cc

@@ -20,6 +20,7 @@
 #include "request.h"
 #include "rpc_engine.h"
 #include "sasl_protocol.h"
+#include "hdfspp/ioservice.h"
 
 #include "RpcHeader.pb.h"
 #include "ProtobufRpcEngine.pb.h"
@@ -118,7 +119,7 @@ Request::Request(std::shared_ptr<LockFreeRpcEngine> engine, const std::string &m
     : engine_(engine),
       method_name_(method_name),
       call_id_(call_id),
-      timer_(engine->io_service()),
+      timer_(engine->io_service()->GetRaw()),
       handler_(std::move(handler)),
       retry_count_(engine->retry_policy() ? 0 : kNoRetry),
       failover_count_(0)
@@ -129,7 +130,7 @@ Request::Request(std::shared_ptr<LockFreeRpcEngine> engine, const std::string &m
 Request::Request(std::shared_ptr<LockFreeRpcEngine> engine, Handler &&handler)
     : engine_(engine),
       call_id_(-1/*Handshake ID*/),
-      timer_(engine->io_service()),
+      timer_(engine->io_service()->GetRaw()),
       handler_(std::move(handler)),
       retry_count_(engine->retry_policy() ? 0 : kNoRetry),
       failover_count_(0) {

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/rpc/rpc_connection.h

@@ -83,7 +83,7 @@ class RpcConnection : public std::enable_shared_from_this<RpcConnection> {
   void SetAuthInfo(const AuthInfo& auth_info);
 
   std::weak_ptr<LockFreeRpcEngine> engine() { return engine_; }
-  ::asio::io_service *GetIoService();
+  std::shared_ptr<IoService> GetIoService();
 
  protected:
   struct Response {

+ 19 - 13
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/rpc/rpc_connection_impl.cc

@@ -70,27 +70,27 @@ RpcConnection::RpcConnection(std::shared_ptr<LockFreeRpcEngine> engine)
     : engine_(engine),
       connected_(kNotYetConnected) {}
 
-::asio::io_service *RpcConnection::GetIoService() {
+std::shared_ptr<IoService> RpcConnection::GetIoService() {
   std::shared_ptr<LockFreeRpcEngine> pinnedEngine = engine_.lock();
   if(!pinnedEngine) {
     LOG_ERROR(kRPC, << "RpcConnection@" << this << " attempted to access invalid RpcEngine");
     return nullptr;
   }
 
-  return &pinnedEngine->io_service();
+  return pinnedEngine->io_service();
 }
 
 void RpcConnection::StartReading() {
   auto shared_this = shared_from_this();
-  ::asio::io_service *service = GetIoService();
+  std::shared_ptr<IoService> service = GetIoService();
   if(!service) {
     LOG_ERROR(kRPC, << "RpcConnection@" << this << " attempted to access invalid IoService");
     return;
   }
 
-  service->post([shared_this, this] () {
-    OnRecvCompleted(::asio::error_code(), 0);
-  });
+  service->PostLambda(
+    [shared_this, this] () { OnRecvCompleted(::asio::error_code(), 0); }
+  );
 }
 
 void RpcConnection::HandshakeComplete(const Status &s) {
@@ -164,13 +164,14 @@ void RpcConnection::ContextComplete(const Status &s) {
 void RpcConnection::AsyncFlushPendingRequests() {
   std::shared_ptr<RpcConnection> shared_this = shared_from_this();
 
-  ::asio::io_service *service = GetIoService();
+  std::shared_ptr<IoService> service = GetIoService();
   if(!service) {
     LOG_ERROR(kRPC, << "RpcConnection@" << this << " attempted to access invalid IoService");
     return;
   }
 
-  service->post([shared_this, this]() {
+  std::function<void()> task = [shared_this, this]()
+  {
     std::lock_guard<std::mutex> state_lock(connection_state_lock_);
 
     LOG_TRACE(kRPC, << "RpcConnection::AsyncFlushPendingRequests called (connected=" << ToString(connected_) << ")");
@@ -178,7 +179,10 @@ void RpcConnection::AsyncFlushPendingRequests() {
     if (!outgoing_request_) {
       FlushPendingRequests();
     }
-  });
+  };
+
+  service->PostTask(task);
+
 }
 
 Status RpcConnection::HandleRpcResponse(std::shared_ptr<Response> response) {
@@ -228,15 +232,17 @@ Status RpcConnection::HandleRpcResponse(std::shared_ptr<Response> response) {
     return status;
   }
 
-  ::asio::io_service *service = GetIoService();
+  std::shared_ptr<IoService> service = GetIoService();
   if(!service) {
     LOG_ERROR(kRPC, << "RpcConnection@" << this << " attempted to access invalid IoService");
     return Status::Error("RpcConnection attempted to access invalid IoService");
   }
 
-  service->post([req, response, status]() {
-    req->OnResponseArrived(response->in.get(), status);  // Never call back while holding a lock
-  });
+  service->PostLambda(
+    [req, response, status]() {
+      req->OnResponseArrived(response->in.get(), status);  // Never call back while holding a lock
+    }
+  );
 
   return Status::OK();
 }

+ 5 - 4
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/rpc/rpc_connection_impl.h

@@ -26,6 +26,7 @@
 #include "common/logging.h"
 #include "common/util.h"
 #include "common/libhdfs_events_impl.h"
+#include "hdfspp/ioservice.h"
 
 #include <asio/connect.hpp>
 #include <asio/read.hpp>
@@ -76,8 +77,8 @@ template <class Socket>
 RpcConnectionImpl<Socket>::RpcConnectionImpl(std::shared_ptr<RpcEngine> engine)
     : RpcConnection(engine),
       options_(engine->options()),
-      socket_(engine->io_service()),
-      connect_timer_(engine->io_service())
+      socket_(engine->io_service()->GetRaw()),
+      connect_timer_(engine->io_service()->GetRaw())
 {
       LOG_TRACE(kRPC, << "RpcConnectionImpl::RpcConnectionImpl called &" << (void*)this);
 }
@@ -353,7 +354,7 @@ void RpcConnectionImpl<Socket>::FlushPendingRequests() {
                         OnSendCompleted(ec, size);
                       });
   } else {  // Nothing to send for this request, inform the handler immediately
-    ::asio::io_service *service = GetIoService();
+    std::shared_ptr<IoService> service = GetIoService();
     if(!service) {
       LOG_ERROR(kRPC, << "RpcConnectionImpl@" << this << " attempted to access null IoService");
       // No easy way to bail out of this context, but the only way to get here is when
@@ -361,7 +362,7 @@ void RpcConnectionImpl<Socket>::FlushPendingRequests() {
       return;
     }
 
-    service->post(
+    service->PostTask(
         // Never hold locks when calling a callback
         [req]() { req->OnResponseArrived(nullptr, Status::OK()); }
     );

+ 7 - 7
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/rpc/rpc_engine.cc

@@ -30,7 +30,7 @@ template <class T>
 using optional = std::experimental::optional<T>;
 
 
-RpcEngine::RpcEngine(::asio::io_service *io_service, const Options &options,
+RpcEngine::RpcEngine(std::shared_ptr<IoService> io_service, const Options &options,
                      const std::string &client_name, const std::string &user_name,
                      const char *protocol_name, int protocol_version)
     : io_service_(io_service),
@@ -40,7 +40,7 @@ RpcEngine::RpcEngine(::asio::io_service *io_service, const Options &options,
       protocol_name_(protocol_name),
       protocol_version_(protocol_version),
       call_id_(0),
-      retry_timer(*io_service),
+      retry_timer(io_service->GetRaw()),
       event_handlers_(std::make_shared<LibhdfsEvents>()),
       connect_canceled_(false)
 {
@@ -86,7 +86,7 @@ bool RpcEngine::CancelPendingConnect() {
 
 void RpcEngine::Shutdown() {
   LOG_DEBUG(kRPC, << "RpcEngine::Shutdown called");
-  io_service_->post([this]() {
+  io_service_->PostLambda([this]() {
     std::lock_guard<std::mutex> state_lock(engine_state_lock_);
     conn_.reset();
   });
@@ -154,7 +154,7 @@ void RpcEngine::AsyncRpc(
 
   // In case user-side code isn't checking the status of Connect before doing RPC
   if(connect_canceled_) {
-    io_service_->post(
+    io_service_->PostLambda(
         [handler](){ handler(Status::Canceled()); }
     );
     return;
@@ -190,7 +190,7 @@ void RpcEngine::AsyncRpcCommsError(
     std::vector<std::shared_ptr<Request>> pendingRequests) {
   LOG_ERROR(kRPC, << "RpcEngine::AsyncRpcCommsError called; status=\"" << status.ToString() << "\" conn=" << failedConnection.get() << " reqs=" << std::to_string(pendingRequests.size()));
 
-  io_service().post([this, status, failedConnection, pendingRequests]() {
+  io_service_->PostLambda([this, status, failedConnection, pendingRequests]() {
     RpcCommsError(status, failedConnection, pendingRequests);
   });
 }
@@ -238,7 +238,7 @@ void RpcEngine::RpcCommsError(
       //    on.  There might be a good argument for caching the first error
       //    rather than the last one, that gets messy
 
-      io_service().post([req, status]() {
+      io_service()->PostLambda([req, status]() {
         req->OnResponseArrived(nullptr, status);  // Never call back while holding a lock
       });
       it = pendingRequests.erase(it);
@@ -283,7 +283,7 @@ void RpcEngine::RpcCommsError(
 
           for(unsigned int i=0; i<pendingRequests.size(); i++) {
             std::shared_ptr<Request> sharedCurrentRequest = pendingRequests[i];
-            io_service().post([sharedCurrentRequest, badEndpointStatus]() {
+            io_service()->PostLambda([sharedCurrentRequest, badEndpointStatus]() {
               sharedCurrentRequest->OnResponseArrived(nullptr, badEndpointStatus);  // Never call back while holding a lock
             });
           }

+ 5 - 4
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/rpc/rpc_engine.h

@@ -60,6 +60,7 @@ class RpcConnection;
 class SaslProtocol;
 class RpcConnection;
 class Request;
+class IoService;
 
 /*
  * These methods of the RpcEngine will never acquire locks, and are safe for
@@ -83,7 +84,7 @@ public:
   virtual const std::string &user_name() = 0;
   virtual const std::string &protocol_name() = 0;
   virtual int protocol_version() = 0;
-  virtual ::asio::io_service &io_service() = 0;
+  virtual std::shared_ptr<IoService> io_service() const = 0;
   virtual const Options &options() = 0;
 };
 
@@ -107,7 +108,7 @@ class RpcEngine : public LockFreeRpcEngine, public std::enable_shared_from_this<
     kCallIdSasl = -33
   };
 
-  RpcEngine(::asio::io_service *io_service, const Options &options,
+  RpcEngine(std::shared_ptr<IoService> service, const Options &options,
             const std::string &client_name, const std::string &user_name,
             const char *protocol_name, int protocol_version);
 
@@ -145,7 +146,7 @@ class RpcEngine : public LockFreeRpcEngine, public std::enable_shared_from_this<
   const std::string &user_name() override { return auth_info_.getUser(); }
   const std::string &protocol_name() override { return protocol_name_; }
   int protocol_version() override { return protocol_version_; }
-  ::asio::io_service &io_service() override { return *io_service_; }
+  std::shared_ptr<IoService> io_service() const override { return io_service_; }
   const Options &options() override { return options_; }
   static std::string GetRandomClientName();
 
@@ -162,7 +163,7 @@ protected:
   std::vector<::asio::ip::tcp::endpoint> last_endpoints_;
 
 private:
-  ::asio::io_service * const io_service_;
+  mutable std::shared_ptr<IoService> io_service_;
   const Options options_;
   const std::string client_name_;
   const std::string client_id_;

+ 16 - 15
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/bad_datanode_test.cc

@@ -16,11 +16,12 @@
  * limitations under the License.
  */
 
-#include "fs/filesystem.h"
-#include "fs/bad_datanode_tracker.h"
 #include "common/libhdfs_events_impl.h"
-
 #include "common/util.h"
+#include "fs/filesystem.h"
+#include "fs/filehandle.h"
+#include "fs/bad_datanode_tracker.h"
+#include "reader/block_reader.h"
 
 #include <gmock/gmock.h>
 
@@ -54,7 +55,7 @@ public:
     const std::string & client_name,
     const hadoop::hdfs::LocatedBlockProto &block,
     size_t offset,
-    const MutableBuffers &buffers,
+    const MutableBuffer &buffer,
     const std::function<void(const Status &, size_t)> handler));
 
   virtual void CancelOperation() override {
@@ -67,14 +68,14 @@ class MockDNConnection : public DataNodeConnection, public std::enable_shared_fr
       handler(Status::OK(), shared_from_this());
     }
 
-  void async_read_some(const MutableBuffers &buf,
+  void async_read_some(const MutableBuffer &buf,
         std::function<void (const asio::error_code & error,
                                std::size_t bytes_transferred) > handler) override {
       (void)buf;
       handler(asio::error::fault, 0);
   }
 
-  void async_write_some(const ConstBuffers &buf,
+  void async_write_some(const ConstBuffer &buf,
             std::function<void (const asio::error_code & error,
                                  std::size_t bytes_transferred) > handler) override {
       (void)buf;
@@ -101,7 +102,7 @@ protected:
     return mock_reader_;
   }
   std::shared_ptr<DataNodeConnection> CreateDataNodeConnection(
-      ::asio::io_service *io_service,
+      std::shared_ptr<IoService> io_service,
       const ::hadoop::hdfs::DatanodeInfoProto & dn,
       const hadoop::common::TokenProto * token) override {
     (void) io_service; (void) dn; (void) token;
@@ -130,12 +131,12 @@ TEST(BadDataNodeTest, TestNoNodes) {
   char buf[4096] = {
       0,
   };
-  IoServiceImpl io_service;
+  std::shared_ptr<IoService> io_service = IoService::MakeShared();
   auto bad_node_tracker = std::make_shared<BadDataNodeTracker>();
   auto monitors = std::make_shared<LibhdfsEvents>();
   bad_node_tracker->AddBadNode("foo");
 
-  PartialMockFileHandle is("cluster", "file", &io_service.io_service(), GetRandomClientName(), file_info, bad_node_tracker, monitors);
+  PartialMockFileHandle is("cluster", "file", io_service, GetRandomClientName(), file_info, bad_node_tracker, monitors);
   Status stat;
   size_t read = 0;
 
@@ -170,7 +171,7 @@ TEST(BadDataNodeTest, NNEventCallback) {
   char buf[4096] = {
       0,
   };
-  IoServiceImpl io_service;
+  std::shared_ptr<IoService> io_service = IoService::MakeShared();
   auto tracker = std::make_shared<BadDataNodeTracker>();
 
 
@@ -191,7 +192,7 @@ TEST(BadDataNodeTest, NNEventCallback) {
 
     return event_response::make_ok();
   });
-  PartialMockFileHandle is("cluster", "file", &io_service.io_service(), GetRandomClientName(),  file_info, tracker, monitors);
+  PartialMockFileHandle is("cluster", "file", io_service, GetRandomClientName(),  file_info, tracker, monitors);
   Status stat;
   size_t read = 0;
 
@@ -234,10 +235,10 @@ TEST(BadDataNodeTest, RecoverableError) {
   char buf[4096] = {
       0,
   };
-  IoServiceImpl io_service;
+  std::shared_ptr<IoService> io_service = IoService::MakeShared();
   auto tracker = std::make_shared<BadDataNodeTracker>();
   auto monitors = std::make_shared<LibhdfsEvents>();
-  PartialMockFileHandle is("cluster", "file", &io_service.io_service(), GetRandomClientName(),  file_info, tracker, monitors);
+  PartialMockFileHandle is("cluster", "file", io_service, GetRandomClientName(),  file_info, tracker, monitors);
   Status stat;
   size_t read = 0;
   EXPECT_CALL(*is.mock_reader_, AsyncReadBlock(_,_,_,_,_))
@@ -285,10 +286,10 @@ TEST(BadDataNodeTest, InternalError) {
   char buf[4096] = {
       0,
   };
-  IoServiceImpl io_service;
+  std::shared_ptr<IoService> io_service = IoService::MakeShared();
   auto tracker = std::make_shared<BadDataNodeTracker>();
   auto monitors = std::make_shared<LibhdfsEvents>();
-  PartialMockFileHandle is("cluster", "file", &io_service.io_service(), GetRandomClientName(),  file_info, tracker, monitors);
+  PartialMockFileHandle is("cluster", "file", io_service, GetRandomClientName(),  file_info, tracker, monitors);
   Status stat;
   size_t read = 0;
   EXPECT_CALL(*is.mock_reader_, AsyncReadBlock(_,_,_,_,_))

+ 6 - 4
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/hdfs_ioservice_test.cc

@@ -16,13 +16,15 @@
  * limitations under the License.
  */
 
-#include "common/hdfs_ioservice.h"
+#include "hdfspp/ioservice.h"
 
 #include <future>
 #include <functional>
 #include <thread>
 #include <string>
 
+
+#include <google/protobuf/stubs/common.h>
 #include <gmock/gmock.h>
 
 using ::testing::_;
@@ -34,7 +36,7 @@ using namespace hdfs;
 // Make sure IoService spins up specified number of threads
 TEST(IoServiceTest, InitThreads) {
 #ifndef DISABLE_CONCURRENT_WORKERS
-  std::shared_ptr<IoServiceImpl> service = std::static_pointer_cast<IoServiceImpl>(IoService::MakeShared());
+  std::shared_ptr<IoService> service = IoService::MakeShared();
   EXPECT_NE(service, nullptr);
 
   unsigned int thread_count = 4;
@@ -50,7 +52,7 @@ TEST(IoServiceTest, InitThreads) {
 // Make sure IoService defaults to logical thread count
 TEST(IoServiceTest, InitDefaultThreads) {
 #ifndef DISABLE_CONCURRENT_WORKERS
-  std::shared_ptr<IoServiceImpl> service = std::static_pointer_cast<IoServiceImpl>(IoService::MakeShared());
+  std::shared_ptr<IoService> service = IoService::MakeShared();
   EXPECT_NE(service, nullptr);
 
   unsigned int thread_count = std::thread::hardware_concurrency();
@@ -66,7 +68,7 @@ TEST(IoServiceTest, InitDefaultThreads) {
 
 // Check IoService::PostTask
 TEST(IoServiceTest, SimplePost) {
-  std::shared_ptr<IoServiceImpl> service = std::static_pointer_cast<IoServiceImpl>(IoService::MakeShared());
+  std::shared_ptr<IoService> service = IoService::MakeShared();
   EXPECT_NE(service, nullptr);
 
   unsigned int thread_count = std::thread::hardware_concurrency();

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/mock_connection.h

@@ -49,7 +49,7 @@ public:
   virtual ~MockConnectionBase();
   typedef std::pair<asio::error_code, std::string> ProducerResult;
 
-  void async_read_some(const MutableBuffers &buf,
+  void async_read_some(const MutableBuffer &buf,
           std::function<void (const asio::error_code & error,
                                  std::size_t bytes_transferred) > handler) override {
     if (produced_.size() == 0) {
@@ -72,7 +72,7 @@ public:
     io_service_->post(std::bind(handler, asio::error_code(), len));
   }
 
-  void async_write_some(const ConstBuffers &buf,
+  void async_write_some(const ConstBuffer &buf,
             std::function<void (const asio::error_code & error,
                                  std::size_t bytes_transferred) > handler) override {
     // CompletionResult res = OnWrite(buf);

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/remote_block_reader_test.cc

@@ -69,14 +69,14 @@ public:
   /* event handler to trigger side effects */
   std::function<void(void)> OnRead;
 
-  void async_read_some(const MutableBuffers &buf,
+  void async_read_some(const MutableBuffer &buf,
         std::function<void (const asio::error_code & error,
                                std::size_t bytes_transferred) > handler) override {
       this->OnRead();
       this->MockConnectionBase::async_read_some(buf, handler);
   }
 
-  void async_write_some(const ConstBuffers &buf,
+  void async_write_some(const ConstBuffer &buf,
             std::function<void (const asio::error_code & error,
                                  std::size_t bytes_transferred) > handler) override {
     this->MockConnectionBase::async_write_some(buf, handler);

+ 58 - 54
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/rpc_engine_test.cc

@@ -16,6 +16,8 @@
  * limitations under the License.
  */
 
+#include "hdfspp/ioservice.h"
+
 #include "mock_connection.h"
 #include "test.pb.h"
 #include "RpcHeader.pb.h"
@@ -23,7 +25,6 @@
 #include "common/namenode_info.h"
 
 #include <google/protobuf/io/coded_stream.h>
-
 #include <gmock/gmock.h>
 
 using ::hadoop::common::RpcResponseHeaderProto;
@@ -104,9 +105,10 @@ static inline std::pair<error_code, string> RpcResponse(
 using namespace hdfs;
 
 TEST(RpcEngineTest, TestRoundTrip) {
-  ::asio::io_service io_service;
+
+  std::shared_ptr<IoService> io_service = IoService::MakeShared();
   Options options;
-  std::shared_ptr<RpcEngine> engine = std::make_shared<RpcEngine>(&io_service, options, "foo", "", "protocol", 1);
+  std::shared_ptr<RpcEngine> engine = std::make_shared<RpcEngine>(io_service, options, "foo", "", "protocol", 1);
   auto conn =
       std::make_shared<RpcConnectionImpl<MockRPCConnection> >(engine);
   conn->TEST_set_connected(true);
@@ -129,20 +131,20 @@ TEST(RpcEngineTest, TestRoundTrip) {
   EchoRequestProto req;
   req.set_message("foo");
   std::shared_ptr<EchoResponseProto> resp(new EchoResponseProto());
-  engine->AsyncRpc("test", &req, resp, [resp, &complete,&io_service](const Status &stat) {
+  engine->AsyncRpc("test", &req, resp, [resp, &complete,io_service](const Status &stat) {
     ASSERT_TRUE(stat.ok());
     ASSERT_EQ("foo", resp->message());
     complete = true;
-    io_service.stop();
+    io_service->Stop();
   });
-  io_service.run();
+  io_service->Run();
   ASSERT_TRUE(complete);
 }
 
 TEST(RpcEngineTest, TestConnectionResetAndFail) {
-  ::asio::io_service io_service;
+  std::shared_ptr<IoService> io_service = IoService::MakeShared();
   Options options;
-  std::shared_ptr<RpcEngine> engine = std::make_shared<RpcEngine>(&io_service, options, "foo", "", "protocol", 1);
+  std::shared_ptr<RpcEngine> engine = std::make_shared<RpcEngine>(io_service, options, "foo", "", "protocol", 1);
   auto conn =
       std::make_shared<RpcConnectionImpl<MockRPCConnection> >(engine);
   conn->TEST_set_connected(true);
@@ -164,23 +166,23 @@ TEST(RpcEngineTest, TestConnectionResetAndFail) {
   req.set_message("foo");
   std::shared_ptr<EchoResponseProto> resp(new EchoResponseProto());
 
-  engine->AsyncRpc("test", &req, resp, [&complete, &io_service](const Status &stat) {
+  engine->AsyncRpc("test", &req, resp, [&complete, io_service](const Status &stat) {
     complete = true;
-    io_service.stop();
+    io_service->Stop();
     ASSERT_FALSE(stat.ok());
   });
-  io_service.run();
+  io_service->Run();
   ASSERT_TRUE(complete);
 }
 
 
 TEST(RpcEngineTest, TestConnectionResetAndRecover) {
-  ::asio::io_service io_service;
+  std::shared_ptr<IoService> io_service = IoService::MakeShared();
   Options options;
   options.max_rpc_retries = 1;
   options.rpc_retry_delay_ms = 0;
   std::shared_ptr<SharedConnectionEngine> engine
-      = std::make_shared<SharedConnectionEngine>(&io_service, options, "foo", "", "protocol", 1);
+      = std::make_shared<SharedConnectionEngine>(io_service, options, "foo", "", "protocol", 1);
 
   // Normally determined during RpcEngine::Connect, but in this case options
   // provides enough info to determine policy here.
@@ -206,22 +208,22 @@ TEST(RpcEngineTest, TestConnectionResetAndRecover) {
   req.set_message("foo");
   std::shared_ptr<EchoResponseProto> resp(new EchoResponseProto());
 
-  engine->AsyncRpc("test", &req, resp, [&complete, &io_service](const Status &stat) {
+  engine->AsyncRpc("test", &req, resp, [&complete, io_service](const Status &stat) {
     complete = true;
-    io_service.stop();
+    io_service->Stop();
     ASSERT_TRUE(stat.ok());
   });
-  io_service.run();
+  io_service->Run();
   ASSERT_TRUE(complete);
 }
 
 TEST(RpcEngineTest, TestConnectionResetAndRecoverWithDelay) {
-  ::asio::io_service io_service;
+  std::shared_ptr<IoService> io_service = IoService::MakeShared();
   Options options;
   options.max_rpc_retries = 1;
   options.rpc_retry_delay_ms = 1;
   std::shared_ptr<SharedConnectionEngine> engine =
-      std::make_shared<SharedConnectionEngine>(&io_service, options, "foo", "", "protocol", 1);
+      std::make_shared<SharedConnectionEngine>(io_service, options, "foo", "", "protocol", 1);
 
   // Normally determined during RpcEngine::Connect, but in this case options
   // provides enough info to determine policy here.
@@ -246,17 +248,17 @@ TEST(RpcEngineTest, TestConnectionResetAndRecoverWithDelay) {
   req.set_message("foo");
   std::shared_ptr<EchoResponseProto> resp(new EchoResponseProto());
 
-  engine->AsyncRpc("test", &req, resp, [&complete, &io_service](const Status &stat) {
+  engine->AsyncRpc("test", &req, resp, [&complete, io_service](const Status &stat) {
     complete = true;
-    io_service.stop();
+    io_service->Stop();
     ASSERT_TRUE(stat.ok());
   });
 
-  ::asio::deadline_timer timer(io_service);
+  ::asio::deadline_timer timer(io_service->GetRaw());
   timer.expires_from_now(std::chrono::hours(100));
   timer.async_wait([](const asio::error_code & err){(void)err; ASSERT_FALSE("Timed out"); });
 
-  io_service.run();
+  io_service->Run();
   ASSERT_TRUE(complete);
 }
 
@@ -267,7 +269,7 @@ TEST(RpcEngineTest, TestConnectionFailure)
   SharedMockConnection::SetSharedConnectionData(producer);
 
   // Error and no retry
-  ::asio::io_service io_service;
+  std::shared_ptr<IoService> io_service = IoService::MakeShared();
 
   bool complete = false;
 
@@ -275,16 +277,16 @@ TEST(RpcEngineTest, TestConnectionFailure)
   options.max_rpc_retries = 0;
   options.rpc_retry_delay_ms = 0;
   std::shared_ptr<SharedConnectionEngine> engine
-      = std::make_shared<SharedConnectionEngine>(&io_service, options, "foo", "", "protocol", 1);
+      = std::make_shared<SharedConnectionEngine>(io_service, options, "foo", "", "protocol", 1);
   EXPECT_CALL(*producer, Produce())
       .WillOnce(Return(std::make_pair(make_error_code(::asio::error::connection_reset), "")));
 
-  engine->Connect("", make_endpoint(), [&complete, &io_service](const Status &stat) {
+  engine->Connect("", make_endpoint(), [&complete, io_service](const Status &stat) {
     complete = true;
-    io_service.stop();
+    io_service->Stop();
     ASSERT_FALSE(stat.ok());
   });
-  io_service.run();
+  io_service->Run();
   ASSERT_TRUE(complete);
 }
 
@@ -294,7 +296,7 @@ TEST(RpcEngineTest, TestConnectionFailureRetryAndFailure)
   producer->checkProducerForConnect = true;
   SharedMockConnection::SetSharedConnectionData(producer);
 
-  ::asio::io_service io_service;
+  std::shared_ptr<IoService> io_service = IoService::MakeShared();
 
   bool complete = false;
 
@@ -302,18 +304,18 @@ TEST(RpcEngineTest, TestConnectionFailureRetryAndFailure)
   options.max_rpc_retries = 2;
   options.rpc_retry_delay_ms = 0;
   std::shared_ptr<SharedConnectionEngine> engine =
-      std::make_shared<SharedConnectionEngine>(&io_service, options, "foo", "", "protocol", 1);
+      std::make_shared<SharedConnectionEngine>(io_service, options, "foo", "", "protocol", 1);
   EXPECT_CALL(*producer, Produce())
       .WillOnce(Return(std::make_pair(make_error_code(::asio::error::connection_reset), "")))
       .WillOnce(Return(std::make_pair(make_error_code(::asio::error::connection_reset), "")))
       .WillOnce(Return(std::make_pair(make_error_code(::asio::error::connection_reset), "")));
 
-  engine->Connect("", make_endpoint(), [&complete, &io_service](const Status &stat) {
+  engine->Connect("", make_endpoint(), [&complete, io_service](const Status &stat) {
     complete = true;
-    io_service.stop();
+    io_service->Stop();
     ASSERT_FALSE(stat.ok());
   });
-  io_service.run();
+  io_service->Run();
   ASSERT_TRUE(complete);
 }
 
@@ -323,7 +325,7 @@ TEST(RpcEngineTest, TestConnectionFailureAndRecover)
   producer->checkProducerForConnect = true;
   SharedMockConnection::SetSharedConnectionData(producer);
 
-  ::asio::io_service io_service;
+  std::shared_ptr<IoService> io_service = IoService::MakeShared();
 
   bool complete = false;
 
@@ -331,29 +333,30 @@ TEST(RpcEngineTest, TestConnectionFailureAndRecover)
   options.max_rpc_retries = 1;
   options.rpc_retry_delay_ms = 0;
   std::shared_ptr<SharedConnectionEngine> engine =
-      std::make_shared<SharedConnectionEngine>(&io_service, options, "foo", "", "protocol", 1);
+      std::make_shared<SharedConnectionEngine>(io_service, options, "foo", "", "protocol", 1);
   EXPECT_CALL(*producer, Produce())
       .WillOnce(Return(std::make_pair(make_error_code(::asio::error::connection_reset), "")))
       .WillOnce(Return(std::make_pair(::asio::error_code(), "")))
       .WillOnce(Return(std::make_pair(::asio::error::would_block, "")));
 
-  engine->Connect("", make_endpoint(), [&complete, &io_service](const Status &stat) {
+  engine->Connect("", make_endpoint(), [&complete, io_service](const Status &stat) {
     complete = true;
-    io_service.stop();
+    io_service->Stop();
     ASSERT_TRUE(stat.ok());
   });
-  io_service.run();
+  io_service->Run();
   ASSERT_TRUE(complete);
 }
 
 TEST(RpcEngineTest, TestEventCallbacks)
 {
-  ::asio::io_service io_service;
+  std::shared_ptr<IoService> io_service = IoService::MakeShared();
+
   Options options;
   options.max_rpc_retries = 99;
   options.rpc_retry_delay_ms = 0;
   std::shared_ptr<SharedConnectionEngine> engine =
-      std::make_shared<SharedConnectionEngine>(&io_service, options, "foo", "", "protocol", 1);
+      std::make_shared<SharedConnectionEngine>(io_service, options, "foo", "", "protocol", 1);
 
   // Normally determined during RpcEngine::Connect, but in this case options
   // provides enough info to determine policy here.
@@ -399,17 +402,18 @@ TEST(RpcEngineTest, TestEventCallbacks)
   std::shared_ptr<EchoResponseProto> resp(new EchoResponseProto());
 
   bool complete = false;
-  engine->AsyncRpc("test", &req, resp, [&complete, &io_service](const Status &stat) {
+  engine->AsyncRpc("test", &req, resp, [&complete, io_service](const Status &stat) {
     complete = true;
-    io_service.stop();
+    io_service->Stop();
     ASSERT_TRUE(stat.ok());
   });
 
   // If you're adding event hooks you'll most likely need to update this.
   // It's a brittle test but makes it hard to miss control flow changes in RPC retry.
-  for(const auto& m : callbacks)
+  for(const auto& m : callbacks) {
     std::cerr << m << std::endl;
-  io_service.run();
+  }
+  io_service->Run();
   ASSERT_TRUE(complete);
   ASSERT_EQ(9, callbacks.size());
   ASSERT_EQ(FS_NN_CONNECT_EVENT, callbacks[0]); // error
@@ -430,7 +434,7 @@ TEST(RpcEngineTest, TestConnectionFailureAndAsyncRecover)
   producer->checkProducerForConnect = true;
   SharedMockConnection::SetSharedConnectionData(producer);
 
-  ::asio::io_service io_service;
+  std::shared_ptr<IoService> io_service = IoService::MakeShared();
 
   bool complete = false;
 
@@ -438,31 +442,31 @@ TEST(RpcEngineTest, TestConnectionFailureAndAsyncRecover)
   options.max_rpc_retries = 1;
   options.rpc_retry_delay_ms = 1;
   std::shared_ptr<SharedConnectionEngine> engine =
-      std::make_shared<SharedConnectionEngine>(&io_service, options, "foo", "", "protocol", 1);
+      std::make_shared<SharedConnectionEngine>(io_service, options, "foo", "", "protocol", 1);
   EXPECT_CALL(*producer, Produce())
       .WillOnce(Return(std::make_pair(make_error_code(::asio::error::connection_reset), "")))
       .WillOnce(Return(std::make_pair(::asio::error_code(), "")))
       .WillOnce(Return(std::make_pair(::asio::error::would_block, "")));
 
-  engine->Connect("", make_endpoint(), [&complete, &io_service](const Status &stat) {
+  engine->Connect("", make_endpoint(), [&complete, io_service](const Status &stat) {
     complete = true;
-    io_service.stop();
+    io_service->Stop();
     ASSERT_TRUE(stat.ok());
   });
 
-  ::asio::deadline_timer timer(io_service);
+  ::asio::deadline_timer timer(io_service->GetRaw());
   timer.expires_from_now(std::chrono::hours(100));
   timer.async_wait([](const asio::error_code & err){(void)err; ASSERT_FALSE("Timed out"); });
 
-  io_service.run();
+  io_service->Run();
   ASSERT_TRUE(complete);
 }
 
 TEST(RpcEngineTest, TestTimeout) {
-  ::asio::io_service io_service;
+  std::shared_ptr<IoService> io_service = IoService::MakeShared();
   Options options;
   options.rpc_timeout = 1;
-  std::shared_ptr<RpcEngine> engine = std::make_shared<RpcEngine>(&io_service, options, "foo", "", "protocol", 1);
+  std::shared_ptr<RpcEngine> engine = std::make_shared<RpcEngine>(io_service, options, "foo", "", "protocol", 1);
   auto conn =
       std::make_shared<RpcConnectionImpl<MockRPCConnection> >(engine);
   conn->TEST_set_connected(true);
@@ -481,15 +485,15 @@ TEST(RpcEngineTest, TestTimeout) {
   std::shared_ptr<EchoResponseProto> resp(new EchoResponseProto());
   engine->AsyncRpc("test", &req, resp, [resp, &complete,&io_service](const Status &stat) {
     complete = true;
-    io_service.stop();
+    io_service->Stop();
     ASSERT_FALSE(stat.ok());
   });
 
-  ::asio::deadline_timer timer(io_service);
+  ::asio::deadline_timer timer(io_service->GetRaw());
   timer.expires_from_now(std::chrono::hours(100));
   timer.async_wait([](const asio::error_code & err){(void)err; ASSERT_FALSE("Timed out"); });
 
-  io_service.run();
+  io_service->Run();
   ASSERT_TRUE(complete);
 }
 

+ 5 - 0
hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/FederationNamespaceInfo.java

@@ -48,6 +48,11 @@ public class FederationNamespaceInfo extends RemoteLocationContext {
     return this.nameserviceId;
   }
 
+  @Override
+  public String getSrc() {
+    return null;
+  }
+
   /**
    * The HDFS cluster id for this namespace.
    *

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/MountTableResolver.java

@@ -388,7 +388,7 @@ public class MountTableResolver
     } else {
       // Not found, use default location
       RemoteLocation remoteLocation =
-          new RemoteLocation(defaultNameService, path);
+          new RemoteLocation(defaultNameService, path, path);
       List<RemoteLocation> locations =
           Collections.singletonList(remoteLocation);
       ret = new PathLocation(null, locations);
@@ -519,7 +519,7 @@ public class MountTableResolver
         newPath += Path.SEPARATOR;
       }
       newPath += remainingPath;
-      RemoteLocation remoteLocation = new RemoteLocation(nsId, newPath);
+      RemoteLocation remoteLocation = new RemoteLocation(nsId, newPath, path);
       locations.add(remoteLocation);
     }
     DestinationOrder order = entry.getDestOrder();

+ 23 - 12
hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/RemoteLocation.java

@@ -20,8 +20,8 @@ package org.apache.hadoop.hdfs.server.federation.resolver;
 import org.apache.hadoop.hdfs.server.federation.router.RemoteLocationContext;
 
 /**
- * A single in a remote namespace consisting of a nameservice ID
- * and a HDFS path.
+ * A location in a remote namespace consisting of a nameservice ID and a HDFS
+ * path (destination). It also contains the federated location (source).
  */
 public class RemoteLocation extends RemoteLocationContext {
 
@@ -30,16 +30,19 @@ public class RemoteLocation extends RemoteLocationContext {
   /** Identifier of the namenode in the namespace for this location. */
   private final String namenodeId;
   /** Path in the remote location. */
-  private final String path;
+  private final String dstPath;
+  /** Original path in federation. */
+  private final String srcPath;
 
   /**
    * Create a new remote location.
    *
-   * @param nsId
-   * @param pPath
+   * @param nsId Destination namespace.
+   * @param dPath Path in the destination namespace.
+   * @param sPath Path in the federated level.
    */
-  public RemoteLocation(String nsId, String pPath) {
-    this(nsId, null, pPath);
+  public RemoteLocation(String nsId, String dPath, String sPath) {
+    this(nsId, null, dPath, sPath);
   }
 
   /**
@@ -47,12 +50,15 @@ public class RemoteLocation extends RemoteLocationContext {
    * namespace.
    *
    * @param nsId Destination namespace.
-   * @param pPath Path in the destination namespace.
+   * @param nnId Destination namenode.
+   * @param dPath Path in the destination namespace.
+   * @param sPath Path in the federated level
    */
-  public RemoteLocation(String nsId, String nnId, String pPath) {
+  public RemoteLocation(String nsId, String nnId, String dPath, String sPath) {
     this.nameserviceId = nsId;
     this.namenodeId = nnId;
-    this.path = pPath;
+    this.dstPath = dPath;
+    this.srcPath = sPath;
   }
 
   @Override
@@ -66,11 +72,16 @@ public class RemoteLocation extends RemoteLocationContext {
 
   @Override
   public String getDest() {
-    return this.path;
+    return this.dstPath;
+  }
+
+  @Override
+  public String getSrc() {
+    return this.srcPath;
   }
 
   @Override
   public String toString() {
-    return getNameserviceId() + "->" + this.path;
+    return getNameserviceId() + "->" + this.dstPath;
   }
 }

+ 7 - 0
hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RemoteLocationContext.java

@@ -39,6 +39,13 @@ public abstract class RemoteLocationContext
    */
   public abstract String getDest();
 
+  /**
+   * Original source location.
+   *
+   * @return Source path.
+   */
+  public abstract String getSrc();
+
   @Override
   public int hashCode() {
     return new HashCodeBuilder(17, 31)

+ 65 - 2
hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcClient.java

@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.hdfs.server.federation.router;
 
+import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.lang.reflect.Constructor;
 import java.lang.reflect.InvocationTargetException;
@@ -62,6 +63,7 @@ import org.apache.hadoop.security.UserGroupInformation;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.annotations.VisibleForTesting;
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
 
 /**
@@ -611,7 +613,7 @@ public class RouterRpcClient {
     UserGroupInformation ugi = RouterRpcServer.getRemoteUser();
     List<? extends FederationNamenodeContext> nns =
         getNamenodesForNameservice(nsId);
-    RemoteLocationContext loc = new RemoteLocation(nsId, "/");
+    RemoteLocationContext loc = new RemoteLocation(nsId, "/", "/");
     Class<?> proto = method.getProtocol();
     Method m = method.getMethod();
     Object[] params = method.getParams(loc);
@@ -727,8 +729,12 @@ public class RouterRpcClient {
           firstResult = result;
         }
       } catch (IOException ioe) {
+        // Localize the exception
+
+        ioe = processException(ioe, loc);
+
         // Record it and move on
-        lastThrownException = (IOException) ioe;
+        lastThrownException =  ioe;
         if (firstThrownException == null) {
           firstThrownException = lastThrownException;
         }
@@ -756,6 +762,63 @@ public class RouterRpcClient {
     return ret;
   }
 
+  /**
+   * Exception messages might contain local subcluster paths. This method
+   * generates a new exception with the proper message.
+   * @param ioe Original IOException.
+   * @param loc Location we are processing.
+   * @return Exception processed for federation.
+   */
+  private IOException processException(
+      IOException ioe, RemoteLocationContext loc) {
+
+    if (ioe instanceof RemoteException) {
+      RemoteException re = (RemoteException)ioe;
+      String newMsg = processExceptionMsg(
+          re.getMessage(), loc.getDest(), loc.getSrc());
+      RemoteException newException =
+          new RemoteException(re.getClassName(), newMsg);
+      newException.setStackTrace(ioe.getStackTrace());
+      return newException;
+    }
+
+    if (ioe instanceof FileNotFoundException) {
+      String newMsg = processExceptionMsg(
+          ioe.getMessage(), loc.getDest(), loc.getSrc());
+      FileNotFoundException newException = new FileNotFoundException(newMsg);
+      newException.setStackTrace(ioe.getStackTrace());
+      return newException;
+    }
+
+    return ioe;
+  }
+
+  /**
+   * Process a subcluster message and make it federated.
+   * @param msg Original exception message.
+   * @param dst Path in federation.
+   * @param src Path in the subcluster.
+   * @return Message processed for federation.
+   */
+  @VisibleForTesting
+  static String processExceptionMsg(
+      final String msg, final String dst, final String src) {
+    if (dst.equals(src) || !dst.startsWith("/") || !src.startsWith("/")) {
+      return msg;
+    }
+
+    String newMsg = msg.replaceFirst(dst, src);
+    int minLen = Math.min(dst.length(), src.length());
+    for (int i = 0; newMsg.equals(msg) && i < minLen; i++) {
+      // Check if we can replace sub folders
+      String dst1 = dst.substring(0, dst.length() - 1 - i);
+      String src1 = src.substring(0, src.length() - 1 - i);
+      newMsg = msg.replaceFirst(dst1, src1);
+    }
+
+    return newMsg;
+  }
+
   /**
    * Checks if a result matches the required result class.
    *

+ 18 - 1
hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java

@@ -2328,7 +2328,24 @@ public class RouterRpcServer extends AbstractService
    */
   private Map<String, Long> getMountPointDates(String path) {
     Map<String, Long> ret = new TreeMap<>();
-    // TODO add when we have a Mount Table
+    if (subclusterResolver instanceof MountTableResolver) {
+      MountTableResolver mountTable = (MountTableResolver)subclusterResolver;
+      String srcPath;
+      try {
+        final List<String> children = subclusterResolver.getMountPoints(path);
+        for (String child : children) {
+          if (path.equals(Path.SEPARATOR)) {
+            srcPath = Path.SEPARATOR + child;
+          } else {
+            srcPath = path + Path.SEPARATOR + child;
+          }
+          MountTable entry = mountTable.getMountPoint(srcPath);
+          ret.put(child, entry.getDateModified());
+        }
+      } catch (IOException e) {
+        LOG.error("Cannot get mount point: {}", e.getMessage());
+      }
+    }
     return ret;
   }
 

+ 7 - 4
hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/impl/StateStoreFileImpl.java

@@ -26,9 +26,11 @@ import java.io.IOException;
 import java.io.InputStreamReader;
 import java.io.OutputStreamWriter;
 import java.nio.charset.StandardCharsets;
-import java.util.LinkedList;
+import java.util.ArrayList;
+import java.util.Collections;
 import java.util.List;
 
+import org.apache.commons.lang.ArrayUtils;
 import org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys;
 import org.apache.hadoop.hdfs.server.federation.store.records.BaseRecord;
 import org.slf4j.Logger;
@@ -134,15 +136,16 @@ public class StateStoreFileImpl extends StateStoreFileBaseImpl {
 
   @Override
   protected List<String> getChildren(String path) {
-    List<String> ret = new LinkedList<>();
     File dir = new File(path);
     File[] files = dir.listFiles();
-    if (files != null) {
+    if (ArrayUtils.isNotEmpty(files)) {
+      List<String> ret = new ArrayList<>(files.length);
       for (File file : files) {
         String filename = file.getName();
         ret.add(filename);
       }
+      return ret;
     }
-    return ret;
+    return Collections.emptyList();
   }
 }

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/MountTable.java

@@ -134,7 +134,7 @@ public abstract class MountTable extends BaseRecord {
     for (Entry<String, String> entry : destinations.entrySet()) {
       String nsId = entry.getKey();
       String path = normalizeFileSystemPath(entry.getValue());
-      RemoteLocation location = new RemoteLocation(nsId, path);
+      RemoteLocation location = new RemoteLocation(nsId, path, src);
       locations.add(location);
     }
 

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/impl/pb/MountTablePBImpl.java

@@ -102,7 +102,7 @@ public class MountTablePBImpl extends MountTable implements PBRecord {
     for (RemoteLocationProto dest : destList) {
       String nsId = dest.getNameserviceId();
       String path = dest.getPath();
-      RemoteLocation loc = new RemoteLocation(nsId, path);
+      RemoteLocation loc = new RemoteLocation(nsId, path, getSourcePath());
       ret.add(loc);
     }
     return ret;

+ 9 - 3
hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/MockResolver.java

@@ -80,7 +80,8 @@ public class MockResolver
       this.locations.put(mount, locationsList);
     }
 
-    final RemoteLocation remoteLocation = new RemoteLocation(nsId, location);
+    final RemoteLocation remoteLocation =
+        new RemoteLocation(nsId, location, mount);
     if (!locationsList.contains(remoteLocation)) {
       locationsList.add(remoteLocation);
     }
@@ -270,10 +271,15 @@ public class MockResolver
     for (String key : keys) {
       if (path.startsWith(key)) {
         for (RemoteLocation location : this.locations.get(key)) {
-          String finalPath = location.getDest() + path.substring(key.length());
+          String finalPath = location.getDest();
+          String extraPath = path.substring(key.length());
+          if (finalPath.endsWith("/") && extraPath.startsWith("/")) {
+            extraPath = extraPath.substring(1);
+          }
+          finalPath += extraPath;
           String nameservice = location.getNameserviceId();
           RemoteLocation remoteLocation =
-              new RemoteLocation(nameservice, finalPath);
+              new RemoteLocation(nameservice, finalPath, path);
           remoteLocations.add(remoteLocation);
         }
         break;

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterAdmin.java

@@ -251,7 +251,7 @@ public class TestRouterAdmin {
     // Verify starting condition
     MountTable entry = getMountTableEntry("/");
     assertEquals(
-        Collections.singletonList(new RemoteLocation("ns0", "/")),
+        Collections.singletonList(new RemoteLocation("ns0", "/", "/")),
         entry.getDestinations());
 
     // Edit the entry for /
@@ -264,7 +264,7 @@ public class TestRouterAdmin {
     // Verify edited condition
     entry = getMountTableEntry("/");
     assertEquals(
-        Collections.singletonList(new RemoteLocation("ns1", "/")),
+        Collections.singletonList(new RemoteLocation("ns1", "/", "/")),
         entry.getDestinations());
   }
 

+ 61 - 1
hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterMountTable.java

@@ -17,25 +17,33 @@
  */
 package org.apache.hadoop.hdfs.server.federation.router;
 
+import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
 import java.io.IOException;
 import java.util.Collections;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.TreeMap;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hdfs.server.federation.RouterConfigBuilder;
+import org.apache.hadoop.hdfs.protocol.ClientProtocol;
+import org.apache.hadoop.hdfs.protocol.DirectoryListing;
+import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.server.federation.MiniRouterDFSCluster.NamenodeContext;
 import org.apache.hadoop.hdfs.server.federation.MiniRouterDFSCluster.RouterContext;
+import org.apache.hadoop.hdfs.server.federation.RouterConfigBuilder;
 import org.apache.hadoop.hdfs.server.federation.StateStoreDFSCluster;
 import org.apache.hadoop.hdfs.server.federation.resolver.MountTableManager;
 import org.apache.hadoop.hdfs.server.federation.resolver.MountTableResolver;
 import org.apache.hadoop.hdfs.server.federation.store.protocol.AddMountTableEntryRequest;
 import org.apache.hadoop.hdfs.server.federation.store.protocol.AddMountTableEntryResponse;
 import org.apache.hadoop.hdfs.server.federation.store.records.MountTable;
+import org.apache.hadoop.util.Time;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -49,6 +57,7 @@ public class TestRouterMountTable {
   private static NamenodeContext nnContext;
   private static RouterContext routerContext;
   private static MountTableResolver mountTable;
+  private static ClientProtocol routerProtocol;
 
   @BeforeClass
   public static void globalSetUp() throws Exception {
@@ -69,6 +78,7 @@ public class TestRouterMountTable {
     nnContext = cluster.getRandomNamenode();
     routerContext = cluster.getRandomRouter();
     Router router = routerContext.getRouter();
+    routerProtocol = routerContext.getClient().getNamenode();
     mountTable = (MountTableResolver) router.getSubclusterResolver();
   }
 
@@ -140,4 +150,54 @@ public class TestRouterMountTable {
 
     return addResponse.getStatus();
   }
+
+  /**
+   * Verify that the file/dir listing contains correct date/time information.
+   */
+  @Test
+  public void testListFilesTime() throws Exception {
+    Long beforeCreatingTime = Time.now();
+    // Add mount table entry
+    MountTable addEntry = MountTable.newInstance(
+        "/testdir", Collections.singletonMap("ns0", "/testdir"));
+    assertTrue(addMountTable(addEntry));
+    addEntry = MountTable.newInstance(
+        "/testdir2", Collections.singletonMap("ns0", "/testdir2"));
+    assertTrue(addMountTable(addEntry));
+    addEntry = MountTable.newInstance(
+        "/testdir/subdir", Collections.singletonMap("ns0", "/testdir/subdir"));
+    assertTrue(addMountTable(addEntry));
+
+    // Create test dir in NN
+    final FileSystem nnFs = nnContext.getFileSystem();
+    assertTrue(nnFs.mkdirs(new Path("/newdir")));
+
+    Map<String, Long> pathModTime = new TreeMap<>();
+    for (String mount : mountTable.getMountPoints("/")) {
+      pathModTime.put(mount, mountTable.getMountPoint("/"+mount)
+          .getDateModified());
+    }
+    FileStatus[] iterator = nnFs.listStatus(new Path("/"));
+    for (FileStatus file : iterator) {
+      pathModTime.put(file.getPath().getName(), file.getModificationTime());
+    }
+    // Fetch listing
+    DirectoryListing listing =
+        routerProtocol.getListing("/", HdfsFileStatus.EMPTY_NAME, false);
+    Iterator<String> pathModTimeIterator = pathModTime.keySet().iterator();
+
+    // Match date/time for each path returned
+    for(HdfsFileStatus f : listing.getPartialListing()) {
+      String fileName = pathModTimeIterator.next();
+      String currentFile = f.getFullPath(new Path("/")).getName();
+      Long currentTime = f.getModificationTime();
+      Long expectedTime = pathModTime.get(currentFile);
+
+      assertEquals(currentFile, fileName);
+      assertTrue(currentTime > beforeCreatingTime);
+      assertEquals(currentTime, expectedTime);
+    }
+    // Verify the total number of results found/matched
+    assertEquals(pathModTime.size(), listing.getPartialListing().length);
+  }
 }

+ 48 - 0
hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterRpc.java

@@ -24,6 +24,7 @@ import static org.apache.hadoop.hdfs.server.federation.FederationTestUtils.delet
 import static org.apache.hadoop.hdfs.server.federation.FederationTestUtils.getFileStatus;
 import static org.apache.hadoop.hdfs.server.federation.FederationTestUtils.verifyFileExists;
 import static org.apache.hadoop.hdfs.server.federation.MiniRouterDFSCluster.TEST_STRING;
+import static org.apache.hadoop.test.GenericTestUtils.assertExceptionContains;
 import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
@@ -31,6 +32,7 @@ import static org.junit.Assert.assertNotEquals;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 import java.io.IOException;
 import java.lang.reflect.Method;
@@ -1022,6 +1024,52 @@ public class TestRouterRpc {
     assertEquals(nnSuccess, routerSuccess);
   }
 
+  @Test
+  public void testProxyExceptionMessages() throws IOException {
+
+    // Install a mount point to a different path to check
+    MockResolver resolver =
+        (MockResolver)router.getRouter().getSubclusterResolver();
+    String ns0 = cluster.getNameservices().get(0);
+    resolver.addLocation("/mnt", ns0, "/");
+
+    try {
+      FsPermission permission = new FsPermission("777");
+      routerProtocol.mkdirs("/mnt/folder0/folder1", permission, false);
+      fail("mkdirs for non-existing parent folder should have failed");
+    } catch (IOException ioe) {
+      assertExceptionContains("/mnt/folder0", ioe,
+          "Wrong path in exception for mkdirs");
+    }
+
+    try {
+      FsPermission permission = new FsPermission("777");
+      routerProtocol.setPermission("/mnt/testfile.txt", permission);
+      fail("setPermission for non-existing file should have failed");
+    } catch (IOException ioe) {
+      assertExceptionContains("/mnt/testfile.txt", ioe,
+          "Wrong path in exception for setPermission");
+    }
+
+    try {
+      FsPermission permission = new FsPermission("777");
+      routerProtocol.mkdirs("/mnt/folder0/folder1", permission, false);
+      routerProtocol.delete("/mnt/folder0", false);
+      fail("delete for non-existing file should have failed");
+    } catch (IOException ioe) {
+      assertExceptionContains("/mnt/folder0", ioe,
+          "Wrong path in exception for delete");
+    }
+
+    resolver.cleanRegistrations();
+
+    // Check corner cases
+    assertEquals(
+        "Parent directory doesn't exist: /ns1/a/a/b",
+        RouterRpcClient.processExceptionMsg(
+            "Parent directory doesn't exist: /a/a/b", "/a", "/ns1/a"));
+  }
+
   @Test
   public void testErasureCoding() throws IOException {
 

Kaikkia tiedostoja ei voida näyttää, sillä liian monta tiedostoa muuttui tässä diffissä