You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@calcite.apache.org by kr...@apache.org on 2021/07/08 14:05:57 UTC

[calcite-avatica] branch master updated: [CALCITE-4676] Avatica client leaks TCP connections

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

krisden 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 ed45a8d  [CALCITE-4676] Avatica client leaks TCP connections
ed45a8d is described below

commit ed45a8d0fdcd9790d37e25ea1da3addf0efe12f9
Author: Istvan Toth <st...@cloudera.com>
AuthorDate: Tue Jul 6 07:19:32 2021 +0200

    [CALCITE-4676] Avatica client leaks TCP connections
    
    Refactor the default client implementation to use global HTTP Connection
    Pools instead of one per JDBC connection.
---
 .../remote/AvaticaCommonsHttpClientImpl.java       | 193 ++++++--------------
 .../remote/AvaticaCommonsHttpClientSpnegoImpl.java | 160 -----------------
 .../remote/AvaticaHttpClientFactoryImpl.java       |  99 ++++++-----
 .../avatica/remote/CommonsHttpClientPoolCache.java | 196 +++++++++++++++++++++
 ...eConfigurable.java => GSSAuthenticateable.java} |  18 +-
 .../remote/HostnameVerificationConfigurable.java   |   2 +-
 ...urable.java => HttpClientPoolConfigurable.java} |  19 +-
 .../avatica/remote/KeyStoreConfigurable.java       |   2 +-
 .../avatica/remote/TrustStoreConfigurable.java     |   2 +-
 ...ticaCommonsHttpClientImplSocketFactoryTest.java | 113 ------------
 .../remote/AvaticaCommonsHttpClientImplTest.java   |  27 ---
 server/build.gradle.kts                            |   2 +-
 .../org/apache/calcite/avatica/SslDriverTest.java  |   2 +-
 .../server/HttpServerSpnegoWithJaasTest.java       |  18 +-
 .../server/HttpServerSpnegoWithoutJaasTest.java    |  18 +-
 15 files changed, 356 insertions(+), 515 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 b30b573..d44682e 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
@@ -20,6 +20,8 @@ import org.apache.http.HttpHost;
 import org.apache.http.NoHttpResponseException;
 import org.apache.http.auth.AuthSchemeProvider;
 import org.apache.http.auth.AuthScope;
+import org.apache.http.auth.Credentials;
+import org.apache.http.auth.KerberosCredentials;
 import org.apache.http.auth.UsernamePasswordCredentials;
 import org.apache.http.client.ClientProtocolException;
 import org.apache.http.client.CredentialsProvider;
@@ -31,50 +33,42 @@ 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.conn.socket.PlainConnectionSocketFactory;
-import org.apache.http.conn.ssl.NoopHostnameVerifier;
-import org.apache.http.conn.ssl.SSLConnectionSocketFactory;
 import org.apache.http.entity.ByteArrayEntity;
 import org.apache.http.entity.ContentType;
 import org.apache.http.impl.auth.BasicSchemeFactory;
 import org.apache.http.impl.auth.DigestSchemeFactory;
+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.ssl.SSLContextBuilder;
-import org.apache.http.ssl.SSLContexts;
 import org.apache.http.util.EntityUtils;
 
+import org.ietf.jgss.GSSCredential;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.io.File;
 import java.io.IOException;
 import java.net.HttpURLConnection;
 import java.net.URI;
 import java.net.URISyntaxException;
 import java.net.URL;
+import java.security.Principal;
 import java.util.Objects;
-import javax.net.ssl.HostnameVerifier;
-import javax.net.ssl.SSLContext;
 
 /**
  * A common class to invoke HTTP requests against the Avatica server agnostic of the data being
  * sent and received across the wire.
  */
-public class AvaticaCommonsHttpClientImpl implements AvaticaHttpClient,
-    UsernamePasswordAuthenticateable, TrustStoreConfigurable,
-        KeyStoreConfigurable, HostnameVerificationConfigurable {
+public class AvaticaCommonsHttpClientImpl implements AvaticaHttpClient, HttpClientPoolConfigurable,
+    UsernamePasswordAuthenticateable, GSSAuthenticateable {
   private static final Logger LOG = LoggerFactory.getLogger(AvaticaCommonsHttpClientImpl.class);
 
-  // Some basic exposed configurations
-  private static final String MAX_POOLED_CONNECTION_PER_ROUTE_KEY =
-      "avatica.pooled.connections.per.route";
-  private static final String MAX_POOLED_CONNECTION_PER_ROUTE_DEFAULT = "25";
-  private static final String MAX_POOLED_CONNECTIONS_KEY = "avatica.pooled.connections.max";
-  private static final String MAX_POOLED_CONNECTIONS_DEFAULT = "100";
+  // SPNEGO specific settings
+  private static final boolean USE_CANONICAL_HOSTNAME = Boolean
+      .parseBoolean(System.getProperty("avatica.http.spnego.use_canonical_hostname", "true"));
+  private static final boolean STRIP_PORT_ON_SERVER_LOOKUP = true;
 
   protected final HttpHost host;
   protected final URI uri;
@@ -88,110 +82,18 @@ public class AvaticaCommonsHttpClientImpl implements AvaticaHttpClient,
   protected Lookup<AuthSchemeProvider> authRegistry = null;
   protected Object userToken;
 
-  protected File truststore = null;
-  protected File keystore = null;
-  protected String truststorePassword = null;
-  protected String keystorePassword = null;
-  protected String keyPassword = null;
-  protected HostnameVerification hostnameVerification = null;
-
   public AvaticaCommonsHttpClientImpl(URL url) {
     this.host = new HttpHost(url.getHost(), url.getPort(), url.getProtocol());
     this.uri = toURI(Objects.requireNonNull(url));
-    initializeClient();
   }
 
-  protected void initializeClient() {
-    socketFactoryRegistry = this.configureSocketFactories();
-    configureConnectionPool(socketFactoryRegistry);
+  protected void initializeClient(PoolingHttpClientConnectionManager pool) {
     this.authCache = new BasicAuthCache();
-    // A single thread-safe HttpClient, pooling connections via the ConnectionManager
+    // A single thread-safe HttpClient, pooling connections via the
+    // ConnectionManager
     this.client = HttpClients.custom().setConnectionManager(pool).build();
   }
 
-  protected void configureConnectionPool(Registry<ConnectionSocketFactory> registry) {
-    pool = new PoolingHttpClientConnectionManager(registry);
-    // Increase max total connection to 100
-    final String maxCnxns =
-        System.getProperty(MAX_POOLED_CONNECTIONS_KEY,
-            MAX_POOLED_CONNECTIONS_DEFAULT);
-    pool.setMaxTotal(Integer.parseInt(maxCnxns));
-    // Increase default max connection per route to 25
-    final String maxCnxnsPerRoute = System.getProperty(MAX_POOLED_CONNECTION_PER_ROUTE_KEY,
-        MAX_POOLED_CONNECTION_PER_ROUTE_DEFAULT);
-    pool.setDefaultMaxPerRoute(Integer.parseInt(maxCnxnsPerRoute));
-  }
-
-  protected Registry<ConnectionSocketFactory> configureSocketFactories() {
-    RegistryBuilder<ConnectionSocketFactory> registryBuilder = RegistryBuilder.create();
-    if (host.getSchemeName().equalsIgnoreCase("https")) {
-      configureHttpsRegistry(registryBuilder);
-    } else {
-      configureHttpRegistry(registryBuilder);
-    }
-    return registryBuilder.build();
-  }
-
-  protected void configureHttpsRegistry(RegistryBuilder<ConnectionSocketFactory> registryBuilder) {
-    try {
-      SSLContext sslContext = getSSLContext();
-      final HostnameVerifier verifier = getHostnameVerifier(hostnameVerification);
-      SSLConnectionSocketFactory sslFactory = new SSLConnectionSocketFactory(sslContext, verifier);
-      registryBuilder.register("https", sslFactory);
-    } catch (Exception e) {
-      LOG.error("HTTPS registry configuration failed");
-      throw new RuntimeException(e);
-    }
-  }
-
-  private SSLContext getSSLContext() throws Exception {
-    SSLContextBuilder sslContextBuilder = SSLContexts.custom();
-    if (null != truststore && null != truststorePassword) {
-      loadTrustStore(sslContextBuilder);
-    }
-    if (null != keystore && null != keystorePassword && null != keyPassword) {
-      loadKeyStore(sslContextBuilder);
-    }
-    return sslContextBuilder.build();
-  }
-
-  protected void loadKeyStore(SSLContextBuilder sslContextBuilder) throws Exception {
-    sslContextBuilder.loadKeyMaterial(keystore,
-            keystorePassword.toCharArray(), keyPassword.toCharArray());
-  }
-
-  protected void loadTrustStore(SSLContextBuilder sslContextBuilder) throws Exception {
-    sslContextBuilder.loadTrustMaterial(truststore, truststorePassword.toCharArray());
-  }
-
-  protected void configureHttpRegistry(RegistryBuilder<ConnectionSocketFactory> registryBuilder) {
-    registryBuilder.register("http", PlainConnectionSocketFactory.getSocketFactory());
-  }
-
-  /**
-   * Creates the {@code HostnameVerifier} given the provided {@code verification}.
-   *
-   * @param verification The intended hostname verification action.
-   * @return A verifier for the request verification.
-   * @throws IllegalArgumentException if the provided verification cannot be handled.
-   */
-  HostnameVerifier getHostnameVerifier(HostnameVerification verification) {
-    // Normally, the configuration logic would give us a default of STRICT if it was not
-    // provided by the user. It's easy for us to do a double-check.
-    if (verification == null) {
-      verification = HostnameVerification.STRICT;
-    }
-    switch (verification) {
-    case STRICT:
-      return SSLConnectionSocketFactory.getDefaultHostnameVerifier();
-    case NONE:
-      return NoopHostnameVerifier.INSTANCE;
-    default:
-      throw new IllegalArgumentException("Unhandled HostnameVerification: "
-          + hostnameVerification);
-    }
-  }
-
   @Override public byte[] send(byte[] request) {
     while (true) {
       HttpClientContext context = HttpClientContext.create();
@@ -199,7 +101,7 @@ public class AvaticaCommonsHttpClientImpl implements AvaticaHttpClient,
       context.setTargetHost(host);
 
       // Set the credentials if they were provided.
-      if (null != this.credentials) {
+      if (null != this.credentialsProvider) {
         context.setCredentialsProvider(credentialsProvider);
         context.setAuthSchemeRegistry(authRegistry);
         context.setAuthCache(authCache);
@@ -226,7 +128,8 @@ public class AvaticaCommonsHttpClientImpl implements AvaticaHttpClient,
           continue;
         }
 
-        throw new RuntimeException("Failed to execute HTTP Request, got HTTP/" + statusCode);
+        throw new RuntimeException(
+            "Failed to execute HTTP Request, got HTTP/" + statusCode);
       } catch (NoHttpResponseException e) {
         // This can happen when sitting behind a load balancer and a backend server dies
         LOG.debug("The server failed to issue an HTTP response, retrying");
@@ -248,8 +151,8 @@ public class AvaticaCommonsHttpClientImpl implements AvaticaHttpClient,
 
   @Override public void setUsernamePassword(AuthenticationType authType, String username,
       String password) {
-    this.credentials = new UsernamePasswordCredentials(
-        Objects.requireNonNull(username), Objects.requireNonNull(password));
+    this.credentials = new UsernamePasswordCredentials(Objects.requireNonNull(username),
+        Objects.requireNonNull(password));
 
     this.credentialsProvider = new BasicCredentialsProvider();
     credentialsProvider.setCredentials(AuthScope.ANY, credentials);
@@ -268,39 +171,51 @@ public class AvaticaCommonsHttpClientImpl implements AvaticaHttpClient,
     this.authRegistry = authRegistryBuilder.build();
   }
 
-  private static URI toURI(URL url) throws RuntimeException {
-    try {
-      return url.toURI();
-    } catch (URISyntaxException e) {
-      throw new RuntimeException(e);
+  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);
     }
   }
 
-  @Override public void setTrustStore(File truststore, String password) {
-    this.truststore = Objects.requireNonNull(truststore);
-    if (!truststore.exists() || !truststore.isFile()) {
-      throw new IllegalArgumentException(
-          "Truststore is must be an existing, regular file: " + truststore);
+  /**
+   * A credentials implementation which returns null.
+   */
+  private static class EmptyCredentials implements Credentials {
+    public static final EmptyCredentials INSTANCE = new EmptyCredentials();
+
+    @Override public String getPassword() {
+      return null;
+    }
+
+    @Override public Principal getUserPrincipal() {
+      return null;
     }
-    this.truststorePassword = Objects.requireNonNull(password);
-    initializeClient();
   }
 
-  @Override public void setKeyStore(File keystore, String keystorepassword, String keypassword) {
-    this.keystore = Objects.requireNonNull(keystore);
-    if (!keystore.exists() || !keystore.isFile()) {
-      throw new IllegalArgumentException(
-              "Keystore is must be an existing, regular file: " + keystore);
+  private static URI toURI(URL url) throws RuntimeException {
+    try {
+      return url.toURI();
+    } catch (URISyntaxException e) {
+      throw new RuntimeException(e);
     }
-    this.keystorePassword = Objects.requireNonNull(keystorepassword);
-    this.keyPassword = Objects.requireNonNull(keypassword);
-    initializeClient();
   }
 
-  @Override public void setHostnameVerification(HostnameVerification verification) {
-    this.hostnameVerification = Objects.requireNonNull(verification);
-    initializeClient();
+  @Override public void setHttpClientPool(PoolingHttpClientConnectionManager pool) {
+    initializeClient(pool);
   }
+
 }
 
 // End AvaticaCommonsHttpClientImpl.java
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
deleted file mode 100644
index b7ae0a0..0000000
--- a/core/src/main/java/org/apache/calcite/avatica/remote/AvaticaCommonsHttpClientSpnegoImpl.java
+++ /dev/null
@@ -1,160 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to you under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.calcite.avatica.remote;
-
-import org.apache.http.auth.AuthSchemeProvider;
-import org.apache.http.auth.AuthScope;
-import org.apache.http.auth.Credentials;
-import org.apache.http.auth.KerberosCredentials;
-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.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.BasicCredentialsProvider;
-import org.apache.http.util.EntityUtils;
-
-import org.ietf.jgss.GSSCredential;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.net.HttpURLConnection;
-import java.net.URL;
-import java.security.Principal;
-
-/**
- * Implementation of an AvaticaHttpClient which uses SPNEGO.
- *
- * <p>(At this point it could probably be just merged back into
- * {@link AvaticaCommonsHttpClientImpl}.)
- */
-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_PER_ROUTE_KEY =
-      "avatica.http.spnego.max_per_route";
-
-  private static final boolean USE_CANONICAL_HOSTNAME =
-      Boolean.parseBoolean(
-           System.getProperty("avatica.http.spnego.use_canonical_hostname", "true"));
-  private static final boolean STRIP_PORT_ON_SERVER_LOOKUP = true;
-
-  /**
-   * Constructs an http client with the expectation that the user is already logged in with their
-   * Kerberos identity via JAAS.
-   *
-   * @param url The URL for the Avatica server
-   */
-  public AvaticaCommonsHttpClientSpnegoImpl(URL url) {
-    this(url, null);
-  }
-
-  /**
-   * Constructs an HTTP client with user specified by the given credentials.
-   *
-   * @param url The URL for the Avatica server
-   * @param credential The GSS credentials
-   */
-  public AvaticaCommonsHttpClientSpnegoImpl(URL url, GSSCredential credential) {
-    super(url);
-    setGSSCredential(credential);
-  }
-
-  @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);
-    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);
-    }
-  }
-
-  @Override public byte[] send(byte[] request) {
-    HttpClientContext context = HttpClientContext.create();
-
-    context.setTargetHost(host);
-    context.setCredentialsProvider(credentialsProvider);
-    context.setAuthSchemeRegistry(authRegistry);
-    context.setAuthCache(authCache);
-
-    ByteArrayEntity entity = new ByteArrayEntity(request, ContentType.APPLICATION_OCTET_STREAM);
-
-    // Create the client with the AuthSchemeRegistry and manager
-    HttpPost post = new HttpPost(uri);
-    post.setEntity(entity);
-
-    try (CloseableHttpResponse response = client.execute(post, context)) {
-      final int statusCode = response.getStatusLine().getStatusCode();
-      if (HttpURLConnection.HTTP_OK == statusCode
-          || HttpURLConnection.HTTP_INTERNAL_ERROR == statusCode) {
-        return EntityUtils.toByteArray(response.getEntity());
-      }
-
-      throw new RuntimeException("Failed to execute HTTP Request, got HTTP/" + statusCode);
-    } catch (RuntimeException e) {
-      throw e;
-    } catch (Exception e) {
-      LOG.debug("Failed to execute HTTP request", e);
-      throw new RuntimeException(e);
-    }
-  }
-
-  /**
-   * A credentials implementation which returns null.
-   */
-  private static class EmptyCredentials implements Credentials {
-    public static final EmptyCredentials INSTANCE = new EmptyCredentials();
-
-    @Override public String getPassword() {
-      return null;
-    }
-    @Override public Principal getUserPrincipal() {
-      return null;
-    }
-  }
-}
-
-// End AvaticaCommonsHttpClientSpnegoImpl.java
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 596e023..d3cf849 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
@@ -18,6 +18,8 @@ package org.apache.calcite.avatica.remote;
 
 import org.apache.calcite.avatica.ConnectionConfig;
 
+import org.apache.http.impl.conn.PoolingHttpClientConnectionManager;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -35,8 +37,6 @@ public class AvaticaHttpClientFactoryImpl implements AvaticaHttpClientFactory {
 
   public static final String HTTP_CLIENT_IMPL_DEFAULT =
       AvaticaCommonsHttpClientImpl.class.getName();
-  public static final String SPNEGO_HTTP_CLIENT_IMPL_DEFAULT =
-      AvaticaCommonsHttpClientSpnegoImpl.class.getName();
 
   // Public for Type.PLUGIN
   public static final AvaticaHttpClientFactoryImpl INSTANCE = new AvaticaHttpClientFactoryImpl();
@@ -57,70 +57,77 @@ public class AvaticaHttpClientFactoryImpl implements AvaticaHttpClientFactory {
       KerberosConnection kerberosUtil) {
     String className = config.httpClientClass();
     if (null == className) {
-      // Provide an implementation that works with SPNEGO if that's the authentication is use.
-      if ("SPNEGO".equalsIgnoreCase(config.authentication())) {
-        className = SPNEGO_HTTP_CLIENT_IMPL_DEFAULT;
-      } else {
-        className = HTTP_CLIENT_IMPL_DEFAULT;
-      }
+      className = HTTP_CLIENT_IMPL_DEFAULT;
     }
 
     AvaticaHttpClient client = instantiateClient(className, url);
 
-    if (client instanceof TrustStoreConfigurable) {
-      File truststore = config.truststore();
-      String truststorePassword = config.truststorePassword();
-      if (null != truststore && null != truststorePassword) {
-        ((TrustStoreConfigurable) client)
-                .setTrustStore(truststore, truststorePassword);
-      }
+    if (client instanceof HttpClientPoolConfigurable) {
+      PoolingHttpClientConnectionManager pool = CommonsHttpClientPoolCache.getPool(config);
+      ((HttpClientPoolConfigurable) client).setHttpClientPool(pool);
     } else {
-      LOG.debug("{} is not capable of SSL/TLS communication", client.getClass().getName());
-    }
+      // Kept for backwards compatibility, the current AvaticaCommonsHttpClientImpl
+      // does not implement these interfaces
+      if (client instanceof TrustStoreConfigurable) {
+        File truststore = config.truststore();
+        String truststorePassword = config.truststorePassword();
+        if (null != truststore && null != truststorePassword) {
+          ((TrustStoreConfigurable) client).setTrustStore(truststore, truststorePassword);
+        }
+      } else {
+        LOG.debug("{} is not capable of SSL/TLS communication", client.getClass().getName());
+      }
 
-    if (client instanceof KeyStoreConfigurable) {
-      File keystore = config.keystore();
-      String keystorePassword = config.keystorePassword();
-      String keyPassword = config.keyPassword();
-      if (null != keystore && null != keystorePassword && null != keyPassword) {
-        ((KeyStoreConfigurable) client)
-                .setKeyStore(keystore, keystorePassword, keyPassword);
+      if (client instanceof KeyStoreConfigurable) {
+        File keystore = config.keystore();
+        String keystorePassword = config.keystorePassword();
+        String keyPassword = config.keyPassword();
+        if (null != keystore && null != keystorePassword && null != keyPassword) {
+          ((KeyStoreConfigurable) client).setKeyStore(keystore, keystorePassword, keyPassword);
+        }
+      } else {
+        LOG.debug("{} is not capable of Mutual authentication", client.getClass().getName());
       }
-    } else {
-      LOG.debug("{} is not capable of Mutual authentication", client.getClass().getName());
-    }
 
-    // Set the SSL hostname verification if the client supports it
-    if (client instanceof HostnameVerificationConfigurable) {
-      ((HostnameVerificationConfigurable) client)
-          .setHostnameVerification(config.hostnameVerification());
-    } else {
-      LOG.debug("{} is not capable of configurable SSL/TLS hostname verification",
-          client.getClass().getName());
+      // Set the SSL hostname verification if the client supports it
+      if (client instanceof HostnameVerificationConfigurable) {
+        ((HostnameVerificationConfigurable) client)
+            .setHostnameVerification(config.hostnameVerification());
+      } else {
+        LOG.debug("{} is not capable of configurable SSL/TLS hostname verification",
+            client.getClass().getName());
+      }
     }
 
+    final String authString = config.authentication();
+    final AuthenticationType authType = authString == null ? null
+        : AuthenticationType.valueOf(authString);
+
     if (client instanceof UsernamePasswordAuthenticateable) {
-      // Shortcircuit quickly if authentication wasn't provided (implies NONE)
-      final String authString = config.authentication();
-      if (null == authString) {
-        return client;
-      }
+      if (isUserPasswordAuth(authType)) {
 
-      final AuthenticationType authType = AuthenticationType.valueOf(authString);
-      final String username = config.avaticaUser();
-      final String password = config.avaticaPassword();
+        final String username = config.avaticaUser();
+        final String password = config.avaticaPassword();
 
-      // Can't authenticate with NONE or w/o username and password
-      if (isUserPasswordAuth(authType)) {
+        // Can't authenticate with NONE or w/o username and password
         if (null != username && null != password) {
           ((UsernamePasswordAuthenticateable) client)
               .setUsernamePassword(authType, username, password);
         } else {
           LOG.debug("Username or password was null");
         }
-      } else {
-        LOG.debug("{} is not capable of username/password authentication.", authType);
       }
+    } else {
+      LOG.debug("{} is not capable of username/password authentication.", authType);
+    }
+
+    if (client instanceof GSSAuthenticateable) {
+      if (AuthenticationType.SPNEGO == authType) {
+        // The actual principal is set in DoAsAvaticaHttpClient below
+        ((GSSAuthenticateable) client).setGSSCredential(null);
+      }
+    } else {
+      LOG.debug("{} is not capable of kerberos authentication.", authType);
     }
 
     if (null != kerberosUtil) {
diff --git a/core/src/main/java/org/apache/calcite/avatica/remote/CommonsHttpClientPoolCache.java b/core/src/main/java/org/apache/calcite/avatica/remote/CommonsHttpClientPoolCache.java
new file mode 100644
index 0000000..c032738
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/avatica/remote/CommonsHttpClientPoolCache.java
@@ -0,0 +1,196 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.avatica.remote;
+
+import org.apache.calcite.avatica.ConnectionConfig;
+import org.apache.calcite.avatica.remote.HostnameVerificationConfigurable.HostnameVerification;
+
+import org.apache.http.config.Registry;
+import org.apache.http.config.RegistryBuilder;
+import org.apache.http.conn.socket.ConnectionSocketFactory;
+import org.apache.http.conn.socket.PlainConnectionSocketFactory;
+import org.apache.http.conn.ssl.NoopHostnameVerifier;
+import org.apache.http.conn.ssl.SSLConnectionSocketFactory;
+import org.apache.http.impl.conn.PoolingHttpClientConnectionManager;
+import org.apache.http.ssl.SSLContextBuilder;
+import org.apache.http.ssl.SSLContexts;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.attribute.BasicFileAttributes;
+import java.util.concurrent.ConcurrentHashMap;
+import javax.net.ssl.HostnameVerifier;
+import javax.net.ssl.SSLContext;
+
+/**
+ * Creates and returns a PoolingHttpClientConnectionManager object.
+ * If a pool exists for a given set of keystore, trustore, and hostanmeVerification
+ * parameters, then the existing pool is returned.
+ *
+ */
+public class CommonsHttpClientPoolCache {
+
+  // Some basic exposed configurations
+  private static final String MAX_POOLED_CONNECTION_PER_ROUTE_KEY =
+      "avatica.pooled.connections.per.route";
+  private static final String MAX_POOLED_CONNECTION_PER_ROUTE_DEFAULT = "25";
+  private static final String MAX_POOLED_CONNECTIONS_KEY = "avatica.pooled.connections.max";
+  private static final String MAX_POOLED_CONNECTIONS_DEFAULT = "100";
+
+  private static final Logger LOG = LoggerFactory.getLogger(CommonsHttpClientPoolCache.class);
+
+  private CommonsHttpClientPoolCache() {
+    //do not instantiate
+  }
+
+  private static final ConcurrentHashMap<String, PoolingHttpClientConnectionManager> CACHED_POOLS =
+      new ConcurrentHashMap<>();
+
+  public static PoolingHttpClientConnectionManager getPool(ConnectionConfig config) {
+    String sslDisc = extractSSLParameters(config);
+
+    return CACHED_POOLS.computeIfAbsent(sslDisc, k -> setupPool(config));
+  }
+
+  private static PoolingHttpClientConnectionManager setupPool(ConnectionConfig config) {
+    Registry<ConnectionSocketFactory> csfr = createCSFRegistry(config);
+    PoolingHttpClientConnectionManager pool = new PoolingHttpClientConnectionManager(csfr);
+    final String maxCnxns =
+        System.getProperty(MAX_POOLED_CONNECTIONS_KEY, MAX_POOLED_CONNECTIONS_DEFAULT);
+    pool.setMaxTotal(Integer.parseInt(maxCnxns));
+    // Increase default max connection per route to 25
+    final String maxCnxnsPerRoute = System.getProperty(MAX_POOLED_CONNECTION_PER_ROUTE_KEY,
+        MAX_POOLED_CONNECTION_PER_ROUTE_DEFAULT);
+    pool.setDefaultMaxPerRoute(Integer.parseInt(maxCnxnsPerRoute));
+    LOG.debug("Created new pool {}", pool);
+    return pool;
+  }
+
+  private static Registry<ConnectionSocketFactory> createCSFRegistry(ConnectionConfig config) {
+    RegistryBuilder<ConnectionSocketFactory> registryBuilder = RegistryBuilder.create();
+    configureHttpRegistry(registryBuilder);
+    configureHttpsRegistry(registryBuilder, config);
+
+    return registryBuilder.build();
+  }
+
+  private static void configureHttpsRegistry(
+      RegistryBuilder<ConnectionSocketFactory> registryBuilder, ConnectionConfig config) {
+    try {
+      SSLContext sslContext = getSSLContext(config);
+      final HostnameVerifier verifier = getHostnameVerifier(config.hostnameVerification());
+      SSLConnectionSocketFactory sslFactory = new SSLConnectionSocketFactory(sslContext, verifier);
+      registryBuilder.register("https", sslFactory);
+    } catch (Exception e) {
+      LOG.error("HTTPS registry configuration failed");
+      throw new RuntimeException(e);
+    }
+  }
+
+  private static SSLContext getSSLContext(ConnectionConfig config) throws Exception {
+    SSLContextBuilder sslContextBuilder = SSLContexts.custom();
+    if (null != config.truststore() && null != config.truststorePassword()) {
+      loadTrustStore(sslContextBuilder, config);
+    }
+    if (null != config.keystore() && null != config.keystorePassword()
+        && null != config.keyPassword()) {
+      loadKeyStore(sslContextBuilder, config);
+    }
+    return sslContextBuilder.build();
+  }
+
+  private static void loadKeyStore(SSLContextBuilder sslContextBuilder, ConnectionConfig config)
+      throws Exception {
+    sslContextBuilder.loadKeyMaterial(config.keystore(), config.keystorePassword().toCharArray(),
+        config.keyPassword().toCharArray());
+  }
+
+  private static void loadTrustStore(SSLContextBuilder sslContextBuilder, ConnectionConfig config)
+      throws Exception {
+    sslContextBuilder.loadTrustMaterial(config.truststore(),
+        config.truststorePassword().toCharArray());
+    System.out.println("truststore loaded. truststore:" + config.truststore()
+        + "pw:" + config.truststorePassword());
+  }
+
+  private static void configureHttpRegistry(
+      RegistryBuilder<ConnectionSocketFactory> registryBuilder) {
+    registryBuilder.register("http", PlainConnectionSocketFactory.getSocketFactory());
+  }
+
+  /**
+   * Creates the {@code HostnameVerifier} given the provided {@code verification}.
+   *
+   * @param verification The intended hostname verification action.
+   * @return A verifier for the request verification.
+   * @throws IllegalArgumentException if the provided verification cannot be
+   *                                  handled.
+   */
+  private static HostnameVerifier getHostnameVerifier(HostnameVerification verification) {
+    // Normally, the configuration logic would give us a default of STRICT if it was
+    // not
+    // provided by the user. It's easy for us to do a double-check.
+    if (verification == null) {
+      verification = HostnameVerification.STRICT;
+    }
+    switch (verification) {
+    case STRICT:
+      return SSLConnectionSocketFactory.getDefaultHostnameVerifier();
+    case NONE:
+      return NoopHostnameVerifier.INSTANCE;
+    default:
+      throw new IllegalArgumentException("Unhandled HostnameVerification: " + verification.name());
+    }
+  }
+
+  private static String extractSSLParameters(ConnectionConfig config) {
+    //Check the mtimes, in case the file was reused (as it happens in the test suite)
+    long keyMtime = 0;
+    if (config.keystore() != null && config.keystore().canRead()) {
+      try {
+        keyMtime = Files.readAttributes(config.keystore().toPath(),
+            BasicFileAttributes.class).lastModifiedTime().toMillis();
+      } catch (IOException e) {
+        //Fall through
+      }
+    }
+    long trustMtime = 0;
+    if (config.truststore() != null && config.truststore().canRead()) {
+      try {
+        trustMtime = Files.readAttributes(config.truststore().toPath(),
+            BasicFileAttributes.class).lastModifiedTime().toMillis();
+      } catch (IOException e) {
+        //Fall through
+      }
+    }
+    StringBuilder sb = new StringBuilder();
+    sb.append(config.hostnameVerification().toString()).append(":")
+    .append(config.truststore()).append(":")
+    .append(config.truststorePassword()).append(":")
+    .append(config.keystore()).append(":")
+    .append(config.keystorePassword()).append(":")
+    .append(config.keyPassword()).append(":")
+    .append(keyMtime).append(":")
+    .append(trustMtime);
+    return sb.toString();
+  }
+}
+
+// End CommonsHttpClientPoolCache.java
diff --git a/core/src/main/java/org/apache/calcite/avatica/remote/TrustStoreConfigurable.java b/core/src/main/java/org/apache/calcite/avatica/remote/GSSAuthenticateable.java
similarity index 63%
copy from core/src/main/java/org/apache/calcite/avatica/remote/TrustStoreConfigurable.java
copy to core/src/main/java/org/apache/calcite/avatica/remote/GSSAuthenticateable.java
index 676f35b..8c8b3be 100644
--- a/core/src/main/java/org/apache/calcite/avatica/remote/TrustStoreConfigurable.java
+++ b/core/src/main/java/org/apache/calcite/avatica/remote/GSSAuthenticateable.java
@@ -16,21 +16,21 @@
  */
 package org.apache.calcite.avatica.remote;
 
-import java.io.File;
+import org.ietf.jgss.GSSCredential;
 
 /**
- * Allows a truststore (and password) to be provided to enable TLS authentication.
+ * Interface that allows configuration of a GSS credential with SPENGO HTTP authentication.
  */
-public interface TrustStoreConfigurable {
+public interface GSSAuthenticateable {
 
   /**
-   * Sets a truststore containing the collection of trust SSL/TLS server certificates
-   * to use for HTTPS calls and the password for that truststore.
+   * Sets the GSS Credential
+   * If credential is null, then the current subject will be used
    *
-   * @param truststore The truststore on the local filesystem
-   * @param password The truststore's password
+   * @param credential GSS Credentials
    */
-  void setTrustStore(File truststore, String password);
+  void setGSSCredential(GSSCredential credential);
+
 }
 
-// End TrustStoreConfigurable.java
+// End GSSAuthenticateable.java
diff --git a/core/src/main/java/org/apache/calcite/avatica/remote/HostnameVerificationConfigurable.java b/core/src/main/java/org/apache/calcite/avatica/remote/HostnameVerificationConfigurable.java
index db4fa1a..63bb51b 100644
--- a/core/src/main/java/org/apache/calcite/avatica/remote/HostnameVerificationConfigurable.java
+++ b/core/src/main/java/org/apache/calcite/avatica/remote/HostnameVerificationConfigurable.java
@@ -20,7 +20,7 @@ package org.apache.calcite.avatica.remote;
  * An interface to decorate an {@link AvaticaHttpClient} that can support configuration on
  * SSL hostname verification.
  */
-public interface HostnameVerificationConfigurable {
+@Deprecated public interface HostnameVerificationConfigurable {
   /**
    * Describes the support hostname verification methods of {@link AvaticaHttpClient}.
    */
diff --git a/core/src/main/java/org/apache/calcite/avatica/remote/TrustStoreConfigurable.java b/core/src/main/java/org/apache/calcite/avatica/remote/HttpClientPoolConfigurable.java
similarity index 60%
copy from core/src/main/java/org/apache/calcite/avatica/remote/TrustStoreConfigurable.java
copy to core/src/main/java/org/apache/calcite/avatica/remote/HttpClientPoolConfigurable.java
index 676f35b..2d074be 100644
--- a/core/src/main/java/org/apache/calcite/avatica/remote/TrustStoreConfigurable.java
+++ b/core/src/main/java/org/apache/calcite/avatica/remote/HttpClientPoolConfigurable.java
@@ -16,21 +16,20 @@
  */
 package org.apache.calcite.avatica.remote;
 
-import java.io.File;
+import org.apache.http.impl.conn.PoolingHttpClientConnectionManager;
 
 /**
- * Allows a truststore (and password) to be provided to enable TLS authentication.
+ * Allows a http connection pool to be provided to enable TLS authentication.
+ * On clients with this interface setHttpClientPool() MUST be called before using them.
  */
-public interface TrustStoreConfigurable {
-
+public interface HttpClientPoolConfigurable {
   /**
-   * Sets a truststore containing the collection of trust SSL/TLS server certificates
-   * to use for HTTPS calls and the password for that truststore.
+   * Sets a PoolingHttpClientConnectionManager containing the collection of SSL/TLS server
+   * keys and truststores to use for HTTPS calls.
    *
-   * @param truststore The truststore on the local filesystem
-   * @param password The truststore's password
+   * @param pool The http connection pool
    */
-  void setTrustStore(File truststore, String password);
+  void setHttpClientPool(PoolingHttpClientConnectionManager pool);
 }
 
-// End TrustStoreConfigurable.java
+// End HttpClientPoolConfigurable.java
diff --git a/core/src/main/java/org/apache/calcite/avatica/remote/KeyStoreConfigurable.java b/core/src/main/java/org/apache/calcite/avatica/remote/KeyStoreConfigurable.java
index eaffd2a..26e501f 100644
--- a/core/src/main/java/org/apache/calcite/avatica/remote/KeyStoreConfigurable.java
+++ b/core/src/main/java/org/apache/calcite/avatica/remote/KeyStoreConfigurable.java
@@ -22,7 +22,7 @@ import java.io.File;
  * Allows a keystore (and keystorepassword, keypassword) to be
  * provided to enable MTLS authentication
  */
-public interface KeyStoreConfigurable {
+@Deprecated public interface KeyStoreConfigurable {
 
     /**
      * Sets a keystore containing the collection of client side certificates
diff --git a/core/src/main/java/org/apache/calcite/avatica/remote/TrustStoreConfigurable.java b/core/src/main/java/org/apache/calcite/avatica/remote/TrustStoreConfigurable.java
index 676f35b..61ea03b 100644
--- a/core/src/main/java/org/apache/calcite/avatica/remote/TrustStoreConfigurable.java
+++ b/core/src/main/java/org/apache/calcite/avatica/remote/TrustStoreConfigurable.java
@@ -21,7 +21,7 @@ import java.io.File;
 /**
  * Allows a truststore (and password) to be provided to enable TLS authentication.
  */
-public interface TrustStoreConfigurable {
+@Deprecated public interface TrustStoreConfigurable {
 
   /**
    * Sets a truststore containing the collection of trust SSL/TLS server certificates
diff --git a/core/src/test/java/org/apache/calcite/avatica/remote/AvaticaCommonsHttpClientImplSocketFactoryTest.java b/core/src/test/java/org/apache/calcite/avatica/remote/AvaticaCommonsHttpClientImplSocketFactoryTest.java
deleted file mode 100644
index b05b8dc..0000000
--- a/core/src/test/java/org/apache/calcite/avatica/remote/AvaticaCommonsHttpClientImplSocketFactoryTest.java
+++ /dev/null
@@ -1,113 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to you under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.calcite.avatica.remote;
-
-import org.apache.http.conn.socket.ConnectionSocketFactory;
-import org.apache.http.conn.socket.PlainConnectionSocketFactory;
-import org.apache.http.conn.ssl.SSLConnectionSocketFactory;
-import org.apache.http.ssl.SSLContextBuilder;
-
-import org.junit.Test;
-
-import java.io.File;
-import java.net.URL;
-
-import static org.junit.Assert.assertTrue;
-import static org.mockito.ArgumentMatchers.any;
-import static org.mockito.Mockito.doNothing;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.spy;
-import static org.mockito.Mockito.times;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.when;
-
-/**
- * Tests to verify loading of truststore/keystore in AvaticaCommonsHttpClientImpl
- */
-public class AvaticaCommonsHttpClientImplSocketFactoryTest {
-
-  private static final String HTTP_REGISTRY = "http";
-  private static final String HTTPS_REGISTRY = "https";
-
-  private URL url;
-  private AvaticaCommonsHttpClientImpl client;
-  private File storeFile;
-  private String password;
-
-  @Test public void testPlainSocketFactory() throws Exception {
-    configureHttpClient();
-    verifyFactoryInstance(client, HTTP_REGISTRY, PlainConnectionSocketFactory.class);
-    verifyFactoryInstance(client, HTTPS_REGISTRY, null);
-    verify(client, times(0)).loadTrustStore(any(SSLContextBuilder.class));
-    verify(client, times(0)).loadKeyStore(any(SSLContextBuilder.class));
-  }
-
-  @Test public void testTrustStoreLoadedInFactory() throws Exception {
-    configureHttpsClient();
-    client.setTrustStore(storeFile, password);
-    verifyFactoryInstance(client, HTTP_REGISTRY, null);
-    verifyFactoryInstance(client, HTTPS_REGISTRY, SSLConnectionSocketFactory.class);
-    verify(client, times(1)).configureSocketFactories();
-    verify(client, times(1)).loadTrustStore(any(SSLContextBuilder.class));
-    verify(client, times(0)).loadKeyStore(any(SSLContextBuilder.class));
-  }
-
-  @Test public void testKeyStoreLoadedInFactory() throws Exception {
-    configureHttpsClient();
-    client.setKeyStore(storeFile, password, password);
-    verifyFactoryInstance(client, HTTP_REGISTRY, null);
-    verifyFactoryInstance(client, HTTPS_REGISTRY, SSLConnectionSocketFactory.class);
-    verify(client, times(1)).configureSocketFactories();
-    verify(client, times(0)).loadTrustStore(any(SSLContextBuilder.class));
-    verify(client, times(1)).loadKeyStore(any(SSLContextBuilder.class));
-  }
-
-  private void configureHttpClient() throws Exception {
-    url = new URL("http://fake_url.com");
-    configureClient();
-  }
-
-  private void configureHttpsClient() throws Exception {
-    url = new URL("https://fake_url.com");
-    configureClient();
-  }
-
-  private void configureClient() throws Exception {
-    client = spy(new AvaticaCommonsHttpClientImpl(url));
-    // storeFile can be used as either Keystore/Truststore
-    storeFile = mock(File.class);
-    when(storeFile.exists()).thenReturn(true);
-    when(storeFile.isFile()).thenReturn(true);
-    password = "";
-
-    doNothing().when(client).loadTrustStore(any(SSLContextBuilder.class));
-    doNothing().when(client).loadKeyStore(any(SSLContextBuilder.class));
-  }
-
-  <T> void verifyFactoryInstance(AvaticaCommonsHttpClientImpl client,
-      String registry, Class<T> expected) {
-    ConnectionSocketFactory factory = client.socketFactoryRegistry.lookup(registry);
-    if (expected == null) {
-      assertTrue("Factory for registry " + registry + " expected as null", factory == null);
-    } else {
-      assertTrue("Factory for registry " + registry + " expected of type " + expected.getName(),
-              expected.equals(factory.getClass()));
-    }
-  }
-}
-
-// End AvaticaCommonsHttpClientImplSocketFactoryTest.java
diff --git a/core/src/test/java/org/apache/calcite/avatica/remote/AvaticaCommonsHttpClientImplTest.java b/core/src/test/java/org/apache/calcite/avatica/remote/AvaticaCommonsHttpClientImplTest.java
index 9918143..e95c741 100644
--- a/core/src/test/java/org/apache/calcite/avatica/remote/AvaticaCommonsHttpClientImplTest.java
+++ b/core/src/test/java/org/apache/calcite/avatica/remote/AvaticaCommonsHttpClientImplTest.java
@@ -16,15 +16,11 @@
  */
 package org.apache.calcite.avatica.remote;
 
-import org.apache.calcite.avatica.remote.HostnameVerificationConfigurable.HostnameVerification;
-
 import org.apache.http.NoHttpResponseException;
 import org.apache.http.StatusLine;
 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.conn.ssl.DefaultHostnameVerifier;
-import org.apache.http.conn.ssl.NoopHostnameVerifier;
 import org.apache.http.entity.StringEntity;
 
 import org.junit.Test;
@@ -32,14 +28,10 @@ import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
 
 import java.net.HttpURLConnection;
-import javax.net.ssl.HostnameVerifier;
 
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
 import static org.mockito.Mockito.any;
 import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.nullable;
 import static org.mockito.Mockito.when;
 
 import static java.nio.charset.StandardCharsets.UTF_8;
@@ -119,25 +111,6 @@ public class AvaticaCommonsHttpClientImplTest {
     assertEquals("success", new String(responseBytes, UTF_8));
   }
 
-  @Test public void testHostnameVerification() throws Exception {
-    AvaticaCommonsHttpClientImpl client = mock(AvaticaCommonsHttpClientImpl.class);
-    // Call the real method
-    when(client.getHostnameVerifier(nullable(HostnameVerification.class)))
-        .thenCallRealMethod();
-
-    // No verification should give the default (strict) verifier
-    HostnameVerifier actualVerifier = client.getHostnameVerifier(null);
-    assertNotNull(actualVerifier);
-    assertTrue(actualVerifier instanceof DefaultHostnameVerifier);
-
-    actualVerifier = client.getHostnameVerifier(HostnameVerification.STRICT);
-    assertNotNull(actualVerifier);
-    assertTrue(actualVerifier instanceof DefaultHostnameVerifier);
-
-    actualVerifier = client.getHostnameVerifier(HostnameVerification.NONE);
-    assertNotNull(actualVerifier);
-    assertTrue(actualVerifier instanceof NoopHostnameVerifier);
-  }
 }
 
 // End AvaticaCommonsHttpClientImplTest.java
diff --git a/server/build.gradle.kts b/server/build.gradle.kts
index 9217949..a1a48a5 100644
--- a/server/build.gradle.kts
+++ b/server/build.gradle.kts
@@ -45,7 +45,7 @@ dependencies {
     testImplementation("org.bouncycastle:bcprov-jdk15on")
     testImplementation("org.hamcrest:hamcrest-core")
     testImplementation("org.mockito:mockito-core")
-
+    testImplementation("org.apache.httpcomponents:httpclient")
     testRuntimeOnly("org.hsqldb:hsqldb")
     testRuntimeOnly("org.slf4j:slf4j-log4j12")
 }
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 7421ddd..a99f7ba 100644
--- a/server/src/test/java/org/apache/calcite/avatica/SslDriverTest.java
+++ b/server/src/test/java/org/apache/calcite/avatica/SslDriverTest.java
@@ -47,7 +47,7 @@ public class SslDriverTest extends HttpBaseTest {
     super(jdbcUrl);
   }
 
-  @Parameters public static List<Object[]> parameters() throws Exception {
+  @Parameters(name = "{0}") public static List<Object[]> parameters() throws Exception {
     // 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
diff --git a/server/src/test/java/org/apache/calcite/avatica/server/HttpServerSpnegoWithJaasTest.java b/server/src/test/java/org/apache/calcite/avatica/server/HttpServerSpnegoWithJaasTest.java
index dd400db..93794b9 100644
--- a/server/src/test/java/org/apache/calcite/avatica/server/HttpServerSpnegoWithJaasTest.java
+++ b/server/src/test/java/org/apache/calcite/avatica/server/HttpServerSpnegoWithJaasTest.java
@@ -16,9 +16,13 @@
  */
 package org.apache.calcite.avatica.server;
 
+import org.apache.calcite.avatica.ConnectionConfig;
+import org.apache.calcite.avatica.ConnectionConfigImpl;
 import org.apache.calcite.avatica.SpnegoTestUtil;
-import org.apache.calcite.avatica.remote.AvaticaCommonsHttpClientSpnegoImpl;
+import org.apache.calcite.avatica.remote.AvaticaCommonsHttpClientImpl;
+import org.apache.calcite.avatica.remote.CommonsHttpClientPoolCache;
 
+import org.apache.http.impl.conn.PoolingHttpClientConnectionManager;
 import org.apache.kerby.kerberos.kerb.KrbException;
 import org.apache.kerby.kerberos.kerb.client.KrbConfig;
 import org.apache.kerby.kerberos.kerb.client.KrbConfigKey;
@@ -44,6 +48,7 @@ import java.net.URL;
 import java.nio.charset.StandardCharsets;
 import java.security.Principal;
 import java.security.PrivilegedExceptionAction;
+import java.util.Properties;
 import java.util.Set;
 import javax.security.auth.Subject;
 import javax.security.auth.kerberos.KerberosTicket;
@@ -225,9 +230,16 @@ public class HttpServerSpnegoWithJaasTest {
         GSSCredential credential = gssManager.createCredential(gssClient,
             GSSCredential.DEFAULT_LIFETIME, oid, GSSCredential.INITIATE_ONLY);
 
+        Properties props = new Properties();
+        ConnectionConfig config = new ConnectionConfigImpl(props);
+
+        PoolingHttpClientConnectionManager pool = CommonsHttpClientPoolCache.getPool(config);
+
         // Passes the GSSCredential into the HTTP client implementation
-        final AvaticaCommonsHttpClientSpnegoImpl httpClient =
-            new AvaticaCommonsHttpClientSpnegoImpl(httpServerUrl, credential);
+        final AvaticaCommonsHttpClientImpl httpClient =
+            new AvaticaCommonsHttpClientImpl(httpServerUrl);
+        httpClient.setGSSCredential(credential);
+        httpClient.setHttpClientPool(pool);
 
         return httpClient.send(new byte[0]);
       }
diff --git a/server/src/test/java/org/apache/calcite/avatica/server/HttpServerSpnegoWithoutJaasTest.java b/server/src/test/java/org/apache/calcite/avatica/server/HttpServerSpnegoWithoutJaasTest.java
index bcaabdf..43d1895 100644
--- a/server/src/test/java/org/apache/calcite/avatica/server/HttpServerSpnegoWithoutJaasTest.java
+++ b/server/src/test/java/org/apache/calcite/avatica/server/HttpServerSpnegoWithoutJaasTest.java
@@ -16,9 +16,13 @@
  */
 package org.apache.calcite.avatica.server;
 
+import org.apache.calcite.avatica.ConnectionConfig;
+import org.apache.calcite.avatica.ConnectionConfigImpl;
 import org.apache.calcite.avatica.SpnegoTestUtil;
-import org.apache.calcite.avatica.remote.AvaticaCommonsHttpClientSpnegoImpl;
+import org.apache.calcite.avatica.remote.AvaticaCommonsHttpClientImpl;
+import org.apache.calcite.avatica.remote.CommonsHttpClientPoolCache;
 
+import org.apache.http.impl.conn.PoolingHttpClientConnectionManager;
 import org.apache.kerby.kerberos.kerb.KrbException;
 import org.apache.kerby.kerberos.kerb.client.KrbConfig;
 import org.apache.kerby.kerberos.kerb.client.KrbConfigKey;
@@ -43,6 +47,7 @@ import java.net.URL;
 import java.nio.charset.StandardCharsets;
 import java.security.Principal;
 import java.security.PrivilegedExceptionAction;
+import java.util.Properties;
 import java.util.Set;
 import javax.security.auth.Subject;
 import javax.security.auth.kerberos.KerberosTicket;
@@ -212,9 +217,16 @@ public class HttpServerSpnegoWithoutJaasTest {
         GSSCredential credential = gssManager.createCredential(gssClient,
             GSSCredential.DEFAULT_LIFETIME, oid, GSSCredential.INITIATE_ONLY);
 
+        Properties props = new Properties();
+        ConnectionConfig config = new ConnectionConfigImpl(props);
+
+        PoolingHttpClientConnectionManager pool = CommonsHttpClientPoolCache.getPool(config);
+
         // Passes the GSSCredential into the HTTP client implementation
-        final AvaticaCommonsHttpClientSpnegoImpl httpClient =
-            new AvaticaCommonsHttpClientSpnegoImpl(httpServerUrl, credential);
+        final AvaticaCommonsHttpClientImpl httpClient =
+            new AvaticaCommonsHttpClientImpl(httpServerUrl);
+        httpClient.setGSSCredential(credential);
+        httpClient.setHttpClientPool(pool);
 
         return httpClient.send(new byte[0]);
       }