You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by st...@apache.org on 2020/01/10 12:34:05 UTC

[phoenix-queryserver] branch master updated: PHOENIX-5642 Add HTTPS support to Phoenix Query Server and thin client

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

stoty pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/phoenix-queryserver.git


The following commit(s) were added to refs/heads/master by this push:
     new 700f62b  PHOENIX-5642 Add HTTPS support to Phoenix Query Server and thin client
700f62b is described below

commit 700f62bc6f53c9b9ec8920a0cc62dc6b222dcc56
Author: Istvan Toth <st...@apache.org>
AuthorDate: Fri Dec 20 07:01:19 2019 +0100

    PHOENIX-5642 Add HTTPS support to Phoenix Query Server and thin client
    
    Closes #14
---
 pom.xml                                            |  32 +--
 .../HttpParamImpersonationQueryServerIT.java       |  75 ++++---
 ...erisedTest.java => QueryServerEnvironment.java} | 218 +++++++++++++--------
 .../phoenix/end2end/SecureQueryServerIT.java       |  41 ++--
 .../phoenix/end2end/ServerCustomizersIT.java       |   6 +-
 .../java/org/apache/phoenix/end2end/TlsUtil.java   |  93 +++++++++
 .../phoenix/queryserver/QueryServerOptions.java    |   5 +
 .../phoenix/queryserver/QueryServerProperties.java |  10 +
 .../phoenix/queryserver/server/QueryServer.java    |  36 +++-
 .../queryserver/server/ServerCustomizersTest.java  |   5 +-
 10 files changed, 360 insertions(+), 161 deletions(-)

diff --git a/pom.xml b/pom.xml
index cd3c003..bd7eafb 100644
--- a/pom.xml
+++ b/pom.xml
@@ -78,7 +78,7 @@
         <jline.version>2.11</jline.version>
         <commons-logging.version>1.2</commons-logging.version>
         <!-- Do not change jodatime.version until HBASE-15199 is fixed -->
-        <avatica.version>1.12.0</avatica.version>
+        <avatica.version>1.16.0</avatica.version>
         <servlet.api.version>3.1.0</servlet.api.version>
         <!-- Test Dependencies -->
         <mockito-all.version>1.8.5</mockito-all.version>
@@ -371,13 +371,6 @@
                     </exclusion>
                 </exclusions>
             </dependency>
-            <dependency>
-                <groupId>org.apache.hadoop</groupId>
-                <artifactId>hadoop-minicluster</artifactId>
-                <version>${hadoop-two.version}</version>
-                <optional>true</optional>
-                <scope>test</scope>
-            </dependency>
 
             <!-- General Dependencies -->
             <dependency>
@@ -477,27 +470,9 @@
             <!-- Required for mini-cluster since hbase built against old version of hadoop -->
             <dependency>
                 <groupId>org.apache.hadoop</groupId>
-                <artifactId>hadoop-mapreduce-client-jobclient</artifactId>
-                <version>${hadoop-two.version}</version>
-                <scope>test</scope>
-            </dependency>
-            <dependency>
-                <groupId>org.apache.hadoop</groupId>
-                <artifactId>hadoop-client</artifactId>
-                <version>${hadoop-two.version}</version>
-                <scope>test</scope>
-            </dependency>
-            <dependency>
-                <groupId>org.apache.hadoop</groupId>
-                <artifactId>hadoop-hdfs</artifactId>
-                <version>${hadoop-two.version}</version>
-                <scope>test</scope>
-            </dependency>
-            <dependency>
-                <groupId>org.apache.hadoop</groupId>
-                <artifactId>hadoop-hdfs</artifactId>
+                <artifactId>hadoop-minicluster</artifactId>
                 <version>${hadoop-two.version}</version>
-                <type>test-jar</type> <!-- this does not work which is typical for maven.-->
+                <optional>true</optional>
                 <scope>test</scope>
             </dependency>
             <dependency>
@@ -506,7 +481,6 @@
                 <version>${hadoop-two.version}</version>
                 <scope>test</scope>
             </dependency>
-
         </dependencies>
     </dependencyManagement>
 
diff --git a/queryserver/src/it/java/org/apache/phoenix/end2end/HttpParamImpersonationQueryServerIT.java b/queryserver/src/it/java/org/apache/phoenix/end2end/HttpParamImpersonationQueryServerIT.java
index e92a837..6d160b7 100644
--- a/queryserver/src/it/java/org/apache/phoenix/end2end/HttpParamImpersonationQueryServerIT.java
+++ b/queryserver/src/it/java/org/apache/phoenix/end2end/HttpParamImpersonationQueryServerIT.java
@@ -17,10 +17,9 @@
 package org.apache.phoenix.end2end;
 
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
 import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 import java.io.File;
 import java.security.PrivilegedExceptionAction;
@@ -44,18 +43,22 @@ import org.apache.hadoop.hbase.security.access.Permission.Action;
 import org.apache.hadoop.hbase.security.token.TokenProvider;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
-import org.apache.phoenix.query.QueryServices;
-import org.apache.phoenix.query.QueryServicesOptions;
 import org.apache.phoenix.queryserver.QueryServerOptions;
 import org.apache.phoenix.queryserver.QueryServerProperties;
 import org.apache.phoenix.queryserver.client.Driver;
-import org.junit.BeforeClass;
+import org.junit.AfterClass;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
 
+@RunWith(Parameterized.class)
 @Category(NeedsOwnMiniClusterTest.class)
-public class HttpParamImpersonationQueryServerIT extends AbstractKerberisedTest {
+public class HttpParamImpersonationQueryServerIT {
+
     private static final Log LOG = LogFactory.getLog(HttpParamImpersonationQueryServerIT.class);
+    private static QueryServerEnvironment environment;
 
     private static final List<TableName> SYSTEM_TABLE_NAMES = Arrays.asList(PhoenixDatabaseMetaData.SYSTEM_CATALOG_HBASE_TABLE_NAME,
         PhoenixDatabaseMetaData.SYSTEM_MUTEX_HBASE_TABLE_NAME,
@@ -63,12 +66,19 @@ public class HttpParamImpersonationQueryServerIT extends AbstractKerberisedTest
         PhoenixDatabaseMetaData.SYSTEM_SCHEMA_HBASE_TABLE_NAME,
         PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_HBASE_TABLE_NAME,
         PhoenixDatabaseMetaData.SYSTEM_STATS_HBASE_TABLE_NAME);
-    /**
-     * Setup and start kerberos, hbase
-     */
-    @BeforeClass
-    public static void setUp() throws Exception {
-        final Configuration conf = UTIL.getConfiguration();
+
+    @Parameters(name = "tls = {0}")
+    public static Iterable<Boolean> data() {
+        return Arrays.asList(new Boolean[] {false, true});
+    }
+
+    public HttpParamImpersonationQueryServerIT(Boolean tls) throws Exception {
+        //Clean up previous environment if any (Junit 4.13 @BeforeParam / @AfterParam would be an alternative)
+        if(environment != null) {
+            stopEnvironment();
+        }
+
+        final Configuration conf = new Configuration();
         conf.setStrings(CoprocessorHost.MASTER_COPROCESSOR_CONF_KEY, AccessController.class.getName());
         conf.setStrings(CoprocessorHost.REGIONSERVER_COPROCESSOR_CONF_KEY, AccessController.class.getName());
         conf.setStrings(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY, AccessController.class.getName(), TokenProvider.class.getName());
@@ -77,20 +87,34 @@ public class HttpParamImpersonationQueryServerIT extends AbstractKerberisedTest
         conf.set("hadoop.proxyuser.user1.groups", "*");
         conf.set("hadoop.proxyuser.user1.hosts", "*");
         conf.setBoolean(QueryServerProperties.QUERY_SERVER_WITH_REMOTEUSEREXTRACTOR_ATTRIB, true);
+        environment = new QueryServerEnvironment(conf, 2, tls);
+    }
 
-        configureAndStartQueryServer(conf, 2);
+    @AfterClass
+    public static void stopEnvironment() throws Exception {
+        environment.stop();
+    }
+
+    private String getUrlTemplate() {
+        String url = Driver.CONNECT_STRING_PREFIX + "url=%s://localhost:" + environment.getPqsPort() + "?"
+                + QueryServerOptions.DEFAULT_QUERY_SERVER_REMOTEUSEREXTRACTOR_PARAM + "=%s;authentication=SPNEGO;serialization=PROTOBUF%s";
+        if(environment.getTls()) {
+            return String.format(url, "https", "%s", ";truststore=" +TlsUtil.getTrustStoreFile().getAbsolutePath() 
+                + ";truststore_password="+TlsUtil.getTrustStorePassword());
+        } else {
+            return String.format(url, "http", "%s", "");
+        }
     }
 
     @Test
     public void testSuccessfulImpersonation() throws Exception {
-        final Entry<String,File> user1 = getUser(1);
-        final Entry<String,File> user2 = getUser(2);
+        final Entry<String,File> user1 = environment.getUser(1);
+        final Entry<String,File> user2 = environment.getUser(2);
         // Build the JDBC URL by hand with the doAs
-        final String doAsUrlTemplate = Driver.CONNECT_STRING_PREFIX + "url=http://localhost:" + PQS_PORT + "?"
-            + QueryServerOptions.DEFAULT_QUERY_SERVER_REMOTEUSEREXTRACTOR_PARAM + "=%s;authentication=SPNEGO;serialization=PROTOBUF";
+        final String doAsUrlTemplate = getUrlTemplate();
         final String tableName = "POSITIVE_IMPERSONATION";
         final int numRows = 5;
-        final UserGroupInformation serviceUgi = UserGroupInformation.loginUserFromKeytabAndReturnUGI(SERVICE_PRINCIPAL, KEYTAB.getAbsolutePath());
+        final UserGroupInformation serviceUgi = UserGroupInformation.loginUserFromKeytabAndReturnUGI(environment.getServicePrincipal(), environment.getServiceKeytab().getAbsolutePath());
         serviceUgi.doAs(new PrivilegedExceptionAction<Void>() {
             @Override public Void run() throws Exception {
                 createTable(tableName, numRows);
@@ -102,7 +126,7 @@ public class HttpParamImpersonationQueryServerIT extends AbstractKerberisedTest
         user1Ugi.doAs(new PrivilegedExceptionAction<Void>() {
             @Override public Void run() throws Exception {
                 // This user should not be able to read the table
-                readAndExpectPermissionError(PQS_URL, tableName, numRows);
+                readAndExpectPermissionError(environment.getPqsUrl(), tableName, numRows);
                 // Run the same query with the same credentials, but with a doAs. We should be permitted since the user we're impersonating can run the query
                 final String doAsUrl = String.format(doAsUrlTemplate, serviceUgi.getShortUserName());
                 try (Connection conn = DriverManager.getConnection(doAsUrl);
@@ -117,13 +141,12 @@ public class HttpParamImpersonationQueryServerIT extends AbstractKerberisedTest
 
     @Test
     public void testDisallowedImpersonation() throws Exception {
-        final Entry<String,File> user2 = getUser(2);
+        final Entry<String,File> user2 = environment.getUser(2);
         // Build the JDBC URL by hand with the doAs
-        final String doAsUrlTemplate = Driver.CONNECT_STRING_PREFIX + "url=http://localhost:" + PQS_PORT + "?"
-            + QueryServerOptions.DEFAULT_QUERY_SERVER_REMOTEUSEREXTRACTOR_PARAM + "=%s;authentication=SPNEGO;serialization=PROTOBUF";
+        final String doAsUrlTemplate = getUrlTemplate();
         final String tableName = "DISALLOWED_IMPERSONATION";
         final int numRows = 5;
-        final UserGroupInformation serviceUgi = UserGroupInformation.loginUserFromKeytabAndReturnUGI(SERVICE_PRINCIPAL, KEYTAB.getAbsolutePath());
+        final UserGroupInformation serviceUgi = UserGroupInformation.loginUserFromKeytabAndReturnUGI(environment.getServicePrincipal(), environment.getServiceKeytab().getAbsolutePath());
         serviceUgi.doAs(new PrivilegedExceptionAction<Void>() {
             @Override public Void run() throws Exception {
                 createTable(tableName, numRows);
@@ -135,7 +158,7 @@ public class HttpParamImpersonationQueryServerIT extends AbstractKerberisedTest
         user2Ugi.doAs(new PrivilegedExceptionAction<Void>() {
             @Override public Void run() throws Exception {
                 // This user is disallowed to read this table
-                readAndExpectPermissionError(PQS_URL, tableName, numRows);
+                readAndExpectPermissionError(environment.getPqsUrl(), tableName, numRows);
                 // This user is also not allowed to impersonate
                 final String doAsUrl = String.format(doAsUrlTemplate, serviceUgi.getShortUserName());
                 try (Connection conn = DriverManager.getConnection(doAsUrl);
@@ -152,7 +175,7 @@ public class HttpParamImpersonationQueryServerIT extends AbstractKerberisedTest
     }
 
     void createTable(String tableName, int numRows) throws Exception {
-        try (Connection conn = DriverManager.getConnection(PQS_URL);
+        try (Connection conn = DriverManager.getConnection(environment.getPqsUrl());
             Statement stmt = conn.createStatement()) {
             conn.setAutoCommit(true);
             assertFalse(stmt.execute("CREATE TABLE " + tableName + "(pk integer not null primary key)"));
@@ -168,7 +191,7 @@ public class HttpParamImpersonationQueryServerIT extends AbstractKerberisedTest
         try {
             for (String user : usersToGrant) {
                 for (TableName tn : SYSTEM_TABLE_NAMES) {
-                    AccessControlClient.grant(UTIL.getConnection(), tn, user, null, null, Action.READ, Action.EXEC);
+                    AccessControlClient.grant(environment.getUtil().getConnection(), tn, user, null, null, Action.READ, Action.EXEC);
                 }
             }
         } catch (Throwable e) {
diff --git a/queryserver/src/it/java/org/apache/phoenix/end2end/AbstractKerberisedTest.java b/queryserver/src/it/java/org/apache/phoenix/end2end/QueryServerEnvironment.java
similarity index 61%
rename from queryserver/src/it/java/org/apache/phoenix/end2end/AbstractKerberisedTest.java
rename to queryserver/src/it/java/org/apache/phoenix/end2end/QueryServerEnvironment.java
index db2d9b6..165e643 100644
--- a/queryserver/src/it/java/org/apache/phoenix/end2end/AbstractKerberisedTest.java
+++ b/queryserver/src/it/java/org/apache/phoenix/end2end/QueryServerEnvironment.java
@@ -1,16 +1,31 @@
 /*
- * 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.
+ * 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.phoenix.end2end;
 
-import com.google.common.base.Preconditions;
-import com.google.common.collect.Maps;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
+import java.io.File;
+import java.io.IOException;
+import java.lang.reflect.Field;
+import java.net.InetAddress;
+import java.security.PrivilegedAction;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+
 import org.apache.commons.io.FileUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -19,10 +34,8 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.LocalHBaseCluster;
-import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
 import org.apache.hadoop.hbase.http.ssl.KeyStoreTestUtil;
 import org.apache.hadoop.hbase.security.HBaseKerberosUtils;
-import org.apache.hadoop.hbase.security.token.TokenProvider;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.http.HttpConfig;
@@ -31,78 +44,93 @@ import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authentication.util.KerberosName;
 import org.apache.phoenix.query.ConfigurationFactory;
 import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.queryserver.QueryServerProperties;
 import org.apache.phoenix.queryserver.client.ThinClientUtil;
 import org.apache.phoenix.queryserver.server.QueryServer;
 import org.apache.phoenix.util.InstanceResolver;
-import org.junit.AfterClass;
-
-import java.io.File;
-import java.io.IOException;
-import java.lang.reflect.Field;
-import java.net.InetAddress;
-import java.security.PrivilegedAction;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-import java.util.UUID;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.TimeUnit;
 
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Maps;
 
 /**
- * Due to this bug https://bugzilla.redhat.com/show_bug.cgi?id=668830
- * We need to use `localhost.localdomain` as host name when running these tests on Jenkins (Centos)
- * but for Mac OS it should be `localhost` to pass.
- * The reason is kerberos principals in this tests are looked up from /etc/hosts
- * and a reverse DNS lookup of 127.0.0.1 is resolved to `localhost.localdomain` rather than `localhost` on Centos.
- * KDC sees `localhost` != `localhost.localdomain` and as the result test fails with authentication error.
- * It's also important to note these principals are shared between HDFs and HBase in this mini HBase cluster.
- * Some more reading https://access.redhat.com/solutions/57330
+ * Due to this bug https://bugzilla.redhat.com/show_bug.cgi?id=668830 We need to use
+ * `localhost.localdomain` as host name when running these tests on Jenkins (Centos) but for Mac OS
+ * it should be `localhost` to pass. The reason is kerberos principals in this tests are looked up
+ * from /etc/hosts and a reverse DNS lookup of 127.0.0.1 is resolved to `localhost.localdomain`
+ * rather than `localhost` on Centos. KDC sees `localhost` != `localhost.localdomain` and as the
+ * result test fails with authentication error. It's also important to note these principals are
+ * shared between HDFs and HBase in this mini HBase cluster. Some more reading
+ * https://access.redhat.com/solutions/57330
  */
-public class AbstractKerberisedTest {
-	private static final Log LOG = LogFactory.getLog(AbstractKerberisedTest.class);
+public class QueryServerEnvironment {
+    private static final Log LOG = LogFactory.getLog(QueryServerEnvironment.class);
 
-	private static final File TEMP_DIR = new File(getTempDirForClass());
-    private static final File KEYTAB_DIR = new File(TEMP_DIR, "keytabs");
-    private static final List<File> USER_KEYTAB_FILES = new ArrayList<>();
+    private final File TEMP_DIR = new File(getTempDir());
+    private final File KEYTAB_DIR = new File(TEMP_DIR, "keytabs");
+    private final List<File> USER_KEYTAB_FILES = new ArrayList<>();
 
     private static final String LOCAL_HOST_REVERSE_DNS_LOOKUP_NAME;
 
     static {
         try {
-            LOCAL_HOST_REVERSE_DNS_LOOKUP_NAME = InetAddress.getByName("127.0.0.1").getCanonicalHostName();
+            LOCAL_HOST_REVERSE_DNS_LOOKUP_NAME =
+                    InetAddress.getByName("127.0.0.1").getCanonicalHostName();
         } catch (Exception e) {
             throw new RuntimeException(e);
         }
     }
 
-    protected static final String SPNEGO_PRINCIPAL = "HTTP/" + LOCAL_HOST_REVERSE_DNS_LOOKUP_NAME;
-    protected static final String PQS_PRINCIPAL = "phoenixqs/" + LOCAL_HOST_REVERSE_DNS_LOOKUP_NAME;
-    protected static final String SERVICE_PRINCIPAL = "securecluster/" + LOCAL_HOST_REVERSE_DNS_LOOKUP_NAME;
-    protected static File KEYTAB;
+    private static final String SPNEGO_PRINCIPAL = "HTTP/" + LOCAL_HOST_REVERSE_DNS_LOOKUP_NAME;
+    private static final String PQS_PRINCIPAL = "phoenixqs/" + LOCAL_HOST_REVERSE_DNS_LOOKUP_NAME;
+    private static final String SERVICE_PRINCIPAL =
+            "securecluster/" + LOCAL_HOST_REVERSE_DNS_LOOKUP_NAME;
+    private File KEYTAB;
+
+    private MiniKdc KDC;
+    private HBaseTestingUtility UTIL = new HBaseTestingUtility();
+    private LocalHBaseCluster HBASE_CLUSTER;
+    private int NUM_CREATED_USERS;
 
-	protected static MiniKdc KDC;
-    protected static HBaseTestingUtility UTIL = new HBaseTestingUtility();
-    protected static LocalHBaseCluster HBASE_CLUSTER;
-    protected static int NUM_CREATED_USERS;
+    private ExecutorService PQS_EXECUTOR;
+    private QueryServer PQS;
+    private int PQS_PORT;
+    private String PQS_URL;
 
-    protected static ExecutorService PQS_EXECUTOR;
-    protected static QueryServer PQS;
-    protected static int PQS_PORT;
-    protected static String PQS_URL;
+    private boolean tls;
 
-    private static String getTempDirForClass() {
+    private static String getTempDir() {
         StringBuilder sb = new StringBuilder(32);
         sb.append(System.getProperty("user.dir")).append(File.separator);
         sb.append("target").append(File.separator);
-        sb.append(AbstractKerberisedTest.class.getSimpleName());
+        sb.append(QueryServerEnvironment.class.getSimpleName());
         sb.append("-").append(UUID.randomUUID());
         return sb.toString();
     }
 
+    public int getPqsPort() {
+        return PQS_PORT;
+    }
+
+    public String getPqsUrl() {
+        return PQS_URL;
+    }
+
+    public boolean getTls() {
+        return tls;
+    }
+
+    public HBaseTestingUtility getUtil() {
+        return UTIL;
+    }
+
+    public String getServicePrincipal() {
+        return SERVICE_PRINCIPAL;
+    }
+
+    public File getServiceKeytab() {
+        return KEYTAB;
+    }
+
     private static void updateDefaultRealm() throws Exception {
         // (at least) one other phoenix test triggers the caching of this field before the KDC is up
         // which causes principal parsing to fail.
@@ -112,7 +140,7 @@ public class AbstractKerberisedTest {
         f.set(null, "EXAMPLE.COM");
     }
 
-    private static void createUsers(int numUsers) throws Exception {
+    private void createUsers(int numUsers) throws Exception {
         assertNotNull("KDC is null, was setup method called?", KDC);
         NUM_CREATED_USERS = numUsers;
         for (int i = 1; i <= numUsers; i++) {
@@ -123,7 +151,7 @@ public class AbstractKerberisedTest {
         }
     }
 
-    protected static Map.Entry<String,File> getUser(int offset) {
+    public Map.Entry<String, File> getUser(int offset) {
         Preconditions.checkArgument(offset > 0 && offset <= NUM_CREATED_USERS);
         return Maps.immutableEntry("user" + offset, USER_KEYTAB_FILES.get(offset - 1));
     }
@@ -131,13 +159,16 @@ public class AbstractKerberisedTest {
     /**
      * Setup the security configuration for hdfs.
      */
-    private static void setHdfsSecuredConfiguration(Configuration conf) throws Exception {
+    private void setHdfsSecuredConfiguration(Configuration conf) throws Exception {
         // Set principal+keytab configuration for HDFS
-        conf.set(DFSConfigKeys.DFS_NAMENODE_KERBEROS_PRINCIPAL_KEY, SERVICE_PRINCIPAL + "@" + KDC.getRealm());
+        conf.set(DFSConfigKeys.DFS_NAMENODE_KERBEROS_PRINCIPAL_KEY,
+            SERVICE_PRINCIPAL + "@" + KDC.getRealm());
         conf.set(DFSConfigKeys.DFS_NAMENODE_KEYTAB_FILE_KEY, KEYTAB.getAbsolutePath());
-        conf.set(DFSConfigKeys.DFS_DATANODE_KERBEROS_PRINCIPAL_KEY, SERVICE_PRINCIPAL + "@" + KDC.getRealm());
+        conf.set(DFSConfigKeys.DFS_DATANODE_KERBEROS_PRINCIPAL_KEY,
+            SERVICE_PRINCIPAL + "@" + KDC.getRealm());
         conf.set(DFSConfigKeys.DFS_DATANODE_KEYTAB_FILE_KEY, KEYTAB.getAbsolutePath());
-        conf.set(DFSConfigKeys.DFS_WEB_AUTHENTICATION_KERBEROS_PRINCIPAL_KEY, SPNEGO_PRINCIPAL + "@" + KDC.getRealm());
+        conf.set(DFSConfigKeys.DFS_WEB_AUTHENTICATION_KERBEROS_PRINCIPAL_KEY,
+            SPNEGO_PRINCIPAL + "@" + KDC.getRealm());
         // Enable token access for HDFS blocks
         conf.setBoolean(DFSConfigKeys.DFS_BLOCK_ACCESS_TOKEN_ENABLE_KEY, true);
         // Only use HTTPS (required because we aren't using "secure" ports)
@@ -149,7 +180,7 @@ public class AbstractKerberisedTest {
         // Generate SSL certs
         File keystoresDir = new File(UTIL.getDataTestDir("keystore").toUri().getPath());
         keystoresDir.mkdirs();
-        String sslConfDir = KeyStoreTestUtil.getClasspathDir(AbstractKerberisedTest.class);
+        String sslConfDir = KeyStoreTestUtil.getClasspathDir(QueryServerEnvironment.class);
         KeyStoreTestUtil.setupSSLConfig(keystoresDir.getAbsolutePath(), sslConfDir, conf, false);
 
         // Magic flag to tell hdfs to not fail on using ports above 1024
@@ -169,8 +200,14 @@ public class AbstractKerberisedTest {
 
     /**
      * Setup and start kerberosed, hbase
+     * @throws Exception
      */
-    public static void configureAndStartQueryServer(final Configuration conf, int numberOfUsers) throws Exception {
+    public QueryServerEnvironment(final Configuration confIn, int numberOfUsers, boolean tls)
+            throws Exception {
+        this.tls = tls;
+
+        Configuration conf = UTIL.getConfiguration();
+        conf.addResource(confIn);
         // Ensure the dirs we need are created/empty
         ensureIsEmptyDirectory(TEMP_DIR);
         ensureIsEmptyDirectory(KEYTAB_DIR);
@@ -179,9 +216,9 @@ public class AbstractKerberisedTest {
         KDC = UTIL.setupMiniKdc(KEYTAB);
         // Create a service principal and spnego principal in one keytab
         // NB. Due to some apparent limitations between HDFS and HBase in the same JVM, trying to
-        //     use separate identies for HBase and HDFS results in a GSS initiate error. The quick
-        //     solution is to just use a single "service" principal instead of "hbase" and "hdfs"
-        //     (or "dn" and "nn") per usual.
+        // use separate identies for HBase and HDFS results in a GSS initiate error. The quick
+        // solution is to just use a single "service" principal instead of "hbase" and "hdfs"
+        // (or "dn" and "nn") per usual.
         KDC.createPrincipal(KEYTAB, SPNEGO_PRINCIPAL, PQS_PRINCIPAL, SERVICE_PRINCIPAL);
         // Start ZK by hand
         UTIL.startMiniZKCluster();
@@ -199,8 +236,21 @@ public class AbstractKerberisedTest {
         conf.setInt(HConstants.REGIONSERVER_PORT, 0);
         conf.setInt(HConstants.REGIONSERVER_INFO_PORT, 0);
 
+        if (tls) {
+            conf.setBoolean(QueryServerProperties.QUERY_SERVER_TLS_ENABLED, true);
+            conf.set(QueryServerProperties.QUERY_SERVER_TLS_KEYSTORE,
+                TlsUtil.getKeyStoreFile().getAbsolutePath());
+            conf.set(QueryServerProperties.QUERY_SERVER_TLS_KEYSTORE_PASSWORD,
+                TlsUtil.getKeyStorePassword());
+            conf.set(QueryServerProperties.QUERY_SERVER_TLS_TRUSTSTORE,
+                TlsUtil.getTrustStoreFile().getAbsolutePath());
+            conf.set(QueryServerProperties.QUERY_SERVER_TLS_TRUSTSTORE_PASSWORD,
+                TlsUtil.getTrustStorePassword());
+        }
+
         // Secure Phoenix setup
-        conf.set("phoenix.queryserver.kerberos.http.principal", SPNEGO_PRINCIPAL + "@" + KDC.getRealm());
+        conf.set("phoenix.queryserver.kerberos.http.principal",
+            SPNEGO_PRINCIPAL + "@" + KDC.getRealm());
         conf.set("phoenix.queryserver.http.keytab.file", KEYTAB.getAbsolutePath());
         conf.set("phoenix.queryserver.kerberos.principal", PQS_PRINCIPAL + "@" + KDC.getRealm());
         conf.set("phoenix.queryserver.keytab.file", KEYTAB.getAbsolutePath());
@@ -218,6 +268,7 @@ public class AbstractKerberisedTest {
             public Configuration getConfiguration() {
                 return conf;
             }
+
             @Override
             public Configuration getConfiguration(Configuration confToClone) {
                 Configuration copy = new Configuration(conf);
@@ -231,27 +282,31 @@ public class AbstractKerberisedTest {
         UTIL.startMiniDFSCluster(1);
         // Use LocalHBaseCluster to avoid HBaseTestingUtility from doing something wrong
         // NB. I'm not actually sure what HTU does incorrect, but this was pulled from some test
-        //     classes in HBase itself. I couldn't get HTU to work myself (2017/07/06)
-        Path rootdir = UTIL.getDataTestDirOnTestFS(AbstractKerberisedTest.class.getSimpleName());
+        // classes in HBase itself. I couldn't get HTU to work myself (2017/07/06)
+        Path rootdir = UTIL.getDataTestDirOnTestFS(QueryServerEnvironment.class.getSimpleName());
         FSUtils.setRootDir(conf, rootdir);
         HBASE_CLUSTER = new LocalHBaseCluster(conf, 1);
         HBASE_CLUSTER.startup();
 
         // Then fork a thread with PQS in it.
-        configureAndStartQueryServer();
+        configureAndStartQueryServer(tls);
     }
 
-    private static void configureAndStartQueryServer() throws Exception {
+    private void configureAndStartQueryServer(boolean tls) throws Exception {
         PQS = new QueryServer(new String[0], UTIL.getConfiguration());
         // Get the PQS ident for PQS to use
-        final UserGroupInformation ugi = UserGroupInformation.loginUserFromKeytabAndReturnUGI(PQS_PRINCIPAL, KEYTAB.getAbsolutePath());
+        final UserGroupInformation ugi =
+                UserGroupInformation.loginUserFromKeytabAndReturnUGI(PQS_PRINCIPAL,
+                    KEYTAB.getAbsolutePath());
         PQS_EXECUTOR = Executors.newSingleThreadExecutor();
         // Launch PQS, doing in the Kerberos login instead of letting PQS do it itself (which would
         // break the HBase/HDFS logins also running in the same test case).
         PQS_EXECUTOR.submit(new Runnable() {
-            @Override public void run() {
+            @Override
+            public void run() {
                 ugi.doAs(new PrivilegedAction<Void>() {
-                    @Override public Void run() {
+                    @Override
+                    public Void run() {
                         PQS.run();
                         return null;
                     }
@@ -260,11 +315,15 @@ public class AbstractKerberisedTest {
         });
         PQS.awaitRunning();
         PQS_PORT = PQS.getPort();
-        PQS_URL = ThinClientUtil.getConnectionUrl("localhost", PQS_PORT) + ";authentication=SPNEGO";
+        PQS_URL =
+                ThinClientUtil.getConnectionUrl(tls ? "https" : "http", "localhost", PQS_PORT)
+                        + ";authentication=SPNEGO" + (tls
+                                ? ";truststore=" + TlsUtil.getTrustStoreFile().getAbsolutePath()
+                                        + ";truststore_password=" + TlsUtil.getTrustStorePassword()
+                                : "");
     }
 
-    @AfterClass
-    public static void stopKdc() throws Exception {
+    public void stop() throws Exception {
         // Remove our custom ConfigurationFactory for future tests
         InstanceResolver.clearSingletons();
         if (PQS_EXECUTOR != null) {
@@ -286,9 +345,4 @@ public class AbstractKerberisedTest {
         }
     }
 
-    protected byte[] copyBytes(byte[] src, int offset, int length) {
-        byte[] dest = new byte[length];
-        System.arraycopy(src, offset, dest, 0, length);
-        return dest;
-    }
 }
diff --git a/queryserver/src/it/java/org/apache/phoenix/end2end/SecureQueryServerIT.java b/queryserver/src/it/java/org/apache/phoenix/end2end/SecureQueryServerIT.java
index 00e89f9..5c59b52 100644
--- a/queryserver/src/it/java/org/apache/phoenix/end2end/SecureQueryServerIT.java
+++ b/queryserver/src/it/java/org/apache/phoenix/end2end/SecureQueryServerIT.java
@@ -18,7 +18,6 @@ package org.apache.phoenix.end2end;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 
 import java.io.File;
@@ -26,6 +25,7 @@ import java.security.PrivilegedExceptionAction;
 import java.sql.DriverManager;
 import java.sql.ResultSet;
 import java.sql.Statement;
+import java.util.Arrays;
 import java.util.Map.Entry;
 
 import org.apache.commons.logging.Log;
@@ -34,34 +34,51 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
 import org.apache.hadoop.hbase.security.token.TokenProvider;
 import org.apache.hadoop.security.UserGroupInformation;
-import org.junit.BeforeClass;
+import org.junit.AfterClass;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
 
+@RunWith(Parameterized.class)
 @Category(NeedsOwnMiniClusterTest.class)
-public class SecureQueryServerIT extends AbstractKerberisedTest {
+public class SecureQueryServerIT {
     private static final Log LOG = LogFactory.getLog(SecureQueryServerIT.class);
+    private static QueryServerEnvironment environment;
 
-    /**
-     * Setup and start kerberos, hbase
-     */
-    @BeforeClass
-    public static void setUp() throws Exception {
-        final Configuration conf = UTIL.getConfiguration();
+    @Parameters(name = "tls = {0}")
+    public static Iterable<Boolean> data() {
+        return Arrays.asList(new Boolean[] {false, true});
+    }
+
+    public SecureQueryServerIT(Boolean tls) throws Exception {
+        //Clean up previous environment if any (Junit 4.13 @BeforeParam / @AfterParam would be an alternative)
+        if(environment != null) {
+            stopEnvironment();
+        }
+
+        final Configuration conf = new Configuration();
         conf.setStrings(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY,
                     TokenProvider.class.getName());
-        configureAndStartQueryServer(conf, 3);
+        environment = new QueryServerEnvironment(conf, 3, tls);
+    }
+
+
+    @AfterClass
+    public static void stopEnvironment() throws Exception {
+        environment.stop();
     }
 
     @Test
     public void testBasicReadWrite() throws Exception {
-        final Entry<String,File> user1 = getUser(1);
+        final Entry<String,File> user1 = environment.getUser(1);
         UserGroupInformation user1Ugi = UserGroupInformation.loginUserFromKeytabAndReturnUGI(user1.getKey(), user1.getValue().getAbsolutePath());
         user1Ugi.doAs(new PrivilegedExceptionAction<Void>() {
             @Override public Void run() throws Exception {
                 // Phoenix
                 final String tableName = "phx_table1";
-                try (java.sql.Connection conn = DriverManager.getConnection(PQS_URL);
+                try (java.sql.Connection conn = DriverManager.getConnection(environment.getPqsUrl());
                         Statement stmt = conn.createStatement()) {
                     conn.setAutoCommit(true);
                     assertFalse(stmt.execute("CREATE TABLE " + tableName + "(pk integer not null primary key)"));
diff --git a/queryserver/src/it/java/org/apache/phoenix/end2end/ServerCustomizersIT.java b/queryserver/src/it/java/org/apache/phoenix/end2end/ServerCustomizersIT.java
index 699000f..9c85337 100644
--- a/queryserver/src/it/java/org/apache/phoenix/end2end/ServerCustomizersIT.java
+++ b/queryserver/src/it/java/org/apache/phoenix/end2end/ServerCustomizersIT.java
@@ -34,6 +34,7 @@ import org.apache.phoenix.util.InstanceResolver;
 import org.eclipse.jetty.security.ConstraintMapping;
 import org.eclipse.jetty.security.ConstraintSecurityHandler;
 import org.eclipse.jetty.security.HashLoginService;
+import org.eclipse.jetty.security.UserStore;
 import org.eclipse.jetty.security.authentication.BasicAuthenticator;
 import org.eclipse.jetty.server.Server;
 import org.eclipse.jetty.util.security.Constraint;
@@ -123,9 +124,12 @@ public class ServerCustomizersIT extends BaseHBaseManagedTimeIT {
         @Override
         public void customize(Server server) {
             LOG.debug("Customizing server to allow requests for {}", USER_AUTHORIZED);
+
+            UserStore store = new UserStore();
+            store.addUser(USER_AUTHORIZED, Credential.getCredential(USER_PW), new String[] {"users"});
             HashLoginService login = new HashLoginService();
-            login.putUser(USER_AUTHORIZED, Credential.getCredential(USER_PW), new String[] {"users"});
             login.setName("users");
+            login.setUserStore(store);
 
             Constraint constraint = new Constraint();
             constraint.setName(Constraint.__BASIC_AUTH);
diff --git a/queryserver/src/it/java/org/apache/phoenix/end2end/TlsUtil.java b/queryserver/src/it/java/org/apache/phoenix/end2end/TlsUtil.java
new file mode 100644
index 0000000..f210a46
--- /dev/null
+++ b/queryserver/src/it/java/org/apache/phoenix/end2end/TlsUtil.java
@@ -0,0 +1,93 @@
+/*
+ * 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.phoenix.end2end;
+
+import java.io.File;
+import java.security.KeyPair;
+import java.security.cert.X509Certificate;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.http.ssl.KeyStoreTestUtil;
+
+public class TlsUtil {
+
+    protected static final String KEYSTORE_PASSWORD = "avaticasecret";
+    protected static final String TRUSTSTORE_PASSWORD = "avaticasecret";
+
+    protected static final String TARGET_DIR_NAME = System.getProperty("target.dir", "target");
+    protected static final File TARGET_DIR =
+            new File(System.getProperty("user.dir"), TARGET_DIR_NAME);
+    protected static final File KEYSTORE = new File(TARGET_DIR, "avatica-test-ks.jks");
+    protected static final File TRUSTSTORE = new File(TARGET_DIR, "avatica-test-ts.jks");
+
+    private static final Log LOG = LogFactory.getLog(QueryServerBasicsIT.class);
+
+    public static File getTrustStoreFile() {
+        return TRUSTSTORE;
+    }
+
+    public static File getKeyStoreFile() {
+        return KEYSTORE;
+    }
+
+    public static String getTrustStorePassword() {
+        return TRUSTSTORE_PASSWORD;
+    }
+
+    public static String getKeyStorePassword() {
+        return KEYSTORE_PASSWORD;
+    }
+
+    static {
+        try {
+            setupTls();
+        } catch (Exception e) {
+            LOG.error("could not set upt TLS for HTTPS tests", e);
+        }
+    }
+
+    /**
+     * This is simplified from org.apache.hadoop.hbase.http.ssl.KeyStoreTestUtil.setupSSLConfig()
+     * Performs setup of SSL configuration in preparation for testing an SSLFactory. This includes
+     * keys, certs, keystores, truststores.
+     */
+    public static void setupTls() throws Exception {
+
+        try {
+            KEYSTORE.delete();
+        } catch (Exception e) {
+            // may not exist
+        }
+
+        try {
+            TRUSTSTORE.delete();
+        } catch (Exception e) {
+            // may not exist
+        }
+
+        KeyPair sKP = KeyStoreTestUtil.generateKeyPair("RSA");
+        X509Certificate sCert =
+                KeyStoreTestUtil.generateCertificate("CN=localhost, O=server", sKP, 30,
+                    "SHA1withRSA");
+        KeyStoreTestUtil.createKeyStore(KEYSTORE.getCanonicalPath(), KEYSTORE_PASSWORD, "server",
+            sKP.getPrivate(), sCert);
+
+        Map<String, X509Certificate> certs = new HashMap<String, X509Certificate>();
+        certs.put("server", sCert);
+
+        KeyStoreTestUtil.createTrustStore(TRUSTSTORE.getCanonicalPath(), TRUSTSTORE_PASSWORD,
+            certs);
+    }
+
+}
diff --git a/queryserver/src/main/java/org/apache/phoenix/queryserver/QueryServerOptions.java b/queryserver/src/main/java/org/apache/phoenix/queryserver/QueryServerOptions.java
index 33f7771..b8b42cb 100644
--- a/queryserver/src/main/java/org/apache/phoenix/queryserver/QueryServerOptions.java
+++ b/queryserver/src/main/java/org/apache/phoenix/queryserver/QueryServerOptions.java
@@ -39,6 +39,11 @@ public class QueryServerOptions {
     public static final boolean DEFAULT_QUERY_SERVER_DISABLE_KERBEROS_LOGIN = false;
     public static final boolean DEFAULT_QUERY_SERVER_CUSTOMIZERS_ENABLED = false;
 
+    public static final boolean DEFAULT_QUERY_SERVER_TLS_ENABLED = false;
+    //We default to empty *store password
+    public static final String DEFAULT_QUERY_SERVER_TLS_KEYSTORE_PASSWORD = "";
+    public static final String DEFAULT_QUERY_SERVER_TLS_TRUSTSTORE_PASSWORD = "";
+
     @SuppressWarnings("serial")
     public static final Set<String> DEFAULT_QUERY_SERVER_SKIP_WORDS = new HashSet<String>() {
         {
diff --git a/queryserver/src/main/java/org/apache/phoenix/queryserver/QueryServerProperties.java b/queryserver/src/main/java/org/apache/phoenix/queryserver/QueryServerProperties.java
index aa081f6..f550cbe 100644
--- a/queryserver/src/main/java/org/apache/phoenix/queryserver/QueryServerProperties.java
+++ b/queryserver/src/main/java/org/apache/phoenix/queryserver/QueryServerProperties.java
@@ -58,6 +58,16 @@ public class QueryServerProperties {
             "phoenix.queryserver.remoteUserExtractor.param";
     public static final String QUERY_SERVER_DISABLE_KERBEROS_LOGIN =
             "phoenix.queryserver.disable.kerberos.login";
+    public static final String QUERY_SERVER_TLS_ENABLED =
+            "phoenix.queryserver.tls.enabled";
+    public static final String QUERY_SERVER_TLS_KEYSTORE =
+            "phoenix.queryserver.tls.keystore";
+    public static final String QUERY_SERVER_TLS_KEYSTORE_PASSWORD =
+            "phoenix.queryserver.tls.keystore.password";
+    public static final String QUERY_SERVER_TLS_TRUSTSTORE =
+            "phoenix.queryserver.tls.truststore";
+    public static final String QUERY_SERVER_TLS_TRUSTSTORE_PASSWORD =
+            "phoenix.queryserver.tls.truststore.password";
 
     // keys for load balancer
     public static final String PHOENIX_QUERY_SERVER_LOADBALANCER_ENABLED =
diff --git a/queryserver/src/main/java/org/apache/phoenix/queryserver/server/QueryServer.java b/queryserver/src/main/java/org/apache/phoenix/queryserver/server/QueryServer.java
index 8486a79..0e755eb 100644
--- a/queryserver/src/main/java/org/apache/phoenix/queryserver/server/QueryServer.java
+++ b/queryserver/src/main/java/org/apache/phoenix/queryserver/server/QueryServer.java
@@ -30,6 +30,7 @@ import org.apache.calcite.avatica.remote.Service;
 import org.apache.calcite.avatica.server.AvaticaServerConfiguration;
 import org.apache.calcite.avatica.server.DoAsRemoteUserCallback;
 import org.apache.calcite.avatica.server.HttpServer;
+import org.apache.calcite.avatica.server.HttpServer.Builder;
 import org.apache.calcite.avatica.server.RemoteUserExtractor;
 import org.apache.calcite.avatica.server.RemoteUserExtractionException;
 import org.apache.calcite.avatica.server.HttpRequestRemoteUserExtractor;
@@ -248,6 +249,8 @@ public final class QueryServer extends Configured implements Tool, Runnable {
           configureClientAuthentication(builder, disableSpnego, ugi);
         }
         setRemoteUserExtractorIfNecessary(builder, getConf());
+        //Avatica doesn't support TLS with custom auth (Why?), hence we only set it in this branch
+        setTlsIfNeccessary(builder, getConf());
         setHandler(args, builder);
       }
 
@@ -273,7 +276,26 @@ public final class QueryServer extends Configured implements Tool, Runnable {
     }
   }
 
-  @VisibleForTesting
+  private void setTlsIfNeccessary(Builder<Server> builder, Configuration conf) throws Exception {
+    final boolean useTls = getConf().getBoolean(QueryServerProperties.QUERY_SERVER_TLS_ENABLED, QueryServerOptions.DEFAULT_QUERY_SERVER_TLS_ENABLED);
+    if(useTls) {
+      final String tlsKeystore = getConf().get(QueryServerProperties.QUERY_SERVER_TLS_KEYSTORE);
+      final String tlsKeystorePassword = getConf().get(QueryServerProperties.QUERY_SERVER_TLS_KEYSTORE_PASSWORD, QueryServerOptions.DEFAULT_QUERY_SERVER_TLS_KEYSTORE_PASSWORD);
+      final String tlsTruststore = getConf().get(QueryServerProperties.QUERY_SERVER_TLS_TRUSTSTORE);
+      final String tlsTruststorePassword = getConf().get(QueryServerProperties.QUERY_SERVER_TLS_TRUSTSTORE_PASSWORD, QueryServerOptions.DEFAULT_QUERY_SERVER_TLS_TRUSTSTORE_PASSWORD);
+      if(tlsKeystore == null) {
+        throw new Exception(String.format("if %s is enabled, %s must be specfified" , QueryServerProperties.QUERY_SERVER_TLS_ENABLED, QueryServerProperties.QUERY_SERVER_TLS_KEYSTORE));
+      }
+      final File tlsKeystoreFile = new File(tlsKeystore);
+      if(tlsTruststore == null) {
+        throw new Exception(String.format("if %s is enabled, %s must be specfified" , QueryServerProperties.QUERY_SERVER_TLS_ENABLED, QueryServerProperties.QUERY_SERVER_TLS_TRUSTSTORE));
+      }
+      final File tlsTruststoreFile = new File(tlsTruststore);
+      builder.withTLS(tlsKeystoreFile, tlsKeystorePassword, tlsTruststoreFile, tlsTruststorePassword);
+    }
+}
+
+@VisibleForTesting
   void configureClientAuthentication(final HttpServer.Builder builder, boolean disableSpnego, UserGroupInformation ugi) throws IOException {
 
     // Enable SPNEGO for client authentication unless it's explicitly disabled
@@ -372,9 +394,8 @@ public final class QueryServer extends Configured implements Tool, Runnable {
     } catch(Throwable ex){
       LOG.debug("Caught an error trying to register with the load balancer", ex);
       success = false;
-    } finally {
-      return success;
     }
+    return success;
   }
 
 
@@ -386,9 +407,8 @@ public final class QueryServer extends Configured implements Tool, Runnable {
         zookeeperConfig = serviceLocator.iterator().next();
     } catch(ServiceConfigurationError ex) {
       LOG.debug("Unable to locate the service provider for load balancer configuration", ex);
-    } finally {
-      return zookeeperConfig;
     }
+    return zookeeperConfig;
   }
 
   public Registry getRegistry()  {
@@ -399,9 +419,8 @@ public final class QueryServer extends Configured implements Tool, Runnable {
         registry = serviceLocator.iterator().next();
     } catch(ServiceConfigurationError ex) {
       LOG.debug("Unable to locate the zookeeper registry for the load balancer", ex);
-    } finally {
-      return registry;
     }
+    return registry;
   }
 
   public boolean unRegister()  {
@@ -411,9 +430,8 @@ public final class QueryServer extends Configured implements Tool, Runnable {
     }catch(Throwable ex) {
       LOG.debug("Caught an error while de-registering the query server from the load balancer",ex);
       success = false;
-    } finally {
-      return success;
     }
+    return success;
   }
   /**
    * Parses the serialization method from the configuration.
diff --git a/queryserver/src/test/java/org/apache/phoenix/queryserver/server/ServerCustomizersTest.java b/queryserver/src/test/java/org/apache/phoenix/queryserver/server/ServerCustomizersTest.java
index 93e1e37..46e57d9 100644
--- a/queryserver/src/test/java/org/apache/phoenix/queryserver/server/ServerCustomizersTest.java
+++ b/queryserver/src/test/java/org/apache/phoenix/queryserver/server/ServerCustomizersTest.java
@@ -25,6 +25,7 @@ import org.apache.calcite.avatica.server.AvaticaServerConfiguration;
 import org.apache.calcite.avatica.server.ServerCustomizer;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.queryserver.QueryServerProperties;
 import org.apache.phoenix.util.InstanceResolver;
 import org.eclipse.jetty.server.Server;
 import org.junit.After;
@@ -72,7 +73,7 @@ public class ServerCustomizersTest {
             }
         });
         Configuration conf = new Configuration(false);
-        conf.set(QueryServices.QUERY_SERVER_CUSTOMIZERS_ENABLED, "true");
+        conf.set(QueryServerProperties.QUERY_SERVER_CUSTOMIZERS_ENABLED, "true");
         QueryServer queryServer = new QueryServer();
         List<ServerCustomizer<Server>> actual = queryServer.createServerCustomizers(conf, avaticaServerConfiguration);
         Assert.assertEquals("Customizers are different", expected, actual);
@@ -84,7 +85,7 @@ public class ServerCustomizersTest {
         AvaticaServerConfiguration avaticaServerConfiguration = null;
         HttpServer.Builder builder = mock(HttpServer.Builder.class);
         Configuration conf = new Configuration(false);
-        conf.set(QueryServices.QUERY_SERVER_CUSTOMIZERS_ENABLED, "true");
+        conf.set(QueryServerProperties.QUERY_SERVER_CUSTOMIZERS_ENABLED, "true");
         QueryServer queryServer = new QueryServer();
         queryServer.enableServerCustomizersIfNecessary(builder, conf, avaticaServerConfiguration);
         verify(builder).withServerCustomizers(anyList(), any(Class.class));