Explorar el Código

Merge branch 'trunk' into HDFS-7240

Anu Engineer hace 7 años
padre
commit
b2b043ef1e
Se han modificado 100 ficheros con 4302 adiciones y 381 borrados
  1. 8 0
      hadoop-common-project/hadoop-common/src/main/conf/hadoop-policy.xml
  2. 2 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java
  3. 2 2
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileContext.java
  4. 2 2
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java
  5. 2 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FsShell.java
  6. 1 9
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/ZKFailoverController.java
  7. 39 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpServer2.java
  8. 69 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/IOUtils.java
  9. 10 5
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/DecayRpcScheduler.java
  10. 2 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/Credentials.java
  11. 26 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/SecurityUtil.java
  12. 36 17
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/ShellBasedUnixGroupsMapping.java
  13. 59 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/DataChecksum.java
  14. 15 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Shell.java
  15. 3 14
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/curator/ZKCuratorManager.java
  16. 59 0
      hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
  17. 1 1
      hadoop-common-project/hadoop-common/src/site/markdown/CLIMiniCluster.md.vm
  18. 2 2
      hadoop-common-project/hadoop-common/src/site/markdown/CredentialProviderAPI.md
  19. 1 1
      hadoop-common-project/hadoop-common/src/site/markdown/SecureMode.md
  20. 14 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFsShellList.java
  21. 60 1
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/TestIOUtils.java
  22. 29 3
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestDecayRpcScheduler.java
  23. 81 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestSecurityUtil.java
  24. 16 1
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/Crc32PerformanceTest.java
  25. 5 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestShell.java
  26. 33 3
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
  27. 2 2
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
  28. 2 2
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsAdmin.java
  29. 24 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/AddErasureCodingPolicyResponse.java
  30. 3 2
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
  31. 8 2
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/OpenFilesIterator.java
  32. 1 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolTranslatorPB.java
  33. 6 2
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
  34. 1 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto
  35. 1 0
      hadoop-hdfs-project/hadoop-hdfs/pom.xml
  36. 18 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
  37. 4 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HDFSPolicyProvider.java
  38. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
  39. 54 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocol/InterQJournalProtocol.java
  40. 40 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocolPB/InterQJournalProtocolPB.java
  41. 64 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocolPB/InterQJournalProtocolServerSideTranslatorPB.java
  42. 96 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocolPB/InterQJournalProtocolTranslatorPB.java
  43. 37 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JournalNodeRpcServer.java
  44. 29 28
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JournalNodeSyncer.java
  45. 19 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java
  46. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ShortCircuitRegistry.java
  47. 3 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/StripedBlockReader.java
  48. 15 8
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/BlockPoolSlice.java
  49. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
  50. 4 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/web/DatanodeHttpServer.java
  51. 65 23
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/MountTableResolver.java
  52. 198 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/ErasureCoding.java
  53. 20 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/FederationUtil.java
  54. 208 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/Quota.java
  55. 37 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/Router.java
  56. 160 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterQuotaManager.java
  57. 228 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterQuotaUpdateService.java
  58. 119 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterQuotaUsage.java
  59. 53 12
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcClient.java
  60. 159 128
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java
  61. 31 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterServiceState.java
  62. 78 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/RouterStore.java
  63. 93 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/impl/RouterStoreImpl.java
  64. 47 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/GetRouterRegistrationRequest.java
  65. 44 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/GetRouterRegistrationResponse.java
  66. 34 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/GetRouterRegistrationsRequest.java
  67. 55 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/GetRouterRegistrationsResponse.java
  68. 50 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/RouterHeartbeatRequest.java
  69. 49 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/RouterHeartbeatResponse.java
  70. 70 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/impl/pb/GetRouterRegistrationRequestPBImpl.java
  71. 79 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/impl/pb/GetRouterRegistrationResponsePBImpl.java
  72. 61 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/impl/pb/GetRouterRegistrationsRequestPBImpl.java
  73. 102 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/impl/pb/GetRouterRegistrationsResponsePBImpl.java
  74. 78 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/impl/pb/RouterHeartbeatRequestPBImpl.java
  75. 70 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/impl/pb/RouterHeartbeatResponsePBImpl.java
  76. 28 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/MountTable.java
  77. 169 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/RouterState.java
  78. 91 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/StateStoreVersion.java
  79. 39 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/impl/pb/MountTablePBImpl.java
  80. 202 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/impl/pb/RouterStatePBImpl.java
  81. 79 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/impl/pb/StateStoreVersionPBImpl.java
  82. 18 9
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
  83. 10 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
  84. 20 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java
  85. 5 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
  86. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Quota.java
  87. 64 51
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java
  88. 18 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java
  89. 140 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/federation/RouterAdmin.java
  90. 47 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/proto/FederationProtocol.proto
  91. 50 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/proto/InterQJournalProtocol.proto
  92. 38 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
  93. 2 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/router/federationhealth.html
  94. 6 2
      hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md
  95. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSHighAvailabilityWithNFS.md
  96. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSHighAvailabilityWithQJM.md
  97. 29 0
      hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSRouterFederation.md
  98. 4 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/StripedFileTestUtil.java
  99. 37 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDecommission.java
  100. 3 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestHdfsAdmin.java

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

@@ -123,6 +123,14 @@
     JNs when using the QuorumJournalManager for edit logs.</description>
   </property>
 
+  <property>
+    <name>security.interqjournal.service.protocol.acl</name>
+    <value>*</value>
+    <description>ACL for InterQJournalProtocol, used by the JN to
+    communicate with other JN
+    </description>
+  </property>
+
   <property>
     <name>security.mrhs.client.protocol.acl</name>
     <value>*</value>

+ 2 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java

@@ -224,6 +224,8 @@ public class CommonConfigurationKeys extends CommonConfigurationKeysPublic {
   SECURITY_NAMENODE_PROTOCOL_ACL = "security.namenode.protocol.acl";
   public static final String SECURITY_QJOURNAL_SERVICE_PROTOCOL_ACL =
       "security.qjournal.service.protocol.acl";
+  public static final String SECURITY_INTERQJOURNAL_SERVICE_PROTOCOL_ACL =
+      "security.interqjournal.service.protocol.acl";
   public static final String HADOOP_SECURITY_TOKEN_SERVICE_USE_IP =
       "hadoop.security.token.service.use_ip";
   public static final boolean HADOOP_SECURITY_TOKEN_SERVICE_USE_IP_DEFAULT =

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

@@ -2012,7 +2012,7 @@ public class FileContext {
      *  </dd>
      * </dl>
      *
-     * @param pathPattern a regular expression specifying a pth pattern
+     * @param pathPattern a glob specifying a path pattern
      *
      * @return an array of paths that match the path pattern
      *
@@ -2040,7 +2040,7 @@ public class FileContext {
      * Return null if pathPattern has no glob and the path does not exist.
      * Return an empty array if pathPattern has a glob and no path matches it. 
      * 
-     * @param pathPattern regular expression specifying the path pattern
+     * @param pathPattern glob specifying the path pattern
      * @param filter user-supplied path filter
      *
      * @return an array of FileStatus objects

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

@@ -2058,7 +2058,7 @@ public abstract class FileSystem extends Configured implements Closeable {
    *  </dd>
    * </dl>
    *
-   * @param pathPattern a regular expression specifying a pth pattern
+   * @param pathPattern a glob specifying a path pattern
 
    * @return an array of paths that match the path pattern
    * @throws IOException IO failure
@@ -2072,7 +2072,7 @@ public abstract class FileSystem extends Configured implements Closeable {
    * {@code pathPattern} and is accepted by the user-supplied path filter.
    * Results are sorted by their path names.
    *
-   * @param pathPattern a regular expression specifying the path pattern
+   * @param pathPattern a glob specifying the path pattern
    * @param filter a user-supplied path filter
    * @return null if {@code pathPattern} has no glob and the path does not exist
    *         an empty array if {@code pathPattern} has a glob and no path

+ 2 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FsShell.java

@@ -30,6 +30,7 @@ import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.fs.shell.Command;
 import org.apache.hadoop.fs.shell.CommandFactory;
 import org.apache.hadoop.fs.shell.FsCommand;
+import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.tools.TableListing;
 import org.apache.hadoop.tracing.TraceUtils;
 import org.apache.hadoop.util.StringUtils;
@@ -99,6 +100,7 @@ public class FsShell extends Configured implements Tool {
   
   protected void init() throws IOException {
     getConf().setQuietMode(true);
+    UserGroupInformation.setConfiguration(getConf());
     if (commandFactory == null) {
       commandFactory = new CommandFactory(getConf());
       commandFactory.addObject(new Help(), "-help");

+ 1 - 9
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/ZKFailoverController.java

@@ -22,7 +22,6 @@ import java.net.InetSocketAddress;
 import java.security.PrivilegedAction;
 import java.security.PrivilegedExceptionAction;
 import java.util.ArrayList;
-import java.util.Collections;
 import java.util.List;
 import java.util.concurrent.Executors;
 import java.util.concurrent.ScheduledExecutorService;
@@ -341,14 +340,7 @@ public abstract class ZKFailoverController {
     }
     
     // Parse authentication from configuration.
-    String zkAuthConf = conf.get(ZK_AUTH_KEY);
-    zkAuthConf = ZKUtil.resolveConfIndirection(zkAuthConf);
-    List<ZKAuthInfo> zkAuths;
-    if (zkAuthConf != null) {
-      zkAuths = ZKUtil.parseAuth(zkAuthConf);
-    } else {
-      zkAuths = Collections.emptyList();
-    }
+    List<ZKAuthInfo> zkAuths = SecurityUtil.getZKAuthInfos(conf, ZK_AUTH_KEY);
 
     // Sanity check configuration.
     Preconditions.checkArgument(zkQuorum != null,

+ 39 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpServer2.java

@@ -864,6 +864,45 @@ public final class HttpServer2 implements FilterContainer {
     }
   }
 
+  /**
+   * Add an internal servlet in the server, with initialization parameters.
+   * Note: This method is to be used for adding servlets that facilitate
+   * internal communication and not for user facing functionality. For
+   * servlets added using this method, filters (except internal Kerberos
+   * filters) are not enabled.
+   *
+   * @param name The name of the servlet (can be passed as null)
+   * @param pathSpec The path spec for the servlet
+   * @param clazz The servlet class
+   * @param params init parameters
+   */
+  public void addInternalServlet(String name, String pathSpec,
+      Class<? extends HttpServlet> clazz, Map<String, String> params) {
+    // Jetty doesn't like the same path spec mapping to different servlets, so
+    // if there's already a mapping for this pathSpec, remove it and assume that
+    // the newest one is the one we want
+    final ServletHolder sh = new ServletHolder(clazz);
+    sh.setName(name);
+    sh.setInitParameters(params);
+    final ServletMapping[] servletMappings =
+        webAppContext.getServletHandler().getServletMappings();
+    for (int i = 0; i < servletMappings.length; i++) {
+      if (servletMappings[i].containsPathSpec(pathSpec)) {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Found existing " + servletMappings[i].getServletName() +
+              " servlet at path " + pathSpec + "; will replace mapping" +
+              " with " + sh.getName() + " servlet");
+        }
+        ServletMapping[] newServletMappings =
+            ArrayUtil.removeFromArray(servletMappings, servletMappings[i]);
+        webAppContext.getServletHandler()
+            .setServletMappings(newServletMappings);
+        break;
+      }
+    }
+    webAppContext.addServlet(sh, pathSpec);
+  }
+
   /**
    * Add the given handler to the front of the list of handlers.
    *

+ 69 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/IOUtils.java

@@ -19,6 +19,7 @@
 package org.apache.hadoop.io;
 
 import java.io.*;
+import java.lang.reflect.Constructor;
 import java.net.Socket;
 import java.nio.ByteBuffer;
 import java.nio.channels.FileChannel;
@@ -35,6 +36,7 @@ import org.apache.commons.logging.Log;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.PathIOException;
 import org.apache.hadoop.util.Shell;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -296,7 +298,19 @@ public class IOUtils {
       cleanupWithLogger(null, stream);
     }
   }
-  
+
+  /**
+   * Closes the streams ignoring {@link Throwable}.
+   * Must only be called in cleaning up from exception handlers.
+   *
+   * @param streams the Streams to close
+   */
+  public static void closeStreams(java.io.Closeable... streams) {
+    if (streams != null) {
+      cleanupWithLogger(null, streams);
+    }
+  }
+
   /**
    * Closes the socket ignoring {@link IOException}
    *
@@ -438,4 +452,58 @@ public class IOUtils {
       throw ioe;
     }
   }
+
+  /**
+   * Takes an IOException, file/directory path, and method name and returns an
+   * IOException with the input exception as the cause and also include the
+   * file,method details. The new exception provides the stack trace of the
+   * place where the exception is thrown and some extra diagnostics
+   * information.
+   *
+   * Return instance of same exception if exception class has a public string
+   * constructor; Otherwise return an PathIOException.
+   * InterruptedIOException and PathIOException are returned unwrapped.
+   *
+   * @param path file/directory path
+   * @param methodName method name
+   * @param exception the caught exception.
+   * @return an exception to throw
+   */
+  public static IOException wrapException(final String path,
+      final String methodName, final IOException exception) {
+
+    if (exception instanceof InterruptedIOException
+        || exception instanceof PathIOException) {
+      return exception;
+    } else {
+      String msg = String
+          .format("Failed with %s while processing file/directory :[%s] in "
+                  + "method:[%s]",
+              exception.getClass().getName(), path, methodName);
+      try {
+        return wrapWithMessage(exception, msg);
+      } catch (Exception ex) {
+        // For subclasses which have no (String) constructor throw IOException
+        // with wrapped message
+
+        return new PathIOException(path, exception);
+      }
+    }
+  }
+
+  @SuppressWarnings("unchecked")
+  private static <T extends IOException> T wrapWithMessage(
+      final T exception, final String msg) throws T {
+    Class<? extends Throwable> clazz = exception.getClass();
+    try {
+      Constructor<? extends Throwable> ctor = clazz
+          .getConstructor(String.class);
+      Throwable t = ctor.newInstance(msg);
+      return (T) (t.initCause(exception));
+    } catch (Throwable e) {
+      LOG.warn("Unable to wrap exception of type " +
+          clazz + ": it has no (String) constructor", e);
+      throw exception;
+    }
+  }
 }

+ 10 - 5
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/DecayRpcScheduler.java

@@ -236,8 +236,8 @@ public class DecayRpcScheduler implements RpcScheduler,
     DecayTask task = new DecayTask(this, timer);
     timer.scheduleAtFixedRate(task, decayPeriodMillis, decayPeriodMillis);
 
-    metricsProxy = MetricsProxy.getInstance(ns, numLevels);
-    metricsProxy.setDelegate(this);
+    metricsProxy = MetricsProxy.getInstance(ns, numLevels, this);
+    recomputeScheduleCache();
   }
 
   // Load configs
@@ -680,21 +680,26 @@ public class DecayRpcScheduler implements RpcScheduler,
     private long[] callCountInLastWindowDefault;
     private ObjectName decayRpcSchedulerInfoBeanName;
 
-    private MetricsProxy(String namespace, int numLevels) {
+    private MetricsProxy(String namespace, int numLevels,
+        DecayRpcScheduler drs) {
       averageResponseTimeDefault = new double[numLevels];
       callCountInLastWindowDefault = new long[numLevels];
+      setDelegate(drs);
       decayRpcSchedulerInfoBeanName =
           MBeans.register(namespace, "DecayRpcScheduler", this);
       this.registerMetrics2Source(namespace);
     }
 
     public static synchronized MetricsProxy getInstance(String namespace,
-        int numLevels) {
+        int numLevels, DecayRpcScheduler drs) {
       MetricsProxy mp = INSTANCES.get(namespace);
       if (mp == null) {
         // We must create one
-        mp = new MetricsProxy(namespace, numLevels);
+        mp = new MetricsProxy(namespace, numLevels, drs);
         INSTANCES.put(namespace, mp);
+      } else  if (drs != mp.delegate.get()){
+        // in case of delegate is reclaimed, we should set it again
+        mp.setDelegate(drs);
       }
       return mp;
     }

+ 2 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/Credentials.java

@@ -207,7 +207,8 @@ public class Credentials implements Writable {
       in.close();
       return credentials;
     } catch(IOException ioe) {
-      throw new IOException("Exception reading " + filename, ioe);
+      throw IOUtils.wrapException(filename.toString(), "Credentials"
+          + ".readTokenStorageFile", ioe);
     } finally {
       IOUtils.cleanupWithLogger(LOG, in);
     }

+ 26 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/SecurityUtil.java

@@ -28,6 +28,7 @@ import java.net.UnknownHostException;
 import java.security.PrivilegedAction;
 import java.security.PrivilegedExceptionAction;
 import java.util.Arrays;
+import java.util.Collections;
 import java.util.List;
 import java.util.ServiceLoader;
 import java.util.concurrent.TimeUnit;
@@ -48,7 +49,7 @@ import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.TokenInfo;
 import org.apache.hadoop.util.StopWatch;
 import org.apache.hadoop.util.StringUtils;
-
+import org.apache.hadoop.util.ZKUtil;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 //this will need to be replaced someday when there is a suitable replacement
@@ -719,4 +720,28 @@ public final class SecurityUtil {
   public static boolean isPrivilegedPort(final int port) {
     return port < 1024;
   }
+
+  /**
+   * Utility method to fetch ZK auth info from the configuration.
+   * @throws java.io.IOException if the Zookeeper ACLs configuration file
+   * cannot be read
+   * @throws ZKUtil.BadAuthFormatException if the auth format is invalid
+   */
+  public static List<ZKUtil.ZKAuthInfo> getZKAuthInfos(Configuration conf,
+      String configKey) throws IOException {
+    char[] zkAuthChars = conf.getPassword(configKey);
+    String zkAuthConf =
+        zkAuthChars != null ? String.valueOf(zkAuthChars) : null;
+    try {
+      zkAuthConf = ZKUtil.resolveConfIndirection(zkAuthConf);
+      if (zkAuthConf != null) {
+        return ZKUtil.parseAuth(zkAuthConf);
+      } else {
+        return Collections.emptyList();
+      }
+    } catch (IOException | ZKUtil.BadAuthFormatException e) {
+      LOG.error("Couldn't read Auth based on {}", configKey);
+      throw e;
+    }
+  }
 }

+ 36 - 17
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/ShellBasedUnixGroupsMapping.java

@@ -157,6 +157,32 @@ public class ShellBasedUnixGroupsMapping extends Configured
     return Shell.getGroupsIDForUserCommand(userName);
   }
 
+  /**
+   * Check if the executor had a timeout and logs the event.
+   * @param executor to check
+   * @param user user to log
+   * @return true if timeout has occurred
+   */
+  private boolean handleExecutorTimeout(
+      ShellCommandExecutor executor,
+      String user) {
+    // If its a shell executor timeout, indicate so in the message
+    // but treat the result as empty instead of throwing it up,
+    // similar to how partial resolution failures are handled above
+    if (executor.isTimedOut()) {
+      LOG.warn(
+          "Unable to return groups for user '{}' as shell group lookup " +
+              "command '{}' ran longer than the configured timeout limit of " +
+              "{} seconds.",
+          user,
+          Joiner.on(' ').join(executor.getExecString()),
+          timeout
+      );
+      return true;
+    }
+    return false;
+  }
+
   /**
    * Get the current user's group list from Unix by running the command 'groups'
    * NOTE. For non-existing user it will return EMPTY list.
@@ -174,26 +200,19 @@ public class ShellBasedUnixGroupsMapping extends Configured
       executor.execute();
       groups = resolveFullGroupNames(executor.getOutput());
     } catch (ExitCodeException e) {
-      try {
-        groups = resolvePartialGroupNames(user, e.getMessage(),
-            executor.getOutput());
-      } catch (PartialGroupNameException pge) {
-        LOG.warn("unable to return groups for user {}", user, pge);
+      if (handleExecutorTimeout(executor, user)) {
         return EMPTY_GROUPS;
+      } else {
+        try {
+          groups = resolvePartialGroupNames(user, e.getMessage(),
+              executor.getOutput());
+        } catch (PartialGroupNameException pge) {
+          LOG.warn("unable to return groups for user {}", user, pge);
+          return EMPTY_GROUPS;
+        }
       }
     } catch (IOException ioe) {
-      // If its a shell executor timeout, indicate so in the message
-      // but treat the result as empty instead of throwing it up,
-      // similar to how partial resolution failures are handled above
-      if (executor.isTimedOut()) {
-        LOG.warn(
-            "Unable to return groups for user '{}' as shell group lookup " +
-            "command '{}' ran longer than the configured timeout limit of " +
-            "{} seconds.",
-            user,
-            Joiner.on(' ').join(executor.getExecString()),
-            timeout
-        );
+      if (handleExecutorTimeout(executor, user)) {
         return EMPTY_GROUPS;
       } else {
         // If its not an executor timeout, we should let the caller handle it

+ 59 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/DataChecksum.java

@@ -28,6 +28,12 @@ import java.util.zip.Checksum;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.ChecksumException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.lang.invoke.MethodHandle;
+import java.lang.invoke.MethodHandles;
+import java.lang.invoke.MethodType;
 
 /**
  * This class provides interface and utilities for processing checksums for
@@ -43,6 +49,9 @@ public class DataChecksum implements Checksum {
   public static final int CHECKSUM_CRC32C  = 2;
   public static final int CHECKSUM_DEFAULT = 3; 
   public static final int CHECKSUM_MIXED   = 4;
+
+  private static final Logger LOG = LoggerFactory.getLogger(DataChecksum.class);
+  private static volatile boolean useJava9Crc32C = Shell.isJavaVersionAtLeast(9);
  
   /** The checksum types */
   public enum Type {
@@ -78,6 +87,23 @@ public class DataChecksum implements Checksum {
     return new CRC32();
   }
 
+
+  /**
+   * The flag is volatile to avoid synchronization here.
+   * Re-entrancy is unlikely except in failure mode (and inexpensive).
+   */
+  static Checksum newCrc32C() {
+    try {
+      return useJava9Crc32C ? Java9Crc32CFactory.createChecksum()
+          : new PureJavaCrc32C();
+    } catch (ExceptionInInitializerError | RuntimeException e) {
+      // should not happen
+      LOG.error("CRC32C creation failed, switching to PureJavaCrc32C", e);
+      useJava9Crc32C = false;
+      return new PureJavaCrc32C();
+    }
+  }
+
   public static DataChecksum newDataChecksum(Type type, int bytesPerChecksum ) {
     if ( bytesPerChecksum <= 0 ) {
       return null;
@@ -89,7 +115,7 @@ public class DataChecksum implements Checksum {
     case CRC32 :
       return new DataChecksum(type, newCrc32(), bytesPerChecksum );
     case CRC32C:
-      return new DataChecksum(type, new PureJavaCrc32C(), bytesPerChecksum);
+      return new DataChecksum(type, newCrc32C(), bytesPerChecksum);
     default:
       return null;  
     }
@@ -528,4 +554,36 @@ public class DataChecksum implements Checksum {
     @Override
     public void update(int b) {}
   };
+
+  /**
+   * Holds constructor handle to let it be initialized on demand.
+   */
+  private static class Java9Crc32CFactory {
+    private static final MethodHandle NEW_CRC32C_MH;
+
+    static {
+      MethodHandle newCRC32C = null;
+      try {
+        newCRC32C = MethodHandles.publicLookup()
+            .findConstructor(
+                Class.forName("java.util.zip.CRC32C"),
+                MethodType.methodType(void.class)
+            );
+      } catch (ReflectiveOperationException e) {
+        // Should not reach here.
+        throw new RuntimeException(e);
+      }
+      NEW_CRC32C_MH = newCRC32C;
+    }
+
+    public static Checksum createChecksum() {
+      try {
+        // Should throw nothing
+        return (Checksum) NEW_CRC32C_MH.invoke();
+      } catch (Throwable t) {
+        throw (t instanceof RuntimeException) ? (RuntimeException) t
+            : new RuntimeException(t);
+      }
+    }
+  };
 }

+ 15 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Shell.java

@@ -89,6 +89,21 @@ public abstract class Shell {
     return true;
   }
 
+  // "1.8"->8, "9"->9, "10"->10
+  private static final int JAVA_SPEC_VER = Math.max(8, Integer.parseInt(
+      System.getProperty("java.specification.version").split("\\.")[0]));
+
+  /**
+   * Query to see if major version of Java specification of the system
+   * is equal or greater than the parameter.
+   *
+   * @param version 8, 9, 10 etc.
+   * @return comparison with system property, always true for 8
+   */
+  public static boolean isJavaVersionAtLeast(int version) {
+    return JAVA_SPEC_VER >= version;
+  }
+
   /**
    * Maximum command line length in Windows
    * KB830473 documents this as 8191

+ 3 - 14
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/curator/ZKCuratorManager.java

@@ -20,7 +20,6 @@ package org.apache.hadoop.util.curator;
 import java.io.IOException;
 import java.nio.charset.Charset;
 import java.util.ArrayList;
-import java.util.Collections;
 import java.util.List;
 
 import org.apache.curator.framework.AuthInfo;
@@ -32,6 +31,7 @@ import org.apache.curator.retry.RetryNTimes;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
+import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.util.ZKUtil;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.data.ACL;
@@ -100,22 +100,11 @@ public final class ZKCuratorManager {
    * Utility method to fetch ZK auth info from the configuration.
    * @throws java.io.IOException if the Zookeeper ACLs configuration file
    * cannot be read
+   * @throws ZKUtil.BadAuthFormatException if the auth format is invalid
    */
   public static List<ZKUtil.ZKAuthInfo> getZKAuths(Configuration conf)
       throws IOException {
-    String zkAuthConf = conf.get(CommonConfigurationKeys.ZK_AUTH);
-    try {
-      zkAuthConf = ZKUtil.resolveConfIndirection(zkAuthConf);
-      if (zkAuthConf != null) {
-        return ZKUtil.parseAuth(zkAuthConf);
-      } else {
-        return Collections.emptyList();
-      }
-    } catch (IOException | ZKUtil.BadAuthFormatException e) {
-      LOG.error("Couldn't read Auth based on {}",
-          CommonConfigurationKeys.ZK_AUTH);
-      throw e;
-    }
+    return SecurityUtil.getZKAuthInfos(conf, CommonConfigurationKeys.ZK_AUTH);
   }
 
   /**

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

@@ -955,6 +955,65 @@
   </description>
 </property>
 
+<property>
+  <name>fs.s3a.assumed.role.arn</name>
+  <value />
+  <description>
+    AWS ARN for the role to be assumed.
+    Required if the fs.s3a.aws.credentials.provider contains
+    org.apache.hadoop.fs.s3a.AssumedRoleCredentialProvider
+  </description>
+</property>
+
+<property>
+  <name>fs.s3a.assumed.role.session.name</name>
+  <value />
+  <description>
+    Session name for the assumed role, must be valid characters according to
+    the AWS APIs.
+    Only used if AssumedRoleCredentialProvider is the AWS credential provider.
+    If not set, one is generated from the current Hadoop/Kerberos username.
+  </description>
+</property>
+
+<property>
+  <name>fs.s3a.assumed.role.session.duration</name>
+  <value>30m</value>
+  <description>
+    Duration of assumed roles before a refresh is attempted.
+    Only used if AssumedRoleCredentialProvider is the AWS credential provider.
+  </description>
+</property>
+
+<property>
+  <name>fs.s3a.assumed.role.policy</name>
+  <value/>
+  <description>
+    JSON policy containing more restrictions to apply to the role.
+    Only used if AssumedRoleCredentialProvider is the AWS credential provider.
+  </description>
+</property>
+
+<property>
+  <name>fs.s3a.assumed.role.sts.endpoint</name>
+  <value/>
+  <description>
+    AWS Simple Token Service Endpoint. If unset, uses the default endpoint.
+    Only used if AssumedRoleCredentialProvider is the AWS credential provider.
+  </description>
+</property>
+
+<property>
+  <name>fs.s3a.assumed.role.credentials.provider</name>
+  <value>org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider</value>
+  <description>
+    List of credential providers to authenticate with the STS endpoint and
+    retrieve short-lived role credentials.
+    Only used if AssumedRoleCredentialProvider is the AWS credential provider.
+    If unset, uses "org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider".
+  </description>
+</property>
+
 <property>
   <name>fs.s3a.connection.maximum</name>
   <value>15</value>

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

@@ -41,7 +41,7 @@ Running the MiniCluster
 
 From inside the root directory of the extracted tarball, you can start the CLI MiniCluster using the following command:
 
-    $ HADOOP_CLASSPATH=share/hadoop/yarn/test/hadoop-yarn-server-tests-${project.version}-tests.jar bin/hadoop jar ./share/hadoop/mapreduce/hadoop-mapreduce-client-jobclient-${project.version}-tests.jar minicluster -rmport RM_PORT -jhsport JHS_PORT
+    $ bin/mapred minicluster -rmport RM_PORT -jhsport JHS_PORT
 
 In the example command above, `RM_PORT` and `JHS_PORT` should be replaced by the user's choice of these port numbers. If not specified, random free ports will be used.
 

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

@@ -98,8 +98,8 @@ In summary, first, provision the credentials into a provider then configure the
 |:---- |:---- |:---|
 |LDAPGroupsMapping    |LDAPGroupsMapping is used to look up the groups for a given user in LDAP. The CredentialProvider API is used to protect the LDAP bind password and those needed for SSL.|[LDAP Groups Mapping](GroupsMapping.html#LDAP_Groups_Mapping)|
 |SSL Passwords        |FileBasedKeyStoresFactory leverages the credential provider API in order to resolve the SSL related passwords.|TODO|
-|HDFS                 |DFSUtil leverages Configuration.getPassword method to use the credential provider API and/or fallback to the clear text value stored in ssl-server.xml.|TODO|
-|YARN                 |WebAppUtils uptakes the use of the credential provider API through the new method on Configuration called getPassword. This provides an alternative to storing the passwords in clear text within the ssl-server.xml file while maintaining backward compatibility.|TODO|
+|HDFS                 |DFSUtil leverages Configuration.getPassword method to use the credential provider API and/or fallback to the clear text value stored in ssl-server.xml. Zookeeper based federation state store and failover controller use Configuration.getPassword to get the Zookeeper authentication info, with fallback provided to clear text auth info.|TODO|
+|YARN                 |WebAppUtils uptakes the use of the credential provider API through the new method on Configuration called getPassword. This provides an alternative to storing the passwords in clear text within the ssl-server.xml file while maintaining backward compatibility. Zookeeper based resource manager state store uses Configuration.getPassword to get the Zookeeper authentication info, with fallback provided to clear text auth info.|TODO|
 |KMS                  |Uses HttpServer2.loadSSLConfiguration that leverages Configuration.getPassword to read SSL related credentials. They may be resolved through Credential Provider and/or from the clear text in the config when allowed.|[KMS](../../hadoop-kms/index.html)|
 |HttpFS               |Uses HttpServer2.loadSSLConfiguration that leverages Configuration.getPassword to read SSL related credentials. They may be resolved through Credential Provider and/or from the clear text in the  config when allowed.|[HttpFS Server Setup](../../hadoop-hdfs-httpfs/ServerSetup.html)|
 |AWS <br/> S3/S3A     |Uses Configuration.getPassword to get the S3 credentials. They may be resolved through the credential provider API or from the config for backward compatibility.|[AWS S3/S3A Usage](../../hadoop-aws/tools/hadoop-aws/index.html)|

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

@@ -572,7 +572,7 @@ or implicitly set to "simple":
 
 Needless to say, an application so configured cannot talk to a secure Hadoop cluster.
 
-#### `--verifyshortname &lt;principal>`: validate the short name of a principal
+#### `--verifyshortname <principal>`: validate the short name of a principal
 
 This verifies that the short name of a principal contains neither the `"@"`
 nor `"/"` characters.

+ 14 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFsShellList.java

@@ -75,4 +75,18 @@ public class TestFsShellList {
     lsArgv = new String[]{"-ls", "-q", testRootDir.toString()};
     assertThat(shell.run(lsArgv), is(0));
   }
+
+  /*
+  UGI params should take effect when we pass.
+ */
+  @Test(expected = IllegalArgumentException.class)
+  public void testListWithUGI() throws Exception {
+    FsShell fsShell = new FsShell(new Configuration());
+    //Passing Dummy such that it should through IAE
+    fsShell.getConf()
+        .set(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION,
+            "DUMMYAUTH");
+    String[] lsArgv = new String[] {"-ls", testRootDir.toString()};
+    fsShell.run(lsArgv);
+  }
 }

+ 60 - 1
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/TestIOUtils.java

@@ -21,9 +21,11 @@ package org.apache.hadoop.io;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.fail;
 
+import java.io.BufferedOutputStream;
 import java.io.ByteArrayInputStream;
 import java.io.EOFException;
 import java.io.File;
+import java.io.FileOutputStream;
 import java.io.FilenameFilter;
 import java.io.IOException;
 import java.io.InputStream;
@@ -31,22 +33,28 @@ import java.io.OutputStream;
 import java.io.RandomAccessFile;
 import java.nio.ByteBuffer;
 import java.nio.channels.FileChannel;
+import java.nio.charset.CharacterCodingException;
 import java.nio.file.Files;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
 
-import org.apache.commons.io.FileUtils;
+import org.apache.commons.io.FileUtils;;
+import org.apache.hadoop.fs.PathIOException;
 import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.hadoop.test.LambdaTestUtils;
 import org.junit.Assert;
 import org.junit.Test;
 import org.mockito.Mockito;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Test cases for IOUtils.java
  */
 public class TestIOUtils {
   private static final String TEST_FILE_NAME = "test_file";
+  private static final Logger LOG = LoggerFactory.getLogger(TestIOUtils.class);
   
   @Test
   public void testCopyBytesShouldCloseStreamsWhenCloseIsTrue() throws Exception {
@@ -289,4 +297,55 @@ public class TestIOUtils {
       FileUtils.deleteDirectory(dir);
     }
   }
+
+  @Test
+  public void testCloseStreams() throws IOException {
+    File tmpFile = null;
+    FileOutputStream fos;
+    BufferedOutputStream bos;
+    FileOutputStream nullStream = null;
+
+    try {
+      tmpFile = new File(GenericTestUtils.getTestDir(), "testCloseStreams.txt");
+      fos = new FileOutputStream(tmpFile) {
+        @Override
+        public void close() throws IOException {
+          throw new IOException();
+        }
+      };
+      bos = new BufferedOutputStream(
+          new FileOutputStream(tmpFile)) {
+        @Override
+        public void close() {
+          throw new NullPointerException();
+        }
+      };
+
+      IOUtils.closeStreams(fos, bos, nullStream);
+      IOUtils.closeStreams();
+    } finally {
+      FileUtils.deleteQuietly(tmpFile);
+    }
+
+  }
+
+  @Test
+  public void testWrapException() throws Exception {
+    // Test for IOException with valid (String) constructor
+    LambdaTestUtils.intercept(EOFException.class,
+        "Failed with java.io.EOFException while processing file/directory "
+            + ":[/tmp/abc.txt] in method:[testWrapException]", () -> {
+          throw IOUtils.wrapException("/tmp/abc.txt", "testWrapException",
+              new EOFException("EOFException "));
+        });
+
+    // Test for IOException with  no (String) constructor
+    PathIOException returnedEx = LambdaTestUtils
+        .intercept(PathIOException.class, "Input/output error:",
+            () -> {
+              throw IOUtils.wrapException("/tmp/abc.txt", "testWrapEx",
+                  new CharacterCodingException());
+            });
+    assertEquals("/tmp/abc.txt", returnedEx.getPath().toString());
+  }
 }

+ 29 - 3
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestDecayRpcScheduler.java

@@ -19,19 +19,22 @@
 package org.apache.hadoop.ipc;
 
 import static java.lang.Thread.sleep;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
 
 import org.junit.Test;
+
+import static org.junit.Assert.*;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
+import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.conf.Configuration;
 
 import javax.management.MBeanServer;
 import javax.management.ObjectName;
+
+import java.io.ByteArrayOutputStream;
+import java.io.PrintStream;
 import java.lang.management.ManagementFactory;
 
 public class TestDecayRpcScheduler {
@@ -248,4 +251,27 @@ public class TestDecayRpcScheduler {
       sleep(10);
     }
   }
+
+  @Test(timeout=60000)
+  public void testNPEatInitialization() throws InterruptedException {
+    // redirect the LOG to and check if there is NPE message while initializing
+    // the DecayRpcScheduler
+    PrintStream output = System.out;
+    try {
+      ByteArrayOutputStream bytes = new ByteArrayOutputStream();
+      System.setOut(new PrintStream(bytes));
+
+      // initializing DefaultMetricsSystem here would set "monitoring" flag in
+      // MetricsSystemImpl to true
+      DefaultMetricsSystem.initialize("NameNode");
+      Configuration conf = new Configuration();
+      scheduler = new DecayRpcScheduler(1, "ns", conf);
+      // check if there is npe in log
+      assertFalse(bytes.toString().contains("NullPointerException"));
+    } finally {
+      //set systout back
+      System.setOut(output);
+    }
+
+  }
 }

+ 81 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestSecurityUtil.java

@@ -21,10 +21,13 @@ import static org.apache.hadoop.security.UserGroupInformation.AuthenticationMeth
 
 import static org.junit.Assert.*;
 
+import java.io.File;
 import java.io.IOException;
 import java.net.InetAddress;
 import java.net.InetSocketAddress;
 import java.net.URI;
+import java.nio.charset.StandardCharsets;
+import java.util.List;
 
 import javax.security.auth.kerberos.KerberosPrincipal;
 
@@ -32,14 +35,23 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.alias.CredentialProvider;
+import org.apache.hadoop.security.alias.CredentialProviderFactory;
+import org.apache.hadoop.security.alias.LocalJavaKeyStoreProvider;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.TokenIdentifier;
 import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.util.ZKUtil.ZKAuthInfo;
 import org.junit.BeforeClass;
 import org.junit.Test;
 import org.mockito.Mockito;
 
+import com.google.common.io.Files;
+
 public class TestSecurityUtil {
+
+  private static final String ZK_AUTH_VALUE = "a_scheme:a_password";
+
   @BeforeClass
   public static void unsetKerberosRealm() {
     // prevent failures if kinit-ed or on os x with no realm
@@ -404,4 +416,73 @@ public class TestSecurityUtil {
     SecurityUtil.setAuthenticationMethod(KERBEROS, conf);
     assertEquals("kerberos", conf.get(HADOOP_SECURITY_AUTHENTICATION));
   }
+
+  @Test
+  public void testAuthPlainPasswordProperty() throws Exception {
+    Configuration conf = new Configuration();
+    conf.set(CommonConfigurationKeys.ZK_AUTH, ZK_AUTH_VALUE);
+    List<ZKAuthInfo> zkAuths = SecurityUtil.getZKAuthInfos(conf,
+        CommonConfigurationKeys.ZK_AUTH);
+    assertEquals(1, zkAuths.size());
+    ZKAuthInfo zkAuthInfo = zkAuths.get(0);
+    assertEquals("a_scheme", zkAuthInfo.getScheme());
+    assertArrayEquals("a_password".getBytes(), zkAuthInfo.getAuth());
+  }
+
+  @Test
+  public void testAuthPlainTextFile() throws Exception {
+    Configuration conf = new Configuration();
+    File passwordTxtFile = File.createTempFile(
+        getClass().getSimpleName() +  ".testAuthAtPathNotation-", ".txt");
+    Files.write(ZK_AUTH_VALUE, passwordTxtFile, StandardCharsets.UTF_8);
+    try {
+      conf.set(CommonConfigurationKeys.ZK_AUTH,
+          "@" + passwordTxtFile.getAbsolutePath());
+      List<ZKAuthInfo> zkAuths = SecurityUtil.getZKAuthInfos(conf,
+          CommonConfigurationKeys.ZK_AUTH);
+      assertEquals(1, zkAuths.size());
+      ZKAuthInfo zkAuthInfo = zkAuths.get(0);
+      assertEquals("a_scheme", zkAuthInfo.getScheme());
+      assertArrayEquals("a_password".getBytes(), zkAuthInfo.getAuth());
+    } finally {
+      boolean deleted = passwordTxtFile.delete();
+      assertTrue(deleted);
+    }
+  }
+
+  @Test
+  public void testAuthLocalJceks() throws Exception {
+    File localJceksFile = File.createTempFile(
+        getClass().getSimpleName() +".testAuthLocalJceks-", ".localjceks");
+    populateLocalJceksTestFile(localJceksFile.getAbsolutePath());
+    try {
+      String localJceksUri = "localjceks://file/" +
+          localJceksFile.getAbsolutePath();
+      Configuration conf = new Configuration();
+      conf.set(CredentialProviderFactory.CREDENTIAL_PROVIDER_PATH,
+          localJceksUri);
+      List<ZKAuthInfo> zkAuths = SecurityUtil.getZKAuthInfos(conf,
+          CommonConfigurationKeys.ZK_AUTH);
+      assertEquals(1, zkAuths.size());
+      ZKAuthInfo zkAuthInfo = zkAuths.get(0);
+      assertEquals("a_scheme", zkAuthInfo.getScheme());
+      assertArrayEquals("a_password".getBytes(), zkAuthInfo.getAuth());
+    } finally {
+      boolean deleted = localJceksFile.delete();
+      assertTrue(deleted);
+    }
+  }
+
+  private void populateLocalJceksTestFile(String path) throws IOException {
+    Configuration conf = new Configuration();
+    conf.set(CredentialProviderFactory.CREDENTIAL_PROVIDER_PATH,
+        "localjceks://file/" + path);
+    CredentialProvider provider =
+        CredentialProviderFactory.getProviders(conf).get(0);
+    assertEquals(LocalJavaKeyStoreProvider.class.getName(),
+        provider.getClass().getName());
+    provider.createCredentialEntry(CommonConfigurationKeys.ZK_AUTH,
+        ZK_AUTH_VALUE.toCharArray());
+    provider.flush();
+  }
 }

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

@@ -129,6 +129,18 @@ public class Crc32PerformanceTest {
       }
     }
 
+    final class ZipC extends AbstractCrc32<Checksum> {
+      @Override
+      public Checksum newAlgorithm() {
+        return DataChecksum.newCrc32C();
+      }
+
+      @Override
+      public DataChecksum.Type crcType() {
+        return DataChecksum.Type.CRC32C;
+      }
+    }
+
     final class PureJava extends AbstractCrc32<PureJavaCrc32> {
       @Override
       public PureJavaCrc32 newAlgorithm() {
@@ -169,6 +181,9 @@ public class Crc32PerformanceTest {
     this.direct = direct;
 
     crcs.add(Crc32.Zip.class);
+    if (Shell.isJavaVersionAtLeast(9)) {
+      crcs.add(Crc32.ZipC.class);
+    }
     crcs.add(Crc32.PureJava.class);
     crcs.add(Crc32.PureJavaC.class);
 
@@ -435,4 +450,4 @@ public class Crc32PerformanceTest {
       outCrc.printf("%" + max + "s = %s\n", n, p.getProperty(n));
     }
   }
-}
+}

+ 5 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestShell.java

@@ -523,4 +523,9 @@ public class TestShell extends Assert {
     shexc1.getProcess().waitFor();
     shexc2.getProcess().waitFor();
   }
+
+  @Test
+  public void testIsJavaVersionAtLeast() {
+    assertTrue(Shell.isJavaVersionAtLeast(8));
+  }
 }

+ 33 - 3
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java

@@ -3088,11 +3088,26 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
   @Deprecated
   public RemoteIterator<OpenFileEntry> listOpenFiles() throws IOException {
     checkOpen();
-    return listOpenFiles(EnumSet.of(OpenFilesType.ALL_OPEN_FILES));
+    return listOpenFiles(EnumSet.of(OpenFilesType.ALL_OPEN_FILES),
+        OpenFilesIterator.FILTER_PATH_DEFAULT);
   }
 
   /**
-   * Get a remote iterator to the open files list by type, managed by NameNode.
+   * Get a remote iterator to the open files list by path,
+   * managed by NameNode.
+   *
+   * @param path
+   * @throws IOException
+   */
+  public RemoteIterator<OpenFileEntry> listOpenFiles(String path)
+      throws IOException {
+    checkOpen();
+    return listOpenFiles(EnumSet.of(OpenFilesType.ALL_OPEN_FILES), path);
+  }
+
+  /**
+   * Get a remote iterator to the open files list by type,
+   * managed by NameNode.
    *
    * @param openFilesTypes
    * @throws IOException
@@ -3100,6 +3115,21 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
   public RemoteIterator<OpenFileEntry> listOpenFiles(
       EnumSet<OpenFilesType> openFilesTypes) throws IOException {
     checkOpen();
-    return new OpenFilesIterator(namenode, tracer, openFilesTypes);
+    return listOpenFiles(openFilesTypes,
+        OpenFilesIterator.FILTER_PATH_DEFAULT);
+  }
+
+  /**
+   * Get a remote iterator to the open files list by type and path,
+   * managed by NameNode.
+   *
+   * @param openFilesTypes
+   * @param path
+   * @throws IOException
+   */
+  public RemoteIterator<OpenFileEntry> listOpenFiles(
+      EnumSet<OpenFilesType> openFilesTypes, String path) throws IOException {
+    checkOpen();
+    return new OpenFilesIterator(namenode, tracer, openFilesTypes, path);
   }
 }

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

@@ -3086,8 +3086,8 @@ public class DistributedFileSystem extends FileSystem
   }
 
   public RemoteIterator<OpenFileEntry> listOpenFiles(
-      EnumSet<OpenFilesType> openFilesTypes) throws IOException {
-    return dfs.listOpenFiles(openFilesTypes);
+      EnumSet<OpenFilesType> openFilesTypes, String path) throws IOException {
+    return dfs.listOpenFiles(openFilesTypes, path);
   }
 
 

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

@@ -659,8 +659,8 @@ public class HdfsAdmin {
   }
 
   public RemoteIterator<OpenFileEntry> listOpenFiles(
-      EnumSet<OpenFilesType> openFilesTypes) throws IOException {
-    return dfs.listOpenFiles(openFilesTypes);
+      EnumSet<OpenFilesType> openFilesTypes, String path) throws IOException {
+    return dfs.listOpenFiles(openFilesTypes, path);
   }
 
 }

