You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by ja...@apache.org on 2018/09/24 15:28:32 UTC

[27/50] [abbrv] phoenix git commit: PHOENIX-4750 Resolve server customizers and provide them to Avatica

PHOENIX-4750 Resolve server customizers and provide them to Avatica

Resolve server customizers on the PQS classpath and provide them to the
HttpServer builder.

Signed-off-by: Josh Elser <el...@apache.org>


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

Branch: refs/heads/omid2
Commit: dbbb1125d1a219b3dec9250f80734410b7a1a8c8
Parents: fe4c053
Author: Alex Araujo <al...@gmail.com>
Authored: Mon Jun 4 16:32:10 2018 -0700
Committer: Josh Elser <el...@apache.org>
Committed: Thu Aug 16 10:41:11 2018 -0400

----------------------------------------------------------------------
 .../org/apache/phoenix/query/QueryServices.java |   1 +
 .../phoenix/query/QueryServicesOptions.java     |   1 +
 .../phoenix/end2end/QueryServerTestUtil.java    | 187 +++++++++++++++++++
 .../phoenix/end2end/ServerCustomizersIT.java    | 147 +++++++++++++++
 .../phoenix/queryserver/server/QueryServer.java |  26 ++-
 .../server/ServerCustomizersFactory.java        |  49 +++++
 .../server/ServerCustomizersTest.java           |  87 +++++++++
 7 files changed, 496 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/dbbb1125/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
