You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by el...@apache.org on 2019/11/26 16:19:36 UTC

[hbase] branch branch-2.1 updated: HBASE-23312 HBase Thrift SPNEGO configs (HBASE-19852) should be backwards compatible

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

elserj pushed a commit to branch branch-2.1
in repository https://gitbox.apache.org/repos/asf/hbase.git


The following commit(s) were added to refs/heads/branch-2.1 by this push:
     new 95ebb59  HBASE-23312 HBase Thrift SPNEGO configs (HBASE-19852) should be backwards compatible
95ebb59 is described below

commit 95ebb5981e5100a94828d266f86db9e5a4e449b8
Author: Kevin Risden <kr...@apache.org>
AuthorDate: Mon Nov 25 17:44:33 2019 -0500

    HBASE-23312 HBase Thrift SPNEGO configs (HBASE-19852) should be backwards compatible
    
    HBase Thrift SPNEGO configs should not be required.
    The `hbase.thrift.spnego.keytab.file` and
    `hbase.thrift.spnego.principal` configs should fall
    back to the `hbase.thrift.keytab.file` and
    `hbase.thrift.kerberos.principal` configs. This will
    avoid any issues during upgrades.
    
    Signed-off-by: Josh Elser <el...@apache.org>
    
    Closes #878
---
 .../hadoop/hbase/thrift/ThriftHttpServlet.java     |  21 +---
 .../hadoop/hbase/thrift/ThriftServerRunner.java    |  46 +++++++-
 ...ava => TestThriftSpnegoHttpFallbackServer.java} | 120 ++++++++-------------
 .../hbase/thrift/TestThriftSpnegoHttpServer.java   | 100 +++++++----------
 4 files changed, 133 insertions(+), 154 deletions(-)

diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftHttpServlet.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftHttpServlet.java
index 4c9a35b..7458162 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftHttpServlet.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftHttpServlet.java
@@ -18,9 +18,6 @@
 
 package org.apache.hadoop.hbase.thrift;
 
-import static org.apache.hadoop.hbase.thrift.ThriftServerRunner.THRIFT_SPNEGO_KEYTAB_FILE_KEY;
-import static org.apache.hadoop.hbase.thrift.ThriftServerRunner.THRIFT_SPNEGO_PRINCIPAL_KEY;
-
 import java.io.IOException;
 import java.security.PrivilegedExceptionAction;
 import java.util.Base64;
@@ -29,7 +26,6 @@ import javax.servlet.ServletException;
 import javax.servlet.http.HttpServletRequest;
 import javax.servlet.http.HttpServletResponse;
 
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.security.SecurityUtil;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authorize.AuthorizationException;
@@ -66,25 +62,14 @@ public class ThriftHttpServlet extends TServlet {
   public static final String NEGOTIATE = "Negotiate";
 
   public ThriftHttpServlet(TProcessor processor, TProtocolFactory protocolFactory,
-      UserGroupInformation serviceUGI, Configuration conf,
-      ThriftServerRunner.HBaseHandler hbaseHandler, boolean securityEnabled, boolean doAsEnabled)
-      throws IOException {
+      UserGroupInformation serviceUGI, UserGroupInformation httpUGI,
+      ThriftServerRunner.HBaseHandler hbaseHandler, boolean securityEnabled, boolean doAsEnabled) {
     super(processor, protocolFactory);
     this.serviceUGI = serviceUGI;
+    this.httpUGI = httpUGI;
     this.hbaseHandler = hbaseHandler;
     this.securityEnabled = securityEnabled;
     this.doAsEnabled = doAsEnabled;
-
-    if (securityEnabled) {
-      // login the spnego principal
-      UserGroupInformation.setConfiguration(conf);
-      this.httpUGI = UserGroupInformation.loginUserFromKeytabAndReturnUGI(
-          conf.get(THRIFT_SPNEGO_PRINCIPAL_KEY),
-          conf.get(THRIFT_SPNEGO_KEYTAB_FILE_KEY)
-      );
-    } else {
-      this.httpUGI = null;
-    }
   }
 
   @Override
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftServerRunner.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftServerRunner.java
index b510ff5..0c34213 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftServerRunner.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftServerRunner.java
@@ -224,6 +224,7 @@ public class ThriftServerRunner implements Runnable {
   private final ThriftMetrics metrics;
   private final HBaseHandler hbaseHandler;
   private final UserGroupInformation serviceUGI;
+  private UserGroupInformation httpUGI;
 
   private SaslUtil.QualityOfProtection qop;
   private String host;
@@ -349,8 +350,19 @@ public class ThriftServerRunner implements Runnable {
         conf.get(THRIFT_DNS_INTERFACE_KEY, "default"),
         conf.get(THRIFT_DNS_NAMESERVER_KEY, "default")));
       userProvider.login(THRIFT_KEYTAB_FILE_KEY, THRIFT_KERBEROS_PRINCIPAL_KEY, host);
+
+      // Setup the SPNEGO user for HTTP if configured
+      String spnegoPrincipal = getSpengoPrincipal(conf, host);
+      String spnegoKeytab = getSpnegoKeytab(conf);
+      UserGroupInformation.setConfiguration(conf);
+      // login the SPNEGO principal using UGI to avoid polluting the login user
+      this.httpUGI = UserGroupInformation.loginUserFromKeytabAndReturnUGI(spnegoPrincipal,
+        spnegoKeytab);
     }
     this.serviceUGI = userProvider.getCurrent().getUGI();
+    if (httpUGI == null) {
+      this.httpUGI = serviceUGI;
+    }
 
     this.conf = HBaseConfiguration.create(conf);
     this.listenPort = conf.getInt(PORT_CONF_KEY, DEFAULT_LISTEN_PORT);
@@ -387,6 +399,38 @@ public class ThriftServerRunner implements Runnable {
     }
   }
 
+
+  private String getSpengoPrincipal(Configuration conf, String host) throws IOException {
+    String principal = conf.get(THRIFT_SPNEGO_PRINCIPAL_KEY);
+    if (principal == null) {
+      // We cannot use the Hadoop configuration deprecation handling here since
+      // the THRIFT_KERBEROS_PRINCIPAL_KEY config is still valid for regular Kerberos
+      // communication. The preference should be to use the THRIFT_SPNEGO_PRINCIPAL_KEY
+      // config so that THRIFT_KERBEROS_PRINCIPAL_KEY doesn't control both backend
+      // Kerberos principal and SPNEGO principal.
+      LOG.info("Using deprecated {} config for SPNEGO principal. Use {} instead.",
+        THRIFT_KERBEROS_PRINCIPAL_KEY, THRIFT_SPNEGO_PRINCIPAL_KEY);
+      principal = conf.get(THRIFT_KERBEROS_PRINCIPAL_KEY);
+    }
+    // Handle _HOST in principal value
+    return org.apache.hadoop.security.SecurityUtil.getServerPrincipal(principal, host);
+  }
+
+  private String getSpnegoKeytab(Configuration conf) {
+    String keytab = conf.get(THRIFT_SPNEGO_KEYTAB_FILE_KEY);
+    if (keytab == null) {
+      // We cannot use the Hadoop configuration deprecation handling here since
+      // the THRIFT_KEYTAB_FILE_KEY config is still valid for regular Kerberos
+      // communication. The preference should be to use the THRIFT_SPNEGO_KEYTAB_FILE_KEY
+      // config so that THRIFT_KEYTAB_FILE_KEY doesn't control both backend
+      // Kerberos keytab and SPNEGO keytab.
+      LOG.info("Using deprecated {} config for SPNEGO keytab. Use {} instead.",
+        THRIFT_KEYTAB_FILE_KEY, THRIFT_SPNEGO_KEYTAB_FILE_KEY);
+      keytab = conf.get(THRIFT_KEYTAB_FILE_KEY);
+    }
+    return keytab;
+  }
+
   private void checkHttpSecurity(QualityOfProtection qop, Configuration conf) {
     if (qop == QualityOfProtection.PRIVACY &&
         conf.getBoolean(USE_HTTP_CONF_KEY, false) &&
@@ -448,7 +492,7 @@ public class ThriftServerRunner implements Runnable {
     TProtocolFactory protocolFactory = new TBinaryProtocol.Factory();
     TProcessor processor = new Hbase.Processor<>(handler);
     TServlet thriftHttpServlet = new ThriftHttpServlet(processor, protocolFactory, serviceUGI,
-        conf, hbaseHandler, securityEnabled, doAsEnabled);
+        httpUGI, hbaseHandler, securityEnabled, doAsEnabled);
 
     // Set the default max thread number to 100 to limit
     // the number of concurrent requests so that Thrfit HTTP server doesn't OOM easily.
diff --git a/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftSpnegoHttpServer.java b/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftSpnegoHttpFallbackServer.java
similarity index 67%
copy from hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftSpnegoHttpServer.java
copy to hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftSpnegoHttpFallbackServer.java
index 1218d18..c8667e4 100644
--- a/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftSpnegoHttpServer.java
+++ b/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftSpnegoHttpFallbackServer.java
@@ -1,34 +1,27 @@
 /*
- * Copyright The Apache Software Foundation
+ * 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
  *
- * 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
+ *     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.
+ * 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.hbase.thrift;
 
-import static org.apache.hadoop.hbase.thrift.ThriftServerRunner.THRIFT_KERBEROS_PRINCIPAL_KEY;
-import static org.apache.hadoop.hbase.thrift.ThriftServerRunner.THRIFT_KEYTAB_FILE_KEY;
-import static org.apache.hadoop.hbase.thrift.ThriftServerRunner.THRIFT_SPNEGO_KEYTAB_FILE_KEY;
-import static org.apache.hadoop.hbase.thrift.ThriftServerRunner.THRIFT_SPNEGO_PRINCIPAL_KEY;
-import static org.apache.hadoop.hbase.thrift.ThriftServerRunner.THRIFT_SUPPORT_PROXYUSER_KEY;
-import static org.apache.hadoop.hbase.thrift.ThriftServerRunner.USE_HTTP_CONF_KEY;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
 
 import java.io.File;
+import java.nio.file.Paths;
 import java.security.Principal;
 import java.security.PrivilegedExceptionAction;
 import java.util.Set;
@@ -36,7 +29,6 @@ import java.util.Set;
 import javax.security.auth.Subject;
 import javax.security.auth.kerberos.KerberosTicket;
 
-import org.apache.commons.io.FileUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
@@ -58,7 +50,6 @@ import org.apache.http.impl.auth.SPNegoSchemeFactory;
 import org.apache.http.impl.client.BasicCredentialsProvider;
 import org.apache.http.impl.client.CloseableHttpClient;
 import org.apache.http.impl.client.HttpClients;
-import org.apache.kerby.kerberos.kerb.KrbException;
 import org.apache.kerby.kerberos.kerb.client.JaasKrbUtil;
 import org.apache.kerby.kerberos.kerb.server.SimpleKdcServer;
 import org.apache.thrift.protocol.TBinaryProtocol;
@@ -78,39 +69,32 @@ import org.slf4j.LoggerFactory;
 /**
  * Start the HBase Thrift HTTP server on a random port through the command-line
  * interface and talk to it from client side with SPNEGO security enabled.
+ *
+ * Supplemental test to TestThriftSpnegoHttpServer which falls back to the original
+ * Kerberos principal and keytab configuration properties, not the separate
+ * SPNEGO-specific properties.
  */
 @Category({ClientTests.class, LargeTests.class})
-public class TestThriftSpnegoHttpServer extends TestThriftHttpServer {
+public class TestThriftSpnegoHttpFallbackServer extends TestThriftHttpServer {
   @ClassRule
   public static final HBaseClassTestRule CLASS_RULE =
-    HBaseClassTestRule.forClass(TestThriftSpnegoHttpServer.class);
+    HBaseClassTestRule.forClass(TestThriftSpnegoHttpFallbackServer.class);
 
   private static final Logger LOG =
-    LoggerFactory.getLogger(TestThriftSpnegoHttpServer.class);
+    LoggerFactory.getLogger(TestThriftSpnegoHttpFallbackServer.class);
 
   private static SimpleKdcServer kdc;
   private static File serverKeytab;
-  private static File spnegoServerKeytab;
   private static File clientKeytab;
 
   private static String clientPrincipal;
   private static String serverPrincipal;
   private static String spnegoServerPrincipal;
 
-  private static void setupUser(SimpleKdcServer kdc, File keytab, String principal)
-      throws KrbException {
-    kdc.createPrincipal(principal);
-    kdc.exportPrincipal(principal, keytab);
-  }
-
   private static SimpleKdcServer buildMiniKdc() throws Exception {
     SimpleKdcServer kdc = new SimpleKdcServer();
 
-    final File target = new File(System.getProperty("user.dir"), "target");
-    File kdcDir = new File(target, TestThriftSpnegoHttpServer.class.getSimpleName());
-    if (kdcDir.exists()) {
-      FileUtils.deleteDirectory(kdcDir);
-    }
+    File kdcDir = Paths.get(TEST_UTIL.getRandomDir().toString()).toAbsolutePath().toFile();
     kdcDir.mkdirs();
     kdc.setWorkDir(kdcDir);
 
@@ -131,66 +115,55 @@ public class TestThriftSpnegoHttpServer extends TestThriftHttpServer {
     KerberosName.setRules("DEFAULT");
 
     HBaseKerberosUtils.setKeytabFileForTesting(serverKeytab.getAbsolutePath());
-    HBaseKerberosUtils.setPrincipalForTesting(serverPrincipal);
+    HBaseKerberosUtils.setPrincipalForTesting(spnegoServerPrincipal);
     HBaseKerberosUtils.setSecuredConfiguration(conf);
 
     // if we drop support for hadoop-2.4.0 and hadoop-2.4.1,
     // the following key should be changed.
     // 1) DFS_NAMENODE_USER_NAME_KEY -> DFS_NAMENODE_KERBEROS_PRINCIPAL_KEY
     // 2) DFS_DATANODE_USER_NAME_KEY -> DFS_DATANODE_KERBEROS_PRINCIPAL_KEY
-    conf.set(DFSConfigKeys.DFS_NAMENODE_USER_NAME_KEY, serverPrincipal);
+    conf.set(DFSConfigKeys.DFS_NAMENODE_USER_NAME_KEY, spnegoServerPrincipal);
     conf.set(DFSConfigKeys.DFS_NAMENODE_KEYTAB_FILE_KEY, serverKeytab.getAbsolutePath());
-    conf.set(DFSConfigKeys.DFS_DATANODE_USER_NAME_KEY, serverPrincipal);
+    conf.set(DFSConfigKeys.DFS_DATANODE_USER_NAME_KEY, spnegoServerPrincipal);
     conf.set(DFSConfigKeys.DFS_DATANODE_KEYTAB_FILE_KEY, serverKeytab.getAbsolutePath());
 
     conf.setBoolean(DFSConfigKeys.DFS_BLOCK_ACCESS_TOKEN_ENABLE_KEY, true);
 
     conf.set(DFSConfigKeys.DFS_WEB_AUTHENTICATION_KERBEROS_PRINCIPAL_KEY, spnegoServerPrincipal);
     conf.set(DFSConfigKeys.DFS_WEB_AUTHENTICATION_KERBEROS_KEYTAB_KEY,
-        spnegoServerKeytab.getAbsolutePath());
+        serverKeytab.getAbsolutePath());
 
     conf.setBoolean("ignore.secure.ports.for.testing", true);
 
-    conf.setBoolean(THRIFT_SUPPORT_PROXYUSER_KEY, true);
-    conf.setBoolean(USE_HTTP_CONF_KEY, true);
-    conf.set("hadoop.proxyuser.hbase.hosts", "*");
-    conf.set("hadoop.proxyuser.hbase.groups", "*");
+    conf.setBoolean(ThriftServerRunner.THRIFT_SUPPORT_PROXYUSER_KEY, true);
+    conf.setBoolean(ThriftServerRunner.USE_HTTP_CONF_KEY, true);
+    conf.set("hadoop.proxyuser.HTTP.hosts", "*");
+    conf.set("hadoop.proxyuser.HTTP.groups", "*");
 
-    conf.set(THRIFT_KERBEROS_PRINCIPAL_KEY, serverPrincipal);
-    conf.set(THRIFT_KEYTAB_FILE_KEY, serverKeytab.getAbsolutePath());
-    conf.set(THRIFT_SPNEGO_PRINCIPAL_KEY, spnegoServerPrincipal);
-    conf.set(THRIFT_SPNEGO_KEYTAB_FILE_KEY, spnegoServerKeytab.getAbsolutePath());
+    conf.set(ThriftServerRunner.THRIFT_KERBEROS_PRINCIPAL_KEY, spnegoServerPrincipal);
+    conf.set(ThriftServerRunner.THRIFT_KEYTAB_FILE_KEY, serverKeytab.getAbsolutePath());
   }
 
   @BeforeClass
   public static void setUpBeforeClass() throws Exception {
-    final File target = new File(System.getProperty("user.dir"), "target");
-    assertTrue(target.exists());
-
-    File keytabDir = new File(target, TestThriftSpnegoHttpServer.class.getSimpleName() +
-      "_keytabs");
-    if (keytabDir.exists()) {
-      FileUtils.deleteDirectory(keytabDir);
-    }
-    keytabDir.mkdirs();
-
     kdc = buildMiniKdc();
     kdc.start();
 
+    File keytabDir = Paths.get(TEST_UTIL.getRandomDir().toString()).toAbsolutePath().toFile();
+    keytabDir.mkdirs();
+
     clientPrincipal = "client@" + kdc.getKdcConfig().getKdcRealm();
     clientKeytab = new File(keytabDir, clientPrincipal + ".keytab");
-    setupUser(kdc, clientKeytab, clientPrincipal);
+    kdc.createAndExportPrincipals(clientKeytab, clientPrincipal);
 
     serverPrincipal = "hbase/" + HConstants.LOCALHOST + "@" + kdc.getKdcConfig().getKdcRealm();
     serverKeytab = new File(keytabDir, serverPrincipal.replace('/', '_') + ".keytab");
-    setupUser(kdc, serverKeytab, serverPrincipal);
 
     spnegoServerPrincipal = "HTTP/" + HConstants.LOCALHOST + "@" + kdc.getKdcConfig().getKdcRealm();
-    spnegoServerKeytab = new File(keytabDir, spnegoServerPrincipal.replace('/', '_') + ".keytab");
-    setupUser(kdc, spnegoServerKeytab, spnegoServerPrincipal);
+    // Add SPNEGO principal to server keytab
+    kdc.createAndExportPrincipals(serverKeytab, serverPrincipal, spnegoServerPrincipal);
 
-    TEST_UTIL.getConfiguration().setBoolean(USE_HTTP_CONF_KEY, true);
-    TEST_UTIL.getConfiguration().setBoolean("hbase.table.sanity.checks", false);
+    TEST_UTIL.getConfiguration().setBoolean(ThriftServerRunner.USE_HTTP_CONF_KEY, true);
     addSecurityConfigurations(TEST_UTIL.getConfiguration());
 
     TestThriftHttpServer.setUpBeforeClass();
@@ -203,6 +176,7 @@ public class TestThriftSpnegoHttpServer extends TestThriftHttpServer {
     try {
       if (null != kdc) {
         kdc.stop();
+        kdc = null;
       }
     } catch (Exception e) {
       LOG.info("Failed to stop mini KDC", e);
@@ -210,7 +184,7 @@ public class TestThriftSpnegoHttpServer extends TestThriftHttpServer {
   }
 
   @Override
-  void talkToThriftServer(String url, int customHeaderSize) throws Exception {
+  protected void talkToThriftServer(String url, int customHeaderSize) throws Exception {
     // Close httpClient and THttpClient automatically on any failures
     try (
         CloseableHttpClient httpClient = createHttpClient();
@@ -227,11 +201,9 @@ public class TestThriftSpnegoHttpServer extends TestThriftHttpServer {
 
       TProtocol prot = new TBinaryProtocol(tHttpClient);
       Hbase.Client client = new Hbase.Client(prot);
-      if (!tableCreated) {
-        TestThriftServer.createTestTables(client);
-        tableCreated = true;
-      }
+      TestThriftServer.createTestTables(client);
       TestThriftServer.checkTableList(client);
+      TestThriftServer.dropTestTables(client);
     }
   }
 
@@ -239,15 +211,17 @@ public class TestThriftSpnegoHttpServer extends TestThriftHttpServer {
     final Subject clientSubject = JaasKrbUtil.loginUsingKeytab(clientPrincipal, clientKeytab);
     final Set<Principal> clientPrincipals = clientSubject.getPrincipals();
     // Make sure the subject has a principal
-    assertFalse(clientPrincipals.isEmpty());
+    assertFalse("Found no client principals in the clientSubject.",
+      clientPrincipals.isEmpty());
 
     // Get a TGT for the subject (might have many, different encryption types). The first should
     // be the default encryption type.
     Set<KerberosTicket> privateCredentials =
         clientSubject.getPrivateCredentials(KerberosTicket.class);
-    assertFalse(privateCredentials.isEmpty());
+    assertFalse("Found no private credentials in the clientSubject.",
+      privateCredentials.isEmpty());
     KerberosTicket tgt = privateCredentials.iterator().next();
-    assertNotNull(tgt);
+    assertNotNull("No kerberos ticket found.", tgt);
 
     // The name of the principal
     final String clientPrincipalName = clientPrincipals.iterator().next().getName();
diff --git a/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftSpnegoHttpServer.java b/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftSpnegoHttpServer.java
index 1218d18..2d5c1f2 100644
--- a/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftSpnegoHttpServer.java
+++ b/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftSpnegoHttpServer.java
@@ -1,34 +1,27 @@
 /*
- * Copyright The Apache Software Foundation
+ * 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
  *
- * 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
+ *     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.
+ * 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.hbase.thrift;
 
-import static org.apache.hadoop.hbase.thrift.ThriftServerRunner.THRIFT_KERBEROS_PRINCIPAL_KEY;
-import static org.apache.hadoop.hbase.thrift.ThriftServerRunner.THRIFT_KEYTAB_FILE_KEY;
-import static org.apache.hadoop.hbase.thrift.ThriftServerRunner.THRIFT_SPNEGO_KEYTAB_FILE_KEY;
-import static org.apache.hadoop.hbase.thrift.ThriftServerRunner.THRIFT_SPNEGO_PRINCIPAL_KEY;
-import static org.apache.hadoop.hbase.thrift.ThriftServerRunner.THRIFT_SUPPORT_PROXYUSER_KEY;
-import static org.apache.hadoop.hbase.thrift.ThriftServerRunner.USE_HTTP_CONF_KEY;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
 
 import java.io.File;
+import java.nio.file.Paths;
 import java.security.Principal;
 import java.security.PrivilegedExceptionAction;
 import java.util.Set;
@@ -36,7 +29,6 @@ import java.util.Set;
 import javax.security.auth.Subject;
 import javax.security.auth.kerberos.KerberosTicket;
 
-import org.apache.commons.io.FileUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
@@ -58,7 +50,6 @@ import org.apache.http.impl.auth.SPNegoSchemeFactory;
 import org.apache.http.impl.client.BasicCredentialsProvider;
 import org.apache.http.impl.client.CloseableHttpClient;
 import org.apache.http.impl.client.HttpClients;
-import org.apache.kerby.kerberos.kerb.KrbException;
 import org.apache.kerby.kerberos.kerb.client.JaasKrbUtil;
 import org.apache.kerby.kerberos.kerb.server.SimpleKdcServer;
 import org.apache.thrift.protocol.TBinaryProtocol;
@@ -97,20 +88,10 @@ public class TestThriftSpnegoHttpServer extends TestThriftHttpServer {
   private static String serverPrincipal;
   private static String spnegoServerPrincipal;
 
-  private static void setupUser(SimpleKdcServer kdc, File keytab, String principal)
-      throws KrbException {
-    kdc.createPrincipal(principal);
-    kdc.exportPrincipal(principal, keytab);
-  }
-
   private static SimpleKdcServer buildMiniKdc() throws Exception {
     SimpleKdcServer kdc = new SimpleKdcServer();
 
-    final File target = new File(System.getProperty("user.dir"), "target");
-    File kdcDir = new File(target, TestThriftSpnegoHttpServer.class.getSimpleName());
-    if (kdcDir.exists()) {
-      FileUtils.deleteDirectory(kdcDir);
-    }
+    File kdcDir = Paths.get(TEST_UTIL.getRandomDir().toString()).toAbsolutePath().toFile();
     kdcDir.mkdirs();
     kdc.setWorkDir(kdcDir);
 
@@ -151,46 +132,40 @@ public class TestThriftSpnegoHttpServer extends TestThriftHttpServer {
 
     conf.setBoolean("ignore.secure.ports.for.testing", true);
 
-    conf.setBoolean(THRIFT_SUPPORT_PROXYUSER_KEY, true);
-    conf.setBoolean(USE_HTTP_CONF_KEY, true);
+    conf.setBoolean(ThriftServerRunner.THRIFT_SUPPORT_PROXYUSER_KEY, true);
+    conf.setBoolean(ThriftServerRunner.USE_HTTP_CONF_KEY, true);
     conf.set("hadoop.proxyuser.hbase.hosts", "*");
     conf.set("hadoop.proxyuser.hbase.groups", "*");
 
-    conf.set(THRIFT_KERBEROS_PRINCIPAL_KEY, serverPrincipal);
-    conf.set(THRIFT_KEYTAB_FILE_KEY, serverKeytab.getAbsolutePath());
-    conf.set(THRIFT_SPNEGO_PRINCIPAL_KEY, spnegoServerPrincipal);
-    conf.set(THRIFT_SPNEGO_KEYTAB_FILE_KEY, spnegoServerKeytab.getAbsolutePath());
+    conf.set(ThriftServerRunner.THRIFT_KERBEROS_PRINCIPAL_KEY, serverPrincipal);
+    conf.set(ThriftServerRunner.THRIFT_KEYTAB_FILE_KEY, serverKeytab.getAbsolutePath());
+    conf.set(ThriftServerRunner.THRIFT_SPNEGO_PRINCIPAL_KEY, spnegoServerPrincipal);
+    conf.set(ThriftServerRunner.THRIFT_SPNEGO_KEYTAB_FILE_KEY,
+      spnegoServerKeytab.getAbsolutePath());
   }
 
   @BeforeClass
   public static void setUpBeforeClass() throws Exception {
-    final File target = new File(System.getProperty("user.dir"), "target");
-    assertTrue(target.exists());
-
-    File keytabDir = new File(target, TestThriftSpnegoHttpServer.class.getSimpleName() +
-      "_keytabs");
-    if (keytabDir.exists()) {
-      FileUtils.deleteDirectory(keytabDir);
-    }
-    keytabDir.mkdirs();
-
     kdc = buildMiniKdc();
     kdc.start();
 
+    File keytabDir = Paths.get(TEST_UTIL.getRandomDir().toString()).toAbsolutePath().toFile();
+    keytabDir.mkdirs();
+
     clientPrincipal = "client@" + kdc.getKdcConfig().getKdcRealm();
     clientKeytab = new File(keytabDir, clientPrincipal + ".keytab");
-    setupUser(kdc, clientKeytab, clientPrincipal);
+    kdc.createAndExportPrincipals(clientKeytab, clientPrincipal);
 
     serverPrincipal = "hbase/" + HConstants.LOCALHOST + "@" + kdc.getKdcConfig().getKdcRealm();
     serverKeytab = new File(keytabDir, serverPrincipal.replace('/', '_') + ".keytab");
-    setupUser(kdc, serverKeytab, serverPrincipal);
 
+    // Setup separate SPNEGO keytab
     spnegoServerPrincipal = "HTTP/" + HConstants.LOCALHOST + "@" + kdc.getKdcConfig().getKdcRealm();
     spnegoServerKeytab = new File(keytabDir, spnegoServerPrincipal.replace('/', '_') + ".keytab");
-    setupUser(kdc, spnegoServerKeytab, spnegoServerPrincipal);
+    kdc.createAndExportPrincipals(spnegoServerKeytab, spnegoServerPrincipal);
+    kdc.createAndExportPrincipals(serverKeytab, serverPrincipal);
 
-    TEST_UTIL.getConfiguration().setBoolean(USE_HTTP_CONF_KEY, true);
-    TEST_UTIL.getConfiguration().setBoolean("hbase.table.sanity.checks", false);
+    TEST_UTIL.getConfiguration().setBoolean(ThriftServerRunner.USE_HTTP_CONF_KEY, true);
     addSecurityConfigurations(TEST_UTIL.getConfiguration());
 
     TestThriftHttpServer.setUpBeforeClass();
@@ -203,6 +178,7 @@ public class TestThriftSpnegoHttpServer extends TestThriftHttpServer {
     try {
       if (null != kdc) {
         kdc.stop();
+        kdc = null;
       }
     } catch (Exception e) {
       LOG.info("Failed to stop mini KDC", e);
@@ -210,7 +186,7 @@ public class TestThriftSpnegoHttpServer extends TestThriftHttpServer {
   }
 
   @Override
-  void talkToThriftServer(String url, int customHeaderSize) throws Exception {
+  protected void talkToThriftServer(String url, int customHeaderSize) throws Exception {
     // Close httpClient and THttpClient automatically on any failures
     try (
         CloseableHttpClient httpClient = createHttpClient();
@@ -227,11 +203,9 @@ public class TestThriftSpnegoHttpServer extends TestThriftHttpServer {
 
       TProtocol prot = new TBinaryProtocol(tHttpClient);
       Hbase.Client client = new Hbase.Client(prot);
-      if (!tableCreated) {
-        TestThriftServer.createTestTables(client);
-        tableCreated = true;
-      }
+      TestThriftServer.createTestTables(client);
       TestThriftServer.checkTableList(client);
+      TestThriftServer.dropTestTables(client);
     }
   }
 
@@ -239,15 +213,17 @@ public class TestThriftSpnegoHttpServer extends TestThriftHttpServer {
     final Subject clientSubject = JaasKrbUtil.loginUsingKeytab(clientPrincipal, clientKeytab);
     final Set<Principal> clientPrincipals = clientSubject.getPrincipals();
     // Make sure the subject has a principal
-    assertFalse(clientPrincipals.isEmpty());
+    assertFalse("Found no client principals in the clientSubject.",
+      clientPrincipals.isEmpty());
 
     // Get a TGT for the subject (might have many, different encryption types). The first should
     // be the default encryption type.
     Set<KerberosTicket> privateCredentials =
         clientSubject.getPrivateCredentials(KerberosTicket.class);
-    assertFalse(privateCredentials.isEmpty());
+    assertFalse("Found no private credentials in the clientSubject.",
+      privateCredentials.isEmpty());
     KerberosTicket tgt = privateCredentials.iterator().next();
-    assertNotNull(tgt);
+    assertNotNull("No kerberos ticket found.", tgt);
 
     // The name of the principal
     final String clientPrincipalName = clientPrincipals.iterator().next().getName();