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 2018/05/30 23:07:21 UTC

calcite-avatica git commit: CALCITE-2294 Allow customization for AvaticaServerConfiguration for plugging new authentication mechanisms

Repository: calcite-avatica
Updated Branches:
  refs/heads/master 0638c6614 -> 3ab9ec6f8


CALCITE-2294 Allow customization for AvaticaServerConfiguration for plugging new authentication mechanisms

Closes #48

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


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

Branch: refs/heads/master
Commit: 3ab9ec6f884607417d8e1badd69a681f958c2703
Parents: 0638c66
Author: Karan Mehta <k....@salesforce.com>
Authored: Wed May 30 19:04:05 2018 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Wed May 30 19:04:25 2018 -0400

----------------------------------------------------------------------
 .../avatica/remote/AuthenticationType.java      |   3 +-
 .../calcite/avatica/server/HttpServer.java      | 180 ++++++----
 .../server/CustomAuthHttpServerTest.java        | 338 +++++++++++++++++++
 .../calcite/avatica/server/HttpAuthBase.java    |  15 +
 ...yStringParameterRemoteUserExtractorTest.java |  14 +-
 site/_docs/security.md                          |  18 +
 6 files changed, 494 insertions(+), 74 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite-avatica/blob/3ab9ec6f/core/src/main/java/org/apache/calcite/avatica/remote/AuthenticationType.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/avatica/remote/AuthenticationType.java b/core/src/main/java/org/apache/calcite/avatica/remote/AuthenticationType.java
index 2662e14..f483be9 100644
--- a/core/src/main/java/org/apache/calcite/avatica/remote/AuthenticationType.java
+++ b/core/src/main/java/org/apache/calcite/avatica/remote/AuthenticationType.java
@@ -23,7 +23,8 @@ public enum AuthenticationType {
   NONE,
   BASIC,
   DIGEST,
-  SPNEGO;
+  SPNEGO,
+  CUSTOM;
 }
 
 // End AuthenticationType.java

http://git-wip-us.apache.org/repos/asf/calcite-avatica/blob/3ab9ec6f/server/src/main/java/org/apache/calcite/avatica/server/HttpServer.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/calcite/avatica/server/HttpServer.java b/server/src/main/java/org/apache/calcite/avatica/server/HttpServer.java
index 08f4274..f7b6e75 100644
--- a/server/src/main/java/org/apache/calcite/avatica/server/HttpServer.java
+++ b/server/src/main/java/org/apache/calcite/avatica/server/HttpServer.java
@@ -215,35 +215,64 @@ public class HttpServer {
     server = new Server(threadPool);
     server.manage(threadPool);
 
-    final ServerConnector connector = configureConnector(getConnector(), port);
-    ConstraintSecurityHandler securityHandler = null;
+    ServerConnector serverConnector = null;
+    HandlerList handlerList = null;
+    if (null != this.config && AuthenticationType.CUSTOM == config.getAuthenticationType()) {
+      if (null != handler || null != sslFactory) {
+        throw new IllegalStateException("Handlers and SSLFactory cannot be configured with "
+                + "the HTTPServer Builder when using CUSTOM Authentication Type.");
+      }
+    } else {
+      serverConnector = configureServerConnector();
+      handlerList = configureHandlers();
+    }
 
-    if (null != this.config) {
-      switch (config.getAuthenticationType()) {
-      case SPNEGO:
-        // Get the Handler for SPNEGO authentication
-        securityHandler = configureSpnego(server, connector, this.config);
-        break;
-      case BASIC:
-        securityHandler = configureBasicAuthentication(server, connector, config);
-        break;
-      case DIGEST:
-        securityHandler = configureDigestAuthentication(server, connector, config);
-        break;
-      default:
-        // Pass
-        break;
+    // Apply server customizers
+    for (ServerCustomizer<Server> customizer : this.serverCustomizers) {
+      LOG.info("Customizing server with customizer: " + customizer.getClass());
+      customizer.customize(server);
+    }
+
+    try {
+      server.start();
+    } catch (Exception e) {
+      throw new RuntimeException(e);
+    }
+
+    if (null != serverConnector && null != handlerList) {
+      port = serverConnector.getLocalPort();
+      LOG.info("Service listening on port {}.", getPort());
+
+      // Set the information about the address for this server
+      try {
+        this.handler.setServerRpcMetadata(createRpcServerMetadata(serverConnector));
+      } catch (UnknownHostException e) {
+        // Failed to do the DNS lookup, bail out.
+        throw new RuntimeException(e);
       }
+    } else if (0 == server.getConnectors().length) {
+      String error = "No server connectors have been configured for this Avatica server";
+      LOG.error(error);
+      throw new RuntimeException(error);
     }
+  }
 
+  private ServerConnector configureServerConnector() {
+    final ServerConnector connector = getServerConnector();
+    connector.setIdleTimeout(60 * 1000);
+    connector.setSoLingerTime(-1);
+    connector.setPort(port);
     server.setConnectors(new Connector[] { connector });
+    return connector;
+  }
 
-    // Default to using the handler that was passed in
+  private HandlerList configureHandlers() {
     final HandlerList handlerList = new HandlerList();
     Handler avaticaHandler = handler;
 
     // Wrap the provided handler for security if we made one
-    if (null != securityHandler) {
+    if (null != config) {
+      ConstraintSecurityHandler securityHandler = getSecurityHandler();
       securityHandler.setHandler(handler);
       avaticaHandler = securityHandler;
     }
@@ -251,30 +280,30 @@ public class HttpServer {
     handlerList.setHandlers(new Handler[] {avaticaHandler, new DefaultHandler()});
 
     server.setHandler(handlerList);
-    // Apply server customizers
-    for (ServerCustomizer<Server> customizer : this.serverCustomizers) {
-      customizer.customize(server);
-    }
-
-    try {
-      server.start();
-    } catch (Exception e) {
-      throw new RuntimeException(e);
-    }
-    port = connector.getLocalPort();
-
-    LOG.info("Service listening on port {}.", getPort());
+    return handlerList;
+  }
 
-    // Set the information about the address for this server
-    try {
-      this.handler.setServerRpcMetadata(createRpcServerMetadata(connector));
-    } catch (UnknownHostException e) {
-      // Failed to do the DNS lookup, bail out.
-      throw new RuntimeException(e);
-    }
+  private ConstraintSecurityHandler getSecurityHandler() {
+    ConstraintSecurityHandler securityHandler = null;
+    switch (config.getAuthenticationType()) {
+    case SPNEGO:
+      // Get the Handler for SPNEGO authentication
+      securityHandler = configureSpnego(server, this.config);
+      break;
+    case BASIC:
+      securityHandler = configureBasicAuthentication(server, config);
+      break;
+    case DIGEST:
+      securityHandler = configureDigestAuthentication(server, config);
+      break;
+    default:
+      // Pass
+      break;
+    }
+    return securityHandler;
   }
 
-  private ServerConnector getConnector() {
+  protected ServerConnector getServerConnector() {
     HttpConnectionFactory factory = new HttpConnectionFactory();
     factory.getHttpConfiguration().setRequestHeaderSize(maxAllowedHeaderSize);
 
@@ -302,10 +331,9 @@ public class HttpServer {
   /**
    * Configures the <code>connector</code> given the <code>config</code> for using SPNEGO.
    *
-   * @param connector The connector to configure
    * @param config The configuration
    */
-  protected ConstraintSecurityHandler configureSpnego(Server server, ServerConnector connector,
+  protected ConstraintSecurityHandler configureSpnego(Server server,
       AvaticaServerConfiguration config) {
     final String realm = Objects.requireNonNull(config.getKerberosRealm());
     final String principal = Objects.requireNonNull(config.getKerberosPrincipal());
@@ -318,7 +346,7 @@ public class HttpServer {
     // Roles are "realms" for Kerberos/SPNEGO
     final String[] allowedRealms = getAllowedRealms(realm, config);
 
-    return configureCommonAuthentication(server, connector, config, Constraint.__SPNEGO_AUTH,
+    return configureCommonAuthentication(Constraint.__SPNEGO_AUTH,
         allowedRealms, new AvaticaSpnegoAuthenticator(), realm, spnegoLoginService);
   }
 
@@ -336,7 +364,7 @@ public class HttpServer {
   }
 
   protected ConstraintSecurityHandler configureBasicAuthentication(Server server,
-      ServerConnector connector, AvaticaServerConfiguration config) {
+      AvaticaServerConfiguration config) {
     final String[] allowedRoles = config.getAllowedRoles();
     final String realm = config.getHashLoginServiceRealm();
     final String loginServiceProperties = config.getHashLoginServiceProperties();
@@ -344,12 +372,12 @@ public class HttpServer {
     HashLoginService loginService = new HashLoginService(realm, loginServiceProperties);
     server.addBean(loginService);
 
-    return configureCommonAuthentication(server, connector, config, Constraint.__BASIC_AUTH,
+    return configureCommonAuthentication(Constraint.__BASIC_AUTH,
         allowedRoles, new BasicAuthenticator(), null, loginService);
   }
 
   protected ConstraintSecurityHandler configureDigestAuthentication(Server server,
-      ServerConnector connector, AvaticaServerConfiguration config) {
+      AvaticaServerConfiguration config) {
     final String[] allowedRoles = config.getAllowedRoles();
     final String realm = config.getHashLoginServiceRealm();
     final String loginServiceProperties = config.getHashLoginServiceProperties();
@@ -357,12 +385,11 @@ public class HttpServer {
     HashLoginService loginService = new HashLoginService(realm, loginServiceProperties);
     server.addBean(loginService);
 
-    return configureCommonAuthentication(server, connector, config, Constraint.__DIGEST_AUTH,
+    return configureCommonAuthentication(Constraint.__DIGEST_AUTH,
         allowedRoles, new DigestAuthenticator(), null, loginService);
   }
 
-  protected ConstraintSecurityHandler configureCommonAuthentication(Server server,
-      ServerConnector connector, AvaticaServerConfiguration config, String constraintName,
+  protected ConstraintSecurityHandler configureCommonAuthentication(String constraintName,
       String[] allowedRoles, Authenticator authenticator, String realm,
       LoginService loginService) {
 
@@ -467,6 +494,8 @@ public class HttpServer {
 
     // The maximum size in bytes of an http header the server will read (64KB)
     private int maxAllowedHeaderSize = MAX_ALLOWED_HEADER_SIZE;
+    private AvaticaServerConfiguration serverConfig;
+    private Subject subject;
 
     public Builder() {}
 
@@ -659,6 +688,22 @@ public class HttpServer {
       return withAuthentication(AuthenticationType.DIGEST, properties, allowedRoles);
     }
 
+    /**
+     * Configures the server to use CUSTOM authentication mechanism, which can allow users to
+     * combine benefits of multiple auth methods. See <code>CustomAuthHttpServerTest</code> for
+     * examples on how to use it.
+     * Note: Default ServerConnectors and Handlers will NOT be used.
+     * Customize them directly using instances <code>{@link ServerCustomizer}</code>
+     * @param config AvaticaServerConfiguration implementation that configures various details
+     *      about the authentication mechanism for <code>{@link HttpServer}</code>
+     * @return <code>this</code>
+     */
+    public Builder<T> withCustomAuthentication(AvaticaServerConfiguration config) {
+      this.authenticationType = AuthenticationType.CUSTOM;
+      this.serverConfig = config;
+      return this;
+    }
+
     private Builder<T> withAuthentication(AuthenticationType authType, String properties,
         String[] allowedRoles) {
       this.loginServiceRealm = "Avatica";
@@ -721,18 +766,18 @@ public class HttpServer {
      */
     @SuppressWarnings("unchecked")
     public HttpServer build() {
-      final AvaticaServerConfiguration serverConfig;
-      final Subject subject;
       switch (authenticationType) {
       case NONE:
         serverConfig = null;
         subject = null;
+        handler = buildHandler(this, serverConfig);
         break;
       case BASIC:
       case DIGEST:
         // Build the configuration for BASIC or DIGEST authentication.
         serverConfig = buildUserAuthenticationConfiguration(this);
         subject = null;
+        handler = buildHandler(this, serverConfig);
         break;
       case SPNEGO:
         if (usingTLS) {
@@ -746,20 +791,19 @@ public class HttpServer {
           subject = null;
         }
         serverConfig = buildSpnegoConfiguration(this);
+        handler = buildHandler(this, serverConfig);
+        break;
+      case CUSTOM:
+        // We don't need to build any Config here since
+        // serverConfig is already assigned the required AvaticaServerConfiguration
+        serverConfig = buildCustomConfiguration(this);
+        subject = null;
         break;
       default:
         throw new IllegalArgumentException("Unhandled AuthenticationType");
       }
 
-      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);
-      }
+      SslContextFactory sslFactory = buildSSLContextFactory();
 
       List<ServerCustomizer<Server>> jettyCustomizers = new ArrayList<>();
       for (ServerCustomizer<?> customizer : this.serverCustomizers) {
@@ -771,6 +815,22 @@ public class HttpServer {
           maxAllowedHeaderSize);
     }
 
+    protected SslContextFactory buildSSLContextFactory() {
+      SslContextFactory sslFactory = null;
+      if (usingTLS) {
+        sslFactory = new SslContextFactory();
+        sslFactory.setKeyStorePath(this.keystore.getAbsolutePath());
+        sslFactory.setKeyStorePassword(keystorePassword);
+        sslFactory.setTrustStorePath(truststore.getAbsolutePath());
+        sslFactory.setTrustStorePassword(truststorePassword);
+      }
+      return sslFactory;
+    }
+
+    private AvaticaServerConfiguration buildCustomConfiguration(Builder<T> tBuilder) {
+      return tBuilder.serverConfig;
+    }
+
     /**
      * Creates the appropriate {@link AvaticaHandler}.
      *

http://git-wip-us.apache.org/repos/asf/calcite-avatica/blob/3ab9ec6f/server/src/test/java/org/apache/calcite/avatica/server/CustomAuthHttpServerTest.java
----------------------------------------------------------------------
diff --git a/server/src/test/java/org/apache/calcite/avatica/server/CustomAuthHttpServerTest.java b/server/src/test/java/org/apache/calcite/avatica/server/CustomAuthHttpServerTest.java
new file mode 100644
index 0000000..dbb2f4c
--- /dev/null
+++ b/server/src/test/java/org/apache/calcite/avatica/server/CustomAuthHttpServerTest.java
@@ -0,0 +1,338 @@
+/*
+ * 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.server;
+
+import org.apache.calcite.avatica.ConnectionSpec;
+import org.apache.calcite.avatica.jdbc.JdbcMeta;
+import org.apache.calcite.avatica.remote.AuthenticationType;
+import org.apache.calcite.avatica.remote.Driver;
+import org.apache.calcite.avatica.remote.LocalService;
+
+import org.eclipse.jetty.security.ConstraintSecurityHandler;
+import org.eclipse.jetty.security.UserAuthentication;
+import org.eclipse.jetty.server.Authentication;
+import org.eclipse.jetty.server.Connector;
+import org.eclipse.jetty.server.Handler;
+import org.eclipse.jetty.server.Request;
+import org.eclipse.jetty.server.Server;
+import org.eclipse.jetty.server.ServerConnector;
+import org.eclipse.jetty.server.UserIdentity;
+import org.eclipse.jetty.server.handler.DefaultHandler;
+import org.eclipse.jetty.server.handler.HandlerList;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import java.sql.SQLException;
+import javax.servlet.http.HttpServletRequest;
+
+import static org.hamcrest.core.StringContains.containsString;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.fail;
+
+import java.util.Arrays;
+import java.util.Properties;
+import java.util.concurrent.Callable;
+
+/**
+ * Test class for providing CustomAvaticaServerConfiguration to the HTTP Server
+ */
+public class CustomAuthHttpServerTest extends HttpAuthBase {
+  private static final ConnectionSpec CONNECTION_SPEC = ConnectionSpec.HSQLDB;
+  private static HttpServer server;
+  private static String url;
+
+  // Counters to keep track of number of function calls
+  private static int methodCallCounter1 = 0;
+  private static int methodCallCounter2 = 0;
+  private static int methodCallCounter3 = 0;
+
+  @Before
+  public void before() {
+    methodCallCounter1 = 0;
+    methodCallCounter2 = 0;
+    methodCallCounter3 = 0;
+  }
+
+  @After
+  public void stopServer() {
+    if (null != server) {
+      server.stop();
+    }
+  }
+
+  @Test
+  public void testCustomImpersonationConfig() throws Exception {
+    AvaticaServerConfiguration configuration = new CustomImpersonationConfig();
+    createServer(configuration, false);
+
+    readWriteData(url, "CUSTOM_CONFIG_1_TABLE", new Properties());
+    Assert.assertEquals("supportsImpersonation should be called same number of "
+            + "times as doAsRemoteUser method", methodCallCounter1, methodCallCounter2);
+    Assert.assertEquals("supportsImpersonation should be called same number of "
+            + "times as getRemoteUserExtractor method", methodCallCounter1, methodCallCounter3);
+  }
+
+  @Test
+  public void testCustomBasicImpersonationConfigWithAllowedUser() throws Exception {
+    AvaticaServerConfiguration configuration = new CustomBasicImpersonationConfig();
+    createServer(configuration, true);
+
+    final Properties props = new Properties();
+    props.put("avatica_user", "USER2");
+    props.put("avatica_password", "password2");
+    props.put("user", "USER2");
+    props.put("password", "password2");
+
+    readWriteData(url, "CUSTOM_CONFIG_2_ALLOWED_TABLE", props);
+    Assert.assertEquals("supportsImpersonation should be called same number of "
+            + "times as doAsRemoteUser method", methodCallCounter1, methodCallCounter2);
+    Assert.assertEquals("supportsImpersonation should be called same number of "
+            + "times as getRemoteUserExtractor method", methodCallCounter1, methodCallCounter3);
+  }
+
+  @Test
+  public void testCustomBasicImpersonationConfigWithDisallowedUser() throws Exception {
+    AvaticaServerConfiguration configuration = new CustomBasicImpersonationConfig();
+    createServer(configuration, true);
+
+    final Properties props = new Properties();
+    props.put("avatica_user", "USER1");
+    props.put("avatica_password", "password1");
+    props.put("user", "USER1");
+    props.put("password", "password1");
+
+    try {
+      readWriteData(url, "CUSTOM_CONFIG_2_DISALLOWED_TABLE", props);
+      fail("Expected an exception");
+    } catch (RuntimeException e) {
+      assertThat(e.getMessage(), containsString("Failed to execute HTTP Request, got HTTP/403"));
+    }
+  }
+
+  @Test(expected = IllegalStateException.class)
+  public void testCustomConfigDisallowsWithHandlerMethod() {
+    AvaticaServerConfiguration configuration = new CustomBasicImpersonationConfig();
+    server = new HttpServer.Builder()
+            .withCustomAuthentication(configuration)
+            .withHandler(Mockito.mock(AvaticaHandler.class))
+            .withPort(0)
+            .build();
+    server.start();
+  }
+
+  public static HttpServer getAvaticaServer() {
+    return server;
+  }
+
+  @SuppressWarnings("unchecked") // needed for the mocked customizers, not the builder
+  protected void createServer(AvaticaServerConfiguration config, boolean isBasicAuth)
+      throws SQLException {
+    final JdbcMeta jdbcMeta = new JdbcMeta(CONNECTION_SPEC.url,
+      CONNECTION_SPEC.username, CONNECTION_SPEC.password);
+    LocalService service = new LocalService(jdbcMeta);
+
+    ConnectorCustomizer connectorCustomizer = new ConnectorCustomizer();
+    BasicAuthHandlerCustomizer basicAuthCustomizer =
+            new BasicAuthHandlerCustomizer(config, service, isBasicAuth);
+
+    server = new HttpServer.Builder()
+            .withCustomAuthentication(config)
+            .withPort(0)
+            .withServerCustomizers(
+                    Arrays.asList(connectorCustomizer, basicAuthCustomizer), Server.class)
+            .build();
+    server.start();
+
+    // Create and grant permissions to our users
+    createHsqldbUsers();
+    url = "jdbc:avatica:remote:url=http://localhost:" + connectorCustomizer.getLocalPort()
+            + ";authentication=BASIC;serialization=PROTOBUF";
+  }
+
+  /**
+   * Customizer to add ServerConnectors to the server
+   */
+  static class ConnectorCustomizer implements ServerCustomizer<Server> {
+
+    ServerConnector connector;
+
+    @Override public void customize(Server server) {
+      HttpServer avaticaServer = getAvaticaServer();
+      connector = avaticaServer.configureConnector(avaticaServer.getServerConnector(), 0);
+      server.setConnectors(new Connector[] { connector });
+    }
+
+    public int getLocalPort() {
+      return connector.getLocalPort();
+    }
+
+  }
+
+  /**
+   * Customizer to add handlers to the server (with or without BasicAuth)
+   */
+  static class BasicAuthHandlerCustomizer implements ServerCustomizer<Server> {
+
+    AvaticaServerConfiguration configuration;
+    LocalService service;
+    boolean isBasicAuth;
+
+    public BasicAuthHandlerCustomizer(AvaticaServerConfiguration configuration
+            , LocalService service, boolean isBasicAuth) {
+      this.configuration = configuration;
+      this.service = service;
+      this.isBasicAuth = isBasicAuth;
+    }
+
+    @Override public void customize(Server server) {
+      HttpServer avaticaServer = getAvaticaServer();
+
+      HandlerFactory factory = new HandlerFactory();
+      Handler avaticaHandler = factory.getHandler(service,
+              Driver.Serialization.PROTOBUF, null, configuration);
+
+      if (isBasicAuth) {
+        ConstraintSecurityHandler securityHandler =
+                avaticaServer.configureBasicAuthentication(server, configuration);
+        securityHandler.setHandler(avaticaHandler);
+        avaticaHandler = securityHandler;
+      }
+
+      HandlerList handlerList = new HandlerList();
+      handlerList.setHandlers(new Handler[] { avaticaHandler, new DefaultHandler()});
+      server.setHandler(handlerList);
+    }
+  }
+
+  /**
+   * CustomImpersonationConfig doesn't authenticates the user but supports user impersonation
+   */
+  static class CustomImpersonationConfig implements AvaticaServerConfiguration {
+
+
+    @Override public AuthenticationType getAuthenticationType() {
+      return AuthenticationType.CUSTOM;
+    }
+
+    @Override public String getKerberosRealm() {
+      return null;
+    }
+
+    @Override public String getKerberosPrincipal() {
+      return null;
+    }
+
+    @Override public String[] getAllowedRoles() {
+      return new String[0];
+    }
+
+    @Override public String getHashLoginServiceRealm() {
+      return null;
+    }
+
+    @Override public String getHashLoginServiceProperties() {
+      return null;
+    }
+
+    @Override public boolean supportsImpersonation() {
+      methodCallCounter1++;
+      return true;
+    }
+
+    @Override public <T> T doAsRemoteUser(String remoteUserName,
+              String remoteAddress, Callable<T> action) throws Exception {
+      methodCallCounter2++;
+      return action.call();
+    }
+    @Override public RemoteUserExtractor getRemoteUserExtractor() {
+      return new RemoteUserExtractor() {
+        @Override public String extract(HttpServletRequest request) {
+          methodCallCounter3++;
+          return "randomUser";
+        }
+      };
+    }
+
+  }
+
+  /**
+   * CustomBasicImpersonationConfig supports BasicAuthentication with user impersonation
+   */
+  static class CustomBasicImpersonationConfig implements AvaticaServerConfiguration {
+
+
+    @Override public AuthenticationType getAuthenticationType() {
+      return AuthenticationType.CUSTOM;
+    }
+
+    @Override public String getKerberosRealm() {
+      return null;
+    }
+
+    @Override public String getKerberosPrincipal() {
+      return null;
+    }
+
+    @Override public String[] getAllowedRoles() {
+      return new String[] { "users" };
+    }
+
+    @Override public String getHashLoginServiceRealm() {
+      return "Avatica";
+    }
+
+    @Override public String getHashLoginServiceProperties() {
+      return HttpAuthBase.getHashLoginServicePropertiesString();
+    }
+
+    @Override public boolean supportsImpersonation() {
+      methodCallCounter1++;
+      return true;
+    }
+
+    @Override public <T> T doAsRemoteUser(String remoteUserName,
+      String remoteAddress, Callable<T> action) throws Exception {
+      methodCallCounter2++;
+      if (remoteUserName.equals("USER1")) {
+        throw new RemoteUserDisallowedException("USER1 is a disallowed user!");
+      }
+      return action.call();
+    }
+    @Override public RemoteUserExtractor getRemoteUserExtractor() {
+      return new RemoteUserExtractor() {
+        @Override public String extract(HttpServletRequest request)
+            throws RemoteUserExtractionException {
+          methodCallCounter3++;
+          if (request instanceof Request) {
+            Authentication authentication = ((Request) request).getAuthentication();
+            if (authentication instanceof UserAuthentication) {
+              UserIdentity userIdentity = ((UserAuthentication) authentication).getUserIdentity();
+              return userIdentity.getUserPrincipal().getName();
+            }
+          }
+          throw new RemoteUserExtractionException("Request doesn't contain user credentials.");
+        }
+      };
+    }
+  }
+
+}
+
+// End CustomAuthHttpServerTest.java

http://git-wip-us.apache.org/repos/asf/calcite-avatica/blob/3ab9ec6f/server/src/test/java/org/apache/calcite/avatica/server/HttpAuthBase.java
----------------------------------------------------------------------
diff --git a/server/src/test/java/org/apache/calcite/avatica/server/HttpAuthBase.java b/server/src/test/java/org/apache/calcite/avatica/server/HttpAuthBase.java
index cfaf302..6ce0afe 100644
--- a/server/src/test/java/org/apache/calcite/avatica/server/HttpAuthBase.java
+++ b/server/src/test/java/org/apache/calcite/avatica/server/HttpAuthBase.java
@@ -18,6 +18,8 @@ package org.apache.calcite.avatica.server;
 
 import org.apache.calcite.avatica.ConnectionSpec;
 
+import java.io.UnsupportedEncodingException;
+import java.net.URLDecoder;
 import java.sql.Connection;
 import java.sql.DriverManager;
 import java.sql.ResultSet;
@@ -75,6 +77,19 @@ public class HttpAuthBase {
       assertEquals(3, results.getInt(1));
     }
   }
+
+  static String getHashLoginServicePropertiesString() {
+    try {
+      final String userPropertiesFile =
+              URLDecoder.decode(HttpQueryStringParameterRemoteUserExtractorTest.class
+                      .getResource("/auth-users.properties").getFile(), "UTF-8");
+      assertNotNull("Could not find properties file for basic auth users", userPropertiesFile);
+      return userPropertiesFile;
+    } catch (UnsupportedEncodingException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
 }
 
 // End HttpAuthBase.java

http://git-wip-us.apache.org/repos/asf/calcite-avatica/blob/3ab9ec6f/server/src/test/java/org/apache/calcite/avatica/server/HttpQueryStringParameterRemoteUserExtractorTest.java
----------------------------------------------------------------------
diff --git a/server/src/test/java/org/apache/calcite/avatica/server/HttpQueryStringParameterRemoteUserExtractorTest.java b/server/src/test/java/org/apache/calcite/avatica/server/HttpQueryStringParameterRemoteUserExtractorTest.java
index 7fcde9a..33d91bc 100644
--- a/server/src/test/java/org/apache/calcite/avatica/server/HttpQueryStringParameterRemoteUserExtractorTest.java
+++ b/server/src/test/java/org/apache/calcite/avatica/server/HttpQueryStringParameterRemoteUserExtractorTest.java
@@ -29,14 +29,11 @@ import org.junit.runners.Parameterized.Parameters;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.io.UnsupportedEncodingException;
-import java.net.URLDecoder;
 import java.util.List;
 import java.util.Properties;
 import java.util.concurrent.Callable;
 
 import static org.hamcrest.core.StringContains.containsString;
-import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertThat;
 import static org.junit.Assert.fail;
 
@@ -129,16 +126,7 @@ public class HttpQueryStringParameterRemoteUserExtractorTest extends HttpAuthBas
     }
 
     @Override public String getHashLoginServiceProperties() {
-      try {
-        final String userPropertiesFile =
-            URLDecoder.decode(HttpQueryStringParameterRemoteUserExtractorTest.class
-                .getResource("/auth-users.properties").getFile(), "UTF-8");
-        assertNotNull("Could not find properties file for basic auth users", userPropertiesFile);
-        return userPropertiesFile;
-      } catch (UnsupportedEncodingException e) {
-        LOG.error("Failed to decode path to Jetty users file", e);
-        throw new RuntimeException(e);
-      }
+      return HttpAuthBase.getHashLoginServicePropertiesString();
     }
   };
 

http://git-wip-us.apache.org/repos/asf/calcite-avatica/blob/3ab9ec6f/site/_docs/security.md
----------------------------------------------------------------------
diff --git a/site/_docs/security.md b/site/_docs/security.md
index 53233bd..a6909dc 100644
--- a/site/_docs/security.md
+++ b/site/_docs/security.md
@@ -7,6 +7,7 @@ auth_types:
   - { name: "HTTP Basic", anchor: "http-basic-authentication" }
   - { name: "HTTP Digest", anchor: "http-digest-authentication" }
   - { name: "Kerberos with SPNEGO", anchor: "kerberos-with-spnego-authentication" }
+  - { name: "Custom Authentication", anchor: "custom-authentication" }
   - { name: "Client implementation", anchor: "client-implementation" }
 ---
 <!--
@@ -256,6 +257,23 @@ config = new AvaticaServerConfiguration() {
 };
 {% endhighlight %}
 
+## Custom Authentication
+
+Avatica server now offers users to plugin their Custom Authentication mechanism through the HTTPServer Builder.
+This is useful if users want to combine features of various authentication types. Examples include combining 
+basic authentication with impersonation or adding mutual authentication with impersonation. More Examples
+are available in `CustomAuthHttpServerTest` class.
+
+Note: Users need to configure their own `ServerConnectors` and `Handlers` with the help of `ServerCustomizers`.
+{% highlight java %}
+AvaticaServerConfiguration configuration = new ExampleAvaticaServerConfiguration();
+HttpServer server = new HttpServer.Builder()
+    .withCustomAuthentication(configuration)
+    .withPort(8765)
+    .build();
+{% endhighlight %}
+
+
 ## Client implementation
 
 Many HTTP client libraries, such as [Apache Commons HttpComponents](https://hc.apache.org/), already have


Re: calcite-avatica git commit: CALCITE-2294 Allow customization for AvaticaServerConfiguration for plugging new authentication mechanisms

Posted by Julian Hyde <jh...@apache.org>.
Josh, per our guidelines, comment should be


> [CALCITE-2294] Allow customization for AvaticaServerConfiguration for plugging new authentication mechanisms (Karan Mehta)

Can you force-push to fix please.


> On May 30, 2018, at 4:07 PM, elserj@apache.org wrote:
> 
> Repository: calcite-avatica
> Updated Branches:
>  refs/heads/master 0638c6614 -> 3ab9ec6f8
> 
> 
> CALCITE-2294 Allow customization for AvaticaServerConfiguration for plugging new authentication mechanisms
> 
> Closes #48
> 
> Signed-off-by: Josh Elser <el...@apache.org>
> 
> 
> Project: http://git-wip-us.apache.org/repos/asf/calcite-avatica/repo
> Commit: http://git-wip-us.apache.org/repos/asf/calcite-avatica/commit/3ab9ec6f
> Tree: http://git-wip-us.apache.org/repos/asf/calcite-avatica/tree/3ab9ec6f
> Diff: http://git-wip-us.apache.org/repos/asf/calcite-avatica/diff/3ab9ec6f
> 
> Branch: refs/heads/master
> Commit: 3ab9ec6f884607417d8e1badd69a681f958c2703
> Parents: 0638c66
> Author: Karan Mehta <k....@salesforce.com>
> Authored: Wed May 30 19:04:05 2018 -0400
> Committer: Josh Elser <el...@apache.org>
> Committed: Wed May 30 19:04:25 2018 -0400
> 
> ----------------------------------------------------------------------
> .../avatica/remote/AuthenticationType.java      |   3 +-
> .../calcite/avatica/server/HttpServer.java      | 180 ++++++----
> .../server/CustomAuthHttpServerTest.java        | 338 +++++++++++++++++++
> .../calcite/avatica/server/HttpAuthBase.java    |  15 +
> ...yStringParameterRemoteUserExtractorTest.java |  14 +-
> site/_docs/security.md                          |  18 +
> 6 files changed, 494 insertions(+), 74 deletions(-)
> ----------------------------------------------------------------------
> 
> 
> http://git-wip-us.apache.org/repos/asf/calcite-avatica/blob/3ab9ec6f/core/src/main/java/org/apache/calcite/avatica/remote/AuthenticationType.java
> ----------------------------------------------------------------------
> diff --git a/core/src/main/java/org/apache/calcite/avatica/remote/AuthenticationType.java b/core/src/main/java/org/apache/calcite/avatica/remote/AuthenticationType.java
> index 2662e14..f483be9 100644
> --- a/core/src/main/java/org/apache/calcite/avatica/remote/AuthenticationType.java
> +++ b/core/src/main/java/org/apache/calcite/avatica/remote/AuthenticationType.java
> @@ -23,7 +23,8 @@ public enum AuthenticationType {
>   NONE,
>   BASIC,
>   DIGEST,
> -  SPNEGO;
> +  SPNEGO,
> +  CUSTOM;
> }
> 
> // End AuthenticationType.java
> 
> http://git-wip-us.apache.org/repos/asf/calcite-avatica/blob/3ab9ec6f/server/src/main/java/org/apache/calcite/avatica/server/HttpServer.java
> ----------------------------------------------------------------------
> diff --git a/server/src/main/java/org/apache/calcite/avatica/server/HttpServer.java b/server/src/main/java/org/apache/calcite/avatica/server/HttpServer.java
> index 08f4274..f7b6e75 100644
> --- a/server/src/main/java/org/apache/calcite/avatica/server/HttpServer.java
> +++ b/server/src/main/java/org/apache/calcite/avatica/server/HttpServer.java
> @@ -215,35 +215,64 @@ public class HttpServer {
>     server = new Server(threadPool);
>     server.manage(threadPool);
> 
> -    final ServerConnector connector = configureConnector(getConnector(), port);
> -    ConstraintSecurityHandler securityHandler = null;
> +    ServerConnector serverConnector = null;
> +    HandlerList handlerList = null;
> +    if (null != this.config && AuthenticationType.CUSTOM == config.getAuthenticationType()) {
> +      if (null != handler || null != sslFactory) {
> +        throw new IllegalStateException("Handlers and SSLFactory cannot be configured with "
> +                + "the HTTPServer Builder when using CUSTOM Authentication Type.");
> +      }
> +    } else {
> +      serverConnector = configureServerConnector();
> +      handlerList = configureHandlers();
> +    }
> 
> -    if (null != this.config) {
> -      switch (config.getAuthenticationType()) {
> -      case SPNEGO:
> -        // Get the Handler for SPNEGO authentication
> -        securityHandler = configureSpnego(server, connector, this.config);
> -        break;
> -      case BASIC:
> -        securityHandler = configureBasicAuthentication(server, connector, config);
> -        break;
> -      case DIGEST:
> -        securityHandler = configureDigestAuthentication(server, connector, config);
> -        break;
> -      default:
> -        // Pass
> -        break;
> +    // Apply server customizers
> +    for (ServerCustomizer<Server> customizer : this.serverCustomizers) {
> +      LOG.info("Customizing server with customizer: " + customizer.getClass());
> +      customizer.customize(server);
> +    }
> +
> +    try {
> +      server.start();
> +    } catch (Exception e) {
> +      throw new RuntimeException(e);
> +    }
> +
> +    if (null != serverConnector && null != handlerList) {
> +      port = serverConnector.getLocalPort();
> +      LOG.info("Service listening on port {}.", getPort());
> +
> +      // Set the information about the address for this server
> +      try {
> +        this.handler.setServerRpcMetadata(createRpcServerMetadata(serverConnector));
> +      } catch (UnknownHostException e) {
> +        // Failed to do the DNS lookup, bail out.
> +        throw new RuntimeException(e);
>       }
> +    } else if (0 == server.getConnectors().length) {
> +      String error = "No server connectors have been configured for this Avatica server";
> +      LOG.error(error);
> +      throw new RuntimeException(error);
>     }
> +  }
> 
> +  private ServerConnector configureServerConnector() {
> +    final ServerConnector connector = getServerConnector();
> +    connector.setIdleTimeout(60 * 1000);
> +    connector.setSoLingerTime(-1);
> +    connector.setPort(port);
>     server.setConnectors(new Connector[] { connector });
> +    return connector;
> +  }
> 
> -    // Default to using the handler that was passed in
> +  private HandlerList configureHandlers() {
>     final HandlerList handlerList = new HandlerList();
>     Handler avaticaHandler = handler;
> 
>     // Wrap the provided handler for security if we made one
> -    if (null != securityHandler) {
> +    if (null != config) {
> +      ConstraintSecurityHandler securityHandler = getSecurityHandler();
>       securityHandler.setHandler(handler);
>       avaticaHandler = securityHandler;
>     }
> @@ -251,30 +280,30 @@ public class HttpServer {
>     handlerList.setHandlers(new Handler[] {avaticaHandler, new DefaultHandler()});
> 
>     server.setHandler(handlerList);
> -    // Apply server customizers
> -    for (ServerCustomizer<Server> customizer : this.serverCustomizers) {
> -      customizer.customize(server);
> -    }
> -
> -    try {
> -      server.start();
> -    } catch (Exception e) {
> -      throw new RuntimeException(e);
> -    }
> -    port = connector.getLocalPort();
> -
> -    LOG.info("Service listening on port {}.", getPort());
> +    return handlerList;
> +  }
> 
> -    // Set the information about the address for this server
> -    try {
> -      this.handler.setServerRpcMetadata(createRpcServerMetadata(connector));
> -    } catch (UnknownHostException e) {
> -      // Failed to do the DNS lookup, bail out.
> -      throw new RuntimeException(e);
> -    }
> +  private ConstraintSecurityHandler getSecurityHandler() {
> +    ConstraintSecurityHandler securityHandler = null;
> +    switch (config.getAuthenticationType()) {
> +    case SPNEGO:
> +      // Get the Handler for SPNEGO authentication
> +      securityHandler = configureSpnego(server, this.config);
> +      break;
> +    case BASIC:
> +      securityHandler = configureBasicAuthentication(server, config);
> +      break;
> +    case DIGEST:
> +      securityHandler = configureDigestAuthentication(server, config);
> +      break;
> +    default:
> +      // Pass
> +      break;
> +    }
> +    return securityHandler;
>   }
> 
> -  private ServerConnector getConnector() {
> +  protected ServerConnector getServerConnector() {
>     HttpConnectionFactory factory = new HttpConnectionFactory();
>     factory.getHttpConfiguration().setRequestHeaderSize(maxAllowedHeaderSize);
> 
> @@ -302,10 +331,9 @@ public class HttpServer {
>   /**
>    * Configures the <code>connector</code> given the <code>config</code> for using SPNEGO.
>    *
> -   * @param connector The connector to configure
>    * @param config The configuration
>    */
> -  protected ConstraintSecurityHandler configureSpnego(Server server, ServerConnector connector,
> +  protected ConstraintSecurityHandler configureSpnego(Server server,
>       AvaticaServerConfiguration config) {
>     final String realm = Objects.requireNonNull(config.getKerberosRealm());
>     final String principal = Objects.requireNonNull(config.getKerberosPrincipal());
> @@ -318,7 +346,7 @@ public class HttpServer {
>     // Roles are "realms" for Kerberos/SPNEGO
>     final String[] allowedRealms = getAllowedRealms(realm, config);
> 
> -    return configureCommonAuthentication(server, connector, config, Constraint.__SPNEGO_AUTH,
> +    return configureCommonAuthentication(Constraint.__SPNEGO_AUTH,
>         allowedRealms, new AvaticaSpnegoAuthenticator(), realm, spnegoLoginService);
>   }
> 
> @@ -336,7 +364,7 @@ public class HttpServer {
>   }
> 
>   protected ConstraintSecurityHandler configureBasicAuthentication(Server server,
> -      ServerConnector connector, AvaticaServerConfiguration config) {
> +      AvaticaServerConfiguration config) {
>     final String[] allowedRoles = config.getAllowedRoles();
>     final String realm = config.getHashLoginServiceRealm();
>     final String loginServiceProperties = config.getHashLoginServiceProperties();
> @@ -344,12 +372,12 @@ public class HttpServer {
>     HashLoginService loginService = new HashLoginService(realm, loginServiceProperties);
>     server.addBean(loginService);
> 
> -    return configureCommonAuthentication(server, connector, config, Constraint.__BASIC_AUTH,
> +    return configureCommonAuthentication(Constraint.__BASIC_AUTH,
>         allowedRoles, new BasicAuthenticator(), null, loginService);
>   }
> 
>   protected ConstraintSecurityHandler configureDigestAuthentication(Server server,
> -      ServerConnector connector, AvaticaServerConfiguration config) {
> +      AvaticaServerConfiguration config) {
>     final String[] allowedRoles = config.getAllowedRoles();
>     final String realm = config.getHashLoginServiceRealm();
>     final String loginServiceProperties = config.getHashLoginServiceProperties();
> @@ -357,12 +385,11 @@ public class HttpServer {
>     HashLoginService loginService = new HashLoginService(realm, loginServiceProperties);
>     server.addBean(loginService);
> 
> -    return configureCommonAuthentication(server, connector, config, Constraint.__DIGEST_AUTH,
> +    return configureCommonAuthentication(Constraint.__DIGEST_AUTH,
>         allowedRoles, new DigestAuthenticator(), null, loginService);
>   }
> 
> -  protected ConstraintSecurityHandler configureCommonAuthentication(Server server,
> -      ServerConnector connector, AvaticaServerConfiguration config, String constraintName,
> +  protected ConstraintSecurityHandler configureCommonAuthentication(String constraintName,
>       String[] allowedRoles, Authenticator authenticator, String realm,
>       LoginService loginService) {
> 
> @@ -467,6 +494,8 @@ public class HttpServer {
> 
>     // The maximum size in bytes of an http header the server will read (64KB)
>     private int maxAllowedHeaderSize = MAX_ALLOWED_HEADER_SIZE;
> +    private AvaticaServerConfiguration serverConfig;
> +    private Subject subject;
> 
>     public Builder() {}
> 
> @@ -659,6 +688,22 @@ public class HttpServer {
>       return withAuthentication(AuthenticationType.DIGEST, properties, allowedRoles);
>     }
> 
> +    /**
> +     * Configures the server to use CUSTOM authentication mechanism, which can allow users to
> +     * combine benefits of multiple auth methods. See <code>CustomAuthHttpServerTest</code> for
> +     * examples on how to use it.
> +     * Note: Default ServerConnectors and Handlers will NOT be used.
> +     * Customize them directly using instances <code>{@link ServerCustomizer}</code>
> +     * @param config AvaticaServerConfiguration implementation that configures various details
> +     *      about the authentication mechanism for <code>{@link HttpServer}</code>
> +     * @return <code>this</code>
> +     */
> +    public Builder<T> withCustomAuthentication(AvaticaServerConfiguration config) {
> +      this.authenticationType = AuthenticationType.CUSTOM;
> +      this.serverConfig = config;
> +      return this;
> +    }
> +
>     private Builder<T> withAuthentication(AuthenticationType authType, String properties,
>         String[] allowedRoles) {
>       this.loginServiceRealm = "Avatica";
> @@ -721,18 +766,18 @@ public class HttpServer {
>      */
>     @SuppressWarnings("unchecked")
>     public HttpServer build() {
> -      final AvaticaServerConfiguration serverConfig;
> -      final Subject subject;
>       switch (authenticationType) {
>       case NONE:
>         serverConfig = null;
>         subject = null;
> +        handler = buildHandler(this, serverConfig);
>         break;
>       case BASIC:
>       case DIGEST:
>         // Build the configuration for BASIC or DIGEST authentication.
>         serverConfig = buildUserAuthenticationConfiguration(this);
>         subject = null;
> +        handler = buildHandler(this, serverConfig);
>         break;
>       case SPNEGO:
>         if (usingTLS) {
> @@ -746,20 +791,19 @@ public class HttpServer {
>           subject = null;
>         }
>         serverConfig = buildSpnegoConfiguration(this);
> +        handler = buildHandler(this, serverConfig);
> +        break;
> +      case CUSTOM:
> +        // We don't need to build any Config here since
> +        // serverConfig is already assigned the required AvaticaServerConfiguration
> +        serverConfig = buildCustomConfiguration(this);
> +        subject = null;
>         break;
>       default:
>         throw new IllegalArgumentException("Unhandled AuthenticationType");
>       }
> 
> -      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);
> -      }
> +      SslContextFactory sslFactory = buildSSLContextFactory();
> 
>       List<ServerCustomizer<Server>> jettyCustomizers = new ArrayList<>();
>       for (ServerCustomizer<?> customizer : this.serverCustomizers) {
> @@ -771,6 +815,22 @@ public class HttpServer {
>           maxAllowedHeaderSize);
>     }
> 
> +    protected SslContextFactory buildSSLContextFactory() {
> +      SslContextFactory sslFactory = null;
> +      if (usingTLS) {
> +        sslFactory = new SslContextFactory();
> +        sslFactory.setKeyStorePath(this.keystore.getAbsolutePath());
> +        sslFactory.setKeyStorePassword(keystorePassword);
> +        sslFactory.setTrustStorePath(truststore.getAbsolutePath());
> +        sslFactory.setTrustStorePassword(truststorePassword);
> +      }
> +      return sslFactory;
> +    }
> +
> +    private AvaticaServerConfiguration buildCustomConfiguration(Builder<T> tBuilder) {
> +      return tBuilder.serverConfig;
> +    }
> +
>     /**
>      * Creates the appropriate {@link AvaticaHandler}.
>      *
> 
> http://git-wip-us.apache.org/repos/asf/calcite-avatica/blob/3ab9ec6f/server/src/test/java/org/apache/calcite/avatica/server/CustomAuthHttpServerTest.java
> ----------------------------------------------------------------------
> diff --git a/server/src/test/java/org/apache/calcite/avatica/server/CustomAuthHttpServerTest.java b/server/src/test/java/org/apache/calcite/avatica/server/CustomAuthHttpServerTest.java
> new file mode 100644
> index 0000000..dbb2f4c
> --- /dev/null
> +++ b/server/src/test/java/org/apache/calcite/avatica/server/CustomAuthHttpServerTest.java
> @@ -0,0 +1,338 @@
> +/*
> + * 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.server;
> +
> +import org.apache.calcite.avatica.ConnectionSpec;
> +import org.apache.calcite.avatica.jdbc.JdbcMeta;
> +import org.apache.calcite.avatica.remote.AuthenticationType;
> +import org.apache.calcite.avatica.remote.Driver;
> +import org.apache.calcite.avatica.remote.LocalService;
> +
> +import org.eclipse.jetty.security.ConstraintSecurityHandler;
> +import org.eclipse.jetty.security.UserAuthentication;
> +import org.eclipse.jetty.server.Authentication;
> +import org.eclipse.jetty.server.Connector;
> +import org.eclipse.jetty.server.Handler;
> +import org.eclipse.jetty.server.Request;
> +import org.eclipse.jetty.server.Server;
> +import org.eclipse.jetty.server.ServerConnector;
> +import org.eclipse.jetty.server.UserIdentity;
> +import org.eclipse.jetty.server.handler.DefaultHandler;
> +import org.eclipse.jetty.server.handler.HandlerList;
> +import org.junit.After;
> +import org.junit.Assert;
> +import org.junit.Before;
> +import org.junit.Test;
> +import org.mockito.Mockito;
> +
> +import java.sql.SQLException;
> +import javax.servlet.http.HttpServletRequest;
> +
> +import static org.hamcrest.core.StringContains.containsString;
> +import static org.junit.Assert.assertThat;
> +import static org.junit.Assert.fail;
> +
> +import java.util.Arrays;
> +import java.util.Properties;
> +import java.util.concurrent.Callable;
> +
> +/**
> + * Test class for providing CustomAvaticaServerConfiguration to the HTTP Server
> + */
> +public class CustomAuthHttpServerTest extends HttpAuthBase {
> +  private static final ConnectionSpec CONNECTION_SPEC = ConnectionSpec.HSQLDB;
> +  private static HttpServer server;
> +  private static String url;
> +
> +  // Counters to keep track of number of function calls
> +  private static int methodCallCounter1 = 0;
> +  private static int methodCallCounter2 = 0;
> +  private static int methodCallCounter3 = 0;
> +
> +  @Before
> +  public void before() {
> +    methodCallCounter1 = 0;
> +    methodCallCounter2 = 0;
> +    methodCallCounter3 = 0;
> +  }
> +
> +  @After
> +  public void stopServer() {
> +    if (null != server) {
> +      server.stop();
> +    }
> +  }
> +
> +  @Test
> +  public void testCustomImpersonationConfig() throws Exception {
> +    AvaticaServerConfiguration configuration = new CustomImpersonationConfig();
> +    createServer(configuration, false);
> +
> +    readWriteData(url, "CUSTOM_CONFIG_1_TABLE", new Properties());
> +    Assert.assertEquals("supportsImpersonation should be called same number of "
> +            + "times as doAsRemoteUser method", methodCallCounter1, methodCallCounter2);
> +    Assert.assertEquals("supportsImpersonation should be called same number of "
> +            + "times as getRemoteUserExtractor method", methodCallCounter1, methodCallCounter3);
> +  }
> +
> +  @Test
> +  public void testCustomBasicImpersonationConfigWithAllowedUser() throws Exception {
> +    AvaticaServerConfiguration configuration = new CustomBasicImpersonationConfig();
> +    createServer(configuration, true);
> +
> +    final Properties props = new Properties();
> +    props.put("avatica_user", "USER2");
> +    props.put("avatica_password", "password2");
> +    props.put("user", "USER2");
> +    props.put("password", "password2");
> +
> +    readWriteData(url, "CUSTOM_CONFIG_2_ALLOWED_TABLE", props);
> +    Assert.assertEquals("supportsImpersonation should be called same number of "
> +            + "times as doAsRemoteUser method", methodCallCounter1, methodCallCounter2);
> +    Assert.assertEquals("supportsImpersonation should be called same number of "
> +            + "times as getRemoteUserExtractor method", methodCallCounter1, methodCallCounter3);
> +  }
> +
> +  @Test
> +  public void testCustomBasicImpersonationConfigWithDisallowedUser() throws Exception {
> +    AvaticaServerConfiguration configuration = new CustomBasicImpersonationConfig();
> +    createServer(configuration, true);
> +
> +    final Properties props = new Properties();
> +    props.put("avatica_user", "USER1");
> +    props.put("avatica_password", "password1");
> +    props.put("user", "USER1");
> +    props.put("password", "password1");
> +
> +    try {
> +      readWriteData(url, "CUSTOM_CONFIG_2_DISALLOWED_TABLE", props);
> +      fail("Expected an exception");
> +    } catch (RuntimeException e) {
> +      assertThat(e.getMessage(), containsString("Failed to execute HTTP Request, got HTTP/403"));
> +    }
> +  }
> +
> +  @Test(expected = IllegalStateException.class)
> +  public void testCustomConfigDisallowsWithHandlerMethod() {
> +    AvaticaServerConfiguration configuration = new CustomBasicImpersonationConfig();
> +    server = new HttpServer.Builder()
> +            .withCustomAuthentication(configuration)
> +            .withHandler(Mockito.mock(AvaticaHandler.class))
> +            .withPort(0)
> +            .build();
> +    server.start();
> +  }
> +
> +  public static HttpServer getAvaticaServer() {
> +    return server;
> +  }
> +
> +  @SuppressWarnings("unchecked") // needed for the mocked customizers, not the builder
> +  protected void createServer(AvaticaServerConfiguration config, boolean isBasicAuth)
> +      throws SQLException {
> +    final JdbcMeta jdbcMeta = new JdbcMeta(CONNECTION_SPEC.url,
> +      CONNECTION_SPEC.username, CONNECTION_SPEC.password);
> +    LocalService service = new LocalService(jdbcMeta);
> +
> +    ConnectorCustomizer connectorCustomizer = new ConnectorCustomizer();
> +    BasicAuthHandlerCustomizer basicAuthCustomizer =
> +            new BasicAuthHandlerCustomizer(config, service, isBasicAuth);
> +
> +    server = new HttpServer.Builder()
> +            .withCustomAuthentication(config)
> +            .withPort(0)
> +            .withServerCustomizers(
> +                    Arrays.asList(connectorCustomizer, basicAuthCustomizer), Server.class)
> +            .build();
> +    server.start();
> +
> +    // Create and grant permissions to our users
> +    createHsqldbUsers();
> +    url = "jdbc:avatica:remote:url=http://localhost:" + connectorCustomizer.getLocalPort()
> +            + ";authentication=BASIC;serialization=PROTOBUF";
> +  }
> +
> +  /**
> +   * Customizer to add ServerConnectors to the server
> +   */
> +  static class ConnectorCustomizer implements ServerCustomizer<Server> {
> +
> +    ServerConnector connector;
> +
> +    @Override public void customize(Server server) {
> +      HttpServer avaticaServer = getAvaticaServer();
> +      connector = avaticaServer.configureConnector(avaticaServer.getServerConnector(), 0);
> +      server.setConnectors(new Connector[] { connector });
> +    }
> +
> +    public int getLocalPort() {
> +      return connector.getLocalPort();
> +    }
> +
> +  }
> +
> +  /**
> +   * Customizer to add handlers to the server (with or without BasicAuth)
> +   */
> +  static class BasicAuthHandlerCustomizer implements ServerCustomizer<Server> {
> +
> +    AvaticaServerConfiguration configuration;
> +    LocalService service;
> +    boolean isBasicAuth;
> +
> +    public BasicAuthHandlerCustomizer(AvaticaServerConfiguration configuration
> +            , LocalService service, boolean isBasicAuth) {
> +      this.configuration = configuration;
> +      this.service = service;
> +      this.isBasicAuth = isBasicAuth;
> +    }
> +
> +    @Override public void customize(Server server) {
> +      HttpServer avaticaServer = getAvaticaServer();
> +
> +      HandlerFactory factory = new HandlerFactory();
> +      Handler avaticaHandler = factory.getHandler(service,
> +              Driver.Serialization.PROTOBUF, null, configuration);
> +
> +      if (isBasicAuth) {
> +        ConstraintSecurityHandler securityHandler =
> +                avaticaServer.configureBasicAuthentication(server, configuration);
> +        securityHandler.setHandler(avaticaHandler);
> +        avaticaHandler = securityHandler;
> +      }
> +
> +      HandlerList handlerList = new HandlerList();
> +      handlerList.setHandlers(new Handler[] { avaticaHandler, new DefaultHandler()});
> +      server.setHandler(handlerList);
> +    }
> +  }
> +
> +  /**
> +   * CustomImpersonationConfig doesn't authenticates the user but supports user impersonation
> +   */
> +  static class CustomImpersonationConfig implements AvaticaServerConfiguration {
> +
> +
> +    @Override public AuthenticationType getAuthenticationType() {
> +      return AuthenticationType.CUSTOM;
> +    }
> +
> +    @Override public String getKerberosRealm() {
> +      return null;
> +    }
> +
> +    @Override public String getKerberosPrincipal() {
> +      return null;
> +    }
> +
> +    @Override public String[] getAllowedRoles() {
> +      return new String[0];
> +    }
> +
> +    @Override public String getHashLoginServiceRealm() {
> +      return null;
> +    }
> +
> +    @Override public String getHashLoginServiceProperties() {
> +      return null;
> +    }
> +
> +    @Override public boolean supportsImpersonation() {
> +      methodCallCounter1++;
> +      return true;
> +    }
> +
> +    @Override public <T> T doAsRemoteUser(String remoteUserName,
> +              String remoteAddress, Callable<T> action) throws Exception {
> +      methodCallCounter2++;
> +      return action.call();
> +    }
> +    @Override public RemoteUserExtractor getRemoteUserExtractor() {
> +      return new RemoteUserExtractor() {
> +        @Override public String extract(HttpServletRequest request) {
> +          methodCallCounter3++;
> +          return "randomUser";
> +        }
> +      };
> +    }
> +
> +  }
> +
> +  /**
> +   * CustomBasicImpersonationConfig supports BasicAuthentication with user impersonation
> +   */
> +  static class CustomBasicImpersonationConfig implements AvaticaServerConfiguration {
> +
> +
> +    @Override public AuthenticationType getAuthenticationType() {
> +      return AuthenticationType.CUSTOM;
> +    }
> +
> +    @Override public String getKerberosRealm() {
> +      return null;
> +    }
> +
> +    @Override public String getKerberosPrincipal() {
> +      return null;
> +    }
> +
> +    @Override public String[] getAllowedRoles() {
> +      return new String[] { "users" };
> +    }
> +
> +    @Override public String getHashLoginServiceRealm() {
> +      return "Avatica";
> +    }
> +
> +    @Override public String getHashLoginServiceProperties() {
> +      return HttpAuthBase.getHashLoginServicePropertiesString();
> +    }
> +
> +    @Override public boolean supportsImpersonation() {
> +      methodCallCounter1++;
> +      return true;
> +    }
> +
> +    @Override public <T> T doAsRemoteUser(String remoteUserName,
> +      String remoteAddress, Callable<T> action) throws Exception {
> +      methodCallCounter2++;
> +      if (remoteUserName.equals("USER1")) {
> +        throw new RemoteUserDisallowedException("USER1 is a disallowed user!");
> +      }
> +      return action.call();
> +    }
> +    @Override public RemoteUserExtractor getRemoteUserExtractor() {
> +      return new RemoteUserExtractor() {
> +        @Override public String extract(HttpServletRequest request)
> +            throws RemoteUserExtractionException {
> +          methodCallCounter3++;
> +          if (request instanceof Request) {
> +            Authentication authentication = ((Request) request).getAuthentication();
> +            if (authentication instanceof UserAuthentication) {
> +              UserIdentity userIdentity = ((UserAuthentication) authentication).getUserIdentity();
> +              return userIdentity.getUserPrincipal().getName();
> +            }
> +          }
> +          throw new RemoteUserExtractionException("Request doesn't contain user credentials.");
> +        }
> +      };
> +    }
> +  }
> +
> +}
> +
> +// End CustomAuthHttpServerTest.java
> 
> http://git-wip-us.apache.org/repos/asf/calcite-avatica/blob/3ab9ec6f/server/src/test/java/org/apache/calcite/avatica/server/HttpAuthBase.java
> ----------------------------------------------------------------------
> diff --git a/server/src/test/java/org/apache/calcite/avatica/server/HttpAuthBase.java b/server/src/test/java/org/apache/calcite/avatica/server/HttpAuthBase.java
> index cfaf302..6ce0afe 100644
> --- a/server/src/test/java/org/apache/calcite/avatica/server/HttpAuthBase.java
> +++ b/server/src/test/java/org/apache/calcite/avatica/server/HttpAuthBase.java
> @@ -18,6 +18,8 @@ package org.apache.calcite.avatica.server;
> 
> import org.apache.calcite.avatica.ConnectionSpec;
> 
> +import java.io.UnsupportedEncodingException;
> +import java.net.URLDecoder;
> import java.sql.Connection;
> import java.sql.DriverManager;
> import java.sql.ResultSet;
> @@ -75,6 +77,19 @@ public class HttpAuthBase {
>       assertEquals(3, results.getInt(1));
>     }
>   }
> +
> +  static String getHashLoginServicePropertiesString() {
> +    try {
> +      final String userPropertiesFile =
> +              URLDecoder.decode(HttpQueryStringParameterRemoteUserExtractorTest.class
> +                      .getResource("/auth-users.properties").getFile(), "UTF-8");
> +      assertNotNull("Could not find properties file for basic auth users", userPropertiesFile);
> +      return userPropertiesFile;
> +    } catch (UnsupportedEncodingException e) {
> +      throw new RuntimeException(e);
> +    }
> +  }
> +
> }
> 
> // End HttpAuthBase.java
> 
> http://git-wip-us.apache.org/repos/asf/calcite-avatica/blob/3ab9ec6f/server/src/test/java/org/apache/calcite/avatica/server/HttpQueryStringParameterRemoteUserExtractorTest.java
> ----------------------------------------------------------------------
> diff --git a/server/src/test/java/org/apache/calcite/avatica/server/HttpQueryStringParameterRemoteUserExtractorTest.java b/server/src/test/java/org/apache/calcite/avatica/server/HttpQueryStringParameterRemoteUserExtractorTest.java
> index 7fcde9a..33d91bc 100644
> --- a/server/src/test/java/org/apache/calcite/avatica/server/HttpQueryStringParameterRemoteUserExtractorTest.java
> +++ b/server/src/test/java/org/apache/calcite/avatica/server/HttpQueryStringParameterRemoteUserExtractorTest.java
> @@ -29,14 +29,11 @@ import org.junit.runners.Parameterized.Parameters;
> import org.slf4j.Logger;
> import org.slf4j.LoggerFactory;
> 
> -import java.io.UnsupportedEncodingException;
> -import java.net.URLDecoder;
> import java.util.List;
> import java.util.Properties;
> import java.util.concurrent.Callable;
> 
> import static org.hamcrest.core.StringContains.containsString;
> -import static org.junit.Assert.assertNotNull;
> import static org.junit.Assert.assertThat;
> import static org.junit.Assert.fail;
> 
> @@ -129,16 +126,7 @@ public class HttpQueryStringParameterRemoteUserExtractorTest extends HttpAuthBas
>     }
> 
>     @Override public String getHashLoginServiceProperties() {
> -      try {
> -        final String userPropertiesFile =
> -            URLDecoder.decode(HttpQueryStringParameterRemoteUserExtractorTest.class
> -                .getResource("/auth-users.properties").getFile(), "UTF-8");
> -        assertNotNull("Could not find properties file for basic auth users", userPropertiesFile);
> -        return userPropertiesFile;
> -      } catch (UnsupportedEncodingException e) {
> -        LOG.error("Failed to decode path to Jetty users file", e);
> -        throw new RuntimeException(e);
> -      }
> +      return HttpAuthBase.getHashLoginServicePropertiesString();
>     }
>   };
> 
> 
> http://git-wip-us.apache.org/repos/asf/calcite-avatica/blob/3ab9ec6f/site/_docs/security.md
> ----------------------------------------------------------------------
> diff --git a/site/_docs/security.md b/site/_docs/security.md
> index 53233bd..a6909dc 100644
> --- a/site/_docs/security.md
> +++ b/site/_docs/security.md
> @@ -7,6 +7,7 @@ auth_types:
>   - { name: "HTTP Basic", anchor: "http-basic-authentication" }
>   - { name: "HTTP Digest", anchor: "http-digest-authentication" }
>   - { name: "Kerberos with SPNEGO", anchor: "kerberos-with-spnego-authentication" }
> +  - { name: "Custom Authentication", anchor: "custom-authentication" }
>   - { name: "Client implementation", anchor: "client-implementation" }
> ---
> <!--
> @@ -256,6 +257,23 @@ config = new AvaticaServerConfiguration() {
> };
> {% endhighlight %}
> 
> +## Custom Authentication
> +
> +Avatica server now offers users to plugin their Custom Authentication mechanism through the HTTPServer Builder.
> +This is useful if users want to combine features of various authentication types. Examples include combining 
> +basic authentication with impersonation or adding mutual authentication with impersonation. More Examples
> +are available in `CustomAuthHttpServerTest` class.
> +
> +Note: Users need to configure their own `ServerConnectors` and `Handlers` with the help of `ServerCustomizers`.
> +{% highlight java %}
> +AvaticaServerConfiguration configuration = new ExampleAvaticaServerConfiguration();
> +HttpServer server = new HttpServer.Builder()
> +    .withCustomAuthentication(configuration)
> +    .withPort(8765)
> +    .build();
> +{% endhighlight %}
> +
> +
> ## Client implementation
> 
> Many HTTP client libraries, such as [Apache Commons HttpComponents](https://hc.apache.org/), already have
>