+ 24 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/AddErasureCodingPolicyResponse.java

@@ -17,6 +17,8 @@
  */
 package org.apache.hadoop.hdfs.protocol;
 
+import org.apache.commons.lang.builder.EqualsBuilder;
+import org.apache.commons.lang.builder.HashCodeBuilder;
 import org.apache.hadoop.HadoopIllegalArgumentException;
 
 /**
@@ -65,4 +67,26 @@ public class AddErasureCodingPolicyResponse {
           + "error message is " + getErrorMsg();
     }
   }
+
+  @Override
+  public boolean equals(Object o) {
+    if (o instanceof AddErasureCodingPolicyResponse) {
+      AddErasureCodingPolicyResponse other = (AddErasureCodingPolicyResponse) o;
+      return new EqualsBuilder()
+          .append(policy, other.policy)
+          .append(succeed, other.succeed)
+          .append(errorMsg, other.errorMsg)
+          .isEquals();
+    }
+    return false;
+  }
+
+  @Override
+  public int hashCode() {
+    return new HashCodeBuilder(303855623, 582626729)
+        .append(policy)
+        .append(succeed)
+        .append(errorMsg)
+        .toHashCode();
+  }
 }

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

@@ -1724,10 +1724,11 @@ public interface ClientProtocol {
    * the the list across batches are not atomic.
    *
    * @param prevId the cursor INode id.
-   * @param openFilesTypes types to filter the open files
+   * @param openFilesTypes types to filter the open files.
+   * @param path path to filter the open files.
    * @throws IOException
    */
   @Idempotent
   BatchedEntries<OpenFileEntry> listOpenFiles(long prevId,
-      EnumSet<OpenFilesType> openFilesTypes) throws IOException;
+      EnumSet<OpenFilesType> openFilesTypes, String path) throws IOException;
 }

+ 8 - 2
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/OpenFilesIterator.java