index 2bb9350..d290174 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
@@ -251,6 +251,7 @@ public interface QueryServices extends SQLCloseable {
     public static final String QUERY_SERVER_SPNEGO_AUTH_DISABLED_ATTRIB = "phoenix.queryserver.spnego.auth.disabled";
     public static final String QUERY_SERVER_WITH_REMOTEUSEREXTRACTOR_ATTRIB = "phoenix.queryserver.withRemoteUserExtractor";
     public static final String QUERY_SERVER_REMOTEUSEREXTRACTOR_PARAM = "phoenix.queryserver.remoteUserExtractor.param";
+    public static final String QUERY_SERVER_CUSTOMIZERS_ENABLED = "phoenix.queryserver.customizers.enabled";
     public static final String QUERY_SERVER_DISABLE_KERBEROS_LOGIN = "phoenix.queryserver.disable.kerberos.login";
 
     // metadata configs

http://git-wip-us.apache.org/repos/asf/phoenix/blob/dbbb1125/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java
index 4be8f81..c68e793 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java
@@ -301,6 +301,7 @@ public class QueryServicesOptions {
     public static final boolean DEFAULT_QUERY_SERVER_WITH_REMOTEUSEREXTRACTOR = false;
     public static final String DEFAULT_QUERY_SERVER_REMOTEUSEREXTRACTOR_PARAM = "doAs";
     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_RENEW_LEASE_ENABLED = true;
     public static final int DEFAULT_RUN_RENEW_LEASE_FREQUENCY_INTERVAL_MILLISECONDS =

http://git-wip-us.apache.org/repos/asf/phoenix/blob/dbbb1125/phoenix-queryserver/src/it/java/org/apache/phoenix/end2end/QueryServerTestUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-queryserver/src/it/java/org/apache/phoenix/end2end/QueryServerTestUtil.java b/phoenix-queryserver/src/it/java/org/apache/phoenix/end2end/QueryServerTestUtil.java
new file mode 100644
index 0000000..01f73ae
--- /dev/null
+++ b/phoenix-queryserver/src/it/java/org/apache/phoenix/end2end/QueryServerTestUtil.java
@@ -0,0 +1,187 @@
+/*
+ * 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.PrivilegedAction;
+import java.util.Map;
+import java.util.Objects;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.LocalHBaseCluster;
+import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.phoenix.query.ConfigurationFactory;
+import org.apache.phoenix.queryserver.client.ThinClientUtil;
+import org.apache.phoenix.queryserver.server.QueryServer;
+import org.apache.phoenix.util.InstanceResolver;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.base.Strings;
+
+public class QueryServerTestUtil {
+    private static final Logger LOG = LoggerFactory.getLogger(QueryServerTestUtil.class);
+
+    private final Configuration conf;
+    private final HBaseTestingUtility util;
+    private LocalHBaseCluster hbase;
+
+    private final QueryServer pqs;
+    private int port;
+    private String url;
+
+    private String principal;
+    private File keytab;
+
+    private ExecutorService executor;
+
+    public QueryServerTestUtil(Configuration conf) {
+        this.conf = Objects.requireNonNull(conf);
+        this.util = new HBaseTestingUtility(conf);
+        this.pqs = new QueryServer(new String[0], conf);
+    }
+
+    public QueryServerTestUtil(Configuration conf, String principal, File keytab) {
+        this.conf = Objects.requireNonNull(conf);
+        this.principal = principal;
+        this.keytab = keytab;
+        this.util = new HBaseTestingUtility(conf);
+        this.pqs = new QueryServer(new String[0], conf);
+    }
+
+    public void startLocalHBaseCluster(Class testClass) throws Exception {
+        startLocalHBaseCluster(testClass.getCanonicalName());
+    }
+
+    public void startLocalHBaseCluster(String uniqueName) throws Exception {
+        LOG.debug("Starting local HBase cluster for '{}'", uniqueName);
+        // Start ZK
+        util.startMiniZKCluster();
+        // Start HDFS
+        util.startMiniDFSCluster(1);
+        // Start HBase
+        Path rootdir = util.getDataTestDirOnTestFS(uniqueName);
+        FSUtils.setRootDir(conf, rootdir);
+        hbase = new LocalHBaseCluster(conf, 1);
+        hbase.startup();
+    }
+
+    public void stopLocalHBaseCluster() throws Exception {
+        LOG.debug("Stopping local HBase cluster");
+        if (hbase != null) {
+            hbase.shutdown();
+            hbase.join();
+        }
+        if (util != null) {
+            util.shutdownMiniDFSCluster();
+            util.shutdownMiniZKCluster();
+        }
+    }
+
+    public void startQueryServer() throws Exception {
+        setupQueryServerConfiguration(conf);
+        executor = Executors.newSingleThreadExecutor();
+        if (!Strings.isNullOrEmpty(principal) && null != keytab) {
+            // Get the PQS ident for PQS to use
+            final UserGroupInformation ugi = UserGroupInformation
+                    .loginUserFromKeytabAndReturnUGI(principal, keytab.getAbsolutePath());
+            // 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).
+            executor.submit(new Runnable() {
+                @Override
+                public void run() {
+                    ugi.doAs(new PrivilegedAction<Void>() {
+                        @Override
+                        public Void run() {
+                            pqs.run();
+                            return null;
+                        }
+                    });
+                }
+            });
+        } else {
+            // Launch PQS without a login
+            executor.submit(new Runnable() {
+                @Override
+                public void run() {
+                    pqs.run();
+                }
+            });
+        }
+        pqs.awaitRunning();
+        port = pqs.getPort();
+        url = ThinClientUtil.getConnectionUrl("localhost", port);
+    }
+
+    public void stopQueryServer() throws Exception {
+        if (pqs != null) {
+            pqs.stop();
+        }
+        if (executor != null) {
+            executor.shutdown();
+            if (!executor.awaitTermination(5, TimeUnit.SECONDS)) {
+                LOG.info("PQS didn't exit in 5 seconds, proceeding anyways.");
+            }
+        }
+    }
+
+    public static void setupQueryServerConfiguration(final Configuration conf) {
+        // Make sure the ConnectionInfo doesn't try to pull a default Configuration
+        InstanceResolver.getSingleton(ConfigurationFactory.class, new ConfigurationFactory() {
+            @Override
+            public Configuration getConfiguration() {
+                return conf;
+            }
+            @Override
+            public Configuration getConfiguration(Configuration confToClone) {
+                Configuration copy = new Configuration(conf);
+                copy.addResource(confToClone);
+                return copy;
+            }
+        });
+    }
+
+    public int getPort() {
+        return port;
+    }
+
+    public String getUrl() {
+        return url;
+    }
+
+    /**
+     * Returns the query server URL with the specified URL params
+     * @param params URL params
+     * @return URL with params
+     */
+    public String getUrl(Map<String, String> params) {
+        if (params == null || params.size() == 0) {
+            return url;
+        }
+        StringBuilder urlParams = new StringBuilder();
+        for (Map.Entry<String, String> param : params.entrySet()) {
+            urlParams.append(";").append(param.getKey()).append("=").append(param.getValue());
+        }
+        return url + urlParams;
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/dbbb1125/phoenix-queryserver/src/it/java/org/apache/phoenix/end2end/ServerCustomizersIT.java
----------------------------------------------------------------------
diff --git a/phoenix-queryserver/src/it/java/org/apache/phoenix/end2end/ServerCustomizersIT.java b/phoenix-queryserver/src/it/java/org/apache/phoenix/end2end/ServerCustomizersIT.java
new file mode 100644
index 0000000..d990adb
--- /dev/null
+++ b/phoenix-queryserver/src/it/java/org/apache/phoenix/end2end/ServerCustomizersIT.java
@@ -0,0 +1,147 @@
+/*
+ * 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.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.Statement;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.calcite.avatica.server.ServerCustomizer;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.queryserver.server.ServerCustomizersFactory;
+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.authentication.BasicAuthenticator;
+import org.eclipse.jetty.server.Server;
+import org.eclipse.jetty.util.security.Constraint;
+import org.eclipse.jetty.util.security.Credential;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ServerCustomizersIT extends BaseHBaseManagedTimeIT {
+    private static final Logger LOG = LoggerFactory.getLogger(ServerCustomizersIT.class);
+    private static final String USER_AUTHORIZED = "user3";
+    private static final String USER_NOT_AUTHORIZED = "user1";
+    private static final String USER_PW = "s3cr3t";
+
+    private static QueryServerTestUtil PQS_UTIL;
+
+    @Rule
+    public ExpectedException expected = ExpectedException.none();
+
+    @BeforeClass
+    public static void setup() throws Exception {
+        Configuration conf = getTestClusterConfig();
+        conf.set(QueryServices.QUERY_SERVER_CUSTOMIZERS_ENABLED, "true");
+        PQS_UTIL = new QueryServerTestUtil(conf);
+        PQS_UTIL.startLocalHBaseCluster(ServerCustomizersIT.class);
+        // Register a test jetty server customizer
+        InstanceResolver.clearSingletons();
+        InstanceResolver.getSingleton(ServerCustomizersFactory.class, new ServerCustomizersFactory() {
+            @Override
+            public List<ServerCustomizer<Server>> createServerCustomizers(Configuration conf) {
+                return Collections.<ServerCustomizer<Server>>singletonList(new TestServerCustomizer());
+            }
+        });
+        PQS_UTIL.startQueryServer();
+    }
+
+    @AfterClass
+    public static void teardown() throws Exception {
+        // Remove custom singletons for future tests
+        InstanceResolver.clearSingletons();
+        if (PQS_UTIL != null) {
+            PQS_UTIL.stopQueryServer();
+            PQS_UTIL.stopLocalHBaseCluster();
+        }
+    }
+
+    @Test
+    public void testUserAuthorized() throws Exception {
+        try (Connection conn = DriverManager.getConnection(PQS_UTIL.getUrl(
+                getBasicAuthParams(USER_AUTHORIZED)));
+                Statement stmt = conn.createStatement()) {
+            Assert.assertFalse("user3 should have access", stmt.execute(
+                "create table "+ServerCustomizersIT.class.getSimpleName()+" (pk integer not null primary key)"));
+        }
+    }
+
+    @Test
+    public void testUserNotAuthorized() throws Exception {
+        expected.expect(RuntimeException.class);
+        expected.expectMessage("HTTP/401");
+        try (Connection conn = DriverManager.getConnection(PQS_UTIL.getUrl(
+                getBasicAuthParams(USER_NOT_AUTHORIZED)));
+                Statement stmt = conn.createStatement()) {
+            Assert.assertFalse(stmt.execute(
+                    "select access from database"));
+        }
+    }
+
+    private Map<String, String> getBasicAuthParams(String user) {
+        Map<String, String> params = new HashMap<>();
+        params.put("authentication", "BASIC");
+        params.put("avatica_user", user);
+        params.put("avatica_password", USER_PW);
+        return params;
+    }
+
+    /**
+     * Contrived customizer that enables BASIC auth for a single user
+     */
+    public static class TestServerCustomizer implements ServerCustomizer<Server> {
+        @Override
+        public void customize(Server server) {
+            LOG.debug("Customizing server to allow requests for {}", USER_AUTHORIZED);
+            HashLoginService login = new HashLoginService();
+            login.putUser(USER_AUTHORIZED, Credential.getCredential(USER_PW), new String[] {"users"});
+            login.setName("users");
+
+            Constraint constraint = new Constraint();
+            constraint.setName(Constraint.__BASIC_AUTH);
+            constraint.setRoles(new String[]{"users"});
+            constraint.setAuthenticate(true);
+
+            ConstraintMapping cm = new ConstraintMapping();
+            cm.setConstraint(constraint);
+            cm.setPathSpec("/*");
+
+            ConstraintSecurityHandler security = new ConstraintSecurityHandler();
+            security.setAuthenticator(new BasicAuthenticator());
+            security.setRealmName("users");
+            security.addConstraintMapping(cm);
+            security.setLoginService(login);
+
+            // chain the PQS handler to security
+            security.setHandler(server.getHandlers()[0]);
+            server.setHandler(security);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/dbbb1125/phoenix-queryserver/src/main/java/org/apache/phoenix/queryserver/server/QueryServer.java
----------------------------------------------------------------------
diff --git a/phoenix-queryserver/src/main/java/org/apache/phoenix/queryserver/server/QueryServer.java b/phoenix-queryserver/src/main/java/org/apache/phoenix/queryserver/server/QueryServer.java
index e3f0f52..47466c8 100644
--- a/phoenix-queryserver/src/main/java/org/apache/phoenix/queryserver/server/QueryServer.java
+++ b/phoenix-queryserver/src/main/java/org/apache/phoenix/queryserver/server/QueryServer.java
@@ -33,6 +33,7 @@ import org.apache.calcite.avatica.server.RemoteUserExtractor;
 import org.apache.calcite.avatica.server.RemoteUserExtractionException;
 import org.apache.calcite.avatica.server.HttpRequestRemoteUserExtractor;
 import org.apache.calcite.avatica.server.HttpQueryStringParameterRemoteUserExtractor;
+import org.apache.calcite.avatica.server.ServerCustomizer;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -52,6 +53,7 @@ import org.apache.phoenix.query.QueryServicesOptions;
 import org.apache.phoenix.loadbalancer.service.LoadBalanceZookeeperConf;
 import org.apache.phoenix.queryserver.register.Registry;
 import org.apache.phoenix.util.InstanceResolver;
+import org.eclipse.jetty.server.Server;
 
 import java.io.File;
 import java.io.IOException;
@@ -61,6 +63,7 @@ import java.net.InetAddress;
 import java.security.PrivilegedExceptionAction;
 import java.util.Arrays;
 import java.util.HashSet;
+import java.util.List;
 import java.util.Map;
 import java.util.Objects;
 import java.util.Set;
@@ -228,14 +231,15 @@ public final class QueryServer extends Configured implements Tool, Runnable {
       Service service = new LocalService(meta);
 
       // Start building the Avatica HttpServer
-      final HttpServer.Builder builder = new HttpServer.Builder().withPort(port)
-          .withHandler(service, getSerialization(getConf()));
+      final HttpServer.Builder<Server> builder = HttpServer.Builder.<Server> newBuilder()
+          .withPort(port).withHandler(service, getSerialization(getConf()));
 
       // Enable client auth when using Kerberos auth for HBase
       if (isKerberos) {
         configureClientAuthentication(builder, disableSpnego);
       }
       setRemoteUserExtractorIfNecessary(builder, getConf());
+      enableServerCustomizersIfNecessary(builder, getConf());
 
       // Build and start the HttpServer
       server = builder.build();
@@ -405,12 +409,30 @@ public final class QueryServer extends Configured implements Tool, Runnable {
     new RemoteUserExtractorFactory.RemoteUserExtractorFactoryImpl();
 
   @VisibleForTesting
+  public void enableServerCustomizersIfNecessary(HttpServer.Builder<Server> builder, Configuration conf) {
+    if (conf.getBoolean(QueryServices.QUERY_SERVER_CUSTOMIZERS_ENABLED,
+            QueryServicesOptions.DEFAULT_QUERY_SERVER_CUSTOMIZERS_ENABLED)) {
+      builder.withServerCustomizers(createServerCustomizers(conf), Server.class);
+    }
+  }
+
+  private static final ServerCustomizersFactory DEFAULT_SERVER_CUSTOMIZERS =
+    new ServerCustomizersFactory.ServerCustomizersFactoryImpl();
+
+  @VisibleForTesting
   RemoteUserExtractor createRemoteUserExtractor(Configuration conf) {
     RemoteUserExtractorFactory factory =
         InstanceResolver.getSingleton(RemoteUserExtractorFactory.class, DEFAULT_USER_EXTRACTOR);
     return factory.createRemoteUserExtractor(conf);
   }
 
+  @VisibleForTesting
+  List<ServerCustomizer<Server>> createServerCustomizers(Configuration conf) {
+    ServerCustomizersFactory factory =
+      InstanceResolver.getSingleton(ServerCustomizersFactory.class, DEFAULT_SERVER_CUSTOMIZERS);
+    return factory.createServerCustomizers(conf);
+  }
+
   /**
    * Use the correctly way to extract end user.
    */

http://git-wip-us.apache.org/repos/asf/phoenix/blob/dbbb1125/phoenix-queryserver/src/main/java/org/apache/phoenix/queryserver/server/ServerCustomizersFactory.java
----------------------------------------------------------------------
diff --git a/phoenix-queryserver/src/main/java/org/apache/phoenix/queryserver/server/ServerCustomizersFactory.java b/phoenix-queryserver/src/main/java/org/apache/phoenix/queryserver/server/ServerCustomizersFactory.java
new file mode 100644
index 0000000..462cd5d
--- /dev/null
+++ b/phoenix-queryserver/src/main/java/org/apache/phoenix/queryserver/server/ServerCustomizersFactory.java
@@ -0,0 +1,49 @@
+/*
+ * 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.queryserver.server;
+
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.calcite.avatica.server.ServerCustomizer;
+import org.apache.hadoop.conf.Configuration;
+import org.eclipse.jetty.server.Server;
+
+/**
+ * Creates customizers for the underlying Avatica HTTP server.
+ * Allows for fine grained control of authentication, etc.
+ */
+public interface ServerCustomizersFactory {
+    /**
+     * Creates a list of customizers that will customize the server.
+     * @param conf Configuration to use
+     * @return List of server suctomizers
+     */
+    List<ServerCustomizer<Server>> createServerCustomizers(Configuration conf);
+
+    /**
+     * Factory that creates an empty list of customizers.
+     */
+    class ServerCustomizersFactoryImpl implements ServerCustomizersFactory {
+        private static final List<ServerCustomizer<Server>> EMPTY_LIST = Collections.emptyList();
+        @Override
+        public List<ServerCustomizer<Server>> createServerCustomizers(Configuration conf) {
+            return EMPTY_LIST;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/dbbb1125/phoenix-queryserver/src/test/java/org/apache/phoenix/queryserver/server/ServerCustomizersTest.java
----------------------------------------------------------------------
diff --git a/phoenix-queryserver/src/test/java/org/apache/phoenix/queryserver/server/ServerCustomizersTest.java b/phoenix-queryserver/src/test/java/org/apache/phoenix/queryserver/server/ServerCustomizersTest.java
new file mode 100644
index 0000000..45fec37
--- /dev/null
+++ b/phoenix-queryserver/src/test/java/org/apache/phoenix/queryserver/server/ServerCustomizersTest.java
@@ -0,0 +1,87 @@
+/*
+ * 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.queryserver.server;
+
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.calcite.avatica.server.HttpServer;
+import org.apache.calcite.avatica.server.ServerCustomizer;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.util.InstanceResolver;
+import org.eclipse.jetty.server.Server;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import static org.mockito.Matchers.*;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
+
+public class ServerCustomizersTest {
+    @Before @After
+    public void clearSingletons() {
+        // clean up singletons
+        InstanceResolver.clearSingletons();
+    }
+
+    @Test
+    public void testDefaultFactory() {
+        QueryServer queryServer = new QueryServer();
+        // the default factory creates an empty list of server customizers
+        List<ServerCustomizer<Server>> customizers =
+            queryServer.createServerCustomizers(new Configuration());
+        Assert.assertEquals(0, customizers.size());
+    }
+
+    @Test
+    public void testUseProvidedCustomizers() {
+        final List<ServerCustomizer<Server>> expected =
+            Collections.<ServerCustomizer<Server>> singletonList(new ServerCustomizer<Server>() {
+              @Override
+              public void customize(Server server) {
+                // no-op customizer
+              }
+        });
+        // Register the server customizer list
+        InstanceResolver.getSingleton(ServerCustomizersFactory.class, new ServerCustomizersFactory() {
+            @Override
+            public List<ServerCustomizer<Server>> createServerCustomizers(Configuration conf) {
+                return expected;
+            }
+        });
+        Configuration conf = new Configuration(false);
+        conf.set(QueryServices.QUERY_SERVER_CUSTOMIZERS_ENABLED, "true");
+        QueryServer queryServer = new QueryServer();
+        List<ServerCustomizer<Server>> actual = queryServer.createServerCustomizers(conf);
+        Assert.assertEquals("Customizers are different", expected, actual);
+    }
+
+    @Test
+    @SuppressWarnings("unchecked")
+    public void testEnableCustomizers() {
+        HttpServer.Builder builder = mock(HttpServer.Builder.class);
+        Configuration conf = new Configuration(false);
+        conf.set(QueryServices.QUERY_SERVER_CUSTOMIZERS_ENABLED, "true");
+        QueryServer queryServer = new QueryServer();
+        queryServer.enableServerCustomizersIfNecessary(builder, conf);
+        verify(builder).withServerCustomizers(anyList(), any(Class.class));
+    }
+}
\ No newline at end of file