You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@calcite.apache.org by el...@apache.org on 2019/10/10 20:30:55 UTC

[calcite-avatica] branch master updated: [CALCITE-3384] Support Kerberos-authentication using SPNEGO over HTTPS

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

elserj pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/calcite-avatica.git


The following commit(s) were added to refs/heads/master by this push:
     new da71884  [CALCITE-3384] Support Kerberos-authentication using SPNEGO over HTTPS
da71884 is described below

commit da71884193726c82df1ae5396a4b1a8dec1ab765
Author: Istvan Toth <st...@stoty.hu>
AuthorDate: Thu Oct 10 16:22:56 2019 -0400

    [CALCITE-3384] Support Kerberos-authentication using SPNEGO over HTTPS
    
    Closes #113
    
    Signed-off-by: Josh Elser <el...@apache.org>
---
 .../remote/AvaticaCommonsHttpClientImpl.java       |   4 +-
 .../remote/AvaticaCommonsHttpClientSpnegoImpl.java |  72 +++------
 .../remote/AvaticaHttpClientFactoryImpl.java       |   7 +-
 .../apache/calcite/avatica/server/HttpServer.java  |   3 -
 .../apache/calcite/avatica/AvaticaSpnegoTest.java  |  87 ++++++-----
 .../{SslDriverTest.java => HttpBaseTest.java}      | 105 ++++---------
 .../org/apache/calcite/avatica/SslDriverTest.java  | 171 +++------------------
 7 files changed, 129 insertions(+), 320 deletions(-)

diff --git a/core/src/main/java/org/apache/calcite/avatica/remote/AvaticaCommonsHttpClientImpl.java b/core/src/main/java/org/apache/calcite/avatica/remote/AvaticaCommonsHttpClientImpl.java
index ba9f9c6..e2bda3a 100644
--- a/core/src/main/java/org/apache/calcite/avatica/remote/AvaticaCommonsHttpClientImpl.java
+++ b/core/src/main/java/org/apache/calcite/avatica/remote/AvaticaCommonsHttpClientImpl.java
@@ -102,7 +102,7 @@ public class AvaticaCommonsHttpClientImpl implements AvaticaHttpClient,
     initializeClient();
   }
 
-  private void initializeClient() {
+  protected void initializeClient() {
     socketFactoryRegistry = this.configureSocketFactories();
     configureConnectionPool(socketFactoryRegistry);
     this.authCache = new BasicAuthCache();
@@ -193,7 +193,7 @@ public class AvaticaCommonsHttpClientImpl implements AvaticaHttpClient,
     }
   }
 
-  public byte[] send(byte[] request) {
+  @Override public byte[] send(byte[] request) {
     while (true) {
       HttpClientContext context = HttpClientContext.create();
 
diff --git a/core/src/main/java/org/apache/calcite/avatica/remote/AvaticaCommonsHttpClientSpnegoImpl.java b/core/src/main/java/org/apache/calcite/avatica/remote/AvaticaCommonsHttpClientSpnegoImpl.java
index c1ca658..635c711 100644
--- a/core/src/main/java/org/apache/calcite/avatica/remote/AvaticaCommonsHttpClientSpnegoImpl.java
+++ b/core/src/main/java/org/apache/calcite/avatica/remote/AvaticaCommonsHttpClientSpnegoImpl.java
@@ -16,7 +16,6 @@
  */
 package org.apache.calcite.avatica.remote;
 
-import org.apache.http.HttpHost;
 import org.apache.http.auth.AuthSchemeProvider;
 import org.apache.http.auth.AuthScope;
 import org.apache.http.auth.Credentials;
@@ -25,16 +24,13 @@ import org.apache.http.client.config.AuthSchemes;
 import org.apache.http.client.methods.CloseableHttpResponse;
 import org.apache.http.client.methods.HttpPost;
 import org.apache.http.client.protocol.HttpClientContext;
-import org.apache.http.config.Lookup;
+import org.apache.http.config.Registry;
 import org.apache.http.config.RegistryBuilder;
+import org.apache.http.conn.socket.ConnectionSocketFactory;
 import org.apache.http.entity.ByteArrayEntity;
 import org.apache.http.entity.ContentType;
 import org.apache.http.impl.auth.SPNegoSchemeFactory;
-import org.apache.http.impl.client.BasicAuthCache;
 import org.apache.http.impl.client.BasicCredentialsProvider;
-import org.apache.http.impl.client.CloseableHttpClient;
-import org.apache.http.impl.client.HttpClients;
-import org.apache.http.impl.conn.PoolingHttpClientConnectionManager;
 import org.apache.http.util.EntityUtils;
 
 import org.ietf.jgss.GSSCredential;
@@ -42,36 +38,24 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.net.HttpURLConnection;
-import java.net.URI;
-import java.net.URISyntaxException;
 import java.net.URL;
 import java.security.Principal;
-import java.util.Objects;
-
 /**
  * Implementation of an AvaticaHttpClient which uses SPNEGO.
+ *
+ * ( at this point it could probably be just merged back into AvaticaCommonsHttpClientImpl)
  */
-public class AvaticaCommonsHttpClientSpnegoImpl implements AvaticaHttpClient {
+public class AvaticaCommonsHttpClientSpnegoImpl extends AvaticaCommonsHttpClientImpl {
   private static final Logger LOG = LoggerFactory
       .getLogger(AvaticaCommonsHttpClientSpnegoImpl.class);
 
   public static final String CACHED_CONNECTIONS_MAX_KEY = "avatica.http.spnego.max_cached";
-  public static final String CACHED_CONNECTIONS_MAX_DEFAULT = "100";
   public static final String CACHED_CONNECTIONS_MAX_PER_ROUTE_KEY =
       "avatica.http.spnego.max_per_route";
-  public static final String CACHED_CONNECTIONS_MAX_PER_ROUTE_DEFAULT = "25";
 
   private static final boolean USE_CANONICAL_HOSTNAME = true;
   private static final boolean STRIP_PORT_ON_SERVER_LOOKUP = true;
 
-  final URL url;
-  final HttpHost host;
-  final PoolingHttpClientConnectionManager pool;
-  final Lookup<AuthSchemeProvider> authRegistry;
-  final BasicCredentialsProvider credentialsProvider;
-  final BasicAuthCache authCache;
-  final CloseableHttpClient client;
-
   /**
    * Constructs an http client with the expectation that the user is already logged in with their
    * Kerberos identity via JAAS.
@@ -89,39 +73,39 @@ public class AvaticaCommonsHttpClientSpnegoImpl implements AvaticaHttpClient {
    * @param credential The GSS credentials
    */
   public AvaticaCommonsHttpClientSpnegoImpl(URL url, GSSCredential credential) {
-    this.url = Objects.requireNonNull(url);
+    super(url);
+    setGSSCredential(credential);
+  }
 
-    pool = new PoolingHttpClientConnectionManager();
-    // Increase max total connection to 100
+  @Override protected void configureConnectionPool(Registry<ConnectionSocketFactory> registry) {
+    super.configureConnectionPool(registry);
+    //For backwards compatibility, override the standard values if set
     final String maxCnxns =
-        System.getProperty(CACHED_CONNECTIONS_MAX_KEY, CACHED_CONNECTIONS_MAX_DEFAULT);
-    pool.setMaxTotal(Integer.parseInt(maxCnxns));
-    // Increase default max connection per route to 25
-    final String maxCnxnsPerRoute = System.getProperty(CACHED_CONNECTIONS_MAX_PER_ROUTE_KEY,
-        CACHED_CONNECTIONS_MAX_PER_ROUTE_DEFAULT);
-    pool.setDefaultMaxPerRoute(Integer.parseInt(maxCnxnsPerRoute));
-
-    this.host = new HttpHost(url.getHost(), url.getPort());
+        System.getProperty(CACHED_CONNECTIONS_MAX_KEY);
+    if (maxCnxns != null) {
+      pool.setMaxTotal(Integer.parseInt(maxCnxns));
+    }
+    //For backwards compatibility, override the standard values if set
+    final String maxCnxnsPerRoute = System.getProperty(CACHED_CONNECTIONS_MAX_PER_ROUTE_KEY);
+    if (maxCnxnsPerRoute != null) {
+      pool.setDefaultMaxPerRoute(Integer.parseInt(maxCnxnsPerRoute));
+    }
+  }
 
+  public void setGSSCredential(GSSCredential credential) {
     this.authRegistry = RegistryBuilder.<AuthSchemeProvider>create().register(AuthSchemes.SPNEGO,
         new SPNegoSchemeFactory(STRIP_PORT_ON_SERVER_LOOKUP, USE_CANONICAL_HOSTNAME)).build();
 
     this.credentialsProvider = new BasicCredentialsProvider();
     if (null != credential) {
       // Non-null credential should be used directly with KerberosCredentials.
+      // This is never set by the JDBC driver, nor the tests
       this.credentialsProvider.setCredentials(AuthScope.ANY, new KerberosCredentials(credential));
     } else {
       // A null credential implies that the user is logged in via JAAS using the
       // java.security.auth.login.config system property
       this.credentialsProvider.setCredentials(AuthScope.ANY, EmptyCredentials.INSTANCE);
     }
-
-    this.authCache = new BasicAuthCache();
-
-    // A single thread-safe HttpClient, pooling connections via the ConnectionManager
-    this.client = HttpClients.custom()
-        .setDefaultAuthSchemeRegistry(authRegistry)
-        .setConnectionManager(pool).build();
   }
 
   @Override public byte[] send(byte[] request) {
@@ -135,7 +119,7 @@ public class AvaticaCommonsHttpClientSpnegoImpl implements AvaticaHttpClient {
     ByteArrayEntity entity = new ByteArrayEntity(request, ContentType.APPLICATION_OCTET_STREAM);
 
     // Create the client with the AuthSchemeRegistry and manager
-    HttpPost post = new HttpPost(toURI(url));
+    HttpPost post = new HttpPost(uri);
     post.setEntity(entity);
 
     try (CloseableHttpResponse response = client.execute(post, context)) {
@@ -154,14 +138,6 @@ public class AvaticaCommonsHttpClientSpnegoImpl implements AvaticaHttpClient {
     }
   }
 
-  private static URI toURI(URL url) throws RuntimeException {
-    try {
-      return url.toURI();
-    } catch (URISyntaxException e) {
-      throw new RuntimeException(e);
-    }
-  }
-
   /**
    * A credentials implementation which returns null.
    */
diff --git a/core/src/main/java/org/apache/calcite/avatica/remote/AvaticaHttpClientFactoryImpl.java b/core/src/main/java/org/apache/calcite/avatica/remote/AvaticaHttpClientFactoryImpl.java
index 8f305dc..596e023 100644
--- a/core/src/main/java/org/apache/calcite/avatica/remote/AvaticaHttpClientFactoryImpl.java
+++ b/core/src/main/java/org/apache/calcite/avatica/remote/AvaticaHttpClientFactoryImpl.java
@@ -66,9 +66,6 @@ public class AvaticaHttpClientFactoryImpl implements AvaticaHttpClientFactory {
     }
 
     AvaticaHttpClient client = instantiateClient(className, url);
-    if (null != kerberosUtil) {
-      client = new DoAsAvaticaHttpClient(client, kerberosUtil);
-    }
 
     if (client instanceof TrustStoreConfigurable) {
       File truststore = config.truststore();
@@ -126,6 +123,10 @@ public class AvaticaHttpClientFactoryImpl implements AvaticaHttpClientFactory {
       }
     }
 
+    if (null != kerberosUtil) {
+      client = new DoAsAvaticaHttpClient(client, kerberosUtil);
+    }
+
     return client;
   }
 
diff --git a/server/src/main/java/org/apache/calcite/avatica/server/HttpServer.java b/server/src/main/java/org/apache/calcite/avatica/server/HttpServer.java
index 2e152f7..a4d5dcb 100644
--- a/server/src/main/java/org/apache/calcite/avatica/server/HttpServer.java
+++ b/server/src/main/java/org/apache/calcite/avatica/server/HttpServer.java
@@ -779,9 +779,6 @@ public class HttpServer {
         handler = buildHandler(this, serverConfig);
         break;
       case SPNEGO:
-        if (usingTLS) {
-          throw new IllegalArgumentException("TLS has not been tested wtih SPNEGO");
-        }
         if (null != keytab) {
           LOG.debug("Performing Kerberos login with {} as {}", keytab, kerberosPrincipal);
           subject = loginViaKerberos(this);
diff --git a/server/src/test/java/org/apache/calcite/avatica/AvaticaSpnegoTest.java b/server/src/test/java/org/apache/calcite/avatica/AvaticaSpnegoTest.java
index 6cc2fed..a5ce879 100644
--- a/server/src/test/java/org/apache/calcite/avatica/AvaticaSpnegoTest.java
+++ b/server/src/test/java/org/apache/calcite/avatica/AvaticaSpnegoTest.java
@@ -16,9 +16,7 @@
  */
 package org.apache.calcite.avatica;
 
-import org.apache.calcite.avatica.jdbc.JdbcMeta;
 import org.apache.calcite.avatica.remote.Driver;
-import org.apache.calcite.avatica.remote.LocalService;
 import org.apache.calcite.avatica.server.AvaticaJaasKrbUtil;
 import org.apache.calcite.avatica.server.HttpServer;
 
@@ -32,6 +30,7 @@ import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 import org.junit.runners.Parameterized.Parameters;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -43,7 +42,6 @@ import java.sql.ResultSet;
 import java.sql.Statement;
 import java.util.ArrayList;
 import java.util.List;
-import java.util.Objects;
 
 import javax.security.auth.Subject;
 
@@ -55,12 +53,9 @@ import static org.junit.Assert.assertTrue;
  * End to end test case for SPNEGO with Avatica.
  */
 @RunWith(Parameterized.class)
-public class AvaticaSpnegoTest {
+public class AvaticaSpnegoTest extends HttpBaseTest {
   private static final Logger LOG = LoggerFactory.getLogger(AvaticaSpnegoTest.class);
 
-  private static final ConnectionSpec CONNECTION_SPEC = ConnectionSpec.HSQLDB;
-  private static final List<HttpServer> SERVERS_TO_STOP = new ArrayList<>();
-
   private static SimpleKdcServer kdc;
   private static KrbConfig clientConfig;
   private static File keytabDir;
@@ -72,6 +67,9 @@ public class AvaticaSpnegoTest {
   private static boolean isKdcStarted = false;
 
   private static void setupKdc() throws Exception {
+    if (isKdcStarted) {
+      return;
+    }
     kdc = new SimpleKdcServer();
     File target = new File(System.getProperty("user.dir"), "target");
     assertTrue(target.exists());
@@ -114,12 +112,7 @@ public class AvaticaSpnegoTest {
     //System.setProperty("sun.security.krb5.debug", "true");
   }
 
-  @AfterClass public static void stopKdc() throws Exception {
-    for (HttpServer server : SERVERS_TO_STOP) {
-      server.stop();
-    }
-    SERVERS_TO_STOP.clear();
-
+  @AfterClass public static void stopKdc() throws KrbException {
     if (isKdcStarted) {
       LOG.info("Stopping KDC on {}", kdcPort);
       kdc.stop();
@@ -151,40 +144,57 @@ public class AvaticaSpnegoTest {
   @Parameters public static List<Object[]> parameters() throws Exception {
     final ArrayList<Object[]> parameters = new ArrayList<>();
 
+    setupClass();
+
     // Start the KDC
     setupKdc();
 
-    // Create a LocalService around HSQLDB
-    final JdbcMeta jdbcMeta = new JdbcMeta(CONNECTION_SPEC.url,
-        CONNECTION_SPEC.username, CONNECTION_SPEC.password);
-    final LocalService localService = new LocalService(jdbcMeta);
-
-    for (Driver.Serialization serialization : new Driver.Serialization[] {
-        Driver.Serialization.JSON, Driver.Serialization.PROTOBUF}) {
-      // Build and start the server
-      HttpServer httpServer = new HttpServer.Builder()
-          .withPort(0)
-          .withAutomaticLogin(serverKeytab)
-          .withSpnego(SpnegoTestUtil.SERVER_PRINCIPAL, SpnegoTestUtil.REALM)
-          .withHandler(localService, serialization)
-          .build();
-      httpServer.start();
-      SERVERS_TO_STOP.add(httpServer);
-
-      final String url = "jdbc:avatica:remote:url=http://" + SpnegoTestUtil.KDC_HOST + ":"
-          + httpServer.getPort() + ";authentication=SPNEGO;serialization=" + serialization;
-      LOG.info("JDBC URL {}", url);
-
-      parameters.add(new Object[] {url});
+    for (boolean tls : new Boolean[] {false, true}) {
+      for (Driver.Serialization serialization : new Driver.Serialization[] {
+          Driver.Serialization.JSON, Driver.Serialization.PROTOBUF}) {
+        if (tls && System.getProperty("java.vendor").contains("IBM")) {
+          // Skip TLS testing on IBM Java due the combination of:
+          // - Jetty 9.4.12+ ignores SSL_* ciphers due to security - eclipse/jetty.project#2807
+          // - IBM uses SSL_* cipher names for ALL ciphers not following RFC cipher names
+          //   See eclipse/jetty.project#2807 for details
+          LOG.info("Skipping HTTPS test on IBM Java");
+          parameters.add(new Object[] {null});
+          continue;
+        }
+
+        // Build and start the server
+        HttpServer.Builder httpServerBuilder = new HttpServer.Builder();
+        if (tls) {
+          httpServerBuilder = httpServerBuilder
+              .withTLS(KEYSTORE, KEYSTORE_PASSWORD, KEYSTORE, KEYSTORE_PASSWORD);
+        }
+        HttpServer httpServer = httpServerBuilder
+            .withPort(0)
+            .withAutomaticLogin(serverKeytab)
+            .withSpnego(SpnegoTestUtil.SERVER_PRINCIPAL, SpnegoTestUtil.REALM)
+            .withHandler(localService, serialization)
+            .build();
+        httpServer.start();
+        SERVERS_TO_STOP.add(httpServer);
+
+        String url = "jdbc:avatica:remote:url=" + (tls ? "https://" : "http://")
+            + SpnegoTestUtil.KDC_HOST + ":" + httpServer.getPort()
+            + ";authentication=SPNEGO;serialization=" + serialization;
+        if (tls) {
+          url += ";truststore=" + KEYSTORE.getAbsolutePath()
+              + ";truststore_password=" + KEYSTORE_PASSWORD;
+        }
+        LOG.info("JDBC URL {}", url);
+
+        parameters.add(new Object[] {url});
+      }
     }
 
     return parameters;
   }
 
-  private final String jdbcUrl;
-
   public AvaticaSpnegoTest(String jdbcUrl) {
-    this.jdbcUrl = Objects.requireNonNull(jdbcUrl);
+    super(jdbcUrl);
   }
 
   @Test public void testAuthenticatedClient() throws Exception {
@@ -240,6 +250,7 @@ public class AvaticaSpnegoTest {
       assertEquals(3, results.getInt(1));
     }
   }
+
 }
 
 // End AvaticaSpnegoTest.java
diff --git a/server/src/test/java/org/apache/calcite/avatica/SslDriverTest.java b/server/src/test/java/org/apache/calcite/avatica/HttpBaseTest.java
similarity index 64%
copy from server/src/test/java/org/apache/calcite/avatica/SslDriverTest.java
copy to server/src/test/java/org/apache/calcite/avatica/HttpBaseTest.java
index 891cf69..7f16ce0 100644
--- a/server/src/test/java/org/apache/calcite/avatica/SslDriverTest.java
+++ b/server/src/test/java/org/apache/calcite/avatica/HttpBaseTest.java
@@ -17,11 +17,12 @@
 package org.apache.calcite.avatica;
 
 import org.apache.calcite.avatica.jdbc.JdbcMeta;
-import org.apache.calcite.avatica.remote.Driver;
 import org.apache.calcite.avatica.remote.LocalService;
 import org.apache.calcite.avatica.server.HttpServer;
 import org.apache.calcite.avatica.util.DateTimeUtils;
 
+import org.apache.kerby.kerberos.kerb.KrbException;
+
 import org.bouncycastle.asn1.x500.X500Name;
 import org.bouncycastle.asn1.x500.style.IETFUtils;
 import org.bouncycastle.asn1.x500.style.RFC4519Style;
@@ -35,12 +36,9 @@ import org.bouncycastle.cert.jcajce.JcaX509v3CertificateBuilder;
 import org.bouncycastle.jce.provider.BouncyCastleProvider;
 import org.bouncycastle.operator.OperatorCreationException;
 import org.bouncycastle.operator.jcajce.JcaContentSignerBuilder;
+
 import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-import org.junit.runners.Parameterized.Parameters;
+import org.junit.Before;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -58,34 +56,31 @@ import java.security.Security;
 import java.security.cert.Certificate;
 import java.security.cert.CertificateException;
 import java.security.cert.X509Certificate;
-import java.sql.Connection;
-import java.sql.DriverManager;
-import java.sql.ResultSet;
-import java.sql.Statement;
+import java.sql.SQLException;
 import java.util.ArrayList;
 import java.util.Calendar;
 import java.util.List;
-import java.util.Objects;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
-import static org.junit.Assume.assumeFalse;
+import static org.junit.Assume.assumeNotNull;
 
 /**
- * Test case for Avatica with TLS connectors.
+ * Common Base for HTTP End2End Tests
  */
-@RunWith(Parameterized.class)
-public class SslDriverTest {
-  private static final Logger LOG = LoggerFactory.getLogger(SslDriverTest.class);
+public abstract class HttpBaseTest {
+  protected static final Logger LOG = LoggerFactory.getLogger(HttpBaseTest.class);
+
+  protected static final String KEYSTORE_PASSWORD = "avaticasecret";
+  protected static final ConnectionSpec CONNECTION_SPEC = ConnectionSpec.HSQLDB;
+  protected static final List<HttpServer> SERVERS_TO_STOP = new ArrayList<>();
 
-  private static final String KEYSTORE_PASSWORD = "avaticasecret";
-  private static final ConnectionSpec CONNECTION_SPEC = ConnectionSpec.HSQLDB;
-  private static final List<HttpServer> SERVERS_TO_STOP = new ArrayList<>();
+  protected static final File TARGET_DIR = new File(System.getProperty("user.dir"), "target");
+  protected static final File KEYSTORE = new File(TARGET_DIR, "avatica-test.jks");
+  protected static LocalService localService;
 
-  @Parameters public static List<Object[]> parameters() throws Exception {
-    final ArrayList<Object[]> parameters = new ArrayList<>();
+  protected final String jdbcUrl;
 
+  public static void setupClass() throws SQLException {
     // Create a self-signed cert
     File target = new File(System.getProperty("user.dir"), "target");
     File keystore = new File(target, "avatica-test.jks");
@@ -95,67 +90,27 @@ public class SslDriverTest {
     new CertTool().createSelfSignedCert(keystore, "avatica", KEYSTORE_PASSWORD);
 
     // Create a LocalService around HSQLDB
-    final JdbcMeta jdbcMeta = new JdbcMeta(CONNECTION_SPEC.url,
+    JdbcMeta jdbcMeta;
+    jdbcMeta = null;
+    jdbcMeta = new JdbcMeta(CONNECTION_SPEC.url,
         CONNECTION_SPEC.username, CONNECTION_SPEC.password);
-    final LocalService localService = new LocalService(jdbcMeta);
-
-    for (Driver.Serialization serialization : new Driver.Serialization[] {
-        Driver.Serialization.JSON, Driver.Serialization.PROTOBUF}) {
-      // Build and start the server, using TLS
-      HttpServer httpServer = new HttpServer.Builder()
-          .withPort(0)
-          .withTLS(keystore, KEYSTORE_PASSWORD, keystore, KEYSTORE_PASSWORD)
-          .withHandler(localService, serialization)
-          .build();
-      httpServer.start();
-      SERVERS_TO_STOP.add(httpServer);
-
-      final String url = "jdbc:avatica:remote:url=https://localhost:" + httpServer.getPort()
-          + ";serialization=" + serialization + ";truststore=" + keystore.getAbsolutePath()
-          + ";truststore_password=" + KEYSTORE_PASSWORD;
-      LOG.info("JDBC URL {}", url);
-
-      parameters.add(new Object[] {url});
-    }
-
-    return parameters;
+    localService = new LocalService(jdbcMeta);
   }
 
-  @BeforeClass public static void setupClass() {
-    // Skip TLS testing on IBM Java due the combination of:
-    // - Jetty 9.4.12+ ignores SSL_* ciphers due to security - eclipse/jetty.project#2807
-    // - IBM uses SSL_* cipher names for ALL ciphers not following RFC cipher names
-    //   See eclipse/jetty.project#2807 for details
-    assumeFalse(System.getProperty("java.vendor").contains("IBM"));
-  }
-
-  @AfterClass public static void stopKdc() {
+  @AfterClass public static void stopServers() throws KrbException {
     for (HttpServer server : SERVERS_TO_STOP) {
       server.stop();
     }
+    SERVERS_TO_STOP.clear();
   }
 
-  private final String jdbcUrl;
-
-  public SslDriverTest(String jdbcUrl) {
-    this.jdbcUrl = Objects.requireNonNull(jdbcUrl);
+  @Before public void checkUrl() {
+    //We signal that we skip the test because of the IBM Java issue by specifying a Null URL
+    assumeNotNull(jdbcUrl);
   }
 
-  @Test
-  public void testReadWrite() throws Exception {
-    final String tableName = "testReadWrite";
-    try (Connection conn = DriverManager.getConnection(jdbcUrl);
-        Statement stmt = conn.createStatement()) {
-      assertFalse(stmt.execute("DROP TABLE IF EXISTS " + tableName));
-      assertFalse(stmt.execute("CREATE TABLE " + tableName + "(pk integer)"));
-      assertEquals(1, stmt.executeUpdate("INSERT INTO " + tableName + " VALUES(1)"));
-      assertEquals(1, stmt.executeUpdate("INSERT INTO " + tableName + " VALUES(2)"));
-      assertEquals(1, stmt.executeUpdate("INSERT INTO " + tableName + " VALUES(3)"));
-
-      ResultSet results = stmt.executeQuery("SELECT count(1) FROM " + tableName);
-      assertTrue(results.next());
-      assertEquals(3, results.getInt(1));
-    }
+  public HttpBaseTest(String jdbcUrl) {
+    this.jdbcUrl = jdbcUrl;
   }
 
   /**
@@ -230,4 +185,4 @@ public class SslDriverTest {
   }
 }
 
-// End SslDriverTest.java
+// End HttpBaseTest.java
diff --git a/server/src/test/java/org/apache/calcite/avatica/SslDriverTest.java b/server/src/test/java/org/apache/calcite/avatica/SslDriverTest.java
index 891cf69..289b0c9 100644
--- a/server/src/test/java/org/apache/calcite/avatica/SslDriverTest.java
+++ b/server/src/test/java/org/apache/calcite/avatica/SslDriverTest.java
@@ -16,102 +16,61 @@
  */
 package org.apache.calcite.avatica;
 
-import org.apache.calcite.avatica.jdbc.JdbcMeta;
 import org.apache.calcite.avatica.remote.Driver;
-import org.apache.calcite.avatica.remote.LocalService;
 import org.apache.calcite.avatica.server.HttpServer;
-import org.apache.calcite.avatica.util.DateTimeUtils;
 
-import org.bouncycastle.asn1.x500.X500Name;
-import org.bouncycastle.asn1.x500.style.IETFUtils;
-import org.bouncycastle.asn1.x500.style.RFC4519Style;
-import org.bouncycastle.asn1.x509.BasicConstraints;
-import org.bouncycastle.asn1.x509.Extension;
-import org.bouncycastle.asn1.x509.KeyUsage;
-import org.bouncycastle.cert.X509CertificateHolder;
-import org.bouncycastle.cert.jcajce.JcaX509CertificateConverter;
-import org.bouncycastle.cert.jcajce.JcaX509ExtensionUtils;
-import org.bouncycastle.cert.jcajce.JcaX509v3CertificateBuilder;
-import org.bouncycastle.jce.provider.BouncyCastleProvider;
-import org.bouncycastle.operator.OperatorCreationException;
-import org.bouncycastle.operator.jcajce.JcaContentSignerBuilder;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 import org.junit.runners.Parameterized.Parameters;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
-import java.io.File;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.math.BigInteger;
-import java.security.KeyPair;
-import java.security.KeyPairGenerator;
-import java.security.KeyStore;
-import java.security.NoSuchAlgorithmException;
-import java.security.PrivateKey;
-import java.security.PublicKey;
-import java.security.Security;
-import java.security.cert.Certificate;
-import java.security.cert.CertificateException;
-import java.security.cert.X509Certificate;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
 import java.sql.Connection;
 import java.sql.DriverManager;
 import java.sql.ResultSet;
 import java.sql.Statement;
 import java.util.ArrayList;
-import java.util.Calendar;
+import java.util.Arrays;
 import java.util.List;
-import java.util.Objects;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assume.assumeFalse;
 
 /**
  * Test case for Avatica with TLS connectors.
  */
 @RunWith(Parameterized.class)
-public class SslDriverTest {
-  private static final Logger LOG = LoggerFactory.getLogger(SslDriverTest.class);
+public class SslDriverTest extends HttpBaseTest {
 
-  private static final String KEYSTORE_PASSWORD = "avaticasecret";
-  private static final ConnectionSpec CONNECTION_SPEC = ConnectionSpec.HSQLDB;
-  private static final List<HttpServer> SERVERS_TO_STOP = new ArrayList<>();
+  public SslDriverTest(String jdbcUrl) {
+    super(jdbcUrl);
+  }
 
   @Parameters public static List<Object[]> parameters() throws Exception {
-    final ArrayList<Object[]> parameters = new ArrayList<>();
-
-    // Create a self-signed cert
-    File target = new File(System.getProperty("user.dir"), "target");
-    File keystore = new File(target, "avatica-test.jks");
-    if (keystore.isFile()) {
-      assertTrue("Failed to delete keystore: " + keystore, keystore.delete());
+    if (System.getProperty("java.vendor").contains("IBM")) {
+      // Skip TLS testing on IBM Java due the combination of:
+      // - Jetty 9.4.12+ ignores SSL_* ciphers due to security - eclipse/jetty.project#2807
+      // - IBM uses SSL_* cipher names for ALL ciphers not following RFC cipher names
+      //   See eclipse/jetty.project#2807 for details
+      LOG.info("Skipping HTTPS test on IBM Java");
+      return Arrays.asList(new Object[][] {{null}, {null}});
     }
-    new CertTool().createSelfSignedCert(keystore, "avatica", KEYSTORE_PASSWORD);
-
-    // Create a LocalService around HSQLDB
-    final JdbcMeta jdbcMeta = new JdbcMeta(CONNECTION_SPEC.url,
-        CONNECTION_SPEC.username, CONNECTION_SPEC.password);
-    final LocalService localService = new LocalService(jdbcMeta);
 
+    final ArrayList<Object[]> parameters = new ArrayList<>();
+    setupClass();
     for (Driver.Serialization serialization : new Driver.Serialization[] {
         Driver.Serialization.JSON, Driver.Serialization.PROTOBUF}) {
       // Build and start the server, using TLS
       HttpServer httpServer = new HttpServer.Builder()
           .withPort(0)
-          .withTLS(keystore, KEYSTORE_PASSWORD, keystore, KEYSTORE_PASSWORD)
+          .withTLS(KEYSTORE, KEYSTORE_PASSWORD, KEYSTORE, KEYSTORE_PASSWORD)
           .withHandler(localService, serialization)
           .build();
       httpServer.start();
       SERVERS_TO_STOP.add(httpServer);
 
       final String url = "jdbc:avatica:remote:url=https://localhost:" + httpServer.getPort()
-          + ";serialization=" + serialization + ";truststore=" + keystore.getAbsolutePath()
+          + ";serialization=" + serialization + ";truststore=" + KEYSTORE.getAbsolutePath()
           + ";truststore_password=" + KEYSTORE_PASSWORD;
       LOG.info("JDBC URL {}", url);
 
@@ -121,26 +80,6 @@ public class SslDriverTest {
     return parameters;
   }
 
-  @BeforeClass public static void setupClass() {
-    // Skip TLS testing on IBM Java due the combination of:
-    // - Jetty 9.4.12+ ignores SSL_* ciphers due to security - eclipse/jetty.project#2807
-    // - IBM uses SSL_* cipher names for ALL ciphers not following RFC cipher names
-    //   See eclipse/jetty.project#2807 for details
-    assumeFalse(System.getProperty("java.vendor").contains("IBM"));
-  }
-
-  @AfterClass public static void stopKdc() {
-    for (HttpServer server : SERVERS_TO_STOP) {
-      server.stop();
-    }
-  }
-
-  private final String jdbcUrl;
-
-  public SslDriverTest(String jdbcUrl) {
-    this.jdbcUrl = Objects.requireNonNull(jdbcUrl);
-  }
-
   @Test
   public void testReadWrite() throws Exception {
     final String tableName = "testReadWrite";
@@ -158,76 +97,6 @@ public class SslDriverTest {
     }
   }
 
-  /**
-   * Utility class for creating certificates for testing.
-   */
-  private static class CertTool {
-    private static final String SIGNING_ALGORITHM = "SHA256WITHRSA";
-    private static final String ENC_ALGORITHM = "RSA";
-
-    static {
-      Security.addProvider(new BouncyCastleProvider());
-    }
-
-    private void createSelfSignedCert(File targetKeystore, String keyName,
-        String keystorePassword) {
-      if (targetKeystore.exists()) {
-        throw new RuntimeException("Keystore already exists: " + targetKeystore);
-      }
-
-      try {
-        KeyPair kp = generateKeyPair();
-
-        X509Certificate cert = generateCert(keyName, kp, true, kp.getPublic(),
-            kp.getPrivate());
-
-        char[] password = keystorePassword.toCharArray();
-        KeyStore keystore = KeyStore.getInstance("JKS");
-        keystore.load(null, null);
-        keystore.setCertificateEntry(keyName + "Cert", cert);
-        keystore.setKeyEntry(keyName + "Key", kp.getPrivate(), password, new Certificate[] {cert});
-        try (FileOutputStream fos = new FileOutputStream(targetKeystore)) {
-          keystore.store(fos, password);
-        }
-      } catch (Exception e) {
-        throw new RuntimeException(e);
-      }
-    }
-
-    private KeyPair generateKeyPair() throws NoSuchAlgorithmException {
-      KeyPairGenerator gen = KeyPairGenerator.getInstance(ENC_ALGORITHM);
-      gen.initialize(2048);
-      return gen.generateKeyPair();
-    }
-
-    private X509Certificate generateCert(String keyName, KeyPair kp, boolean isCertAuthority,
-                                         PublicKey signerPublicKey, PrivateKey signerPrivateKey)
-        throws IOException, OperatorCreationException, CertificateException,
-        NoSuchAlgorithmException {
-      Calendar startDate = DateTimeUtils.calendar();
-      Calendar endDate = DateTimeUtils.calendar();
-      endDate.add(Calendar.YEAR, 100);
-
-      BigInteger serialNumber = BigInteger.valueOf(startDate.getTimeInMillis());
-      X500Name issuer = new X500Name(
-          IETFUtils.rDNsFromString("cn=localhost", RFC4519Style.INSTANCE));
-      JcaX509v3CertificateBuilder certGen = new JcaX509v3CertificateBuilder(issuer,
-          serialNumber, startDate.getTime(), endDate.getTime(), issuer, kp.getPublic());
-      JcaX509ExtensionUtils extensionUtils = new JcaX509ExtensionUtils();
-      certGen.addExtension(Extension.subjectKeyIdentifier, false,
-          extensionUtils.createSubjectKeyIdentifier(kp.getPublic()));
-      certGen.addExtension(Extension.basicConstraints, false,
-          new BasicConstraints(isCertAuthority));
-      certGen.addExtension(Extension.authorityKeyIdentifier, false,
-          extensionUtils.createAuthorityKeyIdentifier(signerPublicKey));
-      if (isCertAuthority) {
-        certGen.addExtension(Extension.keyUsage, true, new KeyUsage(KeyUsage.keyCertSign));
-      }
-      X509CertificateHolder certificateHolder = certGen.build(
-          new JcaContentSignerBuilder(SIGNING_ALGORITHM).build(signerPrivateKey));
-      return new JcaX509CertificateConverter().getCertificate(certificateHolder);
-    }
-  }
 }
 
 // End SslDriverTest.java