@@ -37,6 +37,9 @@ import org.apache.htrace.core.Tracer;
 public class OpenFilesIterator extends
     BatchedRemoteIterator<Long, OpenFileEntry> {
 
+  /** No path to be filtered by default. */
+  public static final String FILTER_PATH_DEFAULT = "/";
+
   /**
    * Open file types to filter the results.
    */
@@ -67,20 +70,23 @@ public class OpenFilesIterator extends
   private final ClientProtocol namenode;
   private final Tracer tracer;
   private final EnumSet<OpenFilesType> types;
+  /** List files filtered by given path. */
+  private String path;
 
   public OpenFilesIterator(ClientProtocol namenode, Tracer tracer,
-      EnumSet<OpenFilesType> types) {
+      EnumSet<OpenFilesType> types, String path) {
     super(HdfsConstants.GRANDFATHER_INODE_ID);
     this.namenode = namenode;
     this.tracer = tracer;
     this.types = types;
+    this.path = path;
   }
 
   @Override
   public BatchedEntries<OpenFileEntry> makeRequest(Long prevId)
       throws IOException {
     try (TraceScope ignored = tracer.newScope("listOpenFiles")) {
-      return namenode.listOpenFiles(prevId, types);
+      return namenode.listOpenFiles(prevId, types, path);
     }
   }
 

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

@@ -1,3 +1,4 @@
+
 /**
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file

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

@@ -78,6 +78,7 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.OpenFilesIterator.OpenFilesType;
 import org.apache.hadoop.hdfs.protocol.ReplicatedBlockStats;
 import org.apache.hadoop.hdfs.protocol.OpenFileEntry;
+import org.apache.hadoop.hdfs.protocol.OpenFilesIterator;
 import org.apache.hadoop.hdfs.protocol.ZoneReencryptionStatus;
 import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
@@ -1898,17 +1899,20 @@ public class ClientNamenodeProtocolTranslatorPB implements
   @Override
   public BatchedEntries<OpenFileEntry> listOpenFiles(long prevId)
       throws IOException {
-    return listOpenFiles(prevId, EnumSet.of(OpenFilesType.ALL_OPEN_FILES));
+    return listOpenFiles(prevId, EnumSet.of(OpenFilesType.ALL_OPEN_FILES),
+        OpenFilesIterator.FILTER_PATH_DEFAULT);
   }
 
   @Override
   public BatchedEntries<OpenFileEntry> listOpenFiles(long prevId,
-      EnumSet<OpenFilesType> openFilesTypes) throws IOException {
+      EnumSet<OpenFilesType> openFilesTypes, String path) throws IOException {
     ListOpenFilesRequestProto.Builder req =
         ListOpenFilesRequestProto.newBuilder().setId(prevId);
     if (openFilesTypes != null) {
       req.addAllTypes(PBHelperClient.convertOpenFileTypes(openFilesTypes));
     }
+    req.setPath(path);
+
     try {
       ListOpenFilesResponseProto response =
           rpcProxy.listOpenFiles(null, req.build());

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

@@ -804,6 +804,7 @@ enum OpenFilesTypeProto {
 message ListOpenFilesRequestProto {
   required int64 id = 1;
   repeated OpenFilesTypeProto types = 2;
+  optional string path = 3;
 }
 
 message OpenFilesBatchResponseProto {

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

@@ -382,6 +382,7 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
                   <include>FederationProtocol.proto</include>
                   <include>RouterProtocol.proto</include>
                   <include>AliasMapProtocol.proto</include>
+                  <include>InterQJournalProtocol.proto</include>
                 </includes>
               </source>
             </configuration>

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

@@ -650,6 +650,8 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final String  DFS_DATANODE_HTTP_ADDRESS_KEY = "dfs.datanode.http.address";
   public static final int     DFS_DATANODE_HTTP_DEFAULT_PORT = 9864;
   public static final String  DFS_DATANODE_HTTP_ADDRESS_DEFAULT = "0.0.0.0:" + DFS_DATANODE_HTTP_DEFAULT_PORT;
+  public static final String  DFS_DATANODE_HTTP_INTERNAL_PROXY_PORT =
+      "dfs.datanode.http.internal-proxy.port";
   public static final String  DFS_DATANODE_MAX_RECEIVER_THREADS_KEY =
       HdfsClientConfigKeys.DeprecatedKeys.DFS_DATANODE_MAX_RECEIVER_THREADS_KEY;
   public static final int     DFS_DATANODE_MAX_RECEIVER_THREADS_DEFAULT = 4096;
@@ -1281,6 +1283,13 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final long FEDERATION_STORE_MEMBERSHIP_EXPIRATION_MS_DEFAULT =
       TimeUnit.MINUTES.toMillis(5);
 
+  // HDFS Router-based federation mount table entries
+  /** Maximum number of cache entries to have. */
+  public static final String FEDERATION_MOUNT_TABLE_MAX_CACHE_SIZE =
+      DFSConfigKeys.FEDERATION_ROUTER_PREFIX + "mount-table.max-cache-size";
+  /** Remove cache entries if we have more than 10k. */
+  public static final int FEDERATION_MOUNT_TABLE_MAX_CACHE_SIZE_DEFAULT = 10000;
+
   // HDFS Router-based federation admin
   public static final String DFS_ROUTER_ADMIN_HANDLER_COUNT_KEY =
       FEDERATION_ROUTER_PREFIX + "admin.handler.count";
@@ -1315,6 +1324,15 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final String DFS_ROUTER_HTTPS_ADDRESS_DEFAULT =
       "0.0.0.0:" + DFS_ROUTER_HTTPS_PORT_DEFAULT;
 
+  // HDFS Router-based federation quota
+  public static final String DFS_ROUTER_QUOTA_ENABLE =
+      FEDERATION_ROUTER_PREFIX + "quota.enable";
+  public static final boolean DFS_ROUTER_QUOTA_ENABLED_DEFAULT = false;
+  public static final String DFS_ROUTER_QUOTA_CACHE_UPATE_INTERVAL =
+      FEDERATION_ROUTER_PREFIX + "quota-cache.update.interval";
+  public static final long DFS_ROUTER_QUOTA_CACHE_UPATE_INTERVAL_DEFAULT =
+      60000;
+
   // dfs.client.retry confs are moved to HdfsClientConfigKeys.Retry 
   @Deprecated
   public static final String  DFS_CLIENT_RETRY_POLICY_ENABLED_KEY

+ 4 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HDFSPolicyProvider.java

@@ -24,6 +24,7 @@ import org.apache.hadoop.ha.ZKFCProtocol;
 import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.ReconfigurationProtocol;
+import org.apache.hadoop.hdfs.qjournal.protocol.InterQJournalProtocol;
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocol;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeLifelineProtocol;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
@@ -57,6 +58,9 @@ public class HDFSPolicyProvider extends PolicyProvider {
         NamenodeProtocol.class),
     new Service(CommonConfigurationKeys.SECURITY_QJOURNAL_SERVICE_PROTOCOL_ACL,
         QJournalProtocol.class),
+        new Service(
+            CommonConfigurationKeys.SECURITY_INTERQJOURNAL_SERVICE_PROTOCOL_ACL,
+            InterQJournalProtocol.class),
     new Service(CommonConfigurationKeys.SECURITY_HA_SERVICE_PROTOCOL_ACL,
         HAServiceProtocol.class),
     new Service(CommonConfigurationKeys.SECURITY_ZKFC_PROTOCOL_ACL,

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

@@ -1856,7 +1856,7 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
       EnumSet<OpenFilesType> openFilesTypes =
           PBHelperClient.convertOpenFileTypes(req.getTypesList());
       BatchedEntries<OpenFileEntry> entries = server.listOpenFiles(req.getId(),
-          openFilesTypes);
+          openFilesTypes, req.getPath());
       ListOpenFilesResponseProto.Builder builder =
           ListOpenFilesResponseProto.newBuilder();
       builder.setHasMore(entries.hasMore());

+ 54 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocol/InterQJournalProtocol.java

@@ -0,0 +1,54 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdfs.qjournal.protocol;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.qjournal.server.JournalNode;
+import org.apache.hadoop.hdfs.qjournal.protocol.InterQJournalProtocolProtos.GetEditLogManifestFromJournalResponseProto;
+import org.apache.hadoop.security.KerberosInfo;
+
+import java.io.IOException;
+
+/**
+ * Protocol used to communicate between {@link JournalNode} for journalsync.
+ *
+ * This is responsible for sending edit log manifest.
+ */
+
+@KerberosInfo(
+    serverPrincipal = DFSConfigKeys.DFS_JOURNALNODE_KERBEROS_PRINCIPAL_KEY,
+    clientPrincipal = DFSConfigKeys.DFS_JOURNALNODE_KERBEROS_PRINCIPAL_KEY)
+@InterfaceAudience.Private
+public interface InterQJournalProtocol {
+
+  long versionID = 1L;
+
+  /**
+   * @param jid the journal from which to enumerate edits
+   * @param sinceTxId the first transaction which the client cares about
+   * @param inProgressOk whether or not to check the in-progress edit log
+   *        segment
+   * @return a list of edit log segments since the given transaction ID.
+   */
+  GetEditLogManifestFromJournalResponseProto getEditLogManifestFromJournal(
+      String jid, String nameServiceId, long sinceTxId, boolean inProgressOk)
+      throws IOException;
+
+}

+ 40 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocolPB/InterQJournalProtocolPB.java

@@ -0,0 +1,40 @@
+/**
+ * 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.qjournal.protocolPB;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.qjournal.protocol.InterQJournalProtocolProtos.InterQJournalProtocolService;
+import org.apache.hadoop.ipc.ProtocolInfo;
+import org.apache.hadoop.security.KerberosInfo;
+/**
+ * Protocol used to communicate between journal nodes for journal sync.
+ * Note: This extends the protocolbuffer service based interface to
+ * add annotations required for security.
+ */
+@KerberosInfo(
+    serverPrincipal = DFSConfigKeys.DFS_JOURNALNODE_KERBEROS_PRINCIPAL_KEY,
+    clientPrincipal = DFSConfigKeys.DFS_JOURNALNODE_KERBEROS_PRINCIPAL_KEY)
+@ProtocolInfo(protocolName =
+    "org.apache.hadoop.hdfs.qjournal.protocol.InterQJournalProtocol",
+    protocolVersion = 1)
+@InterfaceAudience.Private
+public interface InterQJournalProtocolPB extends
+    InterQJournalProtocolService.BlockingInterface {
+}

+ 64 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocolPB/InterQJournalProtocolServerSideTranslatorPB.java

@@ -0,0 +1,64 @@
+/**
+ * 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.qjournal.protocolPB;
+
+import com.google.protobuf.RpcController;
+import com.google.protobuf.ServiceException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hdfs.qjournal.protocol.InterQJournalProtocol;
+import org.apache.hadoop.hdfs.qjournal.protocol.InterQJournalProtocolProtos.GetEditLogManifestFromJournalRequestProto;
+import org.apache.hadoop.hdfs.qjournal.protocol.InterQJournalProtocolProtos.GetEditLogManifestFromJournalResponseProto;
+
+import java.io.IOException;
+
+/**
+ * Implementation for protobuf service that forwards requests
+ * received on {@link InterQJournalProtocolPB} to the
+ * {@link InterQJournalProtocol} server implementation.
+ */
+@InterfaceAudience.Private
+public class InterQJournalProtocolServerSideTranslatorPB implements
+    InterQJournalProtocolPB{
+
+  /* Server side implementation to delegate the requests to. */
+  private final InterQJournalProtocol impl;
+
+  public InterQJournalProtocolServerSideTranslatorPB(InterQJournalProtocol
+                                                         impl) {
+    this.impl = impl;
+  }
+
+  @Override
+  public GetEditLogManifestFromJournalResponseProto
+      getEditLogManifestFromJournal(RpcController controller,
+                                    GetEditLogManifestFromJournalRequestProto
+                                        request) throws ServiceException {
+    try {
+      return impl.getEditLogManifestFromJournal(
+          request.getJid().getIdentifier(),
+          request.hasNameServiceId() ? request.getNameServiceId() : null,
+          request.getSinceTxId(),
+          request.getInProgressOk());
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+}

+ 96 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocolPB/InterQJournalProtocolTranslatorPB.java

@@ -0,0 +1,96 @@
+/**
+ * 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.qjournal.protocolPB;
+
+import com.google.protobuf.RpcController;
+import com.google.protobuf.ServiceException;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hdfs.qjournal.protocol.InterQJournalProtocol;
+import org.apache.hadoop.hdfs.qjournal.protocol.InterQJournalProtocolProtos.GetEditLogManifestFromJournalResponseProto;
+import org.apache.hadoop.hdfs.qjournal.protocol.InterQJournalProtocolProtos.GetEditLogManifestFromJournalRequestProto;
+import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos;
+import org.apache.hadoop.ipc.ProtobufHelper;
+import org.apache.hadoop.ipc.ProtocolMetaInterface;
+import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.ipc.RpcClientUtil;
+
+import java.io.Closeable;
+import java.io.IOException;
+
+/**
+ * This class is the client side translator to translate the requests made on
+ * {@link InterQJournalProtocol} interfaces to the RPC server implementing
+ * {@link InterQJournalProtocolPB}.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Stable
+public class InterQJournalProtocolTranslatorPB implements ProtocolMetaInterface,
+    InterQJournalProtocol, Closeable {
+
+  /* RpcController is not used and hence is set to null. */
+  private final static RpcController NULL_CONTROLLER = null;
+  private final InterQJournalProtocolPB rpcProxy;
+
+  public InterQJournalProtocolTranslatorPB(InterQJournalProtocolPB rpcProxy) {
+    this.rpcProxy = rpcProxy;
+  }
+
+  @Override
+  public void close() {
+    RPC.stopProxy(rpcProxy);
+  }
+
+
+  @Override
+  public GetEditLogManifestFromJournalResponseProto
+      getEditLogManifestFromJournal(String jid, String nameServiceId,
+                                    long sinceTxId, boolean inProgressOk)
+      throws IOException {
+    try {
+      GetEditLogManifestFromJournalRequestProto.Builder req;
+      req = GetEditLogManifestFromJournalRequestProto.newBuilder()
+          .setJid(convertJournalId(jid))
+          .setSinceTxId(sinceTxId)
+          .setInProgressOk(inProgressOk);
+      if (nameServiceId !=null) {
+        req.setNameServiceId(nameServiceId);
+      }
+      return rpcProxy.getEditLogManifestFromJournal(NULL_CONTROLLER,
+          req.build()
+      );
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  private QJournalProtocolProtos.JournalIdProto convertJournalId(String jid) {
+    return QJournalProtocolProtos.JournalIdProto.newBuilder()
+        .setIdentifier(jid)
+        .build();
+  }
+
+  @Override
+  public boolean isMethodSupported(String methodName) throws IOException {
+    return RpcClientUtil.isMethodSupported(rpcProxy,
+        InterQJournalProtocolPB.class, RPC.RpcKind.RPC_PROTOCOL_BUFFER,
+        RPC.getProtocolVersion(InterQJournalProtocolPB.class), methodName);
+  }
+}

+ 37 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JournalNodeRpcServer.java

@@ -26,8 +26,12 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.HDFSPolicyProvider;
 import org.apache.hadoop.hdfs.protocolPB.PBHelper;
+import org.apache.hadoop.hdfs.qjournal.protocol.InterQJournalProtocol;
+import org.apache.hadoop.hdfs.qjournal.protocol.InterQJournalProtocolProtos.InterQJournalProtocolService;
+import org.apache.hadoop.hdfs.qjournal.protocol.InterQJournalProtocolProtos.GetEditLogManifestFromJournalResponseProto;
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocol;
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.GetEditLogManifestResponseProto;
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.GetJournalStateResponseProto;
@@ -36,6 +40,8 @@ import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.PrepareRe
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.QJournalProtocolService;
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.SegmentStateProto;
 import org.apache.hadoop.hdfs.qjournal.protocol.RequestInfo;
+import org.apache.hadoop.hdfs.qjournal.protocolPB.InterQJournalProtocolPB;
+import org.apache.hadoop.hdfs.qjournal.protocolPB.InterQJournalProtocolServerSideTranslatorPB;
 import org.apache.hadoop.hdfs.qjournal.protocolPB.QJournalProtocolPB;
 import org.apache.hadoop.hdfs.qjournal.protocolPB.QJournalProtocolServerSideTranslatorPB;
 import org.apache.hadoop.hdfs.server.common.StorageInfo;
@@ -52,7 +58,8 @@ import com.google.protobuf.BlockingService;
 
 @InterfaceAudience.Private
 @VisibleForTesting
-public class JournalNodeRpcServer implements QJournalProtocol {
+public class JournalNodeRpcServer implements QJournalProtocol,
+    InterQJournalProtocol {
   private static final int HANDLER_COUNT = 5;
   private final JournalNode jn;
   private Server server;
@@ -84,6 +91,19 @@ public class JournalNodeRpcServer implements QJournalProtocol {
       .setVerbose(false)
       .build();
 
+
+    //Adding InterQJournalProtocolPB to server
+    InterQJournalProtocolServerSideTranslatorPB
+        qJournalProtocolServerSideTranslatorPB = new
+        InterQJournalProtocolServerSideTranslatorPB(this);
+
+    BlockingService interQJournalProtocolService = InterQJournalProtocolService
+        .newReflectiveBlockingService(qJournalProtocolServerSideTranslatorPB);
+
+    DFSUtil.addPBProtocol(confCopy, InterQJournalProtocolPB.class,
+        interQJournalProtocolService, server);
+
+
     // set service-level authorization security policy
     if (confCopy.getBoolean(
       CommonConfigurationKeys.HADOOP_SECURITY_AUTHORIZATION, false)) {
@@ -263,4 +283,20 @@ public class JournalNodeRpcServer implements QJournalProtocol {
                               String nameServiceId) throws IOException {
     return jn.getJournalCTime(journalId, nameServiceId);
   }
+
+  @SuppressWarnings("deprecation")
+  @Override
+  public GetEditLogManifestFromJournalResponseProto
+      getEditLogManifestFromJournal(String jid, String nameServiceId,
+                                    long sinceTxId, boolean inProgressOk)
+      throws IOException {
+    RemoteEditLogManifest manifest = jn.getOrCreateJournal(jid, nameServiceId)
+        .getEditLogManifest(sinceTxId, inProgressOk);
+
+    return GetEditLogManifestFromJournalResponseProto.newBuilder()
+        .setManifest(PBHelper.convert(manifest))
+        .setHttpPort(jn.getBoundHttpAddress().getPort())
+        .setFromURL(jn.getHttpServerURI())
+        .build();
+  }
 }

+ 29 - 28
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JournalNodeSyncer.java

@@ -20,7 +20,6 @@ package org.apache.hadoop.hdfs.qjournal.server;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Sets;
-import com.google.protobuf.ServiceException;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileUtil;
@@ -28,19 +27,17 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
 
 import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.apache.hadoop.hdfs.protocolPB.PBHelper;
-import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos;
-import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos
-  .JournalIdProto;
-import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos
-  .GetEditLogManifestRequestProto;
-import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos
-  .GetEditLogManifestResponseProto;
-import org.apache.hadoop.hdfs.qjournal.protocolPB.QJournalProtocolPB;
+import org.apache.hadoop.hdfs.qjournal.protocol.InterQJournalProtocol;
+import org.apache.hadoop.hdfs.qjournal.protocol.InterQJournalProtocolProtos.GetEditLogManifestFromJournalResponseProto;
+import org.apache.hadoop.hdfs.qjournal.protocolPB.InterQJournalProtocolPB;
+import org.apache.hadoop.hdfs.qjournal.protocolPB.InterQJournalProtocolTranslatorPB;
 import org.apache.hadoop.hdfs.server.common.Util;
 import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
 import org.apache.hadoop.hdfs.server.protocol.RemoteEditLog;
 import org.apache.hadoop.hdfs.util.DataTransferThrottler;
+import org.apache.hadoop.ipc.ProtobufRpcEngine;
 import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.util.Daemon;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -52,6 +49,7 @@ import java.net.MalformedURLException;
 import java.net.URI;
 import java.net.URISyntaxException;
 import java.net.URL;
+import java.security.PrivilegedExceptionAction;
 import java.util.Collection;
 import java.util.HashSet;
 import java.util.List;
@@ -69,7 +67,6 @@ public class JournalNodeSyncer {
   private final Journal journal;
   private final String jid;
   private  String nameServiceId;
-  private final JournalIdProto jidProto;
   private final JNStorage jnStorage;
   private final Configuration conf;
   private volatile Daemon syncJournalDaemon;
@@ -90,7 +87,6 @@ public class JournalNodeSyncer {
     this.journal = journal;
     this.jid = jid;
     this.nameServiceId = nameServiceId;
-    this.jidProto = convertJournalId(this.jid);
     this.jnStorage = journal.getStorage();
     this.conf = conf;
     journalSyncInterval = conf.getLong(
@@ -235,7 +231,7 @@ public class JournalNodeSyncer {
     LOG.info("Syncing Journal " + jn.getBoundIpcAddress().getAddress() + ":"
         + jn.getBoundIpcAddress().getPort() + " with "
         + otherJNProxies.get(index) + ", journal id: " + jid);
-    final QJournalProtocolPB jnProxy = otherJNProxies.get(index).jnProxy;
+    final InterQJournalProtocol jnProxy = otherJNProxies.get(index).jnProxy;
     if (jnProxy == null) {
       LOG.error("JournalNode Proxy not found.");
       return;
@@ -249,13 +245,11 @@ public class JournalNodeSyncer {
       return;
     }
 
-    GetEditLogManifestResponseProto editLogManifest;
+    GetEditLogManifestFromJournalResponseProto editLogManifest;
     try {
-      editLogManifest = jnProxy.getEditLogManifest(null,
-          GetEditLogManifestRequestProto.newBuilder().setJid(jidProto)
-              .setSinceTxId(0)
-              .setInProgressOk(false).build());
-    } catch (ServiceException e) {
+      editLogManifest = jnProxy.getEditLogManifestFromJournal(jid,
+          nameServiceId, 0, false);
+    } catch (IOException e) {
       LOG.error("Could not sync with Journal at " +
           otherJNProxies.get(journalNodeIndexForSync), e);
       return;
@@ -323,14 +317,8 @@ public class JournalNodeSyncer {
         Sets.newHashSet(jn.getBoundIpcAddress()));
   }
 
-  private JournalIdProto convertJournalId(String journalId) {
-    return QJournalProtocolProtos.JournalIdProto.newBuilder()
-      .setIdentifier(journalId)
-      .build();
-  }
-
   private void getMissingLogSegments(List<RemoteEditLog> thisJournalEditLogs,
-      GetEditLogManifestResponseProto response,
+      GetEditLogManifestFromJournalResponseProto response,
       JournalNodeProxy remoteJNproxy) {
 
     List<RemoteEditLog> otherJournalEditLogs = PBHelper.convert(
@@ -497,13 +485,26 @@ public class JournalNodeSyncer {
 
   private class JournalNodeProxy {
     private final InetSocketAddress jnAddr;
-    private final QJournalProtocolPB jnProxy;
+    private final InterQJournalProtocol jnProxy;
     private URL httpServerUrl;
 
     JournalNodeProxy(InetSocketAddress jnAddr) throws IOException {
+      final Configuration confCopy = new Configuration(conf);
       this.jnAddr = jnAddr;
-      this.jnProxy = RPC.getProxy(QJournalProtocolPB.class,
-          RPC.getProtocolVersion(QJournalProtocolPB.class), jnAddr, conf);
+      this.jnProxy = SecurityUtil.doAsLoginUser(
+          new PrivilegedExceptionAction<InterQJournalProtocol>() {
+            @Override
+            public InterQJournalProtocol run() throws IOException {
+              RPC.setProtocolEngine(confCopy, InterQJournalProtocolPB.class,
+                  ProtobufRpcEngine.class);
+              InterQJournalProtocolPB interQJournalProtocolPB = RPC.getProxy(
+                  InterQJournalProtocolPB.class,
+                  RPC.getProtocolVersion(InterQJournalProtocolPB.class),
+                  jnAddr, confCopy);
+              return new InterQJournalProtocolTranslatorPB(
+                  interQJournalProtocolPB);
+            }
+          });
     }
 
     @Override

+ 19 - 5
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java

@@ -148,11 +148,25 @@ class BPOfferService {
     }
     Set<InetSocketAddress> newAddrs = Sets.newHashSet(addrs);
     
-    if (!Sets.symmetricDifference(oldAddrs, newAddrs).isEmpty()) {
-      // Keep things simple for now -- we can implement this at a later date.
-      throw new IOException(
-          "HA does not currently support adding a new standby to a running DN. " +
-          "Please do a rolling restart of DNs to reconfigure the list of NNs.");
+    // Process added NNs
+    Set<InetSocketAddress> addedNNs = Sets.difference(newAddrs, oldAddrs);
+    for (InetSocketAddress addedNN : addedNNs) {
+      BPServiceActor actor = new BPServiceActor(addedNN,
+          lifelineAddrs.get(addrs.indexOf(addedNN)), this);
+      actor.start();
+      bpServices.add(actor);
+    }
+
+    // Process removed NNs
+    Set<InetSocketAddress> removedNNs = Sets.difference(oldAddrs, newAddrs);
+    for (InetSocketAddress removedNN : removedNNs) {
+      for (BPServiceActor actor : bpServices) {
+        if (actor.getNNSocketAddress().equals(removedNN)) {
+          actor.stop();
+          shutdownActor(actor);
+          break;
+        }
+      }
     }
   }
 

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

@@ -114,7 +114,7 @@ public class ShortCircuitRegistry {
 
   public synchronized void removeShm(ShortCircuitShm shm) {
     if (LOG.isTraceEnabled()) {
-      LOG.debug("removing shm " + shm);
+      LOG.trace("removing shm " + shm);
     }
     // Stop tracking the shmId.
     RegisteredShm removedShm = segments.remove(shm.getShmId());

+ 3 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/StripedBlockReader.java

@@ -105,6 +105,7 @@ class StripedBlockReader {
     if (offsetInBlock >= block.getNumBytes()) {
       return null;
     }
+    Peer peer = null;
     try {
       InetSocketAddress dnAddr =
           stripedReader.getSocketAddress4Transfer(source);
@@ -120,7 +121,7 @@ class StripedBlockReader {
          *
          * TODO: add proper tracer
          */
-      Peer peer = newConnectedPeer(block, dnAddr, blockToken, source);
+      peer = newConnectedPeer(block, dnAddr, blockToken, source);
       if (peer.isLocal()) {
         this.isLocal = true;
       }
@@ -131,6 +132,7 @@ class StripedBlockReader {
     } catch (IOException e) {
       LOG.info("Exception while creating remote block reader, datanode {}",
           source, e);
+      IOUtils.closeStream(peer);
       return null;
     }
   }

+ 15 - 8
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/BlockPoolSlice.java

@@ -84,6 +84,7 @@ class BlockPoolSlice {
   private final int ioFileBufferSize;
   @VisibleForTesting
   public static final String DU_CACHE_FILE = "dfsUsed";
+  private final Runnable shutdownHook;
   private volatile boolean dfsUsedSaved = false;
   private static final int SHUTDOWN_HOOK_PRIORITY = 30;
   private final boolean deleteDuplicateReplicas;
@@ -162,15 +163,16 @@ class BlockPoolSlice {
                                                      .build();
 
     // Make the dfs usage to be saved during shutdown.
-    ShutdownHookManager.get().addShutdownHook(
-      new Runnable() {
-        @Override
-        public void run() {
-          if (!dfsUsedSaved) {
-            saveDfsUsed();
-          }
+    shutdownHook = new Runnable() {
+      @Override
+      public void run() {
+        if (!dfsUsedSaved) {
+          saveDfsUsed();
         }
-      }, SHUTDOWN_HOOK_PRIORITY);
+      }
+    };
+    ShutdownHookManager.get().addShutdownHook(shutdownHook,
+        SHUTDOWN_HOOK_PRIORITY);
   }
 
   File getDirectory() {
@@ -756,6 +758,11 @@ class BlockPoolSlice {
     saveDfsUsed();
     dfsUsedSaved = true;
 
+    // Remove the shutdown hook to avoid any memory leak
+    if (shutdownHook != null) {
+      ShutdownHookManager.get().removeShutdownHook(shutdownHook);
+    }
+
     if (dfsUsage instanceof CachingGetSpaceUsed) {
       IOUtils.cleanup(LOG, ((CachingGetSpaceUsed) dfsUsage));
     }

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java

@@ -1785,11 +1785,11 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
         case FINALIZED:
         case RBW:
         case RWR:
-          builders.get(b.getVolume().getStorageID()).add(b);
+          builders.get(volStorageID).add(b);
           break;
         case RUR:
           ReplicaInfo orig = b.getOriginalReplica();
-          builders.get(b.getVolume().getStorageID()).add(orig);
+          builders.get(volStorageID).add(orig);
           break;
         case TEMPORARY:
           break;

+ 4 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/web/DatanodeHttpServer.java

@@ -74,6 +74,7 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_ADMIN;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_HTTPS_ADDRESS_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_HTTPS_ADDRESS_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_HTTP_ADDRESS_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_HTTP_INTERNAL_PROXY_PORT;
 
 public class DatanodeHttpServer implements Closeable {
   private final HttpServer2 infoServer;
@@ -107,12 +108,14 @@ public class DatanodeHttpServer implements Closeable {
 
     Configuration confForInfoServer = new Configuration(conf);
     confForInfoServer.setInt(HttpServer2.HTTP_MAX_THREADS_KEY, 10);
+    int proxyPort =
+        confForInfoServer.getInt(DFS_DATANODE_HTTP_INTERNAL_PROXY_PORT, 0);
     HttpServer2.Builder builder = new HttpServer2.Builder()
         .setName("datanode")
         .setConf(confForInfoServer)
         .setACL(new AccessControlList(conf.get(DFS_ADMIN, " ")))
         .hostName(getHostnameForSpnegoPrincipal(confForInfoServer))
-        .addEndpoint(URI.create("http://localhost:0"))
+        .addEndpoint(URI.create("http://localhost:" + proxyPort))
         .setFindPort(true);
 
     final boolean xFrameEnabled = conf.getBoolean(

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

@@ -18,10 +18,13 @@
 package org.apache.hadoop.hdfs.server.federation.resolver;
 
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_ROUTER_DEFAULT_NAMESERVICE;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.FEDERATION_MOUNT_TABLE_MAX_CACHE_SIZE;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.FEDERATION_MOUNT_TABLE_MAX_CACHE_SIZE_DEFAULT;
 
 import java.io.IOException;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.Iterator;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
@@ -30,9 +33,10 @@ import java.util.Set;
 import java.util.SortedMap;
 import java.util.TreeMap;
 import java.util.TreeSet;
+import java.util.concurrent.Callable;
 import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentNavigableMap;
-import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutionException;
 import java.util.concurrent.locks.Lock;
 import java.util.concurrent.locks.ReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
@@ -55,6 +59,8 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import com.google.common.annotations.VisibleForTesting;
+import com.google.common.cache.Cache;
+import com.google.common.cache.CacheBuilder;
 
 /**
  * Mount table to map between global paths and remote locations. This allows the
@@ -81,8 +87,7 @@ public class MountTableResolver
   /** Path -> Remote HDFS location. */
   private final TreeMap<String, MountTable> tree = new TreeMap<>();
   /** Path -> Remote location. */
-  private final ConcurrentNavigableMap<String, PathLocation> locationCache =
-      new ConcurrentSkipListMap<>();
+  private final Cache<String, PathLocation> locationCache;
 
   /** Default nameservice when no mount matches the math. */
   private String defaultNameService = "";
@@ -99,20 +104,30 @@ public class MountTableResolver
   }
 
   public MountTableResolver(Configuration conf, Router routerService) {
+    this(conf, routerService, null);
+  }
+
+  public MountTableResolver(Configuration conf, StateStoreService store) {
+    this(conf, null, store);
+  }
+
+  public MountTableResolver(Configuration conf, Router routerService,
+      StateStoreService store) {
     this.router = routerService;
-    if (this.router != null) {
+    if (store != null) {
+      this.stateStore = store;
+    } else 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;
+    int maxCacheSize = conf.getInt(
+        FEDERATION_MOUNT_TABLE_MAX_CACHE_SIZE,
+        FEDERATION_MOUNT_TABLE_MAX_CACHE_SIZE_DEFAULT);
+    this.locationCache = CacheBuilder.newBuilder()
+        .maximumSize(maxCacheSize)
+        .build();
 
     registerCacheExternal();
     initDefaultNameService(conf);
@@ -210,16 +225,26 @@ public class MountTableResolver
    * @param path Source path.
    */
   private void invalidateLocationCache(final String path) {
-    if (locationCache.isEmpty()) {
+    LOG.debug("Invalidating {} from {}", path, locationCache);
+    if (locationCache.size() == 0) {
       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);
+
+    // Go through the entries and remove the ones from the path to invalidate
+    ConcurrentMap<String, PathLocation> map = locationCache.asMap();
+    Set<Entry<String, PathLocation>> entries = map.entrySet();
+    Iterator<Entry<String, PathLocation>> it = entries.iterator();
+    while (it.hasNext()) {
+      Entry<String, PathLocation> entry = it.next();
+      PathLocation loc = entry.getValue();
+      String src = loc.getSourcePath();
+      if (src.startsWith(path)) {
+        LOG.debug("Removing {}", src);
+        it.remove();
+      }
     }
+
+    LOG.debug("Location cache after invalidation: {}", locationCache);
   }
 
   /**
@@ -267,7 +292,9 @@ public class MountTableResolver
           // Node exists, check for updates
           MountTable existingEntry = this.tree.get(srcPath);
           if (existingEntry != null && !existingEntry.equals(entry)) {
-            // Entry has changed
+            LOG.info("Entry has changed from \"{}\" to \"{}\"",
+                existingEntry, entry);
+            this.tree.put(srcPath, entry);
             invalidateLocationCache(srcPath);
             LOG.info("Updated mount point {} in resolver");
           }
@@ -310,7 +337,7 @@ public class MountTableResolver
     LOG.info("Clearing all mount location caches");
     writeLock.lock();
     try {
-      this.locationCache.clear();
+      this.locationCache.invalidateAll();
       this.tree.clear();
     } finally {
       writeLock.unlock();
@@ -323,8 +350,15 @@ public class MountTableResolver
     verifyMountTable();
     readLock.lock();
     try {
-      return this.locationCache.computeIfAbsent(
-          path, this::lookupLocation);
+      Callable<? extends PathLocation> meh = new Callable<PathLocation>() {
+        @Override
+        public PathLocation call() throws Exception {
+          return lookupLocation(path);
+        }
+      };
+      return this.locationCache.get(path, meh);
+    } catch (ExecutionException e) {
+      throw new IOException(e);
     } finally {
       readLock.unlock();
     }
@@ -542,4 +576,12 @@ public class MountTableResolver
     }
     return ret;
   }
+
+  /**
+   * Get the size of the cache.
+   * @return Size of the cache.
+   */
+  protected long getCacheSize() {
+    return this.locationCache.size();
+  }
 }

+ 198 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/ErasureCoding.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 static org.apache.hadoop.hdfs.server.federation.router.RouterRpcServer.merge;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.hadoop.hdfs.protocol.AddErasureCodingPolicyResponse;
+import org.apache.hadoop.hdfs.protocol.ECBlockGroupStats;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicyInfo;
+import org.apache.hadoop.hdfs.server.federation.resolver.ActiveNamenodeResolver;
+import org.apache.hadoop.hdfs.server.federation.resolver.FederationNamespaceInfo;
+import org.apache.hadoop.hdfs.server.federation.resolver.RemoteLocation;
+import org.apache.hadoop.hdfs.server.namenode.NameNode.OperationCategory;
+
+/**
+ * Module that implements all the RPC calls in
+ * {@link org.apache.hadoop.hdfs.protocol.ClientProtocol} related to
+ * Erasure Coding in the {@link RouterRpcServer}.
+ */
+public class ErasureCoding {
+
+  /** RPC server to receive client calls. */
+  private final RouterRpcServer rpcServer;
+  /** RPC clients to connect to the Namenodes. */
+  private final RouterRpcClient rpcClient;
+  /** Interface to identify the active NN for a nameservice or blockpool ID. */
+  private final ActiveNamenodeResolver namenodeResolver;
+
+
+  public ErasureCoding(RouterRpcServer server) {
+    this.rpcServer = server;
+    this.rpcClient =  this.rpcServer.getRPCClient();
+    this.namenodeResolver = this.rpcClient.getNamenodeResolver();
+  }
+
+  public ErasureCodingPolicyInfo[] getErasureCodingPolicies()
+      throws IOException {
+    rpcServer.checkOperation(OperationCategory.READ);
+
+    RemoteMethod method = new RemoteMethod("getErasureCodingPolicies");
+    Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
+    Map<FederationNamespaceInfo, ErasureCodingPolicyInfo[]> ret =
+        rpcClient.invokeConcurrent(
+            nss, method, true, false, ErasureCodingPolicyInfo[].class);
+    return merge(ret, ErasureCodingPolicyInfo.class);
+  }
+
+  public Map<String, String> getErasureCodingCodecs() throws IOException {
+    rpcServer.checkOperation(OperationCategory.READ);
+
+    RemoteMethod method = new RemoteMethod("getErasureCodingCodecs");
+    Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
+    @SuppressWarnings("rawtypes")
+    Map<FederationNamespaceInfo, Map> retCodecs =
+        rpcClient.invokeConcurrent(
+            nss, method, true, false, Map.class);
+
+    Map<String, String> ret = new HashMap<>();
+    Object obj = retCodecs;
+    @SuppressWarnings("unchecked")
+    Map<FederationNamespaceInfo, Map<String, String>> results =
+        (Map<FederationNamespaceInfo, Map<String, String>>)obj;
+    Collection<Map<String, String>> allCodecs = results.values();
+    for (Map<String, String> codecs : allCodecs) {
+      ret.putAll(codecs);
+    }
+
+    return ret;
+  }
+
+  public AddErasureCodingPolicyResponse[] addErasureCodingPolicies(
+      ErasureCodingPolicy[] policies) throws IOException {
+    rpcServer.checkOperation(OperationCategory.WRITE);
+
+    RemoteMethod method = new RemoteMethod("addErasureCodingPolicies",
+        new Class<?>[] {ErasureCodingPolicy[].class}, new Object[] {policies});
+    Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
+    Map<FederationNamespaceInfo, AddErasureCodingPolicyResponse[]> ret =
+        rpcClient.invokeConcurrent(
+            nss, method, true, false, AddErasureCodingPolicyResponse[].class);
+
+    return merge(ret, AddErasureCodingPolicyResponse.class);
+  }
+
+  public void removeErasureCodingPolicy(String ecPolicyName)
+      throws IOException {
+    rpcServer.checkOperation(OperationCategory.WRITE);
+
+    RemoteMethod method = new RemoteMethod("removeErasureCodingPolicy",
+        new Class<?>[] {String.class}, ecPolicyName);
+    Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
+    rpcClient.invokeConcurrent(nss, method, true, false);
+  }
+
+  public void disableErasureCodingPolicy(String ecPolicyName)
+      throws IOException {
+    rpcServer.checkOperation(OperationCategory.WRITE);
+
+    RemoteMethod method = new RemoteMethod("disableErasureCodingPolicy",
+        new Class<?>[] {String.class}, ecPolicyName);
+    Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
+    rpcClient.invokeConcurrent(nss, method, true, false);
+  }
+
+  public void enableErasureCodingPolicy(String ecPolicyName)
+      throws IOException {
+    rpcServer.checkOperation(OperationCategory.WRITE);
+
+    RemoteMethod method = new RemoteMethod("enableErasureCodingPolicy",
+        new Class<?>[] {String.class}, ecPolicyName);
+    Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
+    rpcClient.invokeConcurrent(nss, method, true, false);
+  }
+
+  public ErasureCodingPolicy getErasureCodingPolicy(String src)
+      throws IOException {
+    rpcServer.checkOperation(OperationCategory.READ);
+
+    final List<RemoteLocation> locations =
+        rpcServer.getLocationsForPath(src, true);
+    RemoteMethod remoteMethod = new RemoteMethod("getErasureCodingPolicy",
+        new Class<?>[] {String.class}, new RemoteParam());
+    ErasureCodingPolicy ret = rpcClient.invokeSequential(
+        locations, remoteMethod, null, null);
+    return ret;
+  }
+
+  public void setErasureCodingPolicy(String src, String ecPolicyName)
+      throws IOException {
+    rpcServer.checkOperation(OperationCategory.WRITE);
+
+    final List<RemoteLocation> locations =
+        rpcServer.getLocationsForPath(src, true);
+    RemoteMethod remoteMethod = new RemoteMethod("setErasureCodingPolicy",
+        new Class<?>[] {String.class, String.class},
+        new RemoteParam(), ecPolicyName);
+    rpcClient.invokeSequential(locations, remoteMethod, null, null);
+  }
+
+  public void unsetErasureCodingPolicy(String src) throws IOException {
+    rpcServer.checkOperation(OperationCategory.WRITE);
+
+    final List<RemoteLocation> locations =
+        rpcServer.getLocationsForPath(src, true);
+    RemoteMethod remoteMethod = new RemoteMethod("unsetErasureCodingPolicy",
+        new Class<?>[] {String.class}, new RemoteParam());
+    rpcClient.invokeSequential(locations, remoteMethod, null, null);
+  }
+
+  public ECBlockGroupStats getECBlockGroupStats() throws IOException {
+    rpcServer.checkOperation(OperationCategory.READ);
+
+    RemoteMethod method = new RemoteMethod("getECBlockGroupStats");
+    Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
+    Map<FederationNamespaceInfo, ECBlockGroupStats> allStats =
+        rpcClient.invokeConcurrent(
+            nss, method, true, false, ECBlockGroupStats.class);
+
+    // Merge the stats from all the namespaces
+    long lowRedundancyBlockGroups = 0;
+    long corruptBlockGroups = 0;
+    long missingBlockGroups = 0;
+    long bytesInFutureBlockGroups = 0;
+    long pendingDeletionBlocks = 0;
+    for (ECBlockGroupStats stats : allStats.values()) {
+      lowRedundancyBlockGroups += stats.getLowRedundancyBlockGroups();
+      corruptBlockGroups += stats.getCorruptBlockGroups();
+      missingBlockGroups += stats.getMissingBlockGroups();
+      bytesInFutureBlockGroups += stats.getBytesInFutureBlockGroups();
+      pendingDeletionBlocks += stats.getPendingDeletionBlocks();
+    }
+    return new ECBlockGroupStats(lowRedundancyBlockGroups, corruptBlockGroups,
+        missingBlockGroups, bytesInFutureBlockGroups, pendingDeletionBlocks);
+  }
+}

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

@@ -30,6 +30,7 @@ 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.apache.hadoop.util.VersionInfo;
 import org.codehaus.jettison.json.JSONArray;
 import org.codehaus.jettison.json.JSONException;
 import org.codehaus.jettison.json.JSONObject;
@@ -105,6 +106,25 @@ public final class FederationUtil {
     return ret;
   }
 
+  /**
+   * Fetch the compile timestamp for this jar.
+   *
+   * @return Date compiled.
+   */
+  public static String getBuildVersion() {
+    return VersionInfo.getBuildVersion();
+  }
+
+  /**
+   * Fetch the build/compile information for this jar.
+   *
+   * @return String Compilation info.
+   */
+  public static String getCompileInfo() {
+    return VersionInfo.getDate() + " by " + VersionInfo.getUser() + " from "
+        + VersionInfo.getBranch();
+  }
+
   /**
    * Create an instance of an interface with a constructor using a context.
    *

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

@@ -0,0 +1,208 @@
+/**
+ * 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.Set;
+
+import org.apache.hadoop.fs.QuotaUsage;
+import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.hdfs.protocol.ClientProtocol;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.server.federation.resolver.RemoteLocation;
+import org.apache.hadoop.hdfs.server.namenode.NameNode.OperationCategory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Module that implements the quota relevant RPC calls
+ * {@link ClientProtocol#setQuota(String, long, long, StorageType)}
+ * and
+ * {@link ClientProtocol#getQuotaUsage(String)}
+ * in the {@link RouterRpcServer}.
+ */
+public class Quota {
+  private static final Logger LOG = LoggerFactory.getLogger(Quota.class);
+
+  /** RPC server to receive client calls. */
+  private final RouterRpcServer rpcServer;
+  /** RPC clients to connect to the Namenodes. */
+  private final RouterRpcClient rpcClient;
+  /** Router used in RouterRpcServer. */
+  private final Router router;
+
+  public Quota(Router router, RouterRpcServer server) {
+    this.router = router;
+    this.rpcServer = server;
+    this.rpcClient = server.getRPCClient();
+  }
+
+  /**
+   * Set quota for the federation path.
+   * @param path Federation path.
+   * @param namespaceQuota Name space quota.
+   * @param storagespaceQuota Storage space quota.
+   * @param type StorageType that the space quota is intended to be set on.
+   * @throws IOException
+   */
+  public void setQuota(String path, long namespaceQuota,
+      long storagespaceQuota, StorageType type) throws IOException {
+    rpcServer.checkOperation(OperationCategory.WRITE);
+
+    // Set quota for current path and its children mount table path.
+    final List<RemoteLocation> locations = getQuotaRemoteLocations(path);
+    if (LOG.isDebugEnabled()) {
+      for (RemoteLocation loc : locations) {
+        LOG.debug("Set quota for path: nsId: {}, dest: {}.",
+            loc.getNameserviceId(), loc.getDest());
+      }
+    }
+
+    RemoteMethod method = new RemoteMethod("setQuota",
+        new Class<?>[] {String.class, long.class, long.class,
+            StorageType.class},
+        new RemoteParam(), namespaceQuota, storagespaceQuota, type);
+    rpcClient.invokeConcurrent(locations, method, false, false);
+  }
+
+  /**
+   * Get quota usage for the federation path.
+   * @param path Federation path.
+   * @return Aggregated quota.
+   * @throws IOException
+   */
+  public QuotaUsage getQuotaUsage(String path) throws IOException {
+    final List<RemoteLocation> quotaLocs = getValidQuotaLocations(path);
+    RemoteMethod method = new RemoteMethod("getQuotaUsage",
+        new Class<?>[] {String.class}, new RemoteParam());
+    Map<RemoteLocation, QuotaUsage> results = rpcClient.invokeConcurrent(
+        quotaLocs, method, true, false, QuotaUsage.class);
+
+    return aggregateQuota(results);
+  }
+
+  /**
+   * Get valid quota remote locations used in {@link #getQuotaUsage(String)}.
+   * Differentiate the method {@link #getQuotaRemoteLocations(String)}, this
+   * method will do some additional filtering.
+   * @param path Federation path.
+   * @return List of valid quota remote locations.
+   * @throws IOException
+   */
+  private List<RemoteLocation> getValidQuotaLocations(String path)
+      throws IOException {
+    final List<RemoteLocation> locations = getQuotaRemoteLocations(path);
+
+    // NameService -> Locations
+    Map<String, List<RemoteLocation>> validLocations = new HashMap<>();
+    for (RemoteLocation loc : locations) {
+      String nsId = loc.getNameserviceId();
+      List<RemoteLocation> dests = validLocations.get(nsId);
+      if (dests == null) {
+        dests = new LinkedList<>();
+        dests.add(loc);
+        validLocations.put(nsId, dests);
+      } else {
+        // Ensure the paths in the same nameservice is different.
+        // Don't include parent-child paths.
+        boolean isChildPath = false;
+        for (RemoteLocation d : dests) {
+          if (loc.getDest().startsWith(d.getDest())) {
+            isChildPath = true;
+            break;
+          }
+        }
+
+        if (!isChildPath) {
+          dests.add(loc);
+        }
+      }
+    }
+
+    List<RemoteLocation> quotaLocs = new LinkedList<>();
+    for (List<RemoteLocation> locs : validLocations.values()) {
+      quotaLocs.addAll(locs);
+    }
+
+    return quotaLocs;
+  }
+
+  /**
+   * Aggregate quota that queried from sub-clusters.
+   * @param results Quota query result.
+   * @return Aggregated Quota.
+   */
+  private QuotaUsage aggregateQuota(Map<RemoteLocation, QuotaUsage> results) {
+    long nsCount = 0;
+    long ssCount = 0;
+    boolean hasQuotaUnSet = false;
+
+    for (Map.Entry<RemoteLocation, QuotaUsage> entry : results.entrySet()) {
+      RemoteLocation loc = entry.getKey();
+      QuotaUsage usage = entry.getValue();
+      if (usage != null) {
+        // If quota is not set in real FileSystem, the usage
+        // value will return -1.
+        if (usage.getQuota() == -1 && usage.getSpaceQuota() == -1) {
+          hasQuotaUnSet = true;
+        }
+
+        nsCount += usage.getFileAndDirectoryCount();
+        ssCount += usage.getSpaceConsumed();
+        LOG.debug(
+            "Get quota usage for path: nsId: {}, dest: {},"
+                + " nsCount: {}, ssCount: {}.",
+            loc.getNameserviceId(), loc.getDest(),
+            usage.getFileAndDirectoryCount(), usage.getSpaceConsumed());
+      }
+    }
+
+    QuotaUsage.Builder builder = new QuotaUsage.Builder()
+        .fileAndDirectoryCount(nsCount).spaceConsumed(ssCount);
+    if (hasQuotaUnSet) {
+      builder.quota(HdfsConstants.QUOTA_DONT_SET);
+    }
+
+    return builder.build();
+  }
+
+  /**
+   * Get all quota remote locations across subclusters under given
+   * federation path.
+   * @param path Federation path.
+   * @return List of quota remote locations.
+   * @throws IOException
+   */
+  private List<RemoteLocation> getQuotaRemoteLocations(String path)
+      throws IOException {
+    List<RemoteLocation> locations = new LinkedList<>();
+    RouterQuotaManager manager = this.router.getQuotaManager();
+    if (manager != null) {
+      Set<String> childrenPaths = manager.getPaths(path);
+      for (String childPath : childrenPaths) {
+        locations.addAll(rpcServer.getLocationsForPath(childPath, true));
+      }
+    }
+
+    return locations;
+  }
+}

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

@@ -45,6 +45,8 @@ import org.apache.hadoop.util.JvmPauseMonitor;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.annotations.VisibleForTesting;
+
 /**
  * Router that provides a unified view of multiple federated HDFS clusters. It
  * has two main roles: (1) federated interface and (2) NameNode heartbeat.
@@ -105,7 +107,10 @@ public class Router extends CompositeService {
   /** JVM pauses (GC and others). */
   private JvmPauseMonitor pauseMonitor;
 
-
+  /** Quota usage update service. */
+  private RouterQuotaUpdateService quotaUpdateService;
+  /** Quota cache manager. */
+  private RouterQuotaManager quotaManager;
 
   /////////////////////////////////////////////////////////
   // Constructor
@@ -200,6 +205,14 @@ public class Router extends CompositeService {
       this.pauseMonitor.init(conf);
     }
 
+    // Initial quota relevant service
+    if (conf.getBoolean(DFSConfigKeys.DFS_ROUTER_QUOTA_ENABLE,
+        DFSConfigKeys.DFS_ROUTER_QUOTA_ENABLED_DEFAULT)) {
+      this.quotaManager = new RouterQuotaManager();
+      this.quotaUpdateService = new RouterQuotaUpdateService(this);
+      addService(this.quotaUpdateService);
+    }
+
     super.serviceInit(conf);
   }
 
@@ -524,4 +537,27 @@ public class Router extends CompositeService {
       this.namenodeResolver.setRouterId(this.routerId);
     }
   }
+
+  /**
+   * If the quota system is enabled in Router.
+   */
+  public boolean isQuotaEnabled() {
+    return this.quotaManager != null;
+  }
+
+  /**
+   * Get route quota manager.
+   * @return RouterQuotaManager Quota manager.
+   */
+  public RouterQuotaManager getQuotaManager() {
+    return this.quotaManager;
+  }
+
+  /**
+   * Get quota cache update service.
+   */
+  @VisibleForTesting
+  RouterQuotaUpdateService getQuotaCacheUpdateService() {
+    return this.quotaUpdateService;
+  }
 }

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

@@ -0,0 +1,160 @@
+/**
+ * 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.Set;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+
+/**
+ * Router quota manager in Router. The manager maintains
+ * {@link RouterQuotaUsage} cache of mount tables and do management
+ * for the quota caches.
+ */
+public class RouterQuotaManager {
+  /** Quota usage <MountTable Path, Aggregated QuotaUsage> cache. */
+  private TreeMap<String, RouterQuotaUsage> cache;
+
+  /** Lock to access the quota cache. */
+  private final ReadWriteLock readWriteLock = new ReentrantReadWriteLock();
+  private final Lock readLock = readWriteLock.readLock();
+  private final Lock writeLock = readWriteLock.writeLock();
+
+  public RouterQuotaManager() {
+    this.cache = new TreeMap<>();
+  }
+
+  /**
+   * Get all the mount quota paths.
+   */
+  public Set<String> getAll() {
+    readLock.lock();
+    try {
+      return this.cache.keySet();
+    } finally {
+      readLock.unlock();
+    }
+  }
+
+  /**
+   * Get the nearest ancestor's quota usage, and meanwhile its quota was set.
+   * @param path The path being written.
+   * @return RouterQuotaUsage Quota usage.
+   */
+  public RouterQuotaUsage getQuotaUsage(String path) {
+    readLock.lock();
+    try {
+      RouterQuotaUsage quotaUsage = this.cache.get(path);
+      if (quotaUsage != null && isQuotaSet(quotaUsage)) {
+        return quotaUsage;
+      }
+
+      // If not found, look for its parent path usage value.
+      int pos = path.lastIndexOf(Path.SEPARATOR);
+      if (pos != -1) {
+        String parentPath = path.substring(0, pos);
+        return getQuotaUsage(parentPath);
+      }
+    } finally {
+      readLock.unlock();
+    }
+
+    return null;
+  }
+
+  /**
+   * Get children paths (can including itself) under specified federation path.
+   * @param parentPath
+   * @return Set<String> Children path set.
+   */
+  public Set<String> getPaths(String parentPath) {
+    readLock.lock();
+    try {
+      String from = parentPath;
+      String to = parentPath + Character.MAX_VALUE;
+      SortedMap<String, RouterQuotaUsage> subMap = this.cache.subMap(from, to);
+      return subMap.keySet();
+    } finally {
+      readLock.unlock();
+    }
+  }
+
+  /**
+   * Put new entity into cache.
+   * @param path Mount table path.
+   * @param quotaUsage Corresponding cache value.
+   */
+  public void put(String path, RouterQuotaUsage quotaUsage) {
+    writeLock.lock();
+    try {
+      this.cache.put(path, quotaUsage);
+    } finally {
+      writeLock.unlock();
+    }
+  }
+
+  /**
+   * Remove the entity from cache.
+   * @param path Mount table path.
+   */
+  public void remove(String path) {
+    writeLock.lock();
+    try {
+      this.cache.remove(path);
+    } finally {
+      writeLock.unlock();
+    }
+  }
+
+  /**
+   * Clean up the cache.
+   */
+  public void clear() {
+    writeLock.lock();
+    try {
+      this.cache.clear();
+    } finally {
+      writeLock.unlock();
+    }
+  }
+
+  /**
+   * Check if the quota was set.
+   * @param quota RouterQuotaUsage set in mount table.
+   */
+  public boolean isQuotaSet(RouterQuotaUsage quota) {
+    if (quota != null) {
+      long nsQuota = quota.getQuota();
+      long ssQuota = quota.getSpaceQuota();
+
+      // once nsQuota or ssQuota was set, this mount table is quota set
+      if (nsQuota != HdfsConstants.QUOTA_DONT_SET
+          || ssQuota != HdfsConstants.QUOTA_DONT_SET) {
+        return true;
+      }
+    }
+
+    return false;
+  }
+}

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

@@ -0,0 +1,228 @@
+/**
+ * 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.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.QuotaUsage;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.server.federation.store.MountTableStore;
+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.UpdateMountTableEntryRequest;
+import org.apache.hadoop.hdfs.server.federation.store.records.MountTable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Service to periodically update the {@link RouterQuotaUsage}
+ * cached information in the {@link Router} and update corresponding
+ * mount table in State Store.
+ */
+public class RouterQuotaUpdateService extends PeriodicService {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(RouterQuotaUpdateService.class);
+
+  private MountTableStore mountTableStore;
+  private RouterRpcServer rpcServer;
+  /** Router using this Service. */
+  private final Router router;
+  /** Router Quota manager. */
+  private RouterQuotaManager quotaManager;
+
+  public RouterQuotaUpdateService(final Router router) throws IOException {
+    super(RouterQuotaUpdateService.class.getName());
+    this.router = router;
+    this.rpcServer = router.getRpcServer();
+    this.quotaManager = router.getQuotaManager();
+
+    if (this.quotaManager == null) {
+      throw new IOException("Router quota manager is not initialized.");
+    }
+  }
+
+  @Override
+  protected void serviceInit(Configuration conf) throws Exception {
+    this.setIntervalMs(conf.getTimeDuration(
+        DFSConfigKeys.DFS_ROUTER_QUOTA_CACHE_UPATE_INTERVAL,
+        DFSConfigKeys.DFS_ROUTER_QUOTA_CACHE_UPATE_INTERVAL_DEFAULT,
+        TimeUnit.MILLISECONDS));
+
+    super.serviceInit(conf);
+  }
+
+  @Override
+  protected void periodicInvoke() {
+    LOG.debug("Start to update quota cache.");
+    try {
+      List<MountTable> updateMountTables = new LinkedList<>();
+      List<MountTable> mountTables = getQuotaSetMountTables();
+      for (MountTable entry : mountTables) {
+        String src = entry.getSourcePath();
+        RouterQuotaUsage oldQuota = entry.getQuota();
+        long nsQuota = oldQuota.getQuota();
+        long ssQuota = oldQuota.getSpaceQuota();
+        // Call RouterRpcServer#getQuotaUsage for getting current quota usage.
+        QuotaUsage currentQuotaUsage = this.rpcServer.getQuotaModule()
+            .getQuotaUsage(src);
+        // If quota is not set in some subclusters under federation path,
+        // set quota for this path.
+        if (currentQuotaUsage.getQuota() == HdfsConstants.QUOTA_DONT_SET) {
+          this.rpcServer.setQuota(src, nsQuota, ssQuota, null);
+        }
+
+        RouterQuotaUsage newQuota = generateNewQuota(oldQuota,
+            currentQuotaUsage);
+        this.quotaManager.put(src, newQuota);
+        entry.setQuota(newQuota);
+
+        // only update mount tables which quota was changed
+        if (!oldQuota.equals(newQuota)) {
+          updateMountTables.add(entry);
+
+          LOG.debug(
+              "Update quota usage entity of path: {}, nsCount: {},"
+                  + " nsQuota: {}, ssCount: {}, ssQuota: {}.",
+              src, newQuota.getFileAndDirectoryCount(),
+              newQuota.getQuota(), newQuota.getSpaceConsumed(),
+              newQuota.getSpaceQuota());
+        }
+      }
+
+      updateMountTableEntries(updateMountTables);
+    } catch (IOException e) {
+      LOG.error("Quota cache updated error.", e);
+    }
+  }
+
+  /**
+   * Get mount table store management interface.
+   * @return MountTableStore instance.
+   * @throws IOException
+   */
+  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 all the existing mount tables.
+   * @return List of mount tables.
+   * @throws IOException
+   */
+  private List<MountTable> getMountTableEntries() throws IOException {
+    // scan mount tables from root path
+    GetMountTableEntriesRequest getRequest = GetMountTableEntriesRequest
+        .newInstance("/");
+    GetMountTableEntriesResponse getResponse = getMountTableStore()
+        .getMountTableEntries(getRequest);
+    return getResponse.getEntries();
+  }
+
+  /**
+   * Get mount tables which quota was set.
+   * During this time, the quota usage cache will also be updated by
+   * quota manager:
+   * 1. Stale paths (entries) will be removed.
+   * 2. Existing entries will be override and updated.
+   * @return List of mount tables which quota was set.
+   * @throws IOException
+   */
+  private List<MountTable> getQuotaSetMountTables() throws IOException {
+    List<MountTable> mountTables = getMountTableEntries();
+    Set<String> stalePaths = new HashSet<>();
+    for (String path : this.quotaManager.getAll()) {
+      stalePaths.add(path);
+    }
+
+    List<MountTable> neededMountTables = new LinkedList<>();
+    for (MountTable entry : mountTables) {
+      // select mount tables which is quota set
+      if (isQuotaSet(entry)) {
+        neededMountTables.add(entry);
+      }
+
+      // update mount table entries info in quota cache
+      String src = entry.getSourcePath();
+      this.quotaManager.put(src, entry.getQuota());
+      stalePaths.remove(src);
+    }
+
+    // remove stale paths that currently cached
+    for (String stalePath : stalePaths) {
+      this.quotaManager.remove(stalePath);
+    }
+
+    return neededMountTables;
+  }
+
+  /**
+   * Check if the quota was set in given MountTable.
+   * @param mountTable Mount table entry.
+   */
+  private boolean isQuotaSet(MountTable mountTable) {
+    if (mountTable != null) {
+      return this.quotaManager.isQuotaSet(mountTable.getQuota());
+    }
+    return false;
+  }
+
+  /**
+   * Generate a new quota based on old quota and current quota usage value.
+   * @param oldQuota Old quota stored in State Store.
+   * @param currentQuotaUsage Current quota usage value queried from
+   *        subcluster.
+   * @return A new RouterQuotaUsage.
+   */
+  private RouterQuotaUsage generateNewQuota(RouterQuotaUsage oldQuota,
+      QuotaUsage currentQuotaUsage) {
+    RouterQuotaUsage newQuota = new RouterQuotaUsage.Builder()
+        .fileAndDirectoryCount(currentQuotaUsage.getFileAndDirectoryCount())
+        .quota(oldQuota.getQuota())
+        .spaceConsumed(currentQuotaUsage.getSpaceConsumed())
+        .spaceQuota(oldQuota.getSpaceQuota()).build();
+    return newQuota;
+  }
+
+  /**
+   * Write out updated mount table entries into State Store.
+   * @param updateMountTables Mount tables to be updated.
+   * @throws IOException
+   */
+  private void updateMountTableEntries(List<MountTable> updateMountTables)
+      throws IOException {
+    for (MountTable entry : updateMountTables) {
+      UpdateMountTableEntryRequest updateRequest = UpdateMountTableEntryRequest
+          .newInstance(entry);
+      getMountTableStore().updateMountTableEntry(updateRequest);
+    }
+  }
+}

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

@@ -0,0 +1,119 @@
+/**
+ * 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.fs.QuotaUsage;
+import org.apache.hadoop.hdfs.protocol.DSQuotaExceededException;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
+import org.apache.hadoop.hdfs.server.namenode.DirectoryWithQuotaFeature;
+import org.apache.hadoop.hdfs.server.namenode.Quota;
+import org.apache.hadoop.util.StringUtils;
+
+/**
+ * The subclass of {@link QuotaUsage} used in Router-based federation.
+ */
+public final class RouterQuotaUsage extends QuotaUsage {
+
+  /** Default quota usage count. */
+  public static final long QUOTA_USAGE_COUNT_DEFAULT = 0;
+
+  private RouterQuotaUsage(Builder builder) {
+    super(builder);
+  }
+
+  /** Build the instance based on the builder. */
+  public static class Builder extends QuotaUsage.Builder {
+
+    public RouterQuotaUsage build() {
+      return new RouterQuotaUsage(this);
+    }
+
+    @Override
+    public Builder fileAndDirectoryCount(long count) {
+      super.fileAndDirectoryCount(count);
+      return this;
+    }
+
+    @Override
+    public Builder quota(long quota) {
+      super.quota(quota);
+      return this;
+    }
+
+    @Override
+    public Builder spaceConsumed(long spaceConsumed) {
+      super.spaceConsumed(spaceConsumed);
+      return this;
+    }
+
+    @Override
+    public Builder spaceQuota(long spaceQuota) {
+      super.spaceQuota(spaceQuota);
+      return this;
+    }
+  }
+
+  /**
+   * Verify if namespace quota is violated once quota is set. Relevant
+   * method {@link DirectoryWithQuotaFeature#verifyNamespaceQuota}.
+   * @throws NSQuotaExceededException
+   */
+  public void verifyNamespaceQuota() throws NSQuotaExceededException {
+    if (Quota.isViolated(getQuota(), getFileAndDirectoryCount())) {
+      throw new NSQuotaExceededException(getQuota(),
+          getFileAndDirectoryCount());
+    }
+  }
+
+  /**
+   * Verify if storage space quota is violated once quota is set. Relevant
+   * method {@link DirectoryWithQuotaFeature#verifyStoragespaceQuota}.
+   * @throws DSQuotaExceededException
+   */
+  public void verifyStoragespaceQuota() throws DSQuotaExceededException {
+    if (Quota.isViolated(getSpaceQuota(), getSpaceConsumed())) {
+      throw new DSQuotaExceededException(getSpaceQuota(), getSpaceConsumed());
+    }
+  }
+
+  @Override
+  public String toString() {
+    String nsQuota = String.valueOf(getQuota());
+    String nsCount = String.valueOf(getFileAndDirectoryCount());
+    if (getQuota() == HdfsConstants.QUOTA_DONT_SET) {
+      nsQuota = "-";
+      nsCount = "-";
+    }
+
+    String ssQuota = StringUtils.byteDesc(getSpaceQuota());
+    String ssCount = StringUtils.byteDesc(getSpaceConsumed());
+    if (getSpaceQuota() == HdfsConstants.QUOTA_DONT_SET) {
+      ssQuota = "-";
+      ssCount = "-";
+    }
+
+    StringBuilder str = new StringBuilder();
+    str.append("[NsQuota: ").append(nsQuota).append("/")
+        .append(nsCount);
+    str.append(", SsQuota: ").append(ssQuota)
+        .append("/").append(ssCount)
+        .append("]");
+    return str.toString();
+  }
+}

+ 53 - 12
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcClient.java

@@ -147,6 +147,14 @@ public class RouterRpcClient {
         failoverSleepBaseMillis, failoverSleepMaxMillis);
   }
 
+  /**
+   * Get the active namenode resolver used by this client.
+   * @return Active namenode resolver.
+   */
+  public ActiveNamenodeResolver getNamenodeResolver() {
+    return this.namenodeResolver;
+  }
+
   /**
    * Shutdown the client.
    */
@@ -617,9 +625,9 @@ public class RouterRpcClient {
    * @throws IOException if the success condition is not met, return the first
    *                     remote exception generated.
    */
-  public Object invokeSequential(
+  public <T> T invokeSequential(
       final List<? extends RemoteLocationContext> locations,
-      final RemoteMethod remoteMethod, Class<?> expectedResultClass,
+      final RemoteMethod remoteMethod, Class<T> expectedResultClass,
       Object expectedResultValue) throws IOException {
 
     final UserGroupInformation ugi = RouterRpcServer.getRemoteUser();
@@ -639,7 +647,9 @@ public class RouterRpcClient {
         if (isExpectedClass(expectedResultClass, result) &&
             isExpectedValue(expectedResultValue, result)) {
           // Valid result, stop here
-          return result;
+          @SuppressWarnings("unchecked")
+          T ret = (T)result;
+          return ret;
         }
         if (firstResult == null) {
           firstResult = result;
@@ -669,7 +679,9 @@ public class RouterRpcClient {
       throw firstThrownException;
     }
     // Return the last result, whether it is the value we are looking for or a
-    return firstResult;
+    @SuppressWarnings("unchecked")
+    T ret = (T)firstResult;
+    return ret;
   }
 
   /**
@@ -708,6 +720,28 @@ public class RouterRpcClient {
     }
   }
 
+  /**
+   * Invoke multiple concurrent proxy calls to different clients. Returns an
+   * array of results.
+   *
+   * Re-throws exceptions generated by the remote RPC call as either
+   * RemoteException or IOException.
+   *
+   * @param <T> The type of the remote location.
+   * @param locations List of remote locations to call concurrently.
+   * @param method The remote method and parameters to invoke.
+   * @param requireResponse If true an exception will be thrown if all calls do
+   *          not complete. If false exceptions are ignored and all data results
+   *          successfully received are returned.
+   * @param standby If the requests should go to the standby namenodes too.
+   * @throws IOException If all the calls throw an exception.
+   */
+  public <T extends RemoteLocationContext, R> void invokeConcurrent(
+      final Collection<T> locations, final RemoteMethod method,
+      boolean requireResponse, boolean standby) throws IOException {
+    invokeConcurrent(locations, method, requireResponse, standby, void.class);
+  }
+
   /**
    * Invokes multiple concurrent proxy calls to different clients. Returns an
    * array of results.
@@ -716,20 +750,24 @@ public class RouterRpcClient {
    * RemoteException or IOException.
    *
    * @param <T> The type of the remote location.
+   * @param <R> The type of the remote method return.
    * @param locations List of remote locations to call concurrently.
    * @param method The remote method and parameters to invoke.
    * @param requireResponse If true an exception will be thrown if all calls do
    *          not complete. If false exceptions are ignored and all data results
    *          successfully received are returned.
    * @param standby If the requests should go to the standby namenodes too.
+   * @param clazz Type of the remote return type.
    * @return Result of invoking the method per subcluster: nsId -> result.
    * @throws IOException If requiredResponse=true and any of the calls throw an
    *           exception.
    */
-  public <T extends RemoteLocationContext> Map<T, Object> invokeConcurrent(
+  public <T extends RemoteLocationContext, R> Map<T, R> invokeConcurrent(
       final Collection<T> locations, final RemoteMethod method,
-      boolean requireResponse, boolean standby) throws IOException {
-    return invokeConcurrent(locations, method, requireResponse, standby, -1);
+      boolean requireResponse, boolean standby, Class<R> clazz)
+          throws IOException {
+    return invokeConcurrent(
+        locations, method, requireResponse, standby, -1, clazz);
   }
 
   /**
@@ -739,6 +777,8 @@ public class RouterRpcClient {
    * Re-throws exceptions generated by the remote RPC call as either
    * RemoteException or IOException.
    *
+   * @param <T> The type of the remote location.
+   * @param <R> The type of the remote method return.
    * @param locations List of remote locations to call concurrently.
    * @param method The remote method and parameters to invoke.
    * @param requireResponse If true an exception will be thrown if all calls do
@@ -746,14 +786,15 @@ public class RouterRpcClient {
    *          successfully received are returned.
    * @param standby If the requests should go to the standby namenodes too.
    * @param timeOutMs Timeout for each individual call.
+   * @param clazz Type of the remote return type.
    * @return Result of invoking the method per subcluster: nsId -> result.
    * @throws IOException If requiredResponse=true and any of the calls throw an
    *           exception.
    */
   @SuppressWarnings("unchecked")
-  public <T extends RemoteLocationContext> Map<T, Object> invokeConcurrent(
+  public <T extends RemoteLocationContext, R> Map<T, R> invokeConcurrent(
       final Collection<T> locations, final RemoteMethod method,
-      boolean requireResponse, boolean standby, long timeOutMs)
+      boolean requireResponse, boolean standby, long timeOutMs, Class<R> clazz)
           throws IOException {
 
     final UserGroupInformation ugi = RouterRpcServer.getRemoteUser();
@@ -767,7 +808,7 @@ public class RouterRpcClient {
           getNamenodesForNameservice(ns);
       Object[] paramList = method.getParams(location);
       Object result = invokeMethod(ugi, namenodes, m, paramList);
-      return Collections.singletonMap(location, result);
+      return Collections.singletonMap(location, clazz.cast(result));
     }
 
     List<T> orderedLocations = new LinkedList<>();
@@ -817,14 +858,14 @@ public class RouterRpcClient {
       } else {
         futures = executorService.invokeAll(callables);
       }
-      Map<T, Object> results = new TreeMap<>();
+      Map<T, R> results = new TreeMap<>();
       Map<T, IOException> exceptions = new TreeMap<>();
       for (int i=0; i<futures.size(); i++) {
         T location = orderedLocations.get(i);
         try {
           Future<Object> future = futures.get(i);
           Object result = future.get();
-          results.put(location, result);
+          results.put(location, clazz.cast(result));
         } catch (CancellationException ce) {
           T loc = orderedLocations.get(i);
           String msg =

+ 159 - 128
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java

@@ -28,12 +28,14 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_ROUTER_READER_QUEUE_SIZE_
 
 import java.io.FileNotFoundException;
 import java.io.IOException;
+import java.lang.reflect.Array;
 import java.net.InetSocketAddress;
 import java.util.Collection;
 import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.Iterator;
 import java.util.LinkedHashMap;
+import java.util.LinkedHashSet;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
@@ -90,6 +92,7 @@ import org.apache.hadoop.hdfs.protocol.LastBlockWithStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.OpenFileEntry;
+import org.apache.hadoop.hdfs.protocol.OpenFilesIterator;
 import org.apache.hadoop.hdfs.protocol.OpenFilesIterator.OpenFilesType;
 import org.apache.hadoop.hdfs.protocol.ReplicatedBlockStats;
 import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
@@ -180,6 +183,12 @@ public class RouterRpcServer extends AbstractService implements ClientProtocol {
   /** Category of the operation that a thread is executing. */
   private final ThreadLocal<OperationCategory> opCategory = new ThreadLocal<>();
 
+  // Modules implementing groups of RPC calls
+  /** Router Quota calls. */
+  private final Quota quotaCall;
+  /** Erasure coding calls. */
+  private final ErasureCoding erasureCoding;
+
 
   /**
    * Construct a router RPC server.
@@ -276,6 +285,10 @@ public class RouterRpcServer extends AbstractService implements ClientProtocol {
     // Create the client
     this.rpcClient = new RouterRpcClient(this.conf, this.router.getRouterId(),
         this.namenodeResolver, this.rpcMonitor);
+
+    // Initialize modules
+    this.quotaCall = new Quota(this.router, this);
+    this.erasureCoding = new ErasureCoding(this);
   }
 
   @Override
@@ -361,7 +374,7 @@ public class RouterRpcServer extends AbstractService implements ClientProtocol {
    *                          client requests.
    * @throws UnsupportedOperationException If the operation is not supported.
    */
-  private void checkOperation(OperationCategory op, boolean supported)
+  protected void checkOperation(OperationCategory op, boolean supported)
       throws StandbyException, UnsupportedOperationException {
     checkOperation(op);
 
@@ -383,7 +396,7 @@ public class RouterRpcServer extends AbstractService implements ClientProtocol {
    * @throws StandbyException If the Router is in safe mode and cannot serve
    *                          client requests.
    */
-  private void checkOperation(OperationCategory op) throws StandbyException {
+  protected void checkOperation(OperationCategory op) throws StandbyException {
     // Log the function we are currently calling.
     if (rpcMonitor != null) {
       rpcMonitor.startOp();
@@ -943,8 +956,9 @@ public class RouterRpcServer extends AbstractService implements ClientProtocol {
     RemoteMethod method = new RemoteMethod("getListing",
         new Class<?>[] {String.class, startAfter.getClass(), boolean.class},
         new RemoteParam(), startAfter, needLocation);
-    Map<RemoteLocation, Object> listings =
-        rpcClient.invokeConcurrent(locations, method, false, false);
+    Map<RemoteLocation, DirectoryListing> listings =
+        rpcClient.invokeConcurrent(
+            locations, method, false, false, DirectoryListing.class);
 
     Map<String, HdfsFileStatus> nnListing = new TreeMap<>();
     int totalRemainingEntries = 0;
@@ -953,9 +967,10 @@ public class RouterRpcServer extends AbstractService implements ClientProtocol {
     if (listings != null) {
       // Check the subcluster listing with the smallest name
       String lastName = null;
-      for (Entry<RemoteLocation, Object> entry : listings.entrySet()) {
+      for (Entry<RemoteLocation, DirectoryListing> entry :
+          listings.entrySet()) {
         RemoteLocation location = entry.getKey();
-        DirectoryListing listing = (DirectoryListing) entry.getValue();
+        DirectoryListing listing = entry.getValue();
         if (listing == null) {
           LOG.debug("Cannot get listing from {}", location);
         } else {
@@ -1091,11 +1106,10 @@ public class RouterRpcServer extends AbstractService implements ClientProtocol {
 
     RemoteMethod method = new RemoteMethod("getStats");
     Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
-    Map<FederationNamespaceInfo, Object> results =
-        rpcClient.invokeConcurrent(nss, method, true, false);
+    Map<FederationNamespaceInfo, long[]> results =
+        rpcClient.invokeConcurrent(nss, method, true, false, long[].class);
     long[] combinedData = new long[STATS_ARRAY_LENGTH];
-    for (Object o : results.values()) {
-      long[] data = (long[]) o;
+    for (long[] data : results.values()) {
       for (int i = 0; i < combinedData.length && i < data.length; i++) {
         if (data[i] >= 0) {
           combinedData[i] += data[i];
@@ -1128,11 +1142,13 @@ public class RouterRpcServer extends AbstractService implements ClientProtocol {
         new Class<?>[] {DatanodeReportType.class}, type);
 
     Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
-    Map<FederationNamespaceInfo, Object> results =
-        rpcClient.invokeConcurrent(nss, method, true, false, timeOutMs);
-    for (Entry<FederationNamespaceInfo, Object> entry : results.entrySet()) {
+    Map<FederationNamespaceInfo, DatanodeInfo[]> results =
+        rpcClient.invokeConcurrent(
+            nss, method, true, false, timeOutMs, DatanodeInfo[].class);
+    for (Entry<FederationNamespaceInfo, DatanodeInfo[]> entry :
+        results.entrySet()) {
       FederationNamespaceInfo ns = entry.getKey();
-      DatanodeInfo[] result = (DatanodeInfo[]) entry.getValue();
+      DatanodeInfo[] result = entry.getValue();
       for (DatanodeInfo node : result) {
         String nodeId = node.getXferAddr();
         if (!datanodesMap.containsKey(nodeId)) {
@@ -1162,10 +1178,10 @@ public class RouterRpcServer extends AbstractService implements ClientProtocol {
     RemoteMethod method = new RemoteMethod("getDatanodeStorageReport",
         new Class<?>[] {DatanodeReportType.class}, type);
     Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
-    Map<FederationNamespaceInfo, Object> results =
-        rpcClient.invokeConcurrent(nss, method, true, false);
-    for (Object r : results.values()) {
-      DatanodeStorageReport[] result = (DatanodeStorageReport[]) r;
+    Map<FederationNamespaceInfo, DatanodeStorageReport[]> results =
+        rpcClient.invokeConcurrent(
+            nss, method, true, false, DatanodeStorageReport[].class);
+    for (DatanodeStorageReport[] result : results.values()) {
       for (DatanodeStorageReport node : result) {
         String nodeId = node.getDatanodeInfo().getXferAddr();
         if (!datanodesMap.containsKey(nodeId)) {
@@ -1193,17 +1209,14 @@ public class RouterRpcServer extends AbstractService implements ClientProtocol {
         new Class<?>[] {SafeModeAction.class, boolean.class},
         action, isChecked);
     Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
-    Map<FederationNamespaceInfo, Object> results =
-        rpcClient.invokeConcurrent(nss, method, true, true);
+    Map<FederationNamespaceInfo, Boolean> results =
+        rpcClient.invokeConcurrent(nss, method, true, true, boolean.class);
 
     // We only report true if all the name space are in safe mode
     int numSafemode = 0;
-    for (Object result : results.values()) {
-      if (result instanceof Boolean) {
-        boolean safemode = (boolean) result;
-        if (safemode) {
-          numSafemode++;
-        }
+    for (boolean safemode : results.values()) {
+      if (safemode) {
+        numSafemode++;
       }
     }
     return numSafemode == results.size();
@@ -1216,18 +1229,14 @@ public class RouterRpcServer extends AbstractService implements ClientProtocol {
     RemoteMethod method = new RemoteMethod("restoreFailedStorage",
         new Class<?>[] {String.class}, arg);
     final Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
-    Map<FederationNamespaceInfo, Object> ret =
-        rpcClient.invokeConcurrent(nss, method, true, false);
+    Map<FederationNamespaceInfo, Boolean> ret =
+        rpcClient.invokeConcurrent(nss, method, true, false, boolean.class);
 
     boolean success = true;
-    Object obj = ret;
-    @SuppressWarnings("unchecked")
-    Map<FederationNamespaceInfo, Boolean> results =
-        (Map<FederationNamespaceInfo, Boolean>)obj;
-    Collection<Boolean> sucesses = results.values();
-    for (boolean s : sucesses) {
+    for (boolean s : ret.values()) {
       if (!s) {
         success = false;
+        break;
       }
     }
     return success;
@@ -1240,18 +1249,14 @@ public class RouterRpcServer extends AbstractService implements ClientProtocol {
     RemoteMethod method = new RemoteMethod("saveNamespace",
         new Class<?>[] {Long.class, Long.class}, timeWindow, txGap);
     final Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
-    Map<FederationNamespaceInfo, Object> ret =
-        rpcClient.invokeConcurrent(nss, method, true, false);
+    Map<FederationNamespaceInfo, Boolean> ret =
+        rpcClient.invokeConcurrent(nss, method, true, false, boolean.class);
 
     boolean success = true;
-    Object obj = ret;
-    @SuppressWarnings("unchecked")
-    Map<FederationNamespaceInfo, Boolean> results =
-        (Map<FederationNamespaceInfo, Boolean>)obj;
-    Collection<Boolean> sucesses = results.values();
-    for (boolean s : sucesses) {
+    for (boolean s : ret.values()) {
       if (!s) {
         success = false;
+        break;
       }
     }
     return success;
@@ -1263,17 +1268,12 @@ public class RouterRpcServer extends AbstractService implements ClientProtocol {
 
     RemoteMethod method = new RemoteMethod("rollEdits", new Class<?>[] {});
     final Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
-    Map<FederationNamespaceInfo, Object> ret =
-        rpcClient.invokeConcurrent(nss, method, true, false);
+    Map<FederationNamespaceInfo, Long> ret =
+        rpcClient.invokeConcurrent(nss, method, true, false, long.class);
 
     // Return the maximum txid
     long txid = 0;
-    Object obj = ret;
-    @SuppressWarnings("unchecked")
-    Map<FederationNamespaceInfo, Long> results =
-        (Map<FederationNamespaceInfo, Long>)obj;
-    Collection<Long> txids = results.values();
-    for (long t : txids) {
+    for (long t : ret.values()) {
       if (t > txid) {
         txid = t;
       }
@@ -1308,17 +1308,13 @@ public class RouterRpcServer extends AbstractService implements ClientProtocol {
     RemoteMethod method = new RemoteMethod("rollingUpgrade",
         new Class<?>[] {RollingUpgradeAction.class}, action);
     final Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
-    Map<FederationNamespaceInfo, Object> ret =
-        rpcClient.invokeConcurrent(nss, method, true, false);
+    Map<FederationNamespaceInfo, RollingUpgradeInfo> ret =
+        rpcClient.invokeConcurrent(
+            nss, method, true, false, RollingUpgradeInfo.class);
 
     // Return the first rolling upgrade info
     RollingUpgradeInfo info = null;
-    Object obj = ret;
-    @SuppressWarnings("unchecked")
-    Map<FederationNamespaceInfo, RollingUpgradeInfo> results =
-        (Map<FederationNamespaceInfo, RollingUpgradeInfo>)obj;
-    Collection<RollingUpgradeInfo> infos = results.values();
-    for (RollingUpgradeInfo infoNs : infos) {
+    for (RollingUpgradeInfo infoNs : ret.values()) {
       if (info == null && infoNs != null) {
         info = infoNs;
       }
@@ -1370,10 +1366,9 @@ public class RouterRpcServer extends AbstractService implements ClientProtocol {
       final List<RemoteLocation> locations = getLocationsForPath(path, false);
       RemoteMethod method = new RemoteMethod("getContentSummary",
           new Class<?>[] {String.class}, new RemoteParam());
-      @SuppressWarnings("unchecked")
-      Map<String, ContentSummary> results =
-          (Map<String, ContentSummary>) ((Object)rpcClient.invokeConcurrent(
-              locations, method, false, false));
+      Map<RemoteLocation, ContentSummary> results =
+          rpcClient.invokeConcurrent(
+              locations, method, false, false, ContentSummary.class);
       summaries.addAll(results.values());
     } catch (FileNotFoundException e) {
       notFoundException = e;
@@ -1767,17 +1762,12 @@ public class RouterRpcServer extends AbstractService implements ClientProtocol {
     RemoteMethod method = new RemoteMethod(
         "getCurrentEditLogTxid", new Class<?>[] {});
     final Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
-    Map<FederationNamespaceInfo, Object> ret =
-        rpcClient.invokeConcurrent(nss, method, true, false);
+    Map<FederationNamespaceInfo, Long> ret =
+        rpcClient.invokeConcurrent(nss, method, true, false, long.class);
 
     // Return the maximum txid
     long txid = 0;
-    Object obj = ret;
-    @SuppressWarnings("unchecked")
-    Map<FederationNamespaceInfo, Long> results =
-        (Map<FederationNamespaceInfo, Long>)obj;
-    Collection<Long> txids = results.values();
-    for (long t : txids) {
+    for (long t : ret.values()) {
       if (t > txid) {
         txid = t;
       }
@@ -1810,49 +1800,16 @@ public class RouterRpcServer extends AbstractService implements ClientProtocol {
     checkOperation(OperationCategory.WRITE, false);
   }
 
-  @Override
-  public ErasureCodingPolicyInfo[] getErasureCodingPolicies()
-      throws IOException {
-    checkOperation(OperationCategory.READ, false);
-    return null;
-  }
-
-  @Override // ClientProtocol
-  public ErasureCodingPolicy getErasureCodingPolicy(String src)
-      throws IOException {
-    checkOperation(OperationCategory.READ, false);
-    return null;
-  }
-
-  @Override // ClientProtocol
-  public void setErasureCodingPolicy(String src, String ecPolicyName)
-      throws IOException {
-    checkOperation(OperationCategory.WRITE, false);
-  }
-
-  @Override // ClientProtocol
-  public void unsetErasureCodingPolicy(String src) throws IOException {
-    checkOperation(OperationCategory.WRITE, false);
-  }
-
   @Override // ClientProtocol
   public void setQuota(String path, long namespaceQuota, long storagespaceQuota,
       StorageType type) throws IOException {
-    checkOperation(OperationCategory.WRITE);
-
-    // TODO assign global replicas instead of applying them to each folder
-    final List<RemoteLocation> locations = getLocationsForPath(path, true);
-    RemoteMethod method = new RemoteMethod("setQuota",
-        new Class<?>[] {String.class, Long.class, Long.class,
-            StorageType.class},
-        new RemoteParam(), namespaceQuota, storagespaceQuota, type);
-    rpcClient.invokeConcurrent(locations, method, false, false);
+    this.quotaCall.setQuota(path, namespaceQuota, storagespaceQuota, type);
   }
 
   @Override // ClientProtocol
   public QuotaUsage getQuotaUsage(String path) throws IOException {
-    checkOperation(OperationCategory.READ, false);
-    return null;
+    checkOperation(OperationCategory.READ);
+    return this.quotaCall.getQuotaUsage(path);
   }
 
   @Override
@@ -1896,38 +1853,61 @@ public class RouterRpcServer extends AbstractService implements ClientProtocol {
     return null;
   }
 
-  @Override
+  @Override // ClientProtocol
+  public ErasureCodingPolicyInfo[] getErasureCodingPolicies()
+      throws IOException {
+    return erasureCoding.getErasureCodingPolicies();
+  }
+
+  @Override // ClientProtocol
+  public Map<String, String> getErasureCodingCodecs() throws IOException {
+    return erasureCoding.getErasureCodingCodecs();
+  }
+
+  @Override // ClientProtocol
   public AddErasureCodingPolicyResponse[] addErasureCodingPolicies(
       ErasureCodingPolicy[] policies) throws IOException {
-    checkOperation(OperationCategory.WRITE, false);
-    return null;
+    return erasureCoding.addErasureCodingPolicies(policies);
   }
 
-  @Override
-  public void removeErasureCodingPolicy(String arg0) throws IOException {
-    checkOperation(OperationCategory.WRITE, false);
+  @Override // ClientProtocol
+  public void removeErasureCodingPolicy(String ecPolicyName)
+      throws IOException {
+    erasureCoding.removeErasureCodingPolicy(ecPolicyName);
   }
 
-  @Override
-  public void disableErasureCodingPolicy(String arg0) throws IOException {
-    checkOperation(OperationCategory.WRITE, false);
+  @Override // ClientProtocol
+  public void disableErasureCodingPolicy(String ecPolicyName)
+      throws IOException {
+    erasureCoding.disableErasureCodingPolicy(ecPolicyName);
   }
 
-  @Override
-  public void enableErasureCodingPolicy(String arg0) throws IOException {
-    checkOperation(OperationCategory.WRITE, false);
+  @Override // ClientProtocol
+  public void enableErasureCodingPolicy(String ecPolicyName)
+      throws IOException {
+    erasureCoding.enableErasureCodingPolicy(ecPolicyName);
   }
 
-  @Override
-  public ECBlockGroupStats getECBlockGroupStats() throws IOException {
-    checkOperation(OperationCategory.READ, false);
-    return null;
+  @Override // ClientProtocol
+  public ErasureCodingPolicy getErasureCodingPolicy(String src)
+      throws IOException {
+    return erasureCoding.getErasureCodingPolicy(src);
+  }
+
+  @Override // ClientProtocol
+  public void setErasureCodingPolicy(String src, String ecPolicyName)
+      throws IOException {
+    erasureCoding.setErasureCodingPolicy(src, ecPolicyName);
+  }
+
+  @Override // ClientProtocol
+  public void unsetErasureCodingPolicy(String src) throws IOException {
+    erasureCoding.unsetErasureCodingPolicy(src);
   }
 
   @Override
-  public Map<String, String> getErasureCodingCodecs() throws IOException {
-    checkOperation(OperationCategory.READ, false);
-    return null;
+  public ECBlockGroupStats getECBlockGroupStats() throws IOException {
+    return erasureCoding.getECBlockGroupStats();
   }
 
   @Override
@@ -1940,12 +1920,13 @@ public class RouterRpcServer extends AbstractService implements ClientProtocol {
   @Override
   public BatchedEntries<OpenFileEntry> listOpenFiles(long prevId)
       throws IOException {
-    return listOpenFiles(prevId, EnumSet.of(OpenFilesType.ALL_OPEN_FILES));
+    return listOpenFiles(prevId, EnumSet.of(OpenFilesType.ALL_OPEN_FILES),
+        OpenFilesIterator.FILTER_PATH_DEFAULT);
   }
 
   @Override
   public BatchedEntries<OpenFileEntry> listOpenFiles(long prevId,
-      EnumSet<OpenFilesType> openFilesTypes) throws IOException {
+      EnumSet<OpenFilesType> openFilesTypes, String path) throws IOException {
     checkOperation(OperationCategory.READ, false);
     return null;
   }
@@ -1994,7 +1975,7 @@ public class RouterRpcServer extends AbstractService implements ClientProtocol {
    * @return Prioritized list of locations in the federated cluster.
    * @throws IOException If the location for this path cannot be determined.
    */
-  private List<RemoteLocation> getLocationsForPath(
+  protected List<RemoteLocation> getLocationsForPath(
       String path, boolean failIfLocked) throws IOException {
     try {
       // Check the location for this path
@@ -2014,6 +1995,16 @@ public class RouterRpcServer extends AbstractService implements ClientProtocol {
           }
           throw new IOException(path + " is in a read only mount point");
         }
+
+        // Check quota
+        if (this.router.isQuotaEnabled()) {
+          RouterQuotaUsage quotaUsage = this.router.getQuotaManager()
+              .getQuotaUsage(path);
+          if (quotaUsage != null) {
+            quotaUsage.verifyNamespaceQuota();
+            quotaUsage.verifyStoragespaceQuota();
+          }
+        }
       }
 
       return location.getDestinations();
@@ -2117,4 +2108,44 @@ public class RouterRpcServer extends AbstractService implements ClientProtocol {
     UserGroupInformation ugi = Server.getRemoteUser();
     return (ugi != null) ? ugi : UserGroupInformation.getCurrentUser();
   }
+
+  /**
+   * Merge the outputs from multiple namespaces.
+   * @param map Namespace -> Output array.
+   * @param clazz Class of the values.
+   * @return Array with the outputs.
+   */
+  protected static <T> T[] merge(
+      Map<FederationNamespaceInfo, T[]> map, Class<T> clazz) {
+
+    // Put all results into a set to avoid repeats
+    Set<T> ret = new LinkedHashSet<>();
+    for (T[] values : map.values()) {
+      for (T val : values) {
+        ret.add(val);
+      }
+    }
+
+    return toArray(ret, clazz);
+  }
+
+  /**
+   * Convert a set of values into an array.
+   * @param set Input set.
+   * @param clazz Class of the values.
+   * @return Array with the values in set.
+   */
+  private static <T> T[] toArray(Set<T> set, Class<T> clazz) {
+    @SuppressWarnings("unchecked")
+    T[] combinedData = (T[]) Array.newInstance(clazz, set.size());
+    combinedData = set.toArray(combinedData);
+    return combinedData;
+  }
+
+  /**
+   * Get quota module implement.
+   */
+  public Quota getQuotaModule() {
+    return this.quotaCall;
+  }
 }

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

@@ -0,0 +1,31 @@
+/**
+ * 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;
+
+/**
+ * States of the Router.
+ */
+public enum RouterServiceState {
+  NONE,
+  INITIALIZING,
+  SAFEMODE,
+  RUNNING,
+  STOPPING,
+  SHUTDOWN,
+  EXPIRED;
+}

+ 78 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/RouterStore.java

@@ -0,0 +1,78 @@
+/**
+ * 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.store;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hdfs.server.federation.store.driver.StateStoreDriver;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.GetRouterRegistrationRequest;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.GetRouterRegistrationResponse;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.GetRouterRegistrationsRequest;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.GetRouterRegistrationsResponse;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.RouterHeartbeatRequest;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.RouterHeartbeatResponse;
+import org.apache.hadoop.hdfs.server.federation.store.records.RouterState;
+
+/**
+ * Management API for
+ * {@link org.apache.hadoop.hdfs.server.federation.store.records.RouterState
+ *  RouterState} records in the state store. Accesses the data store via the
+ * {@link org.apache.hadoop.hdfs.server.federation.store.driver.
+ * StateStoreDriver StateStoreDriver} interface. No data is cached.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public abstract class RouterStore extends CachedRecordStore<RouterState> {
+
+  public RouterStore(StateStoreDriver driver) {
+    super(RouterState.class, driver, true);
+  }
+
+  /**
+   * Fetches the current router state object.
+   *
+   * @param request Fully populated request object.
+   * @return The matching router record or null if none exists.
+   * @throws IOException Throws exception if unable to query the data store or
+   *           if more than one matching record is found.
+   */
+  public abstract GetRouterRegistrationResponse getRouterRegistration(
+      GetRouterRegistrationRequest request) throws IOException;
+
+  /**
+   * Fetches all router status objects.
+   *
+   * @param request Fully populated request object.
+   * @return List of Router records present in the data store.
+   * @throws IOException Throws exception if unable to query the data store
+   */
+  public abstract GetRouterRegistrationsResponse getRouterRegistrations(
+      GetRouterRegistrationsRequest request) throws IOException;
+
+  /**
+   * Update the state of this router in the State Store.
+   *
+   * @param request Fully populated request object.
+   * @return True if the update was successfully recorded, false otherwise.
+   * @throws IOException Throws exception if unable to query the data store
+   */
+  public abstract RouterHeartbeatResponse routerHeartbeat(
+      RouterHeartbeatRequest request) throws IOException;
+}

+ 93 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/impl/RouterStoreImpl.java

@@ -0,0 +1,93 @@
+/**
+ * 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.store.impl;
+
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hdfs.server.federation.store.RouterStore;
+import org.apache.hadoop.hdfs.server.federation.store.driver.StateStoreDriver;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.GetRouterRegistrationRequest;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.GetRouterRegistrationResponse;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.GetRouterRegistrationsRequest;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.GetRouterRegistrationsResponse;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.RouterHeartbeatRequest;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.RouterHeartbeatResponse;
+import org.apache.hadoop.hdfs.server.federation.store.records.Query;
+import org.apache.hadoop.hdfs.server.federation.store.records.QueryResult;
+import org.apache.hadoop.hdfs.server.federation.store.records.RouterState;
+
+/**
+ * Implementation of the {@link RouterStore} state store API.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class RouterStoreImpl extends RouterStore {
+
+  public RouterStoreImpl(StateStoreDriver driver) {
+    super(driver);
+  }
+
+  @Override
+  public GetRouterRegistrationResponse getRouterRegistration(
+      GetRouterRegistrationRequest request) throws IOException {
+
+    final RouterState partial = RouterState.newInstance();
+    partial.setAddress(request.getRouterId());
+    final Query<RouterState> query = new Query<RouterState>(partial);
+    RouterState record = getDriver().get(getRecordClass(), query);
+    if (record != null) {
+      overrideExpiredRecord(record);
+    }
+    GetRouterRegistrationResponse response =
+        GetRouterRegistrationResponse.newInstance();
+    response.setRouter(record);
+    return response;
+  }
+
+  @Override
+  public GetRouterRegistrationsResponse getRouterRegistrations(
+      GetRouterRegistrationsRequest request) throws IOException {
+
+    // Get all values from the cache
+    QueryResult<RouterState> recordsAndTimeStamp =
+        getCachedRecordsAndTimeStamp();
+    List<RouterState> records = recordsAndTimeStamp.getRecords();
+    long timestamp = recordsAndTimeStamp.getTimestamp();
+
+    // Generate response
+    GetRouterRegistrationsResponse response =
+        GetRouterRegistrationsResponse.newInstance();
+    response.setRouters(records);
+    response.setTimestamp(timestamp);
+    return response;
+  }
+
+  @Override
+  public RouterHeartbeatResponse routerHeartbeat(RouterHeartbeatRequest request)
+      throws IOException {
+
+    RouterState record = request.getRouter();
+    boolean status = getDriver().put(record, true, false);
+    RouterHeartbeatResponse response =
+        RouterHeartbeatResponse.newInstance(status);
+    return response;
+  }
+}

+ 47 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/GetRouterRegistrationRequest.java

@@ -0,0 +1,47 @@
+/**
+ * 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.store.protocol;
+
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.hdfs.server.federation.store.driver.StateStoreSerializer;
+
+/**
+ * API request for retrieving a single router registration present in the state
+ * store.
+ */
+public abstract class GetRouterRegistrationRequest {
+
+  public static GetRouterRegistrationRequest newInstance() {
+    return StateStoreSerializer.newRecord(GetRouterRegistrationRequest.class);
+  }
+
+  public static GetRouterRegistrationRequest newInstance(String routerId) {
+    GetRouterRegistrationRequest request = newInstance();
+    request.setRouterId(routerId);
+    return request;
+  }
+
+  @Public
+  @Unstable
+  public abstract String getRouterId();
+
+  @Public
+  @Unstable
+  public abstract void setRouterId(String routerId);
+}

+ 44 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/GetRouterRegistrationResponse.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.server.federation.store.protocol;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.hdfs.server.federation.store.driver.StateStoreSerializer;
+import org.apache.hadoop.hdfs.server.federation.store.records.RouterState;
+
+/**
+ * API response for retrieving a single router registration present in the state
+ * store.
+ */
+public abstract class GetRouterRegistrationResponse {
+
+  public static GetRouterRegistrationResponse newInstance() throws IOException {
+    return StateStoreSerializer.newRecord(GetRouterRegistrationResponse.class);
+  }
+
+  @Public
+  @Unstable
+  public abstract RouterState getRouter() throws IOException;
+
+  @Public
+  @Unstable
+  public abstract void setRouter(RouterState router) throws IOException;
+}

+ 34 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/GetRouterRegistrationsRequest.java

@@ -0,0 +1,34 @@
+/**
+ * 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.store.protocol;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hdfs.server.federation.store.driver.StateStoreSerializer;
+
+/**
+ * API request for retrieving a all non-expired router registrations present in
+ * the state store.
+ */
+public abstract class GetRouterRegistrationsRequest {
+
+  public static GetRouterRegistrationsRequest newInstance() throws IOException {
+    return StateStoreSerializer.newRecord(GetRouterRegistrationsRequest.class);
+  }
+
+}

+ 55 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/GetRouterRegistrationsResponse.java

@@ -0,0 +1,55 @@
+/**
+ * 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.store.protocol;
+
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.hdfs.server.federation.store.driver.StateStoreSerializer;
+import org.apache.hadoop.hdfs.server.federation.store.records.RouterState;
+
+/**
+ * API response for retrieving a all non-expired router registrations present in
+ * the state store.
+ */
+public abstract class GetRouterRegistrationsResponse {
+
+  public static GetRouterRegistrationsResponse newInstance()
+      throws IOException {
+    return StateStoreSerializer.newRecord(GetRouterRegistrationsResponse.class);
+  }
+
+  @Public
+  @Unstable
+  public abstract List<RouterState> getRouters() throws IOException;
+
+  @Public
+  @Unstable
+  public abstract void setRouters(List<RouterState> routers)
+      throws IOException;
+
+  @Public
+  @Unstable
+  public abstract long getTimestamp();
+
+  @Public
+  @Unstable
+  public abstract void setTimestamp(long time);
+}

+ 50 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/RouterHeartbeatRequest.java

@@ -0,0 +1,50 @@
+/**
+ * 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.store.protocol;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.hdfs.server.federation.store.driver.StateStoreSerializer;
+import org.apache.hadoop.hdfs.server.federation.store.records.RouterState;
+
+/**
+ * API request for registering a router with the state store.
+ */
+public abstract class RouterHeartbeatRequest {
+
+  public static RouterHeartbeatRequest newInstance() throws IOException {
+    return StateStoreSerializer.newRecord(RouterHeartbeatRequest.class);
+  }
+
+  public static RouterHeartbeatRequest newInstance(RouterState router)
+      throws IOException {
+    RouterHeartbeatRequest request = newInstance();
+    request.setRouter(router);
+    return request;
+  }
+
+  @Public
+  @Unstable
+  public abstract RouterState getRouter() throws IOException;
+
+  @Public
+  @Unstable
+  public abstract void setRouter(RouterState routerState);
+}

+ 49 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/RouterHeartbeatResponse.java

@@ -0,0 +1,49 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.federation.store.protocol;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.hdfs.server.federation.store.driver.StateStoreSerializer;
+
+/**
+ * API response for registering a router with the state store.
+ */
+public abstract class RouterHeartbeatResponse {
+
+  public static RouterHeartbeatResponse newInstance() throws IOException {
+    return StateStoreSerializer.newRecord(RouterHeartbeatResponse.class);
+  }
+
+  public static RouterHeartbeatResponse newInstance(boolean status)
+      throws IOException {
+    RouterHeartbeatResponse response = newInstance();
+    response.setStatus(status);
+    return response;
+  }
+
+  @Public
+  @Unstable
+  public abstract boolean getStatus();
+
+  @Public
+  @Unstable
+  public abstract void setStatus(boolean result);
+}

+ 70 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/impl/pb/GetRouterRegistrationRequestPBImpl.java

@@ -0,0 +1,70 @@
+/**
+ * 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.store.protocol.impl.pb;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.GetRouterRegistrationRequestProto;
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.GetRouterRegistrationRequestProto.Builder;
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.GetRouterRegistrationRequestProtoOrBuilder;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.GetRouterRegistrationRequest;
+import org.apache.hadoop.hdfs.server.federation.store.records.impl.pb.PBRecord;
+
+import com.google.protobuf.Message;
+
+/**
+ * Protobuf implementation of the state store API object
+ * GetRouterRegistrationRequest.
+ */
+public class GetRouterRegistrationRequestPBImpl
+    extends GetRouterRegistrationRequest implements PBRecord {
+
+  private FederationProtocolPBTranslator<GetRouterRegistrationRequestProto,
+      Builder, GetRouterRegistrationRequestProtoOrBuilder> translator =
+          new FederationProtocolPBTranslator<GetRouterRegistrationRequestProto,
+              Builder, GetRouterRegistrationRequestProtoOrBuilder>(
+                  GetRouterRegistrationRequestProto.class);
+
+  public GetRouterRegistrationRequestPBImpl() {
+  }
+
+  @Override
+  public GetRouterRegistrationRequestProto getProto() {
+    return this.translator.build();
+  }
+
+  @Override
+  public void setProto(Message proto) {
+    this.translator.setProto(proto);
+  }
+
+  @Override
+  public void readInstance(String base64String) throws IOException {
+    this.translator.readInstance(base64String);
+  }
+
+  @Override
+  public String getRouterId() {
+    return this.translator.getProtoOrBuilder().getRouterId();
+  }
+
+  @Override
+  public void setRouterId(String routerId) {
+    this.translator.getBuilder().setRouterId(routerId);
+  }
+}

+ 79 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/impl/pb/GetRouterRegistrationResponsePBImpl.java

@@ -0,0 +1,79 @@
+/**
+ * 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.store.protocol.impl.pb;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.GetRouterRegistrationResponseProto;
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.GetRouterRegistrationResponseProto.Builder;
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.GetRouterRegistrationResponseProtoOrBuilder;
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.RouterRecordProto;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.GetRouterRegistrationResponse;
+import org.apache.hadoop.hdfs.server.federation.store.records.RouterState;
+import org.apache.hadoop.hdfs.server.federation.store.records.impl.pb.PBRecord;
+import org.apache.hadoop.hdfs.server.federation.store.records.impl.pb.RouterStatePBImpl;
+
+import com.google.protobuf.Message;
+
+/**
+ * Protobuf implementation of the state store API object
+ * GetRouterRegistrationResponse.
+ */
+public class GetRouterRegistrationResponsePBImpl
+    extends GetRouterRegistrationResponse implements PBRecord {
+
+  private FederationProtocolPBTranslator<GetRouterRegistrationResponseProto,
+      Builder, GetRouterRegistrationResponseProtoOrBuilder> translator =
+          new FederationProtocolPBTranslator<
+              GetRouterRegistrationResponseProto, Builder,
+              GetRouterRegistrationResponseProtoOrBuilder>(
+                  GetRouterRegistrationResponseProto.class);
+
+  public GetRouterRegistrationResponsePBImpl() {
+  }
+
+  @Override
+  public GetRouterRegistrationResponseProto getProto() {
+    return this.translator.build();
+  }
+
+  @Override
+  public void setProto(Message protocol) {
+    this.translator.setProto(protocol);
+  }
+
+  @Override
+  public void readInstance(String base64String) throws IOException {
+    this.translator.readInstance(base64String);
+  }
+
+  @Override
+  public RouterState getRouter() {
+    RouterRecordProto proto = this.translator.getProtoOrBuilder().getRouter();
+    return new RouterStatePBImpl(proto);
+  }
+
+  @Override
+  public void setRouter(RouterState router) {
+    if (router instanceof RouterStatePBImpl) {
+      RouterStatePBImpl routerPB = (RouterStatePBImpl)router;
+      RouterRecordProto routerProto = routerPB.getProto();
+      this.translator.getBuilder().setRouter(routerProto);
+    }
+  }
+}

+ 61 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/impl/pb/GetRouterRegistrationsRequestPBImpl.java

@@ -0,0 +1,61 @@
+/**
+ * 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.store.protocol.impl.pb;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.GetRouterRegistrationsRequestProto;
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.GetRouterRegistrationsRequestProto.Builder;
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.GetRouterRegistrationsRequestProtoOrBuilder;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.GetRouterRegistrationsRequest;
+import org.apache.hadoop.hdfs.server.federation.store.records.impl.pb.PBRecord;
+
+import com.google.protobuf.Message;
+
+/**
+ * Protobuf implementation of the state store API object
+ * GetRouterRegistrationsRequest.
+ */
+public class GetRouterRegistrationsRequestPBImpl
+    extends GetRouterRegistrationsRequest implements PBRecord {
+
+  private FederationProtocolPBTranslator<GetRouterRegistrationsRequestProto,
+      Builder, GetRouterRegistrationsRequestProtoOrBuilder> translator =
+          new FederationProtocolPBTranslator<
+              GetRouterRegistrationsRequestProto, Builder,
+              GetRouterRegistrationsRequestProtoOrBuilder>(
+                  GetRouterRegistrationsRequestProto.class);
+
+  public GetRouterRegistrationsRequestPBImpl() {
+  }
+
+  @Override
+  public GetRouterRegistrationsRequestProto getProto() {
+    return this.translator.build();
+  }
+
+  @Override
+  public void setProto(Message proto) {
+    this.translator.setProto(proto);
+  }
+
+  @Override
+  public void readInstance(String base64String) throws IOException {
+    this.translator.readInstance(base64String);
+  }
+}

+ 102 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/impl/pb/GetRouterRegistrationsResponsePBImpl.java

@@ -0,0 +1,102 @@
+/**
+ * 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.store.protocol.impl.pb;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.GetRouterRegistrationsResponseProto;
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.GetRouterRegistrationsResponseProto.Builder;
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.GetRouterRegistrationsResponseProtoOrBuilder;
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.RouterRecordProto;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.GetRouterRegistrationsResponse;
+import org.apache.hadoop.hdfs.server.federation.store.records.RouterState;
+import org.apache.hadoop.hdfs.server.federation.store.records.impl.pb.PBRecord;
+import org.apache.hadoop.hdfs.server.federation.store.records.impl.pb.RouterStatePBImpl;
+
+import com.google.protobuf.Message;
+
+/**
+ * Protobuf implementation of the state store API object
+ * GetRouterRegistrationsResponse.
+ */
+public class GetRouterRegistrationsResponsePBImpl
+    extends GetRouterRegistrationsResponse implements PBRecord {
+
+  private FederationProtocolPBTranslator<GetRouterRegistrationsResponseProto,
+      Builder, GetRouterRegistrationsResponseProtoOrBuilder> translator =
+          new FederationProtocolPBTranslator<
+              GetRouterRegistrationsResponseProto, Builder,
+              GetRouterRegistrationsResponseProtoOrBuilder>(
+                  GetRouterRegistrationsResponseProto.class);
+
+  public GetRouterRegistrationsResponsePBImpl() {
+
+  }
+
+  @Override
+  public GetRouterRegistrationsResponseProto getProto() {
+    return this.translator.build();
+  }
+
+  @Override
+  public void setProto(Message proto) {
+    this.translator.setProto(proto);
+  }
+
+  @Override
+  public void readInstance(String base64String) throws IOException {
+    this.translator.readInstance(base64String);
+  }
+
+  @Override
+  public List<RouterState> getRouters() throws IOException {
+
+    List<RouterState> ret = new ArrayList<RouterState>();
+    List<RouterRecordProto> memberships =
+        this.translator.getProtoOrBuilder().getRoutersList();
+    for (RouterRecordProto memberProto : memberships) {
+      RouterState membership = new RouterStatePBImpl(memberProto);
+      ret.add(membership);
+    }
+    return ret;
+  }
+
+  @Override
+  public void setRouters(List<RouterState> records) throws IOException {
+
+    this.translator.getBuilder().clearRouters();
+    for (RouterState router : records) {
+      if (router instanceof RouterStatePBImpl) {
+        RouterStatePBImpl routerPB = (RouterStatePBImpl) router;
+        this.translator.getBuilder().addRouters(routerPB.getProto());
+      }
+    }
+  }
+
+  @Override
+  public long getTimestamp() {
+    return this.translator.getProtoOrBuilder().getTimestamp();
+  }
+
+  @Override
+  public void setTimestamp(long time) {
+    this.translator.getBuilder().setTimestamp(time);
+  }
+}

+ 78 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/impl/pb/RouterHeartbeatRequestPBImpl.java

@@ -0,0 +1,78 @@
+/**
+ * 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.store.protocol.impl.pb;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.RouterHeartbeatRequestProto;
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.RouterHeartbeatRequestProto.Builder;
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.RouterHeartbeatRequestProtoOrBuilder;
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.RouterRecordProto;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.RouterHeartbeatRequest;
+import org.apache.hadoop.hdfs.server.federation.store.records.RouterState;
+import org.apache.hadoop.hdfs.server.federation.store.records.impl.pb.PBRecord;
+import org.apache.hadoop.hdfs.server.federation.store.records.impl.pb.RouterStatePBImpl;
+
+import com.google.protobuf.Message;
+
+/**
+ * Protobuf implementation of the state store API object
+ * RouterHeartbeatRequest.
+ */
+public class RouterHeartbeatRequestPBImpl extends RouterHeartbeatRequest
+    implements PBRecord {
+
+  private FederationProtocolPBTranslator<RouterHeartbeatRequestProto, Builder,
+      RouterHeartbeatRequestProtoOrBuilder> translator =
+          new FederationProtocolPBTranslator<RouterHeartbeatRequestProto,
+              Builder, RouterHeartbeatRequestProtoOrBuilder>(
+                  RouterHeartbeatRequestProto.class);
+
+  public RouterHeartbeatRequestPBImpl() {
+  }
+
+  @Override
+  public RouterHeartbeatRequestProto getProto() {
+    return this.translator.build();
+  }
+
+  @Override
+  public void setProto(Message proto) {
+    this.translator.setProto(proto);
+  }
+
+  @Override
+  public void readInstance(String base64String) throws IOException {
+    this.translator.readInstance(base64String);
+  }
+
+  @Override
+  public RouterState getRouter() throws IOException {
+    RouterRecordProto routerProto =
+        this.translator.getProtoOrBuilder().getRouter();
+    return new RouterStatePBImpl(routerProto);
+  }
+
+  @Override
+  public void setRouter(RouterState routerState) {
+    if (routerState instanceof RouterStatePBImpl) {
+      RouterStatePBImpl routerStatePB = (RouterStatePBImpl)routerState;
+      this.translator.getBuilder().setRouter(routerStatePB.getProto());
+    }
+  }
+}

+ 70 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/impl/pb/RouterHeartbeatResponsePBImpl.java

@@ -0,0 +1,70 @@
+/**
+ * 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.store.protocol.impl.pb;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.RouterHeartbeatResponseProto;
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.RouterHeartbeatResponseProto.Builder;
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.RouterHeartbeatResponseProtoOrBuilder;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.RouterHeartbeatResponse;
+import org.apache.hadoop.hdfs.server.federation.store.records.impl.pb.PBRecord;
+
+import com.google.protobuf.Message;
+
+/**
+ * Protobuf implementation of the state store API object
+ * RouterHeartbeatResponse.
+ */
+public class RouterHeartbeatResponsePBImpl extends RouterHeartbeatResponse
+    implements PBRecord {
+
+  private FederationProtocolPBTranslator<RouterHeartbeatResponseProto, Builder,
+      RouterHeartbeatResponseProtoOrBuilder> translator =
+          new FederationProtocolPBTranslator<RouterHeartbeatResponseProto,
+              Builder, RouterHeartbeatResponseProtoOrBuilder>(
+                  RouterHeartbeatResponseProto.class);
+
+  public RouterHeartbeatResponsePBImpl() {
+  }
+
+  @Override
+  public RouterHeartbeatResponseProto getProto() {
+    return this.translator.build();
+  }
+
+  @Override
+  public void setProto(Message proto) {
+    this.translator.setProto(proto);
+  }
+
+  @Override
+  public void readInstance(String base64String) throws IOException {
+    this.translator.readInstance(base64String);
+  }
+
+  @Override
+  public boolean getStatus() {
+    return this.translator.getProtoOrBuilder().getStatus();
+  }
+
+  @Override
+  public void setStatus(boolean result) {
+    this.translator.getBuilder().setStatus(result);
+  }
+}

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

@@ -29,9 +29,11 @@ import java.util.TreeMap;
 import org.apache.commons.lang.builder.HashCodeBuilder;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.federation.resolver.RemoteLocation;
 import org.apache.hadoop.hdfs.server.federation.resolver.order.DestinationOrder;
 import org.apache.hadoop.hdfs.server.federation.router.RouterPermissionChecker;
+import org.apache.hadoop.hdfs.server.federation.router.RouterQuotaUsage;
 import org.apache.hadoop.hdfs.server.federation.store.driver.StateStoreSerializer;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.security.UserGroupInformation;
@@ -140,6 +142,14 @@ public abstract class MountTable extends BaseRecord {
     record.setMode(new FsPermission(
         RouterPermissionChecker.MOUNT_TABLE_PERMISSION_DEFAULT));
 
+    // Set quota for mount table
+    RouterQuotaUsage quota = new RouterQuotaUsage.Builder()
+        .fileAndDirectoryCount(RouterQuotaUsage.QUOTA_USAGE_COUNT_DEFAULT)
+        .quota(HdfsConstants.QUOTA_DONT_SET)
+        .spaceConsumed(RouterQuotaUsage.QUOTA_USAGE_COUNT_DEFAULT)
+        .spaceQuota(HdfsConstants.QUOTA_DONT_SET).build();
+    record.setQuota(quota);
+
     // Validate
     record.validate();
     return record;
@@ -248,6 +258,20 @@ public abstract class MountTable extends BaseRecord {
    */
   public abstract void setMode(FsPermission mode);
 
+  /**
+   * Get quota of this mount table entry.
+   *
+   * @return RouterQuotaUsage quota usage
+   */
+  public abstract RouterQuotaUsage getQuota();
+
+  /**
+   * Set quota for this mount table entry.
+   *
+   * @param quota QuotaUsage for mount table entry
+   */
+  public abstract void setQuota(RouterQuotaUsage quota);
+
   /**
    * Get the default location.
    * @return The default location.
@@ -303,6 +327,10 @@ public abstract class MountTable extends BaseRecord {
       sb.append("[mode:").append(this.getMode()).append("]");
     }
 
+    if (this.getQuota() != null) {
+      sb.append("[quota:").append(this.getQuota()).append("]");
+    }
+
     return sb.toString();
   }
 

+ 169 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/RouterState.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.store.records;
+
+import java.io.IOException;
+import java.util.SortedMap;
+import java.util.TreeMap;
+
+import org.apache.hadoop.hdfs.server.federation.router.FederationUtil;
+import org.apache.hadoop.hdfs.server.federation.router.RouterServiceState;
+import org.apache.hadoop.hdfs.server.federation.store.driver.StateStoreSerializer;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Entry to log the state of a
+ * {@link org.apache.hadoop.hdfs.server.federation.router.Router Router} in the
+ * {@link org.apache.hadoop.hdfs.server.federation.store.StateStoreService
+ * FederationStateStoreService}.
+ */
+public abstract class RouterState extends BaseRecord {
+
+  private static final Logger LOG = LoggerFactory.getLogger(RouterState.class);
+
+  /** Expiration time in ms for this entry. */
+  private static long expirationMs;
+
+  /**
+   * Constructors.
+   */
+  public RouterState() {
+    super();
+  }
+
+  public static RouterState newInstance() {
+    RouterState record = StateStoreSerializer.newRecord(RouterState.class);
+    record.init();
+    return record;
+  }
+
+  public static RouterState newInstance(String addr, long startTime,
+      RouterServiceState status) {
+    RouterState record = newInstance();
+    record.setDateStarted(startTime);
+    record.setAddress(addr);
+    record.setStatus(status);
+    record.setCompileInfo(FederationUtil.getCompileInfo());
+    record.setBuildVersion(FederationUtil.getBuildVersion());
+    return record;
+  }
+
+  public abstract void setAddress(String address);
+
+  public abstract void setDateStarted(long dateStarted);
+
+  public abstract String getAddress();
+
+  public abstract StateStoreVersion getStateStoreVersion() throws IOException;
+
+  public abstract void setStateStoreVersion(StateStoreVersion version);
+
+  public abstract RouterServiceState getStatus();
+
+  public abstract void setStatus(RouterServiceState newStatus);
+
+  public abstract String getBuildVersion();
+
+  public abstract void setBuildVersion(String version);
+
+  public abstract String getCompileInfo();
+
+  public abstract void setCompileInfo(String info);
+
+  public abstract long getDateStarted();
+
+  /**
+   * Get the identifier for the Router. It uses the address.
+   *
+   * @return Identifier for the Router.
+   */
+  public String getRouterId() {
+    return getAddress();
+  }
+
+  @Override
+  public boolean like(BaseRecord o) {
+    if (o instanceof RouterState) {
+      RouterState other = (RouterState)o;
+      if (getAddress() != null &&
+          !getAddress().equals(other.getAddress())) {
+        return false;
+      }
+      if (getStatus() != null &&
+          !getStatus().equals(other.getStatus())) {
+        return false;
+      }
+      return true;
+    }
+    return false;
+  }
+
+  @Override
+  public String toString() {
+    return getAddress() + " -> " + getStatus() + "," + getBuildVersion();
+  }
+
+  @Override
+  public SortedMap<String, String> getPrimaryKeys() {
+    SortedMap<String, String> map = new TreeMap<>();
+    map.put("address", getAddress());
+    return map;
+  }
+
+  @Override
+  public boolean validate() {
+    boolean ret = super.validate();
+    if ((getAddress() == null || getAddress().length() == 0) &&
+        getStatus() != RouterServiceState.INITIALIZING) {
+      LOG.error("Invalid router entry, no address specified {}", this);
+      ret = false;
+    }
+    return ret;
+  }
+
+  @Override
+  public int compareTo(BaseRecord other) {
+    if (other == null) {
+      return -1;
+    } else if (other instanceof RouterState) {
+      RouterState router = (RouterState) other;
+      return this.getAddress().compareTo(router.getAddress());
+    } else {
+      return super.compareTo(other);
+    }
+  }
+
+  @Override
+  public boolean checkExpired(long currentTime) {
+    if (super.checkExpired(currentTime)) {
+      setStatus(RouterServiceState.EXPIRED);
+      return true;
+    }
+    return false;
+  }
+
+  @Override
+  public long getExpirationMs() {
+    return RouterState.expirationMs;
+  }
+
+  public static void setExpirationMs(long time) {
+    RouterState.expirationMs = time;
+  }
+}

+ 91 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/StateStoreVersion.java

@@ -0,0 +1,91 @@
+/**
+ * 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.store.records;
+
+import java.util.SortedMap;
+import java.util.TreeMap;
+
+import org.apache.hadoop.hdfs.server.federation.store.driver.StateStoreSerializer;
+
+/**
+ * Entry to track the version of the State Store data stored in the State Store
+ * by a Router.
+ */
+public abstract class StateStoreVersion extends BaseRecord {
+
+  public static StateStoreVersion newInstance() {
+    return StateStoreSerializer.newRecord(StateStoreVersion.class);
+  }
+
+  public static StateStoreVersion newInstance(long membershipVersion,
+      long mountTableVersion) {
+    StateStoreVersion record = newInstance();
+    record.setMembershipVersion(membershipVersion);
+    record.setMountTableVersion(mountTableVersion);
+    return record;
+  }
+
+  public abstract long getMembershipVersion();
+
+  public abstract void setMembershipVersion(long version);
+
+  public abstract long getMountTableVersion();
+
+  public abstract void setMountTableVersion(long version);
+
+  @Override
+  public SortedMap<String, String> getPrimaryKeys() {
+    // This record is not stored directly, no key needed
+    SortedMap<String, String> map = new TreeMap<String, String>();
+    return map;
+  }
+
+  @Override
+  public long getExpirationMs() {
+    // This record is not stored directly, no expiration needed
+    return -1;
+  }
+
+  @Override
+  public void setDateModified(long time) {
+    // We don't store this record directly
+  }
+
+  @Override
+  public long getDateModified() {
+    // We don't store this record directly
+    return 0;
+  }
+
+  @Override
+  public void setDateCreated(long time) {
+    // We don't store this record directly
+  }
+
+  @Override
+  public long getDateCreated() {
+    // We don't store this record directly
+    return 0;
+  }
+
+  @Override
+  public String toString() {
+    return "Membership: " + getMembershipVersion() +
+        " Mount Table: " + getMountTableVersion();
+  }
+}

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

@@ -27,10 +27,13 @@ import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProt
 import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.MountTableRecordProto.DestOrder;
 import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.MountTableRecordProtoOrBuilder;
 import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.RemoteLocationProto;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.QuotaUsageProto;
 import org.apache.hadoop.hdfs.server.federation.resolver.RemoteLocation;
 import org.apache.hadoop.hdfs.server.federation.resolver.order.DestinationOrder;
 import org.apache.hadoop.hdfs.server.federation.router.RouterAdminServer;
 import org.apache.hadoop.hdfs.server.federation.router.RouterPermissionChecker;
+import org.apache.hadoop.hdfs.server.federation.router.RouterQuotaUsage;
 import org.apache.hadoop.hdfs.server.federation.store.protocol.impl.pb.FederationProtocolPBTranslator;
 import org.apache.hadoop.hdfs.server.federation.store.records.MountTable;
 
@@ -250,6 +253,42 @@ public class MountTablePBImpl extends MountTable implements PBRecord {
     }
   }
 
+  @Override
+  public RouterQuotaUsage getQuota() {
+    MountTableRecordProtoOrBuilder proto = this.translator.getProtoOrBuilder();
+
+    long nsQuota = HdfsConstants.QUOTA_DONT_SET;
+    long nsCount = RouterQuotaUsage.QUOTA_USAGE_COUNT_DEFAULT;
+    long ssQuota = HdfsConstants.QUOTA_DONT_SET;
+    long ssCount = RouterQuotaUsage.QUOTA_USAGE_COUNT_DEFAULT;
+    if (proto.hasQuota()) {
+      QuotaUsageProto quotaProto = proto.getQuota();
+      nsQuota = quotaProto.getQuota();
+      nsCount = quotaProto.getFileAndDirectoryCount();
+      ssQuota = quotaProto.getSpaceQuota();
+      ssCount = quotaProto.getSpaceConsumed();
+    }
+
+    RouterQuotaUsage.Builder builder = new RouterQuotaUsage.Builder()
+        .quota(nsQuota).fileAndDirectoryCount(nsCount).spaceQuota(ssQuota)
+        .spaceConsumed(ssCount);
+    return builder.build();
+  }
+
+  @Override
+  public void setQuota(RouterQuotaUsage quota) {
+    Builder builder = this.translator.getBuilder();
+    if (quota == null) {
+      builder.clearQuota();
+    } else {
+      QuotaUsageProto quotaUsage = QuotaUsageProto.newBuilder()
+          .setFileAndDirectoryCount(quota.getFileAndDirectoryCount())
+          .setQuota(quota.getQuota()).setSpaceConsumed(quota.getSpaceConsumed())
+          .setSpaceQuota(quota.getSpaceQuota()).build();
+      builder.setQuota(quotaUsage);
+    }
+  }
+
   private DestinationOrder convert(DestOrder order) {
     switch (order) {
     case LOCAL:

+ 202 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/impl/pb/RouterStatePBImpl.java

@@ -0,0 +1,202 @@
+/**
+ * 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.store.records.impl.pb;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.RouterRecordProto;
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.RouterRecordProto.Builder;
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.RouterRecordProtoOrBuilder;
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.StateStoreVersionRecordProto;
+import org.apache.hadoop.hdfs.server.federation.router.RouterServiceState;
+import org.apache.hadoop.hdfs.server.federation.store.driver.StateStoreSerializer;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.impl.pb.FederationProtocolPBTranslator;
+import org.apache.hadoop.hdfs.server.federation.store.records.RouterState;
+import org.apache.hadoop.hdfs.server.federation.store.records.StateStoreVersion;
+
+import com.google.protobuf.Message;
+
+/**
+ * Protobuf implementation of the RouterState record.
+ */
+public class RouterStatePBImpl extends RouterState implements PBRecord {
+
+  private FederationProtocolPBTranslator<RouterRecordProto, Builder,
+      RouterRecordProtoOrBuilder> translator =
+          new FederationProtocolPBTranslator<RouterRecordProto, Builder,
+             RouterRecordProtoOrBuilder>(RouterRecordProto.class);
+
+  public RouterStatePBImpl() {
+  }
+
+  public RouterStatePBImpl(RouterRecordProto proto) {
+    this.translator.setProto(proto);
+  }
+
+  @Override
+  public RouterRecordProto getProto() {
+    return this.translator.build();
+  }
+
+  @Override
+  public void setProto(Message proto) {
+    this.translator.setProto(proto);
+  }
+
+  @Override
+  public void readInstance(String base64String) throws IOException {
+    this.translator.readInstance(base64String);
+  }
+
+  @Override
+  public void setAddress(String address) {
+    RouterRecordProto.Builder builder = this.translator.getBuilder();
+    if (address == null) {
+      builder.clearAddress();
+    } else {
+      builder.setAddress(address);
+    }
+  }
+
+  @Override
+  public String getAddress() {
+    RouterRecordProtoOrBuilder proto = this.translator.getProtoOrBuilder();
+    if (!proto.hasAddress()) {
+      return null;
+    }
+    return proto.getAddress();
+  }
+
+  @Override
+  public void setStateStoreVersion(StateStoreVersion version) {
+    RouterRecordProto.Builder builder = this.translator.getBuilder();
+    if (version instanceof StateStoreVersionPBImpl) {
+      StateStoreVersionPBImpl versionPB = (StateStoreVersionPBImpl)version;
+      StateStoreVersionRecordProto versionProto =
+          (StateStoreVersionRecordProto)versionPB.getProto();
+      builder.setStateStoreVersion(versionProto);
+    } else {
+      builder.clearStateStoreVersion();
+    }
+  }
+
+  @Override
+  public StateStoreVersion getStateStoreVersion() throws IOException {
+    RouterRecordProtoOrBuilder proto = this.translator.getProtoOrBuilder();
+    if (!proto.hasStateStoreVersion()) {
+      return null;
+    }
+    StateStoreVersionRecordProto versionProto = proto.getStateStoreVersion();
+    StateStoreVersion version =
+        StateStoreSerializer.newRecord(StateStoreVersion.class);
+    if (version instanceof StateStoreVersionPBImpl) {
+      StateStoreVersionPBImpl versionPB = (StateStoreVersionPBImpl)version;
+      versionPB.setProto(versionProto);
+      return versionPB;
+    } else {
+      throw new IOException("Cannot get State Store version");
+    }
+  }
+
+  @Override
+  public RouterServiceState getStatus() {
+    RouterRecordProtoOrBuilder proto = this.translator.getProtoOrBuilder();
+    if (!proto.hasStatus()) {
+      return null;
+    }
+    return RouterServiceState.valueOf(proto.getStatus());
+  }
+
+  @Override
+  public void setStatus(RouterServiceState newStatus) {
+    RouterRecordProto.Builder builder = this.translator.getBuilder();
+    if (newStatus == null) {
+      builder.clearStatus();
+    } else {
+      builder.setStatus(newStatus.toString());
+    }
+  }
+
+  @Override
+  public String getBuildVersion() {
+    RouterRecordProtoOrBuilder proto = this.translator.getProtoOrBuilder();
+    if (!proto.hasBuildVersion()) {
+      return null;
+    }
+    return proto.getBuildVersion();
+  }
+
+  @Override
+  public void setBuildVersion(String version) {
+    RouterRecordProto.Builder builder = this.translator.getBuilder();
+    if (version == null) {
+      builder.clearBuildVersion();
+    } else {
+      builder.setBuildVersion(version);
+    }
+  }
+
+  @Override
+  public String getCompileInfo() {
+    RouterRecordProtoOrBuilder proto = this.translator.getProtoOrBuilder();
+    if (!proto.hasCompileInfo()) {
+      return null;
+    }
+    return proto.getCompileInfo();
+  }
+
+  @Override
+  public void setCompileInfo(String info) {
+    RouterRecordProto.Builder builder = this.translator.getBuilder();
+    if (info == null) {
+      builder.clearCompileInfo();
+    } else {
+      builder.setCompileInfo(info);
+    }
+  }
+
+  @Override
+  public void setDateStarted(long dateStarted) {
+    this.translator.getBuilder().setDateStarted(dateStarted);
+  }
+
+  @Override
+  public long getDateStarted() {
+    return this.translator.getProtoOrBuilder().getDateStarted();
+  }
+
+  @Override
+  public void setDateModified(long time) {
+    this.translator.getBuilder().setDateModified(time);
+  }
+
+  @Override
+  public long getDateModified() {
+    return this.translator.getProtoOrBuilder().getDateModified();
+  }
+
+  @Override
+  public void setDateCreated(long time) {
+    this.translator.getBuilder().setDateCreated(time);
+  }
+
+  @Override
+  public long getDateCreated() {
+    return this.translator.getProtoOrBuilder().getDateCreated();
+  }
+}

+ 79 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/impl/pb/StateStoreVersionPBImpl.java

@@ -0,0 +1,79 @@
+/**
+ * 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.store.records.impl.pb;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.StateStoreVersionRecordProto;
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.StateStoreVersionRecordProto.Builder;
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.StateStoreVersionRecordProtoOrBuilder;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.impl.pb.FederationProtocolPBTranslator;
+import org.apache.hadoop.hdfs.server.federation.store.records.StateStoreVersion;
+
+import com.google.protobuf.Message;
+
+/**
+ * Protobuf implementation of the StateStoreVersion record.
+ */
+public class StateStoreVersionPBImpl extends StateStoreVersion
+    implements PBRecord {
+
+  private FederationProtocolPBTranslator<StateStoreVersionRecordProto, Builder,
+      StateStoreVersionRecordProtoOrBuilder> translator =
+          new FederationProtocolPBTranslator<StateStoreVersionRecordProto,
+              Builder, StateStoreVersionRecordProtoOrBuilder>(
+                  StateStoreVersionRecordProto.class);
+
+  public StateStoreVersionPBImpl() {
+  }
+
+  @Override
+  public StateStoreVersionRecordProto getProto() {
+    return this.translator.build();
+  }
+
+  @Override
+  public void setProto(Message proto) {
+    this.translator.setProto(proto);
+  }
+
+  @Override
+  public void readInstance(String base64String) throws IOException {
+    this.translator.readInstance(base64String);
+  }
+
+  @Override
+  public long getMembershipVersion() {
+    return this.translator.getProtoOrBuilder().getMembershipVersion();
+  }
+
+  @Override
+  public void setMembershipVersion(long version) {
+    this.translator.getBuilder().setMembershipVersion(version);
+  }
+
+  @Override
+  public long getMountTableVersion() {
+    return this.translator.getProtoOrBuilder().getMountTableVersion();
+  }
+
+  @Override
+  public void setMountTableVersion(long version) {
+    this.translator.getBuilder().setMountTableVersion(version);
+  }
+}

+ 18 - 9
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -1767,11 +1767,12 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
    * TODO: HDFS-12969 - to report open files by type.
    *
    * @param prevId the cursor INode id.
-   * @param openFilesTypes
+   * @param openFilesTypes types to filter the open files.
+   * @param path path to filter the open files.
    * @throws IOException
    */
   BatchedListEntries<OpenFileEntry> listOpenFiles(long prevId,
-      EnumSet<OpenFilesType> openFilesTypes) throws IOException {
+      EnumSet<OpenFilesType> openFilesTypes, String path) throws IOException {
     final String operationName = "listOpenFiles";
     checkSuperuserPrivilege();
     checkOperation(OperationCategory.READ);
@@ -1780,10 +1781,11 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     try {
       checkOperation(OperationCategory.READ);
       if(openFilesTypes.contains(OpenFilesType.ALL_OPEN_FILES)) {
-        batchedListEntries = leaseManager.getUnderConstructionFiles(prevId);
+        batchedListEntries = leaseManager.getUnderConstructionFiles(prevId,
+            path);
       } else {
         if(openFilesTypes.contains(OpenFilesType.BLOCKING_DECOMMISSION)) {
-          batchedListEntries = getFilesBlockingDecom(prevId);
+          batchedListEntries = getFilesBlockingDecom(prevId, path);
         } else {
           throw new IllegalArgumentException("Unknown OpenFileType: "
               + openFilesTypes);
@@ -1799,7 +1801,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     return batchedListEntries;
   }
 
-  public BatchedListEntries<OpenFileEntry> getFilesBlockingDecom(long prevId) {
+  public BatchedListEntries<OpenFileEntry> getFilesBlockingDecom(long prevId,
+      String path) {
     assert hasReadLock();
     final List<OpenFileEntry> openFileEntries = Lists.newArrayList();
     LightWeightHashSet<Long> openFileIds = new LightWeightHashSet<>();
@@ -1817,10 +1820,16 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
         Preconditions.checkState(ucFile instanceof INodeFile);
         openFileIds.add(ucFileId);
         INodeFile inodeFile = ucFile.asFile();
-        openFileEntries.add(new OpenFileEntry(
-            inodeFile.getId(), inodeFile.getFullPathName(),
-            inodeFile.getFileUnderConstructionFeature().getClientName(),
-            inodeFile.getFileUnderConstructionFeature().getClientMachine()));
+
+        String fullPathName = inodeFile.getFullPathName();
+        if (org.apache.commons.lang.StringUtils.isEmpty(path)
+            || fullPathName.startsWith(path)) {
+          openFileEntries.add(new OpenFileEntry(inodeFile.getId(),
+              inodeFile.getFullPathName(),
+              inodeFile.getFileUnderConstructionFeature().getClientName(),
+              inodeFile.getFileUnderConstructionFeature().getClientMachine()));
+        }
+
         if (openFileIds.size() >= this.maxListOpenFilesResponses) {
           return new BatchedListEntries<>(openFileEntries, true);
         }

+ 10 - 6
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java

@@ -727,6 +727,13 @@ public class INodeFile extends INodeWithAdditionalFields
     this.blocks = BlockInfo.EMPTY_ARRAY;
   }
 
+  private void updateRemovedUnderConstructionFiles(
+      ReclaimContext reclaimContext) {
+    if (isUnderConstruction() && reclaimContext.removedUCFiles != null) {
+      reclaimContext.removedUCFiles.add(getId());
+    }
+  }
+
   @Override
   public void cleanSubtree(ReclaimContext reclaimContext,
       final int snapshot, int priorSnapshotId) {
@@ -735,6 +742,7 @@ public class INodeFile extends INodeWithAdditionalFields
       // TODO: avoid calling getStoragePolicyID
       sf.cleanFile(reclaimContext, this, snapshot, priorSnapshotId,
           getStoragePolicyID());
+      updateRemovedUnderConstructionFiles(reclaimContext);
     } else {
       if (snapshot == CURRENT_STATE_ID) {
         if (priorSnapshotId == NO_SNAPSHOT_ID) {
@@ -747,9 +755,7 @@ public class INodeFile extends INodeWithAdditionalFields
           // clean the 0-sized block if the file is UC
           if (uc != null) {
             uc.cleanZeroSizeBlock(this, reclaimContext.collectedBlocks);
-            if (reclaimContext.removedUCFiles != null) {
-              reclaimContext.removedUCFiles.add(getId());
-            }
+            updateRemovedUnderConstructionFiles(reclaimContext);
           }
         }
       }
@@ -768,9 +774,7 @@ public class INodeFile extends INodeWithAdditionalFields
           reclaimContext.collectedBlocks);
       sf.clearDiffs();
     }
-    if (isUnderConstruction() && reclaimContext.removedUCFiles != null) {
-      reclaimContext.removedUCFiles.add(getId());
-    }
+    updateRemovedUnderConstructionFiles(reclaimContext);
   }
 
   public void clearFile(ReclaimContext reclaimContext) {

+ 20 - 6
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java

@@ -37,10 +37,13 @@ import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
 
 import com.google.common.collect.Lists;
+
+import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.fs.BatchedRemoteIterator.BatchedListEntries;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.OpenFileEntry;
+import org.apache.hadoop.hdfs.protocol.OpenFilesIterator;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.util.Daemon;
@@ -258,6 +261,12 @@ public class LeaseManager {
     return iipSet;
   }
 
+  public BatchedListEntries<OpenFileEntry> getUnderConstructionFiles(
+      final long prevId) throws IOException {
+    return getUnderConstructionFiles(prevId,
+        OpenFilesIterator.FILTER_PATH_DEFAULT);
+  }
+
   /**
    * Get a batch of under construction files from the currently active leases.
    * File INodeID is the cursor used to fetch new batch of results and the
@@ -270,7 +279,7 @@ public class LeaseManager {
    * @throws IOException
    */
   public BatchedListEntries<OpenFileEntry> getUnderConstructionFiles(
-      final long prevId) throws IOException {
+      final long prevId, final String path) throws IOException {
     assert fsnamesystem.hasReadLock();
     SortedMap<Long, Lease> remainingLeases;
     synchronized (this) {
@@ -283,6 +292,7 @@ public class LeaseManager {
         Lists.newArrayListWithExpectedSize(numResponses);
 
     int count = 0;
+    String fullPathName = null;
     for (Long inodeId: inodeIds) {
       final INodeFile inodeFile =
           fsnamesystem.getFSDirectory().getInode(inodeId).asFile();
@@ -291,11 +301,15 @@ public class LeaseManager {
             inodeFile.getFullPathName());
         continue;
       }
-      openFileEntries.add(new OpenFileEntry(
-          inodeFile.getId(), inodeFile.getFullPathName(),
-          inodeFile.getFileUnderConstructionFeature().getClientName(),
-          inodeFile.getFileUnderConstructionFeature().getClientMachine()));
-      count++;
+
+      fullPathName = inodeFile.getFullPathName();
+      if (StringUtils.isEmpty(path) || fullPathName.startsWith(path)) {
+        openFileEntries.add(new OpenFileEntry(inodeFile.getId(), fullPathName,
+            inodeFile.getFileUnderConstructionFeature().getClientName(),
+            inodeFile.getFileUnderConstructionFeature().getClientMachine()));
+        count++;
+      }
+
       if (count >= numResponses) {
         break;
       }

+ 5 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java

@@ -115,6 +115,7 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.OpenFileEntry;
+import org.apache.hadoop.hdfs.protocol.OpenFilesIterator;
 import org.apache.hadoop.hdfs.protocol.OpenFilesIterator.OpenFilesType;
 import org.apache.hadoop.hdfs.protocol.QuotaByStorageTypeExceededException;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
@@ -1339,14 +1340,15 @@ public class NameNodeRpcServer implements NamenodeProtocols {
   @Override // ClientProtocol
   public BatchedEntries<OpenFileEntry> listOpenFiles(long prevId)
       throws IOException {
-    return listOpenFiles(prevId, EnumSet.of(OpenFilesType.ALL_OPEN_FILES));
+    return listOpenFiles(prevId, EnumSet.of(OpenFilesType.ALL_OPEN_FILES),
+        OpenFilesIterator.FILTER_PATH_DEFAULT);
   }
 
   @Override // ClientProtocol
   public BatchedEntries<OpenFileEntry> listOpenFiles(long prevId,
-      EnumSet<OpenFilesType> openFilesTypes) throws IOException {
+      EnumSet<OpenFilesType> openFilesTypes, String path) throws IOException {
     checkNNStartup();
-    return namesystem.listOpenFiles(prevId, openFilesTypes);
+    return namesystem.listOpenFiles(prevId, openFilesTypes, path);
   }
 
   @Override // ClientProtocol

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

@@ -49,7 +49,7 @@ public enum Quota {
    * Is quota violated?
    * The quota is violated if quota is set and usage > quota. 
    */
-  static boolean isViolated(final long quota, final long usage) {
+  public static boolean isViolated(final long quota, final long usage) {
     return quota >= 0 && usage > quota;
   }
 

+ 64 - 51
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java

@@ -68,6 +68,7 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.XAttrHelper;
 import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
+import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
@@ -163,6 +164,15 @@ public class NamenodeWebHdfsMethods {
      return np;
   }
 
+  protected ClientProtocol getRpcClientProtocol() throws IOException {
+    final NameNode namenode = (NameNode)context.getAttribute("name.node");
+    final ClientProtocol cp = namenode.getRpcServer();
+    if (cp == null) {
+      throw new RetriableException("Namenode is in startup mode");
+    }
+    return cp;
+  }
+
   private <T> T doAs(final UserGroupInformation ugi,
       final PrivilegedExceptionAction<T> action)
           throws IOException, InterruptedException {
@@ -566,7 +576,7 @@ public class NamenodeWebHdfsMethods {
 
     final Configuration conf = (Configuration)context.getAttribute(JspHelper.CURRENT_CONF);
     final NameNode namenode = (NameNode)context.getAttribute("name.node");
-    final NamenodeProtocols np = getRPCServer(namenode);
+    final ClientProtocol cp = getRpcClientProtocol();
 
     switch(op.getValue()) {
     case CREATE:
@@ -590,14 +600,14 @@ public class NamenodeWebHdfsMethods {
           permission.getDirFsPermission() :
           FsCreateModes.create(permission.getDirFsPermission(),
               unmaskedPermission.getDirFsPermission());
-      final boolean b = np.mkdirs(fullpath, masked, true);
+      final boolean b = cp.mkdirs(fullpath, masked, true);
       final String js = JsonUtil.toJsonString("boolean", b);
       return Response.ok(js).type(MediaType.APPLICATION_JSON).build();
     }
     case CREATESYMLINK:
     {
       validateOpParams(op, destination);
-      np.createSymlink(destination.getValue(), fullpath,
+      cp.createSymlink(destination.getValue(), fullpath,
           PermissionParam.getDefaultSymLinkFsPermission(),
           createParent.getValue());
       return Response.ok().type(MediaType.APPLICATION_OCTET_STREAM).build();
@@ -607,18 +617,18 @@ public class NamenodeWebHdfsMethods {
       validateOpParams(op, destination);
       final EnumSet<Options.Rename> s = renameOptions.getValue();
       if (s.isEmpty()) {
-        final boolean b = np.rename(fullpath, destination.getValue());
+        final boolean b = cp.rename(fullpath, destination.getValue());
         final String js = JsonUtil.toJsonString("boolean", b);
         return Response.ok(js).type(MediaType.APPLICATION_JSON).build();
       } else {
-        np.rename2(fullpath, destination.getValue(),
+        cp.rename2(fullpath, destination.getValue(),
             s.toArray(new Options.Rename[s.size()]));
         return Response.ok().type(MediaType.APPLICATION_OCTET_STREAM).build();
       }
     }
     case SETREPLICATION:
     {
-      final boolean b = np.setReplication(fullpath, replication.getValue(conf));
+      final boolean b = cp.setReplication(fullpath, replication.getValue(conf));
       final String js = JsonUtil.toJsonString("boolean", b);
       return Response.ok(js).type(MediaType.APPLICATION_JSON).build();
     }
@@ -628,17 +638,17 @@ public class NamenodeWebHdfsMethods {
         throw new IllegalArgumentException("Both owner and group are empty.");
       }
 
-      np.setOwner(fullpath, owner.getValue(), group.getValue());
+      cp.setOwner(fullpath, owner.getValue(), group.getValue());
       return Response.ok().type(MediaType.APPLICATION_OCTET_STREAM).build();
     }
     case SETPERMISSION:
     {
-      np.setPermission(fullpath, permission.getDirFsPermission());
+      cp.setPermission(fullpath, permission.getDirFsPermission());
       return Response.ok().type(MediaType.APPLICATION_OCTET_STREAM).build();
     }
     case SETTIMES:
     {
-      np.setTimes(fullpath, modificationTime.getValue(), accessTime.getValue());
+      cp.setTimes(fullpath, modificationTime.getValue(), accessTime.getValue());
       return Response.ok().type(MediaType.APPLICATION_OCTET_STREAM).build();
     }
     case RENEWDELEGATIONTOKEN:
@@ -646,7 +656,7 @@ public class NamenodeWebHdfsMethods {
       validateOpParams(op, delegationTokenArgument);
       final Token<DelegationTokenIdentifier> token = new Token<DelegationTokenIdentifier>();
       token.decodeFromUrlString(delegationTokenArgument.getValue());
-      final long expiryTime = np.renewDelegationToken(token);
+      final long expiryTime = cp.renewDelegationToken(token);
       final String js = JsonUtil.toJsonString("long", expiryTime);
       return Response.ok(js).type(MediaType.APPLICATION_JSON).build();
     }
@@ -655,35 +665,35 @@ public class NamenodeWebHdfsMethods {
       validateOpParams(op, delegationTokenArgument);
       final Token<DelegationTokenIdentifier> token = new Token<DelegationTokenIdentifier>();
       token.decodeFromUrlString(delegationTokenArgument.getValue());
-      np.cancelDelegationToken(token);
+      cp.cancelDelegationToken(token);
       return Response.ok().type(MediaType.APPLICATION_OCTET_STREAM).build();
     }
     case MODIFYACLENTRIES: {
       validateOpParams(op, aclPermission);
-      np.modifyAclEntries(fullpath, aclPermission.getAclPermission(true));
+      cp.modifyAclEntries(fullpath, aclPermission.getAclPermission(true));
       return Response.ok().type(MediaType.APPLICATION_OCTET_STREAM).build();
     }
     case REMOVEACLENTRIES: {
       validateOpParams(op, aclPermission);
-      np.removeAclEntries(fullpath, aclPermission.getAclPermission(false));
+      cp.removeAclEntries(fullpath, aclPermission.getAclPermission(false));
       return Response.ok().type(MediaType.APPLICATION_OCTET_STREAM).build();
     }
     case REMOVEDEFAULTACL: {
-      np.removeDefaultAcl(fullpath);
+      cp.removeDefaultAcl(fullpath);
       return Response.ok().type(MediaType.APPLICATION_OCTET_STREAM).build();
     }
     case REMOVEACL: {
-      np.removeAcl(fullpath);
+      cp.removeAcl(fullpath);
       return Response.ok().type(MediaType.APPLICATION_OCTET_STREAM).build();
     }
     case SETACL: {
       validateOpParams(op, aclPermission);
-      np.setAcl(fullpath, aclPermission.getAclPermission(true));
+      cp.setAcl(fullpath, aclPermission.getAclPermission(true));
       return Response.ok().type(MediaType.APPLICATION_OCTET_STREAM).build();
     }
     case SETXATTR: {
       validateOpParams(op, xattrName, xattrSetFlag);
-      np.setXAttr(
+      cp.setXAttr(
           fullpath,
           XAttrHelper.buildXAttr(xattrName.getXAttrName(),
               xattrValue.getXAttrValue()), xattrSetFlag.getFlag());
@@ -691,34 +701,36 @@ public class NamenodeWebHdfsMethods {
     }
     case REMOVEXATTR: {
       validateOpParams(op, xattrName);
-      np.removeXAttr(fullpath, XAttrHelper.buildXAttr(xattrName.getXAttrName()));
+      cp.removeXAttr(fullpath,
+          XAttrHelper.buildXAttr(xattrName.getXAttrName()));
       return Response.ok().type(MediaType.APPLICATION_OCTET_STREAM).build();
     }
     case ALLOWSNAPSHOT: {
-      np.allowSnapshot(fullpath);
+      cp.allowSnapshot(fullpath);
       return Response.ok().type(MediaType.APPLICATION_OCTET_STREAM).build();
     }
     case CREATESNAPSHOT: {
-      String snapshotPath = np.createSnapshot(fullpath, snapshotName.getValue());
+      String snapshotPath =
+          cp.createSnapshot(fullpath, snapshotName.getValue());
       final String js = JsonUtil.toJsonString(
           org.apache.hadoop.fs.Path.class.getSimpleName(), snapshotPath);
       return Response.ok(js).type(MediaType.APPLICATION_JSON).build();
     }
     case RENAMESNAPSHOT: {
       validateOpParams(op, oldSnapshotName, snapshotName);
-      np.renameSnapshot(fullpath, oldSnapshotName.getValue(),
+      cp.renameSnapshot(fullpath, oldSnapshotName.getValue(),
           snapshotName.getValue());
       return Response.ok().type(MediaType.APPLICATION_OCTET_STREAM).build();
     }
     case DISALLOWSNAPSHOT: {
-      np.disallowSnapshot(fullpath);
+      cp.disallowSnapshot(fullpath);
       return Response.ok().type(MediaType.APPLICATION_OCTET_STREAM).build();
     }
     case SETSTORAGEPOLICY: {
       if (policyName.getValue() == null) {
         throw new IllegalArgumentException("Storage policy name is empty.");
       }
-      np.setStoragePolicy(fullpath, policyName.getValue());
+      cp.setStoragePolicy(fullpath, policyName.getValue());
       return Response.ok().type(MediaType.APPLICATION_OCTET_STREAM).build();
     }
     default:
@@ -812,12 +824,12 @@ public class NamenodeWebHdfsMethods {
       final NewLengthParam newLength,
       final NoRedirectParam noredirectParam
       ) throws IOException, URISyntaxException {
-    final NameNode namenode = (NameNode)context.getAttribute("name.node");
-    final NamenodeProtocols np = getRPCServer(namenode);
+    final ClientProtocol cp = getRpcClientProtocol();
 
     switch(op.getValue()) {
     case APPEND:
     {
+      final NameNode namenode = (NameNode)context.getAttribute("name.node");
       final URI uri = redirectURI(namenode, ugi, delegation, username,
           doAsUser, fullpath, op.getValue(), -1L, -1L,
           excludeDatanodes.getValue(), bufferSize);
@@ -832,20 +844,20 @@ public class NamenodeWebHdfsMethods {
     case CONCAT:
     {
       validateOpParams(op, concatSrcs);
-      np.concat(fullpath, concatSrcs.getAbsolutePaths());
+      cp.concat(fullpath, concatSrcs.getAbsolutePaths());
       return Response.ok().build();
     }
     case TRUNCATE:
     {
       validateOpParams(op, newLength);
       // We treat each rest request as a separate client.
-      final boolean b = np.truncate(fullpath, newLength.getValue(),
+      final boolean b = cp.truncate(fullpath, newLength.getValue(),
           "DFSClient_" + DFSUtil.getSecureRandom().nextLong());
       final String js = JsonUtil.toJsonString("boolean", b);
       return Response.ok(js).type(MediaType.APPLICATION_JSON).build();
     }
     case UNSETSTORAGEPOLICY: {
-      np.unsetStoragePolicy(fullpath);
+      cp.unsetStoragePolicy(fullpath);
       return Response.ok().build();
     }
     default:
@@ -975,14 +987,14 @@ public class NamenodeWebHdfsMethods {
       final NoRedirectParam noredirectParam,
       final StartAfterParam startAfter
       ) throws IOException, URISyntaxException {
-    final NameNode namenode = (NameNode)context.getAttribute("name.node");
     final Configuration conf = (Configuration) context
         .getAttribute(JspHelper.CURRENT_CONF);
-    final NamenodeProtocols np = getRPCServer(namenode);
+    final ClientProtocol cp = getRpcClientProtocol();
 
     switch(op.getValue()) {
     case OPEN:
     {
+      final NameNode namenode = (NameNode)context.getAttribute("name.node");
       final URI uri = redirectURI(namenode, ugi, delegation, username,
           doAsUser, fullpath, op.getValue(), offset.getValue(), -1L,
           excludeDatanodes.getValue(), offset, length, bufferSize);
@@ -998,14 +1010,14 @@ public class NamenodeWebHdfsMethods {
     {
       final long offsetValue = offset.getValue();
       final Long lengthValue = length.getValue();
-      final LocatedBlocks locatedblocks = np.getBlockLocations(fullpath,
+      final LocatedBlocks locatedblocks = cp.getBlockLocations(fullpath,
           offsetValue, lengthValue != null? lengthValue: Long.MAX_VALUE);
       final String js = JsonUtil.toJsonString(locatedblocks);
       return Response.ok(js).type(MediaType.APPLICATION_JSON).build();
     }
     case GETFILESTATUS:
     {
-      final HdfsFileStatus status = np.getFileInfo(fullpath);
+      final HdfsFileStatus status = cp.getFileInfo(fullpath);
       if (status == null) {
         throw new FileNotFoundException("File does not exist: " + fullpath);
       }
@@ -1015,17 +1027,18 @@ public class NamenodeWebHdfsMethods {
     }
     case LISTSTATUS:
     {
-      final StreamingOutput streaming = getListingStream(np, fullpath);
+      final StreamingOutput streaming = getListingStream(cp, fullpath);
       return Response.ok(streaming).type(MediaType.APPLICATION_JSON).build();
     }
     case GETCONTENTSUMMARY:
     {
-      final ContentSummary contentsummary = np.getContentSummary(fullpath);
+      final ContentSummary contentsummary = cp.getContentSummary(fullpath);
       final String js = JsonUtil.toJsonString(contentsummary);
       return Response.ok(js).type(MediaType.APPLICATION_JSON).build();
     }
     case GETFILECHECKSUM:
     {
+      final NameNode namenode = (NameNode)context.getAttribute("name.node");
       final URI uri = redirectURI(namenode, ugi, delegation, username, doAsUser,
           fullpath, op.getValue(), -1L, -1L, null);
       if(!noredirectParam.getValue()) {
@@ -1042,6 +1055,7 @@ public class NamenodeWebHdfsMethods {
         throw new IllegalArgumentException(delegation.getName()
             + " parameter is not null.");
       }
+      final NameNode namenode = (NameNode)context.getAttribute("name.node");
       final Token<? extends TokenIdentifier> token = generateDelegationToken(
           namenode, ugi, renewer.getValue());
 
@@ -1063,7 +1077,7 @@ public class NamenodeWebHdfsMethods {
       return Response.ok(js).type(MediaType.APPLICATION_JSON).build();
     }
     case GETACLSTATUS: {
-      AclStatus status = np.getAclStatus(fullpath);
+      AclStatus status = cp.getAclStatus(fullpath);
       if (status == null) {
         throw new FileNotFoundException("File does not exist: " + fullpath);
       }
@@ -1082,20 +1096,20 @@ public class NamenodeWebHdfsMethods {
           }
         }
       }
-      List<XAttr> xAttrs = np.getXAttrs(fullpath, (names != null && 
+      List<XAttr> xAttrs = cp.getXAttrs(fullpath, (names != null &&
           !names.isEmpty()) ? XAttrHelper.buildXAttrs(names) : null);
       final String js = JsonUtil.toJsonString(xAttrs,
           xattrEncoding.getEncoding());
       return Response.ok(js).type(MediaType.APPLICATION_JSON).build();
     }
     case LISTXATTRS: {
-      final List<XAttr> xAttrs = np.listXAttrs(fullpath);
+      final List<XAttr> xAttrs = cp.listXAttrs(fullpath);
       final String js = JsonUtil.toJsonString(xAttrs);
       return Response.ok(js).type(MediaType.APPLICATION_JSON).build();
     }
     case CHECKACCESS: {
       validateOpParams(op, fsAction);
-      np.checkAccess(fullpath, FsAction.getFsAction(fsAction.getValue()));
+      cp.checkAccess(fullpath, FsAction.getFsAction(fsAction.getValue()));
       return Response.ok().build();
     }
     case GETTRASHROOT: {
@@ -1109,17 +1123,17 @@ public class NamenodeWebHdfsMethods {
       if (startAfter != null && startAfter.getValue() != null) {
         start = startAfter.getValue().getBytes(Charsets.UTF_8);
       }
-      final DirectoryListing listing = getDirectoryListing(np, fullpath, start);
+      final DirectoryListing listing = getDirectoryListing(cp, fullpath, start);
       final String js = JsonUtil.toJsonString(listing);
       return Response.ok(js).type(MediaType.APPLICATION_JSON).build();
     }
     case GETALLSTORAGEPOLICY: {
-      BlockStoragePolicy[] storagePolicies = np.getStoragePolicies();
+      BlockStoragePolicy[] storagePolicies = cp.getStoragePolicies();
       final String js = JsonUtil.toJsonString(storagePolicies);
       return Response.ok(js).type(MediaType.APPLICATION_JSON).build();
     }
     case GETSTORAGEPOLICY: {
-      BlockStoragePolicy storagePolicy = np.getStoragePolicy(fullpath);
+      BlockStoragePolicy storagePolicy = cp.getStoragePolicy(fullpath);
       final String js = JsonUtil.toJsonString(storagePolicy);
       return Response.ok(js).type(MediaType.APPLICATION_JSON).build();
     }
@@ -1127,7 +1141,7 @@ public class NamenodeWebHdfsMethods {
       // Since none of the server defaults values are hot reloaded, we can
       // cache the output of serverDefaults.
       if (serverDefaultsResponse == null) {
-        FsServerDefaults serverDefaults = np.getServerDefaults();
+        FsServerDefaults serverDefaults = cp.getServerDefaults();
         serverDefaultsResponse = JsonUtil.toJsonString(serverDefaults);
       }
       return Response.ok(serverDefaultsResponse)
@@ -1154,21 +1168,21 @@ public class NamenodeWebHdfsMethods {
         new org.apache.hadoop.fs.Path(fullPath)).toUri().getPath();
   }
 
-  private static DirectoryListing getDirectoryListing(final NamenodeProtocols np,
+  private static DirectoryListing getDirectoryListing(final ClientProtocol cp,
       final String p, byte[] startAfter) throws IOException {
-    final DirectoryListing listing = np.getListing(p, startAfter, false);
+    final DirectoryListing listing = cp.getListing(p, startAfter, false);
     if (listing == null) { // the directory does not exist
       throw new FileNotFoundException("File " + p + " does not exist.");
     }
     return listing;
   }
   
-  private static StreamingOutput getListingStream(final NamenodeProtocols np, 
+  private static StreamingOutput getListingStream(final ClientProtocol cp,
       final String p) throws IOException {
     // allows exceptions like FNF or ACE to prevent http response of 200 for
     // a failure since we can't (currently) return error responses in the
     // middle of a streaming operation
-    final DirectoryListing firstDirList = getDirectoryListing(np, p,
+    final DirectoryListing firstDirList = getDirectoryListing(cp, p,
         HdfsFileStatus.EMPTY_NAME);
 
     // must save ugi because the streaming object will be executed outside
@@ -1189,7 +1203,7 @@ public class NamenodeWebHdfsMethods {
             public Void run() throws IOException {
               long n = 0;
               for (DirectoryListing dirList = firstDirList; ;
-                   dirList = getDirectoryListing(np, p, dirList.getLastName())
+                   dirList = getDirectoryListing(cp, p, dirList.getLastName())
               ) {
                 // send each segment of the directory listing
                 for (HdfsFileStatus s : dirList.getPartialListing()) {
@@ -1282,18 +1296,17 @@ public class NamenodeWebHdfsMethods {
       final RecursiveParam recursive,
       final SnapshotNameParam snapshotName
       ) throws IOException {
-    final NameNode namenode = (NameNode)context.getAttribute("name.node");
-    final NamenodeProtocols np = getRPCServer(namenode);
+    final ClientProtocol cp = getRpcClientProtocol();
 
     switch(op.getValue()) {
     case DELETE: {
-      final boolean b = np.delete(fullpath, recursive.getValue());
+      final boolean b = cp.delete(fullpath, recursive.getValue());
       final String js = JsonUtil.toJsonString("boolean", b);
       return Response.ok(js).type(MediaType.APPLICATION_JSON).build();
     }
     case DELETESNAPSHOT: {
       validateOpParams(op, snapshotName);
-      np.deleteSnapshot(fullpath, snapshotName.getValue());
+      cp.deleteSnapshot(fullpath, snapshotName.getValue());
       return Response.ok().type(MediaType.APPLICATION_OCTET_STREAM).build();
     }
     default:

+ 18 - 5
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java

@@ -464,7 +464,7 @@ public class DFSAdmin extends FsShell {
     "\t[-getDatanodeInfo <datanode_host:ipc_port>]\n" +
     "\t[-metasave filename]\n" +
     "\t[-triggerBlockReport [-incremental] <datanode_host:ipc_port>]\n" +
-    "\t[-listOpenFiles [-blockingDecommission]]\n" +
+    "\t[-listOpenFiles [-blockingDecommission] [-path <path>]]\n" +
     "\t[-help [cmd]]\n";
 
   /**
@@ -918,16 +918,29 @@ public class DFSAdmin extends FsShell {
    * @param argv
    */
   public int listOpenFiles(String[] argv) throws IOException {
+    String path = null;
     List<OpenFilesType> types = new ArrayList<>();
     if (argv != null) {
       List<String> args = new ArrayList<>(Arrays.asList(argv));
       if (StringUtils.popOption("-blockingDecommission", args)) {
         types.add(OpenFilesType.BLOCKING_DECOMMISSION);
       }
+
+      path = StringUtils.popOptionWithArgument("-path", args);
     }
     if (types.isEmpty()) {
       types.add(OpenFilesType.ALL_OPEN_FILES);
     }
+
+    if (path != null) {
+      path = path.trim();
+      if (path.length() == 0) {
+        path = OpenFilesIterator.FILTER_PATH_DEFAULT;
+      }
+    } else {
+      path = OpenFilesIterator.FILTER_PATH_DEFAULT;
+    }
+
     EnumSet<OpenFilesType> openFilesTypes = EnumSet.copyOf(types);
 
     DistributedFileSystem dfs = getDFS();
@@ -941,9 +954,9 @@ public class DFSAdmin extends FsShell {
           dfsConf, HAUtil.getAddressOfActive(getDFS()), ClientProtocol.class,
           UserGroupInformation.getCurrentUser(), false);
       openFilesRemoteIterator = new OpenFilesIterator(proxy.getProxy(),
-          FsTracer.get(dfsConf), openFilesTypes);
+          FsTracer.get(dfsConf), openFilesTypes, path);
     } else {
-      openFilesRemoteIterator = dfs.listOpenFiles(openFilesTypes);
+      openFilesRemoteIterator = dfs.listOpenFiles(openFilesTypes, path);
     }
     printOpenFiles(openFilesRemoteIterator);
     return 0;
@@ -1982,7 +1995,7 @@ public class DFSAdmin extends FsShell {
           + " [-triggerBlockReport [-incremental] <datanode_host:ipc_port>]");
     } else if ("-listOpenFiles".equals(cmd)) {
       System.err.println("Usage: hdfs dfsadmin"
-          + " [-listOpenFiles [-blockingDecommission]]");
+          + " [-listOpenFiles [-blockingDecommission] [-path <path>]]");
     } else {
       System.err.println("Usage: hdfs dfsadmin");
       System.err.println("Note: Administrative commands can only be run as the HDFS superuser.");
@@ -2137,7 +2150,7 @@ public class DFSAdmin extends FsShell {
         return exitCode;
       }
     } else if ("-listOpenFiles".equals(cmd)) {
-      if ((argv.length != 1) && (argv.length != 2)) {
+      if ((argv.length > 4)) {
         printUsage(cmd);
         return exitCode;
       }

+ 140 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/federation/RouterAdmin.java

@@ -29,10 +29,12 @@ import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.federation.resolver.MountTableManager;
 import org.apache.hadoop.hdfs.server.federation.resolver.RemoteLocation;
 import org.apache.hadoop.hdfs.server.federation.resolver.order.DestinationOrder;
 import org.apache.hadoop.hdfs.server.federation.router.RouterClient;
+import org.apache.hadoop.hdfs.server.federation.router.RouterQuotaUsage;
 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;
@@ -45,6 +47,7 @@ import org.apache.hadoop.hdfs.server.federation.store.records.MountTable;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
 import org.slf4j.Logger;
@@ -80,7 +83,10 @@ public class RouterAdmin extends Configured implements Tool {
         + "\t[-add <source> <nameservice> <destination> "
         + "[-readonly] -owner <owner> -group <group> -mode <mode>]\n"
         + "\t[-rm <source>]\n"
-        + "\t[-ls <path>]\n";
+        + "\t[-ls <path>]\n"
+        + "\t[-setQuota <path> -nsQuota <nsQuota> -ssQuota "
+        + "<quota in bytes or quota size string>]\n"
+        + "\t[-clrQuota <path>\n";
     System.out.println(usage);
   }
 
@@ -109,6 +115,18 @@ public class RouterAdmin extends Configured implements Tool {
         printUsage();
         return exitCode;
       }
+    } else if ("-setQuota".equalsIgnoreCase(cmd)) {
+      if (argv.length < 4) {
+        System.err.println("Not enough parameters specificed for cmd " + cmd);
+        printUsage();
+        return exitCode;
+      }
+    } else if ("-clrQuota".equalsIgnoreCase(cmd)) {
+      if (argv.length < 2) {
+        System.err.println("Not enough parameters specificed for cmd " + cmd);
+        printUsage();
+        return exitCode;
+      }
     }
 
     // Initialize RouterClient
@@ -144,6 +162,16 @@ public class RouterAdmin extends Configured implements Tool {
         } else {
           listMounts("/");
         }
+      } else if ("-setQuota".equals(cmd)) {
+        if (setQuota(argv, i)) {
+          System.out.println(
+              "Successfully set quota for mount point " + argv[i]);
+        }
+      } else if ("-clrQuota".equals(cmd)) {
+        if (clrQuota(argv[i])) {
+          System.out.println(
+              "Successfully clear quota for mount point " + argv[i]);
+        }
       } else {
         printUsage();
         return exitCode;
@@ -369,8 +397,8 @@ public class RouterAdmin extends Configured implements Tool {
   private static void printMounts(List<MountTable> entries) {
     System.out.println("Mount Table Entries:");
     System.out.println(String.format(
-        "%-25s %-25s %-25s %-25s %-25s",
-        "Source", "Destinations", "Owner", "Group", "Mode"));
+        "%-25s %-25s %-25s %-25s %-25s %-25s",
+        "Source", "Destinations", "Owner", "Group", "Mode", "Quota/Usage"));
     for (MountTable entry : entries) {
       StringBuilder destBuilder = new StringBuilder();
       for (RemoteLocation location : entry.getDestinations()) {
@@ -383,9 +411,117 @@ public class RouterAdmin extends Configured implements Tool {
       System.out.print(String.format("%-25s %-25s", entry.getSourcePath(),
           destBuilder.toString()));
 
-      System.out.println(String.format(" %-25s %-25s %-25s",
+      System.out.print(String.format(" %-25s %-25s %-25s",
           entry.getOwnerName(), entry.getGroupName(), entry.getMode()));
+
+      System.out.println(String.format(" %-25s", entry.getQuota()));
+    }
+  }
+
+  /**
+   * Set quota for a mount table entry.
+   *
+   * @param parameters Parameters of the quota.
+   * @param i Index in the parameters.
+   */
+  private boolean setQuota(String[] parameters, int i) throws IOException {
+    long nsQuota = HdfsConstants.QUOTA_DONT_SET;
+    long ssQuota = HdfsConstants.QUOTA_DONT_SET;
+
+    String mount = parameters[i++];
+    while (i < parameters.length) {
+      if (parameters[i].equals("-nsQuota")) {
+        i++;
+        try {
+          nsQuota = Long.parseLong(parameters[i]);
+        } catch (Exception e) {
+          System.err.println("Cannot parse nsQuota: " + parameters[i]);
+        }
+      } else if (parameters[i].equals("-ssQuota")) {
+        i++;
+        try {
+          ssQuota = StringUtils.TraditionalBinaryPrefix
+              .string2long(parameters[i]);
+        } catch (Exception e) {
+          System.err.println("Cannot parse ssQuota: " + parameters[i]);
+        }
+      }
+
+      i++;
     }
+
+    if (nsQuota <= 0 || ssQuota <= 0) {
+      System.err.println("Input quota value should be a positive number.");
+      return false;
+    }
+
+    return updateQuota(mount, nsQuota, ssQuota);
+  }
+
+  /**
+   * Clear quota of the mount point.
+   *
+   * @param mount Mount table to clear
+   * @return If the quota was cleared.
+   * @throws IOException Error clearing the mount point.
+   */
+  private boolean clrQuota(String mount) throws IOException {
+    return updateQuota(mount, HdfsConstants.QUOTA_DONT_SET,
+        HdfsConstants.QUOTA_DONT_SET);
+  }
+
+  /**
+   * Update quota of specified mount table.
+   *
+   * @param mount Specified mount table to update.
+   * @param nsQuota Namespace quota.
+   * @param ssQuota Storage space quota.
+   * @return If the quota was updated.
+   * @throws IOException Error updating quota.
+   */
+  private boolean updateQuota(String mount, long nsQuota, long ssQuota)
+      throws IOException {
+    // Get existing entry
+    MountTableManager mountTable = client.getMountTableManager();
+    GetMountTableEntriesRequest getRequest = GetMountTableEntriesRequest
+        .newInstance(mount);
+    GetMountTableEntriesResponse getResponse = mountTable
+        .getMountTableEntries(getRequest);
+    List<MountTable> results = getResponse.getEntries();
+    MountTable existingEntry = null;
+    for (MountTable result : results) {
+      if (mount.equals(result.getSourcePath())) {
+        existingEntry = result;
+        break;
+      }
+    }
+
+    if (existingEntry == null) {
+      return false;
+    } else {
+      long nsCount = existingEntry.getQuota().getFileAndDirectoryCount();
+      long ssCount = existingEntry.getQuota().getSpaceConsumed();
+      // If nsQuota or ssQuota was unset, reset corresponding usage
+      // value to zero.
+      if (nsQuota == HdfsConstants.QUOTA_DONT_SET) {
+        nsCount = RouterQuotaUsage.QUOTA_USAGE_COUNT_DEFAULT;
+      }
+
+      if (nsQuota == HdfsConstants.QUOTA_DONT_SET) {
+        ssCount = RouterQuotaUsage.QUOTA_USAGE_COUNT_DEFAULT;
+      }
+
+      RouterQuotaUsage updatedQuota = new RouterQuotaUsage.Builder()
+          .fileAndDirectoryCount(nsCount).quota(nsQuota)
+          .spaceConsumed(ssCount).spaceQuota(ssQuota).build();
+      existingEntry.setQuota(updatedQuota);
+    }
+
+    UpdateMountTableEntryRequest updateRequest =
+        UpdateMountTableEntryRequest.newInstance(existingEntry);
+    UpdateMountTableEntryResponse updateResponse = mountTable
+        .updateMountTableEntry(updateRequest);
+    return updateResponse.getStatus();
   }
 
   /**

+ 47 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/proto/FederationProtocol.proto

@@ -22,6 +22,7 @@ option java_generic_services = true;
 option java_generate_equals_and_hash = true;
 package hadoop.hdfs;
 
+import "hdfs.proto";
 
 /////////////////////////////////////////////////
 // Membership
@@ -134,6 +135,8 @@ message MountTableRecordProto {
   optional string ownerName = 10;
   optional string groupName = 11;
   optional int32 mode = 12;
+
+  optional QuotaUsageProto quota = 13;
 }
 
 message AddMountTableEntryRequestProto {
@@ -169,3 +172,47 @@ message GetMountTableEntriesResponseProto {
   optional uint64 timestamp = 2;
 }
 
+
+/////////////////////////////////////////////////
+// Routers
+/////////////////////////////////////////////////
+
+message StateStoreVersionRecordProto {
+  optional uint64 membershipVersion = 1;
+  optional uint64 mountTableVersion = 2;
+}
+
+message RouterRecordProto {
+  optional uint64 dateCreated = 1;
+  optional uint64 dateModified = 2;
+  optional string address = 3;
+  optional string status = 4;
+  optional StateStoreVersionRecordProto stateStoreVersion = 5;
+  optional string buildVersion = 6;
+  optional string compileInfo = 7;
+  optional uint64 dateStarted = 8;
+}
+
+message GetRouterRegistrationRequestProto {
+  optional string routerId = 1;
+}
+
+message GetRouterRegistrationResponseProto {
+  optional RouterRecordProto router = 1;
+}
+
+message GetRouterRegistrationsRequestProto {
+}
+
+message GetRouterRegistrationsResponseProto {
+  optional uint64 timestamp = 1;
+  repeated RouterRecordProto routers = 2;
+}
+
+message RouterHeartbeatRequestProto {
+  optional RouterRecordProto router = 1;
+}
+
+message RouterHeartbeatResponseProto {
+  optional bool status = 1;
+}

+ 50 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/proto/InterQJournalProtocol.proto

@@ -0,0 +1,50 @@
+/**
+ * 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.
+ */
+
+/**
+ * These .proto interfaces are private and stable.
+ * Please see http://wiki.apache.org/hadoop/Compatibility
+ * for what changes are allowed for a *stable* .proto interface.
+ */
+
+option java_package = "org.apache.hadoop.hdfs.qjournal.protocol";
+option java_outer_classname = "InterQJournalProtocolProtos";
+option java_generic_services = true;
+option java_generate_equals_and_hash = true;
+package hadoop.hdfs.qjournal;
+
+import "HdfsServer.proto";
+import "QJournalProtocol.proto";
+
+message GetEditLogManifestFromJournalRequestProto {
+    required JournalIdProto jid = 1;
+    required uint64 sinceTxId = 2;  // Transaction ID
+    optional bool inProgressOk = 3 [default = false];
+    optional string nameServiceId = 4;
+}
+
+message GetEditLogManifestFromJournalResponseProto {
+    required RemoteEditLogManifestProto manifest = 1;
+    required uint32 httpPort = 2;
+    optional string fromURL = 3;
+}
+
+service InterQJournalProtocolService {
+    rpc getEditLogManifestFromJournal(GetEditLogManifestFromJournalRequestProto)
+    returns (GetEditLogManifestFromJournalResponseProto);
+}

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

@@ -145,6 +145,15 @@
   </description>
 </property>
 
+<property>
+  <name>dfs.datanode.http.internal-proxy.port</name>
+  <value>0</value>
+  <description>
+    The datanode's internal web proxy port.
+    By default it selects a random port available in runtime.
+  </description>
+</property>
+
 <property>
   <name>dfs.datanode.handler.count</name>
   <value>10</value>
@@ -5117,4 +5126,33 @@
     </description>
   </property>
 
+  <property>
+    <name>dfs.federation.router.mount-table.max-cache-size</name>
+    <value>10000</value>
+    <description>
+      Maximum number of mount table cache entries to have.
+      By default, remove cache entries if we have more than 10k.
+    </description>
+  </property>
+
+  <property>
+    <name>dfs.federation.router.quota.enable</name>
+    <value>false</value>
+    <description>
+      Set to true to enable quota system in Router.
+    </description>
+  </property>
+
+  <property>
+    <name>dfs.federation.router.quota-cache.update.interval</name>
+    <value>60s</value>
+    <description>
+      Interval time for updating quota usage cache in Router.
+      This property is used only if the value of
+      dfs.federation.router.quota.enable is true.
+      This setting supports multiple time unit suffixes as described
+      in dfs.heartbeat.interval. If no suffix is specified then milliseconds
+      is assumed.
+    </description>
+  </property>
 </configuration>

+ 2 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/router/federationhealth.html

@@ -338,6 +338,7 @@
       <th>Owner</th>
       <th>Group</th>
       <th>Permission</th>
+      <th>Quota/Usage</th>
       <th>Date Modified</th>
       <th>Date Created</th>
     </tr>
@@ -353,6 +354,7 @@
       <td>{ownerName}</td>
       <td>{groupName}</td>
       <td>{mode}</td>
+      <td>{quota}</td>
       <td>{dateModified}</td>
       <td>{dateCreated}</td>
     </tr>

+ 6 - 2
hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md

@@ -372,7 +372,7 @@ Usage:
         hdfs dfsadmin [-getDatanodeInfo <datanode_host:ipc_port>]
         hdfs dfsadmin [-metasave filename]
         hdfs dfsadmin [-triggerBlockReport [-incremental] <datanode_host:ipc_port>]
-        hdfs dfsadmin [-listOpenFiles]
+        hdfs dfsadmin [-listOpenFiles [-blockingDecommission] [-path <path>]]
         hdfs dfsadmin [-help [cmd]]
 
 | COMMAND\_OPTION | Description |
@@ -409,7 +409,7 @@ Usage:
 | `-getDatanodeInfo` \<datanode\_host:ipc\_port\> | Get the information about the given datanode. See [Rolling Upgrade document](./HdfsRollingUpgrade.html#dfsadmin_-getDatanodeInfo) for the detail. |
 | `-metasave` filename | Save Namenode's primary data structures to *filename* in the directory specified by hadoop.log.dir property. *filename* is overwritten if it exists. *filename* will contain one line for each of the following<br/>1. Datanodes heart beating with Namenode<br/>2. Blocks waiting to be replicated<br/>3. Blocks currently being replicated<br/>4. Blocks waiting to be deleted |
 | `-triggerBlockReport` `[-incremental]` \<datanode\_host:ipc\_port\> | Trigger a block report for the given datanode. If 'incremental' is specified, it will be otherwise, it will be a full block report. |
-| `-listOpenFiles` `[-blockingDecommission]` | List all open files currently managed by the NameNode along with client name and client machine accessing them. |
+| `-listOpenFiles` `[-blockingDecommission]` `[-path <path>]` | List all open files currently managed by the NameNode along with client name and client machine accessing them. Open files list will be filtered by given type and path. |
 | `-help` [cmd] | Displays help for the given command or all commands if none is specified. |
 
 Runs a HDFS dfsadmin client.
@@ -428,12 +428,16 @@ Usage:
           [-add <source> <nameservice> <destination> [-readonly] -owner <owner> -group <group> -mode <mode>]
           [-rm <source>]
           [-ls <path>]
+          [-setQuota <path> -nsQuota <nsQuota> -ssQuota <quota in bytes or quota size string>]
+          [-clrQuota <path>]
 
 | COMMAND\_OPTION | Description |
 |:---- |:---- |
 | `-add` *source* *nameservice* *destination* | Add a mount table entry or update if it exists. |
 | `-rm` *source* | Remove mount point of specified path. |
 | `-ls` *path* | List mount points under specified path. |
+| `-setQuota` *path* `-nsQuota` *nsQuota* `-ssQuota` *ssQuota* | Set quota for specified path. See [HDFS Quotas Guide](./HdfsQuotaAdminGuide.html) for the quota detail. |
+| `-clrQuota` *path* | Clear quota of given mount point. See [HDFS Quotas Guide](./HdfsQuotaAdminGuide.html) for the quota detail. |
 
 The commands for managing Router-based federation. See [Mount table management](./HDFSRouterFederation.html#Mount_table_management) for more info.
 

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSHighAvailabilityWithNFS.md

@@ -487,7 +487,7 @@ In order to secure the information in ZooKeeper, first add the following to your
        <value>@/path/to/zk-acl.txt</value>
      </property>
 
-Please note the '@' character in these values -- this specifies that the configurations are not inline, but rather point to a file on disk.
+Please note the '@' character in these values -- this specifies that the configurations are not inline, but rather point to a file on disk. The authentication info may also be read via a CredentialProvider (pls see the CredentialProviderAPI Guide in the hadoop-common project).
 
 The first configured file specifies a list of ZooKeeper authentications, in the same format as used by the ZK CLI. For example, you may specify something like:
 

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSHighAvailabilityWithQJM.md

@@ -535,7 +535,7 @@ In order to secure the information in ZooKeeper, first add the following to your
        <value>@/path/to/zk-acl.txt</value>
      </property>
 
-Please note the '@' character in these values -- this specifies that the configurations are not inline, but rather point to a file on disk.
+Please note the '@' character in these values -- this specifies that the configurations are not inline, but rather point to a file on disk. The authentication info may also be read via a CredentialProvider (pls see the CredentialProviderAPI Guide in the hadoop-common project).
 
 The first configured file specifies a list of ZooKeeper authentications, in the same format as used by the ZK CLI. For example, you may specify something like:
 

+ 29 - 0
hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSRouterFederation.md

@@ -127,6 +127,11 @@ Examples users may encounter include the following.
 * Copy file/folder in two different nameservices.
 * Write into a file/folder being rebalanced.
 
+### Quota management
+Federation supports and controls global quota at mount table level.
+For performance reasons, the Router caches the quota usage and updates it periodically. These quota usage values
+will be used for quota-verification during each WRITE RPC call invoked in RouterRPCSever. See [HDFS Quotas Guide](./HdfsQuotaAdminGuide.html)
+for the quota detail.
 
 ### State Store
 The (logically centralized, but physically distributed) State Store maintains:
@@ -199,6 +204,21 @@ Mount table permission can be set by following command:
 
 The option mode is UNIX-style permissions for the mount table. Permissions are specified in octal, e.g. 0755. By default, this is set to 0755.
 
+Router-based federation supports global quota at mount table level. Mount table entries may spread multiple subclusters and the global quota will be
+accounted across these subclusters.
+
+The federation admin tool supports setting quotas for specified mount table entries:
+
+    [hdfs]$ $HADOOP_HOME/bin/hdfs dfsrouteradmin -setQuota /path -nsQuota 100 -ssQuota 1024
+
+The above command means that we allow the path to have a maximum of 100 file/directories and use at most 1024 bytes storage space. The parameter for *ssQuota*
+supports multiple size-unit suffix (e.g. 1k is 1KB, 5m is 5MB). If no suffix is specified then bytes is assumed.
+
+Ls command will show below information for each mount table entry:
+
+    Source                    Destinations              Owner                     Group                     Mode                      Quota/Usage
+    /path                     ns0->/path                root                      supergroup                rwxr-xr-x                 [NsQuota: 50/0, SsQuota: 100 B/0 B]
+
 Client configuration
 --------------------
 
@@ -322,6 +342,15 @@ Monitor the namenodes in the subclusters for forwarding the client requests.
 | dfs.federation.router.monitor.namenode | | The identifier of the namenodes to monitor and heartbeat. |
 | dfs.federation.router.monitor.localnamenode.enable | `true` | If `true`, the Router should monitor the namenode in the local machine. |
 
+### Quota
+
+Global quota supported in federation.
+
+| Property | Default | Description|
+|:---- |:---- |:---- |
+| dfs.federation.router.quota.enable | `false` | If `true`, the quota system enabled in the Router. |
+| dfs.federation.router.quota-cache.update.interval | 60s | How often the Router updates quota cache. This setting supports multiple time unit suffixes. If no suffix is specified then milliseconds is assumed. |
+
 Metrics
 -------
 

+ 4 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/StripedFileTestUtil.java

@@ -363,11 +363,12 @@ public class StripedFileTestUtil {
     List<List<LocatedBlock>> blockGroupList = new ArrayList<>();
     LocatedBlocks lbs = dfs.getClient().getLocatedBlocks(srcPath.toString(), 0L,
         Long.MAX_VALUE);
-    int expectedNumGroup = 0;
+
     if (length > 0) {
-      expectedNumGroup = (length - 1) / blkGroupSize + 1;
+      int expectedNumGroup = (length - 1) / blkGroupSize + 1;
+
+      assertEquals(expectedNumGroup, lbs.getLocatedBlocks().size());
     }
-    assertEquals(expectedNumGroup, lbs.getLocatedBlocks().size());
 
     final ErasureCodingPolicy ecPolicy = dfs.getErasureCodingPolicy(srcPath);
     final int cellSize = ecPolicy.getCellSize();

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

@@ -710,13 +710,49 @@ public class TestDecommission extends AdminStatesBaseTest {
         @Override
         public Boolean get() {
           try {
+            boolean result1 = false;
+            boolean result2 = false;
             toolOut.reset();
             assertEquals(0, ToolRunner.run(dfsAdmin,
                 new String[]{"-listOpenFiles", "-blockingDecommission"}));
             toolOut.flush();
-            return verifyOpenFilesListing(
+            result1 = verifyOpenFilesListing(
                 "dfsadmin -listOpenFiles -blockingDecommission",
                 closedFileSet, openFilesMap, toolOut, maxOpenFiles);
+
+            // test -blockingDecommission with option -path
+            if (openFilesMap.size() > 0) {
+              String firstOpenFile = null;
+              // Construct a new open-file and close-file map.
+              // Pick the first open file into new open-file map, remaining
+              //  open files move into close-files map.
+              HashMap<Path, FSDataOutputStream> newOpenFilesMap =
+                  new HashMap<>();
+              HashSet<Path> newClosedFileSet = new HashSet<>();
+              for (Map.Entry<Path, FSDataOutputStream> entry : openFilesMap
+                  .entrySet()) {
+                if (firstOpenFile == null) {
+                  newOpenFilesMap.put(entry.getKey(), entry.getValue());
+                  firstOpenFile = entry.getKey().toString();
+                } else {
+                  newClosedFileSet.add(entry.getKey());
+                }
+              }
+
+              toolOut.reset();
+              assertEquals(0,
+                  ToolRunner.run(dfsAdmin, new String[] {"-listOpenFiles",
+                      "-blockingDecommission", "-path", firstOpenFile}));
+              toolOut.flush();
+              result2 = verifyOpenFilesListing(
+                  "dfsadmin -listOpenFiles -blockingDecommission -path"
+                      + firstOpenFile,
+                  newClosedFileSet, newOpenFilesMap, toolOut, 1);
+            } else {
+              result2 = true;
+            }
+
+            return result1 && result2;
           } catch (Exception e) {
             LOG.warn("Unexpected exception: " + e);
           }

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

@@ -42,6 +42,7 @@ import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.hdfs.client.HdfsAdmin;
 import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.protocol.OpenFileEntry;
+import org.apache.hadoop.hdfs.protocol.OpenFilesIterator;
 import org.apache.hadoop.hdfs.protocol.OpenFilesIterator.OpenFilesType;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.junit.After;
@@ -256,7 +257,8 @@ public class TestHdfsAdmin {
     HdfsAdmin hdfsAdmin = new HdfsAdmin(FileSystem.getDefaultUri(conf), conf);
     HashSet<Path> openFiles = new HashSet<>(openFileMap.keySet());
     RemoteIterator<OpenFileEntry> openFilesRemoteItr =
-        hdfsAdmin.listOpenFiles(EnumSet.of(OpenFilesType.ALL_OPEN_FILES));
+        hdfsAdmin.listOpenFiles(EnumSet.of(OpenFilesType.ALL_OPEN_FILES),
+            OpenFilesIterator.FILTER_PATH_DEFAULT);
     while (openFilesRemoteItr.hasNext()) {
       String filePath = openFilesRemoteItr.next().getFilePath();
       assertFalse(filePath + " should not be listed under open files!",

Algunos archivos no se mostraron porque demasiados archivos cambiaron en este cambio