You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@drill.apache.org by dz...@apache.org on 2022/07/13 14:25:34 UTC

[drill] branch master updated: DRILL-8256: Fix unit tests of Kerberos auth in RPC (#2592)

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

dzamo pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/drill.git


The following commit(s) were added to refs/heads/master by this push:
     new 2cb1cec4af DRILL-8256: Fix unit tests of Kerberos auth in RPC (#2592)
2cb1cec4af is described below

commit 2cb1cec4af8c65348511a126100323b3eee0b252
Author: James Turton <91...@users.noreply.github.com>
AuthorDate: Wed Jul 13 16:25:30 2022 +0200

    DRILL-8256: Fix unit tests of Kerberos auth in RPC (#2592)
---
 .../rpc/security/AuthenticatorProviderImpl.java    |   7 +-
 .../drill/exec/rpc/data/TestBitBitKerberos.java    | 371 ++++++++-------------
 .../rpc/user/security/TestUserBitKerberos.java     |   3 +-
 .../security/TestUserBitKerberosEncryption.java    |   3 +-
 .../rest/spnego/TestDrillSpnegoAuthenticator.java  |   2 +-
 .../rest/spnego/TestSpnegoAuthentication.java      |   2 +-
 .../exec/server/rest/spnego/TestSpnegoConfig.java  |   2 +-
 .../java/org/apache/drill/test/QueryBuilder.java   |  28 +-
 8 files changed, 173 insertions(+), 245 deletions(-)

diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/security/AuthenticatorProviderImpl.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/security/AuthenticatorProviderImpl.java
index 60ffdec1d6..178a185ead 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/security/AuthenticatorProviderImpl.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/security/AuthenticatorProviderImpl.java
@@ -45,8 +45,11 @@ public class AuthenticatorProviderImpl implements AuthenticatorProvider {
 
   @SuppressWarnings("unchecked")
   public AuthenticatorProviderImpl(final DrillConfig config, final ScanResult scan) throws DrillbitStartupException {
-    // Skip auth mechanisms setup if user authentication is disabled
-    if (!config.getBoolean(ExecConstants.USER_AUTHENTICATION_ENABLED)) {
+    // Skip auth mechanisms setup if no authentication is enabled
+    if (
+      !config.getBoolean(ExecConstants.USER_AUTHENTICATION_ENABLED) &&
+      !config.getBoolean(ExecConstants.BIT_AUTHENTICATION_ENABLED)
+    ) {
       return;
     }
 
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/data/TestBitBitKerberos.java b/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/data/TestBitBitKerberos.java
index 7571efb296..65b0351dce 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/data/TestBitBitKerberos.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/data/TestBitBitKerberos.java
@@ -21,13 +21,15 @@ import org.apache.drill.exec.proto.BitData;
 import org.apache.drill.shaded.guava.com.google.common.base.Preconditions;
 import org.apache.drill.shaded.guava.com.google.common.base.Stopwatch;
 import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
-import com.typesafe.config.Config;
-import com.typesafe.config.ConfigValueFactory;
+import org.apache.drill.test.BaseDirTestWatcher;
+import org.apache.drill.test.ClientFixture;
+import org.apache.drill.test.ClusterFixture;
+import org.apache.drill.test.ClusterFixtureBuilder;
+import org.apache.drill.test.ClusterTest;
+
 import io.netty.buffer.ByteBuf;
 import org.apache.drill.exec.ops.FragmentContextImpl;
-import org.apache.drill.test.BaseTestQuery;
 import org.apache.drill.categories.SecurityTest;
-import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.common.config.DrillProperties;
 import org.apache.drill.common.exceptions.UserRemoteException;
 import org.apache.drill.common.scanner.ClassPathScanner;
@@ -60,19 +62,14 @@ import org.apache.drill.exec.vector.ValueVector;
 import org.apache.drill.exec.work.WorkManager.WorkerBee;
 import org.apache.drill.exec.work.fragment.FragmentExecutor;
 import org.apache.drill.exec.work.fragment.FragmentManager;
-import org.apache.hadoop.security.authentication.util.KerberosName;
-import org.apache.hadoop.security.authentication.util.KerberosUtil;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
-import org.junit.Ignore;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.mockito.Mockito;
 
 import java.io.IOException;
-import java.lang.reflect.Field;
 import java.util.List;
-import java.util.Properties;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicLong;
 
@@ -81,52 +78,28 @@ import static org.junit.Assert.assertTrue;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
-@Ignore("See DRILL-5387")
 @Category(SecurityTest.class)
-public class TestBitBitKerberos extends BaseTestQuery {
+public class TestBitBitKerberos extends ClusterTest {
   private static KerberosHelper krbHelper;
-  private static DrillConfig newConfig;
 
   private int port = 1234;
 
   @BeforeClass
   public static void setupTest() throws Exception {
-
-    final Config config = DrillConfig.create(cloneDefaultTestConfigProperties());
-
     krbHelper = new KerberosHelper(TestBitBitKerberos.class.getSimpleName(), null);
-    krbHelper.setupKdc(dirTestWatcher.getTmpDir());
-
-    newConfig = new DrillConfig(
-        config.withValue(ExecConstants.AUTHENTICATION_MECHANISMS,
-            ConfigValueFactory.fromIterable(Lists.newArrayList("kerberos")))
-        .withValue(ExecConstants.BIT_AUTHENTICATION_ENABLED,
-            ConfigValueFactory.fromAnyRef(true))
-        .withValue(ExecConstants.BIT_AUTHENTICATION_MECHANISM,
-            ConfigValueFactory.fromAnyRef("kerberos"))
-        .withValue(ExecConstants.USE_LOGIN_PRINCIPAL,
-            ConfigValueFactory.fromAnyRef(true))
-        .withValue(ExecConstants.SERVICE_PRINCIPAL,
-            ConfigValueFactory.fromAnyRef(krbHelper.SERVER_PRINCIPAL))
-        .withValue(ExecConstants.SERVICE_KEYTAB_LOCATION,
-            ConfigValueFactory.fromAnyRef(krbHelper.serverKeytab.toString())));
-
-    // Ignore the compile time warning caused by the code below.
-
-    // Config is statically initialized at this point. But the above configuration results in a different
-    // initialization which causes the tests to fail. So the following two changes are required.
-
-    // (1) Refresh Kerberos config.
-    // This disabled call to an unsupported internal API does not appear to be
-    // required and it prevents compiling with a target of JDK 8 on newer JDKs.
-    // sun.security.krb5.Config.refresh();
-
-    // (2) Reset the default realm.
-    final Field defaultRealm = KerberosName.class.getDeclaredField("defaultRealm");
-    defaultRealm.setAccessible(true);
-    defaultRealm.set(null, KerberosUtil.getDefaultRealm());
-
-    updateTestCluster(1, newConfig);
+    krbHelper.setupKdc(BaseDirTestWatcher.createTempDir(dirTestWatcher.getTmpDir()));
+    cluster = defaultClusterConfig().build();
+  }
+
+  private static ClusterFixtureBuilder defaultClusterConfig() {
+    return ClusterFixture.bareBuilder(dirTestWatcher)
+      .clusterSize(1)
+      .configNonStringProperty(ExecConstants.AUTHENTICATION_MECHANISMS, Lists.newArrayList("kerberos"))
+      .configProperty(ExecConstants.BIT_AUTHENTICATION_ENABLED, true)
+      .configProperty(ExecConstants.BIT_AUTHENTICATION_MECHANISM, "kerberos")
+      .configProperty(ExecConstants.USE_LOGIN_PRINCIPAL, true)
+      .configProperty(ExecConstants.SERVICE_PRINCIPAL, krbHelper.SERVER_PRINCIPAL)
+      .configProperty(ExecConstants.SERVICE_KEYTAB_LOCATION, krbHelper.serverKeytab.toString());
   }
 
   private FragmentManager setupFragmentContextAndManager(BufferAllocator allocator) {
@@ -188,23 +161,9 @@ public class TestBitBitKerberos extends BaseTestQuery {
     final WorkerBee bee = mock(WorkerBee.class);
     final WorkEventBus workBus = mock(WorkEventBus.class);
 
-    newConfig = new DrillConfig(DrillConfig.create(cloneDefaultTestConfigProperties())
-        .withValue(ExecConstants.AUTHENTICATION_MECHANISMS,
-          ConfigValueFactory.fromIterable(Lists.newArrayList("kerberos")))
-        .withValue(ExecConstants.BIT_AUTHENTICATION_ENABLED,
-          ConfigValueFactory.fromAnyRef(true))
-        .withValue(ExecConstants.BIT_AUTHENTICATION_MECHANISM,
-          ConfigValueFactory.fromAnyRef("kerberos"))
-        .withValue(ExecConstants.USE_LOGIN_PRINCIPAL,
-          ConfigValueFactory.fromAnyRef(true))
-        .withValue(ExecConstants.SERVICE_PRINCIPAL,
-          ConfigValueFactory.fromAnyRef(krbHelper.SERVER_PRINCIPAL))
-        .withValue(ExecConstants.SERVICE_KEYTAB_LOCATION,
-          ConfigValueFactory.fromAnyRef(krbHelper.serverKeytab.toString())));
-
-    final ScanResult result = ClassPathScanner.fromPrescan(newConfig);
+    final ScanResult result = ClassPathScanner.fromPrescan(cluster.config());
     final BootStrapContext c1 =
-      new BootStrapContext(newConfig, SystemOptionManager.createDefaultOptionDefinitions(), result);
+      new BootStrapContext(cluster.config(), SystemOptionManager.createDefaultOptionDefinitions(), result);
 
     final FragmentManager manager = setupFragmentContextAndManager(c1.getAllocator());
     when(workBus.getFragmentManager(Mockito.<FragmentHandle>any())).thenReturn(manager);
@@ -242,51 +201,42 @@ public class TestBitBitKerberos extends BaseTestQuery {
 
     final WorkerBee bee = mock(WorkerBee.class);
     final WorkEventBus workBus = mock(WorkEventBus.class);
-    newConfig = new DrillConfig(DrillConfig.create(cloneDefaultTestConfigProperties())
-      .withValue(ExecConstants.AUTHENTICATION_MECHANISMS,
-        ConfigValueFactory.fromIterable(Lists.newArrayList("kerberos")))
-      .withValue(ExecConstants.BIT_AUTHENTICATION_ENABLED,
-        ConfigValueFactory.fromAnyRef(true))
-      .withValue(ExecConstants.BIT_AUTHENTICATION_MECHANISM,
-        ConfigValueFactory.fromAnyRef("kerberos"))
-      .withValue(ExecConstants.BIT_ENCRYPTION_SASL_ENABLED,
-        ConfigValueFactory.fromAnyRef(true))
-      .withValue(ExecConstants.USE_LOGIN_PRINCIPAL,
-        ConfigValueFactory.fromAnyRef(true))
-      .withValue(ExecConstants.SERVICE_PRINCIPAL,
-        ConfigValueFactory.fromAnyRef(krbHelper.SERVER_PRINCIPAL))
-      .withValue(ExecConstants.SERVICE_KEYTAB_LOCATION,
-        ConfigValueFactory.fromAnyRef(krbHelper.serverKeytab.toString())));
-
-    final ScanResult result = ClassPathScanner.fromPrescan(newConfig);
-    final BootStrapContext c2 =
-      new BootStrapContext(newConfig, SystemOptionManager.createDefaultOptionDefinitions(), result);
-
-    final FragmentManager manager = setupFragmentContextAndManager(c2.getAllocator());
-    when(workBus.getFragmentManager(Mockito.<FragmentHandle>any())).thenReturn(manager);
-
-    final DataConnectionConfig config =
-      new DataConnectionConfig(c2.getAllocator(), c2, new DataServerRequestHandler(workBus, bee));
-    final DataServer server = new DataServer(config);
-
-    port = server.bind(port, true);
-    DrillbitEndpoint ep = DrillbitEndpoint.newBuilder().setAddress("localhost").setDataPort(port).build();
-    final DataConnectionManager connectionManager = new DataConnectionManager(ep, config);
-    final DataTunnel tunnel = new DataTunnel(connectionManager);
-    AtomicLong max = new AtomicLong(0);
-    try {
-      for (int i = 0; i < 40; i++) {
-        long t1 = System.currentTimeMillis();
-        tunnel.sendRecordBatch(new TimingOutcome(max),
-          new FragmentWritableBatch(false, QueryId.getDefaultInstance(), 1, 1, 1, 1,
-            getRandomBatch(c2.getAllocator(), 5000)));
+    try (
+      ClusterFixture cluster = defaultClusterConfig()
+        .configProperty(ExecConstants.BIT_ENCRYPTION_SASL_ENABLED, true)
+        .build();
+    ) {
+
+      final ScanResult result = ClassPathScanner.fromPrescan(cluster.config());
+      final BootStrapContext c2 =
+        new BootStrapContext(cluster.config(), SystemOptionManager.createDefaultOptionDefinitions(), result);
+
+      final FragmentManager manager = setupFragmentContextAndManager(c2.getAllocator());
+      when(workBus.getFragmentManager(Mockito.<FragmentHandle>any())).thenReturn(manager);
+
+      final DataConnectionConfig config =
+        new DataConnectionConfig(c2.getAllocator(), c2, new DataServerRequestHandler(workBus, bee));
+      final DataServer server = new DataServer(config);
+
+      port = server.bind(port, true);
+      DrillbitEndpoint ep = DrillbitEndpoint.newBuilder().setAddress("localhost").setDataPort(port).build();
+      final DataConnectionManager connectionManager = new DataConnectionManager(ep, config);
+      final DataTunnel tunnel = new DataTunnel(connectionManager);
+      AtomicLong max = new AtomicLong(0);
+      try {
+        for (int i = 0; i < 40; i++) {
+          long t1 = System.currentTimeMillis();
+          tunnel.sendRecordBatch(new TimingOutcome(max),
+            new FragmentWritableBatch(false, QueryId.getDefaultInstance(), 1, 1, 1, 1,
+              getRandomBatch(c2.getAllocator(), 5000)));
+        }
+        assertTrue(max.get() > 2700);
+        Thread.sleep(5000);
+      } finally {
+        server.close();
+        connectionManager.close();
+        c2.close();
       }
-      assertTrue(max.get() > 2700);
-      Thread.sleep(5000);
-    } finally {
-      server.close();
-      connectionManager.close();
-      c2.close();
     }
   }
 
@@ -297,79 +247,58 @@ public class TestBitBitKerberos extends BaseTestQuery {
     final WorkerBee bee = mock(WorkerBee.class);
     final WorkEventBus workBus = mock(WorkEventBus.class);
 
-    newConfig = new DrillConfig(DrillConfig.create(cloneDefaultTestConfigProperties())
-      .withValue(ExecConstants.AUTHENTICATION_MECHANISMS,
-        ConfigValueFactory.fromIterable(Lists.newArrayList("kerberos")))
-        .withValue(ExecConstants.BIT_AUTHENTICATION_ENABLED,
-          ConfigValueFactory.fromAnyRef(true))
-        .withValue(ExecConstants.BIT_AUTHENTICATION_MECHANISM,
-          ConfigValueFactory.fromAnyRef("kerberos"))
-        .withValue(ExecConstants.BIT_ENCRYPTION_SASL_ENABLED,
-          ConfigValueFactory.fromAnyRef(true))
-        .withValue(ExecConstants.BIT_ENCRYPTION_SASL_MAX_WRAPPED_SIZE,
-          ConfigValueFactory.fromAnyRef(100000))
-        .withValue(ExecConstants.USE_LOGIN_PRINCIPAL,
-          ConfigValueFactory.fromAnyRef(true))
-        .withValue(ExecConstants.SERVICE_PRINCIPAL,
-          ConfigValueFactory.fromAnyRef(krbHelper.SERVER_PRINCIPAL))
-        .withValue(ExecConstants.SERVICE_KEYTAB_LOCATION,
-          ConfigValueFactory.fromAnyRef(krbHelper.serverKeytab.toString())));
-
-    final ScanResult result = ClassPathScanner.fromPrescan(newConfig);
-    final BootStrapContext c2 =
-      new BootStrapContext(newConfig, SystemOptionManager.createDefaultOptionDefinitions(), result);
-
-    final FragmentManager manager = setupFragmentContextAndManager(c2.getAllocator());
-    when(workBus.getFragmentManager(Mockito.<FragmentHandle>any())).thenReturn(manager);
+    try (
+      ClusterFixture cluster = defaultClusterConfig()
+        .configProperty(ExecConstants.BIT_ENCRYPTION_SASL_ENABLED, true)
+        .configProperty(ExecConstants.BIT_ENCRYPTION_SASL_MAX_WRAPPED_SIZE, 100000)
+        .build();
+    ) {
 
-    final DataConnectionConfig config = new DataConnectionConfig(c2.getAllocator(), c2,
-      new DataServerRequestHandler(workBus, bee));
-    final DataServer server = new DataServer(config);
+      final ScanResult result = ClassPathScanner.fromPrescan(cluster.config());
+      final BootStrapContext c2 =
+        new BootStrapContext(cluster.config(), SystemOptionManager.createDefaultOptionDefinitions(), result);
 
-    port = server.bind(port, true);
-    final DrillbitEndpoint ep = DrillbitEndpoint.newBuilder().setAddress("localhost").setDataPort(port).build();
-    final DataConnectionManager connectionManager = new DataConnectionManager(ep, config);
-    final DataTunnel tunnel = new DataTunnel(connectionManager);
-    AtomicLong max = new AtomicLong(0);
+      final FragmentManager manager = setupFragmentContextAndManager(c2.getAllocator());
+      when(workBus.getFragmentManager(Mockito.<FragmentHandle>any())).thenReturn(manager);
 
-    try {
-      for (int i = 0; i < 40; i++) {
-        long t1 = System.currentTimeMillis();
-        tunnel.sendRecordBatch(new TimingOutcome(max),
-          new FragmentWritableBatch(false, QueryId.getDefaultInstance(), 1, 1, 1, 1,
-            getRandomBatch(c2.getAllocator(), 5000)));
+      final DataConnectionConfig config = new DataConnectionConfig(c2.getAllocator(), c2,
+        new DataServerRequestHandler(workBus, bee));
+      final DataServer server = new DataServer(config);
+
+      port = server.bind(port, true);
+      final DrillbitEndpoint ep = DrillbitEndpoint.newBuilder().setAddress("localhost").setDataPort(port).build();
+      final DataConnectionManager connectionManager = new DataConnectionManager(ep, config);
+      final DataTunnel tunnel = new DataTunnel(connectionManager);
+      AtomicLong max = new AtomicLong(0);
+
+      try {
+        for (int i = 0; i < 40; i++) {
+          long t1 = System.currentTimeMillis();
+          tunnel.sendRecordBatch(new TimingOutcome(max),
+            new FragmentWritableBatch(false, QueryId.getDefaultInstance(), 1, 1, 1, 1,
+              getRandomBatch(c2.getAllocator(), 5000)));
+        }
+        assertTrue(max.get() > 2700);
+        Thread.sleep(5000);
+      } catch (Exception | AssertionError ex) {
+        fail();
+      } finally {
+        server.close();
+        connectionManager.close();
+        c2.close();
       }
-      assertTrue(max.get() > 2700);
-      Thread.sleep(5000);
-    } catch (Exception | AssertionError ex) {
-      fail();
-    } finally {
-      server.close();
-      connectionManager.close();
-      c2.close();
     }
   }
 
   @Test
   public void failureEncryptionOnlyPlainMechanism() throws Exception {
-    try{
-      newConfig = new DrillConfig(DrillConfig.create(cloneDefaultTestConfigProperties())
-        .withValue(ExecConstants.AUTHENTICATION_MECHANISMS,
-          ConfigValueFactory.fromIterable(Lists.newArrayList("plain")))
-          .withValue(ExecConstants.BIT_AUTHENTICATION_ENABLED,
-            ConfigValueFactory.fromAnyRef(true))
-          .withValue(ExecConstants.BIT_AUTHENTICATION_MECHANISM,
-            ConfigValueFactory.fromAnyRef("kerberos"))
-          .withValue(ExecConstants.BIT_ENCRYPTION_SASL_ENABLED,
-            ConfigValueFactory.fromAnyRef(true))
-          .withValue(ExecConstants.USE_LOGIN_PRINCIPAL,
-            ConfigValueFactory.fromAnyRef(true))
-          .withValue(ExecConstants.SERVICE_PRINCIPAL,
-            ConfigValueFactory.fromAnyRef(krbHelper.SERVER_PRINCIPAL))
-          .withValue(ExecConstants.SERVICE_KEYTAB_LOCATION,
-            ConfigValueFactory.fromAnyRef(krbHelper.serverKeytab.toString())));
-
-      updateTestCluster(1, newConfig);
+    try {
+      defaultClusterConfig()
+        .configNonStringProperty(ExecConstants.AUTHENTICATION_MECHANISMS, Lists.newArrayList("plain"))
+        .configProperty(ExecConstants.BIT_ENCRYPTION_SASL_ENABLED, true)
+        .build()
+        .close();
+
       fail();
     } catch(Exception ex) {
       assertTrue(ex.getCause() instanceof DrillbitStartupException);
@@ -387,35 +316,25 @@ public class TestBitBitKerberos extends BaseTestQuery {
    */
   @Test
   public void localQuerySuccessWithWrongBitAuthConfig() throws Exception {
-
-    final Properties connectionProps = new Properties();
-    connectionProps.setProperty(DrillProperties.SERVICE_PRINCIPAL, krbHelper.SERVER_PRINCIPAL);
-    connectionProps.setProperty(DrillProperties.USER, krbHelper.CLIENT_PRINCIPAL);
-    connectionProps.setProperty(DrillProperties.KEYTAB, krbHelper.clientKeytab.getAbsolutePath());
-
-    newConfig = new DrillConfig(DrillConfig.create(cloneDefaultTestConfigProperties())
-        .withValue(ExecConstants.USER_AUTHENTICATION_ENABLED,
-            ConfigValueFactory.fromAnyRef(true))
-        .withValue(ExecConstants.USER_AUTHENTICATOR_IMPL,
-            ConfigValueFactory.fromAnyRef(UserAuthenticatorTestImpl.TYPE))
-        .withValue(ExecConstants.SERVICE_PRINCIPAL,
-            ConfigValueFactory.fromAnyRef(krbHelper.SERVER_PRINCIPAL))
-        .withValue(ExecConstants.SERVICE_KEYTAB_LOCATION,
-            ConfigValueFactory.fromAnyRef(krbHelper.serverKeytab.toString()))
-        .withValue(ExecConstants.AUTHENTICATION_MECHANISMS,
-            ConfigValueFactory.fromIterable(Lists.newArrayList("plain", "kerberos")))
-        .withValue(ExecConstants.BIT_AUTHENTICATION_ENABLED,
-            ConfigValueFactory.fromAnyRef(true))
-        .withValue(ExecConstants.BIT_AUTHENTICATION_MECHANISM,
-            ConfigValueFactory.fromAnyRef("kerberos"))
-        .withValue(ExecConstants.USE_LOGIN_PRINCIPAL,
-            ConfigValueFactory.fromAnyRef(false)));
-
-    updateTestCluster(1, newConfig, connectionProps);
-
-    // Run a query using the new client
-    final String query = getFile("queries/tpch/01.sql");
-    test(query);
+    try (
+      ClusterFixture cluster = defaultClusterConfig()
+        .configProperty(ExecConstants.USER_AUTHENTICATION_ENABLED, true)
+        .configProperty(ExecConstants.USER_AUTHENTICATOR_IMPL, UserAuthenticatorTestImpl.TYPE)
+        .configNonStringProperty(
+          ExecConstants.AUTHENTICATION_MECHANISMS,
+          Lists.newArrayList("plain", "kerberos")
+        )
+        .configProperty(ExecConstants.USE_LOGIN_PRINCIPAL, false)
+        .build();
+      ClientFixture client = cluster.clientBuilder()
+      .property(DrillProperties.SERVICE_PRINCIPAL, krbHelper.SERVER_PRINCIPAL)
+      .property(DrillProperties.USER, krbHelper.CLIENT_PRINCIPAL)
+      .property(DrillProperties.KEYTAB, krbHelper.clientKeytab.getAbsolutePath())
+      .build()
+    ) {
+      // Run a query using the new client
+      client.queryBuilder().sqlResource("queries/tpch/01.sql").run();
+    }
   }
 
   /**
@@ -430,36 +349,28 @@ public class TestBitBitKerberos extends BaseTestQuery {
    */
   @Test
   public void queryFailureWithWrongBitAuthConfig() throws Exception {
-    try{
-      final Properties connectionProps = new Properties();
-      connectionProps.setProperty(DrillProperties.SERVICE_PRINCIPAL, krbHelper.SERVER_PRINCIPAL);
-      connectionProps.setProperty(DrillProperties.USER, krbHelper.CLIENT_PRINCIPAL);
-      connectionProps.setProperty(DrillProperties.KEYTAB, krbHelper.clientKeytab.getAbsolutePath());
-
-      newConfig = new DrillConfig(DrillConfig.create(cloneDefaultTestConfigProperties())
-          .withValue(ExecConstants.USER_AUTHENTICATION_ENABLED,
-              ConfigValueFactory.fromAnyRef(true))
-          .withValue(ExecConstants.USER_AUTHENTICATOR_IMPL,
-              ConfigValueFactory.fromAnyRef(UserAuthenticatorTestImpl.TYPE))
-          .withValue(ExecConstants.SERVICE_PRINCIPAL,
-              ConfigValueFactory.fromAnyRef(krbHelper.SERVER_PRINCIPAL))
-          .withValue(ExecConstants.SERVICE_KEYTAB_LOCATION,
-              ConfigValueFactory.fromAnyRef(krbHelper.serverKeytab.toString()))
-          .withValue(ExecConstants.AUTHENTICATION_MECHANISMS,
-              ConfigValueFactory.fromIterable(Lists.newArrayList("plain", "kerberos")))
-          .withValue(ExecConstants.BIT_AUTHENTICATION_ENABLED,
-              ConfigValueFactory.fromAnyRef(true))
-          .withValue(ExecConstants.BIT_AUTHENTICATION_MECHANISM,
-              ConfigValueFactory.fromAnyRef("kerberos"))
-          .withValue(ExecConstants.USE_LOGIN_PRINCIPAL,
-              ConfigValueFactory.fromAnyRef(false)));
-
-      updateTestCluster(2, newConfig, connectionProps);
-
-      test("alter session set `planner.slice_target` = 10");
-      final String query = getFile("queries/tpch/01.sql");
-      test(query);
-      fail();
+    try {
+      try (
+        ClusterFixture cluster = defaultClusterConfig()
+          .clusterSize(2)
+          .configProperty(ExecConstants.USER_AUTHENTICATION_ENABLED, true)
+          .configProperty(ExecConstants.USER_AUTHENTICATOR_IMPL, UserAuthenticatorTestImpl.TYPE)
+          .configNonStringProperty(
+            ExecConstants.AUTHENTICATION_MECHANISMS,
+            Lists.newArrayList("plain", "kerberos")
+          )
+          .configProperty(ExecConstants.USE_LOGIN_PRINCIPAL, false)
+          .build();
+        ClientFixture client = cluster.clientBuilder()
+        .property(DrillProperties.SERVICE_PRINCIPAL, krbHelper.SERVER_PRINCIPAL)
+        .property(DrillProperties.USER, krbHelper.CLIENT_PRINCIPAL)
+        .property(DrillProperties.KEYTAB, krbHelper.clientKeytab.getAbsolutePath())
+        .build()
+      ) {
+        client.alterSession("planner.slice_target", 10);
+        client.queryBuilder().sqlResource("queries/tpch/01.sql").run();
+        fail();
+      }
     } catch(Exception ex) {
       assertTrue(ex instanceof UserRemoteException);
       assertTrue(((UserRemoteException)ex).getErrorType() == UserBitShared.DrillPBError.ErrorType.CONNECTION);
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/user/security/TestUserBitKerberos.java b/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/user/security/TestUserBitKerberos.java
index 755ae06174..0777ebdbe9 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/user/security/TestUserBitKerberos.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/user/security/TestUserBitKerberos.java
@@ -27,6 +27,7 @@ import org.apache.drill.exec.rpc.data.DataRpcMetrics;
 import org.apache.drill.exec.rpc.security.KerberosHelper;
 import org.apache.drill.exec.rpc.user.UserRpcMetrics;
 import org.apache.drill.exec.rpc.user.security.testing.UserAuthenticatorTestImpl;
+import org.apache.drill.test.BaseDirTestWatcher;
 import org.apache.drill.test.ClientFixture;
 import org.apache.drill.test.ClusterFixture;
 import org.apache.drill.test.ClusterFixtureBuilder;
@@ -51,7 +52,7 @@ public class TestUserBitKerberos extends ClusterTest {
   @BeforeClass
   public static void setupTest() throws Exception {
     krbHelper = new KerberosHelper(TestUserBitKerberos.class.getSimpleName(), null);
-    krbHelper.setupKdc(dirTestWatcher.getTmpDir());
+    krbHelper.setupKdc(BaseDirTestWatcher.createTempDir(dirTestWatcher.getTmpDir()));
     cluster = defaultClusterConfig().build();
   }
 
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/user/security/TestUserBitKerberosEncryption.java b/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/user/security/TestUserBitKerberosEncryption.java
index b0449d50be..7324af4692 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/user/security/TestUserBitKerberosEncryption.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/user/security/TestUserBitKerberosEncryption.java
@@ -18,6 +18,7 @@
 package org.apache.drill.exec.rpc.user.security;
 
 import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
+import org.apache.drill.test.BaseDirTestWatcher;
 import org.apache.drill.test.ClientFixture;
 import org.apache.drill.test.ClusterFixture;
 import org.apache.drill.test.ClusterFixtureBuilder;
@@ -56,7 +57,7 @@ public class TestUserBitKerberosEncryption extends ClusterTest {
   @BeforeClass
   public static void setupTest() throws Exception {
     krbHelper = new KerberosHelper(TestUserBitKerberosEncryption.class.getSimpleName(), null);
-    krbHelper.setupKdc(dirTestWatcher.getTmpDir());
+    krbHelper.setupKdc(BaseDirTestWatcher.createTempDir(dirTestWatcher.getTmpDir()));
     cluster = defaultClusterConfig().build();
   }
 
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/server/rest/spnego/TestDrillSpnegoAuthenticator.java b/exec/java-exec/src/test/java/org/apache/drill/exec/server/rest/spnego/TestDrillSpnegoAuthenticator.java
index 22b574889e..ad93742dc3 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/server/rest/spnego/TestDrillSpnegoAuthenticator.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/server/rest/spnego/TestDrillSpnegoAuthenticator.java
@@ -81,7 +81,7 @@ public class TestDrillSpnegoAuthenticator extends BaseTest {
   @BeforeClass
   public static void setupTest() throws Exception {
     spnegoHelper = new KerberosHelper(TestDrillSpnegoAuthenticator.class.getSimpleName(), primaryName);
-    spnegoHelper.setupKdc(dirTestWatcher.getTmpDir());
+    spnegoHelper.setupKdc(BaseDirTestWatcher.createTempDir(dirTestWatcher.getTmpDir()));
 
     // (1) Refresh Kerberos config.
     // This disabled call to an unsupported internal API does not appear to be
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/server/rest/spnego/TestSpnegoAuthentication.java b/exec/java-exec/src/test/java/org/apache/drill/exec/server/rest/spnego/TestSpnegoAuthentication.java
index fb79f173a2..0d3ae96e0a 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/server/rest/spnego/TestSpnegoAuthentication.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/server/rest/spnego/TestSpnegoAuthentication.java
@@ -76,7 +76,7 @@ public class TestSpnegoAuthentication extends BaseTest {
   @BeforeClass
   public static void setupTest() throws Exception {
     spnegoHelper = new KerberosHelper(TestSpnegoAuthentication.class.getSimpleName(), primaryName);
-    spnegoHelper.setupKdc(dirTestWatcher.getTmpDir());
+    spnegoHelper.setupKdc(BaseDirTestWatcher.createTempDir(dirTestWatcher.getTmpDir()));
 
     // (1) Refresh Kerberos config.
     // This disabled call to an unsupported internal API does not appear to be
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/server/rest/spnego/TestSpnegoConfig.java b/exec/java-exec/src/test/java/org/apache/drill/exec/server/rest/spnego/TestSpnegoConfig.java
index 3d6a8cc211..baaac985dc 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/server/rest/spnego/TestSpnegoConfig.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/server/rest/spnego/TestSpnegoConfig.java
@@ -56,7 +56,7 @@ public class TestSpnegoConfig extends BaseTest {
   @BeforeClass
   public static void setupTest() throws Exception {
     spnegoHelper = new KerberosHelper(TestSpnegoConfig.class.getSimpleName(), primaryName);
-    spnegoHelper.setupKdc(dirTestWatcher.getTmpDir());
+    spnegoHelper.setupKdc(BaseDirTestWatcher.createTempDir(dirTestWatcher.getTmpDir()));
 
     // (1) Refresh Kerberos config.
     // This disabled call to an unsupported internal API does not appear to be
diff --git a/exec/java-exec/src/test/java/org/apache/drill/test/QueryBuilder.java b/exec/java-exec/src/test/java/org/apache/drill/test/QueryBuilder.java
index 41b6ecea11..ec0471e255 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/test/QueryBuilder.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/test/QueryBuilder.java
@@ -281,19 +281,31 @@ public class QueryBuilder {
   }
 
   /**
-   * Run a query contained in a resource file.
+   * Parse a single SQL statement (with optional ending semi-colon) from
+   * the resource provided.
    *
-   * @param resource Name of the resource
+   * @param resource the resource containing exactly one SQL statement, with
+   * optional ending semi-colon
    * @return this builder
    */
-  public QueryBuilder sqlResource(String resource) {
-    sql(ClusterFixture.loadResource(resource));
-    return this;
+  public QueryBuilder sqlResource(String resource) throws IOException {
+    String script = ClusterFixture.loadResource(resource);
+    StatementParser parser = new StatementParser(script);
+    String sql = parser.parseNext();
+    if (sql == null) {
+      throw new IllegalArgumentException("No query found");
+    }
+    return sql(sql);
   }
 
-  public QueryBuilder sqlResource(String resource, Object... args) {
-    sql(ClusterFixture.loadResource(resource), args);
-    return this;
+  public QueryBuilder sqlResource(String resource, Object... args) throws IOException {
+    String script = ClusterFixture.loadResource(resource);
+    StatementParser parser = new StatementParser(script);
+    String sql = parser.parseNext();
+    if (sql == null) {
+      throw new IllegalArgumentException("No query found");
+    }
+    return sql(sql, args);
   }
 
   public QueryBuilder physicalResource(String resource) {