You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by in...@apache.org on 2019/03/03 18:36:44 UTC

[hadoop] 05/45: HDFS-12284. RBF: Support for Kerberos authentication. Contributed by Sherwood Zheng and Inigo Goiri.

This is an automated email from the ASF dual-hosted git repository.

inigoiri pushed a commit to branch HDFS-13891
in repository https://gitbox.apache.org/repos/asf/hadoop.git

commit 4519935620f507ebcf56454f99caf9d871fde52b
Author: Brahma Reddy Battula <br...@apache.org>
AuthorDate: Wed Nov 7 07:33:37 2018 +0530

    HDFS-12284. RBF: Support for Kerberos authentication. Contributed by Sherwood Zheng and Inigo Goiri.
---
 hadoop-hdfs-project/hadoop-hdfs-rbf/pom.xml        |  10 ++
 .../server/federation/router/RBFConfigKeys.java    |  11 ++
 .../hdfs/server/federation/router/Router.java      |  28 ++++
 .../federation/router/RouterAdminServer.java       |   7 +
 .../server/federation/router/RouterHttpServer.java |   5 +-
 .../server/federation/router/RouterRpcClient.java  |   9 +-
 .../server/federation/router/RouterRpcServer.java  |  12 ++
 .../src/main/resources/hdfs-rbf-default.xml        |  47 +++++++
 .../fs/contract/router/RouterHDFSContract.java     |   9 +-
 .../fs/contract/router/SecurityConfUtil.java       | 156 +++++++++++++++++++++
 .../router/TestRouterHDFSContractAppendSecure.java |  46 ++++++
 .../router/TestRouterHDFSContractConcatSecure.java |  51 +++++++
 .../router/TestRouterHDFSContractCreateSecure.java |  48 +++++++
 .../router/TestRouterHDFSContractDeleteSecure.java |  46 ++++++
 .../TestRouterHDFSContractGetFileStatusSecure.java |  47 +++++++
 .../router/TestRouterHDFSContractMkdirSecure.java  |  48 +++++++
 .../router/TestRouterHDFSContractOpenSecure.java   |  47 +++++++
 .../router/TestRouterHDFSContractRenameSecure.java |  48 +++++++
 .../TestRouterHDFSContractRootDirectorySecure.java |  63 +++++++++
 .../router/TestRouterHDFSContractSeekSecure.java   |  48 +++++++
 .../TestRouterHDFSContractSetTimesSecure.java      |  48 +++++++
 .../server/federation/MiniRouterDFSCluster.java    |  58 +++++++-
 22 files changed, 879 insertions(+), 13 deletions(-)

diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/pom.xml b/hadoop-hdfs-project/hadoop-hdfs-rbf/pom.xml
index 6886f00..f38205a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-rbf/pom.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/pom.xml
@@ -35,6 +35,16 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
 
   <dependencies>
     <dependency>
+      <groupId>org.bouncycastle</groupId>
+      <artifactId>bcprov-jdk16</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-minikdc</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
       <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-common</artifactId>
       <scope>provided</scope>
diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RBFConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RBFConfigKeys.java
index bbd4250..fa474f4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RBFConfigKeys.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RBFConfigKeys.java
@@ -242,4 +242,15 @@ public class RBFConfigKeys extends CommonConfigurationKeysPublic {
       FEDERATION_ROUTER_PREFIX + "quota-cache.update.interval";
   public static final long DFS_ROUTER_QUOTA_CACHE_UPATE_INTERVAL_DEFAULT =
       60000;
+
+  // HDFS Router security
+  public static final String DFS_ROUTER_KEYTAB_FILE_KEY =
+      FEDERATION_ROUTER_PREFIX + "keytab.file";
+  public static final String DFS_ROUTER_KERBEROS_PRINCIPAL_KEY =
+      FEDERATION_ROUTER_PREFIX + "kerberos.principal";
+  public static final String DFS_ROUTER_KERBEROS_PRINCIPAL_HOSTNAME_KEY =
+      FEDERATION_ROUTER_PREFIX + "kerberos.principal.hostname";
+
+  public static final String DFS_ROUTER_KERBEROS_INTERNAL_SPNEGO_PRINCIPAL_KEY =
+      FEDERATION_ROUTER_PREFIX + "kerberos.internal.spnego.principal";
 }
diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/Router.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/Router.java
index 5ddc129..3288273 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/Router.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/Router.java
@@ -17,6 +17,10 @@
  */
 package org.apache.hadoop.hdfs.server.federation.router;
 
+import static org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys.DFS_ROUTER_KERBEROS_PRINCIPAL_HOSTNAME_KEY;
+import static org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys.DFS_ROUTER_KERBEROS_PRINCIPAL_KEY;
+import static org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys.DFS_ROUTER_KEYTAB_FILE_KEY;
+
 import static org.apache.hadoop.hdfs.server.federation.router.FederationUtil.newActiveNamenodeResolver;
 import static org.apache.hadoop.hdfs.server.federation.router.FederationUtil.newFileSubclusterResolver;
 
@@ -41,6 +45,8 @@ import org.apache.hadoop.hdfs.server.federation.store.RouterStore;
 import org.apache.hadoop.hdfs.server.federation.store.StateStoreService;
 import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
 import org.apache.hadoop.metrics2.source.JvmMetrics;
+import org.apache.hadoop.security.SecurityUtil;
+import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.service.CompositeService;
 import org.apache.hadoop.util.JvmPauseMonitor;
 import org.apache.hadoop.util.Time;
@@ -145,6 +151,11 @@ public class Router extends CompositeService {
     this.conf = configuration;
     updateRouterState(RouterServiceState.INITIALIZING);
 
+    // Enable the security for the Router
+    UserGroupInformation.setConfiguration(conf);
+    SecurityUtil.login(conf, DFS_ROUTER_KEYTAB_FILE_KEY,
+        DFS_ROUTER_KERBEROS_PRINCIPAL_KEY, getHostName(conf));
+
     if (conf.getBoolean(
         RBFConfigKeys.DFS_ROUTER_STORE_ENABLE,
         RBFConfigKeys.DFS_ROUTER_STORE_ENABLE_DEFAULT)) {
@@ -246,6 +257,23 @@ public class Router extends CompositeService {
     super.serviceInit(conf);
   }
 
+  /**
+   * Returns the hostname for this Router. If the hostname is not
+   * explicitly configured in the given config, then it is determined.
+   *
+   * @param config configuration
+   * @return the hostname (NB: may not be a FQDN)
+   * @throws UnknownHostException if the hostname cannot be determined
+   */
+  private static String getHostName(Configuration config)
+      throws UnknownHostException {
+    String name = config.get(DFS_ROUTER_KERBEROS_PRINCIPAL_HOSTNAME_KEY);
+    if (name == null) {
+      name = InetAddress.getLocalHost().getHostName();
+    }
+    return name;
+  }
+
   @Override
   protected void serviceStart() throws Exception {
 
diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterAdminServer.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterAdminServer.java
index e7fec9e..f34dc41 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterAdminServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterAdminServer.java
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hdfs.server.federation.router;
 
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHORIZATION;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_PERMISSIONS_ENABLED_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_PERMISSIONS_ENABLED_KEY;
 
@@ -27,6 +28,7 @@ import java.util.Set;
 import com.google.common.base.Preconditions;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.HDFSPolicyProvider;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.proto.RouterProtocolProtos.RouterAdminProtocolService;
@@ -142,6 +144,11 @@ public class RouterAdminServer extends AbstractService
         .setVerbose(false)
         .build();
 
+    // Set service-level authorization security policy
+    if (conf.getBoolean(HADOOP_SECURITY_AUTHORIZATION, false)) {
+      this.adminServer.refreshServiceAcl(conf, new HDFSPolicyProvider());
+    }
+
     // The RPC-server port can be ephemeral... ensure we have the correct info
     InetSocketAddress listenAddress = this.adminServer.getListenerAddress();
     this.adminAddress = new InetSocketAddress(
diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterHttpServer.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterHttpServer.java
index d223e2a..d6a5146 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterHttpServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterHttpServer.java
@@ -20,7 +20,6 @@ package org.apache.hadoop.hdfs.server.federation.router;
 import java.net.InetSocketAddress;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.server.common.JspHelper;
 import org.apache.hadoop.hdfs.server.namenode.NameNodeHttpServer;
@@ -84,8 +83,8 @@ public class RouterHttpServer extends AbstractService {
     String webApp = "router";
     HttpServer2.Builder builder = DFSUtil.httpServerTemplateForNNAndJN(
         this.conf, this.httpAddress, this.httpsAddress, webApp,
-        DFSConfigKeys.DFS_NAMENODE_KERBEROS_INTERNAL_SPNEGO_PRINCIPAL_KEY,
-        DFSConfigKeys.DFS_NAMENODE_KEYTAB_FILE_KEY);
+        RBFConfigKeys.DFS_ROUTER_KERBEROS_INTERNAL_SPNEGO_PRINCIPAL_KEY,
+        RBFConfigKeys.DFS_ROUTER_KEYTAB_FILE_KEY);
 
     this.httpServer = builder.build();
 
diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcClient.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcClient.java
index 34f51ec..a21e980 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcClient.java
@@ -255,7 +255,14 @@ public class RouterRpcClient {
       // for each individual request.
 
       // TODO Add tokens from the federated UGI
-      connection = this.connectionManager.getConnection(ugi, rpcAddress, proto);
+      UserGroupInformation connUGI = ugi;
+      if (UserGroupInformation.isSecurityEnabled()) {
+        UserGroupInformation routerUser = UserGroupInformation.getLoginUser();
+        connUGI = UserGroupInformation.createProxyUser(
+            ugi.getUserName(), routerUser);
+      }
+      connection = this.connectionManager.getConnection(
+          connUGI, rpcAddress, proto);
       LOG.debug("User {} NN {} is using connection {}",
           ugi.getUserName(), rpcAddress, connection);
     } catch (Exception ex) {
diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java
index 36d3c81..fcb35f4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hdfs.server.federation.router;
 
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHORIZATION;
 import static org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys.DFS_ROUTER_HANDLER_COUNT_DEFAULT;
 import static org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys.DFS_ROUTER_HANDLER_COUNT_KEY;
 import static org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys.DFS_ROUTER_HANDLER_QUEUE_SIZE_DEFAULT;
@@ -61,6 +62,7 @@ import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.ha.HAServiceProtocol;
 import org.apache.hadoop.hdfs.AddBlockFlag;
 import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.HDFSPolicyProvider;
 import org.apache.hadoop.hdfs.inotify.EventBatchList;
 import org.apache.hadoop.hdfs.protocol.AddErasureCodingPolicyResponse;
 import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
@@ -175,6 +177,9 @@ public class RouterRpcServer extends AbstractService
   /** Monitor metrics for the RPC calls. */
   private final RouterRpcMonitor rpcMonitor;
 
+  /** If we use authentication for the connections. */
+  private final boolean serviceAuthEnabled;
+
 
   /** Interface to identify the active NN for a nameservice or blockpool ID. */
   private final ActiveNamenodeResolver namenodeResolver;
@@ -266,6 +271,13 @@ public class RouterRpcServer extends AbstractService
     DFSUtil.addPBProtocol(
         conf, NamenodeProtocolPB.class, nnPbService, this.rpcServer);
 
+    // Set service-level authorization security policy
+    this.serviceAuthEnabled = conf.getBoolean(
+        HADOOP_SECURITY_AUTHORIZATION, false);
+    if (this.serviceAuthEnabled) {
+      rpcServer.refreshServiceAcl(conf, new HDFSPolicyProvider());
+    }
+
     // We don't want the server to log the full stack trace for some exceptions
     this.rpcServer.addTerseExceptions(
         RemoteException.class,
diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/resources/hdfs-rbf-default.xml b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/resources/hdfs-rbf-default.xml
index 3f56043..29c3093 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/resources/hdfs-rbf-default.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/resources/hdfs-rbf-default.xml
@@ -465,4 +465,51 @@
     </description>
   </property>
 
+  <property>
+    <name>dfs.federation.router.keytab.file</name>
+    <value></value>
+    <description>
+      The keytab file used by router to login as its
+      service principal. The principal name is configured with
+      dfs.federation.router.kerberos.principal.
+    </description>
+  </property>
+
+  <property>
+    <name>dfs.federation.router.kerberos.principal</name>
+    <value></value>
+    <description>
+      The Router service principal. This is typically set to
+      router/_HOST@REALM.TLD. Each Router will substitute _HOST with its
+      own fully qualified hostname at startup. The _HOST placeholder
+      allows using the same configuration setting on both Router
+      in an HA setup.
+    </description>
+  </property>
+
+  <property>
+    <name>dfs.federation.router.kerberos.principal.hostname</name>
+    <value></value>
+    <description>
+      Optional.  The hostname for the Router containing this
+      configuration file.  Will be different for each machine.
+      Defaults to current hostname.
+    </description>
+  </property>
+
+  <property>
+    <name>dfs.federation.router.kerberos.internal.spnego.principal</name>
+    <value>${dfs.web.authentication.kerberos.principal}</value>
+    <description>
+      The server principal used by the Router for web UI SPNEGO
+      authentication when Kerberos security is enabled. This is
+      typically set to HTTP/_HOST@REALM.TLD The SPNEGO server principal
+      begins with the prefix HTTP/ by convention.
+
+      If the value is '*', the web server will attempt to login with
+      every principal specified in the keytab file
+      dfs.web.authentication.kerberos.keytab.
+    </description>
+  </property>
+
 </configuration>
\ No newline at end of file
diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/fs/contract/router/RouterHDFSContract.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/fs/contract/router/RouterHDFSContract.java
index 97a426e..510cb95 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/fs/contract/router/RouterHDFSContract.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/fs/contract/router/RouterHDFSContract.java
@@ -43,12 +43,17 @@ public class RouterHDFSContract extends HDFSContract {
   }
 
   public static void createCluster() throws IOException {
+    createCluster(null);
+  }
+
+  public static void createCluster(Configuration conf) throws IOException {
     try {
-      cluster = new MiniRouterDFSCluster(true, 2);
+      cluster = new MiniRouterDFSCluster(true, 2, conf);
 
       // Start NNs and DNs and wait until ready
-      cluster.startCluster();
+      cluster.startCluster(conf);
 
+      cluster.addRouterOverrides(conf);
       // Start routers with only an RPC service
       cluster.startRouters();
 
diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/fs/contract/router/SecurityConfUtil.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/fs/contract/router/SecurityConfUtil.java
new file mode 100644
index 0000000..deb6ace
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/fs/contract/router/SecurityConfUtil.java
@@ -0,0 +1,156 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ *   you may not use this file except in compliance with the License.
+ *   You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *   Unless required by applicable law or agreed to in writing, software
+ *   distributed under the License is distributed on an "AS IS" BASIS,
+ *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *   See the License for the specific language governing permissions and
+ *   limitations under the License. See accompanying LICENSE file.
+ */
+
+package org.apache.hadoop.fs.contract.router;
+
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION;
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_SERVICE_USER_NAME_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCK_ACCESS_TOKEN_ENABLE_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_HTTPS_KEYSTORE_RESOURCE_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_HTTPS_ADDRESS_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_KERBEROS_PRINCIPAL_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_KEYTAB_FILE_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HTTP_POLICY_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_HTTPS_ADDRESS_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_KERBEROS_PRINCIPAL_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_KEYTAB_FILE_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_SERVER_HTTPS_KEYSTORE_RESOURCE_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_WEB_AUTHENTICATION_KERBEROS_KEYTAB_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_WEB_AUTHENTICATION_KERBEROS_PRINCIPAL_KEY;
+import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_DATA_TRANSFER_PROTECTION_KEY;
+import static org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys.DFS_ROUTER_KERBEROS_INTERNAL_SPNEGO_PRINCIPAL_KEY;
+import static org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys.DFS_ROUTER_KERBEROS_PRINCIPAL_KEY;
+import static org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys.DFS_ROUTER_KEYTAB_FILE_KEY;
+import static org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys.DFS_ROUTER_RPC_BIND_HOST_KEY;
+import static org.junit.Assert.assertTrue;
+
+import java.io.File;
+import java.util.Properties;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys;
+import org.apache.hadoop.hdfs.server.federation.store.driver.StateStoreDriver;
+import org.apache.hadoop.hdfs.server.federation.store.driver.impl.StateStoreFileImpl;
+import org.apache.hadoop.http.HttpConfig;
+import org.apache.hadoop.minikdc.MiniKdc;
+import org.apache.hadoop.security.SecurityUtil;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.ssl.KeyStoreTestUtil;
+import org.apache.hadoop.test.GenericTestUtils;
+
+/**
+ * Test utility to provide a standard routine to initialize the configuration
+ * for secure RBF HDFS cluster.
+ */
+public final class SecurityConfUtil {
+
+  // SSL keystore
+  private static String keystoresDir;
+  private static String sslConfDir;
+
+  // State string for mini dfs
+  private static final String SPNEGO_USER_NAME = "HTTP";
+  private static final String ROUTER_USER_NAME = "router";
+
+  private static String spnegoPrincipal;
+  private static String routerPrincipal;
+
+  private SecurityConfUtil() {
+    // Utility Class
+  }
+
+  public static Configuration initSecurity() throws Exception {
+    // delete old test dir
+    File baseDir = GenericTestUtils.getTestDir(
+        SecurityConfUtil.class.getSimpleName());
+    FileUtil.fullyDelete(baseDir);
+    assertTrue(baseDir.mkdirs());
+
+    // start a mini kdc with default conf
+    Properties kdcConf = MiniKdc.createConf();
+    MiniKdc kdc = new MiniKdc(kdcConf, baseDir);
+    kdc.start();
+
+    Configuration conf = new HdfsConfiguration();
+    SecurityUtil.setAuthenticationMethod(
+        UserGroupInformation.AuthenticationMethod.KERBEROS, conf);
+
+    UserGroupInformation.setConfiguration(conf);
+    assertTrue("Expected configuration to enable security",
+        UserGroupInformation.isSecurityEnabled());
+
+    // Setup the keytab
+    File keytabFile = new File(baseDir, "test.keytab");
+    String keytab = keytabFile.getAbsolutePath();
+
+    // Windows will not reverse name lookup "127.0.0.1" to "localhost".
+    String krbInstance = Path.WINDOWS ? "127.0.0.1" : "localhost";
+
+    kdc.createPrincipal(keytabFile,
+        SPNEGO_USER_NAME + "/" + krbInstance,
+        ROUTER_USER_NAME + "/" + krbInstance);
+
+    routerPrincipal =
+        ROUTER_USER_NAME + "/" + krbInstance + "@" + kdc.getRealm();
+    spnegoPrincipal =
+        SPNEGO_USER_NAME + "/" + krbInstance + "@" + kdc.getRealm();
+
+    // Set auth configuration for mini DFS
+    conf.set(HADOOP_SECURITY_AUTHENTICATION, "kerberos");
+    conf.set(HADOOP_SECURITY_SERVICE_USER_NAME_KEY, routerPrincipal);
+
+    // Setup principles and keytabs for dfs
+    conf.set(DFS_NAMENODE_KERBEROS_PRINCIPAL_KEY, routerPrincipal);
+    conf.set(DFS_NAMENODE_KEYTAB_FILE_KEY, keytab);
+    conf.set(DFS_DATANODE_KERBEROS_PRINCIPAL_KEY, routerPrincipal);
+    conf.set(DFS_DATANODE_KEYTAB_FILE_KEY, keytab);
+    conf.set(DFS_WEB_AUTHENTICATION_KERBEROS_PRINCIPAL_KEY, spnegoPrincipal);
+    conf.set(DFS_WEB_AUTHENTICATION_KERBEROS_KEYTAB_KEY, keytab);
+
+    conf.set(DFS_NAMENODE_HTTPS_ADDRESS_KEY, "localhost:0");
+    conf.set(DFS_DATANODE_HTTPS_ADDRESS_KEY, "localhost:0");
+
+    conf.setBoolean(DFS_BLOCK_ACCESS_TOKEN_ENABLE_KEY, true);
+    conf.set(DFS_DATA_TRANSFER_PROTECTION_KEY, "authentication");
+    conf.set(DFS_HTTP_POLICY_KEY, HttpConfig.Policy.HTTPS_ONLY.name());
+
+    // Setup SSL configuration
+    keystoresDir = baseDir.getAbsolutePath();
+    sslConfDir = KeyStoreTestUtil.getClasspathDir(
+        SecurityConfUtil.class);
+    KeyStoreTestUtil.setupSSLConfig(
+        keystoresDir, sslConfDir, conf, false);
+    conf.set(DFS_CLIENT_HTTPS_KEYSTORE_RESOURCE_KEY,
+        KeyStoreTestUtil.getClientSSLConfigFileName());
+    conf.set(DFS_SERVER_HTTPS_KEYSTORE_RESOURCE_KEY,
+        KeyStoreTestUtil.getServerSSLConfigFileName());
+
+    // Setup principals and keytabs for router
+    conf.set(DFS_ROUTER_KEYTAB_FILE_KEY, keytab);
+    conf.set(DFS_ROUTER_KERBEROS_PRINCIPAL_KEY, routerPrincipal);
+    conf.set(DFS_ROUTER_KERBEROS_INTERNAL_SPNEGO_PRINCIPAL_KEY, "*");
+
+    // Setup basic state store
+    conf.setClass(RBFConfigKeys.FEDERATION_STORE_DRIVER_CLASS,
+        StateStoreFileImpl.class, StateStoreDriver.class);
+
+    // We need to specify the host to prevent 0.0.0.0 as the host address
+    conf.set(DFS_ROUTER_RPC_BIND_HOST_KEY, "localhost");
+
+    return conf;
+  }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/fs/contract/router/TestRouterHDFSContractAppendSecure.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/fs/contract/router/TestRouterHDFSContractAppendSecure.java
new file mode 100644
index 0000000..fe4951d
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/fs/contract/router/TestRouterHDFSContractAppendSecure.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ *   you may not use this file except in compliance with the License.
+ *   You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *   Unless required by applicable law or agreed to in writing, software
+ *   distributed under the License is distributed on an "AS IS" BASIS,
+ *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *   See the License for the specific language governing permissions and
+ *   limitations under the License. See accompanying LICENSE file.
+ */
+
+package org.apache.hadoop.fs.contract.router;
+
+import java.io.IOException;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractAppendTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+
+import static org.apache.hadoop.fs.contract.router.SecurityConfUtil.initSecurity;
+
+/**
+ * Test secure append operations on the Router-based FS.
+ */
+public class TestRouterHDFSContractAppendSecure
+    extends AbstractContractAppendTest {
+
+  @BeforeClass
+  public static void createCluster() throws Exception {
+    RouterHDFSContract.createCluster(initSecurity());
+  }
+
+  @AfterClass
+  public static void teardownCluster() throws IOException {
+    RouterHDFSContract.destroyCluster();
+  }
+
+  @Override
+  protected AbstractFSContract createContract(Configuration conf) {
+    return new RouterHDFSContract(conf);
+  }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/fs/contract/router/TestRouterHDFSContractConcatSecure.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/fs/contract/router/TestRouterHDFSContractConcatSecure.java
new file mode 100644
index 0000000..c9a0cc8
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/fs/contract/router/TestRouterHDFSContractConcatSecure.java
@@ -0,0 +1,51 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ *   you may not use this file except in compliance with the License.
+ *   You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *   Unless required by applicable law or agreed to in writing, software
+ *   distributed under the License is distributed on an "AS IS" BASIS,
+ *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *   See the License for the specific language governing permissions and
+ *   limitations under the License. See accompanying LICENSE file.
+ */
+
+package org.apache.hadoop.fs.contract.router;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.contract.AbstractContractConcatTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+
+import java.io.IOException;
+
+import static org.apache.hadoop.fs.contract.router.SecurityConfUtil.initSecurity;
+
+
+/**
+ * Test secure concat operations on the Router-based FS.
+ */
+public class TestRouterHDFSContractConcatSecure
+    extends AbstractContractConcatTest {
+
+  @BeforeClass
+  public static void createCluster() throws Exception {
+    RouterHDFSContract.createCluster(initSecurity());
+    // perform a simple operation on the cluster to verify it is up
+    RouterHDFSContract.getFileSystem().getDefaultBlockSize(new Path("/"));
+  }
+
+  @AfterClass
+  public static void teardownCluster() throws IOException {
+    RouterHDFSContract.destroyCluster();
+  }
+
+  @Override
+  protected AbstractFSContract createContract(Configuration conf) {
+    return new RouterHDFSContract(conf);
+  }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/fs/contract/router/TestRouterHDFSContractCreateSecure.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/fs/contract/router/TestRouterHDFSContractCreateSecure.java
new file mode 100644
index 0000000..dc264b0
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/fs/contract/router/TestRouterHDFSContractCreateSecure.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ *   you may not use this file except in compliance with the License.
+ *   You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *   Unless required by applicable law or agreed to in writing, software
+ *   distributed under the License is distributed on an "AS IS" BASIS,
+ *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *   See the License for the specific language governing permissions and
+ *   limitations under the License. See accompanying LICENSE file.
+ */
+
+package org.apache.hadoop.fs.contract.router;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractCreateTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+
+import java.io.IOException;
+
+import static org.apache.hadoop.fs.contract.router.SecurityConfUtil.initSecurity;
+
+
+/**
+ * Test secure create operations on the Router-based FS.
+ */
+public class TestRouterHDFSContractCreateSecure
+    extends AbstractContractCreateTest {
+
+  @BeforeClass
+  public static void createCluster() throws Exception {
+    RouterHDFSContract.createCluster(initSecurity());
+  }
+
+  @AfterClass
+  public static void teardownCluster() throws IOException {
+    RouterHDFSContract.destroyCluster();
+  }
+
+  @Override
+  protected AbstractFSContract createContract(Configuration conf) {
+    return new RouterHDFSContract(conf);
+  }
+}
\ No newline at end of file
diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/fs/contract/router/TestRouterHDFSContractDeleteSecure.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/fs/contract/router/TestRouterHDFSContractDeleteSecure.java
new file mode 100644
index 0000000..57cc138
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/fs/contract/router/TestRouterHDFSContractDeleteSecure.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ *   you may not use this file except in compliance with the License.
+ *   You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *   Unless required by applicable law or agreed to in writing, software
+ *   distributed under the License is distributed on an "AS IS" BASIS,
+ *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *   See the License for the specific language governing permissions and
+ *   limitations under the License. See accompanying LICENSE file.
+ */
+
+package org.apache.hadoop.fs.contract.router;
+
+import java.io.IOException;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractDeleteTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+
+import static org.apache.hadoop.fs.contract.router.SecurityConfUtil.initSecurity;
+
+/**
+ * Test secure delete operations on the Router-based FS.
+ */
+public class TestRouterHDFSContractDeleteSecure
+    extends AbstractContractDeleteTest {
+
+  @BeforeClass
+  public static void createCluster() throws Exception {
+    RouterHDFSContract.createCluster(initSecurity());
+  }
+
+  @AfterClass
+  public static void teardownCluster() throws IOException {
+    RouterHDFSContract.destroyCluster();
+  }
+
+  @Override
+  protected AbstractFSContract createContract(Configuration conf) {
+    return new RouterHDFSContract(conf);
+  }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/fs/contract/router/TestRouterHDFSContractGetFileStatusSecure.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/fs/contract/router/TestRouterHDFSContractGetFileStatusSecure.java
new file mode 100644
index 0000000..13e4e96
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/fs/contract/router/TestRouterHDFSContractGetFileStatusSecure.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ *   you may not use this file except in compliance with the License.
+ *   You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *   Unless required by applicable law or agreed to in writing, software
+ *   distributed under the License is distributed on an "AS IS" BASIS,
+ *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *   See the License for the specific language governing permissions and
+ *   limitations under the License. See accompanying LICENSE file.
+ */
+
+package org.apache.hadoop.fs.contract.router;
+
+import java.io.IOException;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractGetFileStatusTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+
+import static org.apache.hadoop.fs.contract.router.SecurityConfUtil.initSecurity;
+
+
+/**
+ * Test secure get file status operations on the Router-based FS.
+ */
+public class TestRouterHDFSContractGetFileStatusSecure
+    extends AbstractContractGetFileStatusTest {
+
+  @BeforeClass
+  public static void createCluster() throws Exception {
+    RouterHDFSContract.createCluster(initSecurity());
+  }
+
+  @AfterClass
+  public static void teardownCluster() throws IOException {
+    RouterHDFSContract.destroyCluster();
+  }
+
+  @Override
+  protected AbstractFSContract createContract(Configuration conf) {
+    return new RouterHDFSContract(conf);
+  }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/fs/contract/router/TestRouterHDFSContractMkdirSecure.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/fs/contract/router/TestRouterHDFSContractMkdirSecure.java
new file mode 100644
index 0000000..7c223a6
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/fs/contract/router/TestRouterHDFSContractMkdirSecure.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ *   you may not use this file except in compliance with the License.
+ *   You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *   Unless required by applicable law or agreed to in writing, software
+ *   distributed under the License is distributed on an "AS IS" BASIS,
+ *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *   See the License for the specific language governing permissions and
+ *   limitations under the License. See accompanying LICENSE file.
+ */
+
+package org.apache.hadoop.fs.contract.router;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractMkdirTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+
+import java.io.IOException;
+
+import static org.apache.hadoop.fs.contract.router.SecurityConfUtil.initSecurity;
+
+
+/**
+ * Test secure dir operations on the Router-based FS.
+ */
+public class TestRouterHDFSContractMkdirSecure
+    extends AbstractContractMkdirTest {
+
+  @BeforeClass
+  public static void createCluster() throws Exception {
+    RouterHDFSContract.createCluster(initSecurity());
+  }
+
+  @AfterClass
+  public static void teardownCluster() throws IOException {
+    RouterHDFSContract.destroyCluster();
+  }
+
+  @Override
+  protected AbstractFSContract createContract(Configuration conf) {
+    return new RouterHDFSContract(conf);
+  }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/fs/contract/router/TestRouterHDFSContractOpenSecure.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/fs/contract/router/TestRouterHDFSContractOpenSecure.java
new file mode 100644
index 0000000..434402c
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/fs/contract/router/TestRouterHDFSContractOpenSecure.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ *   you may not use this file except in compliance with the License.
+ *   You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *   Unless required by applicable law or agreed to in writing, software
+ *   distributed under the License is distributed on an "AS IS" BASIS,
+ *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *   See the License for the specific language governing permissions and
+ *   limitations under the License. See accompanying LICENSE file.
+ */
+
+package org.apache.hadoop.fs.contract.router;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractOpenTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+
+import java.io.IOException;
+
+import static org.apache.hadoop.fs.contract.router.SecurityConfUtil.initSecurity;
+
+
+/**
+ * Test secure open operations on the Router-based FS.
+ */
+public class TestRouterHDFSContractOpenSecure extends AbstractContractOpenTest {
+
+  @BeforeClass
+  public static void createCluster() throws Exception {
+    RouterHDFSContract.createCluster(initSecurity());
+  }
+
+  @AfterClass
+  public static void teardownCluster() throws IOException {
+    RouterHDFSContract.destroyCluster();
+  }
+
+  @Override
+  protected AbstractFSContract createContract(Configuration conf) {
+    return new RouterHDFSContract(conf);
+  }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/fs/contract/router/TestRouterHDFSContractRenameSecure.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/fs/contract/router/TestRouterHDFSContractRenameSecure.java
new file mode 100644
index 0000000..29d7398
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/fs/contract/router/TestRouterHDFSContractRenameSecure.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ *   you may not use this file except in compliance with the License.
+ *   You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *   Unless required by applicable law or agreed to in writing, software
+ *   distributed under the License is distributed on an "AS IS" BASIS,
+ *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *   See the License for the specific language governing permissions and
+ *   limitations under the License. See accompanying LICENSE file.
+ */
+
+package org.apache.hadoop.fs.contract.router;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractRenameTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+
+import java.io.IOException;
+
+import static org.apache.hadoop.fs.contract.router.SecurityConfUtil.initSecurity;
+
+
+/**
+ * Test secure rename operations on the Router-based FS.
+ */
+public class TestRouterHDFSContractRenameSecure
+    extends AbstractContractRenameTest {
+
+  @BeforeClass
+  public static void createCluster() throws Exception {
+    RouterHDFSContract.createCluster(initSecurity());
+  }
+
+  @AfterClass
+  public static void teardownCluster() throws IOException {
+    RouterHDFSContract.destroyCluster();
+  }
+
+  @Override
+  protected AbstractFSContract createContract(Configuration conf) {
+    return new RouterHDFSContract(conf);
+  }
+}
\ No newline at end of file
diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/fs/contract/router/TestRouterHDFSContractRootDirectorySecure.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/fs/contract/router/TestRouterHDFSContractRootDirectorySecure.java
new file mode 100644
index 0000000..32ec161
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/fs/contract/router/TestRouterHDFSContractRootDirectorySecure.java
@@ -0,0 +1,63 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ *   you may not use this file except in compliance with the License.
+ *   You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *   Unless required by applicable law or agreed to in writing, software
+ *   distributed under the License is distributed on an "AS IS" BASIS,
+ *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *   See the License for the specific language governing permissions and
+ *   limitations under the License. See accompanying LICENSE file.
+ */
+
+package org.apache.hadoop.fs.contract.router;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractRootDirectoryTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+
+import java.io.IOException;
+
+import static org.apache.hadoop.fs.contract.router.SecurityConfUtil.initSecurity;
+
+
+/**
+ * Test secure root dir operations on the Router-based FS.
+ */
+public class TestRouterHDFSContractRootDirectorySecure
+    extends AbstractContractRootDirectoryTest {
+
+  @BeforeClass
+  public static void createCluster() throws Exception {
+    RouterHDFSContract.createCluster(initSecurity());
+  }
+
+  @AfterClass
+  public static void teardownCluster() throws IOException {
+    RouterHDFSContract.destroyCluster();
+  }
+
+  @Override
+  protected AbstractFSContract createContract(Configuration conf) {
+    return new RouterHDFSContract(conf);
+  }
+
+  @Override
+  public void testListEmptyRootDirectory() throws IOException {
+    // It doesn't apply because we still have the mount points here
+  }
+
+  @Override
+  public void testRmEmptyRootDirNonRecursive() throws IOException {
+    // It doesn't apply because we still have the mount points here
+  }
+
+  @Override
+  public void testRecursiveRootListing() throws IOException {
+    // It doesn't apply because we still have the mount points here
+  }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/fs/contract/router/TestRouterHDFSContractSeekSecure.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/fs/contract/router/TestRouterHDFSContractSeekSecure.java
new file mode 100644
index 0000000..f281b47
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/fs/contract/router/TestRouterHDFSContractSeekSecure.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ *   you may not use this file except in compliance with the License.
+ *   You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *   Unless required by applicable law or agreed to in writing, software
+ *   distributed under the License is distributed on an "AS IS" BASIS,
+ *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *   See the License for the specific language governing permissions and
+ *   limitations under the License. See accompanying LICENSE file.
+ */
+
+package org.apache.hadoop.fs.contract.router;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractSeekTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+
+import java.io.IOException;
+
+import static org.apache.hadoop.fs.contract.router.SecurityConfUtil.initSecurity;
+
+
+/**
+ * Test secure seek operations on the Router-based FS.
+ */
+public class TestRouterHDFSContractSeekSecure extends AbstractContractSeekTest {
+
+  @BeforeClass
+  public static void createCluster() throws Exception {
+    RouterHDFSContract.createCluster(initSecurity());
+  }
+
+  @AfterClass
+  public static void teardownCluster() throws IOException {
+    RouterHDFSContract.destroyCluster();
+  }
+
+
+  @Override
+  protected AbstractFSContract createContract(Configuration conf) {
+    return new RouterHDFSContract(conf);
+  }
+}
\ No newline at end of file
diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/fs/contract/router/TestRouterHDFSContractSetTimesSecure.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/fs/contract/router/TestRouterHDFSContractSetTimesSecure.java
new file mode 100644
index 0000000..8f86b95
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/fs/contract/router/TestRouterHDFSContractSetTimesSecure.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ *   you may not use this file except in compliance with the License.
+ *   You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *   Unless required by applicable law or agreed to in writing, software
+ *   distributed under the License is distributed on an "AS IS" BASIS,
+ *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *   See the License for the specific language governing permissions and
+ *   limitations under the License. See accompanying LICENSE file.
+ */
+
+package org.apache.hadoop.fs.contract.router;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractSetTimesTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+
+import java.io.IOException;
+
+import static org.apache.hadoop.fs.contract.router.SecurityConfUtil.initSecurity;
+
+
+/**
+ * Test secure set times operations on the Router-based FS.
+ */
+public class TestRouterHDFSContractSetTimesSecure
+    extends AbstractContractSetTimesTest {
+
+  @BeforeClass
+  public static void createCluster() throws Exception {
+    RouterHDFSContract.createCluster(initSecurity());
+  }
+
+  @AfterClass
+  public static void teardownCluster() throws IOException {
+    RouterHDFSContract.destroyCluster();
+  }
+
+  @Override
+  protected AbstractFSContract createContract(Configuration conf) {
+    return new RouterHDFSContract(conf);
+  }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/MiniRouterDFSCluster.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/MiniRouterDFSCluster.java
index e34713d..a5693a6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/MiniRouterDFSCluster.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/MiniRouterDFSCluster.java
@@ -28,6 +28,8 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_SERVICE_RPC_ADDR
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_SERVICE_RPC_BIND_HOST_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMESERVICES;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMESERVICE_ID;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HTTP_POLICY_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_HTTPS_ADDRESS_KEY;
 import static org.apache.hadoop.hdfs.server.federation.FederationTestUtils.NAMENODES;
 import static org.apache.hadoop.hdfs.server.federation.FederationTestUtils.addDirectory;
 import static org.apache.hadoop.hdfs.server.federation.FederationTestUtils.waitNamenodeRegistered;
@@ -85,6 +87,7 @@ import org.apache.hadoop.hdfs.server.federation.router.RouterClient;
 import org.apache.hadoop.hdfs.server.namenode.FSImage;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
+import org.apache.hadoop.http.HttpConfig;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.service.Service.STATE;
@@ -270,6 +273,7 @@ public class MiniRouterDFSCluster {
     private int servicePort;
     private int lifelinePort;
     private int httpPort;
+    private int httpsPort;
     private URI fileSystemUri;
     private int index;
     private DFSClient client;
@@ -305,7 +309,12 @@ public class MiniRouterDFSCluster {
       this.rpcPort = nn.getNameNodeAddress().getPort();
       this.servicePort = nn.getServiceRpcAddress().getPort();
       this.lifelinePort = nn.getServiceRpcAddress().getPort();
-      this.httpPort = nn.getHttpAddress().getPort();
+      if (nn.getHttpAddress() != null) {
+        this.httpPort = nn.getHttpAddress().getPort();
+      }
+      if (nn.getHttpsAddress() != null) {
+        this.httpsPort = nn.getHttpsAddress().getPort();
+      }
       this.fileSystemUri = new URI("hdfs://" + namenode.getHostAndPort());
       DistributedFileSystem.setDefaultUri(this.conf, this.fileSystemUri);
 
@@ -328,10 +337,22 @@ public class MiniRouterDFSCluster {
       return namenode.getServiceRpcAddress().getHostName() + ":" + lifelinePort;
     }
 
+    public String getWebAddress() {
+      if (conf.get(DFS_HTTP_POLICY_KEY)
+          .equals(HttpConfig.Policy.HTTPS_ONLY.name())) {
+        return getHttpsAddress();
+      }
+      return getHttpAddress();
+    }
+
     public String getHttpAddress() {
       return namenode.getHttpAddress().getHostName() + ":" + httpPort;
     }
 
+    public String getHttpsAddress() {
+      return namenode.getHttpsAddress().getHostName() + ":" + httpsPort;
+    }
+
     public FileSystem getFileSystem() throws IOException {
       return DistributedFileSystem.get(conf);
     }
@@ -375,22 +396,38 @@ public class MiniRouterDFSCluster {
 
   public MiniRouterDFSCluster(
       boolean ha, int numNameservices, int numNamenodes,
-      long heartbeatInterval, long cacheFlushInterval) {
+      long heartbeatInterval, long cacheFlushInterval,
+      Configuration overrideConf) {
     this.highAvailability = ha;
     this.heartbeatInterval = heartbeatInterval;
     this.cacheFlushInterval = cacheFlushInterval;
-    configureNameservices(numNameservices, numNamenodes);
+    configureNameservices(numNameservices, numNamenodes, overrideConf);
+  }
+
+  public MiniRouterDFSCluster(
+      boolean ha, int numNameservices, int numNamenodes,
+      long heartbeatInterval, long cacheFlushInterval) {
+    this(ha, numNameservices, numNamenodes,
+        heartbeatInterval, cacheFlushInterval, null);
   }
 
   public MiniRouterDFSCluster(boolean ha, int numNameservices) {
     this(ha, numNameservices, 2,
-        DEFAULT_HEARTBEAT_INTERVAL_MS, DEFAULT_CACHE_INTERVAL_MS);
+        DEFAULT_HEARTBEAT_INTERVAL_MS, DEFAULT_CACHE_INTERVAL_MS,
+        null);
   }
 
   public MiniRouterDFSCluster(
       boolean ha, int numNameservices, int numNamenodes) {
     this(ha, numNameservices, numNamenodes,
-        DEFAULT_HEARTBEAT_INTERVAL_MS, DEFAULT_CACHE_INTERVAL_MS);
+        DEFAULT_HEARTBEAT_INTERVAL_MS, DEFAULT_CACHE_INTERVAL_MS,
+        null);
+  }
+
+  public MiniRouterDFSCluster(boolean ha, int numNameservices,
+      Configuration overrideConf) {
+    this(ha, numNameservices, 2,
+        DEFAULT_HEARTBEAT_INTERVAL_MS, DEFAULT_CACHE_INTERVAL_MS, overrideConf);
   }
 
   /**
@@ -447,6 +484,8 @@ public class MiniRouterDFSCluster {
             "127.0.0.1:" + context.httpPort);
         conf.set(DFS_NAMENODE_RPC_BIND_HOST_KEY + "." + suffix,
             "0.0.0.0");
+        conf.set(DFS_NAMENODE_HTTPS_ADDRESS_KEY + "." + suffix,
+            "127.0.0.1:" + context.httpsPort);
 
         // If the service port is enabled by default, we need to set them up
         boolean servicePortEnabled = false;
@@ -543,7 +582,8 @@ public class MiniRouterDFSCluster {
     return conf;
   }
 
-  public void configureNameservices(int numNameservices, int numNamenodes) {
+  public void configureNameservices(int numNameservices, int numNamenodes,
+      Configuration overrideConf) {
     this.nameservices = new ArrayList<>();
     this.namenodes = new ArrayList<>();
 
@@ -554,6 +594,10 @@ public class MiniRouterDFSCluster {
       this.nameservices.add("ns" + i);
 
       Configuration nnConf = generateNamenodeConfiguration(ns);
+      if (overrideConf != null) {
+        nnConf.addResource(overrideConf);
+      }
+
       if (!highAvailability) {
         context = new NamenodeContext(nnConf, ns, null, nnIndex++);
         this.namenodes.add(context);
@@ -788,7 +832,7 @@ public class MiniRouterDFSCluster {
         NamenodeStatusReport report = new NamenodeStatusReport(
             nn.nameserviceId, nn.namenodeId,
             nn.getRpcAddress(), nn.getServiceAddress(),
-            nn.getLifelineAddress(), nn.getHttpAddress());
+            nn.getLifelineAddress(), nn.getWebAddress());
         FSImage fsImage = nn.namenode.getNamesystem().getFSImage();
         NamespaceInfo nsInfo = fsImage.getStorage().getNamespaceInfo();
         report.setNamespaceInfo(nsInfo);


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org