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 2016/12/29 20:34:12 UTC

calcite git commit: [CALCITE-1538] Support truststore and truststore password avatica JDBC options

Repository: calcite
Updated Branches:
  refs/heads/master 70b832513 -> 0ef3660d6


[CALCITE-1538] Support truststore and truststore password avatica JDBC options

Closes apache/calcite#341


Project: http://git-wip-us.apache.org/repos/asf/calcite/repo
Commit: http://git-wip-us.apache.org/repos/asf/calcite/commit/0ef3660d
Tree: http://git-wip-us.apache.org/repos/asf/calcite/tree/0ef3660d
Diff: http://git-wip-us.apache.org/repos/asf/calcite/diff/0ef3660d

Branch: refs/heads/master
Commit: 0ef3660d68d81c9221dfdc6b2eabd8f886fb618d
Parents: 70b8325
Author: Josh Elser <el...@apache.org>
Authored: Tue Dec 13 22:48:06 2016 -0500
Committer: Josh Elser <el...@apache.org>
Committed: Thu Dec 29 15:30:50 2016 -0500

----------------------------------------------------------------------
 .../avatica/BuiltInConnectionProperty.java      |   8 +-
 .../calcite/avatica/ConnectionConfig.java       |   4 +
 .../calcite/avatica/ConnectionConfigImpl.java   |  12 +
 .../remote/AvaticaCommonsHttpClientImpl.java    |  67 ++++--
 .../remote/AvaticaHttpClientFactoryImpl.java    |  11 +
 .../avatica/remote/TrustStoreConfigurable.java  |  36 +++
 .../avatica/ConnectionConfigImplTest.java       |  50 +++++
 avatica/pom.xml                                 |  11 +
 avatica/server/pom.xml                          |  10 +
 .../calcite/avatica/server/HttpServer.java      |  63 +++++-
 .../apache/calcite/avatica/SslDriverTest.java   | 224 +++++++++++++++++++
 avatica/site/_docs/client_reference.md          |  18 ++
 12 files changed, 490 insertions(+), 24 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/0ef3660d/avatica/core/src/main/java/org/apache/calcite/avatica/BuiltInConnectionProperty.java
----------------------------------------------------------------------
diff --git a/avatica/core/src/main/java/org/apache/calcite/avatica/BuiltInConnectionProperty.java b/avatica/core/src/main/java/org/apache/calcite/avatica/BuiltInConnectionProperty.java
index c408f24..aa7d710 100644
--- a/avatica/core/src/main/java/org/apache/calcite/avatica/BuiltInConnectionProperty.java
+++ b/avatica/core/src/main/java/org/apache/calcite/avatica/BuiltInConnectionProperty.java
@@ -66,7 +66,13 @@ public enum BuiltInConnectionProperty implements ConnectionProperty {
   PRINCIPAL("principal", Type.STRING, null, false),
 
   /** Keytab to use to perform Kerberos login. */
-  KEYTAB("keytab", Type.STRING, null, false);
+  KEYTAB("keytab", Type.STRING, null, false),
+
+  /** Truststore for SSL/TLS communication */
+  TRUSTSTORE("truststore", Type.STRING, null, false),
+
+  /** Password for the truststore */
+  TRUSTSTORE_PASSWORD("truststore_password", Type.STRING, null, false);
 
   private final String camelName;
   private final Type type;

http://git-wip-us.apache.org/repos/asf/calcite/blob/0ef3660d/avatica/core/src/main/java/org/apache/calcite/avatica/ConnectionConfig.java
----------------------------------------------------------------------
diff --git a/avatica/core/src/main/java/org/apache/calcite/avatica/ConnectionConfig.java b/avatica/core/src/main/java/org/apache/calcite/avatica/ConnectionConfig.java
index 032fafc..e8ff0be 100644
--- a/avatica/core/src/main/java/org/apache/calcite/avatica/ConnectionConfig.java
+++ b/avatica/core/src/main/java/org/apache/calcite/avatica/ConnectionConfig.java
@@ -49,6 +49,10 @@ public interface ConnectionConfig {
   String kerberosPrincipal();
   /** @see BuiltInConnectionProperty#KEYTAB */
   File kerberosKeytab();
+  /** @see BuiltInConnectionProperty#TRUSTSTORE */
+  File truststore();
+  /** @see BuiltInConnectionProperty#TRUSTSTORE_PASSWORD */
+  String truststorePassword();
 }
 
 // End ConnectionConfig.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/0ef3660d/avatica/core/src/main/java/org/apache/calcite/avatica/ConnectionConfigImpl.java
----------------------------------------------------------------------
diff --git a/avatica/core/src/main/java/org/apache/calcite/avatica/ConnectionConfigImpl.java b/avatica/core/src/main/java/org/apache/calcite/avatica/ConnectionConfigImpl.java
index 9ef1e66..e9d7272 100644
--- a/avatica/core/src/main/java/org/apache/calcite/avatica/ConnectionConfigImpl.java
+++ b/avatica/core/src/main/java/org/apache/calcite/avatica/ConnectionConfigImpl.java
@@ -92,6 +92,18 @@ public class ConnectionConfigImpl implements ConnectionConfig {
     return keytab;
   }
 
+  public File truststore() {
+    String filename = BuiltInConnectionProperty.TRUSTSTORE.wrap(properties).getString();
+    if (null == filename) {
+      return null;
+    }
+    return new File(filename);
+  }
+
+  public String truststorePassword() {
+    return BuiltInConnectionProperty.TRUSTSTORE_PASSWORD.wrap(properties).getString();
+  }
+
   /** Converts a {@link Properties} object containing (name, value)
    * pairs into a map whose keys are
    * {@link org.apache.calcite.avatica.InternalProperty} objects.

http://git-wip-us.apache.org/repos/asf/calcite/blob/0ef3660d/avatica/core/src/main/java/org/apache/calcite/avatica/remote/AvaticaCommonsHttpClientImpl.java
----------------------------------------------------------------------
diff --git a/avatica/core/src/main/java/org/apache/calcite/avatica/remote/AvaticaCommonsHttpClientImpl.java b/avatica/core/src/main/java/org/apache/calcite/avatica/remote/AvaticaCommonsHttpClientImpl.java
index 5e8fb5f..33872d0 100644
--- a/avatica/core/src/main/java/org/apache/calcite/avatica/remote/AvaticaCommonsHttpClientImpl.java
+++ b/avatica/core/src/main/java/org/apache/calcite/avatica/remote/AvaticaCommonsHttpClientImpl.java
@@ -27,9 +27,11 @@ 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.client.protocol.RequestExpectContinue;
 import org.apache.http.config.Lookup;
 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.SSLConnectionSocketFactory;
 import org.apache.http.entity.ByteArrayEntity;
 import org.apache.http.entity.ContentType;
 import org.apache.http.impl.auth.BasicSchemeFactory;
@@ -39,17 +41,13 @@ 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.protocol.HttpProcessor;
-import org.apache.http.protocol.HttpProcessorBuilder;
-import org.apache.http.protocol.HttpRequestExecutor;
-import org.apache.http.protocol.RequestConnControl;
-import org.apache.http.protocol.RequestContent;
-import org.apache.http.protocol.RequestTargetHost;
+import org.apache.http.ssl.SSLContexts;
 import org.apache.http.util.EntityUtils;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.io.File;
 import java.io.IOException;
 import java.net.HttpURLConnection;
 import java.net.URI;
@@ -57,12 +55,14 @@ import java.net.URISyntaxException;
 import java.net.URL;
 import java.util.Objects;
 
+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 {
+    UsernamePasswordAuthenticateable, TrustStoreConfigurable {
   private static final Logger LOG = LoggerFactory.getLogger(AvaticaCommonsHttpClientImpl.class);
 
   // Some basic exposed configurations
@@ -74,29 +74,44 @@ public class AvaticaCommonsHttpClientImpl implements AvaticaHttpClient,
 
   protected final HttpHost host;
   protected final URI uri;
-  protected final HttpProcessor httpProcessor;
-  protected final HttpRequestExecutor httpExecutor;
-  protected final BasicAuthCache authCache;
-  protected final CloseableHttpClient client;
-  final PoolingHttpClientConnectionManager pool;
+  protected BasicAuthCache authCache;
+  protected CloseableHttpClient client;
+  PoolingHttpClientConnectionManager pool;
 
   protected UsernamePasswordCredentials credentials = null;
   protected CredentialsProvider credentialsProvider = null;
   protected Lookup<AuthSchemeProvider> authRegistry = null;
 
+  protected File truststore = null;
+  protected String truststorePassword = null;
+
   public AvaticaCommonsHttpClientImpl(URL url) {
     this.host = new HttpHost(url.getHost(), url.getPort(), url.getProtocol());
     this.uri = toURI(Objects.requireNonNull(url));
+    initializeClient();
+  }
 
-    this.httpProcessor = HttpProcessorBuilder.create()
-        .add(new RequestContent())
-        .add(new RequestTargetHost())
-        .add(new RequestConnControl())
-        .add(new RequestExpectContinue()).build();
-
-    this.httpExecutor = new HttpRequestExecutor();
+  private void initializeClient() {
+    SSLConnectionSocketFactory sslFactory = null;
+    if (null != truststore && null != truststorePassword) {
+      try {
+        SSLContext sslcontext = SSLContexts.custom().loadTrustMaterial(
+            truststore, truststorePassword.toCharArray()).build();
+        sslFactory = new SSLConnectionSocketFactory(sslcontext);
+      } catch (Exception e) {
+        throw new RuntimeException(e);
+      }
+    } else {
+      LOG.debug("Not configuring HTTPS because of missing truststore/password");
+    }
 
-    pool = new PoolingHttpClientConnectionManager();
+    RegistryBuilder<ConnectionSocketFactory> registryBuilder = RegistryBuilder.create();
+    registryBuilder.register("http", PlainConnectionSocketFactory.getSocketFactory());
+    // Only register the SSL factory when provided
+    if (null != sslFactory) {
+      registryBuilder.register("https", sslFactory);
+    }
+    pool = new PoolingHttpClientConnectionManager(registryBuilder.build());
     // Increase max total connection to 100
     final String maxCnxns =
         System.getProperty(MAX_POOLED_CONNECTIONS_KEY,
@@ -191,6 +206,16 @@ public class AvaticaCommonsHttpClientImpl implements AvaticaHttpClient,
       throw new RuntimeException(e);
     }
   }
+
+  @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);
+    }
+    this.truststorePassword = Objects.requireNonNull(password);
+    initializeClient();
+  }
 }
 
 // End AvaticaCommonsHttpClientImpl.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/0ef3660d/avatica/core/src/main/java/org/apache/calcite/avatica/remote/AvaticaHttpClientFactoryImpl.java
----------------------------------------------------------------------
diff --git a/avatica/core/src/main/java/org/apache/calcite/avatica/remote/AvaticaHttpClientFactoryImpl.java b/avatica/core/src/main/java/org/apache/calcite/avatica/remote/AvaticaHttpClientFactoryImpl.java
index cc83b00..8778c3d 100644
--- a/avatica/core/src/main/java/org/apache/calcite/avatica/remote/AvaticaHttpClientFactoryImpl.java
+++ b/avatica/core/src/main/java/org/apache/calcite/avatica/remote/AvaticaHttpClientFactoryImpl.java
@@ -21,6 +21,7 @@ import org.apache.calcite.avatica.ConnectionConfig;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.io.File;
 import java.lang.reflect.Constructor;
 import java.net.URL;
 import java.util.Objects;
@@ -69,6 +70,16 @@ public class AvaticaHttpClientFactoryImpl implements AvaticaHttpClientFactory {
       client = new DoAsAvaticaHttpClient(client, kerberosUtil);
     }
 
+    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 UsernamePasswordAuthenticateable) {
       // Shortcircuit quickly if authentication wasn't provided (implies NONE)
       final String authString = config.authentication();

http://git-wip-us.apache.org/repos/asf/calcite/blob/0ef3660d/avatica/core/src/main/java/org/apache/calcite/avatica/remote/TrustStoreConfigurable.java
----------------------------------------------------------------------
diff --git a/avatica/core/src/main/java/org/apache/calcite/avatica/remote/TrustStoreConfigurable.java b/avatica/core/src/main/java/org/apache/calcite/avatica/remote/TrustStoreConfigurable.java
new file mode 100644
index 0000000..676f35b
--- /dev/null
+++ b/avatica/core/src/main/java/org/apache/calcite/avatica/remote/TrustStoreConfigurable.java
@@ -0,0 +1,36 @@
+/*
+ * 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 java.io.File;
+
+/**
+ * Allows a truststore (and password) to be provided to enable TLS authentication.
+ */
+public interface TrustStoreConfigurable {
+
+  /**
+   * Sets a truststore containing the collection of trust SSL/TLS server certificates
+   * to use for HTTPS calls and the password for that truststore.
+   *
+   * @param truststore The truststore on the local filesystem
+   * @param password The truststore's password
+   */
+  void setTrustStore(File truststore, String password);
+}
+
+// End TrustStoreConfigurable.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/0ef3660d/avatica/core/src/test/java/org/apache/calcite/avatica/ConnectionConfigImplTest.java
----------------------------------------------------------------------
diff --git a/avatica/core/src/test/java/org/apache/calcite/avatica/ConnectionConfigImplTest.java b/avatica/core/src/test/java/org/apache/calcite/avatica/ConnectionConfigImplTest.java
new file mode 100644
index 0000000..bbe30e1
--- /dev/null
+++ b/avatica/core/src/test/java/org/apache/calcite/avatica/ConnectionConfigImplTest.java
@@ -0,0 +1,50 @@
+/*
+ * 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;
+
+import org.junit.Test;
+
+import java.util.Properties;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+
+/**
+ * Test class for {@link ConnectionConfigImpl}.
+ */
+public class ConnectionConfigImplTest {
+
+  @Test public void testTrustStore() {
+    final String truststore = "/my/truststore.jks";
+    final String pw = "supremelysecret";
+    Properties props = new Properties();
+    props.setProperty(BuiltInConnectionProperty.TRUSTSTORE.name(), truststore);
+    props.setProperty(BuiltInConnectionProperty.TRUSTSTORE_PASSWORD.name(), pw);
+    ConnectionConfigImpl config = new ConnectionConfigImpl(props);
+    assertEquals(truststore, config.truststore().getAbsolutePath());
+    assertEquals(pw, config.truststorePassword());
+  }
+
+  @Test public void testNoTruststore() {
+    Properties props = new Properties();
+    ConnectionConfigImpl config = new ConnectionConfigImpl(props);
+    assertNull(config.truststore());
+    assertNull(config.truststorePassword());
+  }
+}
+
+// End ConnectionConfigImplTest.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/0ef3660d/avatica/pom.xml
----------------------------------------------------------------------
diff --git a/avatica/pom.xml b/avatica/pom.xml
index ff8915c..1e0639c 100644
--- a/avatica/pom.xml
+++ b/avatica/pom.xml
@@ -54,6 +54,7 @@ limitations under the License.
     <version.major>1</version.major>
     <version.minor>9</version.minor>
     <!-- This list is in alphabetical order. -->
+    <bouncycastle.version>1.55</bouncycastle.version>
     <build-helper-maven-plugin.version>1.9</build-helper-maven-plugin.version>
     <checksum-maven-plugin.version>1.2</checksum-maven-plugin.version>
     <dropwizard-metrics3.version>3.1.2</dropwizard-metrics3.version>
@@ -235,6 +236,16 @@ limitations under the License.
         <version>${httpcore.version}</version>
       </dependency>
       <dependency>
+        <groupId>org.bouncycastle</groupId>
+        <artifactId>bcpkix-jdk15on</artifactId>
+        <version>${bouncycastle.version}</version>
+      </dependency>
+      <dependency>
+        <groupId>org.bouncycastle</groupId>
+        <artifactId>bcprov-jdk15on</artifactId>
+        <version>${bouncycastle.version}</version>
+      </dependency>
+      <dependency>
         <groupId>org.mockito</groupId>
         <artifactId>mockito-all</artifactId>
         <version>${mockito-all.version}</version>

http://git-wip-us.apache.org/repos/asf/calcite/blob/0ef3660d/avatica/server/pom.xml
----------------------------------------------------------------------
diff --git a/avatica/server/pom.xml b/avatica/server/pom.xml
index 8126720..80864aa 100644
--- a/avatica/server/pom.xml
+++ b/avatica/server/pom.xml
@@ -132,6 +132,16 @@ limitations under the License.
       <artifactId>slf4j-log4j12</artifactId>
       <scope>test</scope>
     </dependency>
+    <dependency>
+      <groupId>org.bouncycastle</groupId>
+      <artifactId>bcpkix-jdk15on</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.bouncycastle</groupId>
+      <artifactId>bcprov-jdk15on</artifactId>
+      <scope>test</scope>
+    </dependency>
   </dependencies>
 
   <build>

http://git-wip-us.apache.org/repos/asf/calcite/blob/0ef3660d/avatica/server/src/main/java/org/apache/calcite/avatica/server/HttpServer.java
----------------------------------------------------------------------
diff --git a/avatica/server/src/main/java/org/apache/calcite/avatica/server/HttpServer.java b/avatica/server/src/main/java/org/apache/calcite/avatica/server/HttpServer.java
index 3050f72..a8524ec 100644
--- a/avatica/server/src/main/java/org/apache/calcite/avatica/server/HttpServer.java
+++ b/avatica/server/src/main/java/org/apache/calcite/avatica/server/HttpServer.java
@@ -37,6 +37,7 @@ import org.eclipse.jetty.server.ServerConnector;
 import org.eclipse.jetty.server.handler.DefaultHandler;
 import org.eclipse.jetty.server.handler.HandlerList;
 import org.eclipse.jetty.util.security.Constraint;
+import org.eclipse.jetty.util.ssl.SslContextFactory;
 import org.eclipse.jetty.util.thread.QueuedThreadPool;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -73,6 +74,7 @@ public class HttpServer {
   private final AvaticaHandler handler;
   private final AvaticaServerConfiguration config;
   private final Subject subject;
+  private final SslContextFactory sslFactory;
 
   @Deprecated
   public HttpServer(Handler handler) {
@@ -120,10 +122,24 @@ public class HttpServer {
    */
   public HttpServer(int port, AvaticaHandler handler, AvaticaServerConfiguration config,
       Subject subject) {
+    this(port, handler, config, subject, null);
+  }
+
+  /**
+   * Constructs an {@link HttpServer}.
+   * @param port The listen port
+   * @param handler The Handler to run
+   * @param config Optional configuration for the server
+   * @param subject The javax.security Subject for the server, or null
+   * @param sslFactory A configured SslContextFactory, or null
+   */
+  public HttpServer(int port, AvaticaHandler handler, AvaticaServerConfiguration config,
+      Subject subject, SslContextFactory sslFactory) {
     this.port = port;
     this.handler = handler;
     this.config = config;
     this.subject = subject;
+    this.sslFactory = sslFactory;
   }
 
   private static AvaticaHandler wrapJettyHandler(Handler handler) {
@@ -158,7 +174,7 @@ public class HttpServer {
     server = new Server(threadPool);
     server.manage(threadPool);
 
-    final ServerConnector connector = configureConnector(new ServerConnector(server), port);
+    final ServerConnector connector = configureConnector(getConnector(), port);
     ConstraintSecurityHandler securityHandler = null;
 
     if (null != this.config) {
@@ -212,6 +228,13 @@ public class HttpServer {
     }
   }
 
+  private ServerConnector getConnector() {
+    if (null == sslFactory) {
+      return new ServerConnector(server);
+    }
+    return new ServerConnector(server, sslFactory);
+  }
+
   private RpcMetadataResponse createRpcServerMetadata(ServerConnector connector) throws
       UnknownHostException {
     String host = connector.getHost();
@@ -383,6 +406,12 @@ public class HttpServer {
     private String loginServiceProperties;
     private String[] loginServiceAllowedRoles;
 
+    private boolean usingTLS = false;
+    private File keystore;
+    private String keystorePassword;
+    private File truststore;
+    private String truststorePassword;
+
     public Builder() {}
 
     public Builder withPort(int port) {
@@ -561,6 +590,25 @@ public class HttpServer {
     }
 
     /**
+     * Configures the server to use TLS for wire encryption.
+     *
+     * @param keystore The server's keystore
+     * @param keystorePassword The keystore's password
+     * @param truststore The truststore containing the key used to generate the server's key
+     * @param truststorePassword The truststore's password
+     * @return <code>this</code>
+     */
+    public Builder withTLS(File keystore, String keystorePassword, File truststore,
+        String truststorePassword) {
+      this.usingTLS = true;
+      this.keystore = Objects.requireNonNull(keystore);
+      this.keystorePassword = Objects.requireNonNull(keystorePassword);
+      this.truststore = Objects.requireNonNull(truststore);
+      this.truststorePassword = Objects.requireNonNull(truststorePassword);
+      return this;
+    }
+
+    /**
      * Builds the HttpServer instance from <code>this</code>.
      * @return An HttpServer.
      */
@@ -579,6 +627,9 @@ public class HttpServer {
         subject = null;
         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);
@@ -593,8 +644,16 @@ public class HttpServer {
       }
 
       AvaticaHandler handler = buildHandler(this, serverConfig);
+      SslContextFactory sslFactory = null;
+      if (usingTLS) {
+        sslFactory = new SslContextFactory();
+        sslFactory.setKeyStorePath(this.keystore.getAbsolutePath());
+        sslFactory.setKeyStorePassword(keystorePassword);
+        sslFactory.setTrustStorePath(truststore.getAbsolutePath());
+        sslFactory.setTrustStorePassword(truststorePassword);
+      }
 
-      return new HttpServer(port, handler, serverConfig, subject);
+      return new HttpServer(port, handler, serverConfig, subject, sslFactory);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/calcite/blob/0ef3660d/avatica/server/src/test/java/org/apache/calcite/avatica/SslDriverTest.java
----------------------------------------------------------------------
diff --git a/avatica/server/src/test/java/org/apache/calcite/avatica/SslDriverTest.java b/avatica/server/src/test/java/org/apache/calcite/avatica/SslDriverTest.java
new file mode 100644
index 0000000..9dcdfa2
--- /dev/null
+++ b/avatica/server/src/test/java/org/apache/calcite/avatica/SslDriverTest.java
@@ -0,0 +1,224 @@
+/*
+ * 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;
+
+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.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.CertIOException;
+import org.bouncycastle.cert.X509CertificateHolder;
+import org.bouncycastle.cert.jcajce.JcaX509ExtensionUtils;
+import org.bouncycastle.cert.jcajce.JcaX509v3CertificateBuilder;
+import org.bouncycastle.jce.provider.BouncyCastleProvider;
+import org.bouncycastle.jce.provider.X509CertificateObject;
+import org.bouncycastle.operator.OperatorCreationException;
+import org.bouncycastle.operator.jcajce.JcaContentSignerBuilder;
+import org.junit.AfterClass;
+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.NoSuchProviderException;
+import java.security.PrivateKey;
+import java.security.PublicKey;
+import java.security.Security;
+import java.security.cert.Certificate;
+import java.security.cert.CertificateException;
+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.List;
+import java.util.Objects;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Test case for Avatica with TLS connectors.
+ */
+@RunWith(Parameterized.class)
+public class SslDriverTest {
+  private static final Logger LOG = LoggerFactory.getLogger(SslDriverTest.class);
+
+  private static File keystore;
+  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<>();
+
+  @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");
+    keystore = new File(target, "avatica-test.jks");
+    if (keystore.isFile()) {
+      assertTrue("Failed to delete keystore: " + keystore, keystore.delete());
+    }
+    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);
+
+    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;
+  }
+
+  @AfterClass public static void stopKdc() throws Exception {
+    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";
+    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));
+    }
+  }
+
+  /**
+   * 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();
+
+        X509CertificateObject 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, NoSuchProviderException {
+      KeyPairGenerator gen = KeyPairGenerator.getInstance(ENC_ALGORITHM);
+      gen.initialize(2048);
+      return gen.generateKeyPair();
+    }
+
+    private X509CertificateObject generateCert(String keyName, KeyPair kp, boolean isCertAuthority,
+        PublicKey signerPublicKey, PrivateKey signerPrivateKey) throws IOException,
+        CertIOException, OperatorCreationException, CertificateException,
+        NoSuchAlgorithmException {
+      Calendar startDate = Calendar.getInstance();
+      Calendar endDate = Calendar.getInstance();
+      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 cert = certGen.build(
+          new JcaContentSignerBuilder(SIGNING_ALGORITHM).build(signerPrivateKey));
+      return new X509CertificateObject(cert.toASN1Structure());
+    }
+  }
+}
+
+// End SslDriverTest.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/0ef3660d/avatica/site/_docs/client_reference.md
----------------------------------------------------------------------
diff --git a/avatica/site/_docs/client_reference.md b/avatica/site/_docs/client_reference.md
index 3024ca6..f49372a 100644
--- a/avatica/site/_docs/client_reference.md
+++ b/avatica/site/_docs/client_reference.md
@@ -154,3 +154,21 @@ on-hover images for the permalink, but oh well.
 : _Default_: `null`.
 
 : _Required_: No.
+
+<strong><a name="truststore" href="#truststore">truststore</a></strong>
+
+: _Description_: A path to a Java KeyStore (JKS) file on the local filesystem
+  which contains the certificate authority to trust in a TLS handshake. Only
+  necessary when using HTTPS.
+
+: _Default_: `null`.
+
+: _Required_: No.
+
+<strong><a name="truststore_password" href="#truststore_password">truststore_password</a></strong>
+
+: _Description_: The password for the Java KeyStore file specified by <a href="#truststore">truststore</a>.
+
+: _Default_: `null`.
+
+: _Required_: Only if `truststore` was provided.