Selaa lähdekoodia

Merge branch 'trunk' into HDFS-7240

 Conflicts:
	hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestCommonConfigurationFields.java
	hadoop-hdfs-project/hadoop-hdfs/dev-support/findbugsExcludeFile.xml
	hadoop-hdfs-project/hadoop-hdfs/pom.xml
Anu Engineer 7 vuotta sitten
vanhempi
commit
f46756559a
100 muutettua tiedostoa jossa 10786 lisäystä ja 432 poistoa
  1. 19 20
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/KMSClientProvider.java
  2. 70 11
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/LoadBalancingKMSClientProvider.java
  3. 29 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeysPublic.java
  4. 26 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileStatus.java
  5. 10 4
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/protocolPB/PBHelper.java
  6. 2 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/InodeTree.java
  7. 6 12
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/serializer/SerializationFactory.java
  8. 5 3
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/FairCallQueue.java
  9. 20 7
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/http/CrossOriginFilter.java
  10. 4 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticationHandler.java
  11. 1 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticator.java
  12. 13 2
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenManager.java
  13. 2 1
      hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/jni_xor_decoder.c
  14. 4 3
      hadoop-common-project/hadoop-common/src/main/proto/FSProtos.proto
  15. 0 1
      hadoop-common-project/hadoop-common/src/main/resources/META-INF/services/org.apache.hadoop.fs.FileSystem
  16. 43 3
      hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
  17. 1 1
      hadoop-common-project/hadoop-common/src/site/markdown/HttpAuthentication.md
  18. 646 0
      hadoop-common-project/hadoop-common/src/site/markdown/release/3.0.0-beta1/CHANGES.3.0.0-beta1.md
  19. 377 0
      hadoop-common-project/hadoop-common/src/site/markdown/release/3.0.0-beta1/RELEASENOTES.3.0.0-beta1.md
  20. 4 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestCommonConfigurationFields.java
  21. 308 5
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/key/kms/TestLoadBalancingKMSClientProvider.java
  22. 13 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/serializer/TestSerializationFactory.java
  23. 79 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/http/TestCrossOriginFilter.java
  24. 52 20
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/delegation/web/TestDelegationTokenAuthenticationHandlerWithMocks.java
  25. 3 3
      hadoop-common-project/hadoop-kms/src/test/java/org/apache/hadoop/crypto/key/kms/server/TestKMS.java
  26. 41 10
      hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/FileHandle.java
  27. 2 2
      hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/request/WRITE3Request.java
  28. 5 2
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java
  29. 11 1
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsFileStatus.java
  30. 5 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
  31. 1 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/hdfs.proto
  32. 13 4
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSFileSystem.java
  33. 7 5
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs/jni_helper.c
  34. 56 25
      hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/mount/RpcProgramMountd.java
  35. 128 46
      hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/DFSClientCache.java
  36. 46 0
      hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/Nfs3Utils.java
  37. 11 8
      hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/OpenFileCtx.java
  38. 3 3
      hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/OpenFileCtxCache.java
  39. 210 152
      hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/RpcProgramNfs3.java
  40. 4 4
      hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/WriteCtx.java
  41. 15 9
      hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/WriteManager.java
  42. 2 1
      hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/nfs3/TestClientAccessPrivilege.java
  43. 8 5
      hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/nfs3/TestDFSClientCache.java
  44. 159 2
      hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/nfs3/TestExportsTable.java
  45. 6 13
      hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/nfs3/TestReaddir.java
  46. 44 22
      hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/nfs3/TestRpcProgramNfs3.java
  47. 330 0
      hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/nfs3/TestViewfsWithNfs3.java
  48. 6 3
      hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/nfs3/TestWrites.java
  49. 3 0
      hadoop-hdfs-project/hadoop-hdfs/dev-support/findbugsExcludeFile.xml
  50. 10 0
      hadoop-hdfs-project/hadoop-hdfs/pom.xml
  51. 9 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs
  52. 13 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs.cmd
  53. 162 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
  54. 38 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
  55. 1 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
  56. 44 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RouterAdminProtocolPB.java
  57. 151 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RouterAdminProtocolServerSideTranslatorPB.java
  58. 150 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RouterAdminProtocolTranslatorPB.java
  59. 10 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
  60. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicy.java
  61. 38 11
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyRackFaultTolerant.java
  62. 39 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlocksMap.java
  63. 1 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/web/webhdfs/ParameterParser.java
  64. 211 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/FederationMBean.java
  65. 692 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/FederationMetrics.java
  66. 90 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/FederationRPCMBean.java
  67. 239 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/FederationRPCMetrics.java
  68. 211 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/FederationRPCPerformanceMonitor.java
  69. 624 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/NamenodeBeanMetrics.java
  70. 45 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/StateStoreMBean.java
  71. 144 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/StateStoreMetrics.java
  72. 27 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/package-info.java
  73. 117 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/ActiveNamenodeResolver.java
  74. 87 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/FederationNamenodeContext.java
  75. 46 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/FederationNamenodeServiceState.java
  76. 73 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/FederationNamespaceInfo.java
  77. 75 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/FileSubclusterResolver.java
  78. 317 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/MembershipNamenodeResolver.java
  79. 80 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/MountTableManager.java
  80. 544 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/MountTableResolver.java
  81. 63 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/NamenodePriorityComparator.java
  82. 388 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/NamenodeStatusReport.java
  83. 212 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/PathLocation.java
  84. 76 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/RemoteLocation.java
  85. 29 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/order/DestinationOrder.java
  86. 29 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/order/package-info.java
  87. 41 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/package-info.java
  88. 104 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/ConnectionContext.java
  89. 431 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/ConnectionManager.java
  90. 337 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/ConnectionPool.java
  91. 117 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/ConnectionPoolId.java
  92. 76 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/DFSRouter.java
  93. 169 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/FederationUtil.java
  94. 361 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/NamenodeHeartbeatService.java
  95. 198 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/PeriodicService.java
  96. 68 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RemoteLocationContext.java
  97. 164 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RemoteMethod.java
  98. 71 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RemoteParam.java
  99. 527 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/Router.java
  100. 183 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterAdminServer.java

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

@@ -250,9 +250,8 @@ public class KMSClientProvider extends KeyProvider implements CryptoExtension,
      * - HOSTNAME = string
      * - PORT = integer
      *
-     * If multiple hosts are provider, the Factory will create a
-     * {@link LoadBalancingKMSClientProvider} that round-robins requests
-     * across the provided list of hosts.
+     * This will always create a {@link LoadBalancingKMSClientProvider}
+     * if the uri is correct.
      */
     @Override
     public KeyProvider createProvider(URI providerUri, Configuration conf)
@@ -279,30 +278,26 @@ public class KMSClientProvider extends KeyProvider implements CryptoExtension,
           }
           hostsPart = t[0];
         }
-        return createProvider(providerUri, conf, origUrl, port, hostsPart);
+        return createProvider(conf, origUrl, port, hostsPart);
       }
       return null;
     }
 
-    private KeyProvider createProvider(URI providerUri, Configuration conf,
+    private KeyProvider createProvider(Configuration conf,
         URL origUrl, int port, String hostsPart) throws IOException {
       String[] hosts = hostsPart.split(";");
-      if (hosts.length == 1) {
-        return new KMSClientProvider(providerUri, conf);
-      } else {
-        KMSClientProvider[] providers = new KMSClientProvider[hosts.length];
-        for (int i = 0; i < hosts.length; i++) {
-          try {
-            providers[i] =
-                new KMSClientProvider(
-                    new URI("kms", origUrl.getProtocol(), hosts[i], port,
-                        origUrl.getPath(), null, null), conf);
-          } catch (URISyntaxException e) {
-            throw new IOException("Could not instantiate KMSProvider..", e);
-          }
+      KMSClientProvider[] providers = new KMSClientProvider[hosts.length];
+      for (int i = 0; i < hosts.length; i++) {
+        try {
+          providers[i] =
+              new KMSClientProvider(
+                  new URI("kms", origUrl.getProtocol(), hosts[i], port,
+                      origUrl.getPath(), null, null), conf);
+        } catch (URISyntaxException e) {
+          throw new IOException("Could not instantiate KMSProvider.", e);
         }
-        return new LoadBalancingKMSClientProvider(providers, conf);
       }
+      return new LoadBalancingKMSClientProvider(providers, conf);
     }
   }
 
@@ -1031,7 +1026,11 @@ public class KMSClientProvider extends KeyProvider implements CryptoExtension,
       } catch (InterruptedException e) {
         Thread.currentThread().interrupt();
       } catch (Exception e) {
-        throw new IOException(e);
+        if (e instanceof IOException) {
+          throw (IOException) e;
+        } else {
+          throw new IOException(e);
+        }
       }
     }
     return tokens;

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

@@ -19,6 +19,7 @@
 package org.apache.hadoop.crypto.key.kms;
 
 import java.io.IOException;
+import java.io.InterruptedIOException;
 import java.security.GeneralSecurityException;
 import java.security.NoSuchAlgorithmException;
 import java.util.Arrays;
@@ -31,9 +32,13 @@ import org.apache.hadoop.crypto.key.KeyProvider;
 import org.apache.hadoop.crypto.key.KeyProviderCryptoExtension.CryptoExtension;
 import org.apache.hadoop.crypto.key.KeyProviderCryptoExtension.EncryptedKeyVersion;
 import org.apache.hadoop.crypto.key.KeyProviderDelegationTokenExtension;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.apache.hadoop.io.retry.RetryPolicies;
+import org.apache.hadoop.io.retry.RetryPolicy;
+import org.apache.hadoop.io.retry.RetryPolicy.RetryAction;
+import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.security.token.Token;
-import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.Time;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -69,6 +74,8 @@ public class LoadBalancingKMSClientProvider extends KeyProvider implements
   private final KMSClientProvider[] providers;
   private final AtomicInteger currentIdx;
 
+  private RetryPolicy retryPolicy = null;
+
   public LoadBalancingKMSClientProvider(KMSClientProvider[] providers,
       Configuration conf) {
     this(shuffle(providers), Time.monotonicNow(), conf);
@@ -80,24 +87,82 @@ public class LoadBalancingKMSClientProvider extends KeyProvider implements
     super(conf);
     this.providers = providers;
     this.currentIdx = new AtomicInteger((int)(seed % providers.length));
+    int maxNumRetries = conf.getInt(CommonConfigurationKeysPublic.
+        KMS_CLIENT_FAILOVER_MAX_RETRIES_KEY, providers.length);
+    int sleepBaseMillis = conf.getInt(CommonConfigurationKeysPublic.
+        KMS_CLIENT_FAILOVER_SLEEP_BASE_MILLIS_KEY,
+        CommonConfigurationKeysPublic.
+            KMS_CLIENT_FAILOVER_SLEEP_BASE_MILLIS_DEFAULT);
+    int sleepMaxMillis = conf.getInt(CommonConfigurationKeysPublic.
+        KMS_CLIENT_FAILOVER_SLEEP_MAX_MILLIS_KEY,
+        CommonConfigurationKeysPublic.
+            KMS_CLIENT_FAILOVER_SLEEP_MAX_MILLIS_DEFAULT);
+    Preconditions.checkState(maxNumRetries >= 0);
+    Preconditions.checkState(sleepBaseMillis >= 0);
+    Preconditions.checkState(sleepMaxMillis >= 0);
+    this.retryPolicy = RetryPolicies.failoverOnNetworkException(
+        RetryPolicies.TRY_ONCE_THEN_FAIL, maxNumRetries, 0, sleepBaseMillis,
+        sleepMaxMillis);
   }
 
   @VisibleForTesting
-  KMSClientProvider[] getProviders() {
+  public KMSClientProvider[] getProviders() {
     return providers;
   }
 
   private <T> T doOp(ProviderCallable<T> op, int currPos)
       throws IOException {
+    if (providers.length == 0) {
+      throw new IOException("No providers configured !");
+    }
     IOException ex = null;
-    for (int i = 0; i < providers.length; i++) {
+    int numFailovers = 0;
+    for (int i = 0;; i++, numFailovers++) {
       KMSClientProvider provider = providers[(currPos + i) % providers.length];
       try {
         return op.call(provider);
+      } catch (AccessControlException ace) {
+        // No need to retry on AccessControlException
+        // and AuthorizationException.
+        // This assumes all the servers are configured with identical
+        // permissions and identical key acls.
+        throw ace;
       } catch (IOException ioe) {
-        LOG.warn("KMS provider at [{}] threw an IOException!! {}",
-            provider.getKMSUrl(), StringUtils.stringifyException(ioe));
+        LOG.warn("KMS provider at [{}] threw an IOException: ",
+            provider.getKMSUrl(), ioe);
         ex = ioe;
+
+        RetryAction action = null;
+        try {
+          action = retryPolicy.shouldRetry(ioe, 0, numFailovers, false);
+        } catch (Exception e) {
+          if (e instanceof IOException) {
+            throw (IOException)e;
+          }
+          throw new IOException(e);
+        }
+        // make sure each provider is tried at least once, to keep behavior
+        // compatible with earlier versions of LBKMSCP
+        if (action.action == RetryAction.RetryDecision.FAIL
+            && numFailovers >= providers.length - 1) {
+          LOG.warn("Aborting since the Request has failed with all KMS"
+              + " providers(depending on {}={} setting and numProviders={})"
+              + " in the group OR the exception is not recoverable",
+              CommonConfigurationKeysPublic.KMS_CLIENT_FAILOVER_MAX_RETRIES_KEY,
+              getConf().getInt(
+                  CommonConfigurationKeysPublic.
+                  KMS_CLIENT_FAILOVER_MAX_RETRIES_KEY, providers.length),
+              providers.length);
+          throw ex;
+        }
+        if (((numFailovers + 1) % providers.length) == 0) {
+          // Sleep only after we try all the providers for every cycle.
+          try {
+            Thread.sleep(action.delayMillis);
+          } catch (InterruptedException e) {
+            throw new InterruptedIOException("Thread Interrupted");
+          }
+        }
       } catch (Exception e) {
         if (e instanceof RuntimeException) {
           throw (RuntimeException)e;
@@ -106,12 +171,6 @@ public class LoadBalancingKMSClientProvider extends KeyProvider implements
         }
       }
     }
-    if (ex != null) {
-      LOG.warn("Aborting since the Request has failed with all KMS"
-          + " providers in the group. !!");
-      throw ex;
-    }
-    throw new IOException("No providers configured !!");
   }
 
   private int nextIdx() {

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

@@ -721,6 +721,35 @@ public class CommonConfigurationKeysPublic {
   /** Default value for KMS_CLIENT_ENC_KEY_CACHE_EXPIRY (12 hrs)*/
   public static final int KMS_CLIENT_ENC_KEY_CACHE_EXPIRY_DEFAULT = 43200000;
 
+  /**
+   * @see
+   * <a href="{@docRoot}/../hadoop-project-dist/hadoop-common/core-default.xml">
+   * core-default.xml</a>
+   */
+  /** Default value is the number of providers specified. */
+  public static final String KMS_CLIENT_FAILOVER_MAX_RETRIES_KEY =
+      "hadoop.security.kms.client.failover.max.retries";
+
+  /**
+   * @see
+   * <a href="{@docRoot}/../hadoop-project-dist/hadoop-common/core-default.xml">
+   * core-default.xml</a>
+   */
+  public static final String KMS_CLIENT_FAILOVER_SLEEP_BASE_MILLIS_KEY =
+      "hadoop.security.kms.client.failover.sleep.base.millis";
+  /**  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_FAILOVER_SLEEP_MAX_MILLIS_KEY =
+      "hadoop.security.kms.client.failover.sleep.max.millis";
+  /** Default value is 2 secs. */
+  public static final int KMS_CLIENT_FAILOVER_SLEEP_MAX_MILLIS_DEFAULT  = 2000;
+
   /**
    * @see
    * <a href="{@docRoot}/../hadoop-project-dist/hadoop-common/core-default.xml">

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

@@ -60,7 +60,8 @@ public class FileStatus implements Writable, Comparable<Object>,
     HAS_ACL,
     HAS_CRYPT,
     HAS_EC,
-  };
+    SNAPSHOT_ENABLED
+  }
   private static final Set<AttrFlags> NONE = Collections.<AttrFlags>emptySet();
   private static Set<AttrFlags> flags(boolean acl, boolean crypt, boolean ec) {
     if (!(acl || crypt || ec)) {
@@ -273,6 +274,15 @@ public class FileStatus implements Writable, Comparable<Object>,
     return attr.contains(AttrFlags.HAS_EC);
   }
 
+  /**
+   * Check if directory is Snapshot enabled or not.
+   *
+   * @return true if directory is snapshot enabled
+   */
+  public boolean isSnapshotEnabled() {
+    return attr.contains(AttrFlags.SNAPSHOT_ENABLED);
+  }
+
   /**
    * Get the owner of the file.
    * @return owner of the file. The string could be empty if there is no
@@ -330,6 +340,19 @@ public class FileStatus implements Writable, Comparable<Object>,
     this.group = (group == null) ? "" :  group;
   }
 
+  /**
+   * Sets Snapshot enabled flag.
+   *
+   * @param isSnapShotEnabled When true, SNAPSHOT_ENABLED flag is set
+   */
+  public void setSnapShotEnabledFlag(boolean isSnapShotEnabled) {
+    if (isSnapShotEnabled) {
+      attr.add(AttrFlags.SNAPSHOT_ENABLED);
+    } else {
+      attr.remove(AttrFlags.SNAPSHOT_ENABLED);
+    }
+  }
+
   /**
    * @return The contents of the symbolic link.
    */
@@ -486,4 +509,6 @@ public class FileStatus implements Writable, Comparable<Object>,
     }
   }
 
+
+
 }

+ 10 - 4
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/protocolPB/PBHelper.java

@@ -94,11 +94,15 @@ public final class PBHelper {
     owner = proto.getOwner();
     group = proto.getGroup();
     int flags = proto.getFlags();
-    return new FileStatus(length, isdir, blockReplication, blocksize,
-        mtime, atime, permission, owner, group, symlink, path,
-        (flags & FileStatusProto.Flags.HAS_ACL_VALUE)   != 0,
+    FileStatus fileStatus = new FileStatus(length, isdir, blockReplication,
+        blocksize, mtime, atime, permission, owner, group, symlink, path,
+        (flags & FileStatusProto.Flags.HAS_ACL_VALUE) != 0,
         (flags & FileStatusProto.Flags.HAS_CRYPT_VALUE) != 0,
-        (flags & FileStatusProto.Flags.HAS_EC_VALUE)    != 0);
+        (flags & FileStatusProto.Flags.HAS_EC_VALUE) != 0);
+
+    fileStatus.setSnapShotEnabledFlag((flags & FileStatusProto.Flags
+        .SNAPSHOT_ENABLED_VALUE) != 0);
+    return fileStatus;
   }
 
   public static FileStatusProto convert(FileStatus stat) throws IOException {
@@ -124,6 +128,8 @@ public final class PBHelper {
     flags |= stat.hasAcl()         ? FileStatusProto.Flags.HAS_ACL_VALUE   : 0;
     flags |= stat.isEncrypted()    ? FileStatusProto.Flags.HAS_CRYPT_VALUE : 0;
     flags |= stat.isErasureCoded() ? FileStatusProto.Flags.HAS_EC_VALUE    : 0;
+    flags |= stat.isSnapshotEnabled() ? FileStatusProto.Flags
+        .SNAPSHOT_ENABLED_VALUE : 0;
     bld.setFlags(flags);
     return bld.build();
   }

+ 2 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/InodeTree.java

@@ -407,7 +407,8 @@ abstract class InodeTree<T> {
         for (int j = 1; j <= i; ++j) {
           failedAt.append('/').append(path[j]);
         }
-        throw (new FileNotFoundException(failedAt.toString()));
+        throw (new FileNotFoundException("File/Directory does not exist: "
+            + failedAt.toString()));
       }
 
       if (nextInode instanceof INodeLink) {

+ 6 - 12
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/serializer/SerializationFactory.java

@@ -55,18 +55,12 @@ public class SerializationFactory extends Configured {
    */
   public SerializationFactory(Configuration conf) {
     super(conf);
-    if (conf.get(CommonConfigurationKeys.IO_SERIALIZATIONS_KEY).equals("")) {
-      LOG.warn("Serialization for various data types may not be available. Please configure "
-          + CommonConfigurationKeys.IO_SERIALIZATIONS_KEY
-          + " properly to have serialization support (it is currently not set).");
-    } else {
-      for (String serializerName : conf.getTrimmedStrings(
-          CommonConfigurationKeys.IO_SERIALIZATIONS_KEY, new String[] {
-              WritableSerialization.class.getName(),
-              AvroSpecificSerialization.class.getName(),
-              AvroReflectSerialization.class.getName() })) {
-        add(conf, serializerName);
-      }
+    for (String serializerName : conf.getTrimmedStrings(
+            CommonConfigurationKeys.IO_SERIALIZATIONS_KEY,
+            new String[]{WritableSerialization.class.getName(),
+                    AvroSpecificSerialization.class.getName(),
+                    AvroReflectSerialization.class.getName()})) {
+      add(conf, serializerName);
     }
   }
 

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

@@ -122,13 +122,15 @@ public class FairCallQueue<E extends Schedulable> extends AbstractQueue<E>
   private E removeNextElement() {
     int priority = multiplexer.getAndAdvanceCurrentIndex();
     E e = queues.get(priority).poll();
-    if (e == null) {
+    // a semaphore permit has been acquired, so an element MUST be extracted
+    // or the semaphore and queued elements will go out of sync.  loop to
+    // avoid race condition if elements are added behind the current position,
+    // awakening other threads that poll the elements ahead of our position.
+    while (e == null) {
       for (int idx = 0; e == null && idx < queues.size(); idx++) {
         e = queues.get(idx).poll();
       }
     }
-    // guaranteed to find an element if caller acquired permit.
-    assert e != null : "consumer didn't acquire semaphore!";
     return e;
   }
 

+ 20 - 7
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/http/CrossOriginFilter.java

@@ -37,6 +37,7 @@ import javax.servlet.http.HttpServletResponse;
 import org.apache.commons.lang.StringUtils;
 
 import com.google.common.annotations.VisibleForTesting;
+import java.util.stream.Collectors;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -66,6 +67,7 @@ public class CrossOriginFilter implements Filter {
   // Filter configuration
   public static final String ALLOWED_ORIGINS = "allowed-origins";
   public static final String ALLOWED_ORIGINS_DEFAULT = "*";
+  public static final String ALLOWED_ORIGINS_REGEX_PREFIX = "regex:";
   public static final String ALLOWED_METHODS = "allowed-methods";
   public static final String ALLOWED_METHODS_DEFAULT = "GET,POST,HEAD";
   public static final String ALLOWED_HEADERS = "allowed-headers";
@@ -194,6 +196,12 @@ public class CrossOriginFilter implements Filter {
     allowAllOrigins = allowedOrigins.contains("*");
     LOG.info("Allowed Origins: " + StringUtils.join(allowedOrigins, ','));
     LOG.info("Allow All Origins: " + allowAllOrigins);
+    List<String> discouragedAllowedOrigins = allowedOrigins.stream()
+            .filter(s -> s.length() > 1 && s.contains("*"))
+            .collect(Collectors.toList());
+    for (String discouragedAllowedOrigin : discouragedAllowedOrigins) {
+        LOG.warn("Allowed Origin pattern '" + discouragedAllowedOrigin + "' is discouraged, use the 'regex:' prefix and use a Java regular expression instead.");
+    }
   }
 
   private void initializeMaxAge(FilterConfig filterConfig) {
@@ -228,15 +236,20 @@ public class CrossOriginFilter implements Filter {
     String[] origins = originsList.trim().split("\\s+");
     for (String origin : origins) {
       for (String allowedOrigin : allowedOrigins) {
-        if (allowedOrigin.contains("*")) {
-          String regex = allowedOrigin.replace(".", "\\.").replace("*", ".*");
-          Pattern p = Pattern.compile(regex);
-          Matcher m = p.matcher(origin);
-          if (m.matches()) {
+        Pattern regexPattern = null;
+        if (allowedOrigin.startsWith(ALLOWED_ORIGINS_REGEX_PREFIX)) {
+            String regex = allowedOrigin.substring(ALLOWED_ORIGINS_REGEX_PREFIX.length());
+            regexPattern = Pattern.compile(regex);
+        } else if (allowedOrigin.contains("*")) {
+            String regex = allowedOrigin.replace(".", "\\.").replace("*", ".*");
+            regexPattern = Pattern.compile(regex);
+        }
+
+        if (regexPattern != null
+                && regexPattern.matcher(origin).matches()) {
             return true;
-          }
         } else if (allowedOrigin.equals(origin)) {
-          return true;
+            return true;
         }
       }
     }

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

@@ -272,8 +272,11 @@ public abstract class DelegationTokenAuthenticationHandler
               }
               String renewer = ServletUtils.getParameter(request,
                   KerberosDelegationTokenAuthenticator.RENEWER_PARAM);
+              String service = ServletUtils.getParameter(request,
+                  KerberosDelegationTokenAuthenticator.SERVICE_PARAM);
               try {
-                Token<?> dToken = tokenManager.createToken(requestUgi, renewer);
+                Token<?> dToken = tokenManager.createToken(requestUgi, renewer,
+                    service);
                 map = delegationTokenToJSON(dToken);
               } catch (IOException ex) {
                 throw new AuthenticationException(ex.toString(), ex);

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

@@ -70,6 +70,7 @@ public abstract class DelegationTokenAuthenticator implements Authenticator {
   public static final String DELEGATION_PARAM = "delegation";
   public static final String TOKEN_PARAM = "token";
   public static final String RENEWER_PARAM = "renewer";
+  public static final String SERVICE_PARAM = "service";
   public static final String DELEGATION_TOKEN_JSON = "Token";
   public static final String DELEGATION_TOKEN_URL_STRING_JSON = "urlString";
   public static final String RENEW_DELEGATION_TOKEN_JSON = "long";

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

@@ -160,7 +160,14 @@ public class DelegationTokenManager {
   @SuppressWarnings("unchecked")
   public Token<? extends AbstractDelegationTokenIdentifier> createToken(
       UserGroupInformation ugi, String renewer) {
-    LOG.debug("Creating token with ugi:{}, renewer:{}.", ugi, renewer);
+    return createToken(ugi, renewer, null);
+  }
+
+  @SuppressWarnings("unchecked")
+  public Token<? extends AbstractDelegationTokenIdentifier> createToken(
+      UserGroupInformation ugi, String renewer, String service) {
+    LOG.debug("Creating token with ugi:{}, renewer:{}, service:{}.",
+        ugi, renewer, service !=null ? service : "");
     renewer = (renewer == null) ? ugi.getShortUserName() : renewer;
     String user = ugi.getUserName();
     Text owner = new Text(user);
@@ -173,7 +180,11 @@ public class DelegationTokenManager {
     tokenIdentifier.setOwner(owner);
     tokenIdentifier.setRenewer(new Text(renewer));
     tokenIdentifier.setRealUser(realUser);
-    return new Token(tokenIdentifier, secretManager);
+    Token token = new Token(tokenIdentifier, secretManager);
+    if (service != null) {
+      token.setService(new Text(service));
+    }
+    return token;
   }
 
   @SuppressWarnings("unchecked")

+ 2 - 1
hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/jni_xor_decoder.c

@@ -58,7 +58,8 @@ Java_org_apache_hadoop_io_erasurecode_rawcoder_NativeXORRawDecoder_decodeImpl(
   numParityUnits = ((IsalCoder*)xorDecoder)->numParityUnits;
   chunkSize = (int)dataLen;
 
-  getInputs(env, inputs, inputOffsets, xorDecoder->inputs, numDataUnits);
+  getInputs(env, inputs, inputOffsets, xorDecoder->inputs,
+      numDataUnits + numParityUnits);
   getOutputs(env, outputs, outputOffsets, xorDecoder->outputs, numParityUnits);
 
   for (i = 0; i < numDataUnits + numParityUnits; i++) {

+ 4 - 3
hadoop-common-project/hadoop-common/src/main/proto/FSProtos.proto

@@ -44,9 +44,10 @@ message FileStatusProto {
     FT_SYMLINK = 3;
   }
   enum Flags {
-    HAS_ACL    = 0x01; // has ACLs
-    HAS_CRYPT  = 0x02; // encrypted
-    HAS_EC     = 0x04; // erasure coded
+    HAS_ACL           = 0x01; // has ACLs
+    HAS_CRYPT         = 0x02; // encrypted
+    HAS_EC            = 0x04; // erasure coded
+    SNAPSHOT_ENABLED  = 0x08; // snapshot enabled
   }
   required FileType fileType            = 1;
   required string path                  = 2;

+ 0 - 1
hadoop-common-project/hadoop-common/src/main/resources/META-INF/services/org.apache.hadoop.fs.FileSystem

@@ -15,7 +15,6 @@
 
 org.apache.hadoop.fs.LocalFileSystem
 org.apache.hadoop.fs.viewfs.ViewFileSystem
-org.apache.hadoop.fs.ftp.FTPFileSystem
 org.apache.hadoop.fs.HarFileSystem
 org.apache.hadoop.fs.http.HttpFileSystem
 org.apache.hadoop.fs.http.HttpsFileSystem

+ 43 - 3
hadoop-common-project/hadoop-common/src/main/resources/core-default.xml

@@ -815,6 +815,12 @@
   <description>The FileSystem for Ftp: uris.</description>
 </property>
 
+<property>
+  <name>fs.ftp.impl</name>
+  <value>org.apache.hadoop.fs.ftp.FTPFileSystem</value>
+  <description>The implementation class of the FTP FileSystem</description>
+</property>
+
 <property>
   <name>fs.AbstractFileSystem.webhdfs.impl</name>
   <value>org.apache.hadoop.fs.WebHdfs</value>
@@ -1874,9 +1880,15 @@
 <property>
   <name>hadoop.http.cross-origin.allowed-origins</name>
   <value>*</value>
-  <description>Comma separated list of origins that are allowed for web
-    services needing cross-origin (CORS) support. Wildcards (*) and patterns
-    allowed</description>
+  <description>Comma separated list of origins that are allowed for web services
+    needing cross-origin (CORS) support. If a value in the list contains an
+    asterix (*), a regex pattern, escaping any dots ('.' -> '\.') and replacing
+    the asterix such that it captures any characters ('*' -> '.*'), is generated.
+    Values prefixed with 'regex:' are interpreted directly as regular expressions,
+    e.g. use the expression 'regex:https?:\/\/foo\.bar:([0-9]+)?' to allow any
+    origin using the 'http' or 'https' protocol in the domain 'foo.bar' on any
+    port. The use of simple wildcards ('*') is discouraged, and only available for
+    backward compatibility.</description>
 </property>
 
 <property>
@@ -2342,6 +2354,34 @@
   </description>
 </property>
 
+<property>
+  <name>hadoop.security.kms.client.failover.sleep.base.millis</name>
+  <value>100</value>
+  <description>
+    Expert only. The time to wait, in milliseconds, between failover
+    attempts increases exponentially as a function of the number of
+    attempts made so far, with a random factor of +/- 50%. This option
+    specifies the base value used in the failover calculation. The
+    first failover will retry immediately. The 2nd failover attempt
+    will delay at least hadoop.security.client.failover.sleep.base.millis
+    milliseconds. And so on.
+  </description>
+</property>
+
+<property>
+  <name>hadoop.security.kms.client.failover.sleep.max.millis</name>
+  <value>2000</value>
+  <description>
+    Expert only. The time to wait, in milliseconds, between failover
+    attempts increases exponentially as a function of the number of
+    attempts made so far, with a random factor of +/- 50%. This option
+    specifies the maximum value to wait between failovers.
+    Specifically, the time between two failover attempts will not
+    exceed +/- 50% of hadoop.security.client.failover.sleep.max.millis
+    milliseconds.
+  </description>
+</property>
+
  <property>
   <name>ipc.server.max.connections</name>
   <value>0</value>

+ 1 - 1
hadoop-common-project/hadoop-common/src/site/markdown/HttpAuthentication.md

@@ -60,7 +60,7 @@ Add org.apache.hadoop.security.HttpCrossOriginFilterInitializer to hadoop.http.f
 | Property                                 | Default Value                                 | Description                                                                            |
 |:---------------------------------------- |:--------------------------------------------- |:-------------------------------------------------------------------------------------  |
 | hadoop.http.cross-origin.enabled         | `false`                                       | Enables cross origin support for all web-services                                      |
-| hadoop.http.cross-origin.allowed-origins | `*`                                           | Comma separated list of origins that are allowed, wildcards (`*`) and patterns allowed |
+| hadoop.http.cross-origin.allowed-origins | `*`                                           | Comma separated list of origins that are allowed. Values prefixed with `regex:` are interpreted as regular expressions. Values containing wildcards (`*`) are possible as well, here a regular expression is generated, the use is discouraged and support is only available for backward compatibility. |
 | hadoop.http.cross-origin.allowed-methods | `GET,POST,HEAD`                               | Comma separated list of methods that are allowed                                       |
 | hadoop.http.cross-origin.allowed-headers | `X-Requested-With,Content-Type,Accept,Origin` | Comma separated list of headers that are allowed                                       |
 | hadoop.http.cross-origin.max-age         | `1800`                                        | Number of seconds a pre-flighted request can be cached                                 |

+ 646 - 0
hadoop-common-project/hadoop-common/src/site/markdown/release/3.0.0-beta1/CHANGES.3.0.0-beta1.md

@@ -0,0 +1,646 @@
+
+<!---
+# 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.
+-->
+# Apache Hadoop Changelog
+
+## Release 3.0.0-beta1 - 2017-09-28
+
+### INCOMPATIBLE CHANGES:
+
+| JIRA | Summary | Priority | Component | Reporter | Contributor |
+|:---- |:---- | :--- |:---- |:---- |:---- |
+| [HADOOP-14539](https://issues.apache.org/jira/browse/HADOOP-14539) | Move commons logging APIs over to slf4j in hadoop-common |  Major | . | Akira Ajisaka | Wenxin He |
+| [HDFS-12206](https://issues.apache.org/jira/browse/HDFS-12206) | Rename the split EC / replicated block metrics |  Major | metrics | Andrew Wang | Andrew Wang |
+| [HADOOP-13595](https://issues.apache.org/jira/browse/HADOOP-13595) | Rework hadoop\_usage to be broken up by clients/daemons/etc. |  Blocker | scripts | Allen Wittenauer | Allen Wittenauer |
+| [HDFS-6984](https://issues.apache.org/jira/browse/HDFS-6984) | Serialize FileStatus via protobuf |  Major | . | Colin P. McCabe | Chris Douglas |
+| [YARN-6961](https://issues.apache.org/jira/browse/YARN-6961) | Remove commons-logging dependency from hadoop-yarn-server-applicationhistoryservice module |  Minor | build | Akira Ajisaka | Yeliang Cang |
+| [HDFS-11957](https://issues.apache.org/jira/browse/HDFS-11957) | Enable POSIX ACL inheritance by default |  Major | security | John Zhuge | John Zhuge |
+| [MAPREDUCE-6870](https://issues.apache.org/jira/browse/MAPREDUCE-6870) | Add configuration for MR job to finish when all reducers are complete (even with unfinished mappers) |  Major | . | Zhe Zhang | Peter Bacsko |
+| [HADOOP-14260](https://issues.apache.org/jira/browse/HADOOP-14260) | Configuration.dumpConfiguration should redact sensitive information |  Major | conf, security | Vihang Karajgaonkar | John Zhuge |
+| [HADOOP-14726](https://issues.apache.org/jira/browse/HADOOP-14726) | Mark FileStatus::isDir as final |  Minor | fs | Chris Douglas | Chris Douglas |
+| [HDFS-12303](https://issues.apache.org/jira/browse/HDFS-12303) | Change default EC cell size to 1MB for better performance |  Blocker | . | Wei Zhou | Wei Zhou |
+| [HDFS-12258](https://issues.apache.org/jira/browse/HDFS-12258) | ec -listPolicies should list all policies in system, no matter it's enabled or disabled |  Major | . | SammiChen | Wei Zhou |
+| [MAPREDUCE-6892](https://issues.apache.org/jira/browse/MAPREDUCE-6892) | Issues with the count of failed/killed tasks in the jhist file |  Major | client, jobhistoryserver | Peter Bacsko | Peter Bacsko |
+| [HADOOP-14414](https://issues.apache.org/jira/browse/HADOOP-14414) | Calling maven-site-plugin directly for docs profile is unnecessary |  Minor | . | Andras Bokor | Andras Bokor |
+| [HDFS-12218](https://issues.apache.org/jira/browse/HDFS-12218) | Rename split EC / replicated block metrics in BlockManager |  Blocker | erasure-coding, metrics | Andrew Wang | Andrew Wang |
+| [HADOOP-14847](https://issues.apache.org/jira/browse/HADOOP-14847) | Remove Guava Supplier and change to java Supplier in AMRMClient and AMRMClientAysnc |  Blocker | . | Bharat Viswanadham | Bharat Viswanadham |
+| [HDFS-12414](https://issues.apache.org/jira/browse/HDFS-12414) | Ensure to use CLI command to enable/disable erasure coding policy |  Major | . | SammiChen | SammiChen |
+| [HDFS-12438](https://issues.apache.org/jira/browse/HDFS-12438) | Rename dfs.datanode.ec.reconstruction.stripedblock.threads.size to dfs.datanode.ec.reconstruction.threads |  Major | . | Andrew Wang | Andrew Wang |
+| [HADOOP-14738](https://issues.apache.org/jira/browse/HADOOP-14738) | Remove S3N and obsolete bits of S3A; rework docs |  Blocker | fs/s3 | Steve Loughran | Steve Loughran |
+| [HDFS-7859](https://issues.apache.org/jira/browse/HDFS-7859) | Erasure Coding: Persist erasure coding policies in NameNode |  Major | . | Kai Zheng | SammiChen |
+| [HDFS-12395](https://issues.apache.org/jira/browse/HDFS-12395) | Support erasure coding policy operations in namenode edit log |  Major | erasure-coding | SammiChen | SammiChen |
+| [HADOOP-14670](https://issues.apache.org/jira/browse/HADOOP-14670) | Increase minimum cmake version for all platforms |  Major | build | Allen Wittenauer | Allen Wittenauer |
+| [HDFS-12447](https://issues.apache.org/jira/browse/HDFS-12447) | Rename AddECPolicyResponse to AddErasureCodingPolicyResponse |  Major | . | SammiChen | SammiChen |
+| [HDFS-7337](https://issues.apache.org/jira/browse/HDFS-7337) | Configurable and pluggable erasure codec and policy |  Critical | erasure-coding | Zhe Zhang | SammiChen |
+
+
+### IMPORTANT ISSUES:
+
+| JIRA | Summary | Priority | Component | Reporter | Contributor |
+|:---- |:---- | :--- |:---- |:---- |:---- |
+| [YARN-6959](https://issues.apache.org/jira/browse/YARN-6959) | RM may allocate wrong AM Container for new attempt |  Major | capacity scheduler, fairscheduler, scheduler | Yuqi Wang | Yuqi Wang |
+
+
+### NEW FEATURES:
+
+| JIRA | Summary | Priority | Component | Reporter | Contributor |
+|:---- |:---- | :--- |:---- |:---- |:---- |
+| [YARN-4161](https://issues.apache.org/jira/browse/YARN-4161) | Capacity Scheduler : Assign single or multiple containers per heart beat driven by configuration |  Major | capacity scheduler | Mayank Bansal | Wei Yan |
+| [HADOOP-14560](https://issues.apache.org/jira/browse/HADOOP-14560) | Make HttpServer2 backlog size configurable |  Major | common | Alexander Krasheninnikov | Alexander Krasheninnikov |
+| [HDFS-10899](https://issues.apache.org/jira/browse/HDFS-10899) | Add functionality to re-encrypt EDEKs |  Major | encryption, kms | Xiao Chen | Xiao Chen |
+| [YARN-5355](https://issues.apache.org/jira/browse/YARN-5355) | YARN Timeline Service v.2: alpha 2 |  Critical | timelineserver | Sangjin Lee | Vrushali C |
+| [HADOOP-13345](https://issues.apache.org/jira/browse/HADOOP-13345) | S3Guard: Improved Consistency for S3A |  Major | fs/s3 | Chris Nauroth | Chris Nauroth |
+| [HADOOP-12077](https://issues.apache.org/jira/browse/HADOOP-12077) | Provide a multi-URI replication Inode for ViewFs |  Major | fs | Gera Shegalov | Gera Shegalov |
+| [HDFS-7877](https://issues.apache.org/jira/browse/HDFS-7877) | Support maintenance state for datanodes |  Major | datanode, namenode | Ming Ma | Ming Ma |
+| [YARN-2915](https://issues.apache.org/jira/browse/YARN-2915) | Enable YARN RM scale out via federation using multiple RM's |  Major | nodemanager, resourcemanager | Sriram Rao | Subru Krishnan |
+
+
+### IMPROVEMENTS:
+
+| JIRA | Summary | Priority | Component | Reporter | Contributor |
+|:---- |:---- | :--- |:---- |:---- |:---- |
+| [HDFS-12042](https://issues.apache.org/jira/browse/HDFS-12042) | Lazy initialize AbstractINodeDiffList#diffs for snapshots to reduce memory consumption |  Major | . | Misha Dmitriev | Misha Dmitriev |
+| [HDFS-12078](https://issues.apache.org/jira/browse/HDFS-12078) | Add time unit to the description of property dfs.namenode.stale.datanode.interval in hdfs-default.xml |  Minor | documentation, hdfs | Weiwei Yang | Weiwei Yang |
+| [YARN-6752](https://issues.apache.org/jira/browse/YARN-6752) | Display reserved resources in web UI per application |  Major | fairscheduler | Abdullah Yousufi | Abdullah Yousufi |
+| [YARN-6746](https://issues.apache.org/jira/browse/YARN-6746) | SchedulerUtils.checkResourceRequestMatchingNodePartition() is dead code |  Minor | scheduler | Daniel Templeton | Deepti Sawhney |
+| [YARN-6410](https://issues.apache.org/jira/browse/YARN-6410) | FSContext.scheduler should be final |  Minor | fairscheduler | Daniel Templeton | Yeliang Cang |
+| [YARN-6764](https://issues.apache.org/jira/browse/YARN-6764) | Simplify the logic in FairScheduler#attemptScheduling |  Trivial | fairscheduler | Yufei Gu | Yufei Gu |
+| [HADOOP-14443](https://issues.apache.org/jira/browse/HADOOP-14443) | Azure: Support retry and client side failover for authorization, SASKey and delegation token generation |  Major | fs/azure | Santhosh G Nayak | Santhosh G Nayak |
+| [HADOOP-14535](https://issues.apache.org/jira/browse/HADOOP-14535) | wasb: implement high-performance random access and seek of block blobs |  Major | fs/azure | Thomas Marquardt | Thomas Marquardt |
+| [HADOOP-14629](https://issues.apache.org/jira/browse/HADOOP-14629) | Improve exception checking in FileContext related JUnit tests |  Major | fs, test | Andras Bokor | Andras Bokor |
+| [YARN-6689](https://issues.apache.org/jira/browse/YARN-6689) | PlacementRule should be configurable |  Major | . | Jonathan Hung | Jonathan Hung |
+| [HDFS-12130](https://issues.apache.org/jira/browse/HDFS-12130) | Optimizing permission check for getContentSummary |  Major | namenode | Chen Liang | Chen Liang |
+| [HDFS-12137](https://issues.apache.org/jira/browse/HDFS-12137) | DN dataset lock should be fair |  Critical | datanode | Daryn Sharp | Daryn Sharp |
+| [HDFS-11786](https://issues.apache.org/jira/browse/HDFS-11786) | Add support to make copyFromLocal multi threaded |  Major | hdfs | Mukul Kumar Singh | Mukul Kumar Singh |
+| [HDFS-12138](https://issues.apache.org/jira/browse/HDFS-12138) | Remove redundant 'public' modifiers from BlockCollection |  Trivial | namenode | Chen Liang | Chen Liang |
+| [HADOOP-14640](https://issues.apache.org/jira/browse/HADOOP-14640) | Azure: Support affinity for service running on localhost and reuse SPNEGO hadoop.auth cookie for authorization, SASKey and delegation token generation |  Major | fs/azure | Santhosh G Nayak | Santhosh G Nayak |
+| [YARN-6778](https://issues.apache.org/jira/browse/YARN-6778) | In ResourceWeights, weights and setWeights() should be final |  Minor | scheduler | Daniel Templeton | Daniel Templeton |
+| [HDFS-12067](https://issues.apache.org/jira/browse/HDFS-12067) | Correct dfsadmin commands usage message to reflects IPC port |  Major | . | steven-wugang | steven-wugang |
+| [HADOOP-14666](https://issues.apache.org/jira/browse/HADOOP-14666) | Tests use assertTrue(....equals(...)) instead of assertEquals() |  Minor | test | Daniel Templeton | Daniel Templeton |
+| [HADOOP-14659](https://issues.apache.org/jira/browse/HADOOP-14659) | UGI getShortUserName does not need to search the Subject |  Major | common | Daryn Sharp | Daryn Sharp |
+| [HADOOP-14557](https://issues.apache.org/jira/browse/HADOOP-14557) | Document  HADOOP-8143  (Change distcp to have -pb on by default) |  Trivial | . | Wei-Chiu Chuang | Bharat Viswanadham |
+| [YARN-6768](https://issues.apache.org/jira/browse/YARN-6768) | Improve performance of yarn api record toString and fromString |  Major | . | Jonathan Eagles | Jonathan Eagles |
+| [YARN-6779](https://issues.apache.org/jira/browse/YARN-6779) | DominantResourceFairnessPolicy.DominantResourceFairnessComparator.calculateShares() should be @VisibleForTesting |  Trivial | fairscheduler | Daniel Templeton | Yeliang Cang |
+| [YARN-6845](https://issues.apache.org/jira/browse/YARN-6845) | Variable scheduler of FSLeafQueue duplicates the one of its parent FSQueue. |  Trivial | fairscheduler | Yufei Gu | Yufei Gu |
+| [HADOOP-14597](https://issues.apache.org/jira/browse/HADOOP-14597) | Native compilation broken with OpenSSL-1.1.0 because EVP\_CIPHER\_CTX has been made opaque |  Major | . | Ravi Prakash | Ravi Prakash |
+| [HDFS-12193](https://issues.apache.org/jira/browse/HDFS-12193) | Fix style issues in HttpFS tests |  Trivial | httpfs | Zoran Dimitrijevic | Zoran Dimitrijevic |
+| [HADOOP-14681](https://issues.apache.org/jira/browse/HADOOP-14681) | Remove MockitoMaker class |  Major | test | Andras Bokor | Andras Bokor |
+| [HDFS-12143](https://issues.apache.org/jira/browse/HDFS-12143) | Improve performance of getting and removing inode features |  Major | namenode | Daryn Sharp | Daryn Sharp |
+| [HDFS-12171](https://issues.apache.org/jira/browse/HDFS-12171) | Reduce IIP object allocations for inode lookup |  Major | namenode | Daryn Sharp | Daryn Sharp |
+| [HDFS-12190](https://issues.apache.org/jira/browse/HDFS-12190) | Enable 'hdfs dfs -stat' to display access time |  Major | hdfs, shell | Yongjun Zhang | Yongjun Zhang |
+| [YARN-6864](https://issues.apache.org/jira/browse/YARN-6864) | FSPreemptionThread cleanup for readability |  Minor | fairscheduler | Daniel Templeton | Daniel Templeton |
+| [HADOOP-14455](https://issues.apache.org/jira/browse/HADOOP-14455) | ViewFileSystem#rename should support be supported within same nameservice with different mountpoints |  Major | viewfs | Brahma Reddy Battula | Brahma Reddy Battula |
+| [HADOOP-14690](https://issues.apache.org/jira/browse/HADOOP-14690) | RetryInvocationHandler$RetryInfo should override toString() |  Minor | . | Akira Ajisaka | Yeliang Cang |
+| [HADOOP-14709](https://issues.apache.org/jira/browse/HADOOP-14709) | Fix checkstyle warnings in ContractTestUtils |  Minor | test | Steve Loughran | Thomas Marquardt |
+| [MAPREDUCE-6914](https://issues.apache.org/jira/browse/MAPREDUCE-6914) | Tests use assertTrue(....equals(...)) instead of assertEquals() |  Minor | test | Daniel Templeton | Daniel Templeton |
+| [YARN-6832](https://issues.apache.org/jira/browse/YARN-6832) | Tests use assertTrue(....equals(...)) instead of assertEquals() |  Minor | test | Daniel Templeton | Daniel Templeton |
+| [HADOOP-14706](https://issues.apache.org/jira/browse/HADOOP-14706) | Adding a helper method to determine whether a log is Log4j implement |  Minor | util | Wenxin He | Wenxin He |
+| [HADOOP-14471](https://issues.apache.org/jira/browse/HADOOP-14471) | Upgrade Jetty to latest 9.3 version |  Major | . | John Zhuge | John Zhuge |
+| [HDFS-12251](https://issues.apache.org/jira/browse/HDFS-12251) | Add document for StreamCapabilities |  Major | . | Lei (Eddy) Xu | Lei (Eddy) Xu |
+| [HDFS-12224](https://issues.apache.org/jira/browse/HDFS-12224) | Add tests to TestJournalNodeSync for sync after JN downtime |  Major | hdfs | Hanisha Koneru | Hanisha Koneru |
+| [YARN-6802](https://issues.apache.org/jira/browse/YARN-6802) | Add Max AM Resource and AM Resource Usage to Leaf Queue View in FairScheduler WebUI |  Major | fairscheduler | YunFan Zhou | YunFan Zhou |
+| [HDFS-12036](https://issues.apache.org/jira/browse/HDFS-12036) | Add audit log for some erasure coding operations |  Major | namenode | Wei-Chiu Chuang | Huafeng Wang |
+| [HDFS-12264](https://issues.apache.org/jira/browse/HDFS-12264) | DataNode uses a deprecated method IoUtils#cleanup. |  Major | . | Ajay Kumar | Ajay Kumar |
+| [YARN-6811](https://issues.apache.org/jira/browse/YARN-6811) | [ATS1.5]  All history logs should be kept under its own User Directory. |  Major | timelineclient, timelineserver | Rohith Sharma K S | Rohith Sharma K S |
+| [YARN-6879](https://issues.apache.org/jira/browse/YARN-6879) | TestLeafQueue.testDRFUserLimits() has commented out code |  Trivial | capacity scheduler, test | Daniel Templeton | Angela Wang |
+| [MAPREDUCE-6923](https://issues.apache.org/jira/browse/MAPREDUCE-6923) | Optimize MapReduce Shuffle I/O for small partitions |  Major | . | Robert Schmidtke | Robert Schmidtke |
+| [HDFS-12287](https://issues.apache.org/jira/browse/HDFS-12287) | Remove a no-longer applicable TODO comment in DatanodeManager |  Trivial | namenode | Chen Liang | Chen Liang |
+| [YARN-6952](https://issues.apache.org/jira/browse/YARN-6952) | Enable scheduling monitor in FS |  Major | fairscheduler, resourcemanager | Yufei Gu | Yufei Gu |
+| [YARN-6884](https://issues.apache.org/jira/browse/YARN-6884) | AllocationFileLoaderService.loadQueue() has an if without braces |  Trivial | fairscheduler | Daniel Templeton | weiyuan |
+| [YARN-6882](https://issues.apache.org/jira/browse/YARN-6882) | AllocationFileLoaderService.reloadAllocations() should use the diamond operator |  Trivial | fairscheduler | Daniel Templeton | Larry Lo |
+| [HADOOP-14627](https://issues.apache.org/jira/browse/HADOOP-14627) | Support MSI and DeviceCode token provider in ADLS |  Major | fs/adl | Atul Sikaria | Atul Sikaria |
+| [HDFS-12221](https://issues.apache.org/jira/browse/HDFS-12221) | Replace xerces in XmlEditsVisitor |  Major | . | Lei (Eddy) Xu | Ajay Kumar |
+| [HADOOP-14741](https://issues.apache.org/jira/browse/HADOOP-14741) | Refactor curator based ZooKeeper communication into common library |  Major | . | Subru Krishnan | Íñigo Goiri |
+| [HDFS-12162](https://issues.apache.org/jira/browse/HDFS-12162) | Update listStatus document to describe the behavior when the argument is a file |  Major | hdfs, httpfs | Yongjun Zhang | Ajay Kumar |
+| [YARN-6881](https://issues.apache.org/jira/browse/YARN-6881) | LOG is unused in AllocationConfiguration |  Major | fairscheduler | Daniel Templeton | weiyuan |
+| [YARN-6917](https://issues.apache.org/jira/browse/YARN-6917) | Queue path is recomputed from scratch on every allocation |  Minor | capacityscheduler | Jason Lowe | Eric Payne |
+| [HADOOP-14673](https://issues.apache.org/jira/browse/HADOOP-14673) | Remove leftover hadoop\_xml\_escape from functions |  Major | scripts | Allen Wittenauer | Ajay Kumar |
+| [HADOOP-14662](https://issues.apache.org/jira/browse/HADOOP-14662) | Update azure-storage sdk to version 5.4.0 |  Major | fs/azure | Thomas Marquardt | Thomas Marquardt |
+| [HDFS-12301](https://issues.apache.org/jira/browse/HDFS-12301) | NN File Browser UI: Navigate to a path when enter is pressed |  Trivial | ui | Ravi Prakash | Ravi Prakash |
+| [HDFS-12269](https://issues.apache.org/jira/browse/HDFS-12269) | Better to return a Map rather than HashMap in getErasureCodingCodecs |  Minor | erasure-coding | Huafeng Wang | Huafeng Wang |
+| [YARN-3254](https://issues.apache.org/jira/browse/YARN-3254) | HealthReport should include disk full information |  Major | nodemanager | Akira Ajisaka | Suma Shivaprasad |
+| [HDFS-12072](https://issues.apache.org/jira/browse/HDFS-12072) | Provide fairness between EC and non-EC recovery tasks. |  Major | erasure-coding | Lei (Eddy) Xu | Lei (Eddy) Xu |
+| [HDFS-12117](https://issues.apache.org/jira/browse/HDFS-12117) | HttpFS does not seem to support SNAPSHOT related methods for WebHDFS REST Interface |  Major | httpfs | Wellington Chevreuil | Wellington Chevreuil |
+| [HADOOP-14705](https://issues.apache.org/jira/browse/HADOOP-14705) | Add batched interface reencryptEncryptedKeys to KMS |  Major | kms | Xiao Chen | Xiao Chen |
+| [YARN-7053](https://issues.apache.org/jira/browse/YARN-7053) | Move curator transaction support to ZKCuratorManager |  Major | . | Jonathan Hung | Jonathan Hung |
+| [HADOOP-14251](https://issues.apache.org/jira/browse/HADOOP-14251) | Credential provider should handle property key deprecation |  Critical | security | John Zhuge | John Zhuge |
+| [YARN-7049](https://issues.apache.org/jira/browse/YARN-7049) | FSAppAttempt preemption related fields have confusing names |  Major | fairscheduler | Karthik Kambatla | Karthik Kambatla |
+| [HDFS-11986](https://issues.apache.org/jira/browse/HDFS-11986) | Dfsadmin should report erasure coding related information separately |  Major | erasure-coding, hdfs | Manoj Govindassamy | Manoj Govindassamy |
+| [YARN-6999](https://issues.apache.org/jira/browse/YARN-6999) | Add log about how to solve Error: Could not find or load main class org.apache.hadoop.mapreduce.v2.app.MRAppMaster |  Minor | documentation, security | Linlin Zhou | Linlin Zhou |
+| [YARN-7037](https://issues.apache.org/jira/browse/YARN-7037) | Optimize data transfer with zero-copy approach for containerlogs REST API in NMWebServices |  Major | nodemanager | Tao Yang | Tao Yang |
+| [HDFS-12356](https://issues.apache.org/jira/browse/HDFS-12356) | Unit test for JournalNode sync during Rolling Upgrade |  Major | ha | Hanisha Koneru | Hanisha Koneru |
+| [YARN-6780](https://issues.apache.org/jira/browse/YARN-6780) | ResourceWeights.toString() cleanup |  Minor | scheduler | Daniel Templeton | weiyuan |
+| [YARN-6721](https://issues.apache.org/jira/browse/YARN-6721) | container-executor should have stack checking |  Critical | nodemanager, security | Allen Wittenauer | Allen Wittenauer |
+| [HADOOP-14781](https://issues.apache.org/jira/browse/HADOOP-14781) | Clarify that HADOOP\_CONF\_DIR shouldn't actually be set in hadoop-env.sh |  Major | documentation, scripts | Allen Wittenauer | Allen Wittenauer |
+| [HDFS-12380](https://issues.apache.org/jira/browse/HDFS-12380) | Simplify dataQueue.wait condition logical operation in DataStreamer::run() |  Major | hdfs-client | liaoyuxiangqin | liaoyuxiangqin |
+| [HDFS-12300](https://issues.apache.org/jira/browse/HDFS-12300) | Audit-log delegation token related operations |  Major | namenode | Xiao Chen | Xiao Chen |
+| [YARN-7022](https://issues.apache.org/jira/browse/YARN-7022) | Improve click interaction in queue topology in new YARN UI |  Major | yarn-ui-v2 | Abdullah Yousufi | Abdullah Yousufi |
+| [HDFS-12182](https://issues.apache.org/jira/browse/HDFS-12182) | BlockManager.metaSave does not distinguish between "under replicated" and "missing" blocks |  Trivial | hdfs | Wellington Chevreuil | Wellington Chevreuil |
+| [HADOOP-14688](https://issues.apache.org/jira/browse/HADOOP-14688) | Intern strings in KeyVersion and EncryptedKeyVersion |  Major | kms | Xiao Chen | Xiao Chen |
+| [HDFS-12377](https://issues.apache.org/jira/browse/HDFS-12377) | Refactor TestReadStripedFileWithDecoding to avoid test timeouts |  Major | erasure-coding | Andrew Wang | Andrew Wang |
+| [HADOOP-14827](https://issues.apache.org/jira/browse/HADOOP-14827) | Allow StopWatch to accept a Timer parameter for tests |  Minor | common, test | Erik Krogen | Erik Krogen |
+| [HDFS-12131](https://issues.apache.org/jira/browse/HDFS-12131) | Add some of the FSNamesystem JMX values as metrics |  Minor | hdfs, namenode | Erik Krogen | Erik Krogen |
+| [HDFS-12402](https://issues.apache.org/jira/browse/HDFS-12402) | Refactor ErasureCodingPolicyManager and related codes |  Major | erasure-coding | SammiChen | SammiChen |
+| [YARN-6930](https://issues.apache.org/jira/browse/YARN-6930) | Admins should be able to explicitly enable specific LinuxContainerRuntime in the NodeManager |  Major | nodemanager | Vinod Kumar Vavilapalli | Shane Kumpf |
+| [HADOOP-14844](https://issues.apache.org/jira/browse/HADOOP-14844) | Remove requirement to specify TenantGuid for MSI Token Provider |  Major | fs/adl | Atul Sikaria | Atul Sikaria |
+| [YARN-6799](https://issues.apache.org/jira/browse/YARN-6799) | Remove the duplicated code in CGroupsHandlerImp.java |  Trivial | nodemanager | Yufei Gu | weiyuan |
+| [HADOOP-14520](https://issues.apache.org/jira/browse/HADOOP-14520) | WASB: Block compaction for Azure Block Blobs |  Major | fs/azure | Georgi Chalakov | Georgi Chalakov |
+| [HADOOP-14839](https://issues.apache.org/jira/browse/HADOOP-14839) | DistCp log output should contain copied and deleted files and directories |  Major | tools/distcp | Konstantin Shaposhnikov | Yiqun Lin |
+| [YARN-7132](https://issues.apache.org/jira/browse/YARN-7132) | FairScheduler.initScheduler() contains a surprising unary plus |  Minor | fairscheduler | Daniel Templeton | Yeliang Cang |
+| [HADOOP-14843](https://issues.apache.org/jira/browse/HADOOP-14843) | Improve FsPermission symbolic parsing unit test coverage |  Minor | fs | Jason Lowe | Bharat Viswanadham |
+| [YARN-7057](https://issues.apache.org/jira/browse/YARN-7057) | FSAppAttempt#getResourceUsage doesn't need to consider resources queued for preemption |  Major | fairscheduler | Karthik Kambatla | Karthik Kambatla |
+| [HDFS-12412](https://issues.apache.org/jira/browse/HDFS-12412) | Change ErasureCodingWorker.stripedReadPool to cached thread pool |  Major | erasure-coding | Lei (Eddy) Xu | Lei (Eddy) Xu |
+| [HADOOP-14804](https://issues.apache.org/jira/browse/HADOOP-14804) | correct wrong parameters format order in core-default.xml |  Trivial | . | Chen Hongfei | Chen Hongfei |
+| [HDFS-12409](https://issues.apache.org/jira/browse/HDFS-12409) | Add metrics of execution time of different stages in EC recovery task |  Minor | erasure-coding | Lei (Eddy) Xu | Lei (Eddy) Xu |
+| [HADOOP-14864](https://issues.apache.org/jira/browse/HADOOP-14864) | FSDataInputStream#unbuffer UOE should include stream class name |  Minor | fs | John Zhuge | Bharat Viswanadham |
+| [HADOOP-14869](https://issues.apache.org/jira/browse/HADOOP-14869) | Upgrade apache kerby verion to v1.0.1 |  Major | . | Wei Zhou | Wei Zhou |
+| [MAPREDUCE-6956](https://issues.apache.org/jira/browse/MAPREDUCE-6956) | FileOutputCommitter to gain abstract superclass PathOutputCommitter |  Minor | mrv2 | Steve Loughran | Steve Loughran |
+| [MAPREDUCE-6954](https://issues.apache.org/jira/browse/MAPREDUCE-6954) | Disable erasure coding for files that are uploaded to the MR staging area |  Major | client | Peter Bacsko | Peter Bacsko |
+| [HDFS-12349](https://issues.apache.org/jira/browse/HDFS-12349) | Improve log message when it could not alloc enough blocks for EC |  Minor | erasure-coding | Lei (Eddy) Xu | Lei (Eddy) Xu |
+| [HDFS-12441](https://issues.apache.org/jira/browse/HDFS-12441) | Suppress UnresolvedPathException in namenode log |  Minor | . | Kihwal Lee | Kihwal Lee |
+| [HADOOP-13714](https://issues.apache.org/jira/browse/HADOOP-13714) | Tighten up our compatibility guidelines for Hadoop 3 |  Blocker | documentation | Karthik Kambatla | Daniel Templeton |
+| [HDFS-12472](https://issues.apache.org/jira/browse/HDFS-12472) | Add JUNIT timeout to TestBlockStatsMXBean |  Minor | . | Lei (Eddy) Xu | Bharat Viswanadham |
+| [HDFS-12460](https://issues.apache.org/jira/browse/HDFS-12460) | Make addErasureCodingPolicy an idempotent operation |  Major | erasure-coding | SammiChen | SammiChen |
+| [HDFS-12479](https://issues.apache.org/jira/browse/HDFS-12479) | Some misuses of lock in DFSStripedOutputStream |  Minor | erasure-coding | Huafeng Wang | Huafeng Wang |
+| [MAPREDUCE-6958](https://issues.apache.org/jira/browse/MAPREDUCE-6958) | Shuffle audit logger should log size of shuffle transfer |  Minor | . | Jason Lowe | Jason Lowe |
+| [HDFS-12444](https://issues.apache.org/jira/browse/HDFS-12444) | Reduce runtime of TestWriteReadStripedFile |  Major | erasure-coding, test | Andrew Wang | Huafeng Wang |
+| [HDFS-12445](https://issues.apache.org/jira/browse/HDFS-12445) | Correct spellings of choosen to chosen. |  Trivial | . | hu xiaodong | hu xiaodong |
+| [HADOOP-7308](https://issues.apache.org/jira/browse/HADOOP-7308) | Remove unused TaskLogAppender configurations from log4j.properties |  Major | conf | Todd Lipcon | Todd Lipcon |
+| [HDFS-12496](https://issues.apache.org/jira/browse/HDFS-12496) | Make QuorumJournalManager timeout properties configurable |  Major | . | Ajay Kumar | Ajay Kumar |
+| [HDFS-12530](https://issues.apache.org/jira/browse/HDFS-12530) | Processor argument in Offline Image Viewer should be case insensitive |  Minor | tools | Hanisha Koneru | Hanisha Koneru |
+| [HDFS-12304](https://issues.apache.org/jira/browse/HDFS-12304) | Remove unused parameter from FsDatasetImpl#addVolume |  Minor | . | Chen Liang | Chen Liang |
+| [YARN-65](https://issues.apache.org/jira/browse/YARN-65) | Reduce RM app memory footprint once app has completed |  Major | resourcemanager | Jason Lowe | Manikandan R |
+
+
+### BUG FIXES:
+
+| JIRA | Summary | Priority | Component | Reporter | Contributor |
+|:---- |:---- | :--- |:---- |:---- |:---- |
+| [YARN-6093](https://issues.apache.org/jira/browse/YARN-6093) | Minor bugs with AMRMtoken renewal and state store availability when using FederationRMFailoverProxyProvider during RM failover |  Minor | amrmproxy, federation | Botong Huang | Botong Huang |
+| [YARN-6370](https://issues.apache.org/jira/browse/YARN-6370) | Properly handle rack requests for non-active subclusters in LocalityMulticastAMRMProxyPolicy |  Minor | . | Botong Huang | Botong Huang |
+| [HDFS-12043](https://issues.apache.org/jira/browse/HDFS-12043) | Add counters for block re-replication |  Major | . | Chen Liang | Chen Liang |
+| [YARN-6694](https://issues.apache.org/jira/browse/YARN-6694) | Add certain envs to the default yarn.nodemanager.env-whitelist |  Major | . | Jian He | Jian He |
+| [MAPREDUCE-6905](https://issues.apache.org/jira/browse/MAPREDUCE-6905) | Fix meaningless operations in TestDFSIO in some situation. |  Major | tools/rumen | LiXin Ge | LiXin Ge |
+| [HDFS-12079](https://issues.apache.org/jira/browse/HDFS-12079) | Description of dfs.block.invalidate.limit is incorrect in hdfs-default.xml |  Minor | documentation | Weiwei Yang | Weiwei Yang |
+| [HADOOP-13414](https://issues.apache.org/jira/browse/HADOOP-13414) | Hide Jetty Server version header in HTTP responses |  Major | security | Vinayakumar B | Surendra Singh Lilhore |
+| [HDFS-12089](https://issues.apache.org/jira/browse/HDFS-12089) | Fix ambiguous NN retry log message |  Major | webhdfs | Eric Badger | Eric Badger |
+| [HADOOP-14608](https://issues.apache.org/jira/browse/HADOOP-14608) | KMS JMX servlet path not backwards compatible |  Minor | kms | John Zhuge | John Zhuge |
+| [YARN-6708](https://issues.apache.org/jira/browse/YARN-6708) | Nodemanager container crash after ext3 folder limit |  Critical | . | Bibin A Chundatt | Bibin A Chundatt |
+| [HADOOP-14563](https://issues.apache.org/jira/browse/HADOOP-14563) | LoadBalancingKMSClientProvider#warmUpEncryptedKeys swallows IOException |  Major | . | Rushabh S Shah | Rushabh S Shah |
+| [MAPREDUCE-6246](https://issues.apache.org/jira/browse/MAPREDUCE-6246) | DBOutputFormat.java appending extra semicolon to query which is incompatible with DB2 |  Major | mrv1, mrv2 | ramtin | Gergely Novák |
+| [HADOOP-14634](https://issues.apache.org/jira/browse/HADOOP-14634) | Remove jline from main Hadoop pom.xml |  Major | . | Ray Chiang | Ray Chiang |
+| [YARN-6428](https://issues.apache.org/jira/browse/YARN-6428) | Queue AM limit is not honored  in CS always |  Major | . | Bibin A Chundatt | Bibin A Chundatt |
+| [YARN-6770](https://issues.apache.org/jira/browse/YARN-6770) | [Docs] A small mistake in the example of TimelineClient |  Trivial | docs | Jinjiang Ling | Jinjiang Ling |
+| [HADOOP-10829](https://issues.apache.org/jira/browse/HADOOP-10829) | Iteration on CredentialProviderFactory.serviceLoader  is thread-unsafe |  Major | security | Benoy Antony | Benoy Antony |
+| [HDFS-12085](https://issues.apache.org/jira/browse/HDFS-12085) | Reconfigure namenode heartbeat interval fails if the interval was set with time unit |  Minor | hdfs, tools | Weiwei Yang | Weiwei Yang |
+| [HDFS-12052](https://issues.apache.org/jira/browse/HDFS-12052) | Set SWEBHDFS delegation token kind when ssl is enabled in HttpFS |  Major | httpfs, webhdfs | Zoran Dimitrijevic | Zoran Dimitrijevic |
+| [HDFS-12114](https://issues.apache.org/jira/browse/HDFS-12114) | Consistent HttpFS property names |  Major | httpfs | John Zhuge | John Zhuge |
+| [HADOOP-14581](https://issues.apache.org/jira/browse/HADOOP-14581) | Restrict setOwner to list of user when security is enabled in wasb |  Major | fs/azure | Varada Hemeswari | Varada Hemeswari |
+| [YARN-6809](https://issues.apache.org/jira/browse/YARN-6809) | Fix typo in ResourceManagerHA.md |  Trivial | documentation | Akira Ajisaka | Yeliang Cang |
+| [YARN-6797](https://issues.apache.org/jira/browse/YARN-6797) | TimelineWriter does not fully consume the POST response |  Major | timelineclient | Jason Lowe | Jason Lowe |
+| [HDFS-11502](https://issues.apache.org/jira/browse/HDFS-11502) | Datanode UI should display hostname based on JMX bean instead of window.location.hostname |  Major | hdfs | Jeffrey E  Rodriguez | Jeffrey E  Rodriguez |
+| [HADOOP-14646](https://issues.apache.org/jira/browse/HADOOP-14646) | FileContextMainOperationsBaseTest#testListStatusFilterWithSomeMatches never runs |  Minor | test | Andras Bokor | Andras Bokor |
+| [YARN-6654](https://issues.apache.org/jira/browse/YARN-6654) | RollingLevelDBTimelineStore backwards incompatible after fst upgrade |  Blocker | . | Jonathan Eagles | Jonathan Eagles |
+| [YARN-6805](https://issues.apache.org/jira/browse/YARN-6805) | NPE in LinuxContainerExecutor due to null PrivilegedOperationException exit code |  Major | nodemanager | Jason Lowe | Jason Lowe |
+| [YARN-6792](https://issues.apache.org/jira/browse/YARN-6792) | Incorrect XML convertion in NodeIDsInfo and LabelsToNodesInfo |  Blocker | resourcemanager | Giovanni Matteo Fumarola | Giovanni Matteo Fumarola |
+| [YARN-6769](https://issues.apache.org/jira/browse/YARN-6769) | Make schedulables without demand less needy in FairSharePolicy#compare |  Major | fairscheduler | YunFan Zhou | YunFan Zhou |
+| [YARN-6759](https://issues.apache.org/jira/browse/YARN-6759) | Fix TestRMRestart.testRMRestartWaitForPreviousAMToFinish failure |  Major | . | Naganarasimha G R | Naganarasimha G R |
+| [YARN-3260](https://issues.apache.org/jira/browse/YARN-3260) | AM attempt fail to register before RM processes launch event |  Critical | resourcemanager | Jason Lowe | Bibin A Chundatt |
+| [HDFS-12140](https://issues.apache.org/jira/browse/HDFS-12140) | Remove BPOfferService lock contention to get block pool id |  Critical | datanode | Daryn Sharp | Daryn Sharp |
+| [YARN-6625](https://issues.apache.org/jira/browse/YARN-6625) | yarn application -list returns a tracking URL for AM that doesn't work in secured and HA environment |  Major | amrmproxy | Yufei Gu | Yufei Gu |
+| [HDFS-12112](https://issues.apache.org/jira/browse/HDFS-12112) | TestBlockManager#testBlockManagerMachinesArray sometimes fails with NPE |  Minor | . | Wei-Chiu Chuang | Wei-Chiu Chuang |
+| [YARN-6714](https://issues.apache.org/jira/browse/YARN-6714) | IllegalStateException while handling APP\_ATTEMPT\_REMOVED event when async-scheduling enabled in CapacityScheduler |  Major | . | Tao Yang | Tao Yang |
+| [MAPREDUCE-6889](https://issues.apache.org/jira/browse/MAPREDUCE-6889) | Add Job#close API to shutdown MR client services. |  Major | . | Rohith Sharma K S | Rohith Sharma K S |
+| [MAPREDUCE-6910](https://issues.apache.org/jira/browse/MAPREDUCE-6910) | MapReduceTrackingUriPlugin can not return the right URI of history server with HTTPS |  Major | jobhistoryserver | Lantao Jin | Lantao Jin |
+| [HDFS-12154](https://issues.apache.org/jira/browse/HDFS-12154) | Incorrect javadoc description in StorageLocationChecker#check |  Major | . | Nandakumar | Nandakumar |
+| [YARN-6798](https://issues.apache.org/jira/browse/YARN-6798) | Fix NM startup failure with old state store due to version mismatch |  Major | nodemanager | Ray Chiang | Botong Huang |
+| [HADOOP-14637](https://issues.apache.org/jira/browse/HADOOP-14637) | GenericTestUtils.waitFor needs to check condition again after max wait time |  Major | . | Daniel Templeton | Daniel Templeton |
+| [YARN-6819](https://issues.apache.org/jira/browse/YARN-6819) | Application report fails if app rejected due to nodesize |  Major | . | Bibin A Chundatt | Bibin A Chundatt |
+| [HADOOP-14669](https://issues.apache.org/jira/browse/HADOOP-14669) | GenericTestUtils.waitFor should use monotonic time |  Trivial | test | Jason Lowe | Daniel Templeton |
+| [HDFS-12133](https://issues.apache.org/jira/browse/HDFS-12133) | Correct ContentSummaryComputationContext Logger class name. |  Minor | namenode | Surendra Singh Lilhore | Surendra Singh Lilhore |
+| [HDFS-12158](https://issues.apache.org/jira/browse/HDFS-12158) | Secondary Namenode's web interface lack configs for X-FRAME-OPTIONS protection |  Major | namenode | Mukul Kumar Singh | Mukul Kumar Singh |
+| [HDFS-12139](https://issues.apache.org/jira/browse/HDFS-12139) | HTTPFS liststatus returns incorrect pathSuffix for path of file |  Major | httpfs | Yongjun Zhang | Yongjun Zhang |
+| [YARN-6837](https://issues.apache.org/jira/browse/YARN-6837) | Null LocalResource visibility or resource type can crash the nodemanager |  Major | . | Jinjiang Ling | Jinjiang Ling |
+| [HDFS-11472](https://issues.apache.org/jira/browse/HDFS-11472) | Fix inconsistent replica size after a data pipeline failure |  Critical | datanode | Wei-Chiu Chuang | Erik Krogen |
+| [HDFS-12166](https://issues.apache.org/jira/browse/HDFS-12166) | Do not deprecate HTTPFS\_TEMP |  Minor | httpfs | John Zhuge | John Zhuge |
+| [HDFS-11742](https://issues.apache.org/jira/browse/HDFS-11742) | Improve balancer usability after HDFS-8818 |  Blocker | . | Kihwal Lee | Kihwal Lee |
+| [HDFS-12094](https://issues.apache.org/jira/browse/HDFS-12094) | Log torrent when none isa-l EC is used. |  Minor | erasure-coding | LiXin Ge | LiXin Ge |
+| [HDFS-12176](https://issues.apache.org/jira/browse/HDFS-12176) | dfsadmin shows DFS Used%: NaN% if the cluster has zero block. |  Trivial | . | Wei-Chiu Chuang | Weiwei Yang |
+| [YARN-6844](https://issues.apache.org/jira/browse/YARN-6844) | AMRMClientImpl.checkNodeLabelExpression() has wrong error message |  Minor | . | Daniel Templeton | Manikandan R |
+| [YARN-6150](https://issues.apache.org/jira/browse/YARN-6150) | TestContainerManagerSecurity tests for Yarn Server are flakey |  Major | test | Daniel Sturman | Daniel Sturman |
+| [YARN-6307](https://issues.apache.org/jira/browse/YARN-6307) | Refactor FairShareComparator#compare |  Major | fairscheduler | Yufei Gu | Yufei Gu |
+| [YARN-6102](https://issues.apache.org/jira/browse/YARN-6102) | RMActiveService context to be updated with new RMContext on failover |  Critical | . | Ajith S | Rohith Sharma K S |
+| [HADOOP-14578](https://issues.apache.org/jira/browse/HADOOP-14578) | Bind IPC connections to kerberos UPN host for proxy users |  Major | ipc | Daryn Sharp | Daryn Sharp |
+| [HDFS-11896](https://issues.apache.org/jira/browse/HDFS-11896) | Non-dfsUsed will be doubled on dead node re-registration |  Blocker | . | Brahma Reddy Battula | Brahma Reddy Battula |
+| [HADOOP-14692](https://issues.apache.org/jira/browse/HADOOP-14692) | Upgrade Apache Rat |  Trivial | build | Allen Wittenauer | Allen Wittenauer |
+| [HDFS-12062](https://issues.apache.org/jira/browse/HDFS-12062) | removeErasureCodingPolicy needs super user permission |  Critical | erasure-coding | Wei-Chiu Chuang | Wei-Chiu Chuang |
+| [HDFS-12044](https://issues.apache.org/jira/browse/HDFS-12044) | Mismatch between BlockManager#maxReplicationStreams and ErasureCodingWorker.stripedReconstructionPool pool size causes slow and bursty recovery |  Major | erasure-coding | Lei (Eddy) Xu | Lei (Eddy) Xu |
+| [HADOOP-14229](https://issues.apache.org/jira/browse/HADOOP-14229) | hadoop.security.auth\_to\_local example is incorrect in the documentation |  Major | . | Andras Bokor | Andras Bokor |
+| [YARN-6870](https://issues.apache.org/jira/browse/YARN-6870) | Fix floating point inaccuracies in resource availability check in AllocationBasedResourceUtilizationTracker |  Major | api, nodemanager | Brook Zhou | Brook Zhou |
+| [YARN-5728](https://issues.apache.org/jira/browse/YARN-5728) | TestMiniYarnClusterNodeUtilization.testUpdateNodeUtilization timeout |  Major | test | Akira Ajisaka | Akira Ajisaka |
+| [HADOOP-14677](https://issues.apache.org/jira/browse/HADOOP-14677) | mvn clean compile fails |  Major | build | Andras Bokor | Andras Bokor |
+| [HDFS-12082](https://issues.apache.org/jira/browse/HDFS-12082) | BlockInvalidateLimit value is incorrectly set after namenode heartbeat interval reconfigured |  Major | hdfs, namenode | Weiwei Yang | Weiwei Yang |
+| [MAPREDUCE-6924](https://issues.apache.org/jira/browse/MAPREDUCE-6924) | Revert MAPREDUCE-6199 MAPREDUCE-6286 and MAPREDUCE-5875 |  Major | . | Andrew Wang | Junping Du |
+| [HADOOP-14420](https://issues.apache.org/jira/browse/HADOOP-14420) | generateReports property is not applicable for maven-site-plugin:attach-descriptor goal |  Minor | . | Andras Bokor | Andras Bokor |
+| [HADOOP-14644](https://issues.apache.org/jira/browse/HADOOP-14644) | Increase max heap size of Maven javadoc plugin |  Major | test | Andras Bokor | Andras Bokor |
+| [HADOOP-14343](https://issues.apache.org/jira/browse/HADOOP-14343) | Wrong pid file name in error message when starting secure daemon |  Minor | . | Andras Bokor | Andras Bokor |
+| [MAPREDUCE-6921](https://issues.apache.org/jira/browse/MAPREDUCE-6921) | TestUmbilicalProtocolWithJobToken#testJobTokenRpc fails |  Major | . | Sonia Garudi | Sonia Garudi |
+| [HADOOP-14676](https://issues.apache.org/jira/browse/HADOOP-14676) | Wrong default value for "fs.df.interval" |  Major | common, conf, fs | Konstantin Shvachko | xiangguang zheng |
+| [HADOOP-14701](https://issues.apache.org/jira/browse/HADOOP-14701) | Configuration can log misleading warnings about an attempt to override final parameter |  Major | conf | Andrew Sherman | Andrew Sherman |
+| [YARN-5731](https://issues.apache.org/jira/browse/YARN-5731) | Preemption calculation is not accurate when reserved containers are present in queue. |  Major | capacity scheduler | Sunil G | Wangda Tan |
+| [HADOOP-14683](https://issues.apache.org/jira/browse/HADOOP-14683) | FileStatus.compareTo binary compatible issue |  Blocker | . | Sergey Shelukhin | Akira Ajisaka |
+| [HDFS-12107](https://issues.apache.org/jira/browse/HDFS-12107) | FsDatasetImpl#removeVolumes floods the logs when removing the volume |  Major | . | Haohui Mai | Kelvin Chu |
+| [HADOOP-14702](https://issues.apache.org/jira/browse/HADOOP-14702) | Fix formatting issue and regression caused by conversion from APT to Markdown |  Minor | documentation | Doris Gu | Doris Gu |
+| [YARN-6872](https://issues.apache.org/jira/browse/YARN-6872) | Ensure apps could run given NodeLabels are disabled post RM switchover/restart |  Major | resourcemanager | Sunil G | Sunil G |
+| [HDFS-12217](https://issues.apache.org/jira/browse/HDFS-12217) | HDFS snapshots doesn't capture all open files when one of the open files is deleted |  Major | snapshots | Manoj Govindassamy | Manoj Govindassamy |
+| [YARN-6846](https://issues.apache.org/jira/browse/YARN-6846) | Nodemanager can fail to fully delete application local directories when applications are killed |  Critical | nodemanager | Jason Lowe | Jason Lowe |
+| [YARN-6678](https://issues.apache.org/jira/browse/YARN-6678) | Handle IllegalStateException in Async Scheduling mode of CapacityScheduler |  Major | capacityscheduler | Tao Yang | Tao Yang |
+| [YARN-6932](https://issues.apache.org/jira/browse/YARN-6932) | Fix TestFederationRMFailoverProxyProvider test case failure |  Major | . | Arun Suresh | Subru Krishnan |
+| [YARN-6895](https://issues.apache.org/jira/browse/YARN-6895) | [FairScheduler] Preemption reservation may cause regular reservation leaks |  Blocker | fairscheduler | Miklos Szegedi | Miklos Szegedi |
+| [HADOOP-13963](https://issues.apache.org/jira/browse/HADOOP-13963) | /bin/bash is hard coded in some of the scripts |  Major | scripts | Miklos Szegedi | Ajay Kumar |
+| [HADOOP-14722](https://issues.apache.org/jira/browse/HADOOP-14722) | Azure: BlockBlobInputStream position incorrect after seek |  Major | fs/azure | Thomas Marquardt | Thomas Marquardt |
+| [YARN-6951](https://issues.apache.org/jira/browse/YARN-6951) | Fix debug log when Resource Handler chain is enabled |  Minor | . | Yang Wang | Yang Wang |
+| [HADOOP-14727](https://issues.apache.org/jira/browse/HADOOP-14727) | Socket not closed properly when reading Configurations with BlockReaderRemote |  Blocker | conf | Xiao Chen | Jonathan Eagles |
+| [YARN-6920](https://issues.apache.org/jira/browse/YARN-6920) | Fix resource leak that happens during container re-initialization. |  Major | nodemanager | Arun Suresh | Arun Suresh |
+| [HADOOP-14730](https://issues.apache.org/jira/browse/HADOOP-14730) | Support protobuf FileStatus in AdlFileSystem |  Major | . | Vishwajeet Dusane | Chris Douglas |
+| [HDFS-12198](https://issues.apache.org/jira/browse/HDFS-12198) | Document missing namenode metrics that were added recently |  Minor | documentation | Yiqun Lin | Yiqun Lin |
+| [HADOOP-14680](https://issues.apache.org/jira/browse/HADOOP-14680) | Azure: IndexOutOfBoundsException in BlockBlobInputStream |  Minor | fs/azure | Rajesh Balamohan | Thomas Marquardt |
+| [YARN-6757](https://issues.apache.org/jira/browse/YARN-6757) | Refactor the usage of yarn.nodemanager.linux-container-executor.cgroups.mount-path |  Minor | nodemanager | Miklos Szegedi | Miklos Szegedi |
+| [MAPREDUCE-6927](https://issues.apache.org/jira/browse/MAPREDUCE-6927) | MR job should only set tracking url if history was successfully written |  Major | . | Eric Badger | Eric Badger |
+| [YARN-6890](https://issues.apache.org/jira/browse/YARN-6890) | If UI is not secured, we allow user to kill other users' job even yarn cluster is secured. |  Critical | . | Sumana Sathish | Junping Du |
+| [HDFS-10326](https://issues.apache.org/jira/browse/HDFS-10326) | Disable setting tcp socket send/receive buffers for write pipelines |  Major | datanode, hdfs | Daryn Sharp | Daryn Sharp |
+| [HADOOP-14598](https://issues.apache.org/jira/browse/HADOOP-14598) | Blacklist Http/HttpsFileSystem in FsUrlStreamHandlerFactory |  Major | fs/azure, test | Steve Loughran | Steve Loughran |
+| [YARN-6515](https://issues.apache.org/jira/browse/YARN-6515) | Fix warnings from Spotbugs in hadoop-yarn-server-nodemanager |  Major | nodemanager | Naganarasimha G R | Naganarasimha G R |
+| [HDFS-12157](https://issues.apache.org/jira/browse/HDFS-12157) | Do fsyncDirectory(..) outside of FSDataset lock |  Critical | datanode | Vinayakumar B | Vinayakumar B |
+| [HDFS-12278](https://issues.apache.org/jira/browse/HDFS-12278) | LeaseManager operations are inefficient in 2.8. |  Blocker | namenode | Rushabh S Shah | Rushabh S Shah |
+| [HADOOP-14743](https://issues.apache.org/jira/browse/HADOOP-14743) | CompositeGroupsMapping should not swallow exceptions |  Major | security | Wei-Chiu Chuang | Wei-Chiu Chuang |
+| [HADOOP-14754](https://issues.apache.org/jira/browse/HADOOP-14754) | TestCommonConfigurationFields failed: core-default.xml has 2 wasb properties missing in classes |  Minor | common, fs/azure | John Zhuge | John Zhuge |
+| [HADOOP-14760](https://issues.apache.org/jira/browse/HADOOP-14760) | Add missing override to LoadBalancingKMSClientProvider |  Minor | kms | Xiao Chen | Xiao Chen |
+| [YARN-5927](https://issues.apache.org/jira/browse/YARN-5927) | BaseContainerManagerTest::waitForNMContainerState timeout accounting is not accurate |  Trivial | . | Miklos Szegedi | Kai Sasaki |
+| [YARN-6967](https://issues.apache.org/jira/browse/YARN-6967) | Limit application attempt's diagnostic message size thoroughly |  Major | resourcemanager | Chengbing Liu | Chengbing Liu |
+| [HDFS-11303](https://issues.apache.org/jira/browse/HDFS-11303) | Hedged read might hang infinitely if read data from all DN failed |  Major | hdfs-client | Chen Zhang | Chen Zhang |
+| [YARN-6996](https://issues.apache.org/jira/browse/YARN-6996) | Change javax.cache library implementation from JSR107 to Apache Geronimo |  Blocker | . | Ray Chiang | Ray Chiang |
+| [YARN-6987](https://issues.apache.org/jira/browse/YARN-6987) | Log app attempt during InvalidStateTransition |  Major | . | Jonathan Eagles | Jonathan Eagles |
+| [HDFS-11696](https://issues.apache.org/jira/browse/HDFS-11696) | Fix warnings from Spotbugs in hadoop-hdfs |  Major | . | Yiqun Lin | Yiqun Lin |
+| [HDFS-12066](https://issues.apache.org/jira/browse/HDFS-12066) | When Namenode is in safemode,may not allowed to remove an user's erasure coding policy |  Major | hdfs | lufei | lufei |
+| [HDFS-12054](https://issues.apache.org/jira/browse/HDFS-12054) | FSNamesystem#addErasureCodingPolicies should call checkNameNodeSafeMode() to ensure Namenode is not in safemode |  Major | hdfs | lufei | lufei |
+| [YARN-7014](https://issues.apache.org/jira/browse/YARN-7014) | container-executor has off-by-one error which can corrupt the heap |  Critical | yarn | Shane Kumpf | Jason Lowe |
+| [HADOOP-14773](https://issues.apache.org/jira/browse/HADOOP-14773) | Extend ZKCuratorManager API for more reusability |  Major | . | Íñigo Goiri | Íñigo Goiri |
+| [YARN-6965](https://issues.apache.org/jira/browse/YARN-6965) | Duplicate instantiation in FairSchedulerQueueInfo |  Minor | fairscheduler | Masahiro Tanaka | Masahiro Tanaka |
+| [YARN-7020](https://issues.apache.org/jira/browse/YARN-7020) | TestAMRMProxy#testAMRMProxyTokenRenewal is flakey |  Major | . | Robert Kanter | Robert Kanter |
+| [MAPREDUCE-6940](https://issues.apache.org/jira/browse/MAPREDUCE-6940) | Copy-paste error in the TaskAttemptUnsuccessfulCompletionEvent constructor |  Minor | . | Oleg Danilov | Oleg Danilov |
+| [MAPREDUCE-6936](https://issues.apache.org/jira/browse/MAPREDUCE-6936) | Remove unnecessary dependency of hadoop-yarn-server-common from hadoop-mapreduce-client-common |  Major | mrv2 | Haibo Chen | Haibo Chen |
+| [HDFS-12250](https://issues.apache.org/jira/browse/HDFS-12250) | Reduce usage of FsPermissionExtension in unit tests |  Minor | test | Chris Douglas | Chris Douglas |
+| [HDFS-12316](https://issues.apache.org/jira/browse/HDFS-12316) | Verify HDFS snapshot deletion doesn't crash the ongoing file writes |  Major | . | Manoj Govindassamy | Manoj Govindassamy |
+| [YARN-7007](https://issues.apache.org/jira/browse/YARN-7007) | NPE in RM while using YarnClient.getApplications() |  Major | . | Lingfeng Su | Lingfeng Su |
+| [HDFS-12325](https://issues.apache.org/jira/browse/HDFS-12325) | SFTPFileSystem operations should restore cwd |  Major | . | Namit Maheshwari | Chen Liang |
+| [HDFS-11738](https://issues.apache.org/jira/browse/HDFS-11738) | Hedged pread takes more time when block moved from initial locations |  Major | hdfs-client | Vinayakumar B | Vinayakumar B |
+| [YARN-2416](https://issues.apache.org/jira/browse/YARN-2416) | InvalidStateTransitonException in ResourceManager if AMLauncher does not receive response for startContainers() call in time |  Critical | resourcemanager | Jian Fang | Jonathan Eagles |
+| [YARN-7048](https://issues.apache.org/jira/browse/YARN-7048) | Fix tests faking kerberos to explicitly set ugi auth type |  Major | yarn | Daryn Sharp | Daryn Sharp |
+| [HADOOP-14687](https://issues.apache.org/jira/browse/HADOOP-14687) | AuthenticatedURL will reuse bad/expired session cookies |  Critical | common | Daryn Sharp | Daryn Sharp |
+| [YARN-6251](https://issues.apache.org/jira/browse/YARN-6251) | Do async container release to prevent deadlock during container updates |  Major | . | Arun Suresh | Arun Suresh |
+| [YARN-7090](https://issues.apache.org/jira/browse/YARN-7090) | testRMRestartAfterNodeLabelDisabled get failed when CapacityScheduler is configured |  Major | test | Yesha Vora | Wangda Tan |
+| [HDFS-12318](https://issues.apache.org/jira/browse/HDFS-12318) | Fix IOException condition for openInfo in DFSInputStream |  Major | . | legend | legend |
+| [YARN-7074](https://issues.apache.org/jira/browse/YARN-7074) | Fix NM state store update comment |  Minor | nodemanager | Botong Huang | Botong Huang |
+| [HDFS-12344](https://issues.apache.org/jira/browse/HDFS-12344) | LocatedFileStatus regression: no longer accepting null FSPermission |  Minor | fs | Ewan Higgs | Ewan Higgs |
+| [YARN-6640](https://issues.apache.org/jira/browse/YARN-6640) |  AM heartbeat stuck when responseId overflows MAX\_INT |  Blocker | . | Botong Huang | Botong Huang |
+| [HDFS-12319](https://issues.apache.org/jira/browse/HDFS-12319) | DirectoryScanner will throw IllegalStateException when Multiple BP's are present |  Blocker | . | Brahma Reddy Battula | Brahma Reddy Battula |
+| [HDFS-12299](https://issues.apache.org/jira/browse/HDFS-12299) | Race Between update pipeline and DN Re-Registration |  Critical | . | Brahma Reddy Battula | Brahma Reddy Battula |
+| [YARN-7052](https://issues.apache.org/jira/browse/YARN-7052) | RM SchedulingMonitor gives no indication why the spawned thread crashed. |  Critical | yarn | Eric Payne | Eric Payne |
+| [YARN-7087](https://issues.apache.org/jira/browse/YARN-7087) | NM failed to perform log aggregation due to absent container |  Blocker | log-aggregation | Jason Lowe | Jason Lowe |
+| [HDFS-12215](https://issues.apache.org/jira/browse/HDFS-12215) | DataNode#transferBlock does not create its daemon in the xceiver thread group |  Major | datanode | Lei (Eddy) Xu | Lei (Eddy) Xu |
+| [HDFS-12248](https://issues.apache.org/jira/browse/HDFS-12248) | SNN will not upload fsimage on IOE and Interrupted exceptions |  Critical | ha, namenode, rolling upgrades | Brahma Reddy Battula | Brahma Reddy Battula |
+| [HDFS-12358](https://issues.apache.org/jira/browse/HDFS-12358) | Handle IOException when transferring edit log to Journal current dir through JN sync |  Major | . | Hanisha Koneru | Hanisha Koneru |
+| [MAPREDUCE-6945](https://issues.apache.org/jira/browse/MAPREDUCE-6945) | TestMapFileOutputFormat missing @after annotation |  Minor | . | Ajay Kumar | Ajay Kumar |
+| [YARN-7051](https://issues.apache.org/jira/browse/YARN-7051) | Avoid concurrent modification exception in FifoIntraQueuePreemptionPlugin |  Critical | capacity scheduler, scheduler preemption, yarn | Eric Payne | Eric Payne |
+| [YARN-7099](https://issues.apache.org/jira/browse/YARN-7099) | ResourceHandlerModule.parseConfiguredCGroupPath only works for privileged yarn users. |  Minor | nodemanager | Miklos Szegedi | Miklos Szegedi |
+| [YARN-7112](https://issues.apache.org/jira/browse/YARN-7112) | TestAMRMProxy is failing with invalid request |  Major | . | Jason Lowe | Jason Lowe |
+| [HDFS-12293](https://issues.apache.org/jira/browse/HDFS-12293) | DataNode should log file name on disk error |  Major | datanode | Wei-Chiu Chuang | Ajay Kumar |
+| [YARN-7076](https://issues.apache.org/jira/browse/YARN-7076) | yarn application -list -appTypes \<appType\> is not working |  Blocker | . | Jian He | Jian He |
+| [YARN-5816](https://issues.apache.org/jira/browse/YARN-5816) | TestDelegationTokenRenewer#testCancelWithMultipleAppSubmissions is still flakey |  Minor | resourcemanager, test | Daniel Templeton | Robert Kanter |
+| [YARN-6756](https://issues.apache.org/jira/browse/YARN-6756) | ContainerRequest#executionTypeRequest causes NPE |  Critical | . | Jian He | Jian He |
+| [HDFS-12191](https://issues.apache.org/jira/browse/HDFS-12191) | Provide option to not capture the accessTime change of a file to snapshot if no other modification has been done to this file |  Major | hdfs, namenode | Yongjun Zhang | Yongjun Zhang |
+| [YARN-6982](https://issues.apache.org/jira/browse/YARN-6982) | Potential issue on setting AMContainerSpec#tokenConf to null before app is completed |  Major | . | Rohith Sharma K S | Manikandan R |
+| [YARN-6868](https://issues.apache.org/jira/browse/YARN-6868) | Add test scope to certain entries in hadoop-yarn-server-resourcemanager pom.xml |  Major | yarn | Ray Chiang | Ray Chiang |
+| [HDFS-12336](https://issues.apache.org/jira/browse/HDFS-12336) | Listing encryption zones still fails when deleted EZ is not a direct child of snapshottable directory |  Minor | encryption, hdfs | Wellington Chevreuil | Wellington Chevreuil |
+| [HADOOP-14814](https://issues.apache.org/jira/browse/HADOOP-14814) | Fix incompatible API change on FsServerDefaults to HADOOP-14104 |  Blocker | . | Junping Du | Junping Du |
+| [MAPREDUCE-6931](https://issues.apache.org/jira/browse/MAPREDUCE-6931) | Remove TestDFSIO "Total Throughput" calculation |  Critical | benchmarks, test | Dennis Huo | Dennis Huo |
+| [YARN-7115](https://issues.apache.org/jira/browse/YARN-7115) | Move BoundedAppender to org.hadoop.yarn.util pacakge |  Major | . | Jian He | Jian He |
+| [YARN-7077](https://issues.apache.org/jira/browse/YARN-7077) | TestAMSimulator and TestNMSimulator fail |  Major | test | Akira Ajisaka | Akira Ajisaka |
+| [YARN-7116](https://issues.apache.org/jira/browse/YARN-7116) | CapacityScheduler Web UI: Queue's AM usage is always show on per-user's AM usage. |  Major | capacity scheduler, webapp | Wangda Tan | Wangda Tan |
+| [YARN-6726](https://issues.apache.org/jira/browse/YARN-6726) | Fix issues with docker commands executed by container-executor |  Major | nodemanager | Shane Kumpf | Shane Kumpf |
+| [HADOOP-14364](https://issues.apache.org/jira/browse/HADOOP-14364) | refresh changelog/release notes with newer Apache Yetus build |  Major | build, documentation | Allen Wittenauer | Allen Wittenauer |
+| [HDFS-12317](https://issues.apache.org/jira/browse/HDFS-12317) | HDFS metrics render error in the page of Github |  Minor | documentation, metrics | Yiqun Lin | Yiqun Lin |
+| [HADOOP-14824](https://issues.apache.org/jira/browse/HADOOP-14824) | Update ADLS SDK to 2.2.2 for MSI fix |  Major | fs/adl | Atul Sikaria | Atul Sikaria |
+| [HDFS-12363](https://issues.apache.org/jira/browse/HDFS-12363) | Possible NPE in BlockManager$StorageInfoDefragmenter#scanAndCompactStorages |  Major | namenode | Xiao Chen | Xiao Chen |
+| [YARN-7141](https://issues.apache.org/jira/browse/YARN-7141) | Move logging APIs to slf4j in timelineservice after ATSv2 merge |  Minor | . | Varun Saxena | Varun Saxena |
+| [HDFS-11964](https://issues.apache.org/jira/browse/HDFS-11964) | Decoding inputs should be correctly prepared in pread |  Major | erasure-coding | Lei (Eddy) Xu | Takanobu Asanuma |
+| [YARN-7120](https://issues.apache.org/jira/browse/YARN-7120) | CapacitySchedulerPage NPE in "Aggregate scheduler counts" section |  Minor | . | Eric Payne | Eric Payne |
+| [HADOOP-14674](https://issues.apache.org/jira/browse/HADOOP-14674) | Correct javadoc for getRandomizedTempPath |  Major | common | Mukul Kumar Singh | Mukul Kumar Singh |
+| [YARN-7023](https://issues.apache.org/jira/browse/YARN-7023) | Incorrect ReservationId.compareTo() implementation |  Minor | reservation system | Oleg Danilov | Oleg Danilov |
+| [HDFS-12383](https://issues.apache.org/jira/browse/HDFS-12383) | Re-encryption updater should handle canceled tasks better |  Major | encryption | Xiao Chen | Xiao Chen |
+| [YARN-7152](https://issues.apache.org/jira/browse/YARN-7152) | [ATSv2] Registering timeline client before AMRMClient service init throw exception. |  Major | timelineclient | Rohith Sharma K S | Rohith Sharma K S |
+| [HDFS-12388](https://issues.apache.org/jira/browse/HDFS-12388) | A bad error message in DFSStripedOutputStream |  Major | erasure-coding | Kai Zheng | Huafeng Wang |
+| [HADOOP-14820](https://issues.apache.org/jira/browse/HADOOP-14820) | Wasb mkdirs security checks inconsistent with HDFS |  Major | fs/azure | Sivaguru Sankaridurg | Sivaguru Sankaridurg |
+| [HDFS-12359](https://issues.apache.org/jira/browse/HDFS-12359) | Re-encryption should operate with minimum KMS ACL requirements. |  Major | encryption | Xiao Chen | Xiao Chen |
+| [HDFS-11882](https://issues.apache.org/jira/browse/HDFS-11882) | Precisely calculate acked length of striped block groups in updatePipeline |  Critical | erasure-coding, test | Akira Ajisaka | Andrew Wang |
+| [HDFS-12392](https://issues.apache.org/jira/browse/HDFS-12392) | Writing striped file failed due to different cell size |  Major | erasure-coding | SammiChen | SammiChen |
+| [YARN-7164](https://issues.apache.org/jira/browse/YARN-7164) | TestAMRMClientOnRMRestart fails sporadically with bind address in use |  Major | test | Jason Lowe | Jason Lowe |
+| [YARN-6992](https://issues.apache.org/jira/browse/YARN-6992) | Kill application button is visible even if the application is FINISHED in RM UI |  Major | . | Sumana Sathish | Suma Shivaprasad |
+| [HDFS-12357](https://issues.apache.org/jira/browse/HDFS-12357) | Let NameNode to bypass external attribute provider for special user |  Major | . | Yongjun Zhang | Yongjun Zhang |
+| [HDFS-12369](https://issues.apache.org/jira/browse/HDFS-12369) | Edit log corruption due to hard lease recovery of not-closed file which has snapshots |  Major | namenode | Xiao Chen | Xiao Chen |
+| [HDFS-12404](https://issues.apache.org/jira/browse/HDFS-12404) | Rename hdfs config authorization.provider.bypass.users to attributes.provider.bypass.users |  Major | hdfs | Yongjun Zhang | Manoj Govindassamy |
+| [HDFS-12400](https://issues.apache.org/jira/browse/HDFS-12400) | Provide a way for NN to drain the local key cache before re-encryption |  Major | encryption | Xiao Chen | Xiao Chen |
+| [YARN-7140](https://issues.apache.org/jira/browse/YARN-7140) | CollectorInfo should have Public visibility |  Minor | . | Varun Saxena | Varun Saxena |
+| [YARN-7130](https://issues.apache.org/jira/browse/YARN-7130) | ATSv2 documentation changes post merge |  Major | timelineserver | Varun Saxena | Varun Saxena |
+| [HADOOP-14851](https://issues.apache.org/jira/browse/HADOOP-14851) | LambdaTestUtils.eventually() doesn't spin on Assertion failures |  Major | test | Steve Loughran | Steve Loughran |
+| [YARN-7181](https://issues.apache.org/jira/browse/YARN-7181) | CPUTimeTracker.updateElapsedJiffies can report negative usage |  Major | . | Miklos Szegedi | Miklos Szegedi |
+| [HDFS-12417](https://issues.apache.org/jira/browse/HDFS-12417) | Disable flaky TestDFSStripedOutputStreamWithFailure |  Major | test | Chris Douglas | Andrew Wang |
+| [HADOOP-14856](https://issues.apache.org/jira/browse/HADOOP-14856) | Fix AWS, Jetty, HBase, Ehcache entries for NOTICE.txt |  Major | . | Ray Chiang | Ray Chiang |
+| [HDFS-12407](https://issues.apache.org/jira/browse/HDFS-12407) | Journal nodes fails to shutdown cleanly if JournalNodeHttpServer or JournalNodeRpcServer fails to start |  Major | . | Ajay Kumar | Ajay Kumar |
+| [YARN-7185](https://issues.apache.org/jira/browse/YARN-7185) | ContainerScheduler should only look at availableResource for GUARANTEED containers when OPPORTUNISTIC container queuing is enabled. |  Blocker | yarn | Sumana Sathish | Tan, Wangda |
+| [HDFS-12222](https://issues.apache.org/jira/browse/HDFS-12222) | Document and test BlockLocation for erasure-coded files |  Major | . | Andrew Wang | Huafeng Wang |
+| [HADOOP-14867](https://issues.apache.org/jira/browse/HADOOP-14867) | Update HDFS Federation setup document, for incorrect property name for secondary name node http address |  Major | . | Bharat Viswanadham | Bharat Viswanadham |
+| [YARN-4727](https://issues.apache.org/jira/browse/YARN-4727) | Unable to override the $HADOOP\_CONF\_DIR env variable for container |  Major | nodemanager | Terence Yim | Jason Lowe |
+| [MAPREDUCE-6957](https://issues.apache.org/jira/browse/MAPREDUCE-6957) | shuffle hangs after a node manager connection timeout |  Major | mrv2 | Jooseong Kim | Jooseong Kim |
+| [YARN-7146](https://issues.apache.org/jira/browse/YARN-7146) | Many RM unit tests failing with FairScheduler |  Major | test | Robert Kanter | Robert Kanter |
+| [HDFS-12457](https://issues.apache.org/jira/browse/HDFS-12457) | Revert HDFS-11156 Add new op GETFILEBLOCKLOCATIONS to WebHDFS REST API |  Major | webhdfs | Andrew Wang | Andrew Wang |
+| [HDFS-12378](https://issues.apache.org/jira/browse/HDFS-12378) | TestClientProtocolForPipelineRecovery#testZeroByteBlockRecovery fails on trunk |  Blocker | test | Xiao Chen | Lei (Eddy) Xu |
+| [HDFS-12456](https://issues.apache.org/jira/browse/HDFS-12456) | TestNamenodeMetrics.testSyncAndBlockReportMetric fails |  Minor | hdfs, metrics | Bharat Viswanadham | Bharat Viswanadham |
+| [YARN-7163](https://issues.apache.org/jira/browse/YARN-7163) | RMContext need not to be injected to webapp and other Always Running services. |  Blocker | resourcemanager | Rohith Sharma K S | Rohith Sharma K S |
+| [HDFS-12424](https://issues.apache.org/jira/browse/HDFS-12424) | Datatable sorting on the Datanode Information page in the Namenode UI is broken |  Major | . | Shawna Martell | Shawna Martell |
+| [HADOOP-14853](https://issues.apache.org/jira/browse/HADOOP-14853) | hadoop-mapreduce-client-app is not a client module |  Major | . | Haibo Chen | Haibo Chen |
+| [HDFS-12323](https://issues.apache.org/jira/browse/HDFS-12323) | NameNode terminates after full GC thinking QJM unresponsive if full GC is much longer than timeout |  Major | namenode, qjm | Erik Krogen | Erik Krogen |
+| [HDFS-10701](https://issues.apache.org/jira/browse/HDFS-10701) | TestDFSStripedOutputStreamWithFailure#testBlockTokenExpired occasionally fails |  Major | erasure-coding | Wei-Chiu Chuang | SammiChen |
+| [YARN-6977](https://issues.apache.org/jira/browse/YARN-6977) | Node information is not provided for non am containers in RM logs |  Major | capacity scheduler | Sumana Sathish | Suma Shivaprasad |
+| [YARN-7149](https://issues.apache.org/jira/browse/YARN-7149) | Cross-queue preemption sometimes starves an underserved queue |  Major | capacity scheduler | Eric Payne | Eric Payne |
+| [YARN-7192](https://issues.apache.org/jira/browse/YARN-7192) | Add a pluggable StateMachine Listener that is notified of NM Container State changes |  Major | . | Arun Suresh | Arun Suresh |
+| [HADOOP-14835](https://issues.apache.org/jira/browse/HADOOP-14835) | mvn site build throws SAX errors |  Blocker | build, site | Allen Wittenauer | Andrew Wang |
+| [MAPREDUCE-6960](https://issues.apache.org/jira/browse/MAPREDUCE-6960) | Shuffle Handler prints disk error stack traces for every read failure. |  Major | . | Kuhu Shukla | Kuhu Shukla |
+| [HDFS-12480](https://issues.apache.org/jira/browse/HDFS-12480) | TestNameNodeMetrics#testTransactionAndCheckpointMetrics Fails in trunk |  Blocker | test | Brahma Reddy Battula | Hanisha Koneru |
+| [HDFS-11799](https://issues.apache.org/jira/browse/HDFS-11799) | Introduce a config to allow setting up write pipeline with fewer nodes than replication factor |  Major | . | Yongjun Zhang | Brahma Reddy Battula |
+| [HDFS-12449](https://issues.apache.org/jira/browse/HDFS-12449) | TestReconstructStripedFile.testNNSendsErasureCodingTasks randomly cannot finish in 60s |  Major | erasure-coding | SammiChen | SammiChen |
+| [HDFS-12437](https://issues.apache.org/jira/browse/HDFS-12437) | Fix test setup in TestLeaseRecoveryStriped |  Major | erasure-coding, test | Arpit Agarwal | Andrew Wang |
+| [YARN-7196](https://issues.apache.org/jira/browse/YARN-7196) | Fix finicky TestContainerManager tests |  Major | . | Arun Suresh | Arun Suresh |
+| [YARN-6771](https://issues.apache.org/jira/browse/YARN-6771) | Use classloader inside configuration class to make new classes |  Major | . | Jongyoul Lee | Jongyoul Lee |
+| [HDFS-12526](https://issues.apache.org/jira/browse/HDFS-12526) | FSDirectory should use Time.monotonicNow for durations |  Minor | . | Chetna Chaudhari | Bharat Viswanadham |
+| [YARN-6968](https://issues.apache.org/jira/browse/YARN-6968) | Hardcoded absolute pathname in DockerLinuxContainerRuntime |  Major | nodemanager | Miklos Szegedi | Eric Badger |
+| [HDFS-12371](https://issues.apache.org/jira/browse/HDFS-12371) | "BlockVerificationFailures" and "BlocksVerified" show up as 0 in Datanode JMX |  Major | metrics | Sai Nukavarapu | Hanisha Koneru |
+| [MAPREDUCE-6964](https://issues.apache.org/jira/browse/MAPREDUCE-6964) | BaileyBorweinPlouffe should use Time.monotonicNow for measuring durations |  Minor | examples | Chetna Chaudhari | Chetna Chaudhari |
+| [YARN-6991](https://issues.apache.org/jira/browse/YARN-6991) | "Kill application" button does not show error if other user tries to kill the application for secure cluster |  Major | . | Sumana Sathish | Suma Shivaprasad |
+| [YARN-7034](https://issues.apache.org/jira/browse/YARN-7034) | DefaultLinuxContainerRuntime and DockerLinuxContainerRuntime sends client environment variables to container-executor |  Blocker | nodemanager | Miklos Szegedi | Miklos Szegedi |
+| [HDFS-12523](https://issues.apache.org/jira/browse/HDFS-12523) | Thread pools in ErasureCodingWorker do not shutdown |  Major | erasure-coding | Lei (Eddy) Xu | Huafeng Wang |
+| [MAPREDUCE-6966](https://issues.apache.org/jira/browse/MAPREDUCE-6966) | DistSum should use Time.monotonicNow for measuring durations |  Minor | . | Chetna Chaudhari | Chetna Chaudhari |
+| [MAPREDUCE-6965](https://issues.apache.org/jira/browse/MAPREDUCE-6965) | QuasiMonteCarlo should use Time.monotonicNow for measuring durations |  Minor | examples | Chetna Chaudhari | Chetna Chaudhari |
+| [MAPREDUCE-6967](https://issues.apache.org/jira/browse/MAPREDUCE-6967) | gridmix/SleepReducer should use Time.monotonicNow for measuring durations |  Minor | . | Chetna Chaudhari | Chetna Chaudhari |
+| [YARN-7153](https://issues.apache.org/jira/browse/YARN-7153) | Remove duplicated code in AMRMClientAsyncImpl.java |  Minor | client | Sen Zhao | Sen Zhao |
+| [HADOOP-14897](https://issues.apache.org/jira/browse/HADOOP-14897) | Loosen compatibility guidelines for native dependencies |  Blocker | documentation, native | Chris Douglas | Daniel Templeton |
+| [HADOOP-14903](https://issues.apache.org/jira/browse/HADOOP-14903) | Add json-smart explicitly to pom.xml |  Major | common | Ray Chiang | Ray Chiang |
+| [YARN-7118](https://issues.apache.org/jira/browse/YARN-7118) | AHS REST API can return NullPointerException |  Major | . | Prabhu Joseph | Billie Rinaldi |
+| [HDFS-12495](https://issues.apache.org/jira/browse/HDFS-12495) | TestPendingInvalidateBlock#testPendingDeleteUnknownBlocks fails intermittently |  Major | . | Eric Badger | Eric Badger |
+| [MAPREDUCE-6968](https://issues.apache.org/jira/browse/MAPREDUCE-6968) | Staging directory erasure coding config property has a typo |  Major | client | Jason Lowe | Jason Lowe |
+| [HADOOP-14822](https://issues.apache.org/jira/browse/HADOOP-14822) | hadoop-project/pom.xml is executable |  Minor | . | Akira Ajisaka | Ajay Kumar |
+| [YARN-7157](https://issues.apache.org/jira/browse/YARN-7157) | Add admin configuration to filter per-user's apps in secure cluster |  Major | webapp | Sunil G | Sunil G |
+| [YARN-7253](https://issues.apache.org/jira/browse/YARN-7253) | Shared Cache Manager daemon command listed as admin subcmd in yarn script |  Trivial | . | Chris Trezzo | Chris Trezzo |
+| [YARN-7257](https://issues.apache.org/jira/browse/YARN-7257) | AggregatedLogsBlock reports a bad 'end' value as a bad 'start' value |  Major | log-aggregation | Jason Lowe | Jason Lowe |
+| [HDFS-12458](https://issues.apache.org/jira/browse/HDFS-12458) | TestReencryptionWithKMS fails regularly |  Major | encryption, test | Konstantin Shvachko | Xiao Chen |
+
+
+### TESTS:
+
+| JIRA | Summary | Priority | Component | Reporter | Contributor |
+|:---- |:---- | :--- |:---- |:---- |:---- |
+| [YARN-5548](https://issues.apache.org/jira/browse/YARN-5548) | Use MockRMMemoryStateStore to reduce test failures |  Major | . | Bibin A Chundatt | Bibin A Chundatt |
+| [HDFS-2319](https://issues.apache.org/jira/browse/HDFS-2319) | Add test cases for FSshell -stat |  Trivial | test | XieXianshan | Bharat Viswanadham |
+| [HADOOP-14245](https://issues.apache.org/jira/browse/HADOOP-14245) | Use Mockito.when instead of Mockito.stub |  Minor | test | Akira Ajisaka | Andras Bokor |
+| [YARN-5349](https://issues.apache.org/jira/browse/YARN-5349) | TestWorkPreservingRMRestart#testUAMRecoveryOnRMWorkPreservingRestart  fail intermittently |  Minor | . | sandflee | Jason Lowe |
+| [HDFS-11988](https://issues.apache.org/jira/browse/HDFS-11988) | Verify HDFS Snapshots with open files captured are safe across truncates and appends on current version file |  Major | hdfs, snapshots | Manoj Govindassamy | Manoj Govindassamy |
+| [HADOOP-14729](https://issues.apache.org/jira/browse/HADOOP-14729) | Upgrade JUnit 3 test cases to JUnit 4 |  Major | . | Akira Ajisaka | Ajay Kumar |
+| [HDFS-11912](https://issues.apache.org/jira/browse/HDFS-11912) | Add a snapshot unit test with randomized file IO operations |  Minor | hdfs | George Huang | George Huang |
+| [MAPREDUCE-6953](https://issues.apache.org/jira/browse/MAPREDUCE-6953) | Skip the testcase testJobWithChangePriority if FairScheduler is used |  Major | client | Peter Bacsko | Peter Bacsko |
+
+
+### SUB-TASKS:
+
+| JIRA | Summary | Priority | Component | Reporter | Contributor |
+|:---- |:---- | :--- |:---- |:---- |:---- |
+| [YARN-5300](https://issues.apache.org/jira/browse/YARN-5300) | Exclude generated federation protobuf sources from YARN Javadoc/findbugs build |  Minor | nodemanager, resourcemanager | Subru Krishnan | Subru Krishnan |
+| [YARN-5156](https://issues.apache.org/jira/browse/YARN-5156) | YARN\_CONTAINER\_FINISHED of YARN\_CONTAINERs will always have running state |  Major | timelineserver | Li Lu | Vrushali C |
+| [YARN-3662](https://issues.apache.org/jira/browse/YARN-3662) | Federation Membership State Store internal APIs |  Major | nodemanager, resourcemanager | Subru Krishnan | Subru Krishnan |
+| [YARN-5229](https://issues.apache.org/jira/browse/YARN-5229) | Refactor #isApplicationEntity and #getApplicationEvent from HBaseTimelineWriterImpl |  Minor | timelineserver | Joep Rottinghuis | Vrushali C |
+| [YARN-5406](https://issues.apache.org/jira/browse/YARN-5406) | In-memory based implementation of the FederationMembershipStateStore |  Major | nodemanager, resourcemanager | Subru Krishnan | Ellen Hui |
+| [YARN-5390](https://issues.apache.org/jira/browse/YARN-5390) | Federation Subcluster Resolver |  Major | nodemanager, resourcemanager | Carlo Curino | Ellen Hui |
+| [YARN-5307](https://issues.apache.org/jira/browse/YARN-5307) | Federation Application State Store internal APIs |  Major | nodemanager, resourcemanager | Subru Krishnan | Subru Krishnan |
+| [YARN-3664](https://issues.apache.org/jira/browse/YARN-3664) | Federation PolicyStore internal APIs |  Major | nodemanager, resourcemanager | Subru Krishnan | Subru Krishnan |
+| [YARN-5408](https://issues.apache.org/jira/browse/YARN-5408) | Compose Federation membership/application/policy APIs into an uber FederationStateStore API |  Major | nodemanager, resourcemanager | Subru Krishnan | Ellen Hui |
+| [YARN-5407](https://issues.apache.org/jira/browse/YARN-5407) | In-memory based implementation of the FederationApplicationStateStore, FederationPolicyStateStore |  Major | nodemanager, resourcemanager | Subru Krishnan | Ellen Hui |
+| [YARN-5519](https://issues.apache.org/jira/browse/YARN-5519) | Add SubClusterId in AddApplicationHomeSubClusterResponse for Router Failover |  Major | nodemanager, resourcemanager | Giovanni Matteo Fumarola | Ellen Hui |
+| [YARN-3672](https://issues.apache.org/jira/browse/YARN-3672) | Create Facade for Federation State and Policy Store |  Major | nodemanager, resourcemanager | Subru Krishnan | Subru Krishnan |
+| [YARN-5467](https://issues.apache.org/jira/browse/YARN-5467) | InputValidator for the FederationStateStore internal APIs |  Major | nodemanager, resourcemanager | Subru Krishnan | Giovanni Matteo Fumarola |
+| [YARN-3673](https://issues.apache.org/jira/browse/YARN-3673) | Create a FailoverProxy for Federation services |  Major | nodemanager, resourcemanager | Subru Krishnan | Subru Krishnan |
+| [YARN-3671](https://issues.apache.org/jira/browse/YARN-3671) | Integrate Federation services with ResourceManager |  Major | nodemanager, resourcemanager | Subru Krishnan | Subru Krishnan |
+| [YARN-5612](https://issues.apache.org/jira/browse/YARN-5612) | Return SubClusterId in FederationStateStoreFacade#addApplicationHomeSubCluster for Router Failover |  Major | nodemanager, resourcemanager | Giovanni Matteo Fumarola | Giovanni Matteo Fumarola |
+| [YARN-5601](https://issues.apache.org/jira/browse/YARN-5601) | Make the RM epoch base value configurable |  Major | nodemanager, resourcemanager | Subru Krishnan | Subru Krishnan |
+| [YARN-5323](https://issues.apache.org/jira/browse/YARN-5323) | Policies APIs (for Router and AMRMProxy policies) |  Major | nodemanager, resourcemanager | Carlo Curino | Carlo Curino |
+| [YARN-5324](https://issues.apache.org/jira/browse/YARN-5324) | Stateless Federation router policies implementation |  Major | nodemanager, resourcemanager | Carlo Curino | Carlo Curino |
+| [YARN-5325](https://issues.apache.org/jira/browse/YARN-5325) | Stateless ARMRMProxy policies implementation |  Major | nodemanager, resourcemanager | Carlo Curino | Carlo Curino |
+| [YARN-5638](https://issues.apache.org/jira/browse/YARN-5638) | Introduce a collector timestamp to uniquely identify collectors creation order in collector discovery |  Major | timelineserver | Li Lu | Li Lu |
+| [YARN-3649](https://issues.apache.org/jira/browse/YARN-3649) | Allow configurable prefix for hbase table names (like prod, exp, test etc) |  Major | timelineserver | Vrushali C | Vrushali C |
+| [YARN-5715](https://issues.apache.org/jira/browse/YARN-5715) | introduce entity prefix for return and sort order |  Critical | timelineserver | Sangjin Lee | Rohith Sharma K S |
+| [YARN-5391](https://issues.apache.org/jira/browse/YARN-5391) | PolicyManager to tie together Router/AMRM Federation policies |  Major | nodemanager, resourcemanager | Carlo Curino | Carlo Curino |
+| [YARN-5265](https://issues.apache.org/jira/browse/YARN-5265) | Make HBase configuration for the timeline service configurable |  Major | timelineserver | Joep Rottinghuis | Joep Rottinghuis |
+| [YARN-3359](https://issues.apache.org/jira/browse/YARN-3359) | Recover collector list when RM fails over |  Major | resourcemanager | Junping Du | Li Lu |
+| [YARN-5634](https://issues.apache.org/jira/browse/YARN-5634) | Simplify initialization/use of RouterPolicy via a RouterPolicyFacade |  Major | nodemanager, resourcemanager | Carlo Curino | Carlo Curino |
+| [YARN-5792](https://issues.apache.org/jira/browse/YARN-5792) | adopt the id prefix for YARN, MR, and DS entities |  Major | timelineserver | Sangjin Lee | Varun Saxena |
+| [YARN-5676](https://issues.apache.org/jira/browse/YARN-5676) | Add a HashBasedRouterPolicy, and small policies and test refactoring. |  Major | nodemanager, resourcemanager | Carlo Curino | Carlo Curino |
+| [YARN-5872](https://issues.apache.org/jira/browse/YARN-5872) | Add AlwayReject policies for router and amrmproxy. |  Major | nodemanager, resourcemanager | Carlo Curino | Carlo Curino |
+| [YARN-5905](https://issues.apache.org/jira/browse/YARN-5905) | Update the RM webapp host that is reported as part of Federation membership to current primary RM's IP |  Minor | federation, resourcemanager | Subru Krishnan | Subru Krishnan |
+| [YARN-5739](https://issues.apache.org/jira/browse/YARN-5739) | Provide timeline reader API to list available timeline entity types for one application |  Major | timelinereader | Li Lu | Li Lu |
+| [MAPREDUCE-6818](https://issues.apache.org/jira/browse/MAPREDUCE-6818) | Remove direct reference to TimelineClientImpl |  Major | . | Li Lu | Li Lu |
+| [YARN-5585](https://issues.apache.org/jira/browse/YARN-5585) | [Atsv2] Reader side changes for entity prefix and support for pagination via additional filters |  Critical | timelinereader | Rohith Sharma K S | Rohith Sharma K S |
+| [YARN-5980](https://issues.apache.org/jira/browse/YARN-5980) | Update documentation for single node hbase deploy |  Major | timelineserver | Vrushali C | Vrushali C |
+| [YARN-5378](https://issues.apache.org/jira/browse/YARN-5378) | Accommodate app-id-\>cluster mapping |  Major | timelineserver | Joep Rottinghuis | Sangjin Lee |
+| [YARN-6064](https://issues.apache.org/jira/browse/YARN-6064) | Support fromId for flowRuns and flow/flowRun apps REST API's |  Major | timelinereader | Rohith Sharma K S | Rohith Sharma K S |
+| [YARN-6094](https://issues.apache.org/jira/browse/YARN-6094) | Update the coprocessor to be a dynamically loaded one |  Major | timelineserver | Vrushali C | Vrushali C |
+| [YARN-5410](https://issues.apache.org/jira/browse/YARN-5410) | Bootstrap Router server module |  Major | nodemanager, resourcemanager | Subru Krishnan | Giovanni Matteo Fumarola |
+| [YARN-6253](https://issues.apache.org/jira/browse/YARN-6253) | FlowAcitivityColumnPrefix.store(byte[] rowKey, ...) drops timestamp |  Major | . | Haibo Chen | Haibo Chen |
+| [YARN-6190](https://issues.apache.org/jira/browse/YARN-6190) | Validation and synchronization fixes in LocalityMulticastAMRMProxyPolicy |  Minor | federation | Botong Huang | Botong Huang |
+| [YARN-6027](https://issues.apache.org/jira/browse/YARN-6027) | Support fromid(offset) filter for /flows API |  Major | timelineserver | Rohith Sharma K S | Rohith Sharma K S |
+| [YARN-6247](https://issues.apache.org/jira/browse/YARN-6247) | Share a single instance of SubClusterResolver instead of instantiating one per AM |  Minor | . | Botong Huang | Botong Huang |
+| [YARN-6256](https://issues.apache.org/jira/browse/YARN-6256) | Add FROM\_ID info key for timeline entities in reader response. |  Major | timelineserver | Rohith Sharma K S | Rohith Sharma K S |
+| [YARN-6237](https://issues.apache.org/jira/browse/YARN-6237) | Move UID constant to TimelineReaderUtils |  Major | timelinereader | Rohith Sharma K S | Rohith Sharma K S |
+| [YARN-6318](https://issues.apache.org/jira/browse/YARN-6318) | timeline service schema creator fails if executed from a remote machine |  Minor | timelineserver | Sangjin Lee | Sangjin Lee |
+| [YARN-6146](https://issues.apache.org/jira/browse/YARN-6146) | Add Builder methods for TimelineEntityFilters |  Major | timelineserver | Rohith Sharma K S | Haibo Chen |
+| [YARN-5602](https://issues.apache.org/jira/browse/YARN-5602) | Utils for Federation State and Policy Store |  Major | nodemanager, resourcemanager | Giovanni Matteo Fumarola | Giovanni Matteo Fumarola |
+| [YARN-6203](https://issues.apache.org/jira/browse/YARN-6203) | Occasional test failure in TestWeightedRandomRouterPolicy |  Minor | federation | Botong Huang | Carlo Curino |
+| [YARN-3663](https://issues.apache.org/jira/browse/YARN-3663) | Federation State and Policy Store (DBMS implementation) |  Major | nodemanager, resourcemanager | Giovanni Matteo Fumarola | Giovanni Matteo Fumarola |
+| [YARN-5411](https://issues.apache.org/jira/browse/YARN-5411) | Create a proxy chain for ApplicationClientProtocol in the Router |  Major | nodemanager, resourcemanager | Subru Krishnan | Giovanni Matteo Fumarola |
+| [YARN-5413](https://issues.apache.org/jira/browse/YARN-5413) | Create a proxy chain for ResourceManager Admin API in the Router |  Major | nodemanager, resourcemanager | Subru Krishnan | Giovanni Matteo Fumarola |
+| [YARN-5531](https://issues.apache.org/jira/browse/YARN-5531) | UnmanagedAM pool manager for federating application across clusters |  Major | nodemanager, resourcemanager | Subru Krishnan | Botong Huang |
+| [YARN-6666](https://issues.apache.org/jira/browse/YARN-6666) | Fix unit test failure in TestRouterClientRMService |  Minor | . | Botong Huang | Botong Huang |
+| [YARN-6484](https://issues.apache.org/jira/browse/YARN-6484) | [Documentation] Documenting the YARN Federation feature |  Major | nodemanager, resourcemanager | Subru Krishnan | Carlo Curino |
+| [YARN-6658](https://issues.apache.org/jira/browse/YARN-6658) | Remove columnFor() methods of Columns in HBaseTimeline backend |  Major | . | Haibo Chen | Haibo Chen |
+| [YARN-3666](https://issues.apache.org/jira/browse/YARN-3666) | Federation Intercepting and propagating AM- home RM communications |  Major | nodemanager, resourcemanager | Kishore Chaliparambil | Botong Huang |
+| [YARN-5647](https://issues.apache.org/jira/browse/YARN-5647) | [ATSv2 Security] Collector side changes for loading auth filters and principals |  Major | timelineserver | Varun Saxena | Varun Saxena |
+| [YARN-6511](https://issues.apache.org/jira/browse/YARN-6511) | Federation: transparently spanning application across multiple sub-clusters |  Major | . | Botong Huang | Botong Huang |
+| [YARN-6638](https://issues.apache.org/jira/browse/YARN-6638) | [ATSv2 Security] Timeline reader side changes for loading auth filters and principals |  Major | timelineserver | Varun Saxena | Varun Saxena |
+| [YARN-6724](https://issues.apache.org/jira/browse/YARN-6724) | Add ability to blacklist sub-clusters when invoking Routing policies |  Major | router | Subru Krishnan | Giovanni Matteo Fumarola |
+| [YARN-5648](https://issues.apache.org/jira/browse/YARN-5648) | [ATSv2 Security] Client side changes for authentication |  Major | timelineserver | Varun Saxena | Varun Saxena |
+| [YARN-3659](https://issues.apache.org/jira/browse/YARN-3659) | Federation: routing client invocations transparently to multiple RMs |  Major | client, resourcemanager, router | Giovanni Matteo Fumarola | Giovanni Matteo Fumarola |
+| [YARN-5067](https://issues.apache.org/jira/browse/YARN-5067) | Support specifying resources for AM containers in SLS |  Major | scheduler-load-simulator | Wangda Tan | Yufei Gu |
+| [YARN-6681](https://issues.apache.org/jira/browse/YARN-6681) | Eliminate double-copy of child queues in canAssignToThisQueue |  Major | resourcemanager | Daryn Sharp | Daryn Sharp |
+| [HADOOP-14615](https://issues.apache.org/jira/browse/HADOOP-14615) | Add ServiceOperations.stopQuietly that accept slf4j logger API |  Major | . | Wenxin He | Wenxin He |
+| [HADOOP-14617](https://issues.apache.org/jira/browse/HADOOP-14617) | Add ReflectionUtils.logThreadInfo that accept slf4j logger API |  Major | . | Wenxin He | Wenxin He |
+| [HADOOP-14571](https://issues.apache.org/jira/browse/HADOOP-14571) | Deprecate public APIs relate to log4j1 |  Major | . | Akira Ajisaka | Wenxin He |
+| [HADOOP-14587](https://issues.apache.org/jira/browse/HADOOP-14587) | Use GenericTestUtils.setLogLevel when available in hadoop-common |  Major | . | Wenxin He | Wenxin He |
+| [YARN-6776](https://issues.apache.org/jira/browse/YARN-6776) | Refactor ApplicaitonMasterService to move actual processing logic to a separate class |  Minor | . | Arun Suresh | Arun Suresh |
+| [HADOOP-14638](https://issues.apache.org/jira/browse/HADOOP-14638) | Replace commons-logging APIs with slf4j in StreamPumper |  Major | . | Wenxin He | Wenxin He |
+| [YARN-6801](https://issues.apache.org/jira/browse/YARN-6801) | NPE in RM while setting collectors map in NodeHeartbeatResponse |  Major | timelineserver | Vrushali C | Vrushali C |
+| [YARN-6807](https://issues.apache.org/jira/browse/YARN-6807) | Adding required missing configs to Federation configuration guide based on e2e testing |  Major | documentation, federation | Subru Krishnan | Tanuj Nayak |
+| [YARN-6815](https://issues.apache.org/jira/browse/YARN-6815) | [Bug] FederationStateStoreFacade return behavior should be consistent irrespective of whether caching is enabled or not |  Major | nodemanager, resourcemanager | Subru Krishnan | Subru Krishnan |
+| [YARN-6821](https://issues.apache.org/jira/browse/YARN-6821) | Move FederationStateStore SQL DDL files from test resource to sbin |  Major | nodemanager, resourcemanager | Subru Krishnan | Subru Krishnan |
+| [YARN-6706](https://issues.apache.org/jira/browse/YARN-6706) | Refactor ContainerScheduler to make oversubscription change easier |  Major | . | Haibo Chen | Haibo Chen |
+| [HADOOP-14642](https://issues.apache.org/jira/browse/HADOOP-14642) | wasb: add support for caching Authorization and SASKeys |  Major | fs/azure | Sivaguru Sankaridurg | Sivaguru Sankaridurg |
+| [YARN-6777](https://issues.apache.org/jira/browse/YARN-6777) | Support for ApplicationMasterService processing chain of interceptors |  Major | . | Arun Suresh | Arun Suresh |
+| [YARN-6775](https://issues.apache.org/jira/browse/YARN-6775) | CapacityScheduler: Improvements to assignContainers, avoid unnecessary canAssignToUser/Queue calls |  Major | capacityscheduler | Nathan Roberts | Nathan Roberts |
+| [YARN-4455](https://issues.apache.org/jira/browse/YARN-4455) | Support fetching metrics by time range |  Major | timelineserver | Varun Saxena | Varun Saxena |
+| [YARN-6685](https://issues.apache.org/jira/browse/YARN-6685) | Add job count in to SLS JSON input format |  Major | scheduler-load-simulator | Yufei Gu | Yufei Gu |
+| [YARN-6850](https://issues.apache.org/jira/browse/YARN-6850) | Ensure that supplemented timestamp is stored only for flow run metrics |  Major | timelineserver | Vrushali C | Varun Saxena |
+| [YARN-6733](https://issues.apache.org/jira/browse/YARN-6733) | Add table for storing sub-application entities |  Major | timelineserver | Vrushali C | Vrushali C |
+| [HADOOP-14518](https://issues.apache.org/jira/browse/HADOOP-14518) | Customize User-Agent header sent in HTTP/HTTPS requests by WASB. |  Minor | fs/azure | Georgi Chalakov | Georgi Chalakov |
+| [YARN-6804](https://issues.apache.org/jira/browse/YARN-6804) | Allow custom hostname for docker containers in native services |  Major | yarn-native-services | Billie Rinaldi | Billie Rinaldi |
+| [YARN-6866](https://issues.apache.org/jira/browse/YARN-6866) | Minor clean-up and fixes in anticipation of YARN-2915 merge with trunk |  Major | federation | Subru Krishnan | Botong Huang |
+| [YARN-5412](https://issues.apache.org/jira/browse/YARN-5412) | Create a proxy chain for ResourceManager REST API in the Router |  Major | nodemanager, resourcemanager | Subru Krishnan | Giovanni Matteo Fumarola |
+| [HADOOP-11875](https://issues.apache.org/jira/browse/HADOOP-11875) | [JDK9] Add a second copy of Hamlet without \_ as a one-character identifier |  Major | . | Tsuyoshi Ozawa | Akira Ajisaka |
+| [YARN-6888](https://issues.apache.org/jira/browse/YARN-6888) | Refactor AppLevelTimelineCollector such that RM does not have aggregator threads created |  Major | timelineserver | Vrushali C | Vrushali C |
+| [HADOOP-14678](https://issues.apache.org/jira/browse/HADOOP-14678) | AdlFilesystem#initialize swallows exception when getting user name |  Minor | fs/adl | John Zhuge | John Zhuge |
+| [YARN-6734](https://issues.apache.org/jira/browse/YARN-6734) | Ensure sub-application user is extracted & sent to timeline service |  Major | timelineserver | Vrushali C | Rohith Sharma K S |
+| [YARN-6902](https://issues.apache.org/jira/browse/YARN-6902) | Update Microsoft JDBC Driver for SQL Server version in License.txt |  Minor | federation | Botong Huang | Botong Huang |
+| [HADOOP-14672](https://issues.apache.org/jira/browse/HADOOP-14672) | Shaded Hadoop-client-minicluster include unshaded classes, like: javax, sax, dom, etc. |  Blocker | . | Junping Du | Bharat Viswanadham |
+| [HADOOP-14397](https://issues.apache.org/jira/browse/HADOOP-14397) | Pull up the builder pattern to FileSystem and add AbstractContractCreateTest for it |  Major | common, fs, hdfs-client | Lei (Eddy) Xu | Lei (Eddy) Xu |
+| [HDFS-12151](https://issues.apache.org/jira/browse/HDFS-12151) | Hadoop 2 clients cannot writeBlock to Hadoop 3 DataNodes |  Major | rolling upgrades | Sean Mackrory | Sean Mackrory |
+| [HADOOP-14495](https://issues.apache.org/jira/browse/HADOOP-14495) | Add set options interface to FSDataOutputStreamBuilder |  Major | fs | Lei (Eddy) Xu | Lei (Eddy) Xu |
+| [YARN-6853](https://issues.apache.org/jira/browse/YARN-6853) | Add MySql Scripts for FederationStateStore |  Major | . | Giovanni Matteo Fumarola | Giovanni Matteo Fumarola |
+| [HDFS-9388](https://issues.apache.org/jira/browse/HDFS-9388) | Refactor decommission related code to support maintenance state for datanodes |  Major | . | Ming Ma | Manoj Govindassamy |
+| [YARN-6674](https://issues.apache.org/jira/browse/YARN-6674) | Add memory cgroup settings for opportunistic containers |  Major | nodemanager | Haibo Chen | Miklos Szegedi |
+| [YARN-6673](https://issues.apache.org/jira/browse/YARN-6673) | Add cpu cgroup configurations for opportunistic containers |  Major | . | Haibo Chen | Miklos Szegedi |
+| [YARN-5977](https://issues.apache.org/jira/browse/YARN-5977) | ContainerManagementProtocol changes to support change of container ExecutionType |  Major | . | Arun Suresh | kartheek muthyala |
+| [HADOOP-14126](https://issues.apache.org/jira/browse/HADOOP-14126) | remove jackson, joda and other transient aws SDK dependencies from hadoop-aws |  Minor | build, fs/s3 | Steve Loughran | Steve Loughran |
+| [HADOOP-14685](https://issues.apache.org/jira/browse/HADOOP-14685) | Test jars to exclude from hadoop-client-minicluster jar |  Major | common | Bharat Viswanadham | Bharat Viswanadham |
+| [YARN-6957](https://issues.apache.org/jira/browse/YARN-6957) | Moving logging APIs over to slf4j in hadoop-yarn-server-sharedcachemanager |  Major | . | Yeliang Cang | Yeliang Cang |
+| [YARN-6955](https://issues.apache.org/jira/browse/YARN-6955) | Handle concurrent register AM requests in FederationInterceptor |  Minor | . | Botong Huang | Botong Huang |
+| [YARN-6873](https://issues.apache.org/jira/browse/YARN-6873) | Moving logging APIs over to slf4j in hadoop-yarn-server-applicationhistoryservice |  Major | . | Yeliang Cang | Yeliang Cang |
+| [YARN-6897](https://issues.apache.org/jira/browse/YARN-6897) | Refactoring RMWebServices by moving some util methods to RMWebAppUtil |  Major | . | Giovanni Matteo Fumarola | Giovanni Matteo Fumarola |
+| [HADOOP-14715](https://issues.apache.org/jira/browse/HADOOP-14715) | TestWasbRemoteCallHelper failing |  Major | fs/azure, test | Steve Loughran | Esfandiar Manii |
+| [YARN-6970](https://issues.apache.org/jira/browse/YARN-6970) | Add PoolInitializationException as retriable exception in FederationFacade |  Major | federation | Giovanni Matteo Fumarola | Giovanni Matteo Fumarola |
+| [HDFS-11975](https://issues.apache.org/jira/browse/HDFS-11975) | Provide a system-default EC policy |  Major | erasure-coding | Lei (Eddy) Xu | luhuichun |
+| [HADOOP-14628](https://issues.apache.org/jira/browse/HADOOP-14628) | Upgrade maven enforcer plugin to 3.0.0-M1 |  Major | . | Akira Ajisaka | Akira Ajisaka |
+| [HADOOP-14355](https://issues.apache.org/jira/browse/HADOOP-14355) | Update maven-war-plugin to 3.1.0 |  Major | build | Akira Ajisaka | Akira Ajisaka |
+| [YARN-6631](https://issues.apache.org/jira/browse/YARN-6631) | Refactor loader.js in new Yarn UI |  Major | . | Akhil PB | Akhil PB |
+| [YARN-6874](https://issues.apache.org/jira/browse/YARN-6874) | Supplement timestamp for min start/max end time columns in flow run table to avoid overwrite |  Major | timelineserver | Varun Saxena | Vrushali C |
+| [YARN-6958](https://issues.apache.org/jira/browse/YARN-6958) | Moving logging APIs over to slf4j in hadoop-yarn-server-timelineservice |  Major | . | Yeliang Cang | Yeliang Cang |
+| [HADOOP-14183](https://issues.apache.org/jira/browse/HADOOP-14183) | Remove service loader config file for wasb fs |  Minor | fs/azure | John Zhuge | Esfandiar Manii |
+| [YARN-6130](https://issues.apache.org/jira/browse/YARN-6130) | [ATSv2 Security] Generate a delegation token for AM when app collector is created and pass it to AM via NM and RM |  Major | timelineserver | Varun Saxena | Varun Saxena |
+| [YARN-6133](https://issues.apache.org/jira/browse/YARN-6133) | [ATSv2 Security] Renew delegation token for app automatically if an app collector is active |  Major | timelineserver | Varun Saxena | Varun Saxena |
+| [HADOOP-10392](https://issues.apache.org/jira/browse/HADOOP-10392) | Use FileSystem#makeQualified(Path) instead of Path#makeQualified(FileSystem) |  Minor | fs | Akira Ajisaka | Akira Ajisaka |
+| [YARN-6820](https://issues.apache.org/jira/browse/YARN-6820) | Restrict read access to timelineservice v2 data |  Major | timelinereader | Vrushali C | Vrushali C |
+| [YARN-6896](https://issues.apache.org/jira/browse/YARN-6896) | Federation: routing REST invocations transparently to multiple RMs (part 1 - basic execution) |  Major | . | Giovanni Matteo Fumarola | Giovanni Matteo Fumarola |
+| [YARN-6687](https://issues.apache.org/jira/browse/YARN-6687) | Validate that the duration of the periodic reservation is less than the periodicity |  Major | reservation system | Subru Krishnan | Subru Krishnan |
+| [YARN-6905](https://issues.apache.org/jira/browse/YARN-6905) | Multiple HBaseTimelineStorage test failures due to missing FastNumberFormat |  Major | timelineserver | Sonia Garudi | Haibo Chen |
+| [YARN-5978](https://issues.apache.org/jira/browse/YARN-5978) | ContainerScheduler and ContainerManager changes to support ExecType update |  Major | . | Arun Suresh | kartheek muthyala |
+| [YARN-5146](https://issues.apache.org/jira/browse/YARN-5146) | Support for Fair Scheduler in new YARN UI |  Major | . | Wangda Tan | Abdullah Yousufi |
+| [YARN-6741](https://issues.apache.org/jira/browse/YARN-6741) | Deleting all children of a Parent Queue on refresh throws exception |  Major | capacity scheduler | Naganarasimha G R | Naganarasimha G R |
+| [HADOOP-14660](https://issues.apache.org/jira/browse/HADOOP-14660) | wasb: improve throughput by 34% when account limit exceeded |  Major | fs/azure | Thomas Marquardt | Thomas Marquardt |
+| [YARN-7006](https://issues.apache.org/jira/browse/YARN-7006) | [ATSv2 Security] Changes for authentication for CollectorNodemanagerProtocol |  Major | timelineserver | Varun Saxena | Varun Saxena |
+| [YARN-6900](https://issues.apache.org/jira/browse/YARN-6900) | ZooKeeper based implementation of the FederationStateStore |  Major | federation, nodemanager, resourcemanager | Subru Krishnan | Íñigo Goiri |
+| [HDFS-11082](https://issues.apache.org/jira/browse/HDFS-11082) | Provide replicated EC policy to replicate files |  Critical | erasure-coding | Rakesh R | SammiChen |
+| [YARN-6988](https://issues.apache.org/jira/browse/YARN-6988) | container-executor fails for docker when command length \> 4096 B |  Major | yarn | Eric Badger | Eric Badger |
+| [YARN-7038](https://issues.apache.org/jira/browse/YARN-7038) | [Atsv2 Security] CollectorNodemanagerProtocol RPC interface doesn't work when service authorization is enabled |  Major | . | Rohith Sharma K S | Varun Saxena |
+| [HADOOP-14769](https://issues.apache.org/jira/browse/HADOOP-14769) | WASB: delete recursive should not fail if a file is deleted |  Major | fs/azure | Thomas Marquardt | Thomas Marquardt |
+| [HADOOP-14398](https://issues.apache.org/jira/browse/HADOOP-14398) | Modify documents for the FileSystem Builder API |  Major | fs | Lei (Eddy) Xu | Lei (Eddy) Xu |
+| [YARN-6852](https://issues.apache.org/jira/browse/YARN-6852) | [YARN-6223] Native code changes to support isolate GPU devices by using CGroups |  Major | . | Wangda Tan | Wangda Tan |
+| [YARN-7041](https://issues.apache.org/jira/browse/YARN-7041) | Nodemanager NPE running jobs with security off |  Major | timelineserver | Aaron Gresch | Varun Saxena |
+| [YARN-6134](https://issues.apache.org/jira/browse/YARN-6134) | [ATSv2 Security] Regenerate delegation token for app just before token expires if app collector is active |  Major | timelineserver | Varun Saxena | Varun Saxena |
+| [YARN-6979](https://issues.apache.org/jira/browse/YARN-6979) | Add flag to notify all types of container updates to NM via NodeHeartbeatResponse |  Major | . | Arun Suresh | kartheek muthyala |
+| [HADOOP-14194](https://issues.apache.org/jira/browse/HADOOP-14194) | Aliyun OSS should not use empty endpoint as default |  Major | fs/oss | Mingliang Liu | Genmao Yu |
+| [YARN-6861](https://issues.apache.org/jira/browse/YARN-6861) | Reader API for sub application entities |  Major | timelinereader | Rohith Sharma K S | Rohith Sharma K S |
+| [YARN-6323](https://issues.apache.org/jira/browse/YARN-6323) | Rolling upgrade/config change is broken on timeline v2. |  Major | timelineserver | Li Lu | Vrushali C |
+| [YARN-6047](https://issues.apache.org/jira/browse/YARN-6047) | Documentation updates for TimelineService v2 |  Major | documentation, timelineserver | Varun Saxena | Rohith Sharma K S |
+| [MAPREDUCE-6838](https://issues.apache.org/jira/browse/MAPREDUCE-6838) | [ATSv2 Security] Add timeline delegation token received in allocate response to UGI |  Major | . | Varun Saxena | Varun Saxena |
+| [YARN-5603](https://issues.apache.org/jira/browse/YARN-5603) | Metrics for Federation StateStore |  Major | . | Subru Krishnan | Ellen Hui |
+| [YARN-6923](https://issues.apache.org/jira/browse/YARN-6923) | Metrics for Federation Router |  Major | . | Giovanni Matteo Fumarola | Giovanni Matteo Fumarola |
+| [HADOOP-14787](https://issues.apache.org/jira/browse/HADOOP-14787) | AliyunOSS: Implement the \`createNonRecursive\` operator |  Major | fs, fs/oss | Genmao Yu | Genmao Yu |
+| [HADOOP-14649](https://issues.apache.org/jira/browse/HADOOP-14649) | Update aliyun-sdk-oss version to 2.8.1 |  Major | . | Ray Chiang | Genmao Yu |
+| [YARN-7047](https://issues.apache.org/jira/browse/YARN-7047) | Moving logging APIs over to slf4j in hadoop-yarn-server-nodemanager |  Major | . | Yeliang Cang | Yeliang Cang |
+| [YARN-6876](https://issues.apache.org/jira/browse/YARN-6876) | Create an abstract log writer for extendability |  Major | . | Xuan Gong | Xuan Gong |
+| [YARN-6386](https://issues.apache.org/jira/browse/YARN-6386) | Show decommissioning nodes in new YARN UI |  Major | webapp | Elek, Marton | Elek, Marton |
+| [YARN-7010](https://issues.apache.org/jira/browse/YARN-7010) | Federation: routing REST invocations transparently to multiple RMs (part 2 - getApps) |  Major | . | Giovanni Matteo Fumarola | Giovanni Matteo Fumarola |
+| [YARN-5219](https://issues.apache.org/jira/browse/YARN-5219) | When an export var command fails in launch\_container.sh, the full container launch should fail |  Major | . | Hitesh Shah | Sunil G |
+| [HADOOP-14802](https://issues.apache.org/jira/browse/HADOOP-14802) | Add support for using container saskeys for all accesses |  Major | fs/azure | Sivaguru Sankaridurg | Sivaguru Sankaridurg |
+| [YARN-7094](https://issues.apache.org/jira/browse/YARN-7094) | Document the current known issue with server-side NM graceful decom |  Blocker | graceful | Robert Kanter | Robert Kanter |
+| [YARN-7095](https://issues.apache.org/jira/browse/YARN-7095) | Federation: routing getNode/getNodes/getMetrics REST invocations transparently to multiple RMs |  Major | . | Giovanni Matteo Fumarola | Giovanni Matteo Fumarola |
+| [YARN-6033](https://issues.apache.org/jira/browse/YARN-6033) | Add support for sections in container-executor configuration file |  Major | nodemanager | Varun Vasudev | Varun Vasudev |
+| [YARN-6877](https://issues.apache.org/jira/browse/YARN-6877) | Create an abstract log reader for extendability |  Major | . | Xuan Gong | Xuan Gong |
+| [YARN-7071](https://issues.apache.org/jira/browse/YARN-7071) | Add vcores and number of containers in new YARN UI node heat map |  Major | yarn-ui-v2 | Abdullah Yousufi | Abdullah Yousufi |
+| [YARN-7075](https://issues.apache.org/jira/browse/YARN-7075) | Better styling for donut charts in new YARN UI |  Major | . | Da Ding | Da Ding |
+| [HADOOP-14103](https://issues.apache.org/jira/browse/HADOOP-14103) | Sort out hadoop-aws contract-test-options.xml |  Minor | fs/s3, test | Steve Loughran | John Zhuge |
+| [YARN-7148](https://issues.apache.org/jira/browse/YARN-7148) | TestLogsCLI fails in trunk and branch-2 and javadoc error |  Major | . | Xuan Gong | Xuan Gong |
+| [YARN-6978](https://issues.apache.org/jira/browse/YARN-6978) | Add updateContainer API to NMClient. |  Major | . | Arun Suresh | kartheek muthyala |
+| [HADOOP-14774](https://issues.apache.org/jira/browse/HADOOP-14774) | S3A case "testRandomReadOverBuffer" failed due to improper range parameter |  Minor | fs/s3 | Yonger | Yonger |
+| [YARN-7144](https://issues.apache.org/jira/browse/YARN-7144) | Log Aggregation controller should not swallow the exceptions when it calls closeWriter and closeReader. |  Major | . | Xuan Gong | Xuan Gong |
+| [YARN-7104](https://issues.apache.org/jira/browse/YARN-7104) | Improve Nodes Heatmap in new YARN UI with better color coding |  Major | . | Da Ding | Da Ding |
+| [HADOOP-13421](https://issues.apache.org/jira/browse/HADOOP-13421) | Switch to v2 of the S3 List Objects API in S3A |  Minor | fs/s3 | Steven K. Wong | Aaron Fabbri |
+| [YARN-6600](https://issues.apache.org/jira/browse/YARN-6600) | Introduce default and max lifetime of application at LeafQueue level |  Major | capacity scheduler | Rohith Sharma K S | Rohith Sharma K S |
+| [YARN-6849](https://issues.apache.org/jira/browse/YARN-6849) | NMContainerStatus should have the Container ExecutionType. |  Major | . | Arun Suresh | kartheek muthyala |
+| [YARN-7128](https://issues.apache.org/jira/browse/YARN-7128) | The error message in TimelineSchemaCreator is not enough to find out the error. |  Major | timelineserver | Jinjiang Ling | Jinjiang Ling |
+| [YARN-7173](https://issues.apache.org/jira/browse/YARN-7173) | Container update RM-NM communication fix for backward compatibility |  Major | . | Arun Suresh | Arun Suresh |
+| [HADOOP-14798](https://issues.apache.org/jira/browse/HADOOP-14798) | Update sshd-core and related mina-core library versions |  Major | . | Ray Chiang | Ray Chiang |
+| [HADOOP-14796](https://issues.apache.org/jira/browse/HADOOP-14796) | Update json-simple version to 1.1.1 |  Major | . | Ray Chiang | Ray Chiang |
+| [HADOOP-14648](https://issues.apache.org/jira/browse/HADOOP-14648) | Bump commons-configuration2 to 2.1.1 |  Major | . | Ray Chiang | Ray Chiang |
+| [HADOOP-14653](https://issues.apache.org/jira/browse/HADOOP-14653) | Update joda-time version to 2.9.9 |  Major | . | Ray Chiang | Ray Chiang |
+| [HADOOP-14797](https://issues.apache.org/jira/browse/HADOOP-14797) | Update re2j version to 1.1 |  Major | . | Ray Chiang | Ray Chiang |
+| [HADOOP-14857](https://issues.apache.org/jira/browse/HADOOP-14857) | Fix downstream shaded client integration test |  Blocker | . | Sean Busbey | Sean Busbey |
+| [HADOOP-14089](https://issues.apache.org/jira/browse/HADOOP-14089) | Automated checking for malformed client artifacts. |  Blocker | . | David Phillips | Sean Busbey |
+| [YARN-7162](https://issues.apache.org/jira/browse/YARN-7162) | Remove XML excludes file format |  Blocker | graceful | Robert Kanter | Robert Kanter |
+| [HADOOP-14553](https://issues.apache.org/jira/browse/HADOOP-14553) | Add (parallelized) integration tests to hadoop-azure |  Major | fs/azure | Steve Loughran | Steve Loughran |
+| [YARN-7174](https://issues.apache.org/jira/browse/YARN-7174) | Add retry logic in LogsCLI when fetch running application logs |  Major | . | Xuan Gong | Xuan Gong |
+| [HDFS-12470](https://issues.apache.org/jira/browse/HDFS-12470) | DiskBalancer: Some tests create plan files under system directory |  Major | diskbalancer, test | Akira Ajisaka | Hanisha Koneru |
+| [HADOOP-14583](https://issues.apache.org/jira/browse/HADOOP-14583) | wasb throws an exception if you try to create a file and there's no parent directory |  Minor | fs/azure | Steve Loughran | Esfandiar Manii |
+| [HDFS-12473](https://issues.apache.org/jira/browse/HDFS-12473) | Change hosts JSON file format |  Major | . | Ming Ma | Ming Ma |
+| [HDFS-11035](https://issues.apache.org/jira/browse/HDFS-11035) | Better documentation for maintenace mode and upgrade domain |  Major | datanode, documentation | Wei-Chiu Chuang | Ming Ma |
+| [YARN-4266](https://issues.apache.org/jira/browse/YARN-4266) | Allow users to enter containers as UID:GID pair instead of by username |  Major | yarn | Sidharta Seethana | luhuichun |
+| [MAPREDUCE-6947](https://issues.apache.org/jira/browse/MAPREDUCE-6947) | 	Moving logging APIs over to slf4j in hadoop-mapreduce-examples |  Major | . | Gergely Novák | Gergely Novák |
+| [HADOOP-14799](https://issues.apache.org/jira/browse/HADOOP-14799) | Update nimbus-jose-jwt to 4.41.1 |  Major | . | Ray Chiang | Ray Chiang |
+| [HADOOP-14892](https://issues.apache.org/jira/browse/HADOOP-14892) | MetricsSystemImpl should use Time.monotonicNow for measuring durations |  Minor | . | Chetna Chaudhari | Chetna Chaudhari |
+| [HADOOP-14881](https://issues.apache.org/jira/browse/HADOOP-14881) | LoadGenerator should use Time.monotonicNow() to measure durations |  Major | . | Chetna Chaudhari | Bharat Viswanadham |
+| [HADOOP-14220](https://issues.apache.org/jira/browse/HADOOP-14220) | Enhance S3GuardTool with bucket-info and set-capacity commands, tests |  Major | fs/s3 | Steve Loughran | Steve Loughran |
+| [HADOOP-14893](https://issues.apache.org/jira/browse/HADOOP-14893) | WritableRpcEngine should use Time.monotonicNow |  Minor | . | Chetna Chaudhari | Chetna Chaudhari |
+| [HADOOP-14890](https://issues.apache.org/jira/browse/HADOOP-14890) | Move up to AWS SDK 1.11.199 |  Blocker | build, fs/s3 | Steve Loughran | Steve Loughran |
+| [HDFS-12386](https://issues.apache.org/jira/browse/HDFS-12386) | Add fsserver defaults call to WebhdfsFileSystem. |  Minor | webhdfs | Rushabh S Shah | Rushabh S Shah |
+| [YARN-6691](https://issues.apache.org/jira/browse/YARN-6691) | Update YARN daemon startup/shutdown scripts to include Router service |  Major | nodemanager, resourcemanager | Subru Krishnan | Giovanni Matteo Fumarola |
+
+
+### OTHER:
+
+| JIRA | Summary | Priority | Component | Reporter | Contributor |
+|:---- |:---- | :--- |:---- |:---- |:---- |
+| [YARN-6969](https://issues.apache.org/jira/browse/YARN-6969) | Clean up unused code in class FairSchedulerQueueInfo |  Trivial | fairscheduler | Yufei Gu | Larry Lo |
+| [YARN-6622](https://issues.apache.org/jira/browse/YARN-6622) | Document Docker work as experimental |  Blocker | documentation | Varun Vasudev | Varun Vasudev |
+| [YARN-7203](https://issues.apache.org/jira/browse/YARN-7203) | Add container ExecutionType into ContainerReport |  Minor | . | Botong Huang | Botong Huang |
+
+

+ 377 - 0
hadoop-common-project/hadoop-common/src/site/markdown/release/3.0.0-beta1/RELEASENOTES.3.0.0-beta1.md

@@ -0,0 +1,377 @@
+
+<!---
+# 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.
+-->
+# Apache Hadoop  3.0.0-beta1 Release Notes
+
+These release notes cover new developer and user-facing incompatibilities, important issues, features, and major improvements.
+
+
+---
+
+* [HADOOP-14535](https://issues.apache.org/jira/browse/HADOOP-14535) | *Major* | **wasb: implement high-performance random access and seek of block blobs**
+
+Random access and seek improvements for the wasb:// (Azure) file system.
+
+
+---
+
+* [YARN-6798](https://issues.apache.org/jira/browse/YARN-6798) | *Major* | **Fix NM startup failure with old state store due to version mismatch**
+
+<!-- markdown -->
+
+This fixes the LevelDB state store for the NodeManager.  As of this patch, the state store versions now correspond to the following table.
+
+* Previous Patch: YARN-5049
+  * LevelDB Key: queued
+  * Hadoop Versions: 2.9.0, 3.0.0-alpha1
+  * Corresponding LevelDB Version: 1.2
+* Previous Patch: YARN-6127
+  * LevelDB Key: AMRMProxy/NextMasterKey
+  * Hadoop Versions: 2.9.0, 3.0.0-alpha4
+  * Corresponding LevelDB Version: 1.1
+
+
+---
+
+* [HADOOP-14539](https://issues.apache.org/jira/browse/HADOOP-14539) | *Major* | **Move commons logging APIs over to slf4j in hadoop-common**
+
+In Hadoop common, fatal log level is changed to error because slf4j API does not support fatal log level.
+
+
+---
+
+* [HADOOP-14518](https://issues.apache.org/jira/browse/HADOOP-14518) | *Minor* | **Customize User-Agent header sent in HTTP/HTTPS requests by WASB.**
+
+WASB now includes the current Apache Hadoop version in the User-Agent string passed to Azure Blob service. Users also may include optional additional information to identify their application. See the documentation of configuration property fs.wasb.user.agent.id for further details.
+
+
+---
+
+* [HADOOP-11875](https://issues.apache.org/jira/browse/HADOOP-11875) | *Major* | **[JDK9] Add a second copy of Hamlet without \_ as a one-character identifier**
+
+Added org.apache.hadoop.yarn.webapp.hamlet2 package without \_ as a one-character identifier. Please use this package instead of org.apache.hadoop.yarn.webapp.hamlet.
+
+
+---
+
+* [HDFS-12206](https://issues.apache.org/jira/browse/HDFS-12206) | *Major* | **Rename the split EC / replicated block metrics**
+
+The metrics and MBeans introduced in HDFS-10999 have been renamed for brevity and clarity.
+
+
+---
+
+* [HADOOP-13595](https://issues.apache.org/jira/browse/HADOOP-13595) | *Blocker* | **Rework hadoop\_usage to be broken up by clients/daemons/etc.**
+
+This patch changes how usage output is generated to now require a sub-command type.  This allows users to see who the intended audience for  a command is or it is a daemon.
+
+
+---
+
+* [HDFS-6984](https://issues.apache.org/jira/browse/HDFS-6984) | *Major* | **Serialize FileStatus via protobuf**
+
+FileStatus and FsPermission Writable serialization is deprecated and its implementation (incompatibly) replaced with protocol buffers. The FsPermissionProto record moved from hdfs.proto to acl.proto. HdfsFileStatus is now a subtype of FileStatus. FsPermissionExtension with its associated flags for ACLs, encryption, and erasure coding has been deprecated; users should query these attributes on the FileStatus object directly. The FsPermission instance in AclStatus no longer retains or reports these extended attributes (likely unused).
+
+
+---
+
+* [HADOOP-14722](https://issues.apache.org/jira/browse/HADOOP-14722) | *Major* | **Azure: BlockBlobInputStream position incorrect after seek**
+
+Bug fix to Azure Filesystem related to HADOOP-14535.
+
+
+---
+
+* [YARN-6961](https://issues.apache.org/jira/browse/YARN-6961) | *Minor* | **Remove commons-logging dependency from hadoop-yarn-server-applicationhistoryservice module**
+
+commons-logging dependency was removed from hadoop-yarn-server-applicationhistoryservice. If you rely on the transitive commons-logging dependency, please define the dependency explicitly.
+
+
+---
+
+* [HADOOP-14680](https://issues.apache.org/jira/browse/HADOOP-14680) | *Minor* | **Azure: IndexOutOfBoundsException in BlockBlobInputStream**
+
+Bug fix to Azure Filesystem related to HADOOP-14535
+
+
+---
+
+* [HDFS-10326](https://issues.apache.org/jira/browse/HDFS-10326) | *Major* | **Disable setting tcp socket send/receive buffers for write pipelines**
+
+The size of the TCP socket buffers are no longer hardcoded by default. Instead the OS now will automatically tune the size for the buffer.
+
+
+---
+
+* [HDFS-11957](https://issues.apache.org/jira/browse/HDFS-11957) | *Major* | **Enable POSIX ACL inheritance by default**
+
+<!-- markdown -->
+HDFS-6962 introduced POSIX ACL inheritance feature but it is disable by
+default. Now enable the feature by default. Please be aware any code
+expecting the old ACL inheritance behavior will have to be updated.
+Please see the HDFS Permissions Guide for further details.
+
+
+---
+
+* [MAPREDUCE-6870](https://issues.apache.org/jira/browse/MAPREDUCE-6870) | *Major* | **Add configuration for MR job to finish when all reducers are complete (even with unfinished mappers)**
+
+Enables mapreduce.job.finish-when-all-reducers-done by default. With this enabled, a MapReduce job will complete as soon as all of its reducers are complete, even if some mappers are still running. This can occur if a mapper was relaunched after node failure but the relaunched task's output is not actually needed. Previously the job would wait for all mappers to complete.
+
+
+---
+
+* [HADOOP-14260](https://issues.apache.org/jira/browse/HADOOP-14260) | *Major* | **Configuration.dumpConfiguration should redact sensitive information**
+
+<!-- markdown -->
+Configuration.dumpConfiguration no longer prints out the clear text values for the sensitive keys listed in `hadoop.security.sensitive-config-keys`. Callers can override the default list of sensitive keys either to redact more keys or print the clear text values for a few extra keys for debugging purpose.
+
+
+---
+
+* [HDFS-12221](https://issues.apache.org/jira/browse/HDFS-12221) | *Major* | **Replace xerces in XmlEditsVisitor**
+
+New patch with changes in maven dependency. (removed apache xcerces as dependency)
+
+
+---
+
+* [HADOOP-14726](https://issues.apache.org/jira/browse/HADOOP-14726) | *Minor* | **Mark FileStatus::isDir as final**
+
+The deprecated FileStatus::isDir method has been marked as final. FileSystems should override FileStatus::isDirectory
+
+
+---
+
+* [HADOOP-14660](https://issues.apache.org/jira/browse/HADOOP-14660) | *Major* | **wasb: improve throughput by 34% when account limit exceeded**
+
+Up to 34% throughput improvement for the wasb:// (Azure) file system when fs.azure.selfthrottling.enable is false fs.azure.autothrottling.enable is true.
+
+
+---
+
+* [HADOOP-14769](https://issues.apache.org/jira/browse/HADOOP-14769) | *Major* | **WASB: delete recursive should not fail if a file is deleted**
+
+Recursive directory delete improvement for the wasb filesystem.
+
+
+---
+
+* [YARN-6959](https://issues.apache.org/jira/browse/YARN-6959) | *Major* | **RM may allocate wrong AM Container for new attempt**
+
+ResourceManager will now record ResourceRequests from different attempts into different objects.
+
+
+---
+
+* [HDFS-12303](https://issues.apache.org/jira/browse/HDFS-12303) | *Blocker* | **Change default EC cell size to 1MB for better performance**
+
+The cell size of the provided HDFS erasure coding policies has been changed from 64k to 1024k for better performance. The policy names have all been changed accordingly, i.e. RS-6-3.1024k.
+
+
+---
+
+* [HDFS-12258](https://issues.apache.org/jira/browse/HDFS-12258) | *Major* | **ec -listPolicies should list all policies in system, no matter it's enabled or disabled**
+
+<!-- markdown -->
+
+`hdfs ec -listPolicies` now lists enabled, disabled, and removed policies, rather than just enabled policies.
+
+
+---
+
+* [MAPREDUCE-6892](https://issues.apache.org/jira/browse/MAPREDUCE-6892) | *Major* | **Issues with the count of failed/killed tasks in the jhist file**
+
+This adds some new job counters, the number of failed MAP/REDUCE tasks and the number of killed MAP/REDUCE tasks.
+
+
+---
+
+* [YARN-5355](https://issues.apache.org/jira/browse/YARN-5355) | *Critical* | **YARN Timeline Service v.2: alpha 2**
+
+We are releasing the alpha2 version of a major revision of YARN Timeline Service: v.2. YARN Timeline Service v.2 addresses two major challenges: improving scalability and reliability of Timeline Service, and enhancing usability by introducing flows and aggregation.
+
+YARN Timeline Service v.2 alpha1 was introduced in 3.0.0-alpha1 via YARN-2928.
+
+YARN Timeline Service v.2 alpha2 is now being provided so that users and developers can test it and provide feedback and suggestions for making it a ready replacement for Timeline Service v.1.x. Security is provided via Kerberos Authentication and delegation tokens. There is also a simple read level authorization provided via whitelists.
+
+Some of the notable improvements since alpha-1 are:
+- Security via Kerberos Authentication and delegation tokens
+- Read side simple authorization via whitelist
+- Client configurable entity sort ordering
+- New REST APIs for apps, app attempts, containers, fetching metrics by timerange, pagination, sub-app entities
+- Support for storing sub-application entities (entities that exist outside the scope of an application)
+- Configurable TTLs (time-to-live) for tables, configurable table prefixes, configurable hbase cluster
+- Flow level aggregations done as dynamic (table level) coprocessors
+- Uses latest stable HBase release 1.2.6
+
+More details are available in the [YARN Timeline Service v.2](./hadoop-yarn/hadoop-yarn-site/TimelineServiceV2.html) documentation.
+
+
+---
+
+* [HADOOP-14364](https://issues.apache.org/jira/browse/HADOOP-14364) | *Major* | **refresh changelog/release notes with newer Apache Yetus build**
+
+Additionally, this patch updates maven-site-plugin to 3.6 and doxia-module-markdown to 1.8-SNAPSHOT to work around problems with unknown schemas in URLs in markdown formatted content.
+
+
+---
+
+* [HADOOP-13345](https://issues.apache.org/jira/browse/HADOOP-13345) | *Major* | **S3Guard: Improved Consistency for S3A**
+
+S3Guard (pronounced see-guard) is a new feature for the S3A connector to Amazon S3, which uses DynamoDB for a high performance and consistent metadata repository. Essentially: S3Guard caches directory information, so your S3A clients get faster lookups and resilience to inconsistency between S3 list operations and the status of objects. When files are created, with S3Guard, they'll always be found. 
+
+S3Guard does not address update consistency: if a file is updated, while the directory information will be updated, calling open() on the path may still return the old data. Similarly, deleted objects may also potentially be opened.
+
+Please consult the S3Guard documentation in the Amazon S3 section of our documentation.
+
+Note: part of this update includes moving to a new version of the AWS SDK 1.11, one which includes the Dynamo DB client and its a shaded version of Jackson 2. The large aws-sdk-bundle JAR is needed to use the S3A client with or without S3Guard enabled. The good news: because Jackson is shaded, there will be no conflict between any Jackson version used in your application and that which the AWS SDK needs.
+
+
+---
+
+* [HADOOP-14414](https://issues.apache.org/jira/browse/HADOOP-14414) | *Minor* | **Calling maven-site-plugin directly for docs profile is unnecessary**
+
+maven-site-plugin is no longer called directly at package phase.
+
+
+---
+
+* [HDFS-12300](https://issues.apache.org/jira/browse/HDFS-12300) | *Major* | **Audit-log delegation token related operations**
+
+<!-- markdown -->
+
+NameNode now audit-logs `getDelegationToken`, `renewDelegationToken`, `cancelDelegationToken`.
+
+
+---
+
+* [HDFS-12218](https://issues.apache.org/jira/browse/HDFS-12218) | *Blocker* | **Rename split EC / replicated block metrics in BlockManager**
+
+This renames ClientProtocol#getECBlockGroupsStats to ClientProtocol#getEcBlockGroupStats and ClientProtocol#getBlockStats to ClientProtocol#getReplicatedBlockStats. The return-type classes have also been similarly renamed. Their fields have also been renamed to drop the "stats" suffix.
+
+Additionally, ECBlockGroupStats#pendingDeletionBlockGroups has been renamed to ECBlockGroupStats#pendingDeletionBlocks, to clarify that this is the number of blocks, not block groups, pending deletion. The corresponding NameNode metric has also been renamed to PendingDeletionECBlocks.
+
+
+---
+
+* [HADOOP-13421](https://issues.apache.org/jira/browse/HADOOP-13421) | *Minor* | **Switch to v2 of the S3 List Objects API in S3A**
+
+S3A now defaults to using the "v2" S3 list API, which speeds up large-scale path listings. Non-AWS S3 implementations may not support this API: consult the S3A documentation on how to revert to the v1 API.
+
+
+---
+
+* [HADOOP-14847](https://issues.apache.org/jira/browse/HADOOP-14847) | *Blocker* | **Remove Guava Supplier and change to java Supplier in AMRMClient and AMRMClientAysnc**
+
+AMRMClient#waitFor and AMRMClientAsync#waitFor now take java.util.function.Supplier as an argument, rather than com.google.common.base.Supplier.
+
+
+---
+
+* [HADOOP-14520](https://issues.apache.org/jira/browse/HADOOP-14520) | *Major* | **WASB: Block compaction for Azure Block Blobs**
+
+Block Compaction for Azure Block Blobs. When the number of blocks in a block blob is above 32000, the process of compaction replaces a sequence of small blocks with with one big block.
+
+
+---
+
+* [HDFS-12412](https://issues.apache.org/jira/browse/HDFS-12412) | *Major* | **Change ErasureCodingWorker.stripedReadPool to cached thread pool**
+
+Changed {{stripedReadPool}} to unbounded cachedThreadPool.  User should combine {{dfs.datanode.ec.reconstruction.stripedblock.threads}} and {{dfs.namenode.replication.max-streams}} to tune recovery performance.
+
+
+---
+
+* [HDFS-12414](https://issues.apache.org/jira/browse/HDFS-12414) | *Major* | **Ensure to use CLI command to enable/disable erasure coding policy**
+
+dfs.namenode.ec.policies.enabled was removed in order to ensure there is only one approach to enable/disable erasure coding policies to avoid sync up.
+
+
+---
+
+* [HDFS-12438](https://issues.apache.org/jira/browse/HDFS-12438) | *Major* | **Rename dfs.datanode.ec.reconstruction.stripedblock.threads.size to dfs.datanode.ec.reconstruction.threads**
+
+<!-- markdown -->
+
+Config key `dfs.datanode.ec.reconstruction.stripedblock.threads.size` has been renamed to `dfs.datanode.ec.reconstruction.threads`.
+
+
+---
+
+* [HADOOP-14738](https://issues.apache.org/jira/browse/HADOOP-14738) | *Blocker* | **Remove S3N and obsolete bits of S3A; rework docs**
+
+\* The s3n:// client has been removed. Please upgrade to the s3a:// client.
+\* The s3a's original output stream has been removed, the "fast" output stream is the sole option available. There is no need to explicitly enable this, and trying to disable it (fs.s3a.fast.upload=false) will have no effect.
+
+
+---
+
+* [HDFS-7859](https://issues.apache.org/jira/browse/HDFS-7859) | *Major* | **Erasure Coding: Persist erasure coding policies in NameNode**
+
+Persist all built-in erasure coding policies and user defined erasure coding policies into NameNode fsImage and editlog reliably, so that all erasure coding policies remain consistent after NameNode restart.
+
+
+---
+
+* [HDFS-12395](https://issues.apache.org/jira/browse/HDFS-12395) | *Major* | **Support erasure coding policy operations in namenode edit log**
+
+See RN in HDFS-7859 as part of the work.
+
+
+---
+
+* [HADOOP-14670](https://issues.apache.org/jira/browse/HADOOP-14670) | *Major* | **Increase minimum cmake version for all platforms**
+
+CMake v3.1.0 is now the minimum version required to build Apache Hadoop's native components.
+
+
+---
+
+* [HDFS-11799](https://issues.apache.org/jira/browse/HDFS-11799) | *Major* | **Introduce a config to allow setting up write pipeline with fewer nodes than replication factor**
+
+Added new configuration "dfs.client.block.write.replace-datanode-on-failure.min-replication".
+     
+    The minimum number of replications that are needed to not to fail
+      the write pipeline if new datanodes can not be found to replace
+      failed datanodes (could be due to network failure) in the write pipeline.
+      If the number of the remaining datanodes in the write pipeline is greater
+      than or equal to this property value, continue writing to the remaining nodes.
+      Otherwise throw exception.
+
+      If this is set to 0, an exception will be thrown, when a replacement
+      can not be found.
+
+
+---
+
+* [HDFS-12447](https://issues.apache.org/jira/browse/HDFS-12447) | *Major* | **Rename AddECPolicyResponse to AddErasureCodingPolicyResponse**
+
+HdfsAdmin#addErasureCodingPolicies now returns an AddErasureCodingPolicyResponse[] rather than AddECPolicyResponse[]. The corresponding RPC definition and PB message have also been renamed.
+
+
+---
+
+* [HDFS-7337](https://issues.apache.org/jira/browse/HDFS-7337) | *Critical* | **Configurable and pluggable erasure codec and policy**
+
+This allows users to:
+\* develop and plugin their own erasure codec and coders. The plugin will be loaded automatically from hadoop jars, the corresponding codec and coder will be registered for runtime use.
+\* define their own erasure coding policies thru an xml file and CLI command. The added policies will be persisted into fsimage.
+
+
+

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

@@ -101,6 +101,10 @@ public class TestCommonConfigurationFields extends TestConfigurationFieldsBase {
 
     // O3 properties are in a different subtree.
     xmlPrefixToSkipCompare.add("fs.o3.");
+    
+    //ftp properties are in a different subtree.
+    // - org.apache.hadoop.fs.ftp.FTPFileSystem.
+    xmlPrefixToSkipCompare.add("fs.ftp.impl");
 
     // WASB properties are in a different subtree.
     // - org.apache.hadoop.fs.azure.NativeAzureFileSystem

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

@@ -23,9 +23,12 @@ import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
+import static org.mockito.Mockito.verify;
 
 import java.io.IOException;
+import java.net.NoRouteToHostException;
 import java.net.URI;
+import java.net.UnknownHostException;
 import java.security.GeneralSecurityException;
 import java.security.NoSuchAlgorithmException;
 
@@ -33,6 +36,9 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.crypto.key.KeyProvider;
 import org.apache.hadoop.crypto.key.KeyProvider.Options;
 import org.apache.hadoop.crypto.key.KeyProviderCryptoExtension;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.apache.hadoop.net.ConnectTimeoutException;
+import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.authentication.client.AuthenticationException;
 import org.apache.hadoop.security.authorize.AuthorizationException;
 import org.junit.Test;
@@ -47,14 +53,17 @@ public class TestLoadBalancingKMSClientProvider {
     Configuration conf = new Configuration();
     KeyProvider kp = new KMSClientProvider.Factory().createProvider(new URI(
         "kms://http@host1/kms/foo"), conf);
-    assertTrue(kp instanceof KMSClientProvider);
-    assertEquals("http://host1/kms/foo/v1/",
-        ((KMSClientProvider) kp).getKMSUrl());
+    assertTrue(kp instanceof LoadBalancingKMSClientProvider);
+    KMSClientProvider[] providers =
+        ((LoadBalancingKMSClientProvider) kp).getProviders();
+    assertEquals(1, providers.length);
+    assertEquals(Sets.newHashSet("http://host1/kms/foo/v1/"),
+        Sets.newHashSet(providers[0].getKMSUrl()));
 
     kp = new KMSClientProvider.Factory().createProvider(new URI(
         "kms://http@host1;host2;host3/kms/foo"), conf);
     assertTrue(kp instanceof LoadBalancingKMSClientProvider);
-    KMSClientProvider[] providers =
+    providers =
         ((LoadBalancingKMSClientProvider) kp).getProviders();
     assertEquals(3, providers.length);
     assertEquals(Sets.newHashSet("http://host1/kms/foo/v1/",
@@ -320,4 +329,298 @@ public class TestLoadBalancingKMSClientProvider {
     Mockito.verify(p1, Mockito.times(1)).warmUpEncryptedKeys(keyName);
     Mockito.verify(p2, Mockito.times(1)).warmUpEncryptedKeys(keyName);
   }
-}
+
+  /**
+   * Tests whether retryPolicy fails immediately, after trying each provider
+   * once, on encountering IOException which is not SocketException.
+   * @throws Exception
+   */
+  @Test
+  public void testClientRetriesWithIOException() throws Exception {
+    Configuration conf = new Configuration();
+    // Setting total failover attempts to .
+    conf.setInt(
+        CommonConfigurationKeysPublic.KMS_CLIENT_FAILOVER_MAX_RETRIES_KEY, 10);
+    KMSClientProvider p1 = mock(KMSClientProvider.class);
+    when(p1.getMetadata(Mockito.anyString()))
+        .thenThrow(new IOException("p1"));
+    KMSClientProvider p2 = mock(KMSClientProvider.class);
+    when(p2.getMetadata(Mockito.anyString()))
+        .thenThrow(new IOException("p2"));
+    KMSClientProvider p3 = mock(KMSClientProvider.class);
+    when(p3.getMetadata(Mockito.anyString()))
+        .thenThrow(new IOException("p3"));
+
+    when(p1.getKMSUrl()).thenReturn("p1");
+    when(p2.getKMSUrl()).thenReturn("p2");
+    when(p3.getKMSUrl()).thenReturn("p3");
+    LoadBalancingKMSClientProvider kp = new LoadBalancingKMSClientProvider(
+        new KMSClientProvider[] {p1, p2, p3}, 0, conf);
+    try {
+      kp.getMetadata("test3");
+      fail("Should fail since all providers threw an IOException");
+    } catch (Exception e) {
+      assertTrue(e instanceof IOException);
+    }
+    verify(kp.getProviders()[0], Mockito.times(1))
+        .getMetadata(Mockito.eq("test3"));
+    verify(kp.getProviders()[1], Mockito.times(1))
+        .getMetadata(Mockito.eq("test3"));
+    verify(kp.getProviders()[2], Mockito.times(1))
+        .getMetadata(Mockito.eq("test3"));
+  }
+
+  /**
+   * Tests that client doesn't retry once it encounters AccessControlException
+   * from first provider.
+   * This assumes all the kms servers are configured with identical access to
+   * keys.
+   * @throws Exception
+   */
+  @Test
+  public void testClientRetriesWithAccessControlException() throws Exception {
+    Configuration conf = new Configuration();
+    conf.setInt(
+        CommonConfigurationKeysPublic.KMS_CLIENT_FAILOVER_MAX_RETRIES_KEY, 3);
+    KMSClientProvider p1 = mock(KMSClientProvider.class);
+    when(p1.createKey(Mockito.anyString(), Mockito.any(Options.class)))
+        .thenThrow(new AccessControlException("p1"));
+    KMSClientProvider p2 = mock(KMSClientProvider.class);
+    when(p2.createKey(Mockito.anyString(), Mockito.any(Options.class)))
+        .thenThrow(new IOException("p2"));
+    KMSClientProvider p3 = mock(KMSClientProvider.class);
+    when(p3.createKey(Mockito.anyString(), Mockito.any(Options.class)))
+        .thenThrow(new IOException("p3"));
+
+    when(p1.getKMSUrl()).thenReturn("p1");
+    when(p2.getKMSUrl()).thenReturn("p2");
+    when(p3.getKMSUrl()).thenReturn("p3");
+    LoadBalancingKMSClientProvider kp = new LoadBalancingKMSClientProvider(
+        new KMSClientProvider[] {p1, p2, p3}, 0, conf);
+    try {
+      kp.createKey("test3", new Options(conf));
+      fail("Should fail because provider p1 threw an AccessControlException");
+    } catch (Exception e) {
+      assertTrue(e instanceof AccessControlException);
+    }
+    verify(p1, Mockito.times(1)).createKey(Mockito.eq("test3"),
+            Mockito.any(Options.class));
+    verify(p2, Mockito.never()).createKey(Mockito.eq("test3"),
+            Mockito.any(Options.class));
+    verify(p3, Mockito.never()).createKey(Mockito.eq("test3"),
+            Mockito.any(Options.class));
+  }
+
+  /**
+   * Tests that client doesn't retry once it encounters RunTimeException
+   * from first provider.
+   * This assumes all the kms servers are configured with identical access to
+   * keys.
+   * @throws Exception
+   */
+  @Test
+  public void testClientRetriesWithRuntimeException() throws Exception {
+    Configuration conf = new Configuration();
+    conf.setInt(
+        CommonConfigurationKeysPublic.KMS_CLIENT_FAILOVER_MAX_RETRIES_KEY, 3);
+    KMSClientProvider p1 = mock(KMSClientProvider.class);
+    when(p1.createKey(Mockito.anyString(), Mockito.any(Options.class)))
+        .thenThrow(new RuntimeException("p1"));
+    KMSClientProvider p2 = mock(KMSClientProvider.class);
+    when(p2.createKey(Mockito.anyString(), Mockito.any(Options.class)))
+        .thenThrow(new IOException("p2"));
+
+    when(p1.getKMSUrl()).thenReturn("p1");
+    when(p2.getKMSUrl()).thenReturn("p2");
+
+    LoadBalancingKMSClientProvider kp = new LoadBalancingKMSClientProvider(
+        new KMSClientProvider[] {p1, p2}, 0, conf);
+    try {
+      kp.createKey("test3", new Options(conf));
+      fail("Should fail since provider p1 threw RuntimeException");
+    } catch (Exception e) {
+      assertTrue(e instanceof RuntimeException);
+    }
+    verify(p1, Mockito.times(1)).createKey(Mockito.eq("test3"),
+            Mockito.any(Options.class));
+    verify(p2, Mockito.never()).createKey(Mockito.eq("test3"),
+            Mockito.any(Options.class));
+  }
+
+  /**
+   * Tests the client retries until it finds a good provider.
+   * @throws Exception
+   */
+  @Test
+  public void testClientRetriesWithTimeoutsException() throws Exception {
+    Configuration conf = new Configuration();
+    conf.setInt(
+        CommonConfigurationKeysPublic.KMS_CLIENT_FAILOVER_MAX_RETRIES_KEY, 4);
+    KMSClientProvider p1 = mock(KMSClientProvider.class);
+    when(p1.createKey(Mockito.anyString(), Mockito.any(Options.class)))
+        .thenThrow(new ConnectTimeoutException("p1"));
+    KMSClientProvider p2 = mock(KMSClientProvider.class);
+    when(p2.createKey(Mockito.anyString(), Mockito.any(Options.class)))
+        .thenThrow(new UnknownHostException("p2"));
+    KMSClientProvider p3 = mock(KMSClientProvider.class);
+    when(p3.createKey(Mockito.anyString(), Mockito.any(Options.class)))
+        .thenThrow(new NoRouteToHostException("p3"));
+    KMSClientProvider p4 = mock(KMSClientProvider.class);
+    when(p4.createKey(Mockito.anyString(), Mockito.any(Options.class)))
+        .thenReturn(
+            new KMSClientProvider.KMSKeyVersion("test3", "v1", new byte[0]));
+    when(p1.getKMSUrl()).thenReturn("p1");
+    when(p2.getKMSUrl()).thenReturn("p2");
+    when(p3.getKMSUrl()).thenReturn("p3");
+    when(p4.getKMSUrl()).thenReturn("p4");
+    LoadBalancingKMSClientProvider kp = new LoadBalancingKMSClientProvider(
+        new KMSClientProvider[] {p1, p2, p3, p4}, 0, conf);
+    try {
+      kp.createKey("test3", new Options(conf));
+    } catch (Exception e) {
+      fail("Provider p4 should have answered the request.");
+    }
+    verify(p1, Mockito.times(1)).createKey(Mockito.eq("test3"),
+            Mockito.any(Options.class));
+    verify(p2, Mockito.times(1)).createKey(Mockito.eq("test3"),
+            Mockito.any(Options.class));
+    verify(p3, Mockito.times(1)).createKey(Mockito.eq("test3"),
+            Mockito.any(Options.class));
+    verify(p4, Mockito.times(1)).createKey(Mockito.eq("test3"),
+            Mockito.any(Options.class));
+  }
+
+  /**
+   * Tests the operation succeeds second time after ConnectTimeoutException.
+   * @throws Exception
+   */
+  @Test
+  public void testClientRetriesSucceedsSecondTime() throws Exception {
+    Configuration conf = new Configuration();
+    conf.setInt(
+        CommonConfigurationKeysPublic.KMS_CLIENT_FAILOVER_MAX_RETRIES_KEY, 3);
+    KMSClientProvider p1 = mock(KMSClientProvider.class);
+    when(p1.createKey(Mockito.anyString(), Mockito.any(Options.class)))
+        .thenThrow(new ConnectTimeoutException("p1"))
+        .thenReturn(new KMSClientProvider.KMSKeyVersion("test3", "v1",
+                new byte[0]));
+    KMSClientProvider p2 = mock(KMSClientProvider.class);
+    when(p2.createKey(Mockito.anyString(), Mockito.any(Options.class)))
+        .thenThrow(new ConnectTimeoutException("p2"));
+
+    when(p1.getKMSUrl()).thenReturn("p1");
+    when(p2.getKMSUrl()).thenReturn("p2");
+
+    LoadBalancingKMSClientProvider kp = new LoadBalancingKMSClientProvider(
+        new KMSClientProvider[] {p1, p2}, 0, conf);
+    try {
+      kp.createKey("test3", new Options(conf));
+    } catch (Exception e) {
+      fail("Provider p1 should have answered the request second time.");
+    }
+    verify(p1, Mockito.times(2)).createKey(Mockito.eq("test3"),
+            Mockito.any(Options.class));
+    verify(p2, Mockito.times(1)).createKey(Mockito.eq("test3"),
+            Mockito.any(Options.class));
+  }
+
+  /**
+   * Tests whether retryPolicy retries specified number of times.
+   * @throws Exception
+   */
+  @Test
+  public void testClientRetriesSpecifiedNumberOfTimes() throws Exception {
+    Configuration conf = new Configuration();
+    conf.setInt(
+        CommonConfigurationKeysPublic.KMS_CLIENT_FAILOVER_MAX_RETRIES_KEY, 10);
+    KMSClientProvider p1 = mock(KMSClientProvider.class);
+    when(p1.createKey(Mockito.anyString(), Mockito.any(Options.class)))
+        .thenThrow(new ConnectTimeoutException("p1"));
+    KMSClientProvider p2 = mock(KMSClientProvider.class);
+    when(p2.createKey(Mockito.anyString(), Mockito.any(Options.class)))
+        .thenThrow(new ConnectTimeoutException("p2"));
+
+    when(p1.getKMSUrl()).thenReturn("p1");
+    when(p2.getKMSUrl()).thenReturn("p2");
+
+    LoadBalancingKMSClientProvider kp = new LoadBalancingKMSClientProvider(
+        new KMSClientProvider[] {p1, p2}, 0, conf);
+    try {
+      kp.createKey("test3", new Options(conf));
+      fail("Should fail");
+    } catch (Exception e) {
+     assert (e instanceof ConnectTimeoutException);
+    }
+    verify(p1, Mockito.times(6)).createKey(Mockito.eq("test3"),
+            Mockito.any(Options.class));
+    verify(p2, Mockito.times(5)).createKey(Mockito.eq("test3"),
+            Mockito.any(Options.class));
+  }
+
+  /**
+   * Tests whether retryPolicy retries number of times equals to number of
+   * providers if conf kms.client.failover.max.attempts is not set.
+   * @throws Exception
+   */
+  @Test
+  public void testClientRetriesIfMaxAttemptsNotSet() throws Exception {
+    Configuration conf = new Configuration();
+    KMSClientProvider p1 = mock(KMSClientProvider.class);
+    when(p1.createKey(Mockito.anyString(), Mockito.any(Options.class)))
+        .thenThrow(new ConnectTimeoutException("p1"));
+    KMSClientProvider p2 = mock(KMSClientProvider.class);
+    when(p2.createKey(Mockito.anyString(), Mockito.any(Options.class)))
+        .thenThrow(new ConnectTimeoutException("p2"));
+
+    when(p1.getKMSUrl()).thenReturn("p1");
+    when(p2.getKMSUrl()).thenReturn("p2");
+
+    LoadBalancingKMSClientProvider kp = new LoadBalancingKMSClientProvider(
+        new KMSClientProvider[] {p1, p2}, 0, conf);
+    try {
+      kp.createKey("test3", new Options(conf));
+      fail("Should fail");
+    } catch (Exception e) {
+     assert (e instanceof ConnectTimeoutException);
+    }
+    verify(p1, Mockito.times(2)).createKey(Mockito.eq("test3"),
+            Mockito.any(Options.class));
+    verify(p2, Mockito.times(1)).createKey(Mockito.eq("test3"),
+            Mockito.any(Options.class));
+  }
+
+  /**
+   * Tests that client reties each provider once, when it encounters
+   * AuthenticationException wrapped in an IOException from first provider.
+   * @throws Exception
+   */
+  @Test
+  public void testClientRetriesWithAuthenticationExceptionWrappedinIOException()
+      throws Exception {
+    Configuration conf = new Configuration();
+    conf.setInt(
+        CommonConfigurationKeysPublic.KMS_CLIENT_FAILOVER_MAX_RETRIES_KEY, 3);
+    KMSClientProvider p1 = mock(KMSClientProvider.class);
+    when(p1.createKey(Mockito.anyString(), Mockito.any(Options.class)))
+        .thenThrow(new IOException(new AuthenticationException("p1")));
+    KMSClientProvider p2 = mock(KMSClientProvider.class);
+    when(p2.createKey(Mockito.anyString(), Mockito.any(Options.class)))
+        .thenThrow(new IOException(new AuthenticationException("p1")));
+
+    when(p1.getKMSUrl()).thenReturn("p1");
+    when(p2.getKMSUrl()).thenReturn("p2");
+
+    LoadBalancingKMSClientProvider kp = new LoadBalancingKMSClientProvider(
+        new KMSClientProvider[] {p1, p2}, 0, conf);
+    try {
+      kp.createKey("test3", new Options(conf));
+      fail("Should fail since provider p1 threw AuthenticationException");
+    } catch (Exception e) {
+      assertTrue(e.getCause() instanceof AuthenticationException);
+    }
+    verify(p1, Mockito.times(1)).createKey(Mockito.eq("test3"),
+            Mockito.any(Options.class));
+    verify(p2, Mockito.times(1)).createKey(Mockito.eq("test3"),
+            Mockito.any(Options.class));
+  }
+}

+ 13 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/serializer/TestSerializationFactory.java

@@ -51,6 +51,19 @@ public class TestSerializationFactory {
     SerializationFactory factory = new SerializationFactory(conf);
   }
 
+  /**
+   * Test the case when {@code IO_SERIALIZATIONS_KEY}
+   * is not set at all, because something unset this key.
+   * This shouldn't result in any error, the defaults present
+   * in construction should be used in this case.
+   */
+  @Test
+  public void testSerializationKeyIsUnset() {
+    Configuration conf = new Configuration();
+    conf.unset(CommonConfigurationKeys.IO_SERIALIZATIONS_KEY);
+    SerializationFactory factory = new SerializationFactory(conf);
+  }
+
   @Test
   public void testSerializationKeyIsInvalid() {
     Configuration conf = new Configuration();

+ 79 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/http/TestCrossOriginFilter.java

@@ -127,6 +127,85 @@ public class TestCrossOriginFilter {
     Assert.assertFalse(filter.areOriginsAllowed("foo.nomatch1.com foo.nomatch2.com"));
   }
 
+  @Test
+  public void testRegexPatternMatchingOrigins() throws ServletException, IOException {
+
+    // Setup the configuration settings of the server
+    Map<String, String> conf = new HashMap<String, String>();
+    conf.put(CrossOriginFilter.ALLOWED_ORIGINS, "regex:.*[.]example[.]com");
+    FilterConfig filterConfig = new FilterConfigTest(conf);
+
+    // Object under test
+    CrossOriginFilter filter = new CrossOriginFilter();
+    filter.init(filterConfig);
+
+    // match multiple sub-domains
+    Assert.assertFalse(filter.areOriginsAllowed("example.com"));
+    Assert.assertFalse(filter.areOriginsAllowed("foo:example.com"));
+    Assert.assertTrue(filter.areOriginsAllowed("foo.example.com"));
+    Assert.assertTrue(filter.areOriginsAllowed("foo.bar.example.com"));
+
+    // First origin is allowed
+    Assert.assertTrue(filter.areOriginsAllowed("foo.example.com foo.nomatch.com"));
+    // Second origin is allowed
+    Assert.assertTrue(filter.areOriginsAllowed("foo.nomatch.com foo.example.com"));
+    // No origin in list is allowed
+    Assert.assertFalse(filter.areOriginsAllowed("foo.nomatch1.com foo.nomatch2.com"));
+  }
+
+  @Test
+  public void testComplexRegexPatternMatchingOrigins() throws ServletException, IOException {
+
+    // Setup the configuration settings of the server
+    Map<String, String> conf = new HashMap<String, String>();
+    conf.put(CrossOriginFilter.ALLOWED_ORIGINS, "regex:https?:\\/\\/sub1[.]example[.]com(:[0-9]+)?");
+    FilterConfig filterConfig = new FilterConfigTest(conf);
+
+    // Object under test
+    CrossOriginFilter filter = new CrossOriginFilter();
+    filter.init(filterConfig);
+
+    Assert.assertTrue(filter.areOriginsAllowed("http://sub1.example.com"));
+    Assert.assertTrue(filter.areOriginsAllowed("https://sub1.example.com"));
+    Assert.assertTrue(filter.areOriginsAllowed("http://sub1.example.com:1234"));
+    Assert.assertTrue(filter.areOriginsAllowed("https://sub1.example.com:8080"));
+
+    // No origin in list is allowed
+    Assert.assertFalse(filter.areOriginsAllowed("foo.nomatch1.com foo.nomatch2.com"));
+  }
+
+  @Test
+  public void testMixedRegexPatternMatchingOrigins() throws ServletException, IOException {
+
+    // Setup the configuration settings of the server
+    Map<String, String> conf = new HashMap<String, String>();
+    conf.put(CrossOriginFilter.ALLOWED_ORIGINS, "regex:https?:\\/\\/sub1[.]example[.]com(:[0-9]+)?, "
+            + "*.example2.com");
+    FilterConfig filterConfig = new FilterConfigTest(conf);
+
+    // Object under test
+    CrossOriginFilter filter = new CrossOriginFilter();
+    filter.init(filterConfig);
+
+    Assert.assertTrue(filter.areOriginsAllowed("http://sub1.example.com"));
+    Assert.assertTrue(filter.areOriginsAllowed("https://sub1.example.com"));
+    Assert.assertTrue(filter.areOriginsAllowed("http://sub1.example.com:1234"));
+    Assert.assertTrue(filter.areOriginsAllowed("https://sub1.example.com:8080"));
+
+    // match multiple sub-domains
+    Assert.assertFalse(filter.areOriginsAllowed("example2.com"));
+    Assert.assertFalse(filter.areOriginsAllowed("foo:example2.com"));
+    Assert.assertTrue(filter.areOriginsAllowed("foo.example2.com"));
+    Assert.assertTrue(filter.areOriginsAllowed("foo.bar.example2.com"));
+
+    // First origin is allowed
+    Assert.assertTrue(filter.areOriginsAllowed("foo.example2.com foo.nomatch.com"));
+    // Second origin is allowed
+    Assert.assertTrue(filter.areOriginsAllowed("foo.nomatch.com foo.example2.com"));
+    // No origin in list is allowed
+    Assert.assertFalse(filter.areOriginsAllowed("foo.nomatch1.com foo.nomatch2.com"));
+  }
+
   @Test
   public void testDisallowedOrigin() throws ServletException, IOException {
 

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

@@ -111,12 +111,21 @@ public class TestDelegationTokenAuthenticationHandlerWithMocks {
 
   @Test
   public void testManagementOperations() throws Exception {
-      testNonManagementOperation();
-      testManagementOperationErrors();
-      testGetToken(null, new Text("foo"));
-      testGetToken("bar", new Text("foo"));
-      testCancelToken();
-      testRenewToken();
+    final Text testTokenKind = new Text("foo");
+    final String testRenewer = "bar";
+    final String testService = "192.168.64.101:8888";
+    testNonManagementOperation();
+    testManagementOperationErrors();
+    testGetToken(null, null, testTokenKind);
+    testGetToken(testRenewer, null, testTokenKind);
+    testCancelToken();
+    testRenewToken(testRenewer);
+
+    // Management operations against token requested with service parameter
+    Token<DelegationTokenIdentifier> testToken =
+        testGetToken(testRenewer, testService, testTokenKind);
+    testRenewToken(testToken, testRenewer);
+    testCancelToken(testToken);
   }
 
   private void testNonManagementOperation() throws Exception {
@@ -156,8 +165,8 @@ public class TestDelegationTokenAuthenticationHandlerWithMocks {
         Mockito.eq("mock"));
   }
 
-  private void testGetToken(String renewer, Text expectedTokenKind)
-      throws Exception {
+  private Token<DelegationTokenIdentifier> testGetToken(String renewer,
+      String service, Text expectedTokenKind) throws Exception {
     DelegationTokenAuthenticator.DelegationTokenOperation op =
         DelegationTokenAuthenticator.DelegationTokenOperation.
             GETDELEGATIONTOKEN;
@@ -173,10 +182,14 @@ public class TestDelegationTokenAuthenticationHandlerWithMocks {
         new StringWriter()));
     Assert.assertFalse(handler.managementOperation(token, request, response));
 
-    Mockito.when(request.getQueryString()).
-        thenReturn(DelegationTokenAuthenticator.OP_PARAM + "=" + op.toString() +
-        "&" + DelegationTokenAuthenticator.RENEWER_PARAM + "=" + renewer);
-
+    String queryString =
+        DelegationTokenAuthenticator.OP_PARAM + "=" + op.toString() + "&" +
+        DelegationTokenAuthenticator.RENEWER_PARAM + "=" + renewer;
+    if (service != null) {
+      queryString += "&" + DelegationTokenAuthenticator.SERVICE_PARAM + "="
+          + service;
+    }
+    Mockito.when(request.getQueryString()).thenReturn(queryString);
     Mockito.reset(response);
     Mockito.reset(token);
     Mockito.when(token.getUserName()).thenReturn("user");
@@ -208,10 +221,25 @@ public class TestDelegationTokenAuthenticationHandlerWithMocks {
     dt.decodeFromUrlString(tokenStr);
     handler.getTokenManager().verifyToken(dt);
     Assert.assertEquals(expectedTokenKind, dt.getKind());
+    if (service != null) {
+      Assert.assertEquals(service, dt.getService().toString());
+    } else {
+      Assert.assertEquals(0, dt.getService().getLength());
+    }
+    return dt;
   }
 
   @SuppressWarnings("unchecked")
   private void testCancelToken() throws Exception {
+    Token<DelegationTokenIdentifier> token =
+        (Token<DelegationTokenIdentifier>) handler.getTokenManager()
+            .createToken(UserGroupInformation.getCurrentUser(), "foo");
+    testCancelToken(token);
+  }
+
+  @SuppressWarnings("unchecked")
+  private void testCancelToken(Token<DelegationTokenIdentifier> token)
+      throws Exception {
     DelegationTokenAuthenticator.DelegationTokenOperation op =
         DelegationTokenAuthenticator.DelegationTokenOperation.
             CANCELDELEGATIONTOKEN;
@@ -228,9 +256,6 @@ public class TestDelegationTokenAuthenticationHandlerWithMocks {
         Mockito.contains("requires the parameter [token]"));
 
     Mockito.reset(response);
-    Token<DelegationTokenIdentifier> token =
-        (Token<DelegationTokenIdentifier>) handler.getTokenManager().createToken(
-            UserGroupInformation.getCurrentUser(), "foo");
     Mockito.when(request.getQueryString()).thenReturn(
         DelegationTokenAuthenticator.OP_PARAM + "=" + op.toString() + "&" +
             DelegationTokenAuthenticator.TOKEN_PARAM + "=" +
@@ -249,7 +274,16 @@ public class TestDelegationTokenAuthenticationHandlerWithMocks {
   }
 
   @SuppressWarnings("unchecked")
-  private void testRenewToken() throws Exception {
+  private void testRenewToken(String testRenewer) throws Exception {
+    Token<DelegationTokenIdentifier> dToken = (Token<DelegationTokenIdentifier>)
+        handler.getTokenManager().createToken(
+            UserGroupInformation.getCurrentUser(), testRenewer);
+    testRenewToken(dToken, testRenewer);
+  }
+
+  @SuppressWarnings("unchecked")
+  private void testRenewToken(Token<DelegationTokenIdentifier> dToken,
+      String testRenewer) throws Exception {
     DelegationTokenAuthenticator.DelegationTokenOperation op =
         DelegationTokenAuthenticator.DelegationTokenOperation.
             RENEWDELEGATIONTOKEN;
@@ -270,7 +304,7 @@ public class TestDelegationTokenAuthenticationHandlerWithMocks {
 
     Mockito.reset(response);
     AuthenticationToken token = Mockito.mock(AuthenticationToken.class);
-    Mockito.when(token.getUserName()).thenReturn("user");
+    Mockito.when(token.getUserName()).thenReturn(testRenewer);
     Assert.assertFalse(handler.managementOperation(token, request, response));
     Mockito.verify(response).sendError(
         Mockito.eq(HttpServletResponse.SC_BAD_REQUEST),
@@ -280,9 +314,7 @@ public class TestDelegationTokenAuthenticationHandlerWithMocks {
     StringWriter writer = new StringWriter();
     PrintWriter pwriter = new PrintWriter(writer);
     Mockito.when(response.getWriter()).thenReturn(pwriter);
-    Token<DelegationTokenIdentifier> dToken =
-        (Token<DelegationTokenIdentifier>) handler.getTokenManager().createToken(
-            UserGroupInformation.getCurrentUser(), "user");
+
     Mockito.when(request.getQueryString()).
         thenReturn(DelegationTokenAuthenticator.OP_PARAM + "=" + op.toString() +
             "&" + DelegationTokenAuthenticator.TOKEN_PARAM + "=" +

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

@@ -2208,7 +2208,7 @@ public class TestKMS {
         "hadoop.kms.authentication.delegation-token.renew-interval.sec", "5");
     writeConf(confDir, conf);
 
-    // Running as a service (e.g. Yarn in practice).
+    // Running as a service (e.g. YARN in practice).
     runServer(null, null, confDir, new KMSCallable<Void>() {
       @Override
       public Void call() throws Exception {
@@ -2223,7 +2223,7 @@ public class TestKMS {
         final InetSocketAddress kmsAddr =
             new InetSocketAddress(getKMSUrl().getHost(), getKMSUrl().getPort());
 
-        // Job 1 (e.g. Yarn log aggregation job), with user DT.
+        // Job 1 (e.g. YARN log aggregation job), with user DT.
         final Collection<Token<?>> job1Token = new HashSet<>();
         doAs("client", new PrivilegedExceptionAction<Void>() {
           @Override
@@ -2268,7 +2268,7 @@ public class TestKMS {
         });
         Assert.assertFalse(job1Token.isEmpty());
 
-        // job 2 (e.g. Another Yarn log aggregation job, with user DT.
+        // job 2 (e.g. Another YARN log aggregation job, with user DT.
         doAs("client", new PrivilegedExceptionAction<Void>() {
           @Override
           public Void run() throws Exception {

+ 41 - 10
hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/FileHandle.java

@@ -38,17 +38,21 @@ public class FileHandle {
   private static final int HANDLE_LEN = 32;
   private byte[] handle; // Opaque handle
   private long fileId = -1;
+  private int namenodeId = -1;
 
   public FileHandle() {
     handle = null;
   }
 
   /**
-   * Handle is a 32 bytes number. For HDFS, the last 8 bytes is fileId.
+   * Handle is a 32 bytes number. For HDFS, the last 8 bytes is fileId
+   * For ViewFs, last 8 byte is fileId while 4 bytes before that is namenodeId
    * @param v file id
+   * @param n namenode id
    */
-  public FileHandle(long v) {
+  public FileHandle(long v, int n) {
     fileId = v;
+    namenodeId = n;
     handle = new byte[HANDLE_LEN];
     handle[0] = (byte)(v >>> 56);
     handle[1] = (byte)(v >>> 48);
@@ -58,11 +62,20 @@ public class FileHandle {
     handle[5] = (byte)(v >>> 16);
     handle[6] = (byte)(v >>>  8);
     handle[7] = (byte)(v >>>  0);
-    for (int i = 8; i < HANDLE_LEN; i++) {
+
+    handle[8] = (byte) (n >>> 24);
+    handle[9] = (byte) (n >>> 16);
+    handle[10] = (byte) (n >>> 8);
+    handle[11] = (byte) (n >>> 0);
+    for (int i = 12; i < HANDLE_LEN; i++) {
       handle[i] = (byte) 0;
     }
   }
-  
+
+  public FileHandle(long v) {
+    this(v, 0);
+  }
+
   public FileHandle(String s) {
     MessageDigest digest;
     try {
@@ -93,22 +106,32 @@ public class FileHandle {
     return true;
   }
 
-  private long bytesToLong(byte[] data) {
+  private long bytesToLong(byte[] data, int offset) {
     ByteBuffer buffer = ByteBuffer.allocate(8);
     for (int i = 0; i < 8; i++) {
-      buffer.put(data[i]);
+      buffer.put(data[i + offset]);
     }
-    buffer.flip();// need flip
+    buffer.flip(); // need flip
     return buffer.getLong();
   }
-  
+
+  private int bytesToInt(byte[] data, int offset) {
+    ByteBuffer buffer = ByteBuffer.allocate(4);
+    for (int i = 0; i < 4; i++) {
+      buffer.put(data[i + offset]);
+    }
+    buffer.flip(); // need flip
+    return buffer.getInt();
+  }
+
   public boolean deserialize(XDR xdr) {
     if (!XDR.verifyLength(xdr, 32)) {
       return false;
     }
     int size = xdr.readInt();
     handle = xdr.readFixedOpaque(size);
-    fileId = bytesToLong(handle);
+    fileId = bytesToLong(handle, 0);
+    namenodeId = bytesToInt(handle, 8);
     return true;
   }
   
@@ -122,11 +145,15 @@ public class FileHandle {
   public long getFileId() {    
     return fileId;
   }
+
+  public int getNamenodeId() {
+    return namenodeId;
+  }
   
   public byte[] getContent() {
     return handle.clone();
   }
-  
+
   @Override
   public String toString() {
     StringBuilder s = new StringBuilder();
@@ -154,4 +181,8 @@ public class FileHandle {
   public int hashCode() {
     return Arrays.hashCode(handle);
   }
+
+  public String dumpFileHandle() {
+    return "fileId: " + fileId + " namenodeId: " + namenodeId;
+  }
 }

+ 2 - 2
hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/request/WRITE3Request.java

@@ -87,7 +87,7 @@ public class WRITE3Request extends RequestWithHandle {
   
   @Override
   public String toString() {
-    return String.format("fileId: %d offset: %d count: %d stableHow: %s",
-        handle.getFileId(), offset, count, stableHow.name());
+    return String.format("fileHandle: %s offset: %d count: %d stableHow: %s",
+        handle.dumpFileHandle(), offset, count, stableHow.name());
   }
 }

+ 5 - 2
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java

@@ -68,7 +68,7 @@ public class DFSStripedInputStream extends DFSInputStream {
   private ByteBuffer curStripeBuf;
   private ByteBuffer parityBuf;
   private final ErasureCodingPolicy ecPolicy;
-  private final RawErasureDecoder decoder;
+  private RawErasureDecoder decoder;
 
   /**
    * Indicate the start/end offset of the current buffered stripe in the
@@ -188,7 +188,10 @@ public class DFSStripedInputStream extends DFSInputStream {
         BUFFER_POOL.putBuffer(parityBuf);
         parityBuf = null;
       }
-      decoder.release();
+      if (decoder != null) {
+        decoder.release();
+        decoder = null;
+      }
     }
   }
 

+ 11 - 1
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsFileStatus.java

@@ -56,7 +56,8 @@ public class HdfsFileStatus extends FileStatus {
   public enum Flags {
     HAS_ACL,
     HAS_CRYPT,
-    HAS_EC;
+    HAS_EC,
+    SNAPSHOT_ENABLED
   }
   private final EnumSet<Flags> flags;
 
@@ -244,6 +245,15 @@ public class HdfsFileStatus extends FileStatus {
     return storagePolicy;
   }
 
+  /**
+   * Check if directory is Snapshot enabled or not.
+   *
+   * @return true if directory is snapshot enabled
+   */
+  public boolean isSnapshotEnabled() {
+    return flags.contains(Flags.SNAPSHOT_ENABLED);
+  }
+
   @Override
   public boolean equals(Object o) {
     // satisfy findbugs

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

@@ -1602,6 +1602,9 @@ public class PBHelperClient {
         case HAS_EC:
           f.add(HdfsFileStatus.Flags.HAS_EC);
           break;
+        case SNAPSHOT_ENABLED:
+          f.add(HdfsFileStatus.Flags.SNAPSHOT_ENABLED);
+          break;
         default:
           // ignore unknown
           break;
@@ -2155,6 +2158,8 @@ public class PBHelperClient {
     int flags = fs.hasAcl()   ? HdfsFileStatusProto.Flags.HAS_ACL_VALUE   : 0;
     flags |= fs.isEncrypted() ? HdfsFileStatusProto.Flags.HAS_CRYPT_VALUE : 0;
     flags |= fs.isErasureCoded() ? HdfsFileStatusProto.Flags.HAS_EC_VALUE : 0;
+    flags |= fs.isSnapshotEnabled() ? HdfsFileStatusProto.Flags
+        .SNAPSHOT_ENABLED_VALUE : 0;
     builder.setFlags(flags);
     return builder.build();
   }

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

@@ -412,6 +412,7 @@ message HdfsFileStatusProto {
     HAS_ACL   = 0x01; // has ACLs
     HAS_CRYPT = 0x02; // encrypted
     HAS_EC    = 0x04; // erasure coded
+    SNAPSHOT_ENABLED    = 0x08; // SNAPSHOT ENABLED
   }
   required FileType fileType = 1;
   required bytes path = 2;          // local name of inode encoded java UTF8

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

@@ -198,6 +198,7 @@ public class HttpFSFileSystem extends FileSystem
 
   public static final String ENC_BIT_JSON = "encBit";
   public static final String EC_BIT_JSON = "ecBit";
+  public static final String SNAPSHOT_BIT_JSON = "seBit";
 
   public static final String DIRECTORY_LISTING_JSON = "DirectoryListing";
   public static final String PARTIAL_LISTING_JSON = "partialListing";
@@ -1066,19 +1067,27 @@ public class HttpFSFileSystem extends FileSystem
     final Boolean aclBit = (Boolean) json.get(ACL_BIT_JSON);
     final Boolean encBit = (Boolean) json.get(ENC_BIT_JSON);
     final Boolean erasureBit = (Boolean) json.get(EC_BIT_JSON);
+    final Boolean snapshotEnabledBit = (Boolean) json.get(SNAPSHOT_BIT_JSON);
     final boolean aBit = (aclBit != null) ? aclBit : false;
     final boolean eBit = (encBit != null) ? encBit : false;
     final boolean ecBit = (erasureBit != null) ? erasureBit : false;
-    if (aBit || eBit || ecBit) {
+    final boolean seBit =
+        (snapshotEnabledBit != null) ? snapshotEnabledBit : false;
+    if (aBit || eBit || ecBit || seBit) {
       // include this for compatibility with 2.x
       FsPermissionExtension deprecatedPerm =
           new FsPermissionExtension(permission, aBit, eBit, ecBit);
-      return new FileStatus(len, FILE_TYPE.DIRECTORY == type,
+      FileStatus fileStatus = new FileStatus(len, FILE_TYPE.DIRECTORY == type,
           replication, blockSize, mTime, aTime, deprecatedPerm, owner, group,
           null, path, aBit, eBit, ecBit);
+      if (seBit) {
+        fileStatus.setSnapShotEnabledFlag(seBit);
+      }
+      return fileStatus;
+    } else {
+      return new FileStatus(len, FILE_TYPE.DIRECTORY == type,
+          replication, blockSize, mTime, aTime, permission, owner, group, path);
     }
-    return new FileStatus(len, FILE_TYPE.DIRECTORY == type,
-        replication, blockSize, mTime, aTime, permission, owner, group, path);
   }
 
   /**

+ 7 - 5
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs/jni_helper.c

@@ -522,19 +522,21 @@ JNIEnv* getJNIEnv(void)
     /* Create a ThreadLocalState for this thread */
     state = threadLocalStorageCreate();
     if (!state) {
+      mutexUnlock(&jvmMutex);
       fprintf(stderr, "getJNIEnv: Unable to create ThreadLocalState\n");
       return NULL;
     }
-    state->env = getGlobalJNIEnv();
-    mutexUnlock(&jvmMutex);
-    if (!state->env) {
-      goto fail;
-    }
     if (threadLocalStorageSet(state)) {
+      mutexUnlock(&jvmMutex);
       goto fail;
     }
     THREAD_LOCAL_STORAGE_SET_QUICK(state);
 
+    state->env = getGlobalJNIEnv();
+    mutexUnlock(&jvmMutex);
+    if (!state->env) {
+      goto fail;
+    }
     return state->env;
 
 fail:

+ 56 - 25
hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/mount/RpcProgramMountd.java

@@ -19,17 +19,20 @@ package org.apache.hadoop.hdfs.nfs.mount;
 import java.io.IOException;
 import java.net.DatagramSocket;
 import java.net.InetAddress;
+import java.net.URI;
 import java.net.InetSocketAddress;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
+import java.util.HashMap;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.hdfs.DFSClient;
-import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.apache.hadoop.hdfs.nfs.conf.NfsConfigKeys;
 import org.apache.hadoop.hdfs.nfs.conf.NfsConfiguration;
+import org.apache.hadoop.hdfs.nfs.nfs3.Nfs3Utils;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.mount.MountEntry;
 import org.apache.hadoop.mount.MountInterface;
@@ -64,14 +67,12 @@ public class RpcProgramMountd extends RpcProgram implements MountInterface {
   public static final int VERSION_2 = 2;
   public static final int VERSION_3 = 3;
 
-  private final DFSClient dfsClient;
-  
-  /** Synchronized list */
+  /** Synchronized list. */
   private final List<MountEntry> mounts;
   
-  /** List that is unmodifiable */
-  private final List<String> exports;
-  
+  /** List that is unmodifiable. */
+  private final HashMap<String, URI> exports;
+  private final NfsConfiguration config;
   private final NfsExports hostsMatcher;
 
   public RpcProgramMountd(NfsConfiguration config,
@@ -84,17 +85,29 @@ public class RpcProgramMountd extends RpcProgram implements MountInterface {
         VERSION_3, registrationSocket, allowInsecurePorts, config.getInt(
               NfsConfigKeys.NFS_UDP_CLIENT_PORTMAP_TIMEOUT_MILLIS_KEY,
               NfsConfigKeys.NFS_UDP_CLIENT_PORTMAP_TIMEOUT_MILLIS_DEFAULT));
-    exports = new ArrayList<String>();
-    exports.add(config.get(NfsConfigKeys.DFS_NFS_EXPORT_POINT_KEY,
-        NfsConfigKeys.DFS_NFS_EXPORT_POINT_DEFAULT));
+    this.config = config;
+    exports = new HashMap<>();
+    addExports();
     this.hostsMatcher = NfsExports.getInstance(config);
     this.mounts = Collections.synchronizedList(new ArrayList<MountEntry>());
     UserGroupInformation.setConfiguration(config);
     SecurityUtil.login(config, NfsConfigKeys.DFS_NFS_KEYTAB_FILE_KEY,
         NfsConfigKeys.DFS_NFS_KERBEROS_PRINCIPAL_KEY);
-    this.dfsClient = new DFSClient(DFSUtilClient.getNNAddress(config), config);
   }
-  
+
+  private void addExports() throws IOException {
+    FileSystem fs = FileSystem.get(config);
+    String[] exportsPath =
+        config.getStrings(NfsConfigKeys.DFS_NFS_EXPORT_POINT_KEY,
+            NfsConfigKeys.DFS_NFS_EXPORT_POINT_DEFAULT);
+    for (String exportPath : exportsPath) {
+      URI exportURI = Nfs3Utils.getResolvedURI(fs, exportPath);
+      LOG.info("FS:" + fs.getScheme() + " adding export Path:" + exportPath +
+          " with URI: " + exportURI.toString());
+      exports.put(exportPath, exportURI);
+    }
+  }
+
   @Override
   public XDR nullOp(XDR out, int xid, InetAddress client) {
     if (LOG.isDebugEnabled()) {
@@ -125,17 +138,28 @@ public class RpcProgramMountd extends RpcProgram implements MountInterface {
     if (LOG.isDebugEnabled()) {
       LOG.debug("Got host: " + host + " path: " + path);
     }
-    if (!exports.contains(path)) {
+    URI exportURI = exports.get(path);
+    if (exportURI == null) {
       LOG.info("Path " + path + " is not shared.");
       MountResponse.writeMNTResponse(Nfs3Status.NFS3ERR_NOENT, out, xid, null);
       return out;
     }
 
+    DFSClient dfsClient = null;
+    try {
+      dfsClient = new DFSClient(exportURI, config);
+    } catch (Exception e) {
+      LOG.error("Can't get handle for export:" + path, e);
+      MountResponse.writeMNTResponse(Nfs3Status.NFS3ERR_NOENT, out, xid, null);
+      return out;
+    }
+
     FileHandle handle = null;
     try {
-      HdfsFileStatus exFileStatus = dfsClient.getFileInfo(path);
-      
-      handle = new FileHandle(exFileStatus.getFileId());
+      HdfsFileStatus exFileStatus = dfsClient.getFileInfo(exportURI.getPath());
+
+      handle = new FileHandle(exFileStatus.getFileId(),
+                              Nfs3Utils.getNamenodeId(config, exportURI));
     } catch (IOException e) {
       LOG.error("Can't get handle for export:" + path, e);
       MountResponse.writeMNTResponse(Nfs3Status.NFS3ERR_NOENT, out, xid, null);
@@ -143,7 +167,8 @@ public class RpcProgramMountd extends RpcProgram implements MountInterface {
     }
 
     assert (handle != null);
-    LOG.info("Giving handle (fileId:" + handle.getFileId()
+    LOG.info("Giving handle (fileHandle:" + handle.dumpFileHandle()
+        + " file URI: " + exportURI
         + ") to client for export " + path);
     mounts.add(new MountEntry(host, path));
 
@@ -195,7 +220,8 @@ public class RpcProgramMountd extends RpcProgram implements MountInterface {
     info.data().readBytes(data);
     XDR xdr = new XDR(data);
     XDR out = new XDR();
-    InetAddress client = ((InetSocketAddress) info.remoteAddress()).getAddress();
+    InetAddress client =
+        ((InetSocketAddress) info.remoteAddress()).getAddress();
 
     if (mntproc == MNTPROC.NULL) {
       out = nullOp(out, xid, client);
@@ -214,16 +240,20 @@ public class RpcProgramMountd extends RpcProgram implements MountInterface {
     } else if (mntproc == MNTPROC.UMNTALL) {
       umntall(out, xid, client);
     } else if (mntproc == MNTPROC.EXPORT) {
-      // Currently only support one NFS export
+      // Currently only support one NFS export per namenode
       List<NfsExports> hostsMatchers = new ArrayList<NfsExports>();
       if (hostsMatcher != null) {
-        hostsMatchers.add(hostsMatcher);
-        out = MountResponse.writeExportList(out, xid, exports, hostsMatchers);
+        List exportsList = getExports();
+        for (int i = 0; i < exportsList.size(); i++) {
+          hostsMatchers.add(hostsMatcher);
+        }
+        out = MountResponse.writeExportList(out, xid,
+                                            exportsList, hostsMatchers);
       } else {
         // This means there are no valid exports provided.
         RpcAcceptedReply.getInstance(xid,
-          RpcAcceptedReply.AcceptState.PROC_UNAVAIL, new VerifierNone()).write(
-          out);
+            RpcAcceptedReply.AcceptState.PROC_UNAVAIL, new VerifierNone())
+            .write(out);
       }
     } else {
       // Invalid procedure
@@ -231,7 +261,8 @@ public class RpcProgramMountd extends RpcProgram implements MountInterface {
           RpcAcceptedReply.AcceptState.PROC_UNAVAIL, new VerifierNone()).write(
           out);
     }
-    ChannelBuffer buf = ChannelBuffers.wrappedBuffer(out.asReadOnlyWrap().buffer());
+    ChannelBuffer buf =
+        ChannelBuffers.wrappedBuffer(out.asReadOnlyWrap().buffer());
     RpcResponse rsp = new RpcResponse(buf, info.remoteAddress());
     RpcUtil.sendRpcResponse(ctx, rsp);
   }
@@ -244,6 +275,6 @@ public class RpcProgramMountd extends RpcProgram implements MountInterface {
 
   @VisibleForTesting
   public List<String> getExports() {
-    return this.exports;
+    return new ArrayList<>(this.exports.keySet());
   }
 }

+ 128 - 46
hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/DFSClientCache.java

@@ -20,8 +20,11 @@ package org.apache.hadoop.hdfs.nfs.nfs3;
 import org.apache.commons.logging.LogFactory;
 
 import java.io.IOException;
+import java.net.URI;
+import java.nio.file.FileSystemException;
 import java.security.PrivilegedExceptionAction;
 import java.util.ArrayList;
+import java.util.HashMap;
 import java.util.List;
 import java.util.Map.Entry;
 import java.util.concurrent.ConcurrentMap;
@@ -31,9 +34,10 @@ import java.util.concurrent.TimeUnit;
 import com.google.common.base.Preconditions;
 import org.apache.commons.logging.Log;
 import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.hdfs.DFSClient;
 import org.apache.hadoop.hdfs.DFSInputStream;
-import org.apache.hadoop.hdfs.DFSUtilClient;
+import org.apache.hadoop.hdfs.nfs.conf.NfsConfigKeys;
 import org.apache.hadoop.hdfs.nfs.conf.NfsConfiguration;
 import org.apache.hadoop.io.MultipleIOException;
 import org.apache.hadoop.security.UserGroupInformation;
@@ -48,63 +52,97 @@ import com.google.common.cache.RemovalListener;
 import com.google.common.cache.RemovalNotification;
 
 /**
- * A cache saves DFSClient objects for different users
+ * A cache saves DFSClient objects for different users.
  */
 class DFSClientCache {
   private static final Log LOG = LogFactory.getLog(DFSClientCache.class);
   /**
    * Cache that maps User id to the corresponding DFSClient.
    */
-  @VisibleForTesting
-  final LoadingCache<String, DFSClient> clientCache;
+  private final LoadingCache<DfsClientKey, DFSClient> clientCache;
 
   final static int DEFAULT_DFS_CLIENT_CACHE_SIZE = 256;
 
   /**
-   * Cache that maps <DFSClient, inode path> to the corresponding
+   * Cache that maps <DFSClient, inode path, nnid> to the corresponding
    * FSDataInputStream.
    */
-  final LoadingCache<DFSInputStreamCaheKey, FSDataInputStream> inputstreamCache;
+  private final LoadingCache<DFSInputStreamCacheKey,
+                          FSDataInputStream> inputstreamCache;
 
   /**
-   * Time to live for a DFSClient (in seconds)
+   * Time to live for a DFSClient (in seconds).
    */
   final static int DEFAULT_DFS_INPUTSTREAM_CACHE_SIZE = 1024;
   final static int DEFAULT_DFS_INPUTSTREAM_CACHE_TTL = 10 * 60;
 
   private final NfsConfiguration config;
+  private final HashMap<Integer, URI> namenodeUriMap;
 
-  private static class DFSInputStreamCaheKey {
-    final String userId;
-    final String inodePath;
+  private static final class DFSInputStreamCacheKey {
+    private final String userId;
+    private final String inodePath;
+    private final int namenodeId;
 
-    private DFSInputStreamCaheKey(String userId, String inodePath) {
+    private DFSInputStreamCacheKey(String userId, String inodePath,
+                                   int namenodeId) {
       super();
       this.userId = userId;
       this.inodePath = inodePath;
+      this.namenodeId = namenodeId;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+      if (obj instanceof DFSInputStreamCacheKey) {
+        DFSInputStreamCacheKey k = (DFSInputStreamCacheKey) obj;
+        return userId.equals(k.userId) &&
+               inodePath.equals(k.inodePath) &&
+               (namenodeId == k.namenodeId);
+      }
+      return false;
+    }
+
+    @Override
+    public int hashCode() {
+      return Objects.hashCode(userId, inodePath, namenodeId);
+    }
+  }
+
+  private static final class DfsClientKey {
+    private final String userName;
+    private final int namenodeId;
+
+    private DfsClientKey(String userName, int namenodeId) {
+      this.userName = userName;
+      this.namenodeId = namenodeId;
     }
 
     @Override
     public boolean equals(Object obj) {
-      if (obj instanceof DFSInputStreamCaheKey) {
-        DFSInputStreamCaheKey k = (DFSInputStreamCaheKey) obj;
-        return userId.equals(k.userId) && inodePath.equals(k.inodePath);
+      if (obj instanceof DfsClientKey) {
+        DfsClientKey k = (DfsClientKey) obj;
+        return userName.equals(k.userName) &&
+            (namenodeId == k.namenodeId);
       }
       return false;
     }
 
     @Override
     public int hashCode() {
-      return Objects.hashCode(userId, inodePath);
+      return Objects.hashCode(userName, namenodeId);
     }
   }
 
-  DFSClientCache(NfsConfiguration config) {
+  DFSClientCache(NfsConfiguration config) throws IOException {
     this(config, DEFAULT_DFS_CLIENT_CACHE_SIZE);
   }
-  
-  DFSClientCache(NfsConfiguration config, int clientCache) {
+
+  DFSClientCache(NfsConfiguration config, int clientCache) throws IOException {
     this.config = config;
+    namenodeUriMap = new HashMap<>();
+    prepareAddressMap();
+
     this.clientCache = CacheBuilder.newBuilder()
         .maximumSize(clientCache)
         .removalListener(clientRemovalListener())
@@ -115,11 +153,36 @@ class DFSClientCache {
         .expireAfterAccess(DEFAULT_DFS_INPUTSTREAM_CACHE_TTL, TimeUnit.SECONDS)
         .removalListener(inputStreamRemovalListener())
         .build(inputStreamLoader());
-    
+
     ShutdownHookManager.get().addShutdownHook(new CacheFinalizer(),
         SHUTDOWN_HOOK_PRIORITY);
   }
 
+  private void prepareAddressMap() throws IOException {
+    FileSystem fs = FileSystem.get(config);
+    String[] exportsPath =
+        config.getStrings(NfsConfigKeys.DFS_NFS_EXPORT_POINT_KEY,
+            NfsConfigKeys.DFS_NFS_EXPORT_POINT_DEFAULT);
+    for (String exportPath : exportsPath) {
+      URI exportURI = Nfs3Utils.getResolvedURI(fs, exportPath);
+      int namenodeId = Nfs3Utils.getNamenodeId(config, exportURI);
+      URI value = namenodeUriMap.get(namenodeId);
+      // if a unique nnid, add it to the map
+      if (value == null) {
+        LOG.info("Added export:" + exportPath + " FileSystem URI:" + exportURI
+              + " with namenodeId:" + namenodeId);
+        namenodeUriMap.put(namenodeId, exportURI);
+      } else {
+        // if the nnid already exists, it better be the for the same namenode
+        String msg = String.format("FS:%s, Namenode ID collision for path:%s "
+                + "nnid:%s uri being added:%s existing uri:%s", fs.getScheme(),
+            exportPath, namenodeId, exportURI, value);
+        LOG.error(msg);
+        throw new FileSystemException(msg);
+      }
+    }
+  }
+
   /**
    * Priority of the FileSystem shutdown hook.
    */
@@ -135,7 +198,12 @@ class DFSClientCache {
       }
     }
   }
-  
+
+  @VisibleForTesting
+  public LoadingCache<DfsClientKey, DFSClient> getClientCache() {
+    return clientCache;
+  }
+
   /**
    * Close all DFSClient instances in the Cache.
    * @param onlyAutomatic only close those that are marked for automatic closing
@@ -143,9 +211,9 @@ class DFSClientCache {
   synchronized void closeAll(boolean onlyAutomatic) throws IOException {
     List<IOException> exceptions = new ArrayList<IOException>();
 
-    ConcurrentMap<String, DFSClient> map = clientCache.asMap();
+    ConcurrentMap<DfsClientKey, DFSClient> map = clientCache.asMap();
 
-    for (Entry<String, DFSClient> item : map.entrySet()) {
+    for (Entry<DfsClientKey, DFSClient> item : map.entrySet()) {
       final DFSClient client = item.getValue();
       if (client != null) {
         try {
@@ -160,20 +228,24 @@ class DFSClientCache {
       throw MultipleIOException.createIOException(exceptions);
     }
   }
-  
-  private CacheLoader<String, DFSClient> clientLoader() {
-    return new CacheLoader<String, DFSClient>() {
+
+  private CacheLoader<DfsClientKey, DFSClient> clientLoader() {
+    return new CacheLoader<DfsClientKey, DFSClient>() {
       @Override
-      public DFSClient load(String userName) throws Exception {
+      public DFSClient load(final DfsClientKey key) throws Exception {
         UserGroupInformation ugi = getUserGroupInformation(
-                userName,
-                UserGroupInformation.getCurrentUser());
+            key.userName, UserGroupInformation.getCurrentUser());
 
         // Guava requires CacheLoader never returns null.
         return ugi.doAs(new PrivilegedExceptionAction<DFSClient>() {
           @Override
           public DFSClient run() throws IOException {
-            return new DFSClient(DFSUtilClient.getNNAddress(config), config);
+            URI namenodeURI = namenodeUriMap.get(key.namenodeId);
+            if (namenodeURI == null) {
+              throw new IOException("No namenode URI found for user:" +
+                  key.userName + " namenodeId:" + key.namenodeId);
+            }
+            return new DFSClient(namenodeURI, config);
           }
         });
       }
@@ -181,7 +253,7 @@ class DFSClientCache {
   }
 
   /**
-   * This method uses the currentUser, and real user to create a proxy
+   * This method uses the currentUser, and real user to create a proxy.
    * @param effectiveUser The user who is being proxied by the real user
    * @param realUser The actual user who does the command
    * @return Proxy UserGroupInformation
@@ -204,10 +276,11 @@ class DFSClientCache {
     return ugi;
   }
 
-  private RemovalListener<String, DFSClient> clientRemovalListener() {
-    return new RemovalListener<String, DFSClient>() {
+  private RemovalListener<DfsClientKey, DFSClient> clientRemovalListener() {
+    return new RemovalListener<DfsClientKey, DFSClient>() {
       @Override
-      public void onRemoval(RemovalNotification<String, DFSClient> notification) {
+      public void onRemoval(
+          RemovalNotification<DfsClientKey, DFSClient> notification) {
         DFSClient client = notification.getValue();
         try {
           client.close();
@@ -220,12 +293,15 @@ class DFSClientCache {
     };
   }
 
-  private RemovalListener<DFSInputStreamCaheKey, FSDataInputStream> inputStreamRemovalListener() {
-    return new RemovalListener<DFSClientCache.DFSInputStreamCaheKey, FSDataInputStream>() {
+  private RemovalListener
+      <DFSInputStreamCacheKey, FSDataInputStream> inputStreamRemovalListener() {
+    return new RemovalListener
+        <DFSClientCache.DFSInputStreamCacheKey, FSDataInputStream>() {
 
       @Override
       public void onRemoval(
-          RemovalNotification<DFSInputStreamCaheKey, FSDataInputStream> notification) {
+          RemovalNotification<DFSInputStreamCacheKey, FSDataInputStream>
+                                                            notification) {
         try {
           notification.getValue().close();
         } catch (IOException ignored) {
@@ -234,22 +310,24 @@ class DFSClientCache {
     };
   }
 
-  private CacheLoader<DFSInputStreamCaheKey, FSDataInputStream> inputStreamLoader() {
-    return new CacheLoader<DFSInputStreamCaheKey, FSDataInputStream>() {
+  private CacheLoader<DFSInputStreamCacheKey, FSDataInputStream>
+                                                      inputStreamLoader() {
+    return new CacheLoader<DFSInputStreamCacheKey, FSDataInputStream>() {
 
       @Override
-      public FSDataInputStream load(DFSInputStreamCaheKey key) throws Exception {
-        DFSClient client = getDfsClient(key.userId);
+      public FSDataInputStream
+                    load(DFSInputStreamCacheKey key) throws Exception {
+        DFSClient client = getDfsClient(key.userId, key.namenodeId);
         DFSInputStream dis = client.open(key.inodePath);
         return client.createWrappedInputStream(dis);
       }
     };
   }
 
-  DFSClient getDfsClient(String userName) {
+  DFSClient getDfsClient(String userName, int namenodeId) {
     DFSClient client = null;
     try {
-      client = clientCache.get(userName);
+      client = clientCache.get(new DfsClientKey(userName, namenodeId));
     } catch (ExecutionException e) {
       LOG.error("Failed to create DFSClient for user:" + userName + " Cause:"
           + e);
@@ -257,8 +335,10 @@ class DFSClientCache {
     return client;
   }
 
-  FSDataInputStream getDfsInputStream(String userName, String inodePath) {
-    DFSInputStreamCaheKey k = new DFSInputStreamCaheKey(userName, inodePath);
+  FSDataInputStream getDfsInputStream(String userName, String inodePath,
+                                      int namenodeId) {
+    DFSInputStreamCacheKey k =
+        new DFSInputStreamCacheKey(userName, inodePath, namenodeId);
     FSDataInputStream s = null;
     try {
       s = inputstreamCache.get(k);
@@ -269,8 +349,10 @@ class DFSClientCache {
     return s;
   }
 
-  public void invalidateDfsInputStream(String userName, String inodePath) {
-    DFSInputStreamCaheKey k = new DFSInputStreamCaheKey(userName, inodePath);
+  public void invalidateDfsInputStream(String userName, String inodePath,
+                                       int namenodeId) {
+    DFSInputStreamCacheKey k =
+        new DFSInputStreamCacheKey(userName, inodePath, namenodeId);
     inputstreamCache.invalidate(k);
   }
 }

+ 46 - 0
hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/Nfs3Utils.java

@@ -18,8 +18,17 @@
 package org.apache.hadoop.hdfs.nfs.nfs3;
 
 import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.net.URI;
+import java.nio.file.FileSystemException;
 
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FsConstants;
+import org.apache.hadoop.fs.viewfs.ViewFileSystem;
 import org.apache.hadoop.hdfs.DFSClient;
+import org.apache.hadoop.hdfs.DFSUtilClient;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.nfs.NfsFileType;
 import org.apache.hadoop.nfs.NfsTime;
@@ -224,4 +233,41 @@ public class Nfs3Utils {
   public static long getElapsedTime(long startTimeNano) {
     return System.nanoTime() - startTimeNano;
   }
+
+  public static int getNamenodeId(Configuration conf) {
+    URI filesystemURI = FileSystem.getDefaultUri(conf);
+    return getNamenodeId(conf, filesystemURI);
+  }
+
+  public static int getNamenodeId(Configuration conf, URI namenodeURI) {
+    InetSocketAddress address =
+        DFSUtilClient.getNNAddressCheckLogical(conf, namenodeURI);
+    return address.hashCode();
+  }
+
+  public static URI getResolvedURI(FileSystem fs, String exportPath)
+      throws IOException {
+    URI fsURI = fs.getUri();
+    String scheme = fs.getScheme();
+    if (scheme.equalsIgnoreCase(FsConstants.VIEWFS_SCHEME)) {
+      ViewFileSystem viewFs = (ViewFileSystem)fs;
+      ViewFileSystem.MountPoint[] mountPoints = viewFs.getMountPoints();
+      for (ViewFileSystem.MountPoint mount : mountPoints) {
+        String mountedPath = mount.getMountedOnPath().toString();
+        if (exportPath.startsWith(mountedPath)) {
+          String subpath = exportPath.substring(mountedPath.length());
+          fsURI = mount.getTargetFileSystemURIs()[0].resolve(subpath);
+          break;
+        }
+      }
+    } else if (scheme.equalsIgnoreCase(HdfsConstants.HDFS_URI_SCHEME)) {
+      fsURI = fsURI.resolve(exportPath);
+    }
+
+    if (!fsURI.getScheme().equalsIgnoreCase(HdfsConstants.HDFS_URI_SCHEME)) {
+      throw new FileSystemException("Only HDFS is supported as underlying"
+          + "FileSystem, fs scheme:" + scheme + " uri to be added" + fsURI);
+    }
+    return fsURI;
+  }
 }

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

@@ -442,7 +442,7 @@ class OpenFileCtx {
     
     if (!activeState) {
       LOG.info("OpenFileCtx is inactive, fileId: "
-          + request.getHandle().getFileId());
+          + request.getHandle().dumpFileHandle());
       WccData fileWcc = new WccData(latestAttr.getWccAttr(), latestAttr);
       WRITE3Response response = new WRITE3Response(Nfs3Status.NFS3ERR_IO,
           fileWcc, 0, request.getStableHow(), Nfs3Constant.WRITE_COMMIT_VERF);
@@ -981,7 +981,8 @@ class OpenFileCtx {
    * Check stream status to decide if it should be closed
    * @return true, remove stream; false, keep stream
    */
-  public synchronized boolean streamCleanup(long fileId, long streamTimeout) {
+  public synchronized boolean streamCleanup(FileHandle handle,
+                                            long streamTimeout) {
     Preconditions
         .checkState(streamTimeout >= NfsConfigKeys.DFS_NFS_STREAM_TIMEOUT_MIN_DEFAULT);
     if (!activeState) {
@@ -992,7 +993,8 @@ class OpenFileCtx {
     // Check the stream timeout
     if (checkStreamTimeout(streamTimeout)) {
       if (LOG.isDebugEnabled()) {
-        LOG.debug("stream can be closed for fileId: " + fileId);
+        LOG.debug("stream can be closed for fileId: "
+            + handle.dumpFileHandle());
       }
       flag = true;
     }
@@ -1188,7 +1190,7 @@ class OpenFileCtx {
     
     FileHandle handle = writeCtx.getHandle();
     if (LOG.isDebugEnabled()) {
-      LOG.debug("do write, fileId: " + handle.getFileId() + " offset: "
+      LOG.debug("do write, fileHandle " + handle.dumpFileHandle() + " offset: "
           + offset + " length: " + count + " stableHow: " + stableHow.name());
     }
 
@@ -1213,8 +1215,9 @@ class OpenFileCtx {
             writeCtx.setDataState(WriteCtx.DataState.NO_DUMP);
             updateNonSequentialWriteInMemory(-count);
             if (LOG.isDebugEnabled()) {
-              LOG.debug("After writing " + handle.getFileId() + " at offset "
-                  + offset + ", updated the memory count, new value: "
+              LOG.debug("After writing " + handle.dumpFileHandle()
+                  + " at offset " + offset
+                  + ", updated the memory count, new value: "
                   + nonSequentialWriteInMemory.get());
             }
           }
@@ -1257,8 +1260,8 @@ class OpenFileCtx {
       processCommits(writeCtx.getOffset() + writeCtx.getCount());
      
     } catch (IOException e) {
-      LOG.error("Error writing to fileId " + handle.getFileId() + " at offset "
-          + offset + " and length " + count, e);
+      LOG.error("Error writing to fileHandle " + handle.dumpFileHandle()
+          + " at offset " + offset + " and length " + count, e);
       if (!writeCtx.getReplied()) {
         WRITE3Response response = new WRITE3Response(Nfs3Status.NFS3ERR_IO);
         Nfs3Utils.writeChannel(channel, response.serialize(

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

@@ -156,7 +156,7 @@ class OpenFileCtxCache {
       Entry<FileHandle, OpenFileCtx> pairs = it.next();
       FileHandle handle = pairs.getKey();
       OpenFileCtx ctx = pairs.getValue();
-      if (!ctx.streamCleanup(handle.getFileId(), streamTimeout)) {
+      if (!ctx.streamCleanup(handle, streamTimeout)) {
         continue;
       }
 
@@ -164,10 +164,10 @@ class OpenFileCtxCache {
       synchronized (this) {
         OpenFileCtx ctx2 = openFileMap.get(handle);
         if (ctx2 != null) {
-          if (ctx2.streamCleanup(handle.getFileId(), streamTimeout)) {
+          if (ctx2.streamCleanup(handle, streamTimeout)) {
             openFileMap.remove(handle);
             if (LOG.isDebugEnabled()) {
-              LOG.debug("After remove stream " + handle.getFileId()
+              LOG.debug("After remove stream " + handle.dumpFileHandle()
                   + ", the stream number:" + size());
             }
             ctxToRemove.add(ctx2);

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

@@ -319,12 +319,6 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
       return response;
     }
 
-    DFSClient dfsClient = clientCache.getDfsClient(securityHandler.getUser());
-    if (dfsClient == null) {
-      response.setStatus(Nfs3Status.NFS3ERR_SERVERFAULT);
-      return response;
-    }
-
     GETATTR3Request request;
     try {
       request = GETATTR3Request.deserialize(xdr);
@@ -335,9 +329,17 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
     }
 
     FileHandle handle = request.getHandle();
+    int namenodeId = handle.getNamenodeId();
     if (LOG.isDebugEnabled()) {
-      LOG.debug("GETATTR for fileId: " + handle.getFileId() + " client: "
-          + remoteAddress);
+      LOG.debug("GETATTR for fileHandle: " + handle.dumpFileHandle()
+          + " client: " + remoteAddress);
+    }
+
+    DFSClient dfsClient =
+        clientCache.getDfsClient(securityHandler.getUser(), namenodeId);
+    if (dfsClient == null) {
+      response.setStatus(Nfs3Status.NFS3ERR_SERVERFAULT);
+      return response;
     }
 
     Nfs3FileAttributes attrs = null;
@@ -412,11 +414,6 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
   SETATTR3Response setattr(XDR xdr, SecurityHandler securityHandler,
       SocketAddress remoteAddress) {
     SETATTR3Response response = new SETATTR3Response(Nfs3Status.NFS3_OK);
-    DFSClient dfsClient = clientCache.getDfsClient(securityHandler.getUser());
-    if (dfsClient == null) {
-      response.setStatus(Nfs3Status.NFS3ERR_SERVERFAULT);
-      return response;
-    }
 
     SETATTR3Request request;
     try {
@@ -428,9 +425,17 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
     }
 
     FileHandle handle = request.getHandle();
+    int namenodeId = handle.getNamenodeId();
     if (LOG.isDebugEnabled()) {
-      LOG.debug("NFS SETATTR fileId: " + handle.getFileId() + " client: "
-          + remoteAddress);
+      LOG.debug("NFS SETATTR fileHandle: " + handle.dumpFileHandle()
+          + " client: " + remoteAddress);
+    }
+
+    DFSClient dfsClient =
+        clientCache.getDfsClient(securityHandler.getUser(), namenodeId);
+    if (dfsClient == null) {
+      response.setStatus(Nfs3Status.NFS3ERR_SERVERFAULT);
+      return response;
     }
 
     if (request.getAttr().getUpdateFields().contains(SetAttrField.SIZE)) {
@@ -498,12 +503,6 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
       return response;
     }
 
-    DFSClient dfsClient = clientCache.getDfsClient(securityHandler.getUser());
-    if (dfsClient == null) {
-      response.setStatus(Nfs3Status.NFS3ERR_SERVERFAULT);
-      return response;
-    }
-
     LOOKUP3Request request;
     try {
       request = LOOKUP3Request.deserialize(xdr);
@@ -514,15 +513,22 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
 
     FileHandle dirHandle = request.getHandle();
     String fileName = request.getName();
+    int namenodeId = dirHandle.getNamenodeId();
     if (LOG.isDebugEnabled()) {
-      LOG.debug("NFS LOOKUP dir fileId: " + dirHandle.getFileId() + " name: "
-          + fileName + " client: " + remoteAddress);
+      LOG.debug("NFS LOOKUP dir fileHandle: " + dirHandle.dumpFileHandle()
+          + " name: " + fileName + " client: " + remoteAddress);
     }
 
+    DFSClient dfsClient =
+        clientCache.getDfsClient(securityHandler.getUser(), namenodeId);
+    if (dfsClient == null) {
+      response.setStatus(Nfs3Status.NFS3ERR_SERVERFAULT);
+      return response;
+    }
     try {
       String dirFileIdPath = Nfs3Utils.getFileIdPath(dirHandle);
       Nfs3FileAttributes postOpObjAttr = writeManager.getFileAttr(dfsClient,
-          dirHandle, fileName);
+          dirHandle, fileName, namenodeId);
       if (postOpObjAttr == null) {
         if (LOG.isDebugEnabled()) {
           LOG.debug("NFS LOOKUP fileId: " + dirHandle.getFileId() + " name: "
@@ -540,7 +546,8 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
         LOG.info("Can't get path for dir fileId: " + dirHandle.getFileId());
         return new LOOKUP3Response(Nfs3Status.NFS3ERR_STALE);
       }
-      FileHandle fileHandle = new FileHandle(postOpObjAttr.getFileId());
+      FileHandle fileHandle =
+          new FileHandle(postOpObjAttr.getFileId(), namenodeId);
       return new LOOKUP3Response(Nfs3Status.NFS3_OK, fileHandle, postOpObjAttr,
           postOpDirAttr);
 
@@ -566,12 +573,6 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
       return response;
     }
 
-    DFSClient dfsClient = clientCache.getDfsClient(securityHandler.getUser());
-    if (dfsClient == null) {
-      response.setStatus(Nfs3Status.NFS3ERR_SERVERFAULT);
-      return response;
-    }
-
     ACCESS3Request request;
     try {
       request = ACCESS3Request.deserialize(xdr);
@@ -581,13 +582,21 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
     }
 
     FileHandle handle = request.getHandle();
-    Nfs3FileAttributes attrs;
+    int namenodeId = handle.getNamenodeId();
+
+    DFSClient dfsClient =
+        clientCache.getDfsClient(securityHandler.getUser(), namenodeId);
+    if (dfsClient == null) {
+      response.setStatus(Nfs3Status.NFS3ERR_SERVERFAULT);
+      return response;
+    }
 
     if (LOG.isDebugEnabled()) {
-      LOG.debug("NFS ACCESS fileId: " + handle.getFileId() + " client: "
-          + remoteAddress);
+      LOG.debug("NFS ACCESS fileHandle: " + handle.dumpFileHandle()
+          + " client: " + remoteAddress);
     }
 
+    Nfs3FileAttributes attrs;
     try {
       attrs = writeManager.getFileAttr(dfsClient, handle, iug);
 
@@ -639,12 +648,6 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
       return response;
     }
 
-    DFSClient dfsClient = clientCache.getDfsClient(securityHandler.getUser());
-    if (dfsClient == null) {
-      response.setStatus(Nfs3Status.NFS3ERR_SERVERFAULT);
-      return response;
-    }
-
     READLINK3Request request;
 
     try {
@@ -655,9 +658,17 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
     }
 
     FileHandle handle = request.getHandle();
+    int namenodeId = handle.getNamenodeId();
     if (LOG.isDebugEnabled()) {
-      LOG.debug("NFS READLINK fileId: " + handle.getFileId() + " client: "
-          + remoteAddress);
+      LOG.debug("NFS READLINK fileHandle: " + handle.dumpFileHandle()
+          + " client: " + remoteAddress);
+    }
+
+    DFSClient dfsClient =
+        clientCache.getDfsClient(securityHandler.getUser(), namenodeId);
+    if (dfsClient == null) {
+      response.setStatus(Nfs3Status.NFS3ERR_SERVERFAULT);
+      return response;
     }
 
     String fileIdPath = Nfs3Utils.getFileIdPath(handle);
@@ -715,12 +726,6 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
       return response;
     }
 
-    DFSClient dfsClient = clientCache.getDfsClient(userName);
-    if (dfsClient == null) {
-      response.setStatus(Nfs3Status.NFS3ERR_SERVERFAULT);
-      return response;
-    }
-
     READ3Request request;
 
     try {
@@ -734,9 +739,16 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
     int count = request.getCount();
 
     FileHandle handle = request.getHandle();
+    int namenodeId = handle.getNamenodeId();
     if (LOG.isDebugEnabled()) {
-      LOG.debug("NFS READ fileId: " + handle.getFileId() + " offset: " + offset
-          + " count: " + count + " client: " + remoteAddress);
+      LOG.debug("NFS READ fileHandle: " + handle.dumpFileHandle()+ " offset: "
+          + offset + " count: " + count + " client: " + remoteAddress);
+    }
+
+    DFSClient dfsClient = clientCache.getDfsClient(userName, namenodeId);
+    if (dfsClient == null) {
+      response.setStatus(Nfs3Status.NFS3ERR_SERVERFAULT);
+      return response;
     }
 
     Nfs3FileAttributes attrs;
@@ -791,7 +803,7 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
        */
       for (int i = 0; i < 1; ++i) {
         FSDataInputStream fis = clientCache.getDfsInputStream(userName,
-            Nfs3Utils.getFileIdPath(handle));
+            Nfs3Utils.getFileIdPath(handle), namenodeId);
 
         if (fis == null) {
             return new READ3Response(Nfs3Status.NFS3ERR_ACCES);
@@ -805,7 +817,7 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
           // which requires incompatible changes.
           if (e.getMessage().equals("Stream closed")) {
             clientCache.invalidateDfsInputStream(userName,
-                Nfs3Utils.getFileIdPath(handle));
+                Nfs3Utils.getFileIdPath(handle), namenodeId);
             continue;
           } else {
             throw e;
@@ -850,11 +862,6 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
       SecurityHandler securityHandler, SocketAddress remoteAddress) {
     WRITE3Response response = new WRITE3Response(Nfs3Status.NFS3_OK);
 
-    DFSClient dfsClient = clientCache.getDfsClient(securityHandler.getUser());
-    if (dfsClient == null) {
-      response.setStatus(Nfs3Status.NFS3ERR_SERVERFAULT);
-      return response;
-    }
 
     WRITE3Request request;
 
@@ -875,12 +882,20 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
     }
 
     FileHandle handle = request.getHandle();
+    int namenodeId = handle.getNamenodeId();
     if (LOG.isDebugEnabled()) {
-      LOG.debug("NFS WRITE fileId: " + handle.getFileId() + " offset: "
+      LOG.debug("NFS WRITE fileHandle: " + handle.dumpFileHandle() + " offset: "
           + offset + " length: " + count + " stableHow: " + stableHow.getValue()
           + " xid: " + xid + " client: " + remoteAddress);
     }
 
+    DFSClient dfsClient =
+        clientCache.getDfsClient(securityHandler.getUser(), namenodeId);
+    if (dfsClient == null) {
+      response.setStatus(Nfs3Status.NFS3ERR_SERVERFAULT);
+      return response;
+    }
+
     Nfs3FileAttributes preOpAttr = null;
     try {
       preOpAttr = writeManager.getFileAttr(dfsClient, handle, iug);
@@ -932,11 +947,6 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
   CREATE3Response create(XDR xdr, SecurityHandler securityHandler,
       SocketAddress remoteAddress) {
     CREATE3Response response = new CREATE3Response(Nfs3Status.NFS3_OK);
-    DFSClient dfsClient = clientCache.getDfsClient(securityHandler.getUser());
-    if (dfsClient == null) {
-      response.setStatus(Nfs3Status.NFS3ERR_SERVERFAULT);
-      return response;
-    }
 
     CREATE3Request request;
 
@@ -949,11 +959,19 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
 
     FileHandle dirHandle = request.getHandle();
     String fileName = request.getName();
+    int namenodeId = dirHandle.getNamenodeId();
     if (LOG.isDebugEnabled()) {
-      LOG.debug("NFS CREATE dir fileId: " + dirHandle.getFileId()
+      LOG.debug("NFS CREATE dir fileHandle: " + dirHandle.dumpFileHandle()
           + " filename: " + fileName + " client: " + remoteAddress);
     }
 
+    DFSClient dfsClient =
+        clientCache.getDfsClient(securityHandler.getUser(), namenodeId);
+    if (dfsClient == null) {
+      response.setStatus(Nfs3Status.NFS3ERR_SERVERFAULT);
+      return response;
+    }
+
     int createMode = request.getMode();
     if ((createMode != Nfs3Constant.CREATE_EXCLUSIVE)
         && request.getObjAttr().getUpdateFields().contains(SetAttrField.SIZE)
@@ -1016,7 +1034,7 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
       OpenFileCtx openFileCtx = new OpenFileCtx(fos, postOpObjAttr,
           writeDumpDir + "/" + postOpObjAttr.getFileId(), dfsClient, iug,
           aixCompatMode, config);
-      fileHandle = new FileHandle(postOpObjAttr.getFileId());
+      fileHandle = new FileHandle(postOpObjAttr.getFileId(), namenodeId);
       if (!writeManager.addOpenFileStream(fileHandle, openFileCtx)) {
         LOG.warn("Can't add more stream, close it."
             + " Future write will become append");
@@ -1066,11 +1084,6 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
   MKDIR3Response mkdir(XDR xdr, SecurityHandler securityHandler,
       SocketAddress remoteAddress) {
     MKDIR3Response response = new MKDIR3Response(Nfs3Status.NFS3_OK);
-    DFSClient dfsClient = clientCache.getDfsClient(securityHandler.getUser());
-    if (dfsClient == null) {
-      response.setStatus(Nfs3Status.NFS3ERR_SERVERFAULT);
-      return response;
-    }
 
     MKDIR3Request request;
 
@@ -1082,9 +1095,18 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
     }
     FileHandle dirHandle = request.getHandle();
     String fileName = request.getName();
+    int namenodeId = dirHandle.getNamenodeId();
+
+    DFSClient dfsClient =
+        clientCache.getDfsClient(securityHandler.getUser(), namenodeId);
+    if (dfsClient == null) {
+      response.setStatus(Nfs3Status.NFS3ERR_SERVERFAULT);
+      return response;
+    }
+
     if (LOG.isDebugEnabled()) {
-      LOG.debug("NFS MKDIR dirId: " + dirHandle.getFileId() + " filename: "
-          + fileName + " client: " + remoteAddress);
+      LOG.debug("NFS MKDIR dirHandle: " + dirHandle.dumpFileHandle()
+          + " filename: " + fileName + " client: " + remoteAddress);
     }
 
     if (request.getObjAttr().getUpdateFields().contains(SetAttrField.SIZE)) {
@@ -1130,11 +1152,11 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
       setattrInternal(dfsClient, fileIdPath, setAttr3, false);
 
       postOpObjAttr = Nfs3Utils.getFileAttr(dfsClient, fileIdPath, iug);
-      objFileHandle = new FileHandle(postOpObjAttr.getFileId());
+      objFileHandle = new FileHandle(postOpObjAttr.getFileId(), namenodeId);
       WccData dirWcc = Nfs3Utils.createWccData(
           Nfs3Utils.getWccAttr(preOpDirAttr), dfsClient, dirFileIdPath, iug);
       return new MKDIR3Response(Nfs3Status.NFS3_OK, new FileHandle(
-          postOpObjAttr.getFileId()), postOpObjAttr, dirWcc);
+          postOpObjAttr.getFileId(), namenodeId), postOpObjAttr, dirWcc);
     } catch (IOException e) {
       LOG.warn("Exception ", e);
       // Try to return correct WccData
@@ -1167,11 +1189,6 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
   REMOVE3Response remove(XDR xdr, SecurityHandler securityHandler,
       SocketAddress remoteAddress) {
     REMOVE3Response response = new REMOVE3Response(Nfs3Status.NFS3_OK);
-    DFSClient dfsClient = clientCache.getDfsClient(securityHandler.getUser());
-    if (dfsClient == null) {
-      response.setStatus(Nfs3Status.NFS3ERR_SERVERFAULT);
-      return response;
-    }
 
     REMOVE3Request request;
     try {
@@ -1181,12 +1198,21 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
       return new REMOVE3Response(Nfs3Status.NFS3ERR_INVAL);
     }
     FileHandle dirHandle = request.getHandle();
+    int namenodeId = dirHandle.getNamenodeId();
+
     String fileName = request.getName();
     if (LOG.isDebugEnabled()) {
-      LOG.debug("NFS REMOVE dir fileId: " + dirHandle.getFileId()
+      LOG.debug("NFS REMOVE dir fileHandle: " + dirHandle.dumpFileHandle()
           + " fileName: " + fileName + " client: " + remoteAddress);
     }
 
+    DFSClient dfsClient =
+        clientCache.getDfsClient(securityHandler.getUser(), namenodeId);
+    if (dfsClient == null) {
+      response.setStatus(Nfs3Status.NFS3ERR_SERVERFAULT);
+      return response;
+    }
+
     String dirFileIdPath = Nfs3Utils.getFileIdPath(dirHandle);
     Nfs3FileAttributes preOpDirAttr = null;
     Nfs3FileAttributes postOpDirAttr = null;
@@ -1247,11 +1273,6 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
   RMDIR3Response rmdir(XDR xdr, SecurityHandler securityHandler,
       SocketAddress remoteAddress) {
     RMDIR3Response response = new RMDIR3Response(Nfs3Status.NFS3_OK);
-    DFSClient dfsClient = clientCache.getDfsClient(securityHandler.getUser());
-    if (dfsClient == null) {
-      response.setStatus(Nfs3Status.NFS3ERR_SERVERFAULT);
-      return response;
-    }
 
     RMDIR3Request request;
     try {
@@ -1262,12 +1283,19 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
     }
     FileHandle dirHandle = request.getHandle();
     String fileName = request.getName();
-
+    int namenodeId = dirHandle.getNamenodeId();
     if (LOG.isDebugEnabled()) {
-      LOG.debug("NFS RMDIR dir fileId: " + dirHandle.getFileId()
+      LOG.debug("NFS RMDIR dir fileHandle: " + dirHandle.dumpFileHandle()
           + " fileName: " + fileName + " client: " + remoteAddress);
     }
 
+    DFSClient dfsClient =
+        clientCache.getDfsClient(securityHandler.getUser(), namenodeId);
+    if (dfsClient == null) {
+      response.setStatus(Nfs3Status.NFS3ERR_SERVERFAULT);
+      return response;
+    }
+
     String dirFileIdPath = Nfs3Utils.getFileIdPath(dirHandle);
     Nfs3FileAttributes preOpDirAttr = null;
     Nfs3FileAttributes postOpDirAttr = null;
@@ -1332,11 +1360,6 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
   RENAME3Response rename(XDR xdr, SecurityHandler securityHandler,
       SocketAddress remoteAddress) {
     RENAME3Response response = new RENAME3Response(Nfs3Status.NFS3_OK);
-    DFSClient dfsClient = clientCache.getDfsClient(securityHandler.getUser());
-    if (dfsClient == null) {
-      response.setStatus(Nfs3Status.NFS3ERR_SERVERFAULT);
-      return response;
-    }
 
     RENAME3Request request = null;
     try {
@@ -1347,13 +1370,28 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
     }
 
     FileHandle fromHandle = request.getFromDirHandle();
+    int fromNamenodeId = fromHandle.getNamenodeId();
     String fromName = request.getFromName();
     FileHandle toHandle = request.getToDirHandle();
+    int toNamenodeId = toHandle.getNamenodeId();
     String toName = request.getToName();
     if (LOG.isDebugEnabled()) {
-      LOG.debug("NFS RENAME from: " + fromHandle.getFileId() + "/" + fromName
-          + " to: " + toHandle.getFileId() + "/" + toName + " client: "
-          + remoteAddress);
+      LOG.debug("NFS RENAME from: " + fromHandle.dumpFileHandle()
+          + "/" + fromName + " to: " + toHandle.dumpFileHandle()
+          + "/" + toName + " client: " + remoteAddress);
+    }
+
+    DFSClient dfsClient =
+        clientCache.getDfsClient(securityHandler.getUser(), fromNamenodeId);
+    if (dfsClient == null) {
+      response.setStatus(Nfs3Status.NFS3ERR_SERVERFAULT);
+      return response;
+    }
+
+    if (fromNamenodeId != toNamenodeId) {
+      // renaming file from one namenode to another is not supported
+      response.setStatus(Nfs3Status.NFS3ERR_INVAL);
+      return response;
     }
 
     String fromDirFileIdPath = Nfs3Utils.getFileIdPath(fromHandle);
@@ -1429,12 +1467,6 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
       return response;
     }
 
-    DFSClient dfsClient = clientCache.getDfsClient(securityHandler.getUser());
-    if (dfsClient == null) {
-      response.setStatus(Nfs3Status.NFS3ERR_SERVERFAULT);
-      return response;
-    }
-
     SYMLINK3Request request;
     try {
       request = SYMLINK3Request.deserialize(xdr);
@@ -1448,11 +1480,20 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
     String name = request.getName();
     String symData = request.getSymData();
     String linkDirIdPath = Nfs3Utils.getFileIdPath(dirHandle);
+    int namenodeId = dirHandle.getNamenodeId();
+
     // Don't do any name check to source path, just leave it to HDFS
     String linkIdPath = linkDirIdPath + "/" + name;
     if (LOG.isDebugEnabled()) {
       LOG.debug("NFS SYMLINK, target: " + symData + " link: " + linkIdPath
-          + " client: " + remoteAddress);
+          + " namenodeId: " + namenodeId + " client: " + remoteAddress);
+    }
+
+    DFSClient dfsClient =
+        clientCache.getDfsClient(securityHandler.getUser(), namenodeId);
+    if (dfsClient == null) {
+      response.setStatus(Nfs3Status.NFS3ERR_SERVERFAULT);
+      return response;
     }
 
     try {
@@ -1471,7 +1512,7 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
           .setPostOpAttr(Nfs3Utils.getFileAttr(dfsClient, linkDirIdPath, iug));
 
       return new SYMLINK3Response(Nfs3Status.NFS3_OK, new FileHandle(
-          objAttr.getFileId()), objAttr, dirWcc);
+          objAttr.getFileId(), namenodeId), objAttr, dirWcc);
 
     } catch (IOException e) {
       LOG.warn("Exception: " + e);
@@ -1524,12 +1565,6 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
       return response;
     }
 
-    DFSClient dfsClient = clientCache.getDfsClient(securityHandler.getUser());
-    if (dfsClient == null) {
-      response.setStatus(Nfs3Status.NFS3ERR_SERVERFAULT);
-      return response;
-    }
-
     READDIR3Request request;
     try {
       request = READDIR3Request.deserialize(xdr);
@@ -1538,6 +1573,8 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
       return new READDIR3Response(Nfs3Status.NFS3ERR_INVAL);
     }
     FileHandle handle = request.getHandle();
+    int namenodeId = handle.getNamenodeId();
+
     long cookie = request.getCookie();
     if (cookie < 0) {
       LOG.error("Invalid READDIR request, with negative cookie: " + cookie);
@@ -1550,8 +1587,16 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
     }
 
     if (LOG.isDebugEnabled()) {
-      LOG.debug("NFS READDIR fileId: " + handle.getFileId() + " cookie: "
-          + cookie + " count: " + count + " client: " + remoteAddress);
+      LOG.debug("NFS READDIR fileHandle: " + handle.dumpFileHandle()
+          + " cookie: " + cookie + " count: " + count + " client: "
+          + remoteAddress);
+    }
+
+    DFSClient dfsClient =
+        clientCache.getDfsClient(securityHandler.getUser(), namenodeId);
+    if (dfsClient == null) {
+      response.setStatus(Nfs3Status.NFS3ERR_SERVERFAULT);
+      return response;
     }
 
     HdfsFileStatus dirStatus;
@@ -1684,10 +1729,6 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
       return new READDIRPLUS3Response(Nfs3Status.NFS3ERR_ACCES);
     }
 
-    DFSClient dfsClient = clientCache.getDfsClient(securityHandler.getUser());
-    if (dfsClient == null) {
-      return new READDIRPLUS3Response(Nfs3Status.NFS3ERR_SERVERFAULT);
-    }
 
     READDIRPLUS3Request request = null;
     try {
@@ -1698,6 +1739,7 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
     }
 
     FileHandle handle = request.getHandle();
+    int namenodeId = handle.getNamenodeId();
     long cookie = request.getCookie();
     if (cookie < 0) {
       LOG.error("Invalid READDIRPLUS request, with negative cookie: " + cookie);
@@ -1715,9 +1757,15 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
     }
 
     if (LOG.isDebugEnabled()) {
-      LOG.debug("NFS READDIRPLUS fileId: " + handle.getFileId() + " cookie: "
-          + cookie + " dirCount: " + dirCount + " maxCount: " + maxCount
-          + " client: " + remoteAddress);
+      LOG.debug("NFS READDIRPLUS fileHandle: " + handle.dumpFileHandle()
+          + " cookie: " + cookie + " dirCount: " + dirCount + " maxCount: "
+          + maxCount + " client: " + remoteAddress);
+    }
+
+    DFSClient dfsClient =
+        clientCache.getDfsClient(securityHandler.getUser(), namenodeId);
+    if (dfsClient == null) {
+      return new READDIRPLUS3Response(Nfs3Status.NFS3ERR_SERVERFAULT);
     }
 
     HdfsFileStatus dirStatus;
@@ -1805,14 +1853,14 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
 
       entries[0] = new READDIRPLUS3Response.EntryPlus3(
           postOpDirAttr.getFileId(), ".", 0, postOpDirAttr, new FileHandle(
-              postOpDirAttr.getFileId()));
+              postOpDirAttr.getFileId(), namenodeId));
       entries[1] = new READDIRPLUS3Response.EntryPlus3(dotdotFileId, "..",
           dotdotFileId, Nfs3Utils.getNfs3FileAttrFromFileStatus(dotdotStatus,
-              iug), new FileHandle(dotdotFileId));
+              iug), new FileHandle(dotdotFileId, namenodeId));
 
       for (int i = 2; i < n + 2; i++) {
         long fileId = fstatus[i - 2].getFileId();
-        FileHandle childHandle = new FileHandle(fileId);
+        FileHandle childHandle = new FileHandle(fileId, namenodeId);
         Nfs3FileAttributes attr;
         try {
           attr = writeManager.getFileAttr(dfsClient, childHandle, iug);
@@ -1829,7 +1877,7 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
       entries = new READDIRPLUS3Response.EntryPlus3[n]; 
       for (int i = 0; i < n; i++) {
         long fileId = fstatus[i].getFileId();
-        FileHandle childHandle = new FileHandle(fileId);
+        FileHandle childHandle = new FileHandle(fileId, namenodeId);
         Nfs3FileAttributes attr;
         try {
           attr = writeManager.getFileAttr(dfsClient, childHandle, iug);
@@ -1863,11 +1911,6 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
       return response;
     }
 
-    DFSClient dfsClient = clientCache.getDfsClient(securityHandler.getUser());
-    if (dfsClient == null) {
-      response.setStatus(Nfs3Status.NFS3ERR_SERVERFAULT);
-      return response;
-    }
 
     FSSTAT3Request request;
     try {
@@ -1878,9 +1921,17 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
     }
 
     FileHandle handle = request.getHandle();
+    int namenodeId = handle.getNamenodeId();
     if (LOG.isDebugEnabled()) {
-      LOG.debug("NFS FSSTAT fileId: " + handle.getFileId() + " client: "
-          + remoteAddress);
+      LOG.debug("NFS FSSTAT fileHandle: " + handle.dumpFileHandle()
+          + " client: " + remoteAddress);
+    }
+
+    DFSClient dfsClient =
+        clientCache.getDfsClient(securityHandler.getUser(), namenodeId);
+    if (dfsClient == null) {
+      response.setStatus(Nfs3Status.NFS3ERR_SERVERFAULT);
+      return response;
     }
 
     try {
@@ -1938,12 +1989,6 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
       return response;
     }
 
-    DFSClient dfsClient = clientCache.getDfsClient(securityHandler.getUser());
-    if (dfsClient == null) {
-      response.setStatus(Nfs3Status.NFS3ERR_SERVERFAULT);
-      return response;
-    }
-
     FSINFO3Request request;
     try {
       request = FSINFO3Request.deserialize(xdr);
@@ -1953,9 +1998,17 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
     }
 
     FileHandle handle = request.getHandle();
+    int namenodeId = handle.getNamenodeId();
     if (LOG.isDebugEnabled()) {
-      LOG.debug("NFS FSINFO fileId: " + handle.getFileId() + " client: "
-          + remoteAddress);
+      LOG.debug("NFS FSINFO fileHandle: " + handle.dumpFileHandle()
+          +" client: " + remoteAddress);
+    }
+
+    DFSClient dfsClient =
+        clientCache.getDfsClient(securityHandler.getUser(), namenodeId);
+    if (dfsClient == null) {
+      response.setStatus(Nfs3Status.NFS3ERR_SERVERFAULT);
+      return response;
     }
 
     try {
@@ -2003,12 +2056,6 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
       return response;
     }
 
-    DFSClient dfsClient = clientCache.getDfsClient(securityHandler.getUser());
-    if (dfsClient == null) {
-      response.setStatus(Nfs3Status.NFS3ERR_SERVERFAULT);
-      return response;
-    }
-
     PATHCONF3Request request;
     try {
       request = PATHCONF3Request.deserialize(xdr);
@@ -2019,10 +2066,18 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
 
     FileHandle handle = request.getHandle();
     Nfs3FileAttributes attrs;
+    int namenodeId = handle.getNamenodeId();
 
     if (LOG.isDebugEnabled()) {
-      LOG.debug("NFS PATHCONF fileId: " + handle.getFileId() + " client: "
-          + remoteAddress);
+      LOG.debug("NFS PATHCONF fileHandle: " + handle.dumpFileHandle()
+          + " client: " + remoteAddress);
+    }
+
+    DFSClient dfsClient =
+        clientCache.getDfsClient(securityHandler.getUser(), namenodeId);
+    if (dfsClient == null) {
+      response.setStatus(Nfs3Status.NFS3ERR_SERVERFAULT);
+      return response;
     }
 
     try {
@@ -2055,11 +2110,6 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
   COMMIT3Response commit(XDR xdr, Channel channel, int xid,
       SecurityHandler securityHandler, SocketAddress remoteAddress) {
     COMMIT3Response response = new COMMIT3Response(Nfs3Status.NFS3_OK);
-    DFSClient dfsClient = clientCache.getDfsClient(securityHandler.getUser());
-    if (dfsClient == null) {
-      response.setStatus(Nfs3Status.NFS3ERR_SERVERFAULT);
-      return response;
-    }
 
     COMMIT3Request request;
     try {
@@ -2071,12 +2121,20 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
     }
 
     FileHandle handle = request.getHandle();
+    int namenodeId = handle.getNamenodeId();
     if (LOG.isDebugEnabled()) {
-      LOG.debug("NFS COMMIT fileId: " + handle.getFileId() + " offset="
+      LOG.debug("NFS COMMIT fileHandle: " + handle.dumpFileHandle() + " offset="
           + request.getOffset() + " count=" + request.getCount() + " client: "
           + remoteAddress);
     }
 
+    DFSClient dfsClient =
+        clientCache.getDfsClient(securityHandler.getUser(), namenodeId);
+    if (dfsClient == null) {
+      response.setStatus(Nfs3Status.NFS3ERR_SERVERFAULT);
+      return response;
+    }
+
     String fileIdPath = Nfs3Utils.getFileIdPath(handle);
     Nfs3FileAttributes preOpAttr = null;
     try {
@@ -2097,7 +2155,7 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
 
       // Insert commit as an async request
       writeManager.handleCommit(dfsClient, handle, commitOffset, channel, xid,
-          preOpAttr);
+          preOpAttr, namenodeId);
       return null;
     } catch (IOException e) {
       LOG.warn("Exception ", e);

+ 4 - 4
hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/WriteCtx.java

@@ -318,9 +318,9 @@ class WriteCtx {
   
   @Override
   public String toString() {
-    return "Id:" + handle.getFileId() + " offset:" + getPlainOffset() + " " +
-        "count:" + count + " originalCount:" + getOriginalCount() +
-        " stableHow:" + stableHow + " replied:" + replied + " dataState:" +
-        dataState + " xid:" + xid;
+    return "FileHandle:" + handle.dumpFileHandle() + " offset:"
+        + getPlainOffset() + " " + "count:" + count + " originalCount:"
+        + getOriginalCount() + " stableHow:" + stableHow + " replied:"
+        + replied + " dataState:" + dataState + " xid:" + xid;
   }
 }

+ 15 - 9
hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/WriteManager.java

@@ -139,7 +139,8 @@ public class WriteManager {
     FileHandle fileHandle = request.getHandle();
     OpenFileCtx openFileCtx = fileContextCache.get(fileHandle);
     if (openFileCtx == null) {
-      LOG.info("No opened stream for fileId: " + fileHandle.getFileId());
+      LOG.info("No opened stream for fileHandle: "
+          + fileHandle.dumpFileHandle());
 
       String fileIdPath = Nfs3Utils.getFileIdPath(fileHandle.getFileId());
       HdfsDataOutputStream fos = null;
@@ -188,7 +189,8 @@ public class WriteManager {
         try {
           fos.close();
         } catch (IOException e) {
-          LOG.error("Can't close stream for fileId: " + handle.getFileId(), e);
+          LOG.error("Can't close stream for fileHandle: "
+              + handle.dumpFileHandle(), e);
         }
         // Notify client to retry
         WccData fileWcc = new WccData(latestAttr.getWccAttr(), latestAttr);
@@ -201,7 +203,8 @@ public class WriteManager {
       }
 
       if (LOG.isDebugEnabled()) {
-        LOG.debug("Opened stream for appending file: " + fileHandle.getFileId());
+        LOG.debug("Opened stream for appending file: "
+            + fileHandle.dumpFileHandle());
       }
     }
 
@@ -220,7 +223,7 @@ public class WriteManager {
 
     if (openFileCtx == null) {
       if (LOG.isDebugEnabled()) {
-        LOG.debug("No opened stream for fileId: " + fileHandle.getFileId()
+        LOG.debug("No opened stream for fileId: " + fileHandle.dumpFileHandle()
             + " commitOffset=" + commitOffset
             + ". Return success in this case.");
       }
@@ -263,13 +266,14 @@ public class WriteManager {
   }
   
   void handleCommit(DFSClient dfsClient, FileHandle fileHandle,
-      long commitOffset, Channel channel, int xid, Nfs3FileAttributes preOpAttr) {
+      long commitOffset, Channel channel, int xid, Nfs3FileAttributes preOpAttr,
+      int namenodeId) {
     long startTime = System.nanoTime();
     int status;
     OpenFileCtx openFileCtx = fileContextCache.get(fileHandle);
 
     if (openFileCtx == null) {
-      LOG.info("No opened stream for fileId: " + fileHandle.getFileId()
+      LOG.info("No opened stream for fileId: " + fileHandle.dumpFileHandle()
           + " commitOffset=" + commitOffset + ". Return success in this case.");
       status = Nfs3Status.NFS3_OK;
       
@@ -304,7 +308,9 @@ public class WriteManager {
     // Send out the response
     Nfs3FileAttributes postOpAttr = null;
     try {
-      postOpAttr = getFileAttr(dfsClient, new FileHandle(preOpAttr.getFileId()), iug);
+      postOpAttr =
+          getFileAttr(dfsClient, new FileHandle(preOpAttr.getFileId(),
+              namenodeId), iug);
     } catch (IOException e1) {
       LOG.info("Can't get postOpAttr for fileId: " + preOpAttr.getFileId(), e1);
     }
@@ -334,13 +340,13 @@ public class WriteManager {
   }
 
   Nfs3FileAttributes getFileAttr(DFSClient client, FileHandle dirHandle,
-      String fileName) throws IOException {
+      String fileName, int namenodeId) throws IOException {
     String fileIdPath = Nfs3Utils.getFileIdPath(dirHandle) + "/" + fileName;
     Nfs3FileAttributes attr = Nfs3Utils.getFileAttr(client, fileIdPath, iug);
 
     if ((attr != null) && (attr.getType() == NfsFileType.NFSREG.toValue())) {
       OpenFileCtx openFileCtx = fileContextCache.get(new FileHandle(attr
-          .getFileId()));
+          .getFileId(), namenodeId));
 
       if (openFileCtx != null) {
         attr.setSize(openFileCtx.getNextOffset());

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/nfs3/TestClientAccessPrivilege.java

@@ -101,9 +101,10 @@ public class TestClientAccessPrivilege {
     // Create a remove request
     HdfsFileStatus status = nn.getRpcServer().getFileInfo(testdir);
     long dirId = status.getFileId();
+    int namenodeId = Nfs3Utils.getNamenodeId(config);
 
     XDR xdr_req = new XDR();
-    FileHandle handle = new FileHandle(dirId);
+    FileHandle handle = new FileHandle(dirId, namenodeId);
     handle.serialize(xdr_req);
     xdr_req.writeString("f1");
 

+ 8 - 5
hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/nfs3/TestDFSClientCache.java

@@ -43,15 +43,17 @@ public class TestDFSClientCache {
 
     DFSClientCache cache = new DFSClientCache(conf, MAX_CACHE_SIZE);
 
-    DFSClient c1 = cache.getDfsClient("test1");
-    assertTrue(cache.getDfsClient("test1").toString().contains("ugi=test1"));
-    assertEquals(c1, cache.getDfsClient("test1"));
+    int namenodeId = Nfs3Utils.getNamenodeId(conf);
+    DFSClient c1 = cache.getDfsClient("test1", namenodeId);
+    assertTrue(cache.getDfsClient("test1", namenodeId)
+        .toString().contains("ugi=test1"));
+    assertEquals(c1, cache.getDfsClient("test1", namenodeId));
     assertFalse(isDfsClientClose(c1));
 
-    cache.getDfsClient("test2");
+    cache.getDfsClient("test2", namenodeId);
     assertTrue(isDfsClientClose(c1));
     assertTrue("cache size should be the max size or less",
-        cache.clientCache.size() <= MAX_CACHE_SIZE);
+        cache.getClientCache().size() <= MAX_CACHE_SIZE);
   }
 
   @Test
@@ -61,6 +63,7 @@ public class TestDFSClientCache {
 
 
     NfsConfiguration conf = new NfsConfiguration();
+    conf.set(FileSystem.FS_DEFAULT_NAME_KEY, "hdfs://localhost");
     UserGroupInformation currentUserUgi
             = UserGroupInformation.createRemoteUser(currentUser);
     currentUserUgi.setAuthenticationMethod(KERBEROS);

+ 159 - 2
hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/nfs3/TestExportsTable.java

@@ -21,7 +21,14 @@ import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
 
+import org.apache.commons.lang3.RandomStringUtils;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.apache.hadoop.fs.FsConstants;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.viewfs.ConfigUtil;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.MiniDFSNNTopology;
 import org.apache.hadoop.hdfs.nfs.conf.NfsConfigKeys;
 import org.apache.hadoop.hdfs.nfs.conf.NfsConfiguration;
 import org.apache.hadoop.hdfs.nfs.mount.Mountd;
@@ -31,7 +38,152 @@ import org.junit.Test;
 public class TestExportsTable {
  
   @Test
-  public void testExportPoint() throws IOException {
+  public void testHdfsExportPoint() throws IOException {
+    NfsConfiguration config = new NfsConfiguration();
+    MiniDFSCluster cluster = null;
+
+    // Use emphral port in case tests are running in parallel
+    config.setInt("nfs3.mountd.port", 0);
+    config.setInt("nfs3.server.port", 0);
+    config.set("nfs.http.address", "0.0.0.0:0");
+
+    try {
+      cluster = new MiniDFSCluster.Builder(config).numDataNodes(1).build();
+      cluster.waitActive();
+
+      // Start nfs
+      final Nfs3 nfsServer = new Nfs3(config);
+      nfsServer.startServiceInternal(false);
+
+      Mountd mountd = nfsServer.getMountd();
+      RpcProgramMountd rpcMount = (RpcProgramMountd) mountd.getRpcProgram();
+      assertTrue(rpcMount.getExports().size() == 1);
+
+      String exportInMountd = rpcMount.getExports().get(0);
+      assertTrue(exportInMountd.equals("/"));
+
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
+
+  @Test
+  public void testViewFsExportPoint() throws IOException {
+    NfsConfiguration config = new NfsConfiguration();
+    MiniDFSCluster cluster = null;
+    String clusterName = RandomStringUtils.randomAlphabetic(10);
+
+    String exportPoint = "/hdfs1,/hdfs2";
+    config.setStrings(NfsConfigKeys.DFS_NFS_EXPORT_POINT_KEY, exportPoint);
+    config.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY,
+        FsConstants.VIEWFS_SCHEME + "://" + clusterName);
+    // Use emphral port in case tests are running in parallel
+    config.setInt("nfs3.mountd.port", 0);
+    config.setInt("nfs3.server.port", 0);
+    config.set("nfs.http.address", "0.0.0.0:0");
+
+    try {
+      cluster =
+          new MiniDFSCluster.Builder(config).nnTopology(
+              MiniDFSNNTopology.simpleFederatedTopology(2))
+              .numDataNodes(2)
+              .build();
+      cluster.waitActive();
+      DistributedFileSystem hdfs1 = cluster.getFileSystem(0);
+      DistributedFileSystem hdfs2 = cluster.getFileSystem(1);
+      cluster.waitActive();
+      Path base1 = new Path("/user1");
+      Path base2 = new Path("/user2");
+      hdfs1.delete(base1, true);
+      hdfs2.delete(base2, true);
+      hdfs1.mkdirs(base1);
+      hdfs2.mkdirs(base2);
+      ConfigUtil.addLink(config, clusterName, "/hdfs1",
+          hdfs1.makeQualified(base1).toUri());
+      ConfigUtil.addLink(config, clusterName, "/hdfs2",
+          hdfs2.makeQualified(base2).toUri());
+
+      // Start nfs
+      final Nfs3 nfsServer = new Nfs3(config);
+      nfsServer.startServiceInternal(false);
+
+      Mountd mountd = nfsServer.getMountd();
+      RpcProgramMountd rpcMount = (RpcProgramMountd) mountd.getRpcProgram();
+      assertTrue(rpcMount.getExports().size() == 2);
+
+      String exportInMountd1 = rpcMount.getExports().get(0);
+      assertTrue(exportInMountd1.equals("/hdfs1"));
+
+      String exportInMountd2 = rpcMount.getExports().get(1);
+      assertTrue(exportInMountd2.equals("/hdfs2"));
+
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
+
+  @Test
+  public void testViewFsInternalExportPoint() throws IOException {
+    NfsConfiguration config = new NfsConfiguration();
+    MiniDFSCluster cluster = null;
+    String clusterName = RandomStringUtils.randomAlphabetic(10);
+
+    String exportPoint = "/hdfs1/subpath";
+    config.setStrings(NfsConfigKeys.DFS_NFS_EXPORT_POINT_KEY, exportPoint);
+    config.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY,
+        FsConstants.VIEWFS_SCHEME + "://" + clusterName);
+    // Use emphral port in case tests are running in parallel
+    config.setInt("nfs3.mountd.port", 0);
+    config.setInt("nfs3.server.port", 0);
+    config.set("nfs.http.address", "0.0.0.0:0");
+
+    try {
+      cluster =
+          new MiniDFSCluster.Builder(config).nnTopology(
+              MiniDFSNNTopology.simpleFederatedTopology(2))
+              .numDataNodes(2)
+              .build();
+      cluster.waitActive();
+      DistributedFileSystem hdfs1 = cluster.getFileSystem(0);
+      DistributedFileSystem hdfs2 = cluster.getFileSystem(1);
+      cluster.waitActive();
+      Path base1 = new Path("/user1");
+      Path base2 = new Path("/user2");
+      hdfs1.delete(base1, true);
+      hdfs2.delete(base2, true);
+      hdfs1.mkdirs(base1);
+      hdfs2.mkdirs(base2);
+      ConfigUtil.addLink(config, clusterName, "/hdfs1",
+          hdfs1.makeQualified(base1).toUri());
+      ConfigUtil.addLink(config, clusterName, "/hdfs2",
+          hdfs2.makeQualified(base2).toUri());
+      Path subPath = new Path(base1, "subpath");
+      hdfs1.delete(subPath, true);
+      hdfs1.mkdirs(subPath);
+
+      // Start nfs
+      final Nfs3 nfsServer = new Nfs3(config);
+      nfsServer.startServiceInternal(false);
+
+      Mountd mountd = nfsServer.getMountd();
+      RpcProgramMountd rpcMount = (RpcProgramMountd) mountd.getRpcProgram();
+      assertTrue(rpcMount.getExports().size() == 1);
+
+      String exportInMountd = rpcMount.getExports().get(0);
+      assertTrue(exportInMountd.equals(exportPoint));
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
+
+  @Test
+  public void testHdfsInternalExportPoint() throws IOException {
     NfsConfiguration config = new NfsConfiguration();
     MiniDFSCluster cluster = null;
 
@@ -40,10 +192,15 @@ public class TestExportsTable {
     // Use emphral port in case tests are running in parallel
     config.setInt("nfs3.mountd.port", 0);
     config.setInt("nfs3.server.port", 0);
-    
+    config.set("nfs.http.address", "0.0.0.0:0");
+    Path base = new Path(exportPoint);
+
     try {
       cluster = new MiniDFSCluster.Builder(config).numDataNodes(1).build();
       cluster.waitActive();
+      DistributedFileSystem hdfs = cluster.getFileSystem(0);
+      hdfs.delete(base, true);
+      hdfs.mkdirs(base);
 
       // Start nfs
       final Nfs3 nfsServer = new Nfs3(config);

+ 6 - 13
hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/nfs3/TestReaddir.java

@@ -21,9 +21,7 @@ package org.apache.hadoop.hdfs.nfs.nfs3;
 import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
-import java.net.InetAddress;
 import java.net.InetSocketAddress;
-import java.net.SocketAddress;
 import java.util.List;
 
 import org.apache.hadoop.fs.Path;
@@ -31,8 +29,6 @@ import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.nfs.conf.NfsConfiguration;
-import org.apache.hadoop.hdfs.nfs.nfs3.Nfs3;
-import org.apache.hadoop.hdfs.nfs.nfs3.RpcProgramNfs3;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.nfs.nfs3.FileHandle;
@@ -40,15 +36,10 @@ import org.apache.hadoop.nfs.nfs3.response.READDIR3Response;
 import org.apache.hadoop.nfs.nfs3.response.READDIR3Response.Entry3;
 import org.apache.hadoop.nfs.nfs3.response.READDIRPLUS3Response;
 import org.apache.hadoop.nfs.nfs3.response.READDIRPLUS3Response.EntryPlus3;
-import org.apache.hadoop.oncrpc.RpcInfo;
-import org.apache.hadoop.oncrpc.RpcMessage;
 import org.apache.hadoop.oncrpc.XDR;
 import org.apache.hadoop.oncrpc.security.SecurityHandler;
 import org.apache.hadoop.security.authorize.DefaultImpersonationProvider;
 import org.apache.hadoop.security.authorize.ProxyUsers;
-import org.jboss.netty.buffer.ChannelBuffer;
-import org.jboss.netty.channel.Channel;
-import org.jboss.netty.channel.ChannelHandlerContext;
 import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.BeforeClass;
@@ -119,10 +110,11 @@ public class TestReaddir {
     // Get inodeId of /tmp
     HdfsFileStatus status = nn.getRpcServer().getFileInfo(testdir);
     long dirId = status.getFileId();
+    int namenodeId = Nfs3Utils.getNamenodeId(config);
 
     // Create related part of the XDR request
     XDR xdr_req = new XDR();
-    FileHandle handle = new FileHandle(dirId);
+    FileHandle handle = new FileHandle(dirId, namenodeId);
     handle.serialize(xdr_req);
     xdr_req.writeLongAsHyper(0); // cookie
     xdr_req.writeLongAsHyper(0); // verifier
@@ -139,7 +131,7 @@ public class TestReaddir {
 
     // Create related part of the XDR request
     xdr_req = new XDR();
-    handle = new FileHandle(dirId);
+    handle = new FileHandle(dirId, namenodeId);
     handle.serialize(xdr_req);
     xdr_req.writeLongAsHyper(f2Id); // cookie
     xdr_req.writeLongAsHyper(0); // verifier
@@ -167,10 +159,11 @@ public class TestReaddir {
     // Get inodeId of /tmp
     HdfsFileStatus status = nn.getRpcServer().getFileInfo(testdir);
     long dirId = status.getFileId();
+    int namenodeId = Nfs3Utils.getNamenodeId(config);
     
     // Create related part of the XDR request
     XDR xdr_req = new XDR();
-    FileHandle handle = new FileHandle(dirId);
+    FileHandle handle = new FileHandle(dirId, namenodeId);
     handle.serialize(xdr_req);
     xdr_req.writeLongAsHyper(0); // cookie
     xdr_req.writeLongAsHyper(0); // verifier
@@ -189,7 +182,7 @@ public class TestReaddir {
 
     // Create related part of the XDR request
     xdr_req = new XDR();
-    handle = new FileHandle(dirId);
+    handle = new FileHandle(dirId, namenodeId);
     handle.serialize(xdr_req);
     xdr_req.writeLongAsHyper(f2Id); // cookie
     xdr_req.writeLongAsHyper(0); // verifier

+ 44 - 22
hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/nfs3/TestRpcProgramNfs3.java

@@ -186,7 +186,8 @@ public class TestRpcProgramNfs3 {
   public void testGetattr() throws Exception {
     HdfsFileStatus status = nn.getRpcServer().getFileInfo("/tmp/bar");
     long dirId = status.getFileId();
-    FileHandle handle = new FileHandle(dirId);
+    int namenodeId = Nfs3Utils.getNamenodeId(config);
+    FileHandle handle = new FileHandle(dirId, namenodeId);
     XDR xdr_req = new XDR();
     GETATTR3Request req = new GETATTR3Request(handle);
     req.serialize(xdr_req);
@@ -209,8 +210,9 @@ public class TestRpcProgramNfs3 {
   public void testSetattr() throws Exception {
     HdfsFileStatus status = nn.getRpcServer().getFileInfo(testdir);
     long dirId = status.getFileId();
+    int namenodeId = Nfs3Utils.getNamenodeId(config);
     XDR xdr_req = new XDR();
-    FileHandle handle = new FileHandle(dirId);
+    FileHandle handle = new FileHandle(dirId, namenodeId);
     SetAttr3 symAttr = new SetAttr3(0, 1, 0, 0, null, null,
         EnumSet.of(SetAttrField.UID));
     SETATTR3Request req = new SETATTR3Request(handle, symAttr, false, null);
@@ -234,7 +236,8 @@ public class TestRpcProgramNfs3 {
   public void testLookup() throws Exception {
     HdfsFileStatus status = nn.getRpcServer().getFileInfo(testdir);
     long dirId = status.getFileId();
-    FileHandle handle = new FileHandle(dirId);
+    int namenodeId = Nfs3Utils.getNamenodeId(config);
+    FileHandle handle = new FileHandle(dirId, namenodeId);
     LOOKUP3Request lookupReq = new LOOKUP3Request(handle, "bar");
     XDR xdr_req = new XDR();
     lookupReq.serialize(xdr_req);
@@ -257,7 +260,8 @@ public class TestRpcProgramNfs3 {
   public void testAccess() throws Exception {
     HdfsFileStatus status = nn.getRpcServer().getFileInfo("/tmp/bar");
     long dirId = status.getFileId();
-    FileHandle handle = new FileHandle(dirId);
+    int namenodeId = Nfs3Utils.getNamenodeId(config);
+    FileHandle handle = new FileHandle(dirId, namenodeId);
     XDR xdr_req = new XDR();
     ACCESS3Request req = new ACCESS3Request(handle);
     req.serialize(xdr_req);
@@ -281,8 +285,9 @@ public class TestRpcProgramNfs3 {
     // Create a symlink first.
     HdfsFileStatus status = nn.getRpcServer().getFileInfo(testdir);
     long dirId = status.getFileId();
+    int namenodeId = Nfs3Utils.getNamenodeId(config);
     XDR xdr_req = new XDR();
-    FileHandle handle = new FileHandle(dirId);
+    FileHandle handle = new FileHandle(dirId, namenodeId);
     SYMLINK3Request req = new SYMLINK3Request(handle, "fubar", new SetAttr3(),
         "bar");
     req.serialize(xdr_req);
@@ -316,7 +321,8 @@ public class TestRpcProgramNfs3 {
   public void testRead() throws Exception {
     HdfsFileStatus status = nn.getRpcServer().getFileInfo("/tmp/bar");
     long dirId = status.getFileId();
-    FileHandle handle = new FileHandle(dirId);
+    int namenodeId = Nfs3Utils.getNamenodeId(config);
+    FileHandle handle = new FileHandle(dirId, namenodeId);
 
     READ3Request readReq = new READ3Request(handle, 0, 5);
     XDR xdr_req = new XDR();
@@ -373,7 +379,8 @@ public class TestRpcProgramNfs3 {
 
     final HdfsFileStatus status = nn.getRpcServer().getFileInfo(fileName);
     final long dirId = status.getFileId();
-    final FileHandle handle = new FileHandle(dirId);
+    final int namenodeId = Nfs3Utils.getNamenodeId(config);
+    final FileHandle handle = new FileHandle(dirId, namenodeId);
 
     final WRITE3Request writeReq = new WRITE3Request(handle, 0,
         buffer.length, WriteStableHow.DATA_SYNC, ByteBuffer.wrap(buffer));
@@ -390,7 +397,8 @@ public class TestRpcProgramNfs3 {
       throws Exception {
     final HdfsFileStatus status = nn.getRpcServer().getFileInfo(fileName);
     final long dirId = status.getFileId();
-    final FileHandle handle = new FileHandle(dirId);
+    final int namenodeId = Nfs3Utils.getNamenodeId(config);
+    final FileHandle handle = new FileHandle(dirId, namenodeId);
 
     final READ3Request readReq = new READ3Request(handle, 0, len);
     final XDR xdr_req = new XDR();
@@ -422,7 +430,8 @@ public class TestRpcProgramNfs3 {
   private void commit(String fileName, int len) throws Exception {
     final HdfsFileStatus status = nn.getRpcServer().getFileInfo(fileName);
     final long dirId = status.getFileId();
-    final FileHandle handle = new FileHandle(dirId);
+    final int namenodeId = Nfs3Utils.getNamenodeId(config);
+    final FileHandle handle = new FileHandle(dirId, namenodeId);
     final XDR xdr_req = new XDR();
     final COMMIT3Request req = new COMMIT3Request(handle, 0, len);
     req.serialize(xdr_req);
@@ -439,7 +448,8 @@ public class TestRpcProgramNfs3 {
   public void testWrite() throws Exception {
     HdfsFileStatus status = nn.getRpcServer().getFileInfo("/tmp/bar");
     long dirId = status.getFileId();
-    FileHandle handle = new FileHandle(dirId);
+    int namenodeId = Nfs3Utils.getNamenodeId(config);
+    FileHandle handle = new FileHandle(dirId, namenodeId);
 
     byte[] buffer = new byte[10];
     for (int i = 0; i < 10; i++) {
@@ -469,8 +479,9 @@ public class TestRpcProgramNfs3 {
   public void testCreate() throws Exception {
     HdfsFileStatus status = nn.getRpcServer().getFileInfo(testdir);
     long dirId = status.getFileId();
+    int namenodeId = Nfs3Utils.getNamenodeId(config);
     XDR xdr_req = new XDR();
-    FileHandle handle = new FileHandle(dirId);
+    FileHandle handle = new FileHandle(dirId, namenodeId);
     CREATE3Request req = new CREATE3Request(handle, "fubar",
         Nfs3Constant.CREATE_UNCHECKED, new SetAttr3(), 0);
     req.serialize(xdr_req);
@@ -493,8 +504,9 @@ public class TestRpcProgramNfs3 {
   public void testMkdir() throws Exception {//FixME
     HdfsFileStatus status = nn.getRpcServer().getFileInfo(testdir);
     long dirId = status.getFileId();
+    int namenodeId = Nfs3Utils.getNamenodeId(config);
     XDR xdr_req = new XDR();
-    FileHandle handle = new FileHandle(dirId);
+    FileHandle handle = new FileHandle(dirId, namenodeId);
     MKDIR3Request req = new MKDIR3Request(handle, "fubar1", new SetAttr3());
     req.serialize(xdr_req);
     
@@ -520,8 +532,9 @@ public class TestRpcProgramNfs3 {
   public void testSymlink() throws Exception {
     HdfsFileStatus status = nn.getRpcServer().getFileInfo(testdir);
     long dirId = status.getFileId();
+    int namenodeId = Nfs3Utils.getNamenodeId(config);
     XDR xdr_req = new XDR();
-    FileHandle handle = new FileHandle(dirId);
+    FileHandle handle = new FileHandle(dirId, namenodeId);
     SYMLINK3Request req = new SYMLINK3Request(handle, "fubar", new SetAttr3(),
         "bar");
     req.serialize(xdr_req);
@@ -544,8 +557,9 @@ public class TestRpcProgramNfs3 {
   public void testRemove() throws Exception {
     HdfsFileStatus status = nn.getRpcServer().getFileInfo(testdir);
     long dirId = status.getFileId();
+    int namenodeId = Nfs3Utils.getNamenodeId(config);
     XDR xdr_req = new XDR();
-    FileHandle handle = new FileHandle(dirId);
+    FileHandle handle = new FileHandle(dirId, namenodeId);
     REMOVE3Request req = new REMOVE3Request(handle, "bar");
     req.serialize(xdr_req);
 
@@ -567,8 +581,9 @@ public class TestRpcProgramNfs3 {
   public void testRmdir() throws Exception {
     HdfsFileStatus status = nn.getRpcServer().getFileInfo(testdir);
     long dirId = status.getFileId();
+    int namenodeId = Nfs3Utils.getNamenodeId(config);
     XDR xdr_req = new XDR();
-    FileHandle handle = new FileHandle(dirId);
+    FileHandle handle = new FileHandle(dirId, namenodeId);
     RMDIR3Request req = new RMDIR3Request(handle, "foo");
     req.serialize(xdr_req);
 
@@ -590,8 +605,9 @@ public class TestRpcProgramNfs3 {
   public void testRename() throws Exception {
     HdfsFileStatus status = nn.getRpcServer().getFileInfo(testdir);
     long dirId = status.getFileId();
+    int namenodeId = Nfs3Utils.getNamenodeId(config);
     XDR xdr_req = new XDR();
-    FileHandle handle = new FileHandle(dirId);
+    FileHandle handle = new FileHandle(dirId, namenodeId);
     RENAME3Request req = new RENAME3Request(handle, "bar", handle, "fubar");
     req.serialize(xdr_req);
     
@@ -613,7 +629,8 @@ public class TestRpcProgramNfs3 {
   public void testReaddir() throws Exception {
     HdfsFileStatus status = nn.getRpcServer().getFileInfo(testdir);
     long dirId = status.getFileId();
-    FileHandle handle = new FileHandle(dirId);
+    int namenodeId = Nfs3Utils.getNamenodeId(config);
+    FileHandle handle = new FileHandle(dirId, namenodeId);
     XDR xdr_req = new XDR();
     READDIR3Request req = new READDIR3Request(handle, 0, 0, 100);
     req.serialize(xdr_req);
@@ -636,7 +653,8 @@ public class TestRpcProgramNfs3 {
   public void testReaddirplus() throws Exception {
     HdfsFileStatus status = nn.getRpcServer().getFileInfo(testdir);
     long dirId = status.getFileId();
-    FileHandle handle = new FileHandle(dirId);
+    int namenodeId = Nfs3Utils.getNamenodeId(config);
+    FileHandle handle = new FileHandle(dirId, namenodeId);
     XDR xdr_req = new XDR();
     READDIRPLUS3Request req = new READDIRPLUS3Request(handle, 0, 0, 3, 2);
     req.serialize(xdr_req);
@@ -659,7 +677,8 @@ public class TestRpcProgramNfs3 {
   public void testFsstat() throws Exception {
     HdfsFileStatus status = nn.getRpcServer().getFileInfo("/tmp/bar");
     long dirId = status.getFileId();
-    FileHandle handle = new FileHandle(dirId);
+    int namenodeId = Nfs3Utils.getNamenodeId(config);
+    FileHandle handle = new FileHandle(dirId, namenodeId);
     XDR xdr_req = new XDR();
     FSSTAT3Request req = new FSSTAT3Request(handle);
     req.serialize(xdr_req);
@@ -682,7 +701,8 @@ public class TestRpcProgramNfs3 {
   public void testFsinfo() throws Exception {
     HdfsFileStatus status = nn.getRpcServer().getFileInfo("/tmp/bar");
     long dirId = status.getFileId();
-    FileHandle handle = new FileHandle(dirId);
+    int namenodeId = Nfs3Utils.getNamenodeId(config);
+    FileHandle handle = new FileHandle(dirId, namenodeId);
     XDR xdr_req = new XDR();
     FSINFO3Request req = new FSINFO3Request(handle);
     req.serialize(xdr_req);
@@ -705,7 +725,8 @@ public class TestRpcProgramNfs3 {
   public void testPathconf() throws Exception {
     HdfsFileStatus status = nn.getRpcServer().getFileInfo("/tmp/bar");
     long dirId = status.getFileId();
-    FileHandle handle = new FileHandle(dirId);
+    int namenodeId = Nfs3Utils.getNamenodeId(config);
+    FileHandle handle = new FileHandle(dirId, namenodeId);
     XDR xdr_req = new XDR();
     PATHCONF3Request req = new PATHCONF3Request(handle);
     req.serialize(xdr_req);
@@ -728,7 +749,8 @@ public class TestRpcProgramNfs3 {
   public void testCommit() throws Exception {
     HdfsFileStatus status = nn.getRpcServer().getFileInfo("/tmp/bar");
     long dirId = status.getFileId();
-    FileHandle handle = new FileHandle(dirId);
+    int namenodeId = Nfs3Utils.getNamenodeId(config);
+    FileHandle handle = new FileHandle(dirId, namenodeId);
     XDR xdr_req = new XDR();
     COMMIT3Request req = new COMMIT3Request(handle, 0, 5);
     req.serialize(xdr_req);

+ 330 - 0
hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/nfs3/TestViewfsWithNfs3.java

@@ -0,0 +1,330 @@
+/**
+ * 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.nfs.nfs3;
+
+import org.apache.hadoop.crypto.key.JavaKeyStoreProvider;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileSystemTestHelper;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.apache.hadoop.fs.FsConstants;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.viewfs.ConfigUtil;
+import org.apache.hadoop.hdfs.DFSTestUtil;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.MiniDFSNNTopology;
+import org.apache.hadoop.hdfs.client.HdfsAdmin;
+import org.apache.hadoop.hdfs.nfs.conf.NfsConfigKeys;
+import org.apache.hadoop.hdfs.nfs.conf.NfsConfiguration;
+import org.apache.hadoop.hdfs.nfs.mount.RpcProgramMountd;
+import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
+import org.apache.hadoop.nfs.nfs3.FileHandle;
+import org.apache.hadoop.nfs.nfs3.Nfs3Constant;
+import org.apache.hadoop.nfs.nfs3.Nfs3Status;
+import org.apache.hadoop.nfs.nfs3.request.GETATTR3Request;
+import org.apache.hadoop.nfs.nfs3.request.RENAME3Request;
+import org.apache.hadoop.nfs.nfs3.response.GETATTR3Response;
+import org.apache.hadoop.nfs.nfs3.request.WRITE3Request;
+import org.apache.hadoop.nfs.nfs3.response.RENAME3Response;
+import org.apache.hadoop.nfs.nfs3.response.WRITE3Response;
+import org.apache.hadoop.oncrpc.XDR;
+import org.apache.hadoop.oncrpc.security.SecurityHandler;
+import org.apache.hadoop.security.authorize.DefaultImpersonationProvider;
+import org.apache.hadoop.security.authorize.ProxyUsers;
+import org.junit.BeforeClass;
+import org.junit.AfterClass;
+import org.junit.Test;
+import org.junit.Assert;
+import org.mockito.Mockito;
+
+import java.io.File;
+import java.net.InetSocketAddress;
+import java.nio.ByteBuffer;
+
+import static org.junit.Assert.assertEquals;
+
+
+/**
+ * Tests for {@link RpcProgramNfs3} with
+ * {@link org.apache.hadoop.fs.viewfs.ViewFileSystem}.
+ */
+public class TestViewfsWithNfs3 {
+  private static DistributedFileSystem hdfs1;
+  private static DistributedFileSystem hdfs2;
+  private static MiniDFSCluster cluster = null;
+  private static NfsConfiguration config = new NfsConfiguration();
+  private static HdfsAdmin dfsAdmin1;
+  private static HdfsAdmin dfsAdmin2;
+  private static FileSystem viewFs;
+
+  private static NameNode nn1;
+  private static NameNode nn2;
+  private static Nfs3 nfs;
+  private static RpcProgramNfs3 nfsd;
+  private static RpcProgramMountd mountd;
+  private static SecurityHandler securityHandler;
+  private static FileSystemTestHelper fsHelper;
+  private static File testRootDir;
+
+  @BeforeClass
+  public static void setup() throws Exception {
+    String currentUser = System.getProperty("user.name");
+
+    config.set("fs.permissions.umask-mode", "u=rwx,g=,o=");
+    config.set(DefaultImpersonationProvider.getTestProvider()
+        .getProxySuperuserGroupConfKey(currentUser), "*");
+    config.set(DefaultImpersonationProvider.getTestProvider()
+        .getProxySuperuserIpConfKey(currentUser), "*");
+    fsHelper = new FileSystemTestHelper();
+    // Set up java key store
+    String testRoot = fsHelper.getTestRootDir();
+    testRootDir = new File(testRoot).getAbsoluteFile();
+    final Path jksPath = new Path(testRootDir.toString(), "test.jks");
+    config.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_KEY_PROVIDER_PATH,
+        JavaKeyStoreProvider.SCHEME_NAME + "://file" + jksPath.toUri());
+    ProxyUsers.refreshSuperUserGroupsConfiguration(config);
+
+    cluster =
+        new MiniDFSCluster.Builder(config).nnTopology(
+            MiniDFSNNTopology.simpleFederatedTopology(2))
+            .numDataNodes(2)
+            .build();
+    cluster.waitActive();
+    hdfs1 = cluster.getFileSystem(0);
+    hdfs2 = cluster.getFileSystem(1);
+
+    nn1 = cluster.getNameNode(0);
+    nn2 = cluster.getNameNode(1);
+    nn2.getServiceRpcAddress();
+    dfsAdmin1 = new HdfsAdmin(cluster.getURI(0), config);
+    dfsAdmin2 = new HdfsAdmin(cluster.getURI(1), config);
+
+    // Use ephemeral ports in case tests are running in parallel
+    config.setInt("nfs3.mountd.port", 0);
+    config.setInt("nfs3.server.port", 0);
+    config.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY,
+        FsConstants.VIEWFS_URI.toString());
+    // Start NFS with allowed.hosts set to "* rw"
+    config.set("dfs.nfs.exports.allowed.hosts", "* rw");
+
+    Path base1 = new Path("/user1");
+    Path base2 = new Path("/user2");
+    hdfs1.delete(base1, true);
+    hdfs2.delete(base2, true);
+    hdfs1.mkdirs(base1);
+    hdfs2.mkdirs(base2);
+    ConfigUtil.addLink(config, "/hdfs1", hdfs1.makeQualified(base1).toUri());
+    ConfigUtil.addLink(config, "/hdfs2", hdfs2.makeQualified(base2).toUri());
+
+
+    viewFs = FileSystem.get(config);
+    config.setStrings(NfsConfigKeys.DFS_NFS_EXPORT_POINT_KEY,
+        "/hdfs1", "/hdfs2");
+
+    nfs = new Nfs3(config);
+    nfs.startServiceInternal(false);
+    nfsd = (RpcProgramNfs3) nfs.getRpcProgram();
+    mountd = (RpcProgramMountd) nfs.getMountd().getRpcProgram();
+
+    // Mock SecurityHandler which returns system user.name
+    securityHandler = Mockito.mock(SecurityHandler.class);
+    Mockito.when(securityHandler.getUser()).thenReturn(currentUser);
+    viewFs.delete(new Path("/hdfs2/dir2"), true);
+    viewFs.mkdirs(new Path("/hdfs2/dir2"));
+    DFSTestUtil.createFile(viewFs, new Path("/hdfs1/file1"), 0, (short) 1, 0);
+    DFSTestUtil.createFile(viewFs, new Path("/hdfs1/file2"), 0, (short) 1, 0);
+    DFSTestUtil.createFile(viewFs, new Path("/hdfs1/write1"), 0, (short) 1, 0);
+    DFSTestUtil.createFile(viewFs, new Path("/hdfs2/write2"), 0, (short) 1, 0);
+    DFSTestUtil.createFile(viewFs, new Path("/hdfs1/renameMultiNN"),
+        0, (short) 1, 0);
+    DFSTestUtil.createFile(viewFs, new Path("/hdfs1/renameSingleNN"),
+        0, (short) 1, 0);
+  }
+
+  @AfterClass
+  public static void shutdown() throws Exception {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+
+  @Test
+  public void testNumExports() throws Exception {
+    Assert.assertEquals(mountd.getExports().size(),
+        viewFs.getChildFileSystems().length);
+  }
+
+  @Test
+  public void testPaths() throws Exception {
+    Assert.assertEquals(hdfs1.resolvePath(new Path("/user1/file1")),
+        viewFs.resolvePath(new Path("/hdfs1/file1")));
+    Assert.assertEquals(hdfs1.resolvePath(new Path("/user1/file2")),
+        viewFs.resolvePath(new Path("/hdfs1/file2")));
+    Assert.assertEquals(hdfs2.resolvePath(new Path("/user2/dir2")),
+        viewFs.resolvePath(new Path("/hdfs2/dir2")));
+  }
+
+  @Test
+  public void testFileStatus() throws Exception {
+    HdfsFileStatus status = nn1.getRpcServer().getFileInfo("/user1/file1");
+    FileStatus st = viewFs.getFileStatus(new Path("/hdfs1/file1"));
+    Assert.assertEquals(st.isDirectory(), status.isDirectory());
+
+    HdfsFileStatus status2 = nn2.getRpcServer().getFileInfo("/user2/dir2");
+    FileStatus st2 = viewFs.getFileStatus(new Path("/hdfs2/dir2"));
+    Assert.assertEquals(st2.isDirectory(), status2.isDirectory());
+  }
+
+  // Test for getattr
+  private void testNfsGetAttrResponse(long fileId, int namenodeId,
+                                      int expectedStatus) {
+    FileHandle handle = new FileHandle(fileId, namenodeId);
+    XDR xdrReq = new XDR();
+    GETATTR3Request req = new GETATTR3Request(handle);
+    req.serialize(xdrReq);
+    GETATTR3Response response = nfsd.getattr(xdrReq.asReadOnlyWrap(),
+        securityHandler, new InetSocketAddress("localhost", 1234));
+    Assert.assertEquals("Incorrect return code",
+        expectedStatus, response.getStatus());
+  }
+
+  @Test (timeout = 60000)
+  public void testNfsAccessNN1() throws Exception {
+    HdfsFileStatus status = nn1.getRpcServer().getFileInfo("/user1/file1");
+    int namenodeId = Nfs3Utils.getNamenodeId(config, hdfs1.getUri());
+    testNfsGetAttrResponse(status.getFileId(), namenodeId, Nfs3Status.NFS3_OK);
+  }
+
+  @Test (timeout = 60000)
+  public void testNfsAccessNN2() throws Exception {
+    HdfsFileStatus status = nn2.getRpcServer().getFileInfo("/user2/dir2");
+    int namenodeId = Nfs3Utils.getNamenodeId(config, hdfs2.getUri());
+    testNfsGetAttrResponse(status.getFileId(), namenodeId, Nfs3Status.NFS3_OK);
+  }
+
+  @Test (timeout = 60000)
+  public void testWrongNfsAccess() throws Exception {
+    DFSTestUtil.createFile(viewFs, new Path("/hdfs1/file3"), 0, (short) 1, 0);
+    HdfsFileStatus status = nn1.getRpcServer().getFileInfo("/user1/file3");
+    int namenodeId = Nfs3Utils.getNamenodeId(config, hdfs2.getUri());
+    testNfsGetAttrResponse(status.getFileId(), namenodeId,
+        Nfs3Status.NFS3ERR_IO);
+  }
+
+  // Test for write
+  private void testNfsWriteResponse(long dirId, int namenodeId)
+      throws Exception {
+    FileHandle handle = new FileHandle(dirId, namenodeId);
+
+    byte[] buffer = new byte[10];
+    for (int i = 0; i < 10; i++) {
+      buffer[i] = (byte) i;
+    }
+
+    WRITE3Request writeReq = new WRITE3Request(handle, 0, 10,
+        Nfs3Constant.WriteStableHow.DATA_SYNC, ByteBuffer.wrap(buffer));
+    XDR xdrReq = new XDR();
+    writeReq.serialize(xdrReq);
+
+    // Attempt by a priviledged user should pass.
+    WRITE3Response response = nfsd.write(xdrReq.asReadOnlyWrap(),
+        null, 1, securityHandler,
+        new InetSocketAddress("localhost", 1234));
+    Assert.assertEquals("Incorrect response:", null, response);
+  }
+
+  @Test (timeout = 60000)
+  public void testNfsWriteNN1() throws Exception {
+    HdfsFileStatus status = nn1.getRpcServer().getFileInfo("/user1/write1");
+    int namenodeId = Nfs3Utils.getNamenodeId(config, hdfs1.getUri());
+    testNfsWriteResponse(status.getFileId(), namenodeId);
+  }
+
+  @Test (timeout = 60000)
+  public void testNfsWriteNN2() throws Exception {
+    HdfsFileStatus status = nn2.getRpcServer().getFileInfo("/user2/write2");
+    int namenodeId = Nfs3Utils.getNamenodeId(config, hdfs2.getUri());
+    testNfsWriteResponse(status.getFileId(), namenodeId);
+  }
+
+  // Test for rename
+  private void testNfsRename(FileHandle fromDirHandle, String fromFileName,
+                            FileHandle toDirHandle, String toFileName,
+                            int expectedStatus) throws Exception {
+    XDR xdrReq = new XDR();
+    RENAME3Request req = new RENAME3Request(fromDirHandle, fromFileName,
+        toDirHandle, toFileName);
+    req.serialize(xdrReq);
+
+    // Attempt by a privileged user should pass.
+    RENAME3Response response = nfsd.rename(xdrReq.asReadOnlyWrap(),
+        securityHandler, new InetSocketAddress("localhost", 1234));
+    assertEquals(expectedStatus, response.getStatus());
+  }
+
+  @Test (timeout = 60000)
+  public void testNfsRenameMultiNN() throws Exception {
+    HdfsFileStatus fromFileStatus = nn1.getRpcServer().getFileInfo("/user1");
+    int fromNNId = Nfs3Utils.getNamenodeId(config, hdfs1.getUri());
+    FileHandle fromHandle =
+        new FileHandle(fromFileStatus.getFileId(), fromNNId);
+
+    HdfsFileStatus toFileStatus = nn2.getRpcServer().getFileInfo("/user2");
+    int toNNId = Nfs3Utils.getNamenodeId(config, hdfs2.getUri());
+    FileHandle toHandle = new FileHandle(toFileStatus.getFileId(), toNNId);
+
+    HdfsFileStatus statusBeforeRename =
+        nn1.getRpcServer().getFileInfo("/user1/renameMultiNN");
+    Assert.assertEquals(statusBeforeRename.isDirectory(), false);
+
+    testNfsRename(fromHandle, "renameMultiNN",
+        toHandle, "renameMultiNNFail", Nfs3Status.NFS3ERR_INVAL);
+
+    HdfsFileStatus statusAfterRename =
+        nn2.getRpcServer().getFileInfo("/user2/renameMultiNNFail");
+    Assert.assertEquals(statusAfterRename, null);
+
+    statusAfterRename = nn1.getRpcServer().getFileInfo("/user1/renameMultiNN");
+    Assert.assertEquals(statusAfterRename.isDirectory(), false);
+  }
+
+  @Test (timeout = 60000)
+  public void testNfsRenameSingleNN() throws Exception {
+    HdfsFileStatus fromFileStatus = nn1.getRpcServer().getFileInfo("/user1");
+    int fromNNId = Nfs3Utils.getNamenodeId(config, hdfs1.getUri());
+    FileHandle fromHandle =
+        new FileHandle(fromFileStatus.getFileId(), fromNNId);
+
+    HdfsFileStatus statusBeforeRename =
+        nn1.getRpcServer().getFileInfo("/user1/renameSingleNN");
+    Assert.assertEquals(statusBeforeRename.isDirectory(), false);
+
+    testNfsRename(fromHandle, "renameSingleNN",
+        fromHandle, "renameSingleNNSucess", Nfs3Status.NFS3_OK);
+
+    HdfsFileStatus statusAfterRename =
+        nn1.getRpcServer().getFileInfo("/user1/renameSingleNNSucess");
+    Assert.assertEquals(statusAfterRename.isDirectory(), false);
+
+    statusAfterRename =
+        nn1.getRpcServer().getFileInfo("/user1/renameSingleNN");
+    Assert.assertEquals(statusAfterRename, null);
+  }
+}

+ 6 - 3
hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/nfs3/TestWrites.java

@@ -481,6 +481,7 @@ public class TestWrites {
       cluster = new MiniDFSCluster.Builder(config).numDataNodes(1).build();
       cluster.waitActive();
       client = new DFSClient(DFSUtilClient.getNNAddress(config), config);
+      int namenodeId = Nfs3Utils.getNamenodeId(config);
 
       // Use emphral port in case tests are running in parallel
       config.setInt("nfs3.mountd.port", 0);
@@ -492,7 +493,7 @@ public class TestWrites {
       nfsd = (RpcProgramNfs3) nfs3.getRpcProgram();
 
       HdfsFileStatus status = client.getFileInfo("/");
-      FileHandle rootHandle = new FileHandle(status.getFileId());
+      FileHandle rootHandle = new FileHandle(status.getFileId(), namenodeId);
       // Create file1
       CREATE3Request createReq = new CREATE3Request(rootHandle, "file1",
           Nfs3Constant.CREATE_UNCHECKED, new SetAttr3(), 0);
@@ -598,8 +599,9 @@ public class TestWrites {
 
       DFSClient dfsClient = new DFSClient(DFSUtilClient.getNNAddress(config),
           config);
+      int namenodeId = Nfs3Utils.getNamenodeId(config);
       HdfsFileStatus status = dfsClient.getFileInfo("/");
-      FileHandle rootHandle = new FileHandle(status.getFileId());
+      FileHandle rootHandle = new FileHandle(status.getFileId(), namenodeId);
 
       CREATE3Request createReq = new CREATE3Request(rootHandle,
           "out-of-order-write" + System.currentTimeMillis(),
@@ -674,8 +676,9 @@ public class TestWrites {
 
       DFSClient dfsClient = new DFSClient(DFSUtilClient.getNNAddress(config),
           config);
+      int namenodeId = Nfs3Utils.getNamenodeId(config);
       HdfsFileStatus status = dfsClient.getFileInfo("/");
-      FileHandle rootHandle = new FileHandle(status.getFileId());
+      FileHandle rootHandle = new FileHandle(status.getFileId(), namenodeId);
 
       CREATE3Request createReq = new CREATE3Request(rootHandle,
           "overlapping-writes" + System.currentTimeMillis(),

+ 3 - 0
hadoop-hdfs-project/hadoop-hdfs/dev-support/findbugsExcludeFile.xml

@@ -23,6 +23,9 @@
      <Match>
        <Package name ="org.apache.hadoop.cblock.protocol.proto" />
      </Match>
+     <Match>
+       <Package name="org.apache.hadoop.hdfs.federation.protocol.proto" />
+     </Match>
      <Match>
        <Bug pattern="EI_EXPOSE_REP" />
      </Match>

+ 10 - 0
hadoop-hdfs-project/hadoop-hdfs/pom.xml

@@ -225,6 +225,11 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
       <artifactId>sqlite-jdbc</artifactId>
       <version>3.8.7</version>
     </dependency>
+    <dependency>
+      <groupId>org.apache.curator</groupId>
+      <artifactId>curator-test</artifactId>
+      <scope>test</scope>
+    </dependency>
   </dependencies>
 
   <build>
@@ -276,6 +281,9 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
                 <copy file="${basedir}/src/main/webapps/proto-web.xml"
                       tofile="${project.build.directory}/webapps/nfs3/WEB-INF/web.xml"
                       filtering="true"/>
+                <copy file="${basedir}/src/main/webapps/proto-web.xml"
+                      tofile="${project.build.directory}/webapps/router/WEB-INF/web.xml"
+                      filtering="true"/>
                 <copy toDir="${project.build.directory}/webapps">
                   <fileset dir="${basedir}/src/main/webapps">
                     <exclude name="**/proto-web.xml"/>
@@ -357,6 +365,8 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
                   <include>StorageContainerDatanodeProtocol.proto</include>
                   <include>CBlockServiceProtocol.proto</include>
                   <include>CBlockClientServerProtocol.proto</include>
+                  <include>FederationProtocol.proto</include>
+                  <include>RouterProtocol.proto</include>
                 </includes>
               </source>
             </configuration>

+ 9 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs

@@ -43,6 +43,8 @@ function hadoop_usage
   hadoop_add_subcommand "debug" admin "run a Debug Admin to execute HDFS debug commands"
   hadoop_add_subcommand "dfs" client "run a filesystem command on the file system"
   hadoop_add_subcommand "dfsadmin" admin "run a DFS admin client"
+  hadoop_add_subcommand "dfsrouter" daemon "run the DFS router"
+  hadoop_add_subcommand "dfsrouteradmin" admin "manage Router-based federation"
   hadoop_add_subcommand "diskbalancer" daemon "Distributes data evenly among disks on a given node"
   hadoop_add_subcommand "envvars" client "display computed Hadoop environment variables"
   hadoop_add_subcommand "ec" admin "run a HDFS ErasureCoding CLI"
@@ -127,6 +129,13 @@ function hdfscmd_case
     dfsadmin)
       HADOOP_CLASSNAME=org.apache.hadoop.hdfs.tools.DFSAdmin
     ;;
+    dfsrouter)
+      HADOOP_SUBCMD_SUPPORTDAEMONIZATION="true"
+      HADOOP_CLASSNAME='org.apache.hadoop.hdfs.server.federation.router.DFSRouter'
+    ;;
+    dfsrouteradmin)
+      HADOOP_CLASSNAME='org.apache.hadoop.hdfs.tools.federation.RouterAdmin'
+    ;;
     diskbalancer)
       HADOOP_CLASSNAME=org.apache.hadoop.hdfs.tools.DiskBalancerCLI
     ;;

+ 13 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs.cmd

@@ -59,7 +59,7 @@ if "%1" == "--loglevel" (
     )
   )
 
-  set hdfscommands=dfs namenode secondarynamenode journalnode zkfc datanode dfsadmin haadmin fsck balancer jmxget oiv oev fetchdt getconf groups snapshotDiff lsSnapshottableDir cacheadmin mover storagepolicies classpath crypto debug
+  set hdfscommands=dfs namenode secondarynamenode journalnode zkfc datanode dfsadmin haadmin fsck balancer jmxget oiv oev fetchdt getconf groups snapshotDiff lsSnapshottableDir cacheadmin mover storagepolicies classpath crypto dfsrouter dfsrouteradmin debug
   for %%i in ( %hdfscommands% ) do (
     if %hdfs-command% == %%i set hdfscommand=true
   )
@@ -179,6 +179,16 @@ goto :eof
   set CLASS=org.apache.hadoop.hdfs.tools.CryptoAdmin
   goto :eof
 
+:dfsrouter
+  set CLASS=org.apache.hadoop.hdfs.server.federation.router.DFSRouter
+  set HADOOP_OPTS=%HADOOP_OPTS% %HADOOP_ROUTER_OPTS%
+  goto :eof
+
+:dfsrouteradmin
+  set CLASS=org.apache.hadoop.hdfs.tools.federation.RouterAdmin
+  set HADOOP_OPTS=%HADOOP_OPTS% %HADOOP_ROUTER_OPTS%
+  goto :eof
+
 :debug
   set CLASS=org.apache.hadoop.hdfs.tools.DebugAdmin
   goto :eof
@@ -219,6 +229,8 @@ goto :eof
   @echo   secondarynamenode    run the DFS secondary namenode
   @echo   namenode             run the DFS namenode
   @echo   journalnode          run the DFS journalnode
+  @echo   dfsrouter            run the DFS router
+  @echo   dfsrouteradmin       manage Router-based federation
   @echo   zkfc                 run the ZK Failover Controller daemon
   @echo   datanode             run a DFS datanode
   @echo   dfsadmin             run a DFS admin client

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

@@ -18,6 +18,8 @@
 
 package org.apache.hadoop.hdfs;
 
+import java.util.concurrent.TimeUnit;
+
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
@@ -25,6 +27,15 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicyDefault;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicyRackFaultTolerant;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.RamDiskReplicaLruTracker;
+import org.apache.hadoop.hdfs.server.federation.resolver.FileSubclusterResolver;
+import org.apache.hadoop.hdfs.server.federation.resolver.MountTableResolver;
+import org.apache.hadoop.hdfs.server.federation.router.RouterRpcMonitor;
+import org.apache.hadoop.hdfs.server.federation.metrics.FederationRPCPerformanceMonitor;
+import org.apache.hadoop.hdfs.server.federation.resolver.ActiveNamenodeResolver;
+import org.apache.hadoop.hdfs.server.federation.resolver.MembershipNamenodeResolver;
+import org.apache.hadoop.hdfs.server.federation.store.driver.StateStoreDriver;
+import org.apache.hadoop.hdfs.server.federation.store.driver.impl.StateStoreFileImpl;
+import org.apache.hadoop.hdfs.server.federation.store.driver.impl.StateStoreSerializerPBImpl;
 import org.apache.hadoop.http.HttpConfig;
 
 /** 
@@ -270,6 +281,9 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
       "dfs.namenode.posix.acl.inheritance.enabled";
   public static final boolean
       DFS_NAMENODE_POSIX_ACL_INHERITANCE_ENABLED_DEFAULT = true;
+  public static final String DFS_REFORMAT_DISABLED = "dfs.reformat.disabled";
+  public static final boolean DFS_REFORMAT_DISABLED_DEFAULT = false;
+
   public static final String  DFS_NAMENODE_XATTRS_ENABLED_KEY = "dfs.namenode.xattrs.enabled";
   public static final boolean DFS_NAMENODE_XATTRS_ENABLED_DEFAULT = true;
   public static final String  DFS_ADMIN = "dfs.cluster.administrators";
@@ -1109,6 +1123,154 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
       "dfs.use.dfs.network.topology";
   public static final boolean DFS_USE_DFS_NETWORK_TOPOLOGY_DEFAULT = true;
 
+  // HDFS federation
+  public static final String FEDERATION_PREFIX = "dfs.federation.";
+
+  // HDFS Router-based federation
+  public static final String FEDERATION_ROUTER_PREFIX =
+      "dfs.federation.router.";
+  public static final String DFS_ROUTER_DEFAULT_NAMESERVICE =
+      FEDERATION_ROUTER_PREFIX + "default.nameserviceId";
+  public static final String DFS_ROUTER_HANDLER_COUNT_KEY =
+      FEDERATION_ROUTER_PREFIX + "handler.count";
+  public static final int DFS_ROUTER_HANDLER_COUNT_DEFAULT = 10;
+  public static final String DFS_ROUTER_READER_QUEUE_SIZE_KEY =
+      FEDERATION_ROUTER_PREFIX + "reader.queue.size";
+  public static final int DFS_ROUTER_READER_QUEUE_SIZE_DEFAULT = 100;
+  public static final String DFS_ROUTER_READER_COUNT_KEY =
+      FEDERATION_ROUTER_PREFIX + "reader.count";
+  public static final int DFS_ROUTER_READER_COUNT_DEFAULT = 1;
+  public static final String DFS_ROUTER_HANDLER_QUEUE_SIZE_KEY =
+      FEDERATION_ROUTER_PREFIX + "handler.queue.size";
+  public static final int DFS_ROUTER_HANDLER_QUEUE_SIZE_DEFAULT = 100;
+  public static final String DFS_ROUTER_RPC_BIND_HOST_KEY =
+      FEDERATION_ROUTER_PREFIX + "rpc-bind-host";
+  public static final int DFS_ROUTER_RPC_PORT_DEFAULT = 8888;
+  public static final String DFS_ROUTER_RPC_ADDRESS_KEY =
+      FEDERATION_ROUTER_PREFIX + "rpc-address";
+  public static final String DFS_ROUTER_RPC_ADDRESS_DEFAULT =
+      "0.0.0.0:" + DFS_ROUTER_RPC_PORT_DEFAULT;
+  public static final String DFS_ROUTER_RPC_ENABLE =
+      FEDERATION_ROUTER_PREFIX + "rpc.enable";
+  public static final boolean DFS_ROUTER_RPC_ENABLE_DEFAULT = true;
+
+  public static final String DFS_ROUTER_METRICS_ENABLE =
+      FEDERATION_ROUTER_PREFIX + "metrics.enable";
+  public static final boolean DFS_ROUTER_METRICS_ENABLE_DEFAULT = true;
+  public static final String DFS_ROUTER_METRICS_CLASS =
+      FEDERATION_ROUTER_PREFIX + "metrics.class";
+  public static final Class<? extends RouterRpcMonitor>
+      DFS_ROUTER_METRICS_CLASS_DEFAULT =
+          FederationRPCPerformanceMonitor.class;
+
+  // HDFS Router heartbeat
+  public static final String DFS_ROUTER_HEARTBEAT_ENABLE =
+      FEDERATION_ROUTER_PREFIX + "heartbeat.enable";
+  public static final boolean DFS_ROUTER_HEARTBEAT_ENABLE_DEFAULT = true;
+  public static final String DFS_ROUTER_HEARTBEAT_INTERVAL_MS =
+      FEDERATION_ROUTER_PREFIX + "heartbeat.interval";
+  public static final long DFS_ROUTER_HEARTBEAT_INTERVAL_MS_DEFAULT =
+      TimeUnit.SECONDS.toMillis(5);
+  public static final String DFS_ROUTER_MONITOR_NAMENODE =
+      FEDERATION_ROUTER_PREFIX + "monitor.namenode";
+  public static final String DFS_ROUTER_MONITOR_LOCAL_NAMENODE =
+      FEDERATION_ROUTER_PREFIX + "monitor.localnamenode.enable";
+  public static final boolean DFS_ROUTER_MONITOR_LOCAL_NAMENODE_DEFAULT = true;
+
+  // HDFS Router NN client
+  public static final String DFS_ROUTER_NAMENODE_CONNECTION_POOL_SIZE =
+      FEDERATION_ROUTER_PREFIX + "connection.pool-size";
+  public static final int DFS_ROUTER_NAMENODE_CONNECTION_POOL_SIZE_DEFAULT =
+      64;
+  public static final String DFS_ROUTER_NAMENODE_CONNECTION_POOL_CLEAN =
+      FEDERATION_ROUTER_PREFIX + "connection.pool.clean.ms";
+  public static final long DFS_ROUTER_NAMENODE_CONNECTION_POOL_CLEAN_DEFAULT =
+      TimeUnit.MINUTES.toMillis(1);
+  public static final String DFS_ROUTER_NAMENODE_CONNECTION_CLEAN_MS =
+      FEDERATION_ROUTER_PREFIX + "connection.clean.ms";
+  public static final long DFS_ROUTER_NAMENODE_CONNECTION_CLEAN_MS_DEFAULT =
+      TimeUnit.SECONDS.toMillis(10);
+
+  // HDFS Router State Store connection
+  public static final String FEDERATION_FILE_RESOLVER_CLIENT_CLASS =
+      FEDERATION_ROUTER_PREFIX + "file.resolver.client.class";
+  public static final Class<? extends FileSubclusterResolver>
+      FEDERATION_FILE_RESOLVER_CLIENT_CLASS_DEFAULT =
+          MountTableResolver.class;
+  public static final String FEDERATION_NAMENODE_RESOLVER_CLIENT_CLASS =
+      FEDERATION_ROUTER_PREFIX + "namenode.resolver.client.class";
+  public static final Class<? extends ActiveNamenodeResolver>
+      FEDERATION_NAMENODE_RESOLVER_CLIENT_CLASS_DEFAULT =
+          MembershipNamenodeResolver.class;
+
+  // HDFS Router-based federation State Store
+  public static final String FEDERATION_STORE_PREFIX =
+      FEDERATION_ROUTER_PREFIX + "store.";
+
+  public static final String DFS_ROUTER_STORE_ENABLE =
+      FEDERATION_STORE_PREFIX + "enable";
+  public static final boolean DFS_ROUTER_STORE_ENABLE_DEFAULT = true;
+
+  public static final String FEDERATION_STORE_SERIALIZER_CLASS =
+      DFSConfigKeys.FEDERATION_STORE_PREFIX + "serializer";
+  public static final Class<StateStoreSerializerPBImpl>
+      FEDERATION_STORE_SERIALIZER_CLASS_DEFAULT =
+          StateStoreSerializerPBImpl.class;
+
+  public static final String FEDERATION_STORE_DRIVER_CLASS =
+      FEDERATION_STORE_PREFIX + "driver.class";
+  public static final Class<? extends StateStoreDriver>
+      FEDERATION_STORE_DRIVER_CLASS_DEFAULT = StateStoreFileImpl.class;
+
+  public static final String FEDERATION_STORE_CONNECTION_TEST_MS =
+      FEDERATION_STORE_PREFIX + "connection.test";
+  public static final long FEDERATION_STORE_CONNECTION_TEST_MS_DEFAULT =
+      TimeUnit.MINUTES.toMillis(1);
+
+  public static final String DFS_ROUTER_CACHE_TIME_TO_LIVE_MS =
+      FEDERATION_ROUTER_PREFIX + "cache.ttl";
+  public static final long DFS_ROUTER_CACHE_TIME_TO_LIVE_MS_DEFAULT =
+      TimeUnit.MINUTES.toMillis(1);
+
+  public static final String FEDERATION_STORE_MEMBERSHIP_EXPIRATION_MS =
+      FEDERATION_STORE_PREFIX + "membership.expiration";
+  public static final long FEDERATION_STORE_MEMBERSHIP_EXPIRATION_MS_DEFAULT =
+      TimeUnit.MINUTES.toMillis(5);
+
+  // HDFS Router-based federation admin
+  public static final String DFS_ROUTER_ADMIN_HANDLER_COUNT_KEY =
+      FEDERATION_ROUTER_PREFIX + "admin.handler.count";
+  public static final int DFS_ROUTER_ADMIN_HANDLER_COUNT_DEFAULT = 1;
+  public static final int    DFS_ROUTER_ADMIN_PORT_DEFAULT = 8111;
+  public static final String DFS_ROUTER_ADMIN_ADDRESS_KEY =
+      FEDERATION_ROUTER_PREFIX + "admin-address";
+  public static final String DFS_ROUTER_ADMIN_ADDRESS_DEFAULT =
+      "0.0.0.0:" + DFS_ROUTER_ADMIN_PORT_DEFAULT;
+  public static final String DFS_ROUTER_ADMIN_BIND_HOST_KEY =
+      FEDERATION_ROUTER_PREFIX + "admin-bind-host";
+  public static final String DFS_ROUTER_ADMIN_ENABLE =
+      FEDERATION_ROUTER_PREFIX + "admin.enable";
+  public static final boolean DFS_ROUTER_ADMIN_ENABLE_DEFAULT = true;
+
+  // HDFS Router-based federation web
+  public static final String DFS_ROUTER_HTTP_ENABLE =
+      FEDERATION_ROUTER_PREFIX + "http.enable";
+  public static final boolean DFS_ROUTER_HTTP_ENABLE_DEFAULT = true;
+  public static final String DFS_ROUTER_HTTP_ADDRESS_KEY =
+      FEDERATION_ROUTER_PREFIX + "http-address";
+  public static final int    DFS_ROUTER_HTTP_PORT_DEFAULT = 50071;
+  public static final String DFS_ROUTER_HTTP_BIND_HOST_KEY =
+      FEDERATION_ROUTER_PREFIX + "http-bind-host";
+  public static final String DFS_ROUTER_HTTP_ADDRESS_DEFAULT =
+      "0.0.0.0:" + DFS_ROUTER_HTTP_PORT_DEFAULT;
+  public static final String DFS_ROUTER_HTTPS_ADDRESS_KEY =
+      FEDERATION_ROUTER_PREFIX + "https-address";
+  public static final int    DFS_ROUTER_HTTPS_PORT_DEFAULT = 50072;
+  public static final String DFS_ROUTER_HTTPS_BIND_HOST_KEY =
+      FEDERATION_ROUTER_PREFIX + "https-bind-host";
+  public static final String DFS_ROUTER_HTTPS_ADDRESS_DEFAULT =
+      "0.0.0.0:" + DFS_ROUTER_HTTPS_PORT_DEFAULT;
+
   // dfs.client.retry confs are moved to HdfsClientConfigKeys.Retry 
   @Deprecated
   public static final String  DFS_CLIENT_RETRY_POLICY_ENABLED_KEY

+ 38 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java

@@ -1323,6 +1323,44 @@ public class DFSUtil {
     return serviceRpcAddr;
   }
 
+  /**
+   * Map a logical namenode ID to its web address. Use the given nameservice if
+   * specified, or the configured one if none is given.
+   *
+   * @param conf Configuration
+   * @param nsId which nameservice nnId is a part of, optional
+   * @param nnId the namenode ID to get the service addr for
+   * @return the service addr, null if it could not be determined
+   */
+  public static String getNamenodeWebAddr(final Configuration conf, String nsId,
+      String nnId) {
+
+    if (nsId == null) {
+      nsId = getOnlyNameServiceIdOrNull(conf);
+    }
+
+    String webAddrKey = DFSUtilClient.concatSuffixes(
+        DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY, nsId, nnId);
+
+    String webAddr =
+        conf.get(webAddrKey, DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_DEFAULT);
+    return webAddr;
+  }
+
+  /**
+   * Get all of the Web addresses of the individual NNs in a given nameservice.
+   *
+   * @param conf Configuration
+   * @param nsId the nameservice whose NNs addresses we want.
+   * @param defaultValue default address to return in case key is not found.
+   * @return A map from nnId -> Web address of each NN in the nameservice.
+   */
+  public static Map<String, InetSocketAddress> getWebAddressesForNameserviceId(
+      Configuration conf, String nsId, String defaultValue) {
+    return DFSUtilClient.getAddressesForNameserviceId(conf, nsId, defaultValue,
+        DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY);
+  }
+
   /**
    * If the configuration refers to only a single nameservice, return the
    * name of that nameservice. If it refers to 0 or more than 1, return null.

+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java

@@ -948,6 +948,7 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
   }
 
   @Override
+
   public GetFileLinkInfoResponseProto getFileLinkInfo(RpcController controller,
       GetFileLinkInfoRequestProto req) throws ServiceException {
     try {

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

@@ -0,0 +1,44 @@
+/**
+ * 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.protocolPB;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.protocol.proto.RouterProtocolProtos.RouterAdminProtocolService;
+import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSelector;
+import org.apache.hadoop.ipc.ProtocolInfo;
+import org.apache.hadoop.security.KerberosInfo;
+import org.apache.hadoop.security.token.TokenInfo;
+
+/**
+ * Protocol that a clients use to communicate with the NameNode.
+ * Note: This extends the protocolbuffer service based interface to
+ * add annotations required for security.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Stable
+@KerberosInfo(
+    serverPrincipal = DFSConfigKeys.DFS_NAMENODE_KERBEROS_PRINCIPAL_KEY)
+@TokenInfo(DelegationTokenSelector.class)
+@ProtocolInfo(protocolName = HdfsConstants.CLIENT_NAMENODE_PROTOCOL_NAME,
+    protocolVersion = 1)
+public interface RouterAdminProtocolPB extends
+    RouterAdminProtocolService.BlockingInterface {
+}

+ 151 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RouterAdminProtocolServerSideTranslatorPB.java

@@ -0,0 +1,151 @@
+/**
+ * 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.protocolPB;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.AddMountTableEntryRequestProto;
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.AddMountTableEntryResponseProto;
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.GetMountTableEntriesRequestProto;
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.GetMountTableEntriesResponseProto;
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.RemoveMountTableEntryRequestProto;
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.RemoveMountTableEntryResponseProto;
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.UpdateMountTableEntryRequestProto;
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.UpdateMountTableEntryResponseProto;
+import org.apache.hadoop.hdfs.server.federation.router.RouterAdminServer;
+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.protocol.GetMountTableEntriesRequest;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.GetMountTableEntriesResponse;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.RemoveMountTableEntryRequest;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.RemoveMountTableEntryResponse;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.UpdateMountTableEntryRequest;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.UpdateMountTableEntryResponse;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.impl.pb.AddMountTableEntryRequestPBImpl;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.impl.pb.AddMountTableEntryResponsePBImpl;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.impl.pb.GetMountTableEntriesRequestPBImpl;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.impl.pb.GetMountTableEntriesResponsePBImpl;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.impl.pb.RemoveMountTableEntryRequestPBImpl;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.impl.pb.RemoveMountTableEntryResponsePBImpl;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.impl.pb.UpdateMountTableEntryRequestPBImpl;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.impl.pb.UpdateMountTableEntryResponsePBImpl;
+
+import com.google.protobuf.RpcController;
+import com.google.protobuf.ServiceException;
+
+/**
+ * This class is used on the server side. Calls come across the wire for the for
+ * protocol {@link RouterAdminProtocolPB}. This class translates the PB data
+ * types to the native data types used inside the HDFS Router as specified in
+ * the generic RouterAdminProtocol.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Stable
+public class RouterAdminProtocolServerSideTranslatorPB implements
+    RouterAdminProtocolPB {
+
+  private final RouterAdminServer server;
+
+  /**
+   * Constructor.
+   * @param server The NN server.
+   * @throws IOException
+   */
+  public RouterAdminProtocolServerSideTranslatorPB(RouterAdminServer server)
+      throws IOException {
+    this.server = server;
+  }
+
+  @Override
+  public AddMountTableEntryResponseProto addMountTableEntry(
+      RpcController controller, AddMountTableEntryRequestProto request)
+      throws ServiceException {
+
+    try {
+      AddMountTableEntryRequest req =
+          new AddMountTableEntryRequestPBImpl(request);
+      AddMountTableEntryResponse response = server.addMountTableEntry(req);
+      AddMountTableEntryResponsePBImpl responsePB =
+          (AddMountTableEntryResponsePBImpl)response;
+      return responsePB.getProto();
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+
+  /**
+   * Remove an entry from the mount table.
+   */
+  @Override
+  public RemoveMountTableEntryResponseProto removeMountTableEntry(
+      RpcController controller, RemoveMountTableEntryRequestProto request)
+      throws ServiceException {
+    try {
+      RemoveMountTableEntryRequest req =
+          new RemoveMountTableEntryRequestPBImpl(request);
+      RemoveMountTableEntryResponse response =
+          server.removeMountTableEntry(req);
+      RemoveMountTableEntryResponsePBImpl responsePB =
+          (RemoveMountTableEntryResponsePBImpl)response;
+      return responsePB.getProto();
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+
+  /**
+   * Get matching mount table entries.
+   */
+  @Override
+  public GetMountTableEntriesResponseProto getMountTableEntries(
+      RpcController controller, GetMountTableEntriesRequestProto request)
+          throws ServiceException {
+    try {
+      GetMountTableEntriesRequest req =
+          new GetMountTableEntriesRequestPBImpl(request);
+      GetMountTableEntriesResponse response = server.getMountTableEntries(req);
+      GetMountTableEntriesResponsePBImpl responsePB =
+          (GetMountTableEntriesResponsePBImpl)response;
+      return responsePB.getProto();
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+
+  /**
+   * Update a single mount table entry.
+   */
+  @Override
+  public UpdateMountTableEntryResponseProto updateMountTableEntry(
+      RpcController controller, UpdateMountTableEntryRequestProto request)
+          throws ServiceException {
+    try {
+      UpdateMountTableEntryRequest req =
+          new UpdateMountTableEntryRequestPBImpl(request);
+      UpdateMountTableEntryResponse response =
+          server.updateMountTableEntry(req);
+      UpdateMountTableEntryResponsePBImpl responsePB =
+          (UpdateMountTableEntryResponsePBImpl)response;
+      return responsePB.getProto();
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+}

+ 150 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RouterAdminProtocolTranslatorPB.java

@@ -0,0 +1,150 @@
+/**
+ * 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.protocolPB;
+
+import java.io.Closeable;
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.AddMountTableEntryRequestProto;
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.AddMountTableEntryResponseProto;
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.GetMountTableEntriesRequestProto;
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.GetMountTableEntriesResponseProto;
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.RemoveMountTableEntryRequestProto;
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.RemoveMountTableEntryResponseProto;
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.UpdateMountTableEntryRequestProto;
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.UpdateMountTableEntryResponseProto;
+import org.apache.hadoop.hdfs.server.federation.resolver.MountTableManager;
+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.protocol.GetMountTableEntriesRequest;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.GetMountTableEntriesResponse;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.RemoveMountTableEntryRequest;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.RemoveMountTableEntryResponse;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.UpdateMountTableEntryRequest;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.UpdateMountTableEntryResponse;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.impl.pb.AddMountTableEntryRequestPBImpl;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.impl.pb.AddMountTableEntryResponsePBImpl;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.impl.pb.GetMountTableEntriesRequestPBImpl;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.impl.pb.GetMountTableEntriesResponsePBImpl;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.impl.pb.RemoveMountTableEntryRequestPBImpl;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.impl.pb.RemoveMountTableEntryResponsePBImpl;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.impl.pb.UpdateMountTableEntryRequestPBImpl;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.impl.pb.UpdateMountTableEntryResponsePBImpl;
+import org.apache.hadoop.ipc.ProtobufHelper;
+import org.apache.hadoop.ipc.ProtocolMetaInterface;
+import org.apache.hadoop.ipc.ProtocolTranslator;
+import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.ipc.RpcClientUtil;
+
+import com.google.protobuf.ServiceException;
+
+/**
+ * This class forwards NN's ClientProtocol calls as RPC calls to the NN server
+ * while translating from the parameter types used in ClientProtocol to the
+ * new PB types.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Stable
+public class RouterAdminProtocolTranslatorPB
+    implements ProtocolMetaInterface, MountTableManager,
+    Closeable, ProtocolTranslator {
+  final private RouterAdminProtocolPB rpcProxy;
+
+  public RouterAdminProtocolTranslatorPB(RouterAdminProtocolPB proxy) {
+    rpcProxy = proxy;
+  }
+
+  @Override
+  public void close() {
+    RPC.stopProxy(rpcProxy);
+  }
+
+  @Override
+  public Object getUnderlyingProxyObject() {
+    return rpcProxy;
+  }
+
+  @Override
+  public boolean isMethodSupported(String methodName) throws IOException {
+    return RpcClientUtil.isMethodSupported(rpcProxy,
+        RouterAdminProtocolPB.class, RPC.RpcKind.RPC_PROTOCOL_BUFFER,
+        RPC.getProtocolVersion(RouterAdminProtocolPB.class), methodName);
+  }
+
+  @Override
+  public AddMountTableEntryResponse addMountTableEntry(
+      AddMountTableEntryRequest request) throws IOException {
+    AddMountTableEntryRequestPBImpl requestPB =
+        (AddMountTableEntryRequestPBImpl)request;
+    AddMountTableEntryRequestProto proto = requestPB.getProto();
+    try {
+      AddMountTableEntryResponseProto response =
+          rpcProxy.addMountTableEntry(null, proto);
+      return new AddMountTableEntryResponsePBImpl(response);
+    } catch (ServiceException e) {
+      throw new IOException(ProtobufHelper.getRemoteException(e).getMessage());
+    }
+  }
+
+  @Override
+  public UpdateMountTableEntryResponse updateMountTableEntry(
+      UpdateMountTableEntryRequest request) throws IOException {
+    UpdateMountTableEntryRequestPBImpl requestPB =
+        (UpdateMountTableEntryRequestPBImpl)request;
+    UpdateMountTableEntryRequestProto proto = requestPB.getProto();
+    try {
+      UpdateMountTableEntryResponseProto response =
+          rpcProxy.updateMountTableEntry(null, proto);
+      return new UpdateMountTableEntryResponsePBImpl(response);
+    } catch (ServiceException e) {
+      throw new IOException(ProtobufHelper.getRemoteException(e).getMessage());
+    }
+  }
+
+  @Override
+  public RemoveMountTableEntryResponse removeMountTableEntry(
+      RemoveMountTableEntryRequest request) throws IOException {
+    RemoveMountTableEntryRequestPBImpl requestPB =
+        (RemoveMountTableEntryRequestPBImpl)request;
+    RemoveMountTableEntryRequestProto proto = requestPB.getProto();
+    try {
+      RemoveMountTableEntryResponseProto responseProto =
+          rpcProxy.removeMountTableEntry(null, proto);
+      return new RemoveMountTableEntryResponsePBImpl(responseProto);
+    } catch (ServiceException e) {
+      throw new IOException(ProtobufHelper.getRemoteException(e).getMessage());
+    }
+  }
+
+  @Override
+  public GetMountTableEntriesResponse getMountTableEntries(
+      GetMountTableEntriesRequest request) throws IOException {
+    GetMountTableEntriesRequestPBImpl requestPB =
+        (GetMountTableEntriesRequestPBImpl)request;
+    GetMountTableEntriesRequestProto proto = requestPB.getProto();
+    try {
+      GetMountTableEntriesResponseProto response =
+          rpcProxy.getMountTableEntries(null, proto);
+      return new GetMountTableEntriesResponsePBImpl(response);
+    } catch (ServiceException e) {
+      throw new IOException(ProtobufHelper.getRemoteException(e).getMessage());
+    }
+  }
+}

+ 10 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java

@@ -256,6 +256,11 @@ public class BlockManager implements BlockStatsMXBean {
     return invalidateBlocks.getBlocks();
   }
 
+  /** Used by metrics. */
+  public long getTotalReplicatedBlocks() {
+    return blocksMap.getReplicatedBlocks();
+  }
+
   /** Used by metrics. */
   public long getLowRedundancyECBlockGroups() {
     return neededReconstruction.getLowRedundancyECBlockGroups();
@@ -276,6 +281,11 @@ public class BlockManager implements BlockStatsMXBean {
     return invalidateBlocks.getECBlocks();
   }
 
+  /** Used by metrics. */
+  public long getTotalECBlockGroups() {
+    return blocksMap.getECBlockGroups();
+  }
+
   /**
    * redundancyRecheckInterval is how often namenode checks for new
    * reconstruction work.

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicy.java

@@ -43,7 +43,7 @@ import org.slf4j.LoggerFactory;
  */
 @InterfaceAudience.Private
 public abstract class BlockPlacementPolicy {
-  static final Logger LOG = LoggerFactory.getLogger(
+  public static final Logger LOG = LoggerFactory.getLogger(
       BlockPlacementPolicy.class);
 
   @InterfaceAudience.Private

+ 38 - 11
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyRackFaultTolerant.java

@@ -46,9 +46,12 @@ public class BlockPlacementPolicyRackFaultTolerant extends BlockPlacementPolicyD
     if (numOfRacks == 1 || totalNumOfReplicas <= 1) {
       return new int[] {numOfReplicas, totalNumOfReplicas};
     }
-    if(totalNumOfReplicas<numOfRacks){
+    // If more racks than replicas, put one replica per rack.
+    if (totalNumOfReplicas < numOfRacks) {
       return new int[] {numOfReplicas, 1};
     }
+    // If more replicas than racks, evenly spread the replicas.
+    // This calculation rounds up.
     int maxNodesPerRack = (totalNumOfReplicas - 1) / numOfRacks + 1;
     return new int[] {numOfReplicas, maxNodesPerRack};
   }
@@ -109,18 +112,42 @@ public class BlockPlacementPolicyRackFaultTolerant extends BlockPlacementPolicyD
     numOfReplicas = Math.min(totalReplicaExpected - results.size(),
         (maxNodesPerRack -1) * numOfRacks - (results.size() - excess));
 
-    // Fill each rack exactly (maxNodesPerRack-1) replicas.
-    writer = chooseOnce(numOfReplicas, writer, new HashSet<>(excludedNodes),
-        blocksize, maxNodesPerRack -1, results, avoidStaleNodes, storageTypes);
+    try {
+      // Try to spread the replicas as evenly as possible across racks.
+      // This is done by first placing with (maxNodesPerRack-1), then spreading
+      // the remainder by calling again with maxNodesPerRack.
+      writer = chooseOnce(numOfReplicas, writer, new HashSet<>(excludedNodes),
+          blocksize, maxNodesPerRack - 1, results, avoidStaleNodes,
+          storageTypes);
 
-    for (DatanodeStorageInfo resultStorage : results) {
-      addToExcludedNodes(resultStorage.getDatanodeDescriptor(), excludedNodes);
-    }
+      // Exclude the chosen nodes
+      for (DatanodeStorageInfo resultStorage : results) {
+        addToExcludedNodes(resultStorage.getDatanodeDescriptor(),
+            excludedNodes);
+      }
+      LOG.trace("Chosen nodes: {}", results);
+      LOG.trace("Excluded nodes: {}", excludedNodes);
 
-    // For some racks, place one more replica to each one of them.
-    numOfReplicas = totalReplicaExpected - results.size();
-    chooseOnce(numOfReplicas, writer, excludedNodes, blocksize,
-        maxNodesPerRack, results, avoidStaleNodes, storageTypes);
+      numOfReplicas = totalReplicaExpected - results.size();
+      chooseOnce(numOfReplicas, writer, excludedNodes, blocksize,
+          maxNodesPerRack, results, avoidStaleNodes, storageTypes);
+    } catch (NotEnoughReplicasException e) {
+      LOG.debug("Only able to place {} of {} (maxNodesPerRack={}) nodes " +
+              "evenly across racks, falling back to uneven placement.",
+          results.size(), numOfReplicas, maxNodesPerRack);
+      LOG.debug("Caught exception was:", e);
+      // Exclude the chosen nodes
+      for (DatanodeStorageInfo resultStorage : results) {
+        addToExcludedNodes(resultStorage.getDatanodeDescriptor(),
+            excludedNodes);
+      }
+
+      LOG.trace("Chosen nodes: {}", results);
+      LOG.trace("Excluded nodes: {}", excludedNodes);
+      numOfReplicas = totalReplicaExpected - results.size();
+      chooseOnce(numOfReplicas, writer, excludedNodes, blocksize,
+          totalReplicaExpected, results, avoidStaleNodes, storageTypes);
+    }
 
     return writer;
   }

+ 39 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlocksMap.java

@@ -19,6 +19,7 @@ package org.apache.hadoop.hdfs.server.blockmanagement;
 
 import java.util.Collections;
 import java.util.Iterator;
+import java.util.concurrent.atomic.LongAdder;
 
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.server.namenode.INodeId;
@@ -37,6 +38,9 @@ class BlocksMap {
   
   private GSet<Block, BlockInfo> blocks;
 
+  private final LongAdder totalReplicatedBlocks = new LongAdder();
+  private final LongAdder totalECBlockGroups = new LongAdder();
+
   BlocksMap(int capacity) {
     // Use 2% of total memory to size the GSet capacity
     this.capacity = capacity;
@@ -65,6 +69,8 @@ class BlocksMap {
   void clear() {
     if (blocks != null) {
       blocks.clear();
+      totalReplicatedBlocks.reset();
+      totalECBlockGroups.reset();
     }
   }
 
@@ -76,6 +82,7 @@ class BlocksMap {
     if (info != b) {
       info = b;
       blocks.put(info);
+      incrementBlockStat(info);
     }
     info.setBlockCollectionId(bc.getId());
     return info;
@@ -88,8 +95,10 @@ class BlocksMap {
    */
   void removeBlock(Block block) {
     BlockInfo blockInfo = blocks.remove(block);
-    if (blockInfo == null)
+    if (blockInfo == null) {
       return;
+    }
+    decrementBlockStat(block);
 
     assert blockInfo.getBlockCollectionId() == INodeId.INVALID_INODE_ID;
     final int size = blockInfo.isStriped() ?
@@ -166,6 +175,7 @@ class BlocksMap {
     if (info.hasNoStorage()    // no datanodes left
         && info.isDeleted()) { // does not belong to a file
       blocks.remove(b);  // remove block from the map
+      decrementBlockStat(b);
     }
     return removed;
   }
@@ -196,4 +206,32 @@ class BlocksMap {
   int getCapacity() {
     return capacity;
   }
+
+  private void incrementBlockStat(Block block) {
+    if (BlockIdManager.isStripedBlockID(block.getBlockId())) {
+      totalECBlockGroups.increment();
+    } else {
+      totalReplicatedBlocks.increment();
+    }
+  }
+
+  private void decrementBlockStat(Block block) {
+    if (BlockIdManager.isStripedBlockID(block.getBlockId())) {
+      totalECBlockGroups.decrement();
+      assert totalECBlockGroups.longValue() >= 0 :
+          "Total number of ec block groups should be non-negative";
+    } else {
+      totalReplicatedBlocks.decrement();
+      assert totalReplicatedBlocks.longValue() >= 0 :
+          "Total number of replicated blocks should be non-negative";
+    }
+  }
+
+  long getReplicatedBlocks() {
+    return totalReplicatedBlocks.longValue();
+  }
+
+  long getECBlockGroups() {
+    return totalECBlockGroups.longValue();
+  }
 }

+ 1 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/web/webhdfs/ParameterParser.java

@@ -59,8 +59,7 @@ class ParameterParser {
   private final Map<String, List<String>> params;
 
   ParameterParser(QueryStringDecoder decoder, Configuration conf) {
-    this.path = decodeComponent(decoder.path().substring(WEBHDFS_PREFIX_LENGTH),
-        StandardCharsets.UTF_8);
+    this.path = decoder.path().substring(WEBHDFS_PREFIX_LENGTH);
     this.params = decoder.parameters();
     this.conf = conf;
   }

+ 211 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/FederationMBean.java

@@ -0,0 +1,211 @@
+/**
+ * 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.server.federation.metrics;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * JMX interface for the federation statistics.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public interface FederationMBean {
+
+  /**
+   * Get information about all the namenodes in the federation or null if
+   * failure.
+   * @return JSON with all the Namenodes.
+   */
+  String getNamenodes();
+
+  /**
+   * Get the latest info for each registered nameservice.
+   * @return JSON with all the nameservices.
+   */
+  String getNameservices();
+
+  /**
+   * Get the mount table for the federated filesystem or null if failure.
+   * @return JSON with the mount table.
+   */
+  String getMountTable();
+
+  /**
+   * Get the total capacity of the federated cluster.
+   * @return Total capacity of the federated cluster.
+   */
+  long getTotalCapacity();
+
+  /**
+   * Get the used capacity of the federated cluster.
+   * @return Used capacity of the federated cluster.
+   */
+  long getUsedCapacity();
+
+  /**
+   * Get the remaining capacity of the federated cluster.
+   * @return Remaining capacity of the federated cluster.
+   */
+  long getRemainingCapacity();
+
+  /**
+   * Get the number of nameservices in the federation.
+   * @return Number of nameservices in the federation.
+   */
+  int getNumNameservices();
+
+  /**
+   * Get the number of namenodes.
+   * @return Number of namenodes.
+   */
+  int getNumNamenodes();
+
+  /**
+   * Get the number of expired namenodes.
+   * @return Number of expired namenodes.
+   */
+  int getNumExpiredNamenodes();
+
+  /**
+   * Get the number of live datanodes.
+   * @return Number of live datanodes.
+   */
+  int getNumLiveNodes();
+
+  /**
+   * Get the number of dead datanodes.
+   * @return Number of dead datanodes.
+   */
+  int getNumDeadNodes();
+
+  /**
+   * Get the number of decommissioning datanodes.
+   * @return Number of decommissioning datanodes.
+   */
+  int getNumDecommissioningNodes();
+
+  /**
+   * Get the number of live decommissioned datanodes.
+   * @return Number of live decommissioned datanodes.
+   */
+  int getNumDecomLiveNodes();
+
+  /**
+   * Get the number of dead decommissioned datanodes.
+   * @return Number of dead decommissioned datanodes.
+   */
+  int getNumDecomDeadNodes();
+
+  /**
+   * Get Max, Median, Min and Standard Deviation of DataNodes usage.
+   * @return the DataNode usage information, as a JSON string.
+   */
+  String getNodeUsage();
+
+  /**
+   * Get the number of blocks in the federation.
+   * @return Number of blocks in the federation.
+   */
+  long getNumBlocks();
+
+  /**
+   * Get the number of missing blocks in the federation.
+   * @return Number of missing blocks in the federation.
+   */
+  long getNumOfMissingBlocks();
+
+  /**
+   * Get the number of pending replication blocks in the federation.
+   * @return Number of pending replication blocks in the federation.
+   */
+  long getNumOfBlocksPendingReplication();
+
+  /**
+   * Get the number of under replicated blocks in the federation.
+   * @return Number of under replicated blocks in the federation.
+   */
+  long getNumOfBlocksUnderReplicated();
+
+  /**
+   * Get the number of pending deletion blocks in the federation.
+   * @return Number of pending deletion blocks in the federation.
+   */
+  long getNumOfBlocksPendingDeletion();
+
+  /**
+   * Get the number of files in the federation.
+   * @return Number of files in the federation.
+   */
+  long getNumFiles();
+
+  /**
+   * When the router started.
+   * @return Date as a string the router started.
+   */
+  String getRouterStarted();
+
+  /**
+   * Get the version of the router.
+   * @return Version of the router.
+   */
+  String getVersion();
+
+  /**
+   * Get the compilation date of the router.
+   * @return Compilation date of the router.
+   */
+  String getCompiledDate();
+
+  /**
+   * Get the compilation info of the router.
+   * @return Compilation info of the router.
+   */
+  String getCompileInfo();
+
+  /**
+   * Get the host and port of the router.
+   * @return Host and port of the router.
+   */
+  String getHostAndPort();
+
+  /**
+   * Get the identifier of the router.
+   * @return Identifier of the router.
+   */
+  String getRouterId();
+
+  /**
+   * Get the host and port of the router.
+   * @return Host and port of the router.
+   */
+  String getClusterId();
+
+  /**
+   * Get the host and port of the router.
+   * @return Host and port of the router.
+   */
+  String getBlockPoolId();
+
+  /**
+   * Get the current state of the router.
+   *
+   * @return String label for the current router state.
+   */
+  String getRouterStatus();
+}

+ 692 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/FederationMetrics.java

@@ -0,0 +1,692 @@
+/**
+ * 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.server.federation.metrics;
+
+import static org.apache.hadoop.util.Time.now;
+
+import java.io.IOException;
+import java.lang.reflect.Method;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.net.UnknownHostException;
+import java.text.SimpleDateFormat;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.LinkedHashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Function;
+import java.util.function.ToIntFunction;
+import java.util.function.ToLongFunction;
+import java.util.stream.Collectors;
+
+import javax.management.NotCompliantMBeanException;
+import javax.management.ObjectName;
+import javax.management.StandardMBean;
+
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
+import org.apache.hadoop.hdfs.server.federation.resolver.ActiveNamenodeResolver;
+import org.apache.hadoop.hdfs.server.federation.resolver.FederationNamenodeContext;
+import org.apache.hadoop.hdfs.server.federation.resolver.FederationNamespaceInfo;
+import org.apache.hadoop.hdfs.server.federation.resolver.RemoteLocation;
+import org.apache.hadoop.hdfs.server.federation.router.Router;
+import org.apache.hadoop.hdfs.server.federation.router.RouterRpcServer;
+import org.apache.hadoop.hdfs.server.federation.store.MembershipStore;
+import org.apache.hadoop.hdfs.server.federation.store.MountTableStore;
+import org.apache.hadoop.hdfs.server.federation.store.StateStoreService;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.GetMountTableEntriesRequest;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.GetMountTableEntriesResponse;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.GetNamenodeRegistrationsRequest;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.GetNamenodeRegistrationsResponse;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.GetNamespaceInfoRequest;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.GetNamespaceInfoResponse;
+import org.apache.hadoop.hdfs.server.federation.store.records.BaseRecord;
+import org.apache.hadoop.hdfs.server.federation.store.records.MembershipState;
+import org.apache.hadoop.hdfs.server.federation.store.records.MembershipStats;
+import org.apache.hadoop.hdfs.server.federation.store.records.MountTable;
+import org.apache.hadoop.metrics2.util.MBeans;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.util.VersionInfo;
+import org.codehaus.jettison.json.JSONObject;
+import org.eclipse.jetty.util.ajax.JSON;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Implementation of the Router metrics collector.
+ */
+public class FederationMetrics implements FederationMBean {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(FederationMetrics.class);
+
+  /** Format for a date. */
+  private static final String DATE_FORMAT = "yyyy/MM/dd HH:mm:ss";
+
+  /** Prevent holding the page from load too long. */
+  private static final long TIME_OUT = TimeUnit.SECONDS.toMillis(1);
+
+
+  /** Router interface. */
+  private final Router router;
+
+  /** FederationState JMX bean. */
+  private ObjectName beanName;
+
+  /** Resolve the namenode for each namespace. */
+  private final ActiveNamenodeResolver namenodeResolver;
+
+  /** State store. */
+  private final StateStoreService stateStore;
+  /** Membership state store. */
+  private MembershipStore membershipStore;
+  /** Mount table store. */
+  private MountTableStore mountTableStore;
+
+
+  public FederationMetrics(Router router) throws IOException {
+    this.router = router;
+
+    try {
+      StandardMBean bean = new StandardMBean(this, FederationMBean.class);
+      this.beanName = MBeans.register("Router", "FederationState", bean);
+      LOG.info("Registered Router MBean: {}", this.beanName);
+    } catch (NotCompliantMBeanException e) {
+      throw new RuntimeException("Bad Router MBean setup", e);
+    }
+
+    // Resolve namenode for each nameservice
+    this.namenodeResolver = this.router.getNamenodeResolver();
+
+    // State store interfaces
+    this.stateStore = this.router.getStateStore();
+    if (this.stateStore == null) {
+      LOG.error("State store not available");
+    } else {
+      this.membershipStore = stateStore.getRegisteredRecordStore(
+          MembershipStore.class);
+      this.mountTableStore = stateStore.getRegisteredRecordStore(
+          MountTableStore.class);
+    }
+  }
+
+  /**
+   * Unregister the JMX beans.
+   */
+  public void close() {
+    if (this.beanName != null) {
+      MBeans.unregister(beanName);
+    }
+  }
+
+  @Override
+  public String getNamenodes() {
+    final Map<String, Map<String, Object>> info = new LinkedHashMap<>();
+    try {
+      // Get the values from the store
+      GetNamenodeRegistrationsRequest request =
+          GetNamenodeRegistrationsRequest.newInstance();
+      GetNamenodeRegistrationsResponse response =
+          membershipStore.getNamenodeRegistrations(request);
+
+      // Order the namenodes
+      final List<MembershipState> namenodes = response.getNamenodeMemberships();
+      if (namenodes == null || namenodes.size() == 0) {
+        return JSON.toString(info);
+      }
+      List<MembershipState> namenodesOrder = new ArrayList<>(namenodes);
+      Collections.sort(namenodesOrder, MembershipState.NAME_COMPARATOR);
+
+      // Dump namenodes information into JSON
+      for (MembershipState namenode : namenodesOrder) {
+        Map<String, Object> innerInfo = new HashMap<>();
+        Map<String, Object> map = getJson(namenode);
+        innerInfo.putAll(map);
+        long dateModified = namenode.getDateModified();
+        long lastHeartbeat = getSecondsSince(dateModified);
+        innerInfo.put("lastHeartbeat", lastHeartbeat);
+        MembershipStats stats = namenode.getStats();
+        long used = stats.getTotalSpace() - stats.getAvailableSpace();
+        innerInfo.put("used", used);
+        info.put(namenode.getNamenodeKey(),
+            Collections.unmodifiableMap(innerInfo));
+      }
+    } catch (IOException e) {
+      LOG.error("Enable to fetch json representation of namenodes {}",
+          e.getMessage());
+      return "{}";
+    }
+    return JSON.toString(info);
+  }
+
+  @Override
+  public String getNameservices() {
+    final Map<String, Map<String, Object>> info = new LinkedHashMap<>();
+    try {
+      final List<MembershipState> namenodes = getActiveNamenodeRegistrations();
+      List<MembershipState> namenodesOrder = new ArrayList<>(namenodes);
+      Collections.sort(namenodesOrder, MembershipState.NAME_COMPARATOR);
+
+      // Dump namenodes information into JSON
+      for (MembershipState namenode : namenodesOrder) {
+        Map<String, Object> innerInfo = new HashMap<>();
+        Map<String, Object> map = getJson(namenode);
+        innerInfo.putAll(map);
+        long dateModified = namenode.getDateModified();
+        long lastHeartbeat = getSecondsSince(dateModified);
+        innerInfo.put("lastHeartbeat", lastHeartbeat);
+        MembershipStats stats = namenode.getStats();
+        long used = stats.getTotalSpace() - stats.getAvailableSpace();
+        innerInfo.put("used", used);
+        info.put(namenode.getNamenodeKey(),
+            Collections.unmodifiableMap(innerInfo));
+      }
+    } catch (IOException e) {
+      LOG.error("Cannot retrieve nameservices for JMX: {}", e.getMessage());
+      return "{}";
+    }
+    return JSON.toString(info);
+  }
+
+  @Override
+  public String getMountTable() {
+    final List<Map<String, Object>> info = new LinkedList<>();
+
+    try {
+      // Get all the mount points in order
+      GetMountTableEntriesRequest request =
+          GetMountTableEntriesRequest.newInstance("/");
+      GetMountTableEntriesResponse response =
+          mountTableStore.getMountTableEntries(request);
+      final List<MountTable> mounts = response.getEntries();
+      List<MountTable> orderedMounts = new ArrayList<>(mounts);
+      Collections.sort(orderedMounts, MountTable.SOURCE_COMPARATOR);
+
+      // Dump mount table entries information into JSON
+      for (MountTable entry : orderedMounts) {
+        // Sumarize destinations
+        Set<String> nameservices = new LinkedHashSet<>();
+        Set<String> paths = new LinkedHashSet<>();
+        for (RemoteLocation location : entry.getDestinations()) {
+          nameservices.add(location.getNameserviceId());
+          paths.add(location.getDest());
+        }
+
+        Map<String, Object> map = getJson(entry);
+        // We add some values with a cleaner format
+        map.put("dateCreated", getDateString(entry.getDateCreated()));
+        map.put("dateModified", getDateString(entry.getDateModified()));
+
+        Map<String, Object> innerInfo = new HashMap<>();
+        innerInfo.putAll(map);
+        innerInfo.put("nameserviceId", StringUtils.join(",", nameservices));
+        innerInfo.put("path", StringUtils.join(",", paths));
+        if (nameservices.size() > 1) {
+          innerInfo.put("order", entry.getDestOrder().toString());
+        } else {
+          innerInfo.put("order", "");
+        }
+        innerInfo.put("readonly", entry.isReadOnly());
+        info.add(Collections.unmodifiableMap(innerInfo));
+      }
+    } catch (IOException e) {
+      LOG.error(
+          "Cannot generate JSON of mount table from store: {}", e.getMessage());
+      return "[]";
+    }
+    return JSON.toString(info);
+  }
+
+  @Override
+  public long getTotalCapacity() {
+    return getNameserviceAggregatedLong(MembershipStats::getTotalSpace);
+  }
+
+  @Override
+  public long getRemainingCapacity() {
+    return getNameserviceAggregatedLong(MembershipStats::getAvailableSpace);
+  }
+
+  @Override
+  public long getUsedCapacity() {
+    return getTotalCapacity() - getRemainingCapacity();
+  }
+
+  @Override
+  public int getNumNameservices() {
+    try {
+      Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
+      return nss.size();
+    } catch (IOException e) {
+      LOG.error(
+          "Cannot fetch number of expired registrations from the store: {}",
+          e.getMessage());
+      return 0;
+    }
+  }
+
+  @Override
+  public int getNumNamenodes() {
+    try {
+      GetNamenodeRegistrationsRequest request =
+          GetNamenodeRegistrationsRequest.newInstance();
+      GetNamenodeRegistrationsResponse response =
+          membershipStore.getNamenodeRegistrations(request);
+      List<MembershipState> memberships = response.getNamenodeMemberships();
+      return memberships.size();
+    } catch (IOException e) {
+      LOG.error("Cannot retrieve numNamenodes for JMX: {}", e.getMessage());
+      return 0;
+    }
+  }
+
+  @Override
+  public int getNumExpiredNamenodes() {
+    try {
+      GetNamenodeRegistrationsRequest request =
+          GetNamenodeRegistrationsRequest.newInstance();
+      GetNamenodeRegistrationsResponse response =
+          membershipStore.getExpiredNamenodeRegistrations(request);
+      List<MembershipState> expiredMemberships =
+          response.getNamenodeMemberships();
+      return expiredMemberships.size();
+    } catch (IOException e) {
+      LOG.error(
+          "Cannot retrieve numExpiredNamenodes for JMX: {}", e.getMessage());
+      return 0;
+    }
+  }
+
+  @Override
+  public int getNumLiveNodes() {
+    return getNameserviceAggregatedInt(
+        MembershipStats::getNumOfActiveDatanodes);
+  }
+
+  @Override
+  public int getNumDeadNodes() {
+    return getNameserviceAggregatedInt(MembershipStats::getNumOfDeadDatanodes);
+  }
+
+  @Override
+  public int getNumDecommissioningNodes() {
+    return getNameserviceAggregatedInt(
+        MembershipStats::getNumOfDecommissioningDatanodes);
+  }
+
+  @Override
+  public int getNumDecomLiveNodes() {
+    return getNameserviceAggregatedInt(
+        MembershipStats::getNumOfDecomActiveDatanodes);
+  }
+
+  @Override
+  public int getNumDecomDeadNodes() {
+    return getNameserviceAggregatedInt(
+        MembershipStats::getNumOfDecomDeadDatanodes);
+  }
+
+  @Override // NameNodeMXBean
+  public String getNodeUsage() {
+    float median = 0;
+    float max = 0;
+    float min = 0;
+    float dev = 0;
+
+    final Map<String, Map<String, Object>> info = new HashMap<>();
+    try {
+      RouterRpcServer rpcServer = this.router.getRpcServer();
+      DatanodeInfo[] live = rpcServer.getDatanodeReport(
+          DatanodeReportType.LIVE, TIME_OUT);
+
+      if (live.length > 0) {
+        float totalDfsUsed = 0;
+        float[] usages = new float[live.length];
+        int i = 0;
+        for (DatanodeInfo dn : live) {
+          usages[i++] = dn.getDfsUsedPercent();
+          totalDfsUsed += dn.getDfsUsedPercent();
+        }
+        totalDfsUsed /= live.length;
+        Arrays.sort(usages);
+        median = usages[usages.length / 2];
+        max = usages[usages.length - 1];
+        min = usages[0];
+
+        for (i = 0; i < usages.length; i++) {
+          dev += (usages[i] - totalDfsUsed) * (usages[i] - totalDfsUsed);
+        }
+        dev = (float) Math.sqrt(dev / usages.length);
+      }
+    } catch (IOException e) {
+      LOG.info("Cannot get the live nodes: {}", e.getMessage());
+    }
+
+    final Map<String, Object> innerInfo = new HashMap<>();
+    innerInfo.put("min", StringUtils.format("%.2f%%", min));
+    innerInfo.put("median", StringUtils.format("%.2f%%", median));
+    innerInfo.put("max", StringUtils.format("%.2f%%", max));
+    innerInfo.put("stdDev", StringUtils.format("%.2f%%", dev));
+    info.put("nodeUsage", innerInfo);
+
+    return JSON.toString(info);
+  }
+
+  @Override
+  public long getNumBlocks() {
+    return getNameserviceAggregatedLong(MembershipStats::getNumOfBlocks);
+  }
+
+  @Override
+  public long getNumOfMissingBlocks() {
+    return getNameserviceAggregatedLong(MembershipStats::getNumOfBlocksMissing);
+  }
+
+  @Override
+  public long getNumOfBlocksPendingReplication() {
+    return getNameserviceAggregatedLong(
+        MembershipStats::getNumOfBlocksPendingReplication);
+  }
+
+  @Override
+  public long getNumOfBlocksUnderReplicated() {
+    return getNameserviceAggregatedLong(
+        MembershipStats::getNumOfBlocksUnderReplicated);
+  }
+
+  @Override
+  public long getNumOfBlocksPendingDeletion() {
+    return getNameserviceAggregatedLong(
+        MembershipStats::getNumOfBlocksPendingDeletion);
+  }
+
+  @Override
+  public long getNumFiles() {
+    return getNameserviceAggregatedLong(MembershipStats::getNumOfFiles);
+  }
+
+  @Override
+  public String getRouterStarted() {
+    long startTime = this.router.getStartTime();
+    return new Date(startTime).toString();
+  }
+
+  @Override
+  public String getVersion() {
+    return VersionInfo.getVersion() + ", r" + VersionInfo.getRevision();
+  }
+
+  @Override
+  public String getCompiledDate() {
+    return VersionInfo.getDate();
+  }
+
+  @Override
+  public String getCompileInfo() {
+    return VersionInfo.getDate() + " by " + VersionInfo.getUser() + " from "
+        + VersionInfo.getBranch();
+  }
+
+  @Override
+  public String getHostAndPort() {
+    InetSocketAddress address = this.router.getHttpServerAddress();
+    if (address != null) {
+      try {
+        String hostname = InetAddress.getLocalHost().getHostName();
+        int port = address.getPort();
+        return hostname + ":" + port;
+      } catch (UnknownHostException ignored) { }
+    }
+    return "Unknown";
+  }
+
+  @Override
+  public String getRouterId() {
+    return this.router.getRouterId();
+  }
+
+  @Override
+  public String getClusterId() {
+    try {
+      Collection<String> clusterIds =
+          getNamespaceInfo(FederationNamespaceInfo::getClusterId);
+      return clusterIds.toString();
+    } catch (IOException e) {
+      LOG.error("Cannot fetch cluster ID metrics: {}", e.getMessage());
+      return "";
+    }
+  }
+
+  @Override
+  public String getBlockPoolId() {
+    try {
+      Collection<String> blockpoolIds =
+          getNamespaceInfo(FederationNamespaceInfo::getBlockPoolId);
+      return blockpoolIds.toString();
+    } catch (IOException e) {
+      LOG.error("Cannot fetch block pool ID metrics: {}", e.getMessage());
+      return "";
+    }
+  }
+
+  @Override
+  public String getRouterStatus() {
+    return "RUNNING";
+  }
+
+  /**
+   * Build a set of unique values found in all namespaces.
+   *
+   * @param f Method reference of the appropriate FederationNamespaceInfo
+   *          getter function
+   * @return Set of unique string values found in all discovered namespaces.
+   * @throws IOException if the query could not be executed.
+   */
+  private Collection<String> getNamespaceInfo(
+      Function<FederationNamespaceInfo, String> f) throws IOException {
+    GetNamespaceInfoRequest request = GetNamespaceInfoRequest.newInstance();
+    GetNamespaceInfoResponse response =
+        membershipStore.getNamespaceInfo(request);
+    return response.getNamespaceInfo().stream()
+      .map(f)
+      .collect(Collectors.toSet());
+  }
+
+  /**
+   * Get the aggregated value for a method for all nameservices.
+   * @param f Method reference
+   * @return Aggregated integer.
+   */
+  private int getNameserviceAggregatedInt(ToIntFunction<MembershipStats> f) {
+    try {
+      return getActiveNamenodeRegistrations().stream()
+               .map(MembershipState::getStats)
+               .collect(Collectors.summingInt(f));
+    } catch (IOException e) {
+      LOG.error("Unable to extract metrics: {}", e.getMessage());
+      return 0;
+    }
+  }
+
+  /**
+   * Get the aggregated value for a method for all nameservices.
+   * @param f Method reference
+   * @return Aggregated long.
+   */
+  private long getNameserviceAggregatedLong(ToLongFunction<MembershipStats> f) {
+    try {
+      return getActiveNamenodeRegistrations().stream()
+               .map(MembershipState::getStats)
+               .collect(Collectors.summingLong(f));
+    } catch (IOException e) {
+      LOG.error("Unable to extract metrics: {}", e.getMessage());
+      return 0;
+    }
+  }
+
+  /**
+   * Fetches the most active namenode memberships for all known nameservices.
+   * The fetched membership may not or may not be active. Excludes expired
+   * memberships.
+   * @throws IOException if the query could not be performed.
+   * @return List of the most active NNs from each known nameservice.
+   */
+  private List<MembershipState> getActiveNamenodeRegistrations()
+      throws IOException {
+
+    List<MembershipState> resultList = new ArrayList<>();
+    GetNamespaceInfoRequest request = GetNamespaceInfoRequest.newInstance();
+    GetNamespaceInfoResponse response =
+        membershipStore.getNamespaceInfo(request);
+    for (FederationNamespaceInfo nsInfo : response.getNamespaceInfo()) {
+      // Fetch the most recent namenode registration
+      String nsId = nsInfo.getNameserviceId();
+      List<? extends FederationNamenodeContext> nns =
+          namenodeResolver.getNamenodesForNameserviceId(nsId);
+      if (nns != null) {
+        FederationNamenodeContext nn = nns.get(0);
+        if (nn != null && nn instanceof MembershipState) {
+          resultList.add((MembershipState) nn);
+        }
+      }
+    }
+    return resultList;
+  }
+
+  /**
+   * Get time as a date string.
+   * @param time Seconds since 1970.
+   * @return String representing the date.
+   */
+  private static String getDateString(long time) {
+    if (time <= 0) {
+      return "-";
+    }
+    Date date = new Date(time);
+
+    SimpleDateFormat sdf = new SimpleDateFormat(DATE_FORMAT);
+    return sdf.format(date);
+  }
+
+  /**
+   * Get the number of seconds passed since a date.
+   *
+   * @param timeMs to use as a reference.
+   * @return Seconds since the date.
+   */
+  private static long getSecondsSince(long timeMs) {
+    if (timeMs < 0) {
+      return -1;
+    }
+    return (now() - timeMs) / 1000;
+  }
+
+  /**
+   * Get JSON for this record.
+   *
+   * @return Map representing the data for the JSON representation.
+   */
+  private static Map<String, Object> getJson(BaseRecord record) {
+    Map<String, Object> json = new HashMap<>();
+    Map<String, Class<?>> fields = getFields(record);
+
+    for (String fieldName : fields.keySet()) {
+      if (!fieldName.equalsIgnoreCase("proto")) {
+        try {
+          Object value = getField(record, fieldName);
+          if (value instanceof BaseRecord) {
+            BaseRecord recordField = (BaseRecord) value;
+            json.putAll(getJson(recordField));
+          } else {
+            json.put(fieldName, value == null ? JSONObject.NULL : value);
+          }
+        } catch (Exception e) {
+          throw new IllegalArgumentException(
+              "Cannot serialize field " + fieldName + " into JSON");
+        }
+      }
+    }
+    return json;
+  }
+
+  /**
+   * Returns all serializable fields in the object.
+   *
+   * @return Map with the fields.
+   */
+  private static Map<String, Class<?>> getFields(BaseRecord record) {
+    Map<String, Class<?>> getters = new HashMap<>();
+    for (Method m : record.getClass().getDeclaredMethods()) {
+      if (m.getName().startsWith("get")) {
+        try {
+          Class<?> type = m.getReturnType();
+          char[] c = m.getName().substring(3).toCharArray();
+          c[0] = Character.toLowerCase(c[0]);
+          String key = new String(c);
+          getters.put(key, type);
+        } catch (Exception e) {
+          LOG.error("Cannot execute getter {} on {}", m.getName(), record);
+        }
+      }
+    }
+    return getters;
+  }
+
+  /**
+   * Fetches the value for a field name.
+   *
+   * @param fieldName the legacy name of the field.
+   * @return The field data or null if not found.
+   */
+  private static Object getField(BaseRecord record, String fieldName) {
+    Object result = null;
+    Method m = locateGetter(record, fieldName);
+    if (m != null) {
+      try {
+        result = m.invoke(record);
+      } catch (Exception e) {
+        LOG.error("Cannot get field {} on {}", fieldName, record);
+      }
+    }
+    return result;
+  }
+
+  /**
+   * Finds the appropriate getter for a field name.
+   *
+   * @param fieldName The legacy name of the field.
+   * @return The matching getter or null if not found.
+   */
+  private static Method locateGetter(BaseRecord record, String fieldName) {
+    for (Method m : record.getClass().getMethods()) {
+      if (m.getName().equalsIgnoreCase("get" + fieldName)) {
+        return m;
+      }
+    }
+    return null;
+  }
+}

+ 90 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/FederationRPCMBean.java

@@ -0,0 +1,90 @@
+/**
+ * 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.server.federation.metrics;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * JMX interface for the RPC server.
+ * TODO use the default RPC MBean.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public interface FederationRPCMBean {
+
+  long getProxyOps();
+
+  double getProxyAvg();
+
+  long getProcessingOps();
+
+  double getProcessingAvg();
+
+  long getProxyOpFailureCommunicate();
+
+  long getProxyOpFailureStandby();
+
+  long getProxyOpNotImplemented();
+
+  long getRouterFailureStateStoreOps();
+
+  long getRouterFailureReadOnlyOps();
+
+  long getRouterFailureLockedOps();
+
+  long getRouterFailureSafemodeOps();
+
+  int getRpcServerCallQueue();
+
+  /**
+   * Get the number of RPC connections between the clients and the Router.
+   * @return Number of RPC connections between the clients and the Router.
+   */
+  int getRpcServerNumOpenConnections();
+
+  /**
+   * Get the number of RPC connections between the Router and the NNs.
+   * @return Number of RPC connections between the Router and the NNs.
+   */
+  int getRpcClientNumConnections();
+
+  /**
+   * Get the number of active RPC connections between the Router and the NNs.
+   * @return Number of active RPC connections between the Router and the NNs.
+   */
+  int getRpcClientNumActiveConnections();
+
+  /**
+   * Get the number of RPC connections to be created.
+   * @return Number of RPC connections to be created.
+   */
+  int getRpcClientNumCreatingConnections();
+
+  /**
+   * Get the number of connection pools between the Router and a NNs.
+   * @return Number of connection pools between the Router and a NNs.
+   */
+  int getRpcClientNumConnectionPools();
+
+  /**
+   * JSON representation of the RPC connections from the Router to the NNs.
+   * @return JSON string representation.
+   */
+  String getRpcClientConnections();
+}

+ 239 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/FederationRPCMetrics.java

@@ -0,0 +1,239 @@
+/**
+ * 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.server.federation.metrics;
+
+import static org.apache.hadoop.metrics2.impl.MsInfo.ProcessName;
+import static org.apache.hadoop.metrics2.impl.MsInfo.SessionId;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.server.federation.router.RouterRpcServer;
+import org.apache.hadoop.metrics2.MetricsSystem;
+import org.apache.hadoop.metrics2.annotation.Metric;
+import org.apache.hadoop.metrics2.annotation.Metrics;
+import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
+import org.apache.hadoop.metrics2.lib.MetricsRegistry;
+import org.apache.hadoop.metrics2.lib.MutableCounterLong;
+import org.apache.hadoop.metrics2.lib.MutableRate;
+
+/**
+ * Implementation of the RPC metrics collector.
+ */
+@Metrics(name = "RouterRPCActivity", about = "Router RPC Activity",
+    context = "router")
+public class FederationRPCMetrics implements FederationRPCMBean {
+
+  private final MetricsRegistry registry = new MetricsRegistry("router");
+
+  private RouterRpcServer rpcServer;
+
+  @Metric("Time for the router to process an operation internally")
+  private MutableRate processing;
+  @Metric("Number of operations the Router processed internally")
+  private MutableCounterLong processingOp;
+  @Metric("Time for the Router to proxy an operation to the Namenodes")
+  private MutableRate proxy;
+  @Metric("Number of operations the Router proxied to a Namenode")
+  private MutableCounterLong proxyOp;
+
+  @Metric("Number of operations to fail to reach NN")
+  private MutableCounterLong proxyOpFailureStandby;
+  @Metric("Number of operations to hit a standby NN")
+  private MutableCounterLong proxyOpFailureCommunicate;
+  @Metric("Number of operations not implemented")
+  private MutableCounterLong proxyOpNotImplemented;
+
+  @Metric("Failed requests due to State Store unavailable")
+  private MutableCounterLong routerFailureStateStore;
+  @Metric("Failed requests due to read only mount point")
+  private MutableCounterLong routerFailureReadOnly;
+  @Metric("Failed requests due to locked path")
+  private MutableCounterLong routerFailureLocked;
+  @Metric("Failed requests due to safe mode")
+  private MutableCounterLong routerFailureSafemode;
+
+  public FederationRPCMetrics(Configuration conf, RouterRpcServer rpcServer) {
+    this.rpcServer = rpcServer;
+
+    registry.tag(SessionId, "RouterRPCSession");
+    registry.tag(ProcessName, "Router");
+  }
+
+  public static FederationRPCMetrics create(Configuration conf,
+      RouterRpcServer rpcServer) {
+    MetricsSystem ms = DefaultMetricsSystem.instance();
+    return ms.register(FederationRPCMetrics.class.getName(),
+        "HDFS Federation RPC Metrics",
+        new FederationRPCMetrics(conf, rpcServer));
+  }
+
+  /**
+   * Convert nanoseconds to milliseconds.
+   * @param ns Time in nanoseconds.
+   * @return Time in milliseconds.
+   */
+  private static double toMs(double ns) {
+    return ns / 1000000;
+  }
+
+  /**
+   * Reset the metrics system.
+   */
+  public static void reset() {
+    MetricsSystem ms = DefaultMetricsSystem.instance();
+    ms.unregisterSource(FederationRPCMetrics.class.getName());
+  }
+
+  public void incrProxyOpFailureStandby() {
+    proxyOpFailureStandby.incr();
+  }
+
+  @Override
+  public long getProxyOpFailureStandby() {
+    return proxyOpFailureStandby.value();
+  }
+
+  public void incrProxyOpFailureCommunicate() {
+    proxyOpFailureCommunicate.incr();
+  }
+
+  @Override
+  public long getProxyOpFailureCommunicate() {
+    return proxyOpFailureCommunicate.value();
+  }
+
+
+  public void incrProxyOpNotImplemented() {
+    proxyOpNotImplemented.incr();
+  }
+
+  @Override
+  public long getProxyOpNotImplemented() {
+    return proxyOpNotImplemented.value();
+  }
+
+  public void incrRouterFailureStateStore() {
+    routerFailureStateStore.incr();
+  }
+
+  @Override
+  public long getRouterFailureStateStoreOps() {
+    return routerFailureStateStore.value();
+  }
+
+  public void incrRouterFailureSafemode() {
+    routerFailureSafemode.incr();
+  }
+
+  @Override
+  public long getRouterFailureSafemodeOps() {
+    return routerFailureSafemode.value();
+  }
+
+  public void incrRouterFailureReadOnly() {
+    routerFailureReadOnly.incr();
+  }
+
+  @Override
+  public long getRouterFailureReadOnlyOps() {
+    return routerFailureReadOnly.value();
+  }
+
+  public void incrRouterFailureLocked() {
+    routerFailureLocked.incr();
+  }
+
+  @Override
+  public long getRouterFailureLockedOps() {
+    return routerFailureLocked.value();
+  }
+
+  @Override
+  public int getRpcServerCallQueue() {
+    return rpcServer.getServer().getCallQueueLen();
+  }
+
+  @Override
+  public int getRpcServerNumOpenConnections() {
+    return rpcServer.getServer().getNumOpenConnections();
+  }
+
+  @Override
+  public int getRpcClientNumConnections() {
+    return rpcServer.getRPCClient().getNumConnections();
+  }
+
+  @Override
+  public int getRpcClientNumActiveConnections() {
+    return rpcServer.getRPCClient().getNumActiveConnections();
+  }
+
+  @Override
+  public int getRpcClientNumCreatingConnections() {
+    return rpcServer.getRPCClient().getNumCreatingConnections();
+  }
+
+  @Override
+  public int getRpcClientNumConnectionPools() {
+    return rpcServer.getRPCClient().getNumConnectionPools();
+  }
+
+  @Override
+  public String getRpcClientConnections() {
+    return rpcServer.getRPCClient().getJSON();
+  }
+
+  /**
+   * Add the time to proxy an operation from the moment the Router sends it to
+   * the Namenode until it replied.
+   * @param time Proxy time of an operation in nanoseconds.
+   */
+  public void addProxyTime(long time) {
+    proxy.add(time);
+    proxyOp.incr();
+  }
+
+  @Override
+  public double getProxyAvg() {
+    return toMs(proxy.lastStat().mean());
+  }
+
+  @Override
+  public long getProxyOps() {
+    return proxyOp.value();
+  }
+
+  /**
+   * Add the time to process a request in the Router from the time we receive
+   * the call until we send it to the Namenode.
+   * @param time Process time of an operation in nanoseconds.
+   */
+  public void addProcessingTime(long time) {
+    processing.add(time);
+    processingOp.incr();
+  }
+
+  @Override
+  public double getProcessingAvg() {
+    return toMs(processing.lastStat().mean());
+  }
+
+  @Override
+  public long getProcessingOps() {
+    return processingOp.value();
+  }
+}

+ 211 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/FederationRPCPerformanceMonitor.java

@@ -0,0 +1,211 @@
+/**
+ * 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.server.federation.metrics;
+
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadFactory;
+
+import javax.management.NotCompliantMBeanException;
+import javax.management.ObjectName;
+import javax.management.StandardMBean;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.server.federation.router.RouterRpcMonitor;
+import org.apache.hadoop.hdfs.server.federation.router.RouterRpcServer;
+import org.apache.hadoop.hdfs.server.federation.store.StateStoreService;
+import org.apache.hadoop.metrics2.util.MBeans;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+
+/**
+ * Customizable RPC performance monitor. Receives events from the RPC server
+ * and aggregates them via JMX.
+ */
+public class FederationRPCPerformanceMonitor implements RouterRpcMonitor {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(FederationRPCPerformanceMonitor.class);
+
+
+  /** Time for an operation to be received in the Router. */
+  private static final ThreadLocal<Long> START_TIME = new ThreadLocal<>();
+  /** Time for an operation to be send to the Namenode. */
+  private static final ThreadLocal<Long> PROXY_TIME = new ThreadLocal<>();
+
+  /** Configuration for the performance monitor. */
+  private Configuration conf;
+  /** RPC server for the Router. */
+  private RouterRpcServer server;
+  /** State Store. */
+  private StateStoreService store;
+
+  /** JMX interface to monitor the RPC metrics. */
+  private FederationRPCMetrics metrics;
+  private ObjectName registeredBean;
+
+  /** Thread pool for logging stats. */
+  private ExecutorService executor;
+
+
+  @Override
+  public void init(Configuration configuration, RouterRpcServer rpcServer,
+      StateStoreService stateStore) {
+
+    this.conf = configuration;
+    this.server = rpcServer;
+    this.store = stateStore;
+
+    // Create metrics
+    this.metrics = FederationRPCMetrics.create(conf, server);
+
+    // Create thread pool
+    ThreadFactory threadFactory = new ThreadFactoryBuilder()
+        .setNameFormat("Federation RPC Performance Monitor-%d").build();
+    this.executor = Executors.newFixedThreadPool(1, threadFactory);
+
+    // Adding JMX interface
+    try {
+      StandardMBean bean =
+          new StandardMBean(this.metrics, FederationRPCMBean.class);
+      registeredBean = MBeans.register("Router", "FederationRPC", bean);
+      LOG.info("Registered FederationRPCMBean: {}", registeredBean);
+    } catch (NotCompliantMBeanException e) {
+      throw new RuntimeException("Bad FederationRPCMBean setup", e);
+    }
+  }
+
+  @Override
+  public void close() {
+    if (registeredBean != null) {
+      MBeans.unregister(registeredBean);
+      registeredBean = null;
+    }
+    if (this.executor != null) {
+      this.executor.shutdown();
+    }
+  }
+
+  /**
+   * Resets all RPC service performance counters to their defaults.
+   */
+  public void resetPerfCounters() {
+    if (registeredBean != null) {
+      MBeans.unregister(registeredBean);
+      registeredBean = null;
+    }
+    if (metrics != null) {
+      FederationRPCMetrics.reset();
+      metrics = null;
+    }
+    init(conf, server, store);
+  }
+
+  @Override
+  public void startOp() {
+    START_TIME.set(this.getNow());
+  }
+
+  @Override
+  public long proxyOp() {
+    PROXY_TIME.set(this.getNow());
+    long processingTime = getProcessingTime();
+    if (processingTime >= 0) {
+      metrics.addProcessingTime(processingTime);
+    }
+    return Thread.currentThread().getId();
+  }
+
+  @Override
+  public void proxyOpComplete(boolean success) {
+    if (success) {
+      long proxyTime = getProxyTime();
+      if (proxyTime >= 0) {
+        metrics.addProxyTime(proxyTime);
+      }
+    }
+  }
+
+  @Override
+  public void proxyOpFailureStandby() {
+    metrics.incrProxyOpFailureStandby();
+  }
+
+  @Override
+  public void proxyOpFailureCommunicate() {
+    metrics.incrProxyOpFailureCommunicate();
+  }
+
+  @Override
+  public void proxyOpNotImplemented() {
+    metrics.incrProxyOpNotImplemented();
+  }
+
+  @Override
+  public void routerFailureStateStore() {
+    metrics.incrRouterFailureStateStore();
+  }
+
+  @Override
+  public void routerFailureSafemode() {
+    metrics.incrRouterFailureSafemode();
+  }
+
+  @Override
+  public void routerFailureReadOnly() {
+    metrics.incrRouterFailureReadOnly();
+  }
+
+  @Override
+  public void routerFailureLocked() {
+    metrics.incrRouterFailureLocked();
+  }
+
+  /**
+   * Get current time.
+   * @return Current time in nanoseconds.
+   */
+  private long getNow() {
+    return System.nanoTime();
+  }
+
+  /**
+   * Get time between we receiving the operation and sending it to the Namenode.
+   * @return Processing time in nanoseconds.
+   */
+  private long getProcessingTime() {
+    if (START_TIME.get() != null && START_TIME.get() > 0 &&
+        PROXY_TIME.get() != null && PROXY_TIME.get() > 0) {
+      return PROXY_TIME.get() - START_TIME.get();
+    }
+    return -1;
+  }
+
+  /**
+   * Get time between now and when the operation was forwarded to the Namenode.
+   * @return Current proxy time in nanoseconds.
+   */
+  private long getProxyTime() {
+    if (PROXY_TIME.get() != null && PROXY_TIME.get() > 0) {
+      return getNow() - PROXY_TIME.get();
+    }
+    return -1;
+  }
+}

+ 624 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/NamenodeBeanMetrics.java

@@ -0,0 +1,624 @@
+/**
+ * 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.server.federation.metrics;
+
+import static org.apache.hadoop.util.Time.now;
+
+import java.io.IOException;
+import java.lang.management.ManagementFactory;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+import javax.management.NotCompliantMBeanException;
+import javax.management.ObjectName;
+import javax.management.StandardMBean;
+
+import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
+import org.apache.hadoop.hdfs.DFSUtilClient;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
+import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
+import org.apache.hadoop.hdfs.server.federation.resolver.FederationNamespaceInfo;
+import org.apache.hadoop.hdfs.server.federation.router.Router;
+import org.apache.hadoop.hdfs.server.federation.router.RouterRpcServer;
+import org.apache.hadoop.hdfs.server.federation.store.MembershipStore;
+import org.apache.hadoop.hdfs.server.federation.store.StateStoreService;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.GetNamespaceInfoRequest;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.GetNamespaceInfoResponse;
+import org.apache.hadoop.hdfs.server.namenode.NameNodeMXBean;
+import org.apache.hadoop.hdfs.server.namenode.NameNodeStatusMXBean;
+import org.apache.hadoop.hdfs.server.namenode.metrics.FSNamesystemMBean;
+import org.apache.hadoop.ipc.StandbyException;
+import org.apache.hadoop.metrics2.util.MBeans;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.util.VersionInfo;
+import org.eclipse.jetty.util.ajax.JSON;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Expose the Namenode metrics as the Router was one.
+ */
+public class NamenodeBeanMetrics
+    implements FSNamesystemMBean, NameNodeMXBean, NameNodeStatusMXBean {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(NamenodeBeanMetrics.class);
+
+  private final Router router;
+
+  /** FSNamesystem bean. */
+  private ObjectName fsBeanName;
+  /** FSNamesystemState bean. */
+  private ObjectName fsStateBeanName;
+  /** NameNodeInfo bean. */
+  private ObjectName nnInfoBeanName;
+  /** NameNodeStatus bean. */
+  private ObjectName nnStatusBeanName;
+
+
+  public NamenodeBeanMetrics(Router router) {
+    this.router = router;
+
+    try {
+      // TODO this needs to be done with the Metrics from FSNamesystem
+      StandardMBean bean = new StandardMBean(this, FSNamesystemMBean.class);
+      this.fsBeanName = MBeans.register("NameNode", "FSNamesystem", bean);
+      LOG.info("Registered FSNamesystem MBean: {}", this.fsBeanName);
+    } catch (NotCompliantMBeanException e) {
+      throw new RuntimeException("Bad FSNamesystem MBean setup", e);
+    }
+
+    try {
+      StandardMBean bean = new StandardMBean(this, FSNamesystemMBean.class);
+      this.fsStateBeanName =
+          MBeans.register("NameNode", "FSNamesystemState", bean);
+      LOG.info("Registered FSNamesystemState MBean: {}", this.fsStateBeanName);
+    } catch (NotCompliantMBeanException e) {
+      throw new RuntimeException("Bad FSNamesystemState MBean setup", e);
+    }
+
+    try {
+      StandardMBean bean = new StandardMBean(this, NameNodeMXBean.class);
+      this.nnInfoBeanName = MBeans.register("NameNode", "NameNodeInfo", bean);
+      LOG.info("Registered NameNodeInfo MBean: {}", this.nnInfoBeanName);
+    } catch (NotCompliantMBeanException e) {
+      throw new RuntimeException("Bad NameNodeInfo MBean setup", e);
+    }
+
+    try {
+      StandardMBean bean = new StandardMBean(this, NameNodeStatusMXBean.class);
+      this.nnStatusBeanName =
+          MBeans.register("NameNode", "NameNodeStatus", bean);
+      LOG.info("Registered NameNodeStatus MBean: {}", this.nnStatusBeanName);
+    } catch (NotCompliantMBeanException e) {
+      throw new RuntimeException("Bad NameNodeStatus MBean setup", e);
+    }
+  }
+
+  /**
+   * De-register the JMX interfaces.
+   */
+  public void close() {
+    if (fsStateBeanName != null) {
+      MBeans.unregister(fsStateBeanName);
+      fsStateBeanName = null;
+    }
+    if (nnInfoBeanName != null) {
+      MBeans.unregister(nnInfoBeanName);
+      nnInfoBeanName = null;
+    }
+    // Remove the NameNode status bean
+    if (nnStatusBeanName != null) {
+      MBeans.unregister(nnStatusBeanName);
+      nnStatusBeanName = null;
+    }
+  }
+
+  private FederationMetrics getFederationMetrics() {
+    return this.router.getMetrics();
+  }
+
+  /////////////////////////////////////////////////////////
+  // NameNodeMXBean
+  /////////////////////////////////////////////////////////
+
+  @Override
+  public String getVersion() {
+    return VersionInfo.getVersion() + ", r" + VersionInfo.getRevision();
+  }
+
+  @Override
+  public String getSoftwareVersion() {
+    return VersionInfo.getVersion();
+  }
+
+  @Override
+  public long getUsed() {
+    return getFederationMetrics().getUsedCapacity();
+  }
+
+  @Override
+  public long getFree() {
+    return getFederationMetrics().getRemainingCapacity();
+  }
+
+  @Override
+  public long getTotal() {
+    return getFederationMetrics().getTotalCapacity();
+  }
+
+  @Override
+  public String getSafemode() {
+    // We assume that the global federated view is never in safe mode
+    return "";
+  }
+
+  @Override
+  public boolean isUpgradeFinalized() {
+    // We assume the upgrade is always finalized in a federated biew
+    return true;
+  }
+
+  @Override
+  public RollingUpgradeInfo.Bean getRollingUpgradeStatus() {
+    return null;
+  }
+
+  @Override
+  public long getNonDfsUsedSpace() {
+    return 0;
+  }
+
+  @Override
+  public float getPercentUsed() {
+    return DFSUtilClient.getPercentUsed(getCapacityUsed(), getCapacityTotal());
+  }
+
+  @Override
+  public float getPercentRemaining() {
+    return DFSUtilClient.getPercentUsed(
+        getCapacityRemaining(), getCapacityTotal());
+  }
+
+  @Override
+  public long getCacheUsed() {
+    return 0;
+  }
+
+  @Override
+  public long getCacheCapacity() {
+    return 0;
+  }
+
+  @Override
+  public long getBlockPoolUsedSpace() {
+    return 0;
+  }
+
+  @Override
+  public float getPercentBlockPoolUsed() {
+    return 0;
+  }
+
+  @Override
+  public long getTotalBlocks() {
+    return getFederationMetrics().getNumBlocks();
+  }
+
+  @Override
+  public long getNumberOfMissingBlocks() {
+    return getFederationMetrics().getNumOfMissingBlocks();
+  }
+
+  @Override
+  @Deprecated
+  public long getPendingReplicationBlocks() {
+    return getFederationMetrics().getNumOfBlocksPendingReplication();
+  }
+
+  @Override
+  public long getPendingReconstructionBlocks() {
+    return getFederationMetrics().getNumOfBlocksPendingReplication();
+  }
+
+  @Override
+  @Deprecated
+  public long getUnderReplicatedBlocks() {
+    return getFederationMetrics().getNumOfBlocksUnderReplicated();
+  }
+
+  @Override
+  public long getLowRedundancyBlocks() {
+    return getFederationMetrics().getNumOfBlocksUnderReplicated();
+  }
+
+  @Override
+  public long getPendingDeletionBlocks() {
+    return getFederationMetrics().getNumOfBlocksPendingDeletion();
+  }
+
+  @Override
+  public long getScheduledReplicationBlocks() {
+    return -1;
+  }
+
+  @Override
+  public long getNumberOfMissingBlocksWithReplicationFactorOne() {
+    return 0;
+  }
+
+  @Override
+  public String getCorruptFiles() {
+    return "N/A";
+  }
+
+  @Override
+  public int getThreads() {
+    return ManagementFactory.getThreadMXBean().getThreadCount();
+  }
+
+  @Override
+  public String getLiveNodes() {
+    return this.getNodes(DatanodeReportType.LIVE);
+  }
+
+  @Override
+  public String getDeadNodes() {
+    return this.getNodes(DatanodeReportType.DEAD);
+  }
+
+  @Override
+  public String getDecomNodes() {
+    return this.getNodes(DatanodeReportType.DECOMMISSIONING);
+  }
+
+  /**
+   * Get all the nodes in the federation from a particular type.
+   * TODO this is expensive, we may want to cache it.
+   * @param type Type of the datanodes to check.
+   * @return JSON with the nodes.
+   */
+  private String getNodes(DatanodeReportType type) {
+    final Map<String, Map<String, Object>> info = new HashMap<>();
+    try {
+      RouterRpcServer rpcServer = this.router.getRpcServer();
+      DatanodeInfo[] datanodes = rpcServer.getDatanodeReport(type);
+      for (DatanodeInfo node : datanodes) {
+        Map<String, Object> innerinfo = new HashMap<>();
+        innerinfo.put("infoAddr", node.getInfoAddr());
+        innerinfo.put("infoSecureAddr", node.getInfoSecureAddr());
+        innerinfo.put("xferaddr", node.getXferAddr());
+        innerinfo.put("location", node.getNetworkLocation());
+        innerinfo.put("lastContact", getLastContact(node));
+        innerinfo.put("usedSpace", node.getDfsUsed());
+        innerinfo.put("adminState", node.getAdminState().toString());
+        innerinfo.put("nonDfsUsedSpace", node.getNonDfsUsed());
+        innerinfo.put("capacity", node.getCapacity());
+        innerinfo.put("numBlocks", -1); // node.numBlocks()
+        innerinfo.put("version", (node.getSoftwareVersion() == null ?
+                        "UNKNOWN" : node.getSoftwareVersion()));
+        innerinfo.put("used", node.getDfsUsed());
+        innerinfo.put("remaining", node.getRemaining());
+        innerinfo.put("blockScheduled", -1); // node.getBlocksScheduled()
+        innerinfo.put("blockPoolUsed", node.getBlockPoolUsed());
+        innerinfo.put("blockPoolUsedPercent", node.getBlockPoolUsedPercent());
+        innerinfo.put("volfails", -1); // node.getVolumeFailures()
+        info.put(node.getHostName() + ":" + node.getXferPort(),
+            Collections.unmodifiableMap(innerinfo));
+      }
+    } catch (StandbyException e) {
+      LOG.error("Cannot get {} nodes, Router in safe mode", type);
+    } catch (IOException e) {
+      LOG.error("Cannot get " + type + " nodes", e);
+    }
+    return JSON.toString(info);
+  }
+
+  @Override
+  public String getClusterId() {
+    try {
+      return getNamespaceInfo(FederationNamespaceInfo::getClusterId).toString();
+    } catch (IOException e) {
+      LOG.error("Cannot fetch cluster ID metrics {}", e.getMessage());
+      return "";
+    }
+  }
+
+  @Override
+  public String getBlockPoolId() {
+    try {
+      return
+          getNamespaceInfo(FederationNamespaceInfo::getBlockPoolId).toString();
+    } catch (IOException e) {
+      LOG.error("Cannot fetch block pool ID metrics {}", e.getMessage());
+      return "";
+    }
+  }
+
+  /**
+   * Build a set of unique values found in all namespaces.
+   *
+   * @param f Method reference of the appropriate FederationNamespaceInfo
+   *          getter function
+   * @return Set of unique string values found in all discovered namespaces.
+   * @throws IOException if the query could not be executed.
+   */
+  private Collection<String> getNamespaceInfo(
+      Function<FederationNamespaceInfo, String> f) throws IOException {
+    StateStoreService stateStore = router.getStateStore();
+    MembershipStore membershipStore =
+        stateStore.getRegisteredRecordStore(MembershipStore.class);
+
+    GetNamespaceInfoRequest request = GetNamespaceInfoRequest.newInstance();
+    GetNamespaceInfoResponse response =
+        membershipStore.getNamespaceInfo(request);
+    return response.getNamespaceInfo().stream()
+      .map(f)
+      .collect(Collectors.toSet());
+  }
+
+  @Override
+  public String getNameDirStatuses() {
+    return "N/A";
+  }
+
+  @Override
+  public String getNodeUsage() {
+    return "N/A";
+  }
+
+  @Override
+  public String getNameJournalStatus() {
+    return "N/A";
+  }
+
+  @Override
+  public String getJournalTransactionInfo() {
+    return "N/A";
+  }
+
+  @Override
+  public long getNNStartedTimeInMillis() {
+    return this.router.getStartTime();
+  }
+
+  @Override
+  public String getCompileInfo() {
+    return VersionInfo.getDate() + " by " + VersionInfo.getUser() +
+        " from " + VersionInfo.getBranch();
+  }
+
+  @Override
+  public int getDistinctVersionCount() {
+    return 0;
+  }
+
+  @Override
+  public Map<String, Integer> getDistinctVersions() {
+    return null;
+  }
+
+  /////////////////////////////////////////////////////////
+  // FSNamesystemMBean
+  /////////////////////////////////////////////////////////
+
+  @Override
+  public String getFSState() {
+    // We assume is not in safe mode
+    return "Operational";
+  }
+
+  @Override
+  public long getBlocksTotal() {
+    return this.getTotalBlocks();
+  }
+
+  @Override
+  public long getCapacityTotal() {
+    return this.getTotal();
+  }
+
+  @Override
+  public long getCapacityRemaining() {
+    return this.getFree();
+  }
+
+  @Override
+  public long getCapacityUsed() {
+    return this.getUsed();
+  }
+
+  @Override
+  public long getFilesTotal() {
+    return getFederationMetrics().getNumFiles();
+  }
+
+  @Override
+  public int getTotalLoad() {
+    return -1;
+  }
+
+  @Override
+  public int getNumLiveDataNodes() {
+    return this.router.getMetrics().getNumLiveNodes();
+  }
+
+  @Override
+  public int getNumDeadDataNodes() {
+    return this.router.getMetrics().getNumDeadNodes();
+  }
+
+  @Override
+  public int getNumStaleDataNodes() {
+    return -1;
+  }
+
+  @Override
+  public int getNumDecomLiveDataNodes() {
+    return this.router.getMetrics().getNumDecomLiveNodes();
+  }
+
+  @Override
+  public int getNumDecomDeadDataNodes() {
+    return this.router.getMetrics().getNumDecomDeadNodes();
+  }
+
+  @Override
+  public int getNumDecommissioningDataNodes() {
+    return this.router.getMetrics().getNumDecommissioningNodes();
+  }
+
+  @Override
+  public int getNumInMaintenanceLiveDataNodes() {
+    return 0;
+  }
+
+  @Override
+  public int getNumInMaintenanceDeadDataNodes() {
+    return 0;
+  }
+
+  @Override
+  public int getNumEnteringMaintenanceDataNodes() {
+    return 0;
+  }
+
+  @Override
+  public int getVolumeFailuresTotal() {
+    return 0;
+  }
+
+  @Override
+  public long getEstimatedCapacityLostTotal() {
+    return 0;
+  }
+
+  @Override
+  public String getSnapshotStats() {
+    return null;
+  }
+
+  @Override
+  public long getMaxObjects() {
+    return 0;
+  }
+
+  @Override
+  public long getBlockDeletionStartTime() {
+    return -1;
+  }
+
+  @Override
+  public int getNumStaleStorages() {
+    return -1;
+  }
+
+  @Override
+  public String getTopUserOpCounts() {
+    return "N/A";
+  }
+
+  @Override
+  public int getFsLockQueueLength() {
+    return 0;
+  }
+
+  @Override
+  public long getTotalSyncCount() {
+    return 0;
+  }
+
+  @Override
+  public String getTotalSyncTimes() {
+    return "";
+  }
+
+  private long getLastContact(DatanodeInfo node) {
+    return (now() - node.getLastUpdate()) / 1000;
+  }
+
+  /////////////////////////////////////////////////////////
+  // NameNodeStatusMXBean
+  /////////////////////////////////////////////////////////
+
+  @Override
+  public String getNNRole() {
+    return NamenodeRole.NAMENODE.toString();
+  }
+
+  @Override
+  public String getState() {
+    return HAServiceState.ACTIVE.toString();
+  }
+
+  @Override
+  public String getHostAndPort() {
+    return NetUtils.getHostPortString(router.getRpcServerAddress());
+  }
+
+  @Override
+  public boolean isSecurityEnabled() {
+    return false;
+  }
+
+  @Override
+  public long getLastHATransitionTime() {
+    return 0;
+  }
+
+  @Override
+  public long getBytesWithFutureGenerationStamps() {
+    return 0;
+  }
+
+  @Override
+  public String getSlowPeersReport() {
+    return "N/A";
+  }
+
+  @Override
+  public String getSlowDisksReport() {
+    return "N/A";
+  }
+
+  @Override
+  public long getNumberOfSnapshottableDirs() {
+    return 0;
+  }
+
+  @Override
+  public String getEnteringMaintenanceNodes() {
+    return "N/A";
+  }
+
+  @Override
+  public String getNameDirSize() {
+    return "N/A";
+  }
+
+  @Override
+  public int getNumEncryptionZones() {
+    return 0;
+  }
+}

+ 45 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/StateStoreMBean.java

@@ -0,0 +1,45 @@
+/**
+ * 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.server.federation.metrics;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * JMX interface for the State Store metrics.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public interface StateStoreMBean {
+
+  long getReadOps();
+
+  double getReadAvg();
+
+  long getWriteOps();
+
+  double getWriteAvg();
+
+  long getFailureOps();
+
+  double getFailureAvg();
+
+  long getRemoveOps();
+
+  double getRemoveAvg();
+}

+ 144 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/StateStoreMetrics.java

@@ -0,0 +1,144 @@
+/**
+ * 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.server.federation.metrics;
+
+import static org.apache.hadoop.metrics2.impl.MsInfo.ProcessName;
+import static org.apache.hadoop.metrics2.impl.MsInfo.SessionId;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.metrics2.MetricsSystem;
+import org.apache.hadoop.metrics2.annotation.Metric;
+import org.apache.hadoop.metrics2.annotation.Metrics;
+import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
+import org.apache.hadoop.metrics2.lib.MetricsRegistry;
+import org.apache.hadoop.metrics2.lib.MutableGaugeInt;
+import org.apache.hadoop.metrics2.lib.MutableRate;
+
+import com.google.common.annotations.VisibleForTesting;
+
+/**
+ * Implementations of the JMX interface for the State Store metrics.
+ */
+@Metrics(name = "StateStoreActivity", about = "Router metrics",
+    context = "router")
+public final class StateStoreMetrics implements StateStoreMBean {
+
+  private final MetricsRegistry registry = new MetricsRegistry("router");
+
+  @Metric("GET transactions")
+  private MutableRate reads;
+  @Metric("PUT transactions")
+  private MutableRate writes;
+  @Metric("REMOVE transactions")
+  private MutableRate removes;
+  @Metric("Failed transactions")
+  private MutableRate failures;
+
+  private Map<String, MutableGaugeInt> cacheSizes;
+
+  private StateStoreMetrics(Configuration conf) {
+    registry.tag(SessionId, "RouterSession");
+    registry.tag(ProcessName, "Router");
+    cacheSizes = new HashMap<>();
+  }
+
+  public static StateStoreMetrics create(Configuration conf) {
+    MetricsSystem ms = DefaultMetricsSystem.instance();
+    return ms.register(new StateStoreMetrics(conf));
+  }
+
+  public void shutdown() {
+    DefaultMetricsSystem.shutdown();
+    reset();
+  }
+
+  public void addRead(long latency) {
+    reads.add(latency);
+  }
+
+  public long getReadOps() {
+    return reads.lastStat().numSamples();
+  }
+
+  public double getReadAvg() {
+    return reads.lastStat().mean();
+  }
+
+  public void addWrite(long latency) {
+    writes.add(latency);
+  }
+
+  public long getWriteOps() {
+    return writes.lastStat().numSamples();
+  }
+
+  public double getWriteAvg() {
+    return writes.lastStat().mean();
+  }
+
+  public void addFailure(long latency) {
+    failures.add(latency);
+  }
+
+  public long getFailureOps() {
+    return failures.lastStat().numSamples();
+  }
+
+  public double getFailureAvg() {
+    return failures.lastStat().mean();
+  }
+
+  public void addRemove(long latency) {
+    removes.add(latency);
+  }
+
+  public long getRemoveOps() {
+    return removes.lastStat().numSamples();
+  }
+
+  public double getRemoveAvg() {
+    return removes.lastStat().mean();
+  }
+
+  /**
+   * Set the size of the cache for a State Store interface.
+   *
+   * @param name Name of the record to cache.
+   * @param size Number of records.
+   */
+  public void setCacheSize(String name, int size) {
+    String counterName = "Cache" + name + "Size";
+    MutableGaugeInt counter = cacheSizes.get(counterName);
+    if (counter == null) {
+      counter = registry.newGauge(counterName, name, size);
+      cacheSizes.put(counterName, counter);
+    }
+    counter.set(size);
+  }
+
+  @VisibleForTesting
+  public void reset() {
+    reads.resetMinMax();
+    writes.resetMinMax();
+    removes.resetMinMax();
+    failures.resetMinMax();
+  }
+}

+ 27 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/package-info.java

@@ -0,0 +1,27 @@
+/**
+ * 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.
+ */
+
+/**
+ * Report metrics for Router-based Federation.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+package org.apache.hadoop.hdfs.server.federation.metrics;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;

+ 117 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/ActiveNamenodeResolver.java

@@ -0,0 +1,117 @@
+/**
+ * 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.server.federation.resolver;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Locates the most active NN for a given nameservice ID or blockpool ID. This
+ * interface is used by the {@link org.apache.hadoop.hdfs.server.federation.
+ * router.RouterRpcServer RouterRpcServer} to:
+ * <ul>
+ * <li>Determine the target NN for a given subcluster.
+ * <li>List of all namespaces discovered/active in the federation.
+ * <li>Update the currently active NN empirically.
+ * </ul>
+ * The interface is also used by the {@link org.apache.hadoop.hdfs.server.
+ * federation.router.NamenodeHeartbeatService NamenodeHeartbeatService} to
+ * register a discovered NN.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public interface ActiveNamenodeResolver {
+
+  /**
+   * Report a successful, active NN address for a nameservice or blockPool.
+   *
+   * @param ns Nameservice identifier.
+   * @param successfulAddress The address the successful responded to the
+   *                          command.
+   * @throws IOException If the state store cannot be accessed.
+   */
+  void updateActiveNamenode(
+      String ns, InetSocketAddress successfulAddress) throws IOException;
+
+  /**
+   * Returns a prioritized list of the most recent cached registration entries
+   * for a single nameservice ID.
+   * Returns an empty list if none are found. Returns entries in preference of:
+   * <ul>
+   * <li>The most recent ACTIVE NN
+   * <li>The most recent STANDBY NN
+   * <li>The most recent UNAVAILABLE NN
+   * </ul>
+   *
+   * @param nameserviceId Nameservice identifier.
+   * @return Prioritized list of namenode contexts.
+   * @throws IOException If the state store cannot be accessed.
+   */
+  List<? extends FederationNamenodeContext>
+      getNamenodesForNameserviceId(String nameserviceId) throws IOException;
+
+  /**
+   * Returns a prioritized list of the most recent cached registration entries
+   * for a single block pool ID.
+   * Returns an empty list if none are found. Returns entries in preference of:
+   * <ul>
+   * <li>The most recent ACTIVE NN
+   * <li>The most recent STANDBY NN
+   * <li>The most recent UNAVAILABLE NN
+   * </ul>
+   *
+   * @param blockPoolId Block pool identifier for the nameservice.
+   * @return Prioritized list of namenode contexts.
+   * @throws IOException If the state store cannot be accessed.
+   */
+  List<? extends FederationNamenodeContext>
+      getNamenodesForBlockPoolId(String blockPoolId) throws IOException;
+
+  /**
+   * Register a namenode in the State Store.
+   *
+   * @param report Namenode status report.
+   * @return True if the node was registered and successfully committed to the
+   *         data store.
+   * @throws IOException Throws exception if the namenode could not be
+   *           registered.
+   */
+  boolean registerNamenode(NamenodeStatusReport report) throws IOException;
+
+  /**
+   * Get a list of all namespaces that are registered and active in the
+   * federation.
+   *
+   * @return List of name spaces in the federation
+   * @throws Throws exception if the namespace list is not available.
+   */
+  Set<FederationNamespaceInfo> getNamespaces() throws IOException;
+
+  /**
+   * Assign a unique identifier for the parent router service.
+   * Required to report the status to the namenode resolver.
+   *
+   * @param router Unique string identifier for the router.
+   */
+  void setRouterId(String routerId);
+}

+ 87 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/FederationNamenodeContext.java

@@ -0,0 +1,87 @@
+/**
+ * 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.server.federation.resolver;
+
+/**
+ * Interface for a discovered NN and its current server endpoints.
+ */
+public interface FederationNamenodeContext {
+
+  /**
+   * Get the RPC server address of the namenode.
+   *
+   * @return RPC server address in the form of host:port.
+   */
+  String getRpcAddress();
+
+  /**
+   * Get the Service RPC server address of the namenode.
+   *
+   * @return Service RPC server address in the form of host:port.
+   */
+  String getServiceAddress();
+
+  /**
+   * Get the Lifeline RPC server address of the namenode.
+   *
+   * @return Lifeline RPC server address in the form of host:port.
+   */
+  String getLifelineAddress();
+
+  /**
+   * Get the HTTP server address of the namenode.
+   *
+   * @return HTTP address in the form of host:port.
+   */
+  String getWebAddress();
+
+  /**
+   * Get the unique key representing the namenode.
+   *
+   * @return Combination of the nameservice and the namenode IDs.
+   */
+  String getNamenodeKey();
+
+  /**
+   * Identifier for the nameservice/namespace.
+   *
+   * @return Namenode nameservice identifier.
+   */
+  String getNameserviceId();
+
+  /**
+   * Identifier for the namenode.
+   *
+   * @return String
+   */
+  String getNamenodeId();
+
+  /**
+   * The current state of the namenode (active, standby, etc).
+   *
+   * @return FederationNamenodeServiceState State of the namenode.
+   */
+  FederationNamenodeServiceState getState();
+
+  /**
+   * The update date.
+   *
+   * @return Long with the update date.
+   */
+  long getDateModified();
+}

+ 46 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/FederationNamenodeServiceState.java

@@ -0,0 +1,46 @@
+/**
+ * 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.server.federation.resolver;
+
+import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
+
+/**
+ * Namenode state in the federation. The order of this enum is used to evaluate
+ * NN priority for RPC calls.
+ */
+public enum FederationNamenodeServiceState {
+  ACTIVE, // HAServiceState.ACTIVE or operational.
+  STANDBY, // HAServiceState.STANDBY.
+  UNAVAILABLE, // When the namenode cannot be reached.
+  EXPIRED; // When the last update is too old.
+
+  public static FederationNamenodeServiceState getState(HAServiceState state) {
+    switch(state) {
+    case ACTIVE:
+      return FederationNamenodeServiceState.ACTIVE;
+    case STANDBY:
+      return FederationNamenodeServiceState.STANDBY;
+    case INITIALIZING:
+      return FederationNamenodeServiceState.UNAVAILABLE;
+    case STOPPING:
+      return FederationNamenodeServiceState.UNAVAILABLE;
+    default:
+      return FederationNamenodeServiceState.UNAVAILABLE;
+    }
+  }
+}

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

@@ -0,0 +1,73 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.federation.resolver;
+
+import org.apache.hadoop.hdfs.server.federation.router.RemoteLocationContext;
+
+/**
+ * Represents information about a single nameservice/namespace in a federated
+ * HDFS cluster.
+ */
+public class FederationNamespaceInfo extends RemoteLocationContext {
+
+  /** Block pool identifier. */
+  private final String blockPoolId;
+  /** Cluster identifier. */
+  private final String clusterId;
+  /** Nameservice identifier. */
+  private final String nameserviceId;
+
+  public FederationNamespaceInfo(String bpId, String clId, String nsId) {
+    this.blockPoolId = bpId;
+    this.clusterId = clId;
+    this.nameserviceId = nsId;
+  }
+
+  @Override
+  public String getNameserviceId() {
+    return this.nameserviceId;
+  }
+
+  @Override
+  public String getDest() {
+    return this.nameserviceId;
+  }
+
+  /**
+   * The HDFS cluster id for this namespace.
+   *
+   * @return Cluster identifier.
+   */
+  public String getClusterId() {
+    return this.clusterId;
+  }
+
+  /**
+   * The HDFS block pool id for this namespace.
+   *
+   * @return Block pool identifier.
+   */
+  public String getBlockPoolId() {
+    return this.blockPoolId;
+  }
+
+  @Override
+  public String toString() {
+    return this.nameserviceId + "->" + this.blockPoolId + ":" + this.clusterId;
+  }
+}

+ 75 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/FileSubclusterResolver.java

@@ -0,0 +1,75 @@
+/**
+ * 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.server.federation.resolver;
+
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Interface to map a file path in the global name space to a specific
+ * subcluster and path in an HDFS name space.
+ * <p>
+ * Each path in the global/federated namespace may map to 1-N different HDFS
+ * locations.  Each location specifies a single nameservice and a single HDFS
+ * path.  The behavior is similar to MergeFS and Nfly and allows the merger
+ * of multiple HDFS locations into a single path.  See HADOOP-8298 and
+ * HADOOP-12077
+ * <p>
+ * For example, a directory listing will fetch listings for each destination
+ * path and combine them into a single set of results.
+ * <p>
+ * When multiple destinations are available for a path, the destinations are
+ * prioritized in a consistent manner.  This allows the proxy server to
+ * guess the best/most likely destination and attempt it first.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public interface FileSubclusterResolver {
+
+  /**
+   * Get the destinations for a global path. Results are from the mount table
+   * cache.  If multiple destinations are available, the first result is the
+   * highest priority destination.
+   *
+   * @param path Global path.
+   * @return Location in a destination namespace or null if it does not exist.
+   * @throws IOException Throws exception if the data is not available.
+   */
+  PathLocation getDestinationForPath(String path) throws IOException;
+
+  /**
+   * Get a list of mount points for a path. Results are from the mount table
+   * cache.
+   *
+   * @return List of mount points present at this path or zero-length list if
+   *         none are found.
+   * @throws IOException Throws exception if the data is not available.
+   */
+  List<String> getMountPoints(String path) throws IOException;
+
+  /**
+   * Get the default namespace for the cluster.
+   *
+   * @return Default namespace identifier.
+   */
+  String getDefaultNamespace();
+}

+ 317 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/MembershipNamenodeResolver.java

@@ -0,0 +1,317 @@
+/**
+ * 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.server.federation.resolver;
+
+import static org.apache.hadoop.hdfs.server.federation.resolver.FederationNamenodeServiceState.ACTIVE;
+import static org.apache.hadoop.hdfs.server.federation.resolver.FederationNamenodeServiceState.EXPIRED;
+import static org.apache.hadoop.hdfs.server.federation.resolver.FederationNamenodeServiceState.UNAVAILABLE;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.server.federation.store.MembershipStore;
+import org.apache.hadoop.hdfs.server.federation.store.StateStoreCache;
+import org.apache.hadoop.hdfs.server.federation.store.StateStoreService;
+import org.apache.hadoop.hdfs.server.federation.store.StateStoreUnavailableException;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.GetNamenodeRegistrationsRequest;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.GetNamenodeRegistrationsResponse;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.GetNamespaceInfoRequest;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.GetNamespaceInfoResponse;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.NamenodeHeartbeatRequest;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.UpdateNamenodeRegistrationRequest;
+import org.apache.hadoop.hdfs.server.federation.store.records.MembershipState;
+import org.apache.hadoop.hdfs.server.federation.store.records.MembershipStats;
+import org.apache.hadoop.util.Time;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Implements a cached lookup of the most recently active namenode for a
+ * particular nameservice. Relies on the {@link StateStoreService} to
+ * discover available nameservices and namenodes.
+ */
+public class MembershipNamenodeResolver
+    implements ActiveNamenodeResolver, StateStoreCache {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(MembershipNamenodeResolver.class);
+
+  /** Reference to the State Store. */
+  private final StateStoreService stateStore;
+  /** Membership State Store interface. */
+  private MembershipStore membershipInterface;
+
+  /** Parent router ID. */
+  private String routerId;
+
+  /** Cached lookup of NN for nameservice. Invalidated on cache refresh. */
+  private Map<String, List<? extends FederationNamenodeContext>> cacheNS;
+  /** Cached lookup of NN for block pool. Invalidated on cache refresh. */
+  private Map<String, List<? extends FederationNamenodeContext>> cacheBP;
+
+
+  public MembershipNamenodeResolver(
+      Configuration conf, StateStoreService store) throws IOException {
+    this.stateStore = store;
+
+    this.cacheNS = new ConcurrentHashMap<>();
+    this.cacheBP = new ConcurrentHashMap<>();
+
+    if (this.stateStore != null) {
+      // Request cache updates from the state store
+      this.stateStore.registerCacheExternal(this);
+    }
+  }
+
+  private synchronized MembershipStore getMembershipStore() throws IOException {
+    if (this.membershipInterface == null) {
+      this.membershipInterface = this.stateStore.getRegisteredRecordStore(
+          MembershipStore.class);
+      if (this.membershipInterface == null) {
+        throw new IOException("State Store does not have an interface for " +
+            MembershipStore.class.getSimpleName());
+      }
+    }
+    return this.membershipInterface;
+  }
+
+  @Override
+  public boolean loadCache(boolean force) {
+    // Our cache depends on the store, update it first
+    try {
+      MembershipStore membership = getMembershipStore();
+      membership.loadCache(force);
+    } catch (IOException e) {
+      LOG.error("Cannot update membership from the State Store", e);
+    }
+
+    // Force refresh of active NN cache
+    cacheBP.clear();
+    cacheNS.clear();
+    return true;
+  }
+
+  @Override
+  public void updateActiveNamenode(
+      final String nsId, final InetSocketAddress address) throws IOException {
+
+    // Called when we have an RPC miss and successful hit on an alternate NN.
+    // Temporarily update our cache, it will be overwritten on the next update.
+    try {
+      MembershipState partial = MembershipState.newInstance();
+      String rpcAddress = address.getHostName() + ":" + address.getPort();
+      partial.setRpcAddress(rpcAddress);
+      partial.setNameserviceId(nsId);
+
+      GetNamenodeRegistrationsRequest request =
+          GetNamenodeRegistrationsRequest.newInstance(partial);
+
+      MembershipStore membership = getMembershipStore();
+      GetNamenodeRegistrationsResponse response =
+          membership.getNamenodeRegistrations(request);
+      List<MembershipState> records = response.getNamenodeMemberships();
+
+      if (records != null && records.size() == 1) {
+        MembershipState record = records.get(0);
+        UpdateNamenodeRegistrationRequest updateRequest =
+            UpdateNamenodeRegistrationRequest.newInstance(
+                record.getNameserviceId(), record.getNamenodeId(), ACTIVE);
+        membership.updateNamenodeRegistration(updateRequest);
+      }
+    } catch (StateStoreUnavailableException e) {
+      LOG.error("Cannot update {} as active, State Store unavailable", address);
+    }
+  }
+
+  @Override
+  public List<? extends FederationNamenodeContext> getNamenodesForNameserviceId(
+      final String nsId) throws IOException {
+
+    List<? extends FederationNamenodeContext> ret = cacheNS.get(nsId);
+    if (ret == null) {
+      try {
+        MembershipState partial = MembershipState.newInstance();
+        partial.setNameserviceId(nsId);
+        GetNamenodeRegistrationsRequest request =
+            GetNamenodeRegistrationsRequest.newInstance(partial);
+
+        final List<MembershipState> result =
+            getRecentRegistrationForQuery(request, true, false);
+        if (result == null || result.isEmpty()) {
+          LOG.error("Cannot locate eligible NNs for {}", nsId);
+          return null;
+        } else {
+          cacheNS.put(nsId, result);
+          ret = result;
+        }
+      } catch (StateStoreUnavailableException e) {
+        LOG.error("Cannot get active NN for {}, State Store unavailable", nsId);
+      }
+    }
+    if (ret == null) {
+      return null;
+    }
+    return Collections.unmodifiableList(ret);
+  }
+
+  @Override
+  public List<? extends FederationNamenodeContext> getNamenodesForBlockPoolId(
+      final String bpId) throws IOException {
+
+    List<? extends FederationNamenodeContext> ret = cacheBP.get(bpId);
+    if (ret == null) {
+      try {
+        MembershipState partial = MembershipState.newInstance();
+        partial.setBlockPoolId(bpId);
+        GetNamenodeRegistrationsRequest request =
+            GetNamenodeRegistrationsRequest.newInstance(partial);
+
+        final List<MembershipState> result =
+            getRecentRegistrationForQuery(request, true, false);
+        if (result == null || result.isEmpty()) {
+          LOG.error("Cannot locate eligible NNs for {}", bpId);
+        } else {
+          cacheBP.put(bpId, result);
+          ret = result;
+        }
+      } catch (StateStoreUnavailableException e) {
+        LOG.error("Cannot get active NN for {}, State Store unavailable", bpId);
+        return null;
+      }
+    }
+    if (ret == null) {
+      return null;
+    }
+    return Collections.unmodifiableList(ret);
+  }
+
+  @Override
+  public boolean registerNamenode(NamenodeStatusReport report)
+      throws IOException {
+
+    if (this.routerId == null) {
+      LOG.warn("Cannot register namenode, router ID is not known {}", report);
+      return false;
+    }
+
+    MembershipState record = MembershipState.newInstance(
+        routerId, report.getNameserviceId(), report.getNamenodeId(),
+        report.getClusterId(), report.getBlockPoolId(), report.getRpcAddress(),
+        report.getServiceAddress(), report.getLifelineAddress(),
+        report.getWebAddress(), report.getState(), report.getSafemode());
+
+    if (report.statsValid()) {
+      MembershipStats stats = MembershipStats.newInstance();
+      stats.setNumOfFiles(report.getNumFiles());
+      stats.setNumOfBlocks(report.getNumBlocks());
+      stats.setNumOfBlocksMissing(report.getNumBlocksMissing());
+      stats.setNumOfBlocksPendingReplication(
+          report.getNumOfBlocksPendingReplication());
+      stats.setNumOfBlocksUnderReplicated(
+          report.getNumOfBlocksUnderReplicated());
+      stats.setNumOfBlocksPendingDeletion(
+          report.getNumOfBlocksPendingDeletion());
+      stats.setAvailableSpace(report.getAvailableSpace());
+      stats.setTotalSpace(report.getTotalSpace());
+      stats.setNumOfDecommissioningDatanodes(
+          report.getNumDecommissioningDatanodes());
+      stats.setNumOfActiveDatanodes(report.getNumLiveDatanodes());
+      stats.setNumOfDeadDatanodes(report.getNumDeadDatanodes());
+      stats.setNumOfDecomActiveDatanodes(report.getNumDecomLiveDatanodes());
+      stats.setNumOfDecomDeadDatanodes(report.getNumDecomDeadDatanodes());
+      record.setStats(stats);
+    }
+
+    if (report.getState() != UNAVAILABLE) {
+      // Set/update our last contact time
+      record.setLastContact(Time.now());
+    }
+
+    NamenodeHeartbeatRequest request = NamenodeHeartbeatRequest.newInstance();
+    request.setNamenodeMembership(record);
+    return getMembershipStore().namenodeHeartbeat(request).getResult();
+  }
+
+  @Override
+  public Set<FederationNamespaceInfo> getNamespaces() throws IOException {
+    GetNamespaceInfoRequest request = GetNamespaceInfoRequest.newInstance();
+    GetNamespaceInfoResponse response =
+        getMembershipStore().getNamespaceInfo(request);
+    return response.getNamespaceInfo();
+  }
+
+  /**
+   * Picks the most relevant record registration that matches the query. Return
+   * registrations matching the query in this preference: 1) Most recently
+   * updated ACTIVE registration 2) Most recently updated STANDBY registration
+   * (if showStandby) 3) Most recently updated UNAVAILABLE registration (if
+   * showUnavailable). EXPIRED registrations are ignored.
+   *
+   * @param query The select query for NN registrations.
+   * @param excludes List of NNs to exclude from matching results.
+   * @param addUnavailable include UNAVAILABLE registrations.
+   * @param addExpired include EXPIRED registrations.
+   * @return List of memberships or null if no registrations that
+   *         both match the query AND the selected states.
+   * @throws IOException
+   */
+  private List<MembershipState> getRecentRegistrationForQuery(
+      GetNamenodeRegistrationsRequest request, boolean addUnavailable,
+      boolean addExpired) throws IOException {
+
+    // Retrieve a list of all registrations that match this query.
+    // This may include all NN records for a namespace/blockpool, including
+    // duplicate records for the same NN from different routers.
+    MembershipStore membershipStore = getMembershipStore();
+    GetNamenodeRegistrationsResponse response =
+        membershipStore.getNamenodeRegistrations(request);
+
+    List<MembershipState> memberships = response.getNamenodeMemberships();
+    if (!addExpired || !addUnavailable) {
+      Iterator<MembershipState> iterator = memberships.iterator();
+      while (iterator.hasNext()) {
+        MembershipState membership = iterator.next();
+        if (membership.getState() == EXPIRED && !addExpired) {
+          iterator.remove();
+        } else if (membership.getState() == UNAVAILABLE && !addUnavailable) {
+          iterator.remove();
+        }
+      }
+    }
+
+    List<MembershipState> priorityList = new ArrayList<>();
+    priorityList.addAll(memberships);
+    Collections.sort(priorityList, new NamenodePriorityComparator());
+
+    LOG.debug("Selected most recent NN {} for query", priorityList);
+    return priorityList;
+  }
+
+  @Override
+  public void setRouterId(String router) {
+    this.routerId = router;
+  }
+}

+ 80 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/MountTableManager.java

@@ -0,0 +1,80 @@
+/**
+ * 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.server.federation.resolver;
+
+import java.io.IOException;
+
+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.protocol.GetMountTableEntriesRequest;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.GetMountTableEntriesResponse;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.RemoveMountTableEntryRequest;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.RemoveMountTableEntryResponse;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.UpdateMountTableEntryRequest;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.UpdateMountTableEntryResponse;
+
+/**
+ * Manage a mount table.
+ */
+public interface MountTableManager {
+
+  /**
+   * Add an entry to the mount table.
+   *
+   * @param request Fully populated request object.
+   * @return True if the mount table entry was successfully committed to the
+   *         data store.
+   * @throws IOException Throws exception if the data store is not initialized.
+   */
+  AddMountTableEntryResponse addMountTableEntry(
+      AddMountTableEntryRequest request) throws IOException;
+
+  /**
+   * Updates an existing entry in the mount table.
+   *
+   * @param request Fully populated request object.
+   * @return True if the mount table entry was successfully committed to the
+   *         data store.
+   * @throws IOException Throws exception if the data store is not initialized.
+   */
+  UpdateMountTableEntryResponse updateMountTableEntry(
+      UpdateMountTableEntryRequest request) throws IOException;
+
+  /**
+   * Remove an entry from the mount table.
+   *
+   * @param request Fully populated request object.
+   * @return True the mount table entry was removed from the data store.
+   * @throws IOException Throws exception if the data store is not initialized.
+   */
+  RemoveMountTableEntryResponse removeMountTableEntry(
+      RemoveMountTableEntryRequest request) throws IOException;
+
+  /**
+   * List all mount table entries present at or below the path. Fetches from the
+   * state store.
+   *
+   * @param request Fully populated request object.
+   *
+   * @return List of all mount table entries under the path. Zero-length list if
+   *         none are found.
+   * @throws IOException Throws exception if the data store cannot be queried.
+   */
+  GetMountTableEntriesResponse getMountTableEntries(
+      GetMountTableEntriesRequest request) throws IOException;
+}

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

@@ -0,0 +1,544 @@
+/**
+ * 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.server.federation.resolver;
+
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_ROUTER_DEFAULT_NAMESERVICE;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import java.util.TreeSet;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentNavigableMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+import org.apache.hadoop.HadoopIllegalArgumentException;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.DFSUtilClient;
+import org.apache.hadoop.hdfs.server.federation.resolver.order.DestinationOrder;
+import org.apache.hadoop.hdfs.server.federation.router.Router;
+import org.apache.hadoop.hdfs.server.federation.store.MountTableStore;
+import org.apache.hadoop.hdfs.server.federation.store.StateStoreCache;
+import org.apache.hadoop.hdfs.server.federation.store.StateStoreService;
+import org.apache.hadoop.hdfs.server.federation.store.StateStoreUnavailableException;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.GetMountTableEntriesRequest;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.GetMountTableEntriesResponse;
+import org.apache.hadoop.hdfs.server.federation.store.records.MountTable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.annotations.VisibleForTesting;
+
+/**
+ * Mount table to map between global paths and remote locations. This allows the
+ * {@link org.apache.hadoop.hdfs.server.federation.router.Router Router} to map
+ * the global HDFS view to the remote namespaces. This is similar to
+ * {@link org.apache.hadoop.fs.viewfs.ViewFs ViewFs}.
+ * This is implemented as a tree.
+ */
+public class MountTableResolver
+    implements FileSubclusterResolver, StateStoreCache {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(MountTableResolver.class);
+
+  /** Reference to Router. */
+  private final Router router;
+  /** Reference to the State Store. */
+  private final StateStoreService stateStore;
+  /** Interface to the mount table store. */
+  private MountTableStore mountTableStore;
+
+  /** If the tree has been initialized. */
+  private boolean init = false;
+  /** Path -> Remote HDFS location. */
+  private final TreeMap<String, MountTable> tree = new TreeMap<>();
+  /** Path -> Remote location. */
+  private final ConcurrentNavigableMap<String, PathLocation> locationCache =
+      new ConcurrentSkipListMap<>();
+
+  /** Default nameservice when no mount matches the math. */
+  private String defaultNameService = "";
+
+  /** Synchronization for both the tree and the cache. */
+  private final ReadWriteLock readWriteLock = new ReentrantReadWriteLock();
+  private final Lock readLock = readWriteLock.readLock();
+  private final Lock writeLock = readWriteLock.writeLock();
+
+
+  @VisibleForTesting
+  public MountTableResolver(Configuration conf) {
+    this(conf, (StateStoreService)null);
+  }
+
+  public MountTableResolver(Configuration conf, Router routerService) {
+    this.router = routerService;
+    if (this.router != null) {
+      this.stateStore = this.router.getStateStore();
+    } else {
+      this.stateStore = null;
+    }
+
+    registerCacheExternal();
+    initDefaultNameService(conf);
+  }
+
+  public MountTableResolver(Configuration conf, StateStoreService store) {
+    this.router = null;
+    this.stateStore = store;
+
+    registerCacheExternal();
+    initDefaultNameService(conf);
+  }
+
+  /**
+   * Request cache updates from the State Store for this resolver.
+   */
+  private void registerCacheExternal() {
+    if (this.stateStore != null) {
+      this.stateStore.registerCacheExternal(this);
+    }
+  }
+
+  /**
+   * Nameservice for APIs that cannot be resolved to a specific one.
+   *
+   * @param conf Configuration for this resolver.
+   */
+  private void initDefaultNameService(Configuration conf) {
+    try {
+      this.defaultNameService = conf.get(
+          DFS_ROUTER_DEFAULT_NAMESERVICE,
+          DFSUtil.getNamenodeNameServiceId(conf));
+    } catch (HadoopIllegalArgumentException e) {
+      LOG.error("Cannot find default name service, setting it to the first");
+      Collection<String> nsIds = DFSUtilClient.getNameServiceIds(conf);
+      this.defaultNameService = nsIds.iterator().next();
+      LOG.info("Default name service: {}", this.defaultNameService);
+    }
+  }
+
+  /**
+   * Get a reference for the Router for this resolver.
+   *
+   * @return Router for this resolver.
+   */
+  protected Router getRouter() {
+    return this.router;
+  }
+
+  /**
+   * Get the mount table store for this resolver.
+   *
+   * @return Mount table store.
+   * @throws IOException If it cannot connect to the State Store.
+   */
+  protected MountTableStore getMountTableStore() throws IOException {
+    if (this.mountTableStore == null) {
+      this.mountTableStore = this.stateStore.getRegisteredRecordStore(
+          MountTableStore.class);
+      if (this.mountTableStore == null) {
+        throw new IOException("State Store does not have an interface for " +
+            MountTableStore.class);
+      }
+    }
+    return this.mountTableStore;
+  }
+
+  /**
+   * Add a mount entry to the table.
+   *
+   * @param entry The mount table record to add from the state store.
+   */
+  public void addEntry(final MountTable entry) {
+    writeLock.lock();
+    try {
+      String srcPath = entry.getSourcePath();
+      this.tree.put(srcPath, entry);
+      invalidateLocationCache(srcPath);
+    } finally {
+      writeLock.unlock();
+    }
+    this.init = true;
+  }
+
+  /**
+   * Remove a mount table entry.
+   *
+   * @param srcPath Source path for the entry to remove.
+   */
+  public void removeEntry(final String srcPath) {
+    writeLock.lock();
+    try {
+      this.tree.remove(srcPath);
+      invalidateLocationCache(srcPath);
+    } finally {
+      writeLock.unlock();
+    }
+  }
+
+  /**
+   * Invalidates all cache entries below this path. It requires the write lock.
+   *
+   * @param src Source path.
+   */
+  private void invalidateLocationCache(final String path) {
+    if (locationCache.isEmpty()) {
+      return;
+    }
+    // Determine next lexicographic entry after source path
+    String nextSrc = path + Character.MAX_VALUE;
+    ConcurrentNavigableMap<String, PathLocation> subMap =
+        locationCache.subMap(path, nextSrc);
+    for (final String key : subMap.keySet()) {
+      locationCache.remove(key);
+    }
+  }
+
+  /**
+   * Updates the mount path tree with a new set of mount table entries. It also
+   * updates the needed caches.
+   *
+   * @param entries Full set of mount table entries to update.
+   */
+  @VisibleForTesting
+  public void refreshEntries(final Collection<MountTable> entries) {
+    // The tree read/write must be atomic
+    writeLock.lock();
+    try {
+      // New entries
+      Map<String, MountTable> newEntries = new ConcurrentHashMap<>();
+      for (MountTable entry : entries) {
+        String srcPath = entry.getSourcePath();
+        newEntries.put(srcPath, entry);
+      }
+
+      // Old entries (reversed to sort from the leaves to the root)
+      Set<String> oldEntries = new TreeSet<>(Collections.reverseOrder());
+      for (MountTable entry : getTreeValues("/")) {
+        String srcPath = entry.getSourcePath();
+        oldEntries.add(srcPath);
+      }
+
+      // Entries that need to be removed
+      for (String srcPath : oldEntries) {
+        if (!newEntries.containsKey(srcPath)) {
+          this.tree.remove(srcPath);
+          invalidateLocationCache(srcPath);
+          LOG.info("Removed stale mount point {} from resolver", srcPath);
+        }
+      }
+
+      // Entries that need to be added
+      for (MountTable entry : entries) {
+        String srcPath = entry.getSourcePath();
+        if (!oldEntries.contains(srcPath)) {
+          // Add node, it does not exist
+          this.tree.put(srcPath, entry);
+          LOG.info("Added new mount point {} to resolver", srcPath);
+        } else {
+          // Node exists, check for updates
+          MountTable existingEntry = this.tree.get(srcPath);
+          if (existingEntry != null && !existingEntry.equals(entry)) {
+            // Entry has changed
+            invalidateLocationCache(srcPath);
+            LOG.info("Updated mount point {} in resolver");
+          }
+        }
+      }
+    } finally {
+      writeLock.unlock();
+    }
+    this.init = true;
+  }
+
+  /**
+   * Replaces the current in-memory cached of the mount table with a new
+   * version fetched from the data store.
+   */
+  @Override
+  public boolean loadCache(boolean force) {
+    try {
+      // Our cache depends on the store, update it first
+      MountTableStore mountTable = this.getMountTableStore();
+      mountTable.loadCache(force);
+
+      GetMountTableEntriesRequest request =
+          GetMountTableEntriesRequest.newInstance("/");
+      GetMountTableEntriesResponse response =
+          mountTable.getMountTableEntries(request);
+      List<MountTable> records = response.getEntries();
+      refreshEntries(records);
+    } catch (IOException e) {
+      LOG.error("Cannot fetch mount table entries from State Store", e);
+      return false;
+    }
+    return true;
+  }
+
+  /**
+   * Clears all data.
+   */
+  public void clear() {
+    LOG.info("Clearing all mount location caches");
+    writeLock.lock();
+    try {
+      this.locationCache.clear();
+      this.tree.clear();
+    } finally {
+      writeLock.unlock();
+    }
+  }
+
+  @Override
+  public PathLocation getDestinationForPath(final String path)
+      throws IOException {
+    verifyMountTable();
+    readLock.lock();
+    try {
+      return this.locationCache.computeIfAbsent(
+          path, this::lookupLocation);
+    } finally {
+      readLock.unlock();
+    }
+  }
+
+  /**
+   * Build the path location to insert into the cache atomically. It must hold
+   * the read lock.
+   * @param path Path to check/insert.
+   * @return New remote location.
+   */
+  public PathLocation lookupLocation(final String path) {
+    PathLocation ret = null;
+    MountTable entry = findDeepest(path);
+    if (entry != null) {
+      ret = buildLocation(path, entry);
+    } else {
+      // Not found, use default location
+      RemoteLocation remoteLocation =
+          new RemoteLocation(defaultNameService, path);
+      List<RemoteLocation> locations =
+          Collections.singletonList(remoteLocation);
+      ret = new PathLocation(null, locations);
+    }
+    return ret;
+  }
+
+  /**
+   * Get the mount table entry for a path.
+   *
+   * @param path Path to look for.
+   * @return Mount table entry the path belongs.
+   * @throws IOException If the State Store could not be reached.
+   */
+  public MountTable getMountPoint(final String path) throws IOException {
+    verifyMountTable();
+    return findDeepest(path);
+  }
+
+  @Override
+  public List<String> getMountPoints(final String path) throws IOException {
+    verifyMountTable();
+
+    Set<String> children = new TreeSet<>();
+    readLock.lock();
+    try {
+      String from = path;
+      String to = path + Character.MAX_VALUE;
+      SortedMap<String, MountTable> subMap = this.tree.subMap(from, to);
+
+      boolean exists = false;
+      for (String subPath : subMap.keySet()) {
+        String child = subPath;
+
+        // Special case for /
+        if (!path.equals(Path.SEPARATOR)) {
+          // Get the children
+          int ini = path.length();
+          child = subPath.substring(ini);
+        }
+
+        if (child.isEmpty()) {
+          // This is a mount point but without children
+          exists = true;
+        } else if (child.startsWith(Path.SEPARATOR)) {
+          // This is a mount point with children
+          exists = true;
+          child = child.substring(1);
+
+          // We only return immediate children
+          int fin = child.indexOf(Path.SEPARATOR);
+          if (fin > -1) {
+            child = child.substring(0, fin);
+          }
+          if (!child.isEmpty()) {
+            children.add(child);
+          }
+        }
+      }
+      if (!exists) {
+        return null;
+      }
+      return new LinkedList<>(children);
+    } finally {
+      readLock.unlock();
+    }
+  }
+
+  /**
+   * Get all the mount records at or beneath a given path.
+   * @param path Path to get the mount points from.
+   * @return List of mount table records under the path or null if the path is
+   *         not found.
+   * @throws IOException If it's not connected to the State Store.
+   */
+  public List<MountTable> getMounts(final String path) throws IOException {
+    verifyMountTable();
+
+    return getTreeValues(path, false);
+  }
+
+  /**
+   * Check if the Mount Table is ready to be used.
+   * @throws StateStoreUnavailableException If it cannot connect to the store.
+   */
+  private void verifyMountTable() throws StateStoreUnavailableException {
+    if (!this.init) {
+      throw new StateStoreUnavailableException("Mount Table not initialized");
+    }
+  }
+
+  @Override
+  public String toString() {
+    readLock.lock();
+    try {
+      return this.tree.toString();
+    } finally {
+      readLock.unlock();
+    }
+  }
+
+  /**
+   * Build a location for this result beneath the discovered mount point.
+   *
+   * @param result Tree node search result.
+   * @return PathLocation containing the namespace, local path.
+   */
+  private static PathLocation buildLocation(
+      final String path, final MountTable entry) {
+
+    String srcPath = entry.getSourcePath();
+    if (!path.startsWith(srcPath)) {
+      LOG.error("Cannot build location, {} not a child of {}", path, srcPath);
+      return null;
+    }
+    String remainingPath = path.substring(srcPath.length());
+    if (remainingPath.startsWith(Path.SEPARATOR)) {
+      remainingPath = remainingPath.substring(1);
+    }
+
+    List<RemoteLocation> locations = new LinkedList<>();
+    for (RemoteLocation oneDst : entry.getDestinations()) {
+      String nsId = oneDst.getNameserviceId();
+      String dest = oneDst.getDest();
+      String newPath = dest;
+      if (!newPath.endsWith(Path.SEPARATOR)) {
+        newPath += Path.SEPARATOR;
+      }
+      newPath += remainingPath;
+      RemoteLocation remoteLocation = new RemoteLocation(nsId, newPath);
+      locations.add(remoteLocation);
+    }
+    DestinationOrder order = entry.getDestOrder();
+    return new PathLocation(srcPath, locations, order);
+  }
+
+  @Override
+  public String getDefaultNamespace() {
+    return this.defaultNameService;
+  }
+
+  /**
+   * Find the deepest mount point for a path.
+   * @param path Path to look for.
+   * @return Mount table entry.
+   */
+  private MountTable findDeepest(final String path) {
+    readLock.lock();
+    try {
+      Entry<String, MountTable> entry = this.tree.floorEntry(path);
+      while (entry != null && !path.startsWith(entry.getKey())) {
+        entry = this.tree.lowerEntry(entry.getKey());
+      }
+      if (entry == null) {
+        return null;
+      }
+      return entry.getValue();
+    } finally {
+      readLock.unlock();
+    }
+  }
+
+  /**
+   * Get the mount table entries under a path.
+   * @param path Path to search from.
+   * @return Mount Table entries.
+   */
+  private List<MountTable> getTreeValues(final String path) {
+    return getTreeValues(path, false);
+  }
+
+  /**
+   * Get the mount table entries under a path.
+   * @param path Path to search from.
+   * @param reverse If the order should be reversed.
+   * @return Mount Table entries.
+   */
+  private List<MountTable> getTreeValues(final String path, boolean reverse) {
+    LinkedList<MountTable> ret = new LinkedList<>();
+    readLock.lock();
+    try {
+      String from = path;
+      String to = path + Character.MAX_VALUE;
+      SortedMap<String, MountTable> subMap = this.tree.subMap(from, to);
+      for (MountTable entry : subMap.values()) {
+        if (!reverse) {
+          ret.add(entry);
+        } else {
+          ret.addFirst(entry);
+        }
+      }
+    } finally {
+      readLock.unlock();
+    }
+    return ret;
+  }
+}

+ 63 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/NamenodePriorityComparator.java

@@ -0,0 +1,63 @@
+/**
+ * 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.server.federation.resolver;
+
+import java.util.Comparator;
+
+/**
+ * Compares NNs in the same namespace and prioritizes by their status. The
+ * priorities are:
+ * <ul>
+ * <li>ACTIVE
+ * <li>STANDBY
+ * <li>UNAVAILABLE
+ * </ul>
+ * When two NNs have the same state, the last modification date is the tie
+ * breaker, newest has priority. Expired NNs are excluded.
+ */
+public class NamenodePriorityComparator
+    implements Comparator<FederationNamenodeContext> {
+
+  @Override
+  public int compare(FederationNamenodeContext o1,
+      FederationNamenodeContext o2) {
+    FederationNamenodeServiceState state1 = o1.getState();
+    FederationNamenodeServiceState state2 = o2.getState();
+
+    if (state1 == state2) {
+      // Both have the same state, use mode dates
+      return compareModDates(o1, o2);
+    } else {
+      // Enum is ordered by priority
+      return state1.compareTo(state2);
+    }
+  }
+
+  /**
+   * Compare the modification dates.
+   *
+   * @param o1 Context 1.
+   * @param o2 Context 2.
+   * @return Comparison between dates.
+   */
+  private int compareModDates(FederationNamenodeContext o1,
+      FederationNamenodeContext o2) {
+    // Reverse sort, lowest position is highest priority.
+    return (int) (o2.getDateModified() - o1.getDateModified());
+  }
+}

+ 388 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/NamenodeStatusReport.java

@@ -0,0 +1,388 @@
+/**
+ * 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.server.federation.resolver;
+
+import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
+import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
+
+/**
+ * Status of the namenode.
+ */
+public class NamenodeStatusReport {
+
+  /** Namenode information. */
+  private String nameserviceId = "";
+  private String namenodeId = "";
+  private String clusterId = "";
+  private String blockPoolId = "";
+  private String rpcAddress = "";
+  private String serviceAddress = "";
+  private String lifelineAddress = "";
+  private String webAddress = "";
+
+  /** Namenode state. */
+  private HAServiceState status = HAServiceState.STANDBY;
+  private boolean safeMode = false;
+
+  /** Datanodes stats. */
+  private int liveDatanodes = -1;
+  private int deadDatanodes = -1;
+  /** Decommissioning datanodes. */
+  private int decomDatanodes = -1;
+  /** Live decommissioned datanodes. */
+  private int liveDecomDatanodes = -1;
+  /** Dead decommissioned datanodes. */
+  private int deadDecomDatanodes = -1;
+
+  /** Space stats. */
+  private long availableSpace = -1;
+  private long numOfFiles = -1;
+  private long numOfBlocks = -1;
+  private long numOfBlocksMissing = -1;
+  private long numOfBlocksPendingReplication = -1;
+  private long numOfBlocksUnderReplicated = -1;
+  private long numOfBlocksPendingDeletion = -1;
+  private long totalSpace = -1;
+
+  /** If the fields are valid. */
+  private boolean registrationValid = false;
+  private boolean statsValid = false;
+  private boolean haStateValid = false;
+
+  public NamenodeStatusReport(String ns, String nn, String rpc, String service,
+      String lifeline, String web) {
+    this.nameserviceId = ns;
+    this.namenodeId = nn;
+    this.rpcAddress = rpc;
+    this.serviceAddress = service;
+    this.lifelineAddress = lifeline;
+    this.webAddress = web;
+  }
+
+  /**
+   * If the statistics are valid.
+   *
+   * @return If the statistics are valid.
+   */
+  public boolean statsValid() {
+    return this.statsValid;
+  }
+
+  /**
+   * If the registration is valid.
+   *
+   * @return If the registration is valid.
+   */
+  public boolean registrationValid() {
+    return this.registrationValid;
+  }
+
+  /**
+   * If the HA state is valid.
+   *
+   * @return If the HA state is valid.
+   */
+  public boolean haStateValid() {
+    return this.haStateValid;
+  }
+
+  /**
+   * Get the state of the Namenode being monitored.
+   *
+   * @return State of the Namenode.
+   */
+  public FederationNamenodeServiceState getState() {
+    if (!registrationValid) {
+      return FederationNamenodeServiceState.UNAVAILABLE;
+    } else if (haStateValid) {
+      return FederationNamenodeServiceState.getState(status);
+    } else {
+      return FederationNamenodeServiceState.ACTIVE;
+    }
+  }
+
+  /**
+   * Get the name service identifier.
+   *
+   * @return The name service identifier.
+   */
+  public String getNameserviceId() {
+    return this.nameserviceId;
+  }
+
+  /**
+   * Get the namenode identifier.
+   *
+   * @return The namenode identifier.
+   */
+  public String getNamenodeId() {
+    return this.namenodeId;
+  }
+
+  /**
+   * Get the cluster identifier.
+   *
+   * @return The cluster identifier.
+   */
+  public String getClusterId() {
+    return this.clusterId;
+  }
+
+  /**
+   * Get the block pool identifier.
+   *
+   * @return The block pool identifier.
+   */
+  public String getBlockPoolId() {
+    return this.blockPoolId;
+  }
+
+  /**
+   * Get the RPC address.
+   *
+   * @return The RPC address.
+   */
+  public String getRpcAddress() {
+    return this.rpcAddress;
+  }
+
+  /**
+   * Get the Service RPC address.
+   *
+   * @return The Service RPC address.
+   */
+  public String getServiceAddress() {
+    return this.serviceAddress;
+  }
+
+  /**
+   * Get the Lifeline RPC address.
+   *
+   * @return The Lifeline RPC address.
+   */
+  public String getLifelineAddress() {
+    return this.lifelineAddress;
+  }
+
+  /**
+   * Get the web address.
+   *
+   * @return The web address.
+   */
+  public String getWebAddress() {
+    return this.webAddress;
+  }
+
+  /**
+   * Get the HA service state.
+   *
+   * @return The HA service state.
+   */
+  public void setHAServiceState(HAServiceState state) {
+    this.status = state;
+    this.haStateValid = true;
+  }
+
+  /**
+   * Set the namespace information.
+   *
+   * @param info Namespace information.
+   */
+  public void setNamespaceInfo(NamespaceInfo info) {
+    this.clusterId = info.getClusterID();
+    this.blockPoolId = info.getBlockPoolID();
+    this.registrationValid = true;
+  }
+
+  public void setSafeMode(boolean safemode) {
+    this.safeMode = safemode;
+  }
+
+  public boolean getSafemode() {
+    return this.safeMode;
+  }
+
+  /**
+   * Set the datanode information.
+   *
+   * @param numLive Number of live nodes.
+   * @param numDead Number of dead nodes.
+   * @param numDecom Number of decommissioning nodes.
+   * @param numLiveDecom Number of decommissioned live nodes.
+   * @param numDeadDecom Number of decommissioned dead nodes.
+   */
+  public void setDatanodeInfo(int numLive, int numDead, int numDecom,
+      int numLiveDecom, int numDeadDecom) {
+    this.liveDatanodes = numLive;
+    this.deadDatanodes = numDead;
+    this.decomDatanodes = numDecom;
+    this.liveDecomDatanodes = numLiveDecom;
+    this.deadDecomDatanodes = numDeadDecom;
+    this.statsValid = true;
+  }
+
+  /**
+   * Get the number of live blocks.
+   *
+   * @return The number of dead nodes.
+   */
+  public int getNumLiveDatanodes() {
+    return this.liveDatanodes;
+  }
+
+  /**
+   * Get the number of dead blocks.
+   *
+   * @return The number of dead nodes.
+   */
+  public int getNumDeadDatanodes() {
+    return this.deadDatanodes;
+  }
+
+  /**
+   * Get the number of decommissionining nodes.
+   *
+   * @return The number of decommissionining nodes.
+   */
+  public int getNumDecommissioningDatanodes() {
+    return this.decomDatanodes;
+  }
+
+  /**
+   * Get the number of live decommissioned nodes.
+   *
+   * @return The number of live decommissioned nodes.
+   */
+  public int getNumDecomLiveDatanodes() {
+    return this.liveDecomDatanodes;
+  }
+
+  /**
+   * Get the number of dead decommissioned nodes.
+   *
+   * @return The number of dead decommissioned nodes.
+   */
+  public int getNumDecomDeadDatanodes() {
+    return this.deadDecomDatanodes;
+  }
+
+  /**
+   * Set the filesystem information.
+   *
+   * @param available Available capacity.
+   * @param total Total capacity.
+   * @param numFiles Number of files.
+   * @param numBlocks Total number of blocks.
+   * @param numBlocksMissing Number of missing blocks.
+   * @param numOfBlocksPendingReplication Number of blocks pending replication.
+   * @param numOfBlocksUnderReplicated Number of blocks under replication.
+   * @param numOfBlocksPendingDeletion Number of blocks pending deletion.
+   */
+  public void setNamesystemInfo(long available, long total,
+      long numFiles, long numBlocks, long numBlocksMissing,
+      long numBlocksPendingReplication, long numBlocksUnderReplicated,
+      long numBlocksPendingDeletion) {
+    this.totalSpace = total;
+    this.availableSpace = available;
+    this.numOfBlocks = numBlocks;
+    this.numOfBlocksMissing = numBlocksMissing;
+    this.numOfBlocksPendingReplication = numBlocksPendingReplication;
+    this.numOfBlocksUnderReplicated = numBlocksUnderReplicated;
+    this.numOfBlocksPendingDeletion = numBlocksPendingDeletion;
+    this.numOfFiles = numFiles;
+    this.statsValid = true;
+  }
+
+  /**
+   * Get the number of blocks.
+   *
+   * @return The number of blocks.
+   */
+  public long getNumBlocks() {
+    return this.numOfBlocks;
+  }
+
+  /**
+   * Get the number of files.
+   *
+   * @return The number of files.
+   */
+  public long getNumFiles() {
+    return this.numOfFiles;
+  }
+
+  /**
+   * Get the total space.
+   *
+   * @return The total space.
+   */
+  public long getTotalSpace() {
+    return this.totalSpace;
+  }
+
+  /**
+   * Get the available space.
+   *
+   * @return The available space.
+   */
+  public long getAvailableSpace() {
+    return this.availableSpace;
+  }
+
+  /**
+   * Get the number of missing blocks.
+   *
+   * @return Number of missing blocks.
+   */
+  public long getNumBlocksMissing() {
+    return this.numOfBlocksMissing;
+  }
+
+  /**
+   * Get the number of pending replication blocks.
+   *
+   * @return Number of pending replication blocks.
+   */
+  public long getNumOfBlocksPendingReplication() {
+    return this.numOfBlocksPendingReplication;
+  }
+
+  /**
+   * Get the number of under replicated blocks.
+   *
+   * @return Number of under replicated blocks.
+   */
+  public long getNumOfBlocksUnderReplicated() {
+    return this.numOfBlocksUnderReplicated;
+  }
+
+  /**
+   * Get the number of pending deletion blocks.
+   *
+   * @return Number of pending deletion blocks.
+   */
+  public long getNumOfBlocksPendingDeletion() {
+    return this.numOfBlocksPendingDeletion;
+  }
+
+  @Override
+  public String toString() {
+    return String.format("%s-%s:%s",
+        nameserviceId, namenodeId, serviceAddress);
+  }
+}

+ 212 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/PathLocation.java

@@ -0,0 +1,212 @@
+/**
+ * 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.server.federation.resolver;
+
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.hadoop.hdfs.server.federation.resolver.order.DestinationOrder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A map of the properties and target destinations (name space + path) for
+ * a path in the global/federated name space.
+ * This data is generated from the @see MountTable records.
+ */
+public class PathLocation {
+
+  private static final Logger LOG = LoggerFactory.getLogger(PathLocation.class);
+
+
+  /** Source path in global namespace. */
+  private final String sourcePath;
+
+  /** Remote paths in the target name spaces. */
+  private final List<RemoteLocation> destinations;
+  /** Order for the destinations. */
+  private final DestinationOrder destOrder;
+
+
+  /**
+   * Create a new PathLocation.
+   *
+   * @param source Source path in the global name space.
+   * @param dest Destinations of the mount table entry.
+   * @param order Order of the locations.
+   */
+  public PathLocation(
+      String source, List<RemoteLocation> dest, DestinationOrder order) {
+    this.sourcePath = source;
+    this.destinations = Collections.unmodifiableList(dest);
+    this.destOrder = order;
+  }
+
+  /**
+   * Create a new PathLocation with default HASH order.
+   *
+   * @param source Source path in the global name space.
+   * @param dest Destinations of the mount table entry.
+   */
+  public PathLocation(String source, List<RemoteLocation> dest) {
+    this(source, dest, DestinationOrder.HASH);
+  }
+
+  /**
+   * Create a path location from another path.
+   *
+   * @param other Other path location to copy from.
+   */
+  public PathLocation(final PathLocation other) {
+    this.sourcePath = other.sourcePath;
+    this.destinations = Collections.unmodifiableList(other.destinations);
+    this.destOrder = other.destOrder;
+  }
+
+  /**
+   * Create a path location from another path with the destinations sorted.
+   *
+   * @param other Other path location to copy from.
+   * @param firstNsId Identifier of the namespace to place first.
+   */
+  public PathLocation(PathLocation other, String firstNsId) {
+    this.sourcePath = other.sourcePath;
+    this.destOrder = other.destOrder;
+    this.destinations = orderedNamespaces(other.destinations, firstNsId);
+  }
+
+  /**
+   * Prioritize a location/destination by its name space/nameserviceId.
+   * This destination might be used by other threads, so the source is not
+   * modifiable.
+   *
+   * @param original List of destinations to order.
+   * @param nsId The name space/nameserviceID to prioritize.
+   * @return Prioritized list of detinations that cannot be modified.
+   */
+  private static List<RemoteLocation> orderedNamespaces(
+      final List<RemoteLocation> original, final String nsId) {
+    if (original.size() <= 1) {
+      return original;
+    }
+
+    LinkedList<RemoteLocation> newDestinations = new LinkedList<>();
+    boolean found = false;
+    for (RemoteLocation dest : original) {
+      if (dest.getNameserviceId().equals(nsId)) {
+        found = true;
+        newDestinations.addFirst(dest);
+      } else {
+        newDestinations.add(dest);
+      }
+    }
+
+    if (!found) {
+      LOG.debug("Cannot find location with namespace {} in {}",
+          nsId, original);
+    }
+    return Collections.unmodifiableList(newDestinations);
+  }
+
+  /**
+   * Get the source path in the global namespace for this path location.
+   *
+   * @return The path in the global namespace.
+   */
+  public String getSourcePath() {
+    return this.sourcePath;
+  }
+
+  /**
+   * Get the subclusters defined for the destinations.
+   *
+   * @return Set containing the subclusters.
+   */
+  public Set<String> getNamespaces() {
+    Set<String> namespaces = new HashSet<>();
+    List<RemoteLocation> locations = this.getDestinations();
+    for (RemoteLocation location : locations) {
+      String nsId = location.getNameserviceId();
+      namespaces.add(nsId);
+    }
+    return namespaces;
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder();
+    for (RemoteLocation destination : this.destinations) {
+      String nsId = destination.getNameserviceId();
+      String path = destination.getDest();
+      if (sb.length() > 0) {
+        sb.append(",");
+      }
+      sb.append(nsId + "->" + path);
+    }
+    if (this.destinations.size() > 1) {
+      sb.append(" [");
+      sb.append(this.destOrder.toString());
+      sb.append("]");
+    }
+    return sb.toString();
+  }
+
+  /**
+   * Check if this location supports multiple clusters/paths.
+   *
+   * @return If it has multiple destinations.
+   */
+  public boolean hasMultipleDestinations() {
+    return this.destinations.size() > 1;
+  }
+
+  /**
+   * Get the list of locations found in the mount table.
+   * The first result is the highest priority path.
+   *
+   * @return List of remote locations.
+   */
+  public List<RemoteLocation> getDestinations() {
+    return Collections.unmodifiableList(this.destinations);
+  }
+
+  /**
+   * Get the order for the destinations.
+   *
+   * @return Order for the destinations.
+   */
+  public DestinationOrder getDestinationOrder() {
+    return this.destOrder;
+  }
+
+  /**
+   * Get the default or highest priority location.
+   *
+   * @return The default location.
+   */
+  public RemoteLocation getDefaultLocation() {
+    if (destinations.isEmpty() || destinations.get(0).getDest() == null) {
+      throw new UnsupportedOperationException(
+          "Unsupported path " + sourcePath + " please check mount table");
+    }
+    return destinations.get(0);
+  }
+}

+ 76 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/RemoteLocation.java

@@ -0,0 +1,76 @@
+/**
+ * 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.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.
+ */
+public class RemoteLocation extends RemoteLocationContext {
+
+  /** Identifier of the remote namespace for this location. */
+  private final String nameserviceId;
+  /** Identifier of the namenode in the namespace for this location. */
+  private final String namenodeId;
+  /** Path in the remote location. */
+  private final String path;
+
+  /**
+   * Create a new remote location.
+   *
+   * @param nsId
+   * @param pPath
+   */
+  public RemoteLocation(String nsId, String pPath) {
+    this(nsId, null, pPath);
+  }
+
+  /**
+   * Create a new remote location pointing to a particular namenode in the
+   * namespace.
+   *
+   * @param nsId Destination namespace.
+   * @param pPath Path in the destination namespace.
+   */
+  public RemoteLocation(String nsId, String nnId, String pPath) {
+    this.nameserviceId = nsId;
+    this.namenodeId = nnId;
+    this.path = pPath;
+  }
+
+  @Override
+  public String getNameserviceId() {
+    String ret = this.nameserviceId;
+    if (this.namenodeId != null) {
+      ret += "-" + this.namenodeId;
+    }
+    return ret;
+  }
+
+  @Override
+  public String getDest() {
+    return this.path;
+  }
+
+  @Override
+  public String toString() {
+    return getNameserviceId() + "->" + this.path;
+  }
+}

+ 29 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/order/DestinationOrder.java

@@ -0,0 +1,29 @@
+/**
+ * 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.server.federation.resolver.order;
+
+/**
+ * Order of the destinations when we have multiple of them. When the resolver
+ * of files to subclusters (FileSubclusterResolver) has multiple destinations,
+ * this determines which location should be checked first.
+ */
+public enum DestinationOrder {
+  HASH, // Follow consistent hashing
+  LOCAL, // Local first
+  RANDOM // Random order
+}

+ 29 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/order/package-info.java

@@ -0,0 +1,29 @@
+/**
+ * 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.
+ */
+
+/**
+ * A federated location can be resolved to multiple subclusters. This package
+ * takes care of the order in which this multiple destinations should be used.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+
+package org.apache.hadoop.hdfs.server.federation.resolver.order;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;

+ 41 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/package-info.java

@@ -0,0 +1,41 @@
+/**
+ * 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.
+ */
+
+/**
+ * The resolver package contains indepedent data resolvers used in HDFS
+ * federation. The data resolvers collect data from the cluster, including from
+ * the state store. The resolvers expose APIs used by HDFS federation to collect
+ * aggregated, cached data for use in Real-time request processing. The
+ * resolvers are perf-sensitive and are used in the flow of the
+ * {@link RouterRpcServer} request path.
+ * <p>
+ * The principal resolvers are:
+ * <ul>
+ * <li>{@link ActiveNamenodeResolver} Real-time interface for locating the most
+ * recently active NN for a nameservice.
+ * <li>{@link FileSubclusterResolver} Real-time interface for determining the NN
+ * and local file path for a given file/folder based on the global namespace
+ * path.
+ * </ul>
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+package org.apache.hadoop.hdfs.server.federation.resolver;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;

+ 104 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/ConnectionContext.java

@@ -0,0 +1,104 @@
+/**
+ * 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.server.federation.router;
+
+import org.apache.hadoop.hdfs.NameNodeProxiesClient.ProxyAndInfo;
+import org.apache.hadoop.hdfs.protocol.ClientProtocol;
+import org.apache.hadoop.ipc.RPC;
+
+/**
+ * Context to track a connection in a {@link ConnectionPool}. When a client uses
+ * a connection, it increments a counter to mark it as active. Once the client
+ * is done with the connection, it decreases the counter. It also takes care of
+ * closing the connection once is not active.
+ */
+public class ConnectionContext {
+
+  /** Client for the connection. */
+  private final ProxyAndInfo<ClientProtocol> client;
+  /** How many threads are using this connection. */
+  private int numThreads = 0;
+  /** If the connection is closed. */
+  private boolean closed = false;
+
+
+  public ConnectionContext(ProxyAndInfo<ClientProtocol> connection) {
+    this.client = connection;
+  }
+
+  /**
+   * Check if the connection is active.
+   *
+   * @return True if the connection is active.
+   */
+  public synchronized boolean isActive() {
+    return this.numThreads > 0;
+  }
+
+  /**
+   * Check if the connection is closed.
+   *
+   * @return If the connection is closed.
+   */
+  public synchronized boolean isClosed() {
+    return this.closed;
+  }
+
+  /**
+   * Check if the connection can be used. It checks if the connection is used by
+   * another thread or already closed.
+   *
+   * @return True if the connection can be used.
+   */
+  public synchronized boolean isUsable() {
+    return !isActive() && !isClosed();
+  }
+
+  /**
+   * Get the connection client.
+   *
+   * @return Connection client.
+   */
+  public synchronized ProxyAndInfo<ClientProtocol> getClient() {
+    this.numThreads++;
+    return this.client;
+  }
+
+  /**
+   * Release this connection. If the connection was closed, close the proxy.
+   * Otherwise, mark the connection as not used by us anymore.
+   */
+  public synchronized void release() {
+    if (--this.numThreads == 0 && this.closed) {
+      close();
+    }
+  }
+
+  /**
+   * We will not use this connection anymore. If it's not being used, we close
+   * it. Otherwise, we let release() do it once we are done with it.
+   */
+  public synchronized void close() {
+    this.closed = true;
+    if (this.numThreads == 0) {
+      ClientProtocol proxy = this.client.getProxy();
+      // Nobody should be using this anymore so it should close right away
+      RPC.stopProxy(proxy);
+    }
+  }
+}

+ 431 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/ConnectionManager.java

@@ -0,0 +1,431 @@
+/**
+ * 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.server.federation.router;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.TreeMap;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.util.Time;
+import org.eclipse.jetty.util.ajax.JSON;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Implements a pool of connections for the {@link Router} to be able to open
+ * many connections to many Namenodes.
+ */
+public class ConnectionManager {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ConnectionManager.class);
+
+  /** Number of parallel new connections to create. */
+  protected static final int MAX_NEW_CONNECTIONS = 100;
+
+  /** Minimum amount of active connections: 50%. */
+  protected static final float MIN_ACTIVE_RATIO = 0.5f;
+
+
+  /** Configuration for the connection manager, pool and sockets. */
+  private final Configuration conf;
+
+  /** Min number of connections per user + nn. */
+  private final int minSize = 1;
+  /** Max number of connections per user + nn. */
+  private final int maxSize;
+
+  /** How often we close a pool for a particular user + nn. */
+  private final long poolCleanupPeriodMs;
+  /** How often we close a connection in a pool. */
+  private final long connectionCleanupPeriodMs;
+
+  /** Map of connection pools, one pool per user + NN. */
+  private final Map<ConnectionPoolId, ConnectionPool> pools;
+  /** Lock for accessing pools. */
+  private final ReadWriteLock readWriteLock = new ReentrantReadWriteLock();
+  private final Lock readLock = readWriteLock.readLock();
+  private final Lock writeLock = readWriteLock.writeLock();
+
+  /** Queue for creating new connections. */
+  private final BlockingQueue<ConnectionPool> creatorQueue =
+      new ArrayBlockingQueue<>(MAX_NEW_CONNECTIONS);
+  /** Create new connections asynchronously. */
+  private final ConnectionCreator creator;
+  /** Periodic executor to remove stale connection pools. */
+  private final ScheduledThreadPoolExecutor cleaner =
+      new ScheduledThreadPoolExecutor(1);
+
+  /** If the connection manager is running. */
+  private boolean running = false;
+
+
+  /**
+   * Creates a proxy client connection pool manager.
+   *
+   * @param config Configuration for the connections.
+   * @param minPoolSize Min size of the connection pool.
+   * @param maxPoolSize Max size of the connection pool.
+   */
+  public ConnectionManager(Configuration config) {
+    this.conf = config;
+
+    // Configure minimum and maximum connection pools
+    this.maxSize = this.conf.getInt(
+        DFSConfigKeys.DFS_ROUTER_NAMENODE_CONNECTION_POOL_SIZE,
+        DFSConfigKeys.DFS_ROUTER_NAMENODE_CONNECTION_POOL_SIZE_DEFAULT);
+
+    // Map with the connections indexed by UGI and Namenode
+    this.pools = new HashMap<>();
+
+    // Create connections in a thread asynchronously
+    this.creator = new ConnectionCreator(creatorQueue);
+    this.creator.setDaemon(true);
+
+    // Cleanup periods
+    this.poolCleanupPeriodMs = this.conf.getLong(
+        DFSConfigKeys.DFS_ROUTER_NAMENODE_CONNECTION_POOL_CLEAN,
+        DFSConfigKeys.DFS_ROUTER_NAMENODE_CONNECTION_POOL_CLEAN_DEFAULT);
+    LOG.info("Cleaning connection pools every {} seconds",
+        TimeUnit.MILLISECONDS.toSeconds(this.poolCleanupPeriodMs));
+    this.connectionCleanupPeriodMs = this.conf.getLong(
+        DFSConfigKeys.DFS_ROUTER_NAMENODE_CONNECTION_CLEAN_MS,
+        DFSConfigKeys.DFS_ROUTER_NAMENODE_CONNECTION_CLEAN_MS_DEFAULT);
+    LOG.info("Cleaning connections every {} seconds",
+        TimeUnit.MILLISECONDS.toSeconds(this.connectionCleanupPeriodMs));
+  }
+
+  /**
+   * Start the connection manager.
+   */
+  public void start() {
+    // Start the thread that creates connections asynchronously
+    this.creator.start();
+
+    // Schedule a task to remove stale connection pools and sockets
+    long recyleTimeMs = Math.min(
+        poolCleanupPeriodMs, connectionCleanupPeriodMs);
+    LOG.info("Cleaning every {} seconds",
+        TimeUnit.MILLISECONDS.toSeconds(recyleTimeMs));
+    this.cleaner.scheduleAtFixedRate(
+        new CleanupTask(), 0, recyleTimeMs, TimeUnit.MILLISECONDS);
+
+    // Mark the manager as running
+    this.running = true;
+  }
+
+  /**
+   * Stop the connection manager by closing all the pools.
+   */
+  public void close() {
+    this.creator.shutdown();
+    this.cleaner.shutdown();
+    this.running = false;
+
+    writeLock.lock();
+    try {
+      for (ConnectionPool pool : this.pools.values()) {
+        pool.close();
+      }
+      this.pools.clear();
+    } finally {
+      writeLock.unlock();
+    }
+  }
+
+  /**
+   * Fetches the next available proxy client in the pool. Each client connection
+   * is reserved for a single user and cannot be reused until free.
+   *
+   * @param ugi User group information.
+   * @param nnAddress Namenode address for the connection.
+   * @return Proxy client to connect to nnId as UGI.
+   * @throws IOException If the connection cannot be obtained.
+   */
+  public ConnectionContext getConnection(
+      UserGroupInformation ugi, String nnAddress) throws IOException {
+
+    // Check if the manager is shutdown
+    if (!this.running) {
+      LOG.error(
+          "Cannot get a connection to {} because the manager isn't running",
+          nnAddress);
+      return null;
+    }
+
+    // Try to get the pool if created
+    ConnectionPoolId connectionId = new ConnectionPoolId(ugi, nnAddress);
+    ConnectionPool pool = null;
+    readLock.lock();
+    try {
+      pool = this.pools.get(connectionId);
+    } finally {
+      readLock.unlock();
+    }
+
+    // Create the pool if not created before
+    if (pool == null) {
+      writeLock.lock();
+      try {
+        pool = this.pools.get(connectionId);
+        if (pool == null) {
+          pool = new ConnectionPool(
+              this.conf, nnAddress, ugi, this.minSize, this.maxSize);
+          this.pools.put(connectionId, pool);
+        }
+      } finally {
+        writeLock.unlock();
+      }
+    }
+
+    ConnectionContext conn = pool.getConnection();
+
+    // Add a new connection to the pool if it wasn't usable
+    if (conn == null || !conn.isUsable()) {
+      if (!this.creatorQueue.offer(pool)) {
+        LOG.error("Cannot add more than {} connections at the same time",
+            MAX_NEW_CONNECTIONS);
+      }
+    }
+
+    if (conn != null && conn.isClosed()) {
+      LOG.error("We got a closed connection from {}", pool);
+      conn = null;
+    }
+
+    return conn;
+  }
+
+  /**
+   * Get the number of connection pools.
+   *
+   * @return Number of connection pools.
+   */
+  public int getNumConnectionPools() {
+    readLock.lock();
+    try {
+      return pools.size();
+    } finally {
+      readLock.unlock();
+    }
+  }
+
+  /**
+   * Get number of open connections.
+   *
+   * @return Number of open connections.
+   */
+  public int getNumConnections() {
+    int total = 0;
+    readLock.lock();
+    try {
+      for (ConnectionPool pool : this.pools.values()) {
+        total += pool.getNumConnections();
+      }
+    } finally {
+      readLock.unlock();
+    }
+    return total;
+  }
+
+  /**
+   * Get number of active connections.
+   *
+   * @return Number of active connections.
+   */
+  public int getNumActiveConnections() {
+    int total = 0;
+    readLock.lock();
+    try {
+      for (ConnectionPool pool : this.pools.values()) {
+        total += pool.getNumActiveConnections();
+      }
+    } finally {
+      readLock.unlock();
+    }
+    return total;
+  }
+
+  /**
+   * Get the number of connections to be created.
+   *
+   * @return Number of connections to be created.
+   */
+  public int getNumCreatingConnections() {
+    return this.creatorQueue.size();
+  }
+
+  /**
+   * Get a JSON representation of the connection pool.
+   *
+   * @return JSON representation of all the connection pools.
+   */
+  public String getJSON() {
+    final Map<String, String> info = new TreeMap<>();
+    readLock.lock();
+    try {
+      for (Entry<ConnectionPoolId, ConnectionPool> entry :
+          this.pools.entrySet()) {
+        ConnectionPoolId connectionPoolId = entry.getKey();
+        ConnectionPool pool = entry.getValue();
+        info.put(connectionPoolId.toString(), pool.getJSON());
+      }
+    } finally {
+      readLock.unlock();
+    }
+    return JSON.toString(info);
+  }
+
+  /**
+   * Removes stale connections not accessed recently from the pool. This is
+   * invoked periodically.
+   */
+  private class CleanupTask implements Runnable {
+
+    @Override
+    public void run() {
+      long currentTime = Time.now();
+      List<ConnectionPoolId> toRemove = new LinkedList<>();
+
+      // Look for stale pools
+      readLock.lock();
+      try {
+        for (Entry<ConnectionPoolId, ConnectionPool> entry : pools.entrySet()) {
+          ConnectionPool pool = entry.getValue();
+          long lastTimeActive = pool.getLastActiveTime();
+          boolean isStale =
+              currentTime > (lastTimeActive + poolCleanupPeriodMs);
+          if (lastTimeActive > 0 && isStale) {
+            // Remove this pool
+            LOG.debug("Closing and removing stale pool {}", pool);
+            pool.close();
+            ConnectionPoolId poolId = entry.getKey();
+            toRemove.add(poolId);
+          } else {
+            // Keep this pool but clean connections inside
+            LOG.debug("Cleaning up {}", pool);
+            cleanup(pool);
+          }
+        }
+      } finally {
+        readLock.unlock();
+      }
+
+      // Remove stale pools
+      if (!toRemove.isEmpty()) {
+        writeLock.lock();
+        try {
+          for (ConnectionPoolId poolId : toRemove) {
+            pools.remove(poolId);
+          }
+        } finally {
+          writeLock.unlock();
+        }
+      }
+    }
+
+    /**
+     * Clean the unused connections for this pool.
+     *
+     * @param pool Connection pool to cleanup.
+     */
+    private void cleanup(ConnectionPool pool) {
+      if (pool.getNumConnections() > pool.getMinSize()) {
+        // Check if the pool hasn't been active in a while or not 50% are used
+        long timeSinceLastActive = Time.now() - pool.getLastActiveTime();
+        int total = pool.getNumConnections();
+        int active = getNumActiveConnections();
+        if (timeSinceLastActive > connectionCleanupPeriodMs ||
+            active < MIN_ACTIVE_RATIO * total) {
+          // Remove and close 1 connection
+          List<ConnectionContext> conns = pool.removeConnections(1);
+          for (ConnectionContext conn : conns) {
+            conn.close();
+          }
+          LOG.debug("Removed connection {} used {} seconds ago. " +
+              "Pool has {}/{} connections", pool.getConnectionPoolId(),
+              TimeUnit.MILLISECONDS.toSeconds(timeSinceLastActive),
+              pool.getNumConnections(), pool.getMaxSize());
+        }
+      }
+    }
+  }
+
+  /**
+   * Thread that creates connections asynchronously.
+   */
+  private static class ConnectionCreator extends Thread {
+    /** If the creator is running. */
+    private boolean running = true;
+    /** Queue to push work to. */
+    private BlockingQueue<ConnectionPool> queue;
+
+    ConnectionCreator(BlockingQueue<ConnectionPool> blockingQueue) {
+      super("Connection creator");
+      this.queue = blockingQueue;
+    }
+
+    @Override
+    public void run() {
+      while (this.running) {
+        try {
+          ConnectionPool pool = this.queue.take();
+          try {
+            int total = pool.getNumConnections();
+            int active = pool.getNumActiveConnections();
+            if (pool.getNumConnections() < pool.getMaxSize() &&
+                active >= MIN_ACTIVE_RATIO * total) {
+              ConnectionContext conn = pool.newConnection();
+              pool.addConnection(conn);
+            } else {
+              LOG.debug("Cannot add more than {} connections to {}",
+                  pool.getMaxSize(), pool);
+            }
+          } catch (IOException e) {
+            LOG.error("Cannot create a new connection", e);
+          }
+        } catch (InterruptedException e) {
+          LOG.error("The connection creator was interrupted");
+          this.running = false;
+        }
+      }
+    }
+
+    /**
+     * Stop this connection creator.
+     */
+    public void shutdown() {
+      this.running = false;
+      this.interrupt();
+    }
+  }
+}

+ 337 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/ConnectionPool.java

@@ -0,0 +1,337 @@
+/**
+ * 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.server.federation.router;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.LinkedHashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import javax.net.SocketFactory;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.NameNodeProxiesClient.ProxyAndInfo;
+import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
+import org.apache.hadoop.hdfs.protocol.ClientProtocol;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolPB;
+import org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolTranslatorPB;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.retry.RetryPolicy;
+import org.apache.hadoop.io.retry.RetryUtils;
+import org.apache.hadoop.ipc.ProtobufRpcEngine;
+import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.SaslRpcServer;
+import org.apache.hadoop.security.SecurityUtil;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.util.Time;
+import org.eclipse.jetty.util.ajax.JSON;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Maintains a pool of connections for each User (including tokens) + NN. The
+ * RPC client maintains a single socket, to achieve throughput similar to a NN,
+ * each request is multiplexed across multiple sockets/connections from a
+ * pool.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class ConnectionPool {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ConnectionPool.class);
+
+
+  /** Configuration settings for the connection pool. */
+  private final Configuration conf;
+
+  /** Identifier for this connection pool. */
+  private final ConnectionPoolId connectionPoolId;
+  /** Namenode this pool connects to. */
+  private final String namenodeAddress;
+  /** User for this connections. */
+  private final UserGroupInformation ugi;
+
+  /** Pool of connections. We mimic a COW array. */
+  private volatile List<ConnectionContext> connections = new ArrayList<>();
+  /** Connection index for round-robin. */
+  private final AtomicInteger clientIndex = new AtomicInteger(0);
+
+  /** Min number of connections per user. */
+  private final int minSize;
+  /** Max number of connections per user. */
+  private final int maxSize;
+
+  /** The last time a connection was active. */
+  private volatile long lastActiveTime = 0;
+
+
+  protected ConnectionPool(Configuration config, String address,
+      UserGroupInformation user, int minPoolSize, int maxPoolSize)
+          throws IOException {
+
+    this.conf = config;
+
+    // Connection pool target
+    this.ugi = user;
+    this.namenodeAddress = address;
+    this.connectionPoolId =
+        new ConnectionPoolId(this.ugi, this.namenodeAddress);
+
+    // Set configuration parameters for the pool
+    this.minSize = minPoolSize;
+    this.maxSize = maxPoolSize;
+
+    // Add minimum connections to the pool
+    for (int i=0; i<this.minSize; i++) {
+      ConnectionContext newConnection = newConnection();
+      this.connections.add(newConnection);
+    }
+    LOG.debug("Created connection pool \"{}\" with {} connections",
+        this.connectionPoolId, this.minSize);
+  }
+
+  /**
+   * Get the maximum number of connections allowed in this pool.
+   *
+   * @return Maximum number of connections.
+   */
+  protected int getMaxSize() {
+    return this.maxSize;
+  }
+
+  /**
+   * Get the minimum number of connections in this pool.
+   *
+   * @return Minimum number of connections.
+   */
+  protected int getMinSize() {
+    return this.minSize;
+  }
+
+  /**
+   * Get the connection pool identifier.
+   *
+   * @return Connection pool identifier.
+   */
+  protected ConnectionPoolId getConnectionPoolId() {
+    return this.connectionPoolId;
+  }
+
+  /**
+   * Return the next connection round-robin.
+   *
+   * @return Connection context.
+   */
+  protected ConnectionContext getConnection() {
+
+    this.lastActiveTime = Time.now();
+
+    // Get a connection from the pool following round-robin
+    ConnectionContext conn = null;
+    List<ConnectionContext> tmpConnections = this.connections;
+    int size = tmpConnections.size();
+    int threadIndex = this.clientIndex.getAndIncrement();
+    for (int i=0; i<size; i++) {
+      int index = (threadIndex + i) % size;
+      conn = tmpConnections.get(index);
+      if (conn != null && !conn.isUsable()) {
+        return conn;
+      }
+    }
+
+    // We return a connection even if it's active
+    return conn;
+  }
+
+  /**
+   * Add a connection to the current pool. It uses a Copy-On-Write approach.
+   *
+   * @param conns New connections to add to the pool.
+   */
+  public synchronized void addConnection(ConnectionContext conn) {
+    List<ConnectionContext> tmpConnections = new ArrayList<>(this.connections);
+    tmpConnections.add(conn);
+    this.connections = tmpConnections;
+
+    this.lastActiveTime = Time.now();
+  }
+
+  /**
+   * Remove connections from the current pool.
+   *
+   * @param num Number of connections to remove.
+   * @return Removed connections.
+   */
+  public synchronized List<ConnectionContext> removeConnections(int num) {
+    List<ConnectionContext> removed = new LinkedList<>();
+
+    // Remove and close the last connection
+    List<ConnectionContext> tmpConnections = new ArrayList<>();
+    for (int i=0; i<this.connections.size(); i++) {
+      ConnectionContext conn = this.connections.get(i);
+      if (i < this.minSize || i < this.connections.size() - num) {
+        tmpConnections.add(conn);
+      } else {
+        removed.add(conn);
+      }
+    }
+    this.connections = tmpConnections;
+
+    return removed;
+  }
+
+  /**
+   * Close the connection pool.
+   */
+  protected synchronized void close() {
+    long timeSinceLastActive = TimeUnit.MILLISECONDS.toSeconds(
+        Time.now() - getLastActiveTime());
+    LOG.debug("Shutting down connection pool \"{}\" used {} seconds ago",
+        this.connectionPoolId, timeSinceLastActive);
+
+    for (ConnectionContext connection : this.connections) {
+      connection.close();
+    }
+    this.connections.clear();
+  }
+
+  /**
+   * Number of connections in the pool.
+   *
+   * @return Number of connections.
+   */
+  protected int getNumConnections() {
+    return this.connections.size();
+  }
+
+  /**
+   * Number of active connections in the pool.
+   *
+   * @return Number of active connections.
+   */
+  protected int getNumActiveConnections() {
+    int ret = 0;
+
+    List<ConnectionContext> tmpConnections = this.connections;
+    for (ConnectionContext conn : tmpConnections) {
+      if (conn.isActive()) {
+        ret++;
+      }
+    }
+    return ret;
+  }
+
+  /**
+   * Get the last time the connection pool was used.
+   *
+   * @return Last time the connection pool was used.
+   */
+  protected long getLastActiveTime() {
+    return this.lastActiveTime;
+  }
+
+  @Override
+  public String toString() {
+    return this.connectionPoolId.toString();
+  }
+
+  /**
+   * JSON representation of the connection pool.
+   *
+   * @return String representation of the JSON.
+   */
+  public String getJSON() {
+    final Map<String, String> info = new LinkedHashMap<>();
+    info.put("active", Integer.toString(getNumActiveConnections()));
+    info.put("total", Integer.toString(getNumConnections()));
+    if (LOG.isDebugEnabled()) {
+      List<ConnectionContext> tmpConnections = this.connections;
+      for (int i=0; i<tmpConnections.size(); i++) {
+        ConnectionContext connection = tmpConnections.get(i);
+        info.put(i + " active", Boolean.toString(connection.isActive()));
+        info.put(i + " closed", Boolean.toString(connection.isClosed()));
+      }
+    }
+    return JSON.toString(info);
+  }
+
+  /**
+   * Create a new proxy wrapper for a client NN connection.
+   * @return Proxy for the target ClientProtocol that contains the user's
+   *         security context.
+   * @throws IOException
+   */
+  public ConnectionContext newConnection() throws IOException {
+    return newConnection(this.conf, this.namenodeAddress, this.ugi);
+  }
+
+  /**
+   * Creates a proxy wrapper for a client NN connection. Each proxy contains
+   * context for a single user/security context. To maximize throughput it is
+   * recommended to use multiple connection per user+server, allowing multiple
+   * writes and reads to be dispatched in parallel.
+   *
+   * @param conf Configuration for the connection.
+   * @param nnAddress Address of server supporting the ClientProtocol.
+   * @param ugi User context.
+   * @return Proxy for the target ClientProtocol that contains the user's
+   *         security context.
+   * @throws IOException If it cannot be created.
+   */
+  protected static ConnectionContext newConnection(Configuration conf,
+      String nnAddress, UserGroupInformation ugi)
+          throws IOException {
+    RPC.setProtocolEngine(
+        conf, ClientNamenodeProtocolPB.class, ProtobufRpcEngine.class);
+
+    final RetryPolicy defaultPolicy = RetryUtils.getDefaultRetryPolicy(
+        conf,
+        HdfsClientConfigKeys.Retry.POLICY_ENABLED_KEY,
+        HdfsClientConfigKeys.Retry.POLICY_ENABLED_DEFAULT,
+        HdfsClientConfigKeys.Retry.POLICY_SPEC_KEY,
+        HdfsClientConfigKeys.Retry.POLICY_SPEC_DEFAULT,
+        HdfsConstants.SAFEMODE_EXCEPTION_CLASS_NAME);
+
+    SocketFactory factory = SocketFactory.getDefault();
+    if (UserGroupInformation.isSecurityEnabled()) {
+      SaslRpcServer.init(conf);
+    }
+    InetSocketAddress socket = NetUtils.createSocketAddr(nnAddress);
+    final long version = RPC.getProtocolVersion(ClientNamenodeProtocolPB.class);
+    ClientNamenodeProtocolPB proxy = RPC.getProtocolProxy(
+        ClientNamenodeProtocolPB.class, version, socket, ugi, conf,
+        factory, RPC.getRpcTimeout(conf), defaultPolicy, null).getProxy();
+    ClientProtocol client = new ClientNamenodeProtocolTranslatorPB(proxy);
+    Text dtService = SecurityUtil.buildTokenService(socket);
+
+    ProxyAndInfo<ClientProtocol> clientProxy =
+        new ProxyAndInfo<ClientProtocol>(client, dtService, socket);
+    ConnectionContext connection = new ConnectionContext(clientProxy);
+    return connection;
+  }
+}

+ 117 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/ConnectionPoolId.java

@@ -0,0 +1,117 @@
+/**
+ * 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.server.federation.router;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.commons.lang.builder.HashCodeBuilder;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
+
+/**
+ * Identifier for a connection for a user to a namenode.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class ConnectionPoolId implements Comparable<ConnectionPoolId> {
+
+  /** Namenode identifier. */
+  private final String nnId;
+  /** Information about the user. */
+  private final UserGroupInformation ugi;
+
+  /**
+   * New connection pool identifier.
+   *
+   * @param ugi Information of the user issuing the request.
+   * @param nnId Namenode address with port.
+   */
+  public ConnectionPoolId(final UserGroupInformation ugi, final String nnId) {
+    this.nnId = nnId;
+    this.ugi = ugi;
+  }
+
+  @Override
+  public int hashCode() {
+    int hash = new HashCodeBuilder(17, 31)
+        .append(this.nnId)
+        .append(this.ugi.toString())
+        .append(this.getTokenIds())
+        .toHashCode();
+    return hash;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (o instanceof ConnectionPoolId) {
+      ConnectionPoolId other = (ConnectionPoolId) o;
+      if (!this.nnId.equals(other.nnId)) {
+        return false;
+      }
+      if (!this.ugi.toString().equals(other.ugi.toString())) {
+        return false;
+      }
+      String thisTokens = this.getTokenIds().toString();
+      String otherTokens = other.getTokenIds().toString();
+      return thisTokens.equals(otherTokens);
+    }
+    return false;
+  }
+
+  @Override
+  public String toString() {
+    return this.ugi + " " + this.getTokenIds() + "->" + this.nnId;
+  }
+
+  @Override
+  public int compareTo(ConnectionPoolId other) {
+    int ret = this.nnId.compareTo(other.nnId);
+    if (ret == 0) {
+      ret = this.ugi.toString().compareTo(other.ugi.toString());
+    }
+    if (ret == 0) {
+      String thisTokens = this.getTokenIds().toString();
+      String otherTokens = other.getTokenIds().toString();
+      ret = thisTokens.compareTo(otherTokens);
+    }
+    return ret;
+  }
+
+  /**
+   * Get the token identifiers for this connection.
+   * @return List with the token identifiers.
+   */
+  private List<String> getTokenIds() {
+    List<String> tokenIds = new ArrayList<>();
+    Collection<Token<? extends TokenIdentifier>> tokens = this.ugi.getTokens();
+    for (Token<? extends TokenIdentifier> token : tokens) {
+      byte[] tokenIdBytes = token.getIdentifier();
+      String tokenId = Arrays.toString(tokenIdBytes);
+      tokenIds.add(tokenId);
+    }
+    Collections.sort(tokenIds);
+    return tokenIds;
+  }
+}

+ 76 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/DFSRouter.java

@@ -0,0 +1,76 @@
+/**
+ * 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.server.federation.router;
+
+import static org.apache.hadoop.util.ExitUtil.terminate;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.service.CompositeService.CompositeServiceShutdownHook;
+import org.apache.hadoop.util.ShutdownHookManager;
+import org.apache.hadoop.util.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Tool to start the {@link Router} for Router-based federation.
+ */
+public final class DFSRouter {
+
+  private static final Logger LOG = LoggerFactory.getLogger(DFSRouter.class);
+
+
+  /** Usage string for help message. */
+  private static final String USAGE = "Usage: hdfs dfsrouter";
+
+  /** Priority of the Router shutdown hook. */
+  public static final int SHUTDOWN_HOOK_PRIORITY = 30;
+
+
+  private DFSRouter() {
+    // This is just a class to trigger the Router
+  }
+
+  /**
+   * Main run loop for the router.
+   *
+   * @param argv parameters.
+   */
+  public static void main(String[] argv) {
+    if (DFSUtil.parseHelpArgument(argv, USAGE, System.out, true)) {
+      System.exit(0);
+    }
+
+    try {
+      StringUtils.startupShutdownMessage(Router.class, argv, LOG);
+
+      Router router = new Router();
+
+      ShutdownHookManager.get().addShutdownHook(
+          new CompositeServiceShutdownHook(router), SHUTDOWN_HOOK_PRIORITY);
+
+      Configuration conf = new HdfsConfiguration();
+      router.init(conf);
+      router.start();
+    } catch (Throwable e) {
+      LOG.error("Failed to start router", e);
+      terminate(1, e);
+    }
+  }
+}

+ 169 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/FederationUtil.java

@@ -0,0 +1,169 @@
+/**
+ * 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.server.federation.router;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.lang.reflect.Constructor;
+import java.net.URL;
+import java.net.URLConnection;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.server.federation.resolver.ActiveNamenodeResolver;
+import org.apache.hadoop.hdfs.server.federation.resolver.FileSubclusterResolver;
+import org.apache.hadoop.hdfs.server.federation.store.StateStoreService;
+import org.codehaus.jettison.json.JSONArray;
+import org.codehaus.jettison.json.JSONException;
+import org.codehaus.jettison.json.JSONObject;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Utilities for managing HDFS federation.
+ */
+public final class FederationUtil {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(FederationUtil.class);
+
+  private FederationUtil() {
+    // Utility Class
+  }
+
+  /**
+   * Get a JMX data from a web endpoint.
+   *
+   * @param beanQuery JMX bean.
+   * @param webAddress Web address of the JMX endpoint.
+   * @return JSON with the JMX data
+   */
+  public static JSONArray getJmx(String beanQuery, String webAddress) {
+    JSONArray ret = null;
+    BufferedReader reader = null;
+    try {
+      String host = webAddress;
+      int port = -1;
+      if (webAddress.indexOf(":") > 0) {
+        String[] webAddressSplit = webAddress.split(":");
+        host = webAddressSplit[0];
+        port = Integer.parseInt(webAddressSplit[1]);
+      }
+      URL jmxURL = new URL("http", host, port, "/jmx?qry=" + beanQuery);
+      URLConnection conn = jmxURL.openConnection();
+      conn.setConnectTimeout(5 * 1000);
+      conn.setReadTimeout(5 * 1000);
+      InputStream in = conn.getInputStream();
+      InputStreamReader isr = new InputStreamReader(in, "UTF-8");
+      reader = new BufferedReader(isr);
+
+      StringBuilder sb = new StringBuilder();
+      String line = null;
+      while ((line = reader.readLine()) != null) {
+        sb.append(line);
+      }
+      String jmxOutput = sb.toString();
+
+      // Parse JSON
+      JSONObject json = new JSONObject(jmxOutput);
+      ret = json.getJSONArray("beans");
+    } catch (IOException e) {
+      LOG.error("Cannot read JMX bean {} from server {}: {}",
+          beanQuery, webAddress, e.getMessage());
+    } catch (JSONException e) {
+      LOG.error("Cannot parse JMX output for {} from server {}: {}",
+          beanQuery, webAddress, e.getMessage());
+    } catch (Exception e) {
+      LOG.error("Cannot parse JMX output for {} from server {}: {}",
+          beanQuery, webAddress, e);
+    } finally {
+      if (reader != null) {
+        try {
+          reader.close();
+        } catch (IOException e) {
+          LOG.error("Problem closing {}", webAddress, e);
+        }
+      }
+    }
+    return ret;
+  }
+
+  /**
+   * Create an instance of an interface with a constructor using a context.
+   *
+   * @param conf Configuration for the class names.
+   * @param context Context object to pass to the instance.
+   * @param contextClass Type of the context passed to the constructor.
+   * @param clazz Class of the object to return.
+   * @return New instance of the specified class that implements the desired
+   *         interface and a single parameter constructor containing a
+   *         StateStore reference.
+   */
+  private static <T, R> T newInstance(final Configuration conf,
+      final R context, final Class<R> contextClass, final Class<T> clazz) {
+    try {
+      if (contextClass == null) {
+        // Default constructor if no context
+        Constructor<T> constructor = clazz.getConstructor();
+        return constructor.newInstance();
+      } else {
+        // Constructor with context
+        Constructor<T> constructor = clazz.getConstructor(
+            Configuration.class, contextClass);
+        return constructor.newInstance(conf, context);
+      }
+    } catch (ReflectiveOperationException e) {
+      LOG.error("Could not instantiate: {}", clazz.getSimpleName(), e);
+      return null;
+    }
+  }
+
+  /**
+   * Creates an instance of a FileSubclusterResolver from the configuration.
+   *
+   * @param conf Configuration that defines the file resolver class.
+   * @param router Router service.
+   * @return New file subcluster resolver.
+   */
+  public static FileSubclusterResolver newFileSubclusterResolver(
+      Configuration conf, Router router) {
+    Class<? extends FileSubclusterResolver> clazz = conf.getClass(
+        DFSConfigKeys.FEDERATION_FILE_RESOLVER_CLIENT_CLASS,
+        DFSConfigKeys.FEDERATION_FILE_RESOLVER_CLIENT_CLASS_DEFAULT,
+        FileSubclusterResolver.class);
+    return newInstance(conf, router, Router.class, clazz);
+  }
+
+  /**
+   * Creates an instance of an ActiveNamenodeResolver from the configuration.
+   *
+   * @param conf Configuration that defines the namenode resolver class.
+   * @param obj Context object passed to class constructor.
+   * @return New active namenode resolver.
+   */
+  public static ActiveNamenodeResolver newActiveNamenodeResolver(
+      Configuration conf, StateStoreService stateStore) {
+    Class<? extends ActiveNamenodeResolver> clazz = conf.getClass(
+        DFSConfigKeys.FEDERATION_NAMENODE_RESOLVER_CLIENT_CLASS,
+        DFSConfigKeys.FEDERATION_NAMENODE_RESOLVER_CLIENT_CLASS_DEFAULT,
+        ActiveNamenodeResolver.class);
+    return newInstance(conf, stateStore, StateStoreService.class, clazz);
+  }
+}

+ 361 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/NamenodeHeartbeatService.java

@@ -0,0 +1,361 @@
+/**
+ * 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.server.federation.router;
+
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_ROUTER_HEARTBEAT_INTERVAL_MS;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_ROUTER_HEARTBEAT_INTERVAL_MS_DEFAULT;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.net.URI;
+import java.util.Map;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.ha.HAServiceProtocol;
+import org.apache.hadoop.ha.HAServiceStatus;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.NameNodeProxies;
+import org.apache.hadoop.hdfs.protocol.ClientProtocol;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
+import org.apache.hadoop.hdfs.server.federation.resolver.ActiveNamenodeResolver;
+import org.apache.hadoop.hdfs.server.federation.resolver.NamenodeStatusReport;
+import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
+import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
+import org.apache.hadoop.hdfs.tools.NNHAServiceTarget;
+import org.codehaus.jettison.json.JSONArray;
+import org.codehaus.jettison.json.JSONObject;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * The {@link Router} periodically checks the state of a Namenode (usually on
+ * the same server) and reports their high availability (HA) state and
+ * load/space status to the
+ * {@link org.apache.hadoop.hdfs.server.federation.store.StateStoreService}
+ * . Note that this is an optional role as a Router can be independent of any
+ * subcluster.
+ * <p>
+ * For performance with Namenode HA, the Router uses the high availability state
+ * information in the State Store to forward the request to the Namenode that is
+ * most likely to be active.
+ * <p>
+ * Note that this service can be embedded into the Namenode itself to simplify
+ * the operation.
+ */
+public class NamenodeHeartbeatService extends PeriodicService {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(NamenodeHeartbeatService.class);
+
+
+  /** Configuration for the heartbeat. */
+  private Configuration conf;
+
+  /** Router performing the heartbeating. */
+  private final ActiveNamenodeResolver resolver;
+
+  /** Interface to the tracked NN. */
+  private final String nameserviceId;
+  private final String namenodeId;
+
+  /** Namenode HA target. */
+  private NNHAServiceTarget localTarget;
+  /** RPC address for the namenode. */
+  private String rpcAddress;
+  /** Service RPC address for the namenode. */
+  private String serviceAddress;
+  /** Service RPC address for the namenode. */
+  private String lifelineAddress;
+  /** HTTP address for the namenode. */
+  private String webAddress;
+
+  /**
+   * Create a new Namenode status updater.
+   * @param resolver Namenode resolver service to handle NN registration.
+   * @param nameserviceId Identifier of the nameservice.
+   * @param namenodeId Identifier of the namenode in HA.
+   */
+  public NamenodeHeartbeatService(
+      ActiveNamenodeResolver resolver, String nsId, String nnId) {
+    super(NamenodeHeartbeatService.class.getSimpleName() +
+        (nsId == null ? "" : " " + nsId) +
+        (nnId == null ? "" : " " + nnId));
+
+    this.resolver = resolver;
+
+    this.nameserviceId = nsId;
+    this.namenodeId = nnId;
+
+  }
+
+  @Override
+  protected void serviceInit(Configuration configuration) throws Exception {
+
+    this.conf = configuration;
+
+    String nnDesc = nameserviceId;
+    if (this.namenodeId != null && !this.namenodeId.isEmpty()) {
+      this.localTarget = new NNHAServiceTarget(
+          conf, nameserviceId, namenodeId);
+      nnDesc += "-" + namenodeId;
+    } else {
+      this.localTarget = null;
+    }
+
+    // Get the RPC address for the clients to connect
+    this.rpcAddress = getRpcAddress(conf, nameserviceId, namenodeId);
+    LOG.info("{} RPC address: {}", nnDesc, rpcAddress);
+
+    // Get the Service RPC address for monitoring
+    this.serviceAddress =
+        DFSUtil.getNamenodeServiceAddr(conf, nameserviceId, namenodeId);
+    if (this.serviceAddress == null) {
+      LOG.error("Cannot locate RPC service address for NN {}, " +
+          "using RPC address {}", nnDesc, this.rpcAddress);
+      this.serviceAddress = this.rpcAddress;
+    }
+    LOG.info("{} Service RPC address: {}", nnDesc, serviceAddress);
+
+    // Get the Lifeline RPC address for faster monitoring
+    this.lifelineAddress =
+        DFSUtil.getNamenodeLifelineAddr(conf, nameserviceId, namenodeId);
+    if (this.lifelineAddress == null) {
+      this.lifelineAddress = this.serviceAddress;
+    }
+    LOG.info("{} Lifeline RPC address: {}", nnDesc, lifelineAddress);
+
+    // Get the Web address for UI
+    this.webAddress =
+        DFSUtil.getNamenodeWebAddr(conf, nameserviceId, namenodeId);
+    LOG.info("{} Web address: {}", nnDesc, webAddress);
+
+    this.setIntervalMs(conf.getLong(
+        DFS_ROUTER_HEARTBEAT_INTERVAL_MS,
+        DFS_ROUTER_HEARTBEAT_INTERVAL_MS_DEFAULT));
+
+
+    super.serviceInit(configuration);
+  }
+
+  @Override
+  public void periodicInvoke() {
+    updateState();
+  }
+
+  /**
+   * Get the RPC address for a Namenode.
+   * @param conf Configuration.
+   * @param nsId Name service identifier.
+   * @param nnId Name node identifier.
+   * @return RPC address in format hostname:1234.
+   */
+  private static String getRpcAddress(
+      Configuration conf, String nsId, String nnId) {
+
+    // Get it from the regular RPC setting
+    String confKey = DFSConfigKeys.DFS_NAMENODE_RPC_ADDRESS_KEY;
+    String ret = conf.get(confKey);
+
+    if (nsId != null || nnId != null) {
+      // Get if for the proper nameservice and namenode
+      confKey = DFSUtil.addKeySuffixes(confKey, nsId, nnId);
+      ret = conf.get(confKey);
+
+      // If not available, get it from the map
+      if (ret == null) {
+        Map<String, InetSocketAddress> rpcAddresses =
+            DFSUtil.getRpcAddressesForNameserviceId(conf, nsId, null);
+        InetSocketAddress sockAddr = null;
+        if (nnId != null) {
+          sockAddr = rpcAddresses.get(nnId);
+        } else if (rpcAddresses.size() == 1) {
+          // Get the only namenode in the namespace
+          sockAddr = rpcAddresses.values().iterator().next();
+        }
+        if (sockAddr != null) {
+          InetAddress addr = sockAddr.getAddress();
+          ret = addr.getHostName() + ":" + sockAddr.getPort();
+        }
+      }
+    }
+    return ret;
+  }
+
+  /**
+   * Update the state of the Namenode.
+   */
+  private void updateState() {
+    NamenodeStatusReport report = getNamenodeStatusReport();
+    if (!report.registrationValid()) {
+      // Not operational
+      LOG.error("Namenode is not operational: {}", getNamenodeDesc());
+    } else if (report.haStateValid()) {
+      // block and HA status available
+      LOG.debug("Received service state: {} from HA namenode: {}",
+          report.getState(), getNamenodeDesc());
+    } else if (localTarget == null) {
+      // block info available, HA status not expected
+      LOG.debug(
+          "Reporting non-HA namenode as operational: " + getNamenodeDesc());
+    } else {
+      // block info available, HA status should be available, but was not
+      // fetched do nothing and let the current state stand
+      return;
+    }
+    try {
+      if (!resolver.registerNamenode(report)) {
+        LOG.warn("Cannot register namenode {}", report);
+      }
+    } catch (IOException e) {
+      LOG.info("Cannot register namenode in the State Store");
+    } catch (Exception ex) {
+      LOG.error("Unhandled exception updating NN registration for {}",
+          getNamenodeDesc(), ex);
+    }
+  }
+
+  /**
+   * Get the status report for the Namenode monitored by this heartbeater.
+   * @return Namenode status report.
+   */
+  protected NamenodeStatusReport getNamenodeStatusReport() {
+    NamenodeStatusReport report = new NamenodeStatusReport(nameserviceId,
+        namenodeId, rpcAddress, serviceAddress, lifelineAddress, webAddress);
+
+    try {
+      LOG.debug("Probing NN at service address: {}", serviceAddress);
+
+      URI serviceURI = new URI("hdfs://" + serviceAddress);
+      // Read the filesystem info from RPC (required)
+      NamenodeProtocol nn = NameNodeProxies
+          .createProxy(this.conf, serviceURI, NamenodeProtocol.class)
+          .getProxy();
+
+      if (nn != null) {
+        NamespaceInfo info = nn.versionRequest();
+        if (info != null) {
+          report.setNamespaceInfo(info);
+        }
+      }
+      if (!report.registrationValid()) {
+        return report;
+      }
+
+      // Check for safemode from the client protocol. Currently optional, but
+      // should be required at some point for QoS
+      try {
+        ClientProtocol client = NameNodeProxies
+            .createProxy(this.conf, serviceURI, ClientProtocol.class)
+            .getProxy();
+        if (client != null) {
+          boolean isSafeMode = client.setSafeMode(
+              SafeModeAction.SAFEMODE_GET, false);
+          report.setSafeMode(isSafeMode);
+        }
+      } catch (Exception e) {
+        LOG.error("Cannot fetch safemode state for {}", getNamenodeDesc(), e);
+      }
+
+      // Read the stats from JMX (optional)
+      updateJMXParameters(webAddress, report);
+
+      if (localTarget != null) {
+        // Try to get the HA status
+        try {
+          // Determine if NN is active
+          // TODO: dynamic timeout
+          HAServiceProtocol haProtocol = localTarget.getProxy(conf, 30*1000);
+          HAServiceStatus status = haProtocol.getServiceStatus();
+          report.setHAServiceState(status.getState());
+        } catch (Throwable e) {
+          if (e.getMessage().startsWith("HA for namenode is not enabled")) {
+            LOG.error("HA for {} is not enabled", getNamenodeDesc());
+            localTarget = null;
+          } else {
+            // Failed to fetch HA status, ignoring failure
+            LOG.error("Cannot fetch HA status for {}: {}",
+                getNamenodeDesc(), e.getMessage(), e);
+          }
+        }
+      }
+    } catch(IOException e) {
+      LOG.error("Cannot communicate with {}: {}",
+          getNamenodeDesc(), e.getMessage());
+    } catch(Throwable e) {
+      // Generic error that we don't know about
+      LOG.error("Unexpected exception while communicating with {}: {}",
+          getNamenodeDesc(), e.getMessage(), e);
+    }
+    return report;
+  }
+
+  /**
+   * Get the description of the Namenode to monitor.
+   * @return Description of the Namenode to monitor.
+   */
+  public String getNamenodeDesc() {
+    if (namenodeId != null && !namenodeId.isEmpty()) {
+      return nameserviceId + "-" + namenodeId + ":" + serviceAddress;
+    } else {
+      return nameserviceId + ":" + serviceAddress;
+    }
+  }
+
+  /**
+   * Get the parameters for a Namenode from JMX and add them to the report.
+   * @param webAddress Web interface of the Namenode to monitor.
+   * @param report Namenode status report to update with JMX data.
+   */
+  private void updateJMXParameters(
+      String address, NamenodeStatusReport report) {
+    try {
+      // TODO part of this should be moved to its own utility
+      String query = "Hadoop:service=NameNode,name=FSNamesystem*";
+      JSONArray aux = FederationUtil.getJmx(query, address);
+      if (aux != null) {
+        for (int i = 0; i < aux.length(); i++) {
+          JSONObject jsonObject = aux.getJSONObject(i);
+          String name = jsonObject.getString("name");
+          if (name.equals("Hadoop:service=NameNode,name=FSNamesystemState")) {
+            report.setDatanodeInfo(
+                jsonObject.getInt("NumLiveDataNodes"),
+                jsonObject.getInt("NumDeadDataNodes"),
+                jsonObject.getInt("NumDecommissioningDataNodes"),
+                jsonObject.getInt("NumDecomLiveDataNodes"),
+                jsonObject.getInt("NumDecomDeadDataNodes"));
+          } else if (name.equals(
+              "Hadoop:service=NameNode,name=FSNamesystem")) {
+            report.setNamesystemInfo(
+                jsonObject.getLong("CapacityRemaining"),
+                jsonObject.getLong("CapacityTotal"),
+                jsonObject.getLong("FilesTotal"),
+                jsonObject.getLong("BlocksTotal"),
+                jsonObject.getLong("MissingBlocks"),
+                jsonObject.getLong("PendingReplicationBlocks"),
+                jsonObject.getLong("UnderReplicatedBlocks"),
+                jsonObject.getLong("PendingDeletionBlocks"));
+          }
+        }
+      }
+    } catch (Exception e) {
+      LOG.error("Cannot get stat from {} using JMX", getNamenodeDesc(), e);
+    }
+  }
+}

+ 198 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/PeriodicService.java

@@ -0,0 +1,198 @@
+/**
+ * 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.server.federation.router;
+
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.hadoop.service.AbstractService;
+import org.apache.hadoop.service.ServiceStateException;
+import org.apache.hadoop.util.Time;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+
+/**
+ * Service to periodically execute a runnable.
+ */
+public abstract class PeriodicService extends AbstractService {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(PeriodicService.class);
+
+  /** Default interval in milliseconds for the periodic service. */
+  private static final long DEFAULT_INTERVAL_MS = TimeUnit.MINUTES.toMillis(1);
+
+
+  /** Interval for running the periodic service in milliseconds. */
+  private long intervalMs;
+  /** Name of the service. */
+  private final String serviceName;
+
+  /** Scheduler for the periodic service. */
+  private final ScheduledExecutorService scheduler;
+
+  /** If the service is running. */
+  private volatile boolean isRunning = false;
+
+  /** How many times we run. */
+  private long runCount;
+  /** How many errors we got. */
+  private long errorCount;
+  /** When was the last time we executed this service successfully. */
+  private long lastRun;
+
+  /**
+   * Create a new periodic update service.
+   *
+   * @param name Name of the service.
+   */
+  public PeriodicService(String name) {
+    this(name, DEFAULT_INTERVAL_MS);
+  }
+
+  /**
+   * Create a new periodic update service.
+   *
+   * @param name Name of the service.
+   * @param interval Interval for the periodic service in milliseconds.
+   */
+  public PeriodicService(String name, long interval) {
+    super(name);
+    this.serviceName = name;
+    this.intervalMs = interval;
+
+    ThreadFactory threadFactory = new ThreadFactoryBuilder()
+        .setNameFormat(this.getName() + "-%d")
+        .build();
+    this.scheduler = Executors.newScheduledThreadPool(1, threadFactory);
+  }
+
+  /**
+   * Set the interval for the periodic service.
+   *
+   * @param interval Interval in milliseconds.
+   */
+  protected void setIntervalMs(long interval) {
+    if (getServiceState() == STATE.STARTED) {
+      throw new ServiceStateException("Periodic service already started");
+    } else {
+      this.intervalMs = interval;
+    }
+  }
+
+  /**
+   * Get the interval for the periodic service.
+   *
+   * @return Interval in milliseconds.
+   */
+  protected long getIntervalMs() {
+    return this.intervalMs;
+  }
+
+  /**
+   * Get how many times we failed to run the periodic service.
+   *
+   * @return Times we failed to run the periodic service.
+   */
+  protected long getErrorCount() {
+    return this.errorCount;
+  }
+
+  /**
+   * Get how many times we run the periodic service.
+   *
+   * @return Times we run the periodic service.
+   */
+  protected long getRunCount() {
+    return this.runCount;
+  }
+
+  /**
+   * Get the last time the periodic service was executed.
+   *
+   * @return Last time the periodic service was executed.
+   */
+  protected long getLastUpdate() {
+    return this.lastRun;
+  }
+
+  @Override
+  protected void serviceStart() throws Exception {
+    super.serviceStart();
+    LOG.info("Starting periodic service {}", this.serviceName);
+    startPeriodic();
+  }
+
+  @Override
+  protected void serviceStop() throws Exception {
+    stopPeriodic();
+    LOG.info("Stopping periodic service {}", this.serviceName);
+    super.serviceStop();
+  }
+
+  /**
+   * Stop the periodic task.
+   */
+  protected synchronized void stopPeriodic() {
+    if (this.isRunning) {
+      LOG.info("{} is shutting down", this.serviceName);
+      this.isRunning = false;
+      this.scheduler.shutdownNow();
+    }
+  }
+
+  /**
+   * Start the periodic execution.
+   */
+  protected synchronized void startPeriodic() {
+    stopPeriodic();
+
+    // Create the runnable service
+    Runnable updateRunnable = new Runnable() {
+      @Override
+      public void run() {
+        LOG.debug("Running {} update task", serviceName);
+        try {
+          if (!isRunning) {
+            return;
+          }
+          periodicInvoke();
+          runCount++;
+          lastRun = Time.now();
+        } catch (Exception ex) {
+          errorCount++;
+          LOG.warn(serviceName + " service threw an exception", ex);
+        }
+      }
+    };
+
+    // Start the execution of the periodic service
+    this.isRunning = true;
+    this.scheduler.scheduleWithFixedDelay(
+        updateRunnable, 0, this.intervalMs, TimeUnit.MILLISECONDS);
+  }
+
+  /**
+   * Method that the service will run periodically.
+   */
+  protected abstract void periodicInvoke();
+}

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

@@ -0,0 +1,68 @@
+/**
+ * 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.server.federation.router;
+
+import org.apache.commons.lang.builder.HashCodeBuilder;
+
+/**
+ * Base class for objects that are unique to a namespace.
+ */
+public abstract class RemoteLocationContext
+    implements Comparable<RemoteLocationContext> {
+
+  /**
+   * Returns an identifier for a unique namespace.
+   *
+   * @return Namespace identifier.
+   */
+  public abstract String getNameserviceId();
+
+  /**
+   * Destination in this location. For example the path in a remote namespace.
+   *
+   * @return Destination in this location.
+   */
+  public abstract String getDest();
+
+  @Override
+  public int hashCode() {
+    return new HashCodeBuilder(17, 31)
+        .append(getNameserviceId())
+        .append(getDest())
+        .toHashCode();
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (obj instanceof RemoteLocationContext) {
+      RemoteLocationContext other = (RemoteLocationContext) obj;
+      return this.getNameserviceId().equals(other.getNameserviceId()) &&
+          this.getDest().equals(other.getDest());
+    }
+    return false;
+  }
+
+  @Override
+  public int compareTo(RemoteLocationContext info) {
+    int ret = this.getNameserviceId().compareTo(info.getNameserviceId());
+    if (ret == 0) {
+      ret = this.getDest().compareTo(info.getDest());
+    }
+    return ret;
+  }
+}

+ 164 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RemoteMethod.java

@@ -0,0 +1,164 @@
+/**
+ * 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.server.federation.router;
+
+import java.io.IOException;
+import java.lang.reflect.Method;
+import java.util.Arrays;
+
+import org.apache.hadoop.hdfs.protocol.ClientProtocol;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Determines the remote client protocol method and the parameter list for a
+ * specific location.
+ */
+public class RemoteMethod {
+
+  private static final Logger LOG = LoggerFactory.getLogger(RemoteMethod.class);
+
+
+  /** List of parameters: static and dynamic values, matchings types. */
+  private final Object[] params;
+  /** List of method parameters types, matches parameters. */
+  private final Class<?>[] types;
+  /** String name of the ClientProtocol method. */
+  private final String methodName;
+
+  /**
+   * Create a method with no parameters.
+   *
+   * @param method The string name of the ClientProtocol method.
+   */
+  public RemoteMethod(String method) {
+    this.params = null;
+    this.types = null;
+    this.methodName = method;
+  }
+
+  /**
+   * Creates a remote method generator.
+   *
+   * @param method The string name of the ClientProtocol method.
+   * @param pTypes A list of types to use to locate the specific method.
+   * @param pParams A list of parameters for the method. The order of the
+   *          parameter list must match the order and number of the types.
+   *          Parameters are grouped into 2 categories:
+   *          <ul>
+   *          <li>Static parameters that are immutable across locations.
+   *          <li>Dynamic parameters that are determined for each location by a
+   *          RemoteParam object. To specify a dynamic parameter, pass an
+   *          instance of RemoteParam in place of the parameter value.
+   *          </ul>
+   * @throws IOException If the types and parameter lists are not valid.
+   */
+  public RemoteMethod(String method, Class<?>[] pTypes, Object... pParams)
+      throws IOException {
+
+    if (pParams.length != pTypes.length) {
+      throw new IOException("Invalid parameters for method " + method);
+    }
+
+    this.params = pParams;
+    this.types = pTypes;
+    this.methodName = method;
+  }
+
+  /**
+   * Get the represented java method.
+   *
+   * @return Method
+   * @throws IOException If the method cannot be found.
+   */
+  public Method getMethod() throws IOException {
+    try {
+      if (types != null) {
+        return ClientProtocol.class.getDeclaredMethod(methodName, types);
+      } else {
+        return ClientProtocol.class.getDeclaredMethod(methodName);
+      }
+    } catch (NoSuchMethodException e) {
+      // Re-throw as an IOException
+      LOG.error("Cannot get method {} with types {}",
+          methodName, Arrays.toString(types), e);
+      throw new IOException(e);
+    } catch (SecurityException e) {
+      LOG.error("Cannot access method {} with types {}",
+          methodName, Arrays.toString(types), e);
+      throw new IOException(e);
+    }
+  }
+
+  /**
+   * Get the calling types for this method.
+   *
+   * @return An array of calling types.
+   */
+  public Class<?>[] getTypes() {
+    return this.types;
+  }
+
+  /**
+   * Generate a list of parameters for this specific location using no context.
+   *
+   * @return A list of parameters for the method customized for the location.
+   */
+  public Object[] getParams() {
+    return this.getParams(null);
+  }
+
+  /**
+   * Get the name of the method.
+   *
+   * @return Name of the method.
+   */
+  public String getMethodName() {
+    return this.methodName;
+  }
+
+  /**
+   * Generate a list of parameters for this specific location. Parameters are
+   * grouped into 2 categories:
+   * <ul>
+   * <li>Static parameters that are immutable across locations.
+   * <li>Dynamic parameters that are determined for each location by a
+   * RemoteParam object.
+   * </ul>
+   *
+   * @param context The context identifying the location.
+   * @return A list of parameters for the method customized for the location.
+   */
+  public Object[] getParams(RemoteLocationContext context) {
+    if (this.params == null) {
+      return new Object[] {};
+    }
+    Object[] objList = new Object[this.params.length];
+    for (int i = 0; i < this.params.length; i++) {
+      Object currentObj = this.params[i];
+      if (currentObj instanceof RemoteParam) {
+        // Map the parameter using the context
+        RemoteParam paramGetter = (RemoteParam) currentObj;
+        objList[i] = paramGetter.getParameterForContext(context);
+      } else {
+        objList[i] = currentObj;
+      }
+    }
+    return objList;
+  }
+}

+ 71 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RemoteParam.java

@@ -0,0 +1,71 @@
+/**
+ * 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.server.federation.router;
+
+import java.util.Map;
+
+/**
+ * A dynamically assignable parameter that is location-specific.
+ * <p>
+ * There are 2 ways this mapping is determined:
+ * <ul>
+ * <li>Default: Uses the RemoteLocationContext's destination
+ * <li>Map: Uses the value of the RemoteLocationContext key provided in the
+ * parameter map.
+ * </ul>
+ */
+public class RemoteParam {
+
+  private final Map<? extends Object, ? extends Object> paramMap;
+
+  /**
+   * Constructs a default remote parameter. Always maps the value to the
+   * destination of the provided RemoveLocationContext.
+   */
+  public RemoteParam() {
+    this.paramMap = null;
+  }
+
+  /**
+   * Constructs a map based remote parameter. Determines the value using the
+   * provided RemoteLocationContext as a key into the map.
+   *
+   * @param map Map with RemoteLocationContext keys.
+   */
+  public RemoteParam(
+      Map<? extends RemoteLocationContext, ? extends Object> map) {
+    this.paramMap = map;
+  }
+
+  /**
+   * Determine the appropriate value for this parameter based on the location.
+   *
+   * @param context Context identifying the location.
+   * @return A parameter specific to this location.
+   */
+  public Object getParameterForContext(RemoteLocationContext context) {
+    if (context == null) {
+      return null;
+    } else if (this.paramMap != null) {
+      return this.paramMap.get(context);
+    } else {
+      // Default case
+      return context.getDest();
+    }
+  }
+}

+ 527 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/Router.java

@@ -0,0 +1,527 @@
+/**
+ * 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.server.federation.router;
+
+import static org.apache.hadoop.hdfs.server.federation.router.FederationUtil.newActiveNamenodeResolver;
+import static org.apache.hadoop.hdfs.server.federation.router.FederationUtil.newFileSubclusterResolver;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.net.UnknownHostException;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.HAUtil;
+import org.apache.hadoop.hdfs.server.federation.metrics.FederationMetrics;
+import org.apache.hadoop.hdfs.server.federation.resolver.ActiveNamenodeResolver;
+import org.apache.hadoop.hdfs.server.federation.resolver.FileSubclusterResolver;
+import org.apache.hadoop.hdfs.server.federation.store.StateStoreService;
+import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
+import org.apache.hadoop.metrics2.source.JvmMetrics;
+import org.apache.hadoop.service.CompositeService;
+import org.apache.hadoop.util.JvmPauseMonitor;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Router that provides a unified view of multiple federated HDFS clusters. It
+ * has two main roles: (1) federated interface and (2) NameNode heartbeat.
+ * <p>
+ * For the federated interface, the Router receives a client request, checks the
+ * State Store for the correct subcluster, and forwards the request to the
+ * active Namenode of that subcluster. The reply from the Namenode then flows in
+ * the opposite direction. The Routers are stateless and can be behind a load
+ * balancer. HDFS clients connect to the router using the same interfaces as are
+ * used to communicate with a namenode, namely the ClientProtocol RPC interface
+ * and the WebHdfs HTTP interface exposed by the router. {@link RouterRpcServer}
+ * {@link RouterHttpServer}
+ * <p>
+ * For NameNode heartbeat, the Router periodically checks the state of a
+ * NameNode (usually on the same server) and reports their high availability
+ * (HA) state and load/space status to the State Store. Note that this is an
+ * optional role as a Router can be independent of any subcluster.
+ * {@link StateStoreService} {@link NamenodeHeartbeatService}
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class Router extends CompositeService {
+
+  private static final Logger LOG = LoggerFactory.getLogger(Router.class);
+
+
+  /** Configuration for the Router. */
+  private Configuration conf;
+
+  /** Router address/identifier. */
+  private String routerId;
+
+  /** RPC interface to the client. */
+  private RouterRpcServer rpcServer;
+  private InetSocketAddress rpcAddress;
+
+  /** RPC interface for the admin. */
+  private RouterAdminServer adminServer;
+  private InetSocketAddress adminAddress;
+
+  /** HTTP interface and web application. */
+  private RouterHttpServer httpServer;
+
+  /** Interface with the State Store. */
+  private StateStoreService stateStore;
+
+  /** Interface to map global name space to HDFS subcluster name spaces. */
+  private FileSubclusterResolver subclusterResolver;
+
+  /** Interface to identify the active NN for a nameservice or blockpool ID. */
+  private ActiveNamenodeResolver namenodeResolver;
+  /** Updates the namenode status in the namenode resolver. */
+  private Collection<NamenodeHeartbeatService> namenodeHearbeatServices;
+
+  /** Router metrics. */
+  private RouterMetricsService metrics;
+
+  /** JVM pauses (GC and others). */
+  private JvmPauseMonitor pauseMonitor;
+
+
+
+  /////////////////////////////////////////////////////////
+  // Constructor
+  /////////////////////////////////////////////////////////
+
+  public Router() {
+    super(Router.class.getName());
+  }
+
+  /////////////////////////////////////////////////////////
+  // Service management
+  /////////////////////////////////////////////////////////
+
+  @Override
+  protected void serviceInit(Configuration configuration) throws Exception {
+    this.conf = configuration;
+
+    if (conf.getBoolean(
+        DFSConfigKeys.DFS_ROUTER_STORE_ENABLE,
+        DFSConfigKeys.DFS_ROUTER_STORE_ENABLE_DEFAULT)) {
+      // Service that maintains the State Store connection
+      this.stateStore = new StateStoreService();
+      addService(this.stateStore);
+    }
+
+    // Resolver to track active NNs
+    this.namenodeResolver = newActiveNamenodeResolver(
+        this.conf, this.stateStore);
+    if (this.namenodeResolver == null) {
+      throw new IOException("Cannot find namenode resolver.");
+    }
+
+    // Lookup interface to map between the global and subcluster name spaces
+    this.subclusterResolver = newFileSubclusterResolver(this.conf, this);
+    if (this.subclusterResolver == null) {
+      throw new IOException("Cannot find subcluster resolver");
+    }
+
+    if (conf.getBoolean(
+        DFSConfigKeys.DFS_ROUTER_RPC_ENABLE,
+        DFSConfigKeys.DFS_ROUTER_RPC_ENABLE_DEFAULT)) {
+      // Create RPC server
+      this.rpcServer = createRpcServer();
+      addService(this.rpcServer);
+      this.setRpcServerAddress(rpcServer.getRpcAddress());
+    }
+
+    if (conf.getBoolean(
+        DFSConfigKeys.DFS_ROUTER_ADMIN_ENABLE,
+        DFSConfigKeys.DFS_ROUTER_ADMIN_ENABLE_DEFAULT)) {
+      // Create admin server
+      this.adminServer = createAdminServer();
+      addService(this.adminServer);
+    }
+
+    if (conf.getBoolean(
+        DFSConfigKeys.DFS_ROUTER_HTTP_ENABLE,
+        DFSConfigKeys.DFS_ROUTER_HTTP_ENABLE_DEFAULT)) {
+      // Create HTTP server
+      this.httpServer = createHttpServer();
+      addService(this.httpServer);
+    }
+
+    if (conf.getBoolean(
+        DFSConfigKeys.DFS_ROUTER_HEARTBEAT_ENABLE,
+        DFSConfigKeys.DFS_ROUTER_HEARTBEAT_ENABLE_DEFAULT)) {
+
+      // Create status updater for each monitored Namenode
+      this.namenodeHearbeatServices = createNamenodeHearbeatServices();
+      for (NamenodeHeartbeatService hearbeatService :
+          this.namenodeHearbeatServices) {
+        addService(hearbeatService);
+      }
+
+      if (this.namenodeHearbeatServices.isEmpty()) {
+        LOG.error("Heartbeat is enabled but there are no namenodes to monitor");
+      }
+    }
+
+    // Router metrics system
+    if (conf.getBoolean(
+        DFSConfigKeys.DFS_ROUTER_METRICS_ENABLE,
+        DFSConfigKeys.DFS_ROUTER_METRICS_ENABLE_DEFAULT)) {
+
+      DefaultMetricsSystem.initialize("Router");
+
+      this.metrics = new RouterMetricsService(this);
+      addService(this.metrics);
+
+      // JVM pause monitor
+      this.pauseMonitor = new JvmPauseMonitor();
+      this.pauseMonitor.init(conf);
+    }
+
+    super.serviceInit(conf);
+  }
+
+  @Override
+  protected void serviceStart() throws Exception {
+
+    if (this.pauseMonitor != null) {
+      this.pauseMonitor.start();
+      JvmMetrics jvmMetrics = this.metrics.getJvmMetrics();
+      if (jvmMetrics != null) {
+        jvmMetrics.setPauseMonitor(pauseMonitor);
+      }
+    }
+
+    super.serviceStart();
+  }
+
+  @Override
+  protected void serviceStop() throws Exception {
+
+    // JVM pause monitor
+    if (this.pauseMonitor != null) {
+      this.pauseMonitor.stop();
+    }
+
+    super.serviceStop();
+  }
+
+  /**
+   * Shutdown the router.
+   */
+  public void shutDown() {
+    new Thread() {
+      @Override
+      public void run() {
+        Router.this.stop();
+      }
+    }.start();
+  }
+
+  /////////////////////////////////////////////////////////
+  // RPC Server
+  /////////////////////////////////////////////////////////
+
+  /**
+   * Create a new Router RPC server to proxy ClientProtocol requests.
+   *
+   * @return New Router RPC Server.
+   * @throws IOException If the router RPC server was not started.
+   */
+  protected RouterRpcServer createRpcServer() throws IOException {
+    return new RouterRpcServer(this.conf, this, this.getNamenodeResolver(),
+        this.getSubclusterResolver());
+  }
+
+  /**
+   * Get the Router RPC server.
+   *
+   * @return Router RPC server.
+   */
+  public RouterRpcServer getRpcServer() {
+    return this.rpcServer;
+  }
+
+  /**
+   * Set the current RPC socket for the router.
+   *
+   * @param rpcAddress RPC address.
+   */
+  protected void setRpcServerAddress(InetSocketAddress address) {
+    this.rpcAddress = address;
+
+    // Use the RPC address as our unique router Id
+    if (this.rpcAddress != null) {
+      try {
+        String hostname = InetAddress.getLocalHost().getHostName();
+        setRouterId(hostname + ":" + this.rpcAddress.getPort());
+      } catch (UnknownHostException ex) {
+        LOG.error("Cannot set unique router ID, address not resolvable {}",
+            this.rpcAddress);
+      }
+    }
+  }
+
+  /**
+   * Get the current RPC socket address for the router.
+   *
+   * @return InetSocketAddress
+   */
+  public InetSocketAddress getRpcServerAddress() {
+    return this.rpcAddress;
+  }
+
+  /////////////////////////////////////////////////////////
+  // Admin server
+  /////////////////////////////////////////////////////////
+
+  /**
+   * Create a new router admin server to handle the router admin interface.
+   *
+   * @return RouterAdminServer
+   * @throws IOException If the admin server was not successfully started.
+   */
+  protected RouterAdminServer createAdminServer() throws IOException {
+    return new RouterAdminServer(this.conf, this);
+  }
+
+  /**
+   * Set the current Admin socket for the router.
+   *
+   * @param adminAddress Admin RPC address.
+   */
+  protected void setAdminServerAddress(InetSocketAddress address) {
+    this.adminAddress = address;
+  }
+
+  /**
+   * Get the current Admin socket address for the router.
+   *
+   * @return InetSocketAddress Admin address.
+   */
+  public InetSocketAddress getAdminServerAddress() {
+    return adminAddress;
+  }
+
+  /////////////////////////////////////////////////////////
+  // HTTP server
+  /////////////////////////////////////////////////////////
+
+  /**
+   * Create an HTTP server for this Router.
+   *
+   * @return HTTP server for this Router.
+   */
+  protected RouterHttpServer createHttpServer() {
+    return new RouterHttpServer(this);
+  }
+
+  /**
+   * Get the current HTTP socket address for the router.
+   *
+   * @return InetSocketAddress HTTP address.
+   */
+  public InetSocketAddress getHttpServerAddress() {
+    if (httpServer != null) {
+      return httpServer.getHttpAddress();
+    }
+    return null;
+  }
+
+  /////////////////////////////////////////////////////////
+  // Namenode heartbeat monitors
+  /////////////////////////////////////////////////////////
+
+  /**
+   * Create each of the services that will monitor a Namenode.
+   *
+   * @return List of heartbeat services.
+   */
+  protected Collection<NamenodeHeartbeatService>
+      createNamenodeHearbeatServices() {
+
+    Map<String, NamenodeHeartbeatService> ret = new HashMap<>();
+
+    if (conf.getBoolean(
+        DFSConfigKeys.DFS_ROUTER_MONITOR_LOCAL_NAMENODE,
+        DFSConfigKeys.DFS_ROUTER_MONITOR_LOCAL_NAMENODE_DEFAULT)) {
+      // Create a local heartbet service
+      NamenodeHeartbeatService localHeartbeatService =
+          createLocalNamenodeHearbeatService();
+      if (localHeartbeatService != null) {
+        String nnDesc = localHeartbeatService.getNamenodeDesc();
+        ret.put(nnDesc, localHeartbeatService);
+      }
+    }
+
+    // Create heartbeat services for a list specified by the admin
+    String namenodes = this.conf.get(
+        DFSConfigKeys.DFS_ROUTER_MONITOR_NAMENODE);
+    if (namenodes != null) {
+      for (String namenode : namenodes.split(",")) {
+        String[] namenodeSplit = namenode.split("\\.");
+        String nsId = null;
+        String nnId = null;
+        if (namenodeSplit.length == 2) {
+          nsId = namenodeSplit[0];
+          nnId = namenodeSplit[1];
+        } else if (namenodeSplit.length == 1) {
+          nsId = namenode;
+        } else {
+          LOG.error("Wrong Namenode to monitor: {}", namenode);
+        }
+        if (nsId != null) {
+          NamenodeHeartbeatService heartbeatService =
+              createNamenodeHearbeatService(nsId, nnId);
+          if (heartbeatService != null) {
+            ret.put(heartbeatService.getNamenodeDesc(), heartbeatService);
+          }
+        }
+      }
+    }
+
+    return ret.values();
+  }
+
+  /**
+   * Create a new status updater for the local Namenode.
+   *
+   * @return Updater of the status for the local Namenode.
+   */
+  protected NamenodeHeartbeatService createLocalNamenodeHearbeatService() {
+    // Detect NN running in this machine
+    String nsId = DFSUtil.getNamenodeNameServiceId(conf);
+    String nnId = null;
+    if (HAUtil.isHAEnabled(conf, nsId)) {
+      nnId = HAUtil.getNameNodeId(conf, nsId);
+      if (nnId == null) {
+        LOG.error("Cannot find namenode id for local {}", nsId);
+      }
+    }
+
+    return createNamenodeHearbeatService(nsId, nnId);
+  }
+
+  /**
+   * Create a heartbeat monitor for a particular Namenode.
+   *
+   * @param nsId Identifier of the nameservice to monitor.
+   * @param nnId Identifier of the namenode (HA) to monitor.
+   * @return Updater of the status for the specified Namenode.
+   */
+  protected NamenodeHeartbeatService createNamenodeHearbeatService(
+      String nsId, String nnId) {
+
+    LOG.info("Creating heartbeat service for Namenode {} in {}", nnId, nsId);
+    NamenodeHeartbeatService ret = new NamenodeHeartbeatService(
+        namenodeResolver, nsId, nnId);
+    return ret;
+  }
+
+  /////////////////////////////////////////////////////////
+  // Submodule getters
+  /////////////////////////////////////////////////////////
+
+  /**
+   * Get the State Store service.
+   *
+   * @return State Store service.
+   */
+  public StateStoreService getStateStore() {
+    return this.stateStore;
+  }
+
+  /**
+   * Get the metrics system for the Router.
+   *
+   * @return Router metrics.
+   */
+  public RouterMetrics getRouterMetrics() {
+    if (this.metrics != null) {
+      return this.metrics.getRouterMetrics();
+    }
+    return null;
+  }
+
+  /**
+   * Get the federation metrics.
+   *
+   * @return Federation metrics.
+   */
+  public FederationMetrics getMetrics() {
+    if (this.metrics != null) {
+      return this.metrics.getFederationMetrics();
+    }
+    return null;
+  }
+
+  /**
+   * Get the subcluster resolver for files.
+   *
+   * @return Subcluster resolver for files.
+   */
+  public FileSubclusterResolver getSubclusterResolver() {
+    return this.subclusterResolver;
+  }
+
+  /**
+   * Get the namenode resolver for a subcluster.
+   *
+   * @return The namenode resolver for a subcluster.
+   */
+  public ActiveNamenodeResolver getNamenodeResolver() {
+    return this.namenodeResolver;
+  }
+
+  /////////////////////////////////////////////////////////
+  // Router info
+  /////////////////////////////////////////////////////////
+
+  /**
+   * Unique ID for the router, typically the hostname:port string for the
+   * router's RPC server. This ID may be null on router startup before the RPC
+   * server has bound to a port.
+   *
+   * @return Router identifier.
+   */
+  public String getRouterId() {
+    return this.routerId;
+  }
+
+  /**
+   * Sets a unique ID for this router.
+   *
+   * @param router Identifier of the Router.
+   */
+  public void setRouterId(String id) {
+    this.routerId = id;
+    if (this.stateStore != null) {
+      this.stateStore.setIdentifier(this.routerId);
+    }
+    if (this.namenodeResolver != null) {
+      this.namenodeResolver.setRouterId(this.routerId);
+    }
+  }
+}

+ 183 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterAdminServer.java

@@ -0,0 +1,183 @@
+/**
+ * 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.server.federation.router;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.protocol.proto.RouterProtocolProtos.RouterAdminProtocolService;
+import org.apache.hadoop.hdfs.protocolPB.RouterAdminProtocolPB;
+import org.apache.hadoop.hdfs.protocolPB.RouterAdminProtocolServerSideTranslatorPB;
+import org.apache.hadoop.hdfs.server.federation.resolver.MountTableManager;
+import org.apache.hadoop.hdfs.server.federation.store.MountTableStore;
+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.protocol.GetMountTableEntriesRequest;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.GetMountTableEntriesResponse;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.RemoveMountTableEntryRequest;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.RemoveMountTableEntryResponse;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.UpdateMountTableEntryRequest;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.UpdateMountTableEntryResponse;
+import org.apache.hadoop.ipc.ProtobufRpcEngine;
+import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.ipc.RPC.Server;
+import org.apache.hadoop.service.AbstractService;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.protobuf.BlockingService;
+
+/**
+ * This class is responsible for handling all of the Admin calls to the HDFS
+ * router. It is created, started, and stopped by {@link Router}.
+ */
+public class RouterAdminServer extends AbstractService
+    implements MountTableManager {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(RouterAdminServer.class);
+
+  private Configuration conf;
+
+  private final Router router;
+
+  private MountTableStore mountTableStore;
+
+  /** The Admin server that listens to requests from clients. */
+  private final Server adminServer;
+  private final InetSocketAddress adminAddress;
+
+  public RouterAdminServer(Configuration conf, Router router)
+      throws IOException {
+    super(RouterAdminServer.class.getName());
+
+    this.conf = conf;
+    this.router = router;
+
+    int handlerCount = this.conf.getInt(
+        DFSConfigKeys.DFS_ROUTER_ADMIN_HANDLER_COUNT_KEY,
+        DFSConfigKeys.DFS_ROUTER_ADMIN_HANDLER_COUNT_DEFAULT);
+
+    RPC.setProtocolEngine(this.conf, RouterAdminProtocolPB.class,
+        ProtobufRpcEngine.class);
+
+    RouterAdminProtocolServerSideTranslatorPB routerAdminProtocolTranslator =
+        new RouterAdminProtocolServerSideTranslatorPB(this);
+    BlockingService clientNNPbService = RouterAdminProtocolService.
+        newReflectiveBlockingService(routerAdminProtocolTranslator);
+
+    InetSocketAddress confRpcAddress = conf.getSocketAddr(
+        DFSConfigKeys.DFS_ROUTER_ADMIN_BIND_HOST_KEY,
+        DFSConfigKeys.DFS_ROUTER_ADMIN_ADDRESS_KEY,
+        DFSConfigKeys.DFS_ROUTER_ADMIN_ADDRESS_DEFAULT,
+        DFSConfigKeys.DFS_ROUTER_ADMIN_PORT_DEFAULT);
+
+    String bindHost = conf.get(
+        DFSConfigKeys.DFS_ROUTER_ADMIN_BIND_HOST_KEY,
+        confRpcAddress.getHostName());
+    LOG.info("Admin server binding to {}:{}",
+        bindHost, confRpcAddress.getPort());
+
+    this.adminServer = new RPC.Builder(this.conf)
+        .setProtocol(RouterAdminProtocolPB.class)
+        .setInstance(clientNNPbService)
+        .setBindAddress(bindHost)
+        .setPort(confRpcAddress.getPort())
+        .setNumHandlers(handlerCount)
+        .setVerbose(false)
+        .build();
+
+    // The RPC-server port can be ephemeral... ensure we have the correct info
+    InetSocketAddress listenAddress = this.adminServer.getListenerAddress();
+    this.adminAddress = new InetSocketAddress(
+        confRpcAddress.getHostName(), listenAddress.getPort());
+    router.setAdminServerAddress(this.adminAddress);
+  }
+
+  /** Allow access to the client RPC server for testing. */
+  @VisibleForTesting
+  Server getAdminServer() {
+    return this.adminServer;
+  }
+
+  private MountTableStore getMountTableStore() throws IOException {
+    if (this.mountTableStore == null) {
+      this.mountTableStore = router.getStateStore().getRegisteredRecordStore(
+          MountTableStore.class);
+      if (this.mountTableStore == null) {
+        throw new IOException("Mount table state store is not available.");
+      }
+    }
+    return this.mountTableStore;
+  }
+
+  /**
+   * Get the RPC address of the admin service.
+   * @return Administration service RPC address.
+   */
+  public InetSocketAddress getRpcAddress() {
+    return this.adminAddress;
+  }
+
+  @Override
+  protected void serviceInit(Configuration configuration) throws Exception {
+    this.conf = configuration;
+    super.serviceInit(conf);
+  }
+
+  @Override
+  protected void serviceStart() throws Exception {
+    this.adminServer.start();
+    super.serviceStart();
+  }
+
+  @Override
+  protected void serviceStop() throws Exception {
+    if (this.adminServer != null) {
+      this.adminServer.stop();
+    }
+    super.serviceStop();
+  }
+
+  @Override
+  public AddMountTableEntryResponse addMountTableEntry(
+      AddMountTableEntryRequest request) throws IOException {
+    return getMountTableStore().addMountTableEntry(request);
+  }
+
+  @Override
+  public UpdateMountTableEntryResponse updateMountTableEntry(
+      UpdateMountTableEntryRequest request) throws IOException {
+    return getMountTableStore().updateMountTableEntry(request);
+  }
+
+  @Override
+  public RemoveMountTableEntryResponse removeMountTableEntry(
+      RemoveMountTableEntryRequest request) throws IOException {
+    return getMountTableStore().removeMountTableEntry(request);
+  }
+
+  @Override
+  public GetMountTableEntriesResponse getMountTableEntries(
+      GetMountTableEntriesRequest request) throws IOException {
+    return getMountTableStore().getMountTableEntries(request);
+  }
+}

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