You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by is...@apache.org on 2016/12/19 15:15:39 UTC

[2/2] lucene-solr:master: SOLR-9513: Generic Hadoop authentication plugins, GenericHadoopAuthPlugin and ConfigurableInternodeAuthHadoopPlugin

SOLR-9513: Generic Hadoop authentication plugins, GenericHadoopAuthPlugin and ConfigurableInternodeAuthHadoopPlugin


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

Branch: refs/heads/master
Commit: a1a8b2864e621c18aa86b21d4a244233e991a47d
Parents: 321c6f0
Author: Ishan Chattopadhyaya <is...@apache.org>
Authored: Mon Dec 19 20:45:04 2016 +0530
Committer: Ishan Chattopadhyaya <is...@apache.org>
Committed: Mon Dec 19 20:45:04 2016 +0530

----------------------------------------------------------------------
 solr/CHANGES.txt                                |   3 +
 .../security/AttributeOnlyServletContext.java   | 291 ++++++++++++++
 .../ConfigurableInternodeAuthHadoopPlugin.java  |  68 ++++
 .../security/DelegationTokenKerberosFilter.java |   6 +-
 .../solr/security/GenericHadoopAuthPlugin.java  | 266 ++++++++++++
 .../apache/solr/security/HadoopAuthFilter.java  | 198 +++++++++
 .../apache/solr/security/HadoopAuthPlugin.java  | 241 +++++++++++
 .../apache/solr/security/KerberosFilter.java    |   6 +-
 .../apache/solr/security/KerberosPlugin.java    | 314 +--------------
 ...tContinuesRecorderAuthenticationHandler.java |  71 ++++
 .../solr/security/hadoop_kerberos_config.json   |  16 +
 .../hadoop_simple_auth_with_delegation.json     |  29 ++
 .../TestSolrCloudWithSecureImpersonation.java   |   8 +-
 .../solr/security/hadoop/ImpersonationUtil.java |  73 ++++
 .../hadoop/ImpersonatorCollectionsHandler.java  |  60 +++
 .../hadoop/TestDelegationWithHadoopAuth.java    | 400 +++++++++++++++++++
 .../hadoop/TestImpersonationWithHadoopAuth.java | 215 ++++++++++
 .../TestSolrCloudWithHadoopAuthPlugin.java      | 136 +++++++
 .../solrj/impl/HttpClientBuilderFactory.java    |  41 ++
 .../solrj/impl/Krb5HttpClientBuilder.java       |  10 +-
 .../apache/solr/cloud/MiniSolrCloudCluster.java |  29 +-
 .../apache/solr/cloud/SolrCloudTestCase.java    |  30 +-
 22 files changed, 2185 insertions(+), 326 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a1a8b286/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index c920575..f783934 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -173,6 +173,9 @@ New Features
 * SOLR-9844: FieldCache information fetched via the mbeans handler or seen via the UI now displays the total size used.
   The individual cache entries in the response are now formatted better as well. (Varun Thacker)
 
+* SOLR-9513: Generic authentication plugins (GenericHadoopAuthPlugin and ConfigurableInternodeAuthHadoopPlugin) that delegate
+  all functionality to Hadoop authentication framework. (Hrishikesh Gadre via Ishan Chattopadhyaya)
+
 Optimizations
 ----------------------
 * SOLR-9704: Facet Module / JSON Facet API: Optimize blockChildren facets that have

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a1a8b286/solr/core/src/java/org/apache/solr/security/AttributeOnlyServletContext.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/security/AttributeOnlyServletContext.java b/solr/core/src/java/org/apache/solr/security/AttributeOnlyServletContext.java
new file mode 100644
index 0000000..4abcd40
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/security/AttributeOnlyServletContext.java
@@ -0,0 +1,291 @@
+/*
+ * 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.solr.security;
+
+import java.io.InputStream;
+import java.net.MalformedURLException;
+import java.net.URL;
+
+import java.util.Collections;
+import java.util.Enumeration;
+import java.util.EventListener;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+
+import javax.servlet.Filter;
+import javax.servlet.FilterRegistration;
+import javax.servlet.RequestDispatcher;
+import javax.servlet.Servlet;
+import javax.servlet.ServletContext;
+import javax.servlet.ServletException;
+import javax.servlet.ServletRegistration;
+import javax.servlet.SessionCookieConfig;
+import javax.servlet.SessionTrackingMode;
+import javax.servlet.FilterRegistration.Dynamic;
+import javax.servlet.descriptor.JspConfigDescriptor;
+
+/**
+ * A concrete implementation of {@linkplain ServletContext} which support only attributes.
+ */
+class AttributeOnlyServletContext implements ServletContext {
+  private Map<String, Object> attributes = new HashMap<String, Object>();
+
+  @Override
+  public void setSessionTrackingModes(Set<SessionTrackingMode> sessionTrackingModes) {}
+
+  @Override
+  public boolean setInitParameter(String name, String value) {
+    return false;
+  }
+
+  @Override
+  public void setAttribute(String name, Object object) {
+    attributes.put(name, object);
+  }
+
+  @Override
+  public void removeAttribute(String name) {
+    attributes.remove(name);
+  }
+
+  @Override
+  public void log(String message, Throwable throwable) {}
+
+  @Override
+  public void log(Exception exception, String msg) {}
+
+  @Override
+  public void log(String msg) {}
+
+  @Override
+  public String getVirtualServerName() {
+    return null;
+  }
+
+  @Override
+  public SessionCookieConfig getSessionCookieConfig() {
+    return null;
+  }
+
+  @Override
+  public Enumeration<Servlet> getServlets() {
+    return null;
+  }
+
+  @Override
+  public Map<String,? extends ServletRegistration> getServletRegistrations() {
+    return null;
+  }
+
+  @Override
+  public ServletRegistration getServletRegistration(String servletName) {
+    return null;
+  }
+
+  @Override
+  public Enumeration<String> getServletNames() {
+    return null;
+  }
+
+  @Override
+  public String getServletContextName() {
+    return null;
+  }
+
+  @Override
+  public Servlet getServlet(String name) throws ServletException {
+    return null;
+  }
+
+  @Override
+  public String getServerInfo() {
+    return null;
+  }
+
+  @Override
+  public Set<String> getResourcePaths(String path) {
+    return null;
+  }
+
+  @Override
+  public InputStream getResourceAsStream(String path) {
+    return null;
+  }
+
+  @Override
+  public URL getResource(String path) throws MalformedURLException {
+    return null;
+  }
+
+  @Override
+  public RequestDispatcher getRequestDispatcher(String path) {
+    return null;
+  }
+
+  @Override
+  public String getRealPath(String path) {
+    return null;
+  }
+
+  @Override
+  public RequestDispatcher getNamedDispatcher(String name) {
+    return null;
+  }
+
+  @Override
+  public int getMinorVersion() {
+    return 0;
+  }
+
+  @Override
+  public String getMimeType(String file) {
+    return null;
+  }
+
+  @Override
+  public int getMajorVersion() {
+    return 0;
+  }
+
+  @Override
+  public JspConfigDescriptor getJspConfigDescriptor() {
+    return null;
+  }
+
+  @Override
+  public Enumeration<String> getInitParameterNames() {
+    return null;
+  }
+
+  @Override
+  public String getInitParameter(String name) {
+    return null;
+  }
+
+  @Override
+  public Map<String,? extends FilterRegistration> getFilterRegistrations() {
+    return null;
+  }
+
+  @Override
+  public FilterRegistration getFilterRegistration(String filterName) {
+    return null;
+  }
+
+  @Override
+  public Set<SessionTrackingMode> getEffectiveSessionTrackingModes() {
+    return null;
+  }
+
+  @Override
+  public int getEffectiveMinorVersion() {
+    return 0;
+  }
+
+  @Override
+  public int getEffectiveMajorVersion() {
+    return 0;
+  }
+
+  @Override
+  public Set<SessionTrackingMode> getDefaultSessionTrackingModes() {
+    return null;
+  }
+
+  @Override
+  public String getContextPath() {
+    return null;
+  }
+
+  @Override
+  public ServletContext getContext(String uripath) {
+    return null;
+  }
+
+  @Override
+  public ClassLoader getClassLoader() {
+    return null;
+  }
+
+  @Override
+  public Enumeration<String> getAttributeNames() {
+    return Collections.enumeration(attributes.keySet());
+  }
+
+  @Override
+  public Object getAttribute(String name) {
+    return attributes.get(name);
+  }
+
+  @Override
+  public void declareRoles(String... roleNames) {}
+
+  @Override
+  public <T extends Servlet> T createServlet(Class<T> clazz) throws ServletException {
+    return null;
+  }
+
+  @Override
+  public <T extends EventListener> T createListener(Class<T> clazz) throws ServletException {
+    return null;
+  }
+
+  @Override
+  public <T extends Filter> T createFilter(Class<T> clazz) throws ServletException {
+    return null;
+  }
+
+  @Override
+  public javax.servlet.ServletRegistration.Dynamic addServlet(String servletName, Class<? extends Servlet> servletClass) {
+    return null;
+  }
+
+  @Override
+  public javax.servlet.ServletRegistration.Dynamic addServlet(String servletName, Servlet servlet) {
+    return null;
+  }
+
+  @Override
+  public javax.servlet.ServletRegistration.Dynamic addServlet(String servletName, String className) {
+    return null;
+  }
+
+  @Override
+  public void addListener(Class<? extends EventListener> listenerClass) {}
+
+  @Override
+  public <T extends EventListener> void addListener(T t) {}
+
+  @Override
+  public void addListener(String className) {}
+
+  @Override
+  public Dynamic addFilter(String filterName, Class<? extends Filter> filterClass) {
+    return null;
+  }
+
+  @Override
+  public Dynamic addFilter(String filterName, Filter filter) {
+    return null;
+  }
+
+  @Override
+  public Dynamic addFilter(String filterName, String className) {
+    return null;
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a1a8b286/solr/core/src/java/org/apache/solr/security/ConfigurableInternodeAuthHadoopPlugin.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/security/ConfigurableInternodeAuthHadoopPlugin.java b/solr/core/src/java/org/apache/solr/security/ConfigurableInternodeAuthHadoopPlugin.java
new file mode 100644
index 0000000..f3bb70f
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/security/ConfigurableInternodeAuthHadoopPlugin.java
@@ -0,0 +1,68 @@
+/*
+ * 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.solr.security;
+
+import java.io.IOException;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+
+import org.apache.solr.client.solrj.impl.HttpClientBuilderFactory;
+import org.apache.solr.client.solrj.impl.SolrHttpClientBuilder;
+import org.apache.solr.core.CoreContainer;
+
+/**
+ * This class extends {@linkplain HadoopAuthPlugin} by enabling configuration of
+ * authentication mechanism for Solr internal communication.
+ **/
+public class ConfigurableInternodeAuthHadoopPlugin extends HadoopAuthPlugin implements HttpClientBuilderPlugin {
+
+  /**
+   * A property specifying the {@linkplain HttpClientBuilderFactory} used for the Solr internal
+   * communication.
+   */
+  private static final String HTTPCLIENT_BUILDER_FACTORY = "clientBuilderFactory";
+
+  private HttpClientBuilderFactory factory = null;
+
+  public ConfigurableInternodeAuthHadoopPlugin(CoreContainer coreContainer) {
+    super(coreContainer);
+  }
+
+  @Override
+  public void init(Map<String,Object> pluginConfig) {
+    super.init(pluginConfig);
+
+    String httpClientBuilderFactory = (String)Objects.requireNonNull(pluginConfig.get(HTTPCLIENT_BUILDER_FACTORY),
+        "Please specify clientBuilderFactory to be used for Solr internal communication.");
+    factory = this.coreContainer.getResourceLoader().newInstance(httpClientBuilderFactory, HttpClientBuilderFactory.class);
+  }
+
+  @Override
+  public SolrHttpClientBuilder getHttpClientBuilder(SolrHttpClientBuilder builder) {
+    return factory.getHttpClientBuilder(Optional.ofNullable(builder));
+  }
+
+  @Override
+  public void close() throws IOException {
+    super.close();
+
+    if (factory != null) {
+      factory.close();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a1a8b286/solr/core/src/java/org/apache/solr/security/DelegationTokenKerberosFilter.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/security/DelegationTokenKerberosFilter.java b/solr/core/src/java/org/apache/solr/security/DelegationTokenKerberosFilter.java
index 421de52..007e0bd 100644
--- a/solr/core/src/java/org/apache/solr/security/DelegationTokenKerberosFilter.java
+++ b/solr/core/src/java/org/apache/solr/security/DelegationTokenKerberosFilter.java
@@ -141,9 +141,9 @@ public class DelegationTokenKerberosFilter extends DelegationTokenAuthentication
     // set the internal authentication handler in order to record whether the request should continue
     super.initializeAuthHandler(authHandlerClassName, filterConfig);
     AuthenticationHandler authHandler = getAuthenticationHandler();
-    super.initializeAuthHandler(KerberosPlugin.RequestContinuesRecorderAuthenticationHandler.class.getName(), filterConfig);
-    KerberosPlugin.RequestContinuesRecorderAuthenticationHandler newAuthHandler =
-        (KerberosPlugin.RequestContinuesRecorderAuthenticationHandler)getAuthenticationHandler();
+    super.initializeAuthHandler(RequestContinuesRecorderAuthenticationHandler.class.getName(), filterConfig);
+    RequestContinuesRecorderAuthenticationHandler newAuthHandler =
+        (RequestContinuesRecorderAuthenticationHandler)getAuthenticationHandler();
     newAuthHandler.setAuthHandler(authHandler);
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a1a8b286/solr/core/src/java/org/apache/solr/security/GenericHadoopAuthPlugin.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/security/GenericHadoopAuthPlugin.java b/solr/core/src/java/org/apache/solr/security/GenericHadoopAuthPlugin.java
new file mode 100644
index 0000000..e5fe349
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/security/GenericHadoopAuthPlugin.java
@@ -0,0 +1,266 @@
+/*
+ * 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.solr.security;
+
+import static org.apache.solr.security.RequestContinuesRecorderAuthenticationHandler.REQUEST_CONTINUES_ATTR;
+import static org.apache.solr.security.HadoopAuthFilter.DELEGATION_TOKEN_ZK_CLIENT;
+
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.lang.invoke.MethodHandles;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Enumeration;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+
+import javax.servlet.FilterChain;
+import javax.servlet.FilterConfig;
+import javax.servlet.ServletContext;
+import javax.servlet.ServletException;
+import javax.servlet.ServletRequest;
+import javax.servlet.ServletResponse;
+import javax.servlet.http.HttpServletResponse;
+import javax.servlet.http.HttpServletResponseWrapper;
+
+import org.apache.commons.collections.iterators.IteratorEnumeration;
+import org.apache.hadoop.security.authentication.server.AuthenticationFilter;
+import org.apache.solr.client.solrj.impl.HttpClientBuilderFactory;
+import org.apache.solr.client.solrj.impl.Krb5HttpClientBuilder;
+import org.apache.solr.client.solrj.impl.SolrHttpClientBuilder;
+import org.apache.solr.cloud.ZkController;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.SolrException.ErrorCode;
+import org.apache.solr.common.util.SuppressForbidden;
+import org.apache.solr.core.CoreContainer;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This class implements a generic plugin which can use authentication schemes exposed by the
+ * Hadoop framework. This plugin supports following features
+ * - integration with authentication mehcanisms (e.g. kerberos)
+ * - Delegation token support
+ * - Proxy users (or secure impersonation) support
+ *
+ * This plugin enables defining configuration parameters required by the undelying Hadoop authentication
+ * mechanism. These configuration parameters can either be specified as a Java system property or the default
+ * value can be specified as part of the plugin configuration.
+ *
+ * The proxy users are configured by specifying relevant Hadoop configuration parameters. Please note that
+ * the delegation token support must be enabled for using the proxy users support.
+ *
+ * For Solr internal communication, this plugin enables configuring {@linkplain HttpClientBuilderFactory}
+ * implementation (e.g. based on kerberos).
+ */
+public class GenericHadoopAuthPlugin extends AuthenticationPlugin implements HttpClientBuilderPlugin {
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  /**
+   * A property specifying the type of authentication scheme to be configured.
+   */
+  private static final String HADOOP_AUTH_TYPE = "type";
+
+  /**
+   * A property specifies the value of the prefix to be used to define Java system property
+   * for configuring the authentication mechanism. The name of the Java system property is
+   * defined by appending the configuration parmeter namne to this prefix value e.g. if prefix
+   * is 'solr' then the Java system property 'solr.kerberos.principal' defines the value of
+   * configuration parameter 'kerberos.principal'.
+   */
+  private static final String SYSPROP_PREFIX_PROPERTY = "sysPropPrefix";
+
+  /**
+   * A property specifying the configuration parameters required by the authentication scheme
+   * defined by {@linkplain #HADOOP_AUTH_TYPE} property.
+   */
+  private static final String AUTH_CONFIG_NAMES_PROPERTY = "authConfigs";
+
+  /**
+   * A property specifying the {@linkplain HttpClientBuilderFactory} used for the Solr internal
+   * communication.
+   */
+  private static final String HTTPCLIENT_BUILDER_FACTORY = "clientBuilderFactory";
+
+  /**
+   * A property specifying the default values for the configuration parameters specified by the
+   * {@linkplain #AUTH_CONFIG_NAMES_PROPERTY} property. The default values are specified as a
+   * collection of key-value pairs (i.e. property-name : default_value).
+   */
+  private static final String DEFAULT_AUTH_CONFIGS_PROPERTY = "defaultConfigs";
+
+  /**
+   * A property which enable (or disable) the delegation tokens functionality.
+   */
+  private static final String DELEGATION_TOKEN_ENABLED_PROPERTY = "enableDelegationToken";
+
+  /**
+   * A property which enables initialization of kerberos before connecting to Zookeeper.
+   */
+  private static final String INIT_KERBEROS_ZK = "initKerberosZk";
+
+  /**
+   * A property which configures proxy users for the underlying Hadoop authentication mechanism.
+   * This configuration is expressed as a collection of key-value pairs  (i.e. property-name : value).
+   */
+  public static final String PROXY_USER_CONFIGS = "proxyUserConfigs";
+
+  private AuthenticationFilter authFilter;
+  private HttpClientBuilderFactory factory = null;
+  private final CoreContainer coreContainer;
+
+  public GenericHadoopAuthPlugin(CoreContainer coreContainer) {
+    this.coreContainer = coreContainer;
+  }
+
+  @SuppressWarnings("rawtypes")
+  @Override
+  public void init(Map<String,Object> pluginConfig) {
+    try {
+      String delegationTokenEnabled = (String)pluginConfig.getOrDefault(DELEGATION_TOKEN_ENABLED_PROPERTY, "false");
+      authFilter = (Boolean.parseBoolean(delegationTokenEnabled)) ? new HadoopAuthFilter() : new AuthenticationFilter();
+
+      // Initialize kerberos before initializing curator instance.
+      boolean initKerberosZk = Boolean.parseBoolean((String)pluginConfig.getOrDefault(INIT_KERBEROS_ZK, "false"));
+      if (initKerberosZk) {
+        (new Krb5HttpClientBuilder()).getBuilder();
+      }
+
+      FilterConfig conf = getInitFilterConfig(pluginConfig);
+      authFilter.init(conf);
+
+      String httpClientBuilderFactory = (String)pluginConfig.get(HTTPCLIENT_BUILDER_FACTORY);
+      if (httpClientBuilderFactory != null) {
+        Class c = Class.forName(httpClientBuilderFactory);
+        factory = (HttpClientBuilderFactory)c.newInstance();
+      }
+
+    } catch (ServletException | ClassNotFoundException | InstantiationException | IllegalAccessException e) {
+      throw new SolrException(ErrorCode.SERVER_ERROR, "Error initializing kerberos authentication plugin: "+e);
+    }
+  }
+
+  @SuppressWarnings("unchecked")
+  protected FilterConfig getInitFilterConfig(Map<String, Object> pluginConfig) {
+    Map<String, String> params = new HashMap<>();
+
+    String type = (String) Objects.requireNonNull(pluginConfig.get(HADOOP_AUTH_TYPE));
+    params.put(HADOOP_AUTH_TYPE, type);
+
+    String sysPropPrefix = (String) pluginConfig.getOrDefault(SYSPROP_PREFIX_PROPERTY, "solr.");
+    Collection<String> authConfigNames = (Collection<String>) pluginConfig.
+        getOrDefault(AUTH_CONFIG_NAMES_PROPERTY, Collections.emptyList());
+    Map<String,String> authConfigDefaults = (Map<String,String>) pluginConfig
+        .getOrDefault(DEFAULT_AUTH_CONFIGS_PROPERTY, Collections.emptyMap());
+    Map<String,String> proxyUserConfigs = (Map<String,String>) pluginConfig
+        .getOrDefault(PROXY_USER_CONFIGS, Collections.emptyMap());
+
+    for ( String configName : authConfigNames) {
+      String systemProperty = sysPropPrefix + configName;
+      String defaultConfigVal = authConfigDefaults.get(configName);
+      String configVal = System.getProperty(systemProperty, defaultConfigVal);
+      if (configVal != null) {
+        params.put(configName, configVal);
+      }
+    }
+
+    // Configure proxy user settings.
+    params.putAll(proxyUserConfigs);
+
+    final ServletContext servletContext = new AttributeOnlyServletContext();
+    log.info("Params: "+params);
+
+    ZkController controller = coreContainer.getZkController();
+    if (controller != null) {
+      servletContext.setAttribute(DELEGATION_TOKEN_ZK_CLIENT, controller.getZkClient());
+    }
+
+    FilterConfig conf = new FilterConfig() {
+      @Override
+      public ServletContext getServletContext() {
+        return servletContext;
+      }
+
+      @Override
+      public Enumeration<String> getInitParameterNames() {
+        return new IteratorEnumeration(params.keySet().iterator());
+      }
+
+      @Override
+      public String getInitParameter(String param) {
+        return params.get(param);
+      }
+
+      @Override
+      public String getFilterName() {
+        return "HadoopAuthFilter";
+      }
+    };
+
+    return conf;
+  }
+
+  @Override
+  public boolean doAuthenticate(ServletRequest request, ServletResponse response, FilterChain filterChain)
+      throws Exception {
+    final HttpServletResponse frsp = (HttpServletResponse)response;
+
+    // Workaround until HADOOP-13346 is fixed.
+    HttpServletResponse rspCloseShield = new HttpServletResponseWrapper(frsp) {
+      @SuppressForbidden(reason = "Hadoop DelegationTokenAuthenticationFilter uses response writer, this" +
+          "is providing a CloseShield on top of that")
+      @Override
+      public PrintWriter getWriter() throws IOException {
+        final PrintWriter pw = new PrintWriterWrapper(frsp.getWriter()) {
+          @Override
+          public void close() {};
+        };
+        return pw;
+      }
+    };
+    authFilter.doFilter(request, rspCloseShield, filterChain);
+
+    if (authFilter instanceof HadoopAuthFilter) { // delegation token mgmt.
+      String requestContinuesAttr = (String)request.getAttribute(REQUEST_CONTINUES_ATTR);
+      if (requestContinuesAttr == null) {
+        log.warn("Could not find " + REQUEST_CONTINUES_ATTR);
+        return false;
+      } else {
+        return Boolean.parseBoolean(requestContinuesAttr);
+      }
+    }
+
+    return true;
+  }
+
+  @Override
+  public SolrHttpClientBuilder getHttpClientBuilder(SolrHttpClientBuilder builder) {
+    return (factory != null) ? factory.getHttpClientBuilder(Optional.ofNullable(builder)) : builder;
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (authFilter != null) {
+      authFilter.destroy();
+    }
+    if (factory != null) {
+      factory.close();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a1a8b286/solr/core/src/java/org/apache/solr/security/HadoopAuthFilter.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/security/HadoopAuthFilter.java b/solr/core/src/java/org/apache/solr/security/HadoopAuthFilter.java
new file mode 100644
index 0000000..fb35e72
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/security/HadoopAuthFilter.java
@@ -0,0 +1,198 @@
+/*
+ * 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.solr.security;
+
+import java.io.IOException;
+import java.util.LinkedList;
+import java.util.List;
+
+import javax.servlet.FilterChain;
+import javax.servlet.FilterConfig;
+import javax.servlet.ServletException;
+import javax.servlet.ServletRequest;
+import javax.servlet.ServletResponse;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletRequestWrapper;
+
+import org.apache.curator.RetryPolicy;
+import org.apache.curator.framework.AuthInfo;
+import org.apache.curator.framework.CuratorFramework;
+import org.apache.curator.framework.CuratorFrameworkFactory;
+import org.apache.curator.framework.api.ACLProvider;
+import org.apache.curator.retry.ExponentialBackoffRetry;
+
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.authentication.server.AuthenticationHandler;
+import org.apache.hadoop.security.token.delegation.web.DelegationTokenAuthenticationFilter;
+import org.apache.hadoop.security.token.delegation.web.HttpUserGroupInformation;
+import org.apache.solr.common.cloud.SecurityAwareZkACLProvider;
+import org.apache.solr.common.cloud.SolrZkClient;
+import org.apache.solr.common.cloud.ZkACLProvider;
+import org.apache.solr.common.cloud.ZkCredentialsProvider;
+import org.apache.zookeeper.data.ACL;
+
+/**
+ * This is an authentication filter based on Hadoop's {@link DelegationTokenAuthenticationFilter}.
+ */
+public class HadoopAuthFilter extends DelegationTokenAuthenticationFilter {
+  /**
+   * This property defines the configuration parameter storing the Solr zookeeper client ref
+   * in the servlet filter config.
+   */
+  static final String DELEGATION_TOKEN_ZK_CLIENT = "solr.kerberos.delegation.token.zk.client";
+
+  private CuratorFramework curatorFramework;
+
+  @Override
+  public void init(FilterConfig conf) throws ServletException {
+    if (conf != null && "zookeeper".equals(conf.getInitParameter("signer.secret.provider"))) {
+      SolrZkClient zkClient =
+          (SolrZkClient)conf.getServletContext().getAttribute(DELEGATION_TOKEN_ZK_CLIENT);
+      conf.getServletContext().setAttribute("signer.secret.provider.zookeeper.curator.client",
+          getCuratorClient(zkClient));
+    }
+    super.init(conf);
+  }
+
+  @Override
+  public void doFilter(ServletRequest request, ServletResponse response,
+      FilterChain filterChain) throws IOException, ServletException {
+    // HttpClient 4.4.x throws NPE if query string is null and parsed through URLEncodedUtils.
+    // See HTTPCLIENT-1746 and HADOOP-12767
+    HttpServletRequest httpRequest = (HttpServletRequest)request;
+    String queryString = httpRequest.getQueryString();
+    final String nonNullQueryString = queryString == null ? "" : queryString;
+    HttpServletRequest requestNonNullQueryString = new HttpServletRequestWrapper(httpRequest){
+      @Override
+      public String getQueryString() {
+        return nonNullQueryString;
+      }
+    };
+
+    // include Impersonator User Name in case someone (e.g. logger) wants it
+    FilterChain filterChainWrapper = new FilterChain() {
+      @Override
+      public void doFilter(ServletRequest servletRequest, ServletResponse servletResponse)
+          throws IOException, ServletException {
+        HttpServletRequest httpRequest = (HttpServletRequest) servletRequest;
+
+        UserGroupInformation ugi = HttpUserGroupInformation.get();
+        if (ugi != null && ugi.getAuthenticationMethod() == UserGroupInformation.AuthenticationMethod.PROXY) {
+          UserGroupInformation realUserUgi = ugi.getRealUser();
+          if (realUserUgi != null) {
+            httpRequest.setAttribute(KerberosPlugin.IMPERSONATOR_USER_NAME, realUserUgi.getShortUserName());
+          }
+        }
+        filterChain.doFilter(servletRequest, servletResponse);
+      }
+    };
+
+    super.doFilter(requestNonNullQueryString, response, filterChainWrapper);
+  }
+
+  @Override
+  public void destroy() {
+    super.destroy();
+    if (curatorFramework != null) {
+      curatorFramework.close();
+    }
+    curatorFramework = null;
+  }
+
+  @Override
+  protected void initializeAuthHandler(String authHandlerClassName,
+                                       FilterConfig filterConfig) throws ServletException {
+    // set the internal authentication handler in order to record whether the request should continue
+    super.initializeAuthHandler(authHandlerClassName, filterConfig);
+    AuthenticationHandler authHandler = getAuthenticationHandler();
+    super.initializeAuthHandler(RequestContinuesRecorderAuthenticationHandler.class.getName(), filterConfig);
+    RequestContinuesRecorderAuthenticationHandler newAuthHandler =
+        (RequestContinuesRecorderAuthenticationHandler)getAuthenticationHandler();
+    newAuthHandler.setAuthHandler(authHandler);
+  }
+
+  protected CuratorFramework getCuratorClient(SolrZkClient zkClient) {
+    // should we try to build a RetryPolicy off of the ZkController?
+    RetryPolicy retryPolicy = new ExponentialBackoffRetry(1000, 3);
+    if (zkClient == null) {
+      throw new IllegalArgumentException("zkClient required");
+    }
+    String zkHost = zkClient.getZkServerAddress();
+    String zkChroot = zkHost.contains("/")? zkHost.substring(zkHost.indexOf("/")): "";
+    String zkNamespace = zkChroot + SecurityAwareZkACLProvider.SECURITY_ZNODE_PATH;
+    zkNamespace = zkNamespace.startsWith("/") ? zkNamespace.substring(1) : zkNamespace;
+    String zkConnectionString = zkHost.contains("/")? zkHost.substring(0, zkHost.indexOf("/")): zkHost;
+    SolrZkToCuratorCredentialsACLs curatorToSolrZk = new SolrZkToCuratorCredentialsACLs(zkClient);
+    final int connectionTimeoutMs = 30000; // this value is currently hard coded, see SOLR-7561.
+
+    curatorFramework = CuratorFrameworkFactory.builder()
+        .namespace(zkNamespace)
+        .connectString(zkConnectionString)
+        .retryPolicy(retryPolicy)
+        .aclProvider(curatorToSolrZk.getACLProvider())
+        .authorization(curatorToSolrZk.getAuthInfos())
+        .sessionTimeoutMs(zkClient.getZkClientTimeout())
+        .connectionTimeoutMs(connectionTimeoutMs)
+        .build();
+    curatorFramework.start();
+    return curatorFramework;
+  }
+
+  /**
+   * Convert Solr Zk Credentials/ACLs to Curator versions
+   */
+  protected static class SolrZkToCuratorCredentialsACLs {
+    private final ACLProvider aclProvider;
+    private final List<AuthInfo> authInfos;
+
+    public SolrZkToCuratorCredentialsACLs(SolrZkClient zkClient) {
+      this.aclProvider = createACLProvider(zkClient);
+      this.authInfos = createAuthInfo(zkClient);
+    }
+
+    public ACLProvider getACLProvider() { return aclProvider; }
+    public List<AuthInfo> getAuthInfos() { return authInfos; }
+
+    private ACLProvider createACLProvider(SolrZkClient zkClient) {
+      final ZkACLProvider zkACLProvider = zkClient.getZkACLProvider();
+      return new ACLProvider() {
+        @Override
+        public List<ACL> getDefaultAcl() {
+          return zkACLProvider.getACLsToAdd(null);
+        }
+
+        @Override
+        public List<ACL> getAclForPath(String path) {
+           List<ACL> acls = zkACLProvider.getACLsToAdd(path);
+           return acls;
+        }
+      };
+    }
+
+    private List<AuthInfo> createAuthInfo(SolrZkClient zkClient) {
+      List<AuthInfo> ret = new LinkedList<AuthInfo>();
+
+      // In theory the credentials to add could change here if zookeeper hasn't been initialized
+      ZkCredentialsProvider credentialsProvider =
+        zkClient.getZkClientConnectionStrategy().getZkCredentialsToAddAutomatically();
+      for (ZkCredentialsProvider.ZkCredentials zkCredentials : credentialsProvider.getCredentials()) {
+        ret.add(new AuthInfo(zkCredentials.getScheme(), zkCredentials.getAuth()));
+      }
+      return ret;
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a1a8b286/solr/core/src/java/org/apache/solr/security/HadoopAuthPlugin.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/security/HadoopAuthPlugin.java b/solr/core/src/java/org/apache/solr/security/HadoopAuthPlugin.java
new file mode 100644
index 0000000..db0f639
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/security/HadoopAuthPlugin.java
@@ -0,0 +1,241 @@
+/*
+ * 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.solr.security;
+
+import static org.apache.solr.security.RequestContinuesRecorderAuthenticationHandler.REQUEST_CONTINUES_ATTR;
+import static org.apache.solr.security.HadoopAuthFilter.DELEGATION_TOKEN_ZK_CLIENT;
+
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.lang.invoke.MethodHandles;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Enumeration;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+import javax.servlet.FilterChain;
+import javax.servlet.FilterConfig;
+import javax.servlet.ServletContext;
+import javax.servlet.ServletException;
+import javax.servlet.ServletRequest;
+import javax.servlet.ServletResponse;
+import javax.servlet.http.HttpServletResponse;
+import javax.servlet.http.HttpServletResponseWrapper;
+
+import org.apache.commons.collections.iterators.IteratorEnumeration;
+import org.apache.hadoop.security.authentication.server.AuthenticationFilter;
+import org.apache.solr.client.solrj.impl.Krb5HttpClientBuilder;
+import org.apache.solr.cloud.ZkController;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.SolrException.ErrorCode;
+import org.apache.solr.common.util.SuppressForbidden;
+import org.apache.solr.core.CoreContainer;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This class implements a generic plugin which can use authentication schemes exposed by the
+ * Hadoop framework. This plugin supports following features
+ * - integration with authentication mehcanisms (e.g. kerberos)
+ * - Delegation token support
+ * - Proxy users (or secure impersonation) support
+ *
+ * This plugin enables defining configuration parameters required by the undelying Hadoop authentication
+ * mechanism. These configuration parameters can either be specified as a Java system property or the default
+ * value can be specified as part of the plugin configuration.
+ *
+ * The proxy users are configured by specifying relevant Hadoop configuration parameters. Please note that
+ * the delegation token support must be enabled for using the proxy users support.
+ *
+ * Note - this class does not support configuring authentication mechanism for Solr internal communication.
+ * For this purpose {@linkplain ConfigurableInternodeAuthHadoopPlugin} should be used. If this plugin is used in the
+ * SolrCloud mode, it will use PKI based authentication mechanism for Solr internal communication.
+ **/
+public class HadoopAuthPlugin extends AuthenticationPlugin {
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  /**
+   * A property specifying the type of authentication scheme to be configured.
+   */
+  private static final String HADOOP_AUTH_TYPE = "type";
+
+  /**
+   * A property specifies the value of the prefix to be used to define Java system property
+   * for configuring the authentication mechanism. The name of the Java system property is
+   * defined by appending the configuration parmeter namne to this prefix value e.g. if prefix
+   * is 'solr' then the Java system property 'solr.kerberos.principal' defines the value of
+   * configuration parameter 'kerberos.principal'.
+   */
+  private static final String SYSPROP_PREFIX_PROPERTY = "sysPropPrefix";
+
+  /**
+   * A property specifying the configuration parameters required by the authentication scheme
+   * defined by {@linkplain #HADOOP_AUTH_TYPE} property.
+   */
+  private static final String AUTH_CONFIG_NAMES_PROPERTY = "authConfigs";
+
+  /**
+   * A property specifying the default values for the configuration parameters specified by the
+   * {@linkplain #AUTH_CONFIG_NAMES_PROPERTY} property. The default values are specified as a
+   * collection of key-value pairs (i.e. property-name : default_value).
+   */
+  private static final String DEFAULT_AUTH_CONFIGS_PROPERTY = "defaultConfigs";
+
+  /**
+   * A property which enable (or disable) the delegation tokens functionality.
+   */
+  private static final String DELEGATION_TOKEN_ENABLED_PROPERTY = "enableDelegationToken";
+
+  /**
+   * A property which enables initialization of kerberos before connecting to Zookeeper.
+   */
+  private static final String INIT_KERBEROS_ZK = "initKerberosZk";
+
+  /**
+   * A property which configures proxy users for the underlying Hadoop authentication mechanism.
+   * This configuration is expressed as a collection of key-value pairs  (i.e. property-name : value).
+   */
+  public static final String PROXY_USER_CONFIGS = "proxyUserConfigs";
+
+  private AuthenticationFilter authFilter;
+  protected final CoreContainer coreContainer;
+
+  public HadoopAuthPlugin(CoreContainer coreContainer) {
+    this.coreContainer = coreContainer;
+  }
+
+  @Override
+  public void init(Map<String,Object> pluginConfig) {
+    try {
+      String delegationTokenEnabled = (String)pluginConfig.getOrDefault(DELEGATION_TOKEN_ENABLED_PROPERTY, "false");
+      authFilter = (Boolean.parseBoolean(delegationTokenEnabled)) ? new HadoopAuthFilter() : new AuthenticationFilter();
+
+      // Initialize kerberos before initializing curator instance.
+      boolean initKerberosZk = Boolean.parseBoolean((String)pluginConfig.getOrDefault(INIT_KERBEROS_ZK, "false"));
+      if (initKerberosZk) {
+        (new Krb5HttpClientBuilder()).getBuilder();
+      }
+
+      FilterConfig conf = getInitFilterConfig(pluginConfig);
+      authFilter.init(conf);
+
+    } catch (ServletException e) {
+      throw new SolrException(ErrorCode.SERVER_ERROR, "Error initializing GenericHadoopAuthPlugin: "+e);
+    }
+  }
+
+  @SuppressWarnings("unchecked")
+  protected FilterConfig getInitFilterConfig(Map<String, Object> pluginConfig) {
+    Map<String, String> params = new HashMap<>();
+
+    String type = (String) Objects.requireNonNull(pluginConfig.get(HADOOP_AUTH_TYPE));
+    params.put(HADOOP_AUTH_TYPE, type);
+
+    String sysPropPrefix = (String) pluginConfig.getOrDefault(SYSPROP_PREFIX_PROPERTY, "solr.");
+    Collection<String> authConfigNames = (Collection<String>) pluginConfig.
+        getOrDefault(AUTH_CONFIG_NAMES_PROPERTY, Collections.emptyList());
+    Map<String,String> authConfigDefaults = (Map<String,String>) pluginConfig
+        .getOrDefault(DEFAULT_AUTH_CONFIGS_PROPERTY, Collections.emptyMap());
+    Map<String,String> proxyUserConfigs = (Map<String,String>) pluginConfig
+        .getOrDefault(PROXY_USER_CONFIGS, Collections.emptyMap());
+
+    for ( String configName : authConfigNames) {
+      String systemProperty = sysPropPrefix + configName;
+      String defaultConfigVal = authConfigDefaults.get(configName);
+      String configVal = System.getProperty(systemProperty, defaultConfigVal);
+      if (configVal != null) {
+        params.put(configName, configVal);
+      }
+    }
+
+    // Configure proxy user settings.
+    params.putAll(proxyUserConfigs);
+
+    final ServletContext servletContext = new AttributeOnlyServletContext();
+    log.info("Params: "+params);
+
+    ZkController controller = coreContainer.getZkController();
+    if (controller != null) {
+      servletContext.setAttribute(DELEGATION_TOKEN_ZK_CLIENT, controller.getZkClient());
+    }
+
+    FilterConfig conf = new FilterConfig() {
+      @Override
+      public ServletContext getServletContext() {
+        return servletContext;
+      }
+
+      @Override
+      public Enumeration<String> getInitParameterNames() {
+        return new IteratorEnumeration(params.keySet().iterator());
+      }
+
+      @Override
+      public String getInitParameter(String param) {
+        return params.get(param);
+      }
+
+      @Override
+      public String getFilterName() {
+        return "HadoopAuthFilter";
+      }
+    };
+
+    return conf;
+  }
+
+  @Override
+  public boolean doAuthenticate(ServletRequest request, ServletResponse response, FilterChain filterChain)
+      throws Exception {
+    final HttpServletResponse frsp = (HttpServletResponse)response;
+
+    // Workaround until HADOOP-13346 is fixed.
+    HttpServletResponse rspCloseShield = new HttpServletResponseWrapper(frsp) {
+      @SuppressForbidden(reason = "Hadoop DelegationTokenAuthenticationFilter uses response writer, this" +
+          "is providing a CloseShield on top of that")
+      @Override
+      public PrintWriter getWriter() throws IOException {
+        final PrintWriter pw = new PrintWriterWrapper(frsp.getWriter()) {
+          @Override
+          public void close() {};
+        };
+        return pw;
+      }
+    };
+    authFilter.doFilter(request, rspCloseShield, filterChain);
+
+    if (authFilter instanceof HadoopAuthFilter) { // delegation token mgmt.
+      String requestContinuesAttr = (String)request.getAttribute(REQUEST_CONTINUES_ATTR);
+      if (requestContinuesAttr == null) {
+        log.warn("Could not find " + REQUEST_CONTINUES_ATTR);
+        return false;
+      } else {
+        return Boolean.parseBoolean(requestContinuesAttr);
+      }
+    }
+
+    return true;
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (authFilter != null) {
+      authFilter.destroy();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a1a8b286/solr/core/src/java/org/apache/solr/security/KerberosFilter.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/security/KerberosFilter.java b/solr/core/src/java/org/apache/solr/security/KerberosFilter.java
index 9c53050..d725d09 100644
--- a/solr/core/src/java/org/apache/solr/security/KerberosFilter.java
+++ b/solr/core/src/java/org/apache/solr/security/KerberosFilter.java
@@ -42,9 +42,9 @@ public class KerberosFilter extends AuthenticationFilter {
     super.initializeAuthHandler(authHandlerClassName, filterConfig);
     AuthenticationHandler authHandler = getAuthenticationHandler();
     super.initializeAuthHandler(
-        KerberosPlugin.RequestContinuesRecorderAuthenticationHandler.class.getName(), filterConfig);
-    KerberosPlugin.RequestContinuesRecorderAuthenticationHandler newAuthHandler =
-        (KerberosPlugin.RequestContinuesRecorderAuthenticationHandler)getAuthenticationHandler();
+        RequestContinuesRecorderAuthenticationHandler.class.getName(), filterConfig);
+    RequestContinuesRecorderAuthenticationHandler newAuthHandler =
+        (RequestContinuesRecorderAuthenticationHandler)getAuthenticationHandler();
     newAuthHandler.setAuthHandler(authHandler);
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a1a8b286/solr/core/src/java/org/apache/solr/security/KerberosPlugin.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/security/KerberosPlugin.java b/solr/core/src/java/org/apache/solr/security/KerberosPlugin.java
index 3655ac9..42d22ca 100644
--- a/solr/core/src/java/org/apache/solr/security/KerberosPlugin.java
+++ b/solr/core/src/java/org/apache/solr/security/KerberosPlugin.java
@@ -17,43 +17,24 @@
 package org.apache.solr.security;
 
 import java.io.IOException;
-import java.io.InputStream;
 import java.io.PrintWriter;
 import java.lang.invoke.MethodHandles;
-import java.net.MalformedURLException;
-import java.net.URL;
-import java.util.Collections;
 import java.util.Enumeration;
-import java.util.EventListener;
 import java.util.HashMap;
 import java.util.Map;
-import java.util.Properties;
-import java.util.Set;
 
 import javax.servlet.Filter;
 import javax.servlet.FilterChain;
 import javax.servlet.FilterConfig;
-import javax.servlet.FilterRegistration;
-import javax.servlet.FilterRegistration.Dynamic;
-import javax.servlet.RequestDispatcher;
-import javax.servlet.Servlet;
 import javax.servlet.ServletContext;
 import javax.servlet.ServletException;
-import javax.servlet.ServletRegistration;
 import javax.servlet.ServletRequest;
 import javax.servlet.ServletResponse;
-import javax.servlet.SessionCookieConfig;
-import javax.servlet.SessionTrackingMode;
-import javax.servlet.descriptor.JspConfigDescriptor;
-import javax.servlet.http.HttpServletRequest;
 import javax.servlet.http.HttpServletResponse;
 import javax.servlet.http.HttpServletResponseWrapper;
 
 import com.google.common.annotations.VisibleForTesting;
 import org.apache.commons.collections.iterators.IteratorEnumeration;
-import org.apache.hadoop.security.authentication.client.AuthenticationException;
-import org.apache.hadoop.security.authentication.server.AuthenticationHandler;
-import org.apache.hadoop.security.authentication.server.AuthenticationToken;
 import org.apache.solr.client.solrj.impl.Krb5HttpClientBuilder;
 import org.apache.solr.client.solrj.impl.SolrHttpClientBuilder;
 import org.apache.solr.cloud.ZkController;
@@ -92,9 +73,6 @@ public class KerberosPlugin extends AuthenticationPlugin implements HttpClientBu
   public static final String IMPERSONATOR_DO_AS_HTTP_PARAM = "doAs";
   public static final String IMPERSONATOR_USER_NAME = "solr.impersonator.user.name";
 
-  // filled in by Plugin/Filter
-  static final String REQUEST_CONTINUES_ATTR =
-      "org.apache.solr.security.kerberosplugin.requestcontinues";
   static final String DELEGATION_TOKEN_ZK_CLIENT =
       "solr.kerberos.delegation.token.zk.client";
 
@@ -263,9 +241,9 @@ public class KerberosPlugin extends AuthenticationPlugin implements HttpClientBu
       }
     };
     kerberosFilter.doFilter(req, rspCloseShield, chain);
-    String requestContinuesAttr = (String)req.getAttribute(REQUEST_CONTINUES_ATTR);
+    String requestContinuesAttr = (String)req.getAttribute(RequestContinuesRecorderAuthenticationHandler.REQUEST_CONTINUES_ATTR);
     if (requestContinuesAttr == null) {
-      log.warn("Could not find " + REQUEST_CONTINUES_ATTR);
+      log.warn("Could not find " + RequestContinuesRecorderAuthenticationHandler.REQUEST_CONTINUES_ATTR);
       return false;
     } else {
       return Boolean.parseBoolean(requestContinuesAttr);
@@ -286,292 +264,4 @@ public class KerberosPlugin extends AuthenticationPlugin implements HttpClientBu
   protected Filter getKerberosFilter() { return kerberosFilter; }
 
   protected void setKerberosFilter(Filter kerberosFilter) { this.kerberosFilter = kerberosFilter; }
-
-  protected static class AttributeOnlyServletContext implements ServletContext {
-    private Map<String, Object> attributes = new HashMap<String, Object>();
-
-    @Override
-    public void setSessionTrackingModes(Set<SessionTrackingMode> sessionTrackingModes) {}
-    
-    @Override
-    public boolean setInitParameter(String name, String value) {
-      return false;
-    }
-
-    @Override
-    public void setAttribute(String name, Object object) {
-      attributes.put(name, object);
-    }
-
-    @Override
-    public void removeAttribute(String name) {
-      attributes.remove(name);
-    }
-    
-    @Override
-    public void log(String message, Throwable throwable) {}
-    
-    @Override
-    public void log(Exception exception, String msg) {}
-    
-    @Override
-    public void log(String msg) {}
-    
-    @Override
-    public String getVirtualServerName() {
-      return null;
-    }
-    
-    @Override
-    public SessionCookieConfig getSessionCookieConfig() {
-      return null;
-    }
-    
-    @Override
-    public Enumeration<Servlet> getServlets() {
-      return null;
-    }
-    
-    @Override
-    public Map<String,? extends ServletRegistration> getServletRegistrations() {
-      return null;
-    }
-    
-    @Override
-    public ServletRegistration getServletRegistration(String servletName) {
-      return null;
-    }
-    
-    @Override
-    public Enumeration<String> getServletNames() {
-      return null;
-    }
-    
-    @Override
-    public String getServletContextName() {
-      return null;
-    }
-    
-    @Override
-    public Servlet getServlet(String name) throws ServletException {
-      return null;
-    }
-    
-    @Override
-    public String getServerInfo() {
-      return null;
-    }
-    
-    @Override
-    public Set<String> getResourcePaths(String path) {
-      return null;
-    }
-    
-    @Override
-    public InputStream getResourceAsStream(String path) {
-      return null;
-    }
-    
-    @Override
-    public URL getResource(String path) throws MalformedURLException {
-      return null;
-    }
-    
-    @Override
-    public RequestDispatcher getRequestDispatcher(String path) {
-      return null;
-    }
-    
-    @Override
-    public String getRealPath(String path) {
-      return null;
-    }
-    
-    @Override
-    public RequestDispatcher getNamedDispatcher(String name) {
-      return null;
-    }
-    
-    @Override
-    public int getMinorVersion() {
-      return 0;
-    }
-    
-    @Override
-    public String getMimeType(String file) {
-      return null;
-    }
-    
-    @Override
-    public int getMajorVersion() {
-      return 0;
-    }
-    
-    @Override
-    public JspConfigDescriptor getJspConfigDescriptor() {
-      return null;
-    }
-    
-    @Override
-    public Enumeration<String> getInitParameterNames() {
-      return null;
-    }
-    
-    @Override
-    public String getInitParameter(String name) {
-      return null;
-    }
-    
-    @Override
-    public Map<String,? extends FilterRegistration> getFilterRegistrations() {
-      return null;
-    }
-    
-    @Override
-    public FilterRegistration getFilterRegistration(String filterName) {
-      return null;
-    }
-    
-    @Override
-    public Set<SessionTrackingMode> getEffectiveSessionTrackingModes() {
-      return null;
-    }
-    
-    @Override
-    public int getEffectiveMinorVersion() {
-      return 0;
-    }
-    
-    @Override
-    public int getEffectiveMajorVersion() {
-      return 0;
-    }
-    
-    @Override
-    public Set<SessionTrackingMode> getDefaultSessionTrackingModes() {
-      return null;
-    }
-    
-    @Override
-    public String getContextPath() {
-      return null;
-    }
-    
-    @Override
-    public ServletContext getContext(String uripath) {
-      return null;
-    }
-    
-    @Override
-    public ClassLoader getClassLoader() {
-      return null;
-    }
-
-    @Override
-    public Enumeration<String> getAttributeNames() {
-      return Collections.enumeration(attributes.keySet());
-    }
-
-    @Override
-    public Object getAttribute(String name) {
-      return attributes.get(name);
-    }
-    
-    @Override
-    public void declareRoles(String... roleNames) {}
-    
-    @Override
-    public <T extends Servlet> T createServlet(Class<T> clazz) throws ServletException {
-      return null;
-    }
-    
-    @Override
-    public <T extends EventListener> T createListener(Class<T> clazz) throws ServletException {
-      return null;
-    }
-    
-    @Override
-    public <T extends Filter> T createFilter(Class<T> clazz) throws ServletException {
-      return null;
-    }
-    
-    @Override
-    public javax.servlet.ServletRegistration.Dynamic addServlet(String servletName, Class<? extends Servlet> servletClass) {
-      return null;
-    }
-    
-    @Override
-    public javax.servlet.ServletRegistration.Dynamic addServlet(String servletName, Servlet servlet) {
-      return null;
-    }
-    
-    @Override
-    public javax.servlet.ServletRegistration.Dynamic addServlet(String servletName, String className) {
-      return null;
-    }
-    
-    @Override
-    public void addListener(Class<? extends EventListener> listenerClass) {}
-    
-    @Override
-    public <T extends EventListener> void addListener(T t) {}
-    
-    @Override
-    public void addListener(String className) {}
-    
-    @Override
-    public Dynamic addFilter(String filterName, Class<? extends Filter> filterClass) {
-      return null;
-    }
-    
-    @Override
-    public Dynamic addFilter(String filterName, Filter filter) {
-      return null;
-    }
-    
-    @Override
-    public Dynamic addFilter(String filterName, String className) {
-      return null;
-    }
-  };
-
-  /*
-   * {@link AuthenticationHandler} that delegates to another {@link AuthenticationHandler}
-   * and records the response of managementOperation (which indicates whether the request
-   * should continue or not).
-   */
-  public static class RequestContinuesRecorderAuthenticationHandler implements AuthenticationHandler {
-    private AuthenticationHandler authHandler;
-
-    public void setAuthHandler(AuthenticationHandler authHandler) {
-      this.authHandler = authHandler;
-    }
-
-    public String getType() {
-      return authHandler.getType();
-    }
-
-    public void init(Properties config) throws ServletException {
-      // authHandler has already been init'ed, nothing to do here
-    }
-
-    public void destroy() {
-      authHandler.destroy();
-    }
-
-    public boolean managementOperation(AuthenticationToken token,
-                                       HttpServletRequest request,
-                                       HttpServletResponse response)
-        throws IOException, AuthenticationException {
-      boolean result = authHandler.managementOperation(token, request, response);
-      request.setAttribute(KerberosPlugin.REQUEST_CONTINUES_ATTR, new Boolean(result).toString());
-      return result;
-    }
-
-
-    public AuthenticationToken authenticate(HttpServletRequest request, HttpServletResponse response)
-        throws IOException, AuthenticationException {
-      return authHandler.authenticate(request, response);
-    }
-  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a1a8b286/solr/core/src/java/org/apache/solr/security/RequestContinuesRecorderAuthenticationHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/security/RequestContinuesRecorderAuthenticationHandler.java b/solr/core/src/java/org/apache/solr/security/RequestContinuesRecorderAuthenticationHandler.java
new file mode 100644
index 0000000..95591ca
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/security/RequestContinuesRecorderAuthenticationHandler.java
@@ -0,0 +1,71 @@
+/*
+ * 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.solr.security;
+
+import java.io.IOException;
+import java.util.Properties;
+
+import javax.servlet.ServletException;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+
+import org.apache.hadoop.security.authentication.client.AuthenticationException;
+import org.apache.hadoop.security.authentication.server.AuthenticationHandler;
+import org.apache.hadoop.security.authentication.server.AuthenticationToken;
+
+/*
+ * {@link AuthenticationHandler} that delegates to another {@link AuthenticationHandler}
+ * and records the response of managementOperation (which indicates whether the request
+ * should continue or not).
+ */
+public class RequestContinuesRecorderAuthenticationHandler implements AuthenticationHandler {
+  // filled in by Plugin/Filter
+  static final String REQUEST_CONTINUES_ATTR =
+      "org.apache.solr.security.authentication.requestcontinues";
+
+  private AuthenticationHandler authHandler;
+
+  public void setAuthHandler(AuthenticationHandler authHandler) {
+    this.authHandler = authHandler;
+  }
+
+  public String getType() {
+    return authHandler.getType();
+  }
+
+  public void init(Properties config) throws ServletException {
+    // authHandler has already been init'ed, nothing to do here
+  }
+
+  public void destroy() {
+    authHandler.destroy();
+  }
+
+  public boolean managementOperation(AuthenticationToken token,
+                                     HttpServletRequest request,
+                                     HttpServletResponse response)
+      throws IOException, AuthenticationException {
+    boolean result = authHandler.managementOperation(token, request, response);
+    request.setAttribute(RequestContinuesRecorderAuthenticationHandler.REQUEST_CONTINUES_ATTR, new Boolean(result).toString());
+    return result;
+  }
+
+  public AuthenticationToken authenticate(HttpServletRequest request, HttpServletResponse response)
+      throws IOException, AuthenticationException {
+    return authHandler.authenticate(request, response);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a1a8b286/solr/core/src/test-files/solr/security/hadoop_kerberos_config.json
----------------------------------------------------------------------
diff --git a/solr/core/src/test-files/solr/security/hadoop_kerberos_config.json b/solr/core/src/test-files/solr/security/hadoop_kerberos_config.json
new file mode 100644
index 0000000..679474e
--- /dev/null
+++ b/solr/core/src/test-files/solr/security/hadoop_kerberos_config.json
@@ -0,0 +1,16 @@
+{
+    "authentication": {
+        "class": "org.apache.solr.security.ConfigurableInternodeAuthHadoopPlugin",
+        "sysPropPrefix": "solr.",
+        "type": "kerberos",
+        "clientBuilderFactory": "org.apache.solr.client.solrj.impl.Krb5HttpClientBuilder",
+        "initKerberosZk": "true",
+        "authConfigs": [
+            "kerberos.principal",
+            "kerberos.keytab",
+            "kerberos.name.rules"
+        ],
+        "defaultConfigs": {
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a1a8b286/solr/core/src/test-files/solr/security/hadoop_simple_auth_with_delegation.json
----------------------------------------------------------------------
diff --git a/solr/core/src/test-files/solr/security/hadoop_simple_auth_with_delegation.json b/solr/core/src/test-files/solr/security/hadoop_simple_auth_with_delegation.json
new file mode 100644
index 0000000..2248329
--- /dev/null
+++ b/solr/core/src/test-files/solr/security/hadoop_simple_auth_with_delegation.json
@@ -0,0 +1,29 @@
+{
+    "authentication": {
+        "class": "org.apache.solr.security.HadoopAuthPlugin",
+        "sysPropPrefix": "solr.",
+        "type": "simple",
+        "enableDelegationToken":"true",
+        "authConfigs": [
+            "delegation-token.token-kind",
+            "delegation-token.update-interval.sec",
+            "delegation-token.max-lifetime.sec",
+            "delegation-token.renewal-interval.sec",
+            "delegation-token.removal-scan-interval.sec",
+            "cookie.domain",
+            "signer.secret.provider",
+            "zk-dt-secret-manager.enable",
+            "zk-dt-secret-manager.znodeWorkingPath",
+            "signer.secret.provider.zookeeper.path"
+        ],
+        "defaultConfigs": {
+            "delegation-token.token-kind": "solr-dt",
+            "signer.secret.provider": "zookeeper",
+            "zk-dt-secret-manager.enable": "true",
+            "token.validity": "36000",
+            "zk-dt-secret-manager.znodeWorkingPath": "solr/security/zkdtsm",
+            "signer.secret.provider.zookeeper.path": "/token",
+            "cookie.domain": "127.0.0.1"
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a1a8b286/solr/core/src/test/org/apache/solr/cloud/TestSolrCloudWithSecureImpersonation.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestSolrCloudWithSecureImpersonation.java b/solr/core/src/test/org/apache/solr/cloud/TestSolrCloudWithSecureImpersonation.java
index 53b7f5b..d2f7873 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestSolrCloudWithSecureImpersonation.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestSolrCloudWithSecureImpersonation.java
@@ -108,11 +108,9 @@ public class TestSolrCloudWithSecureImpersonation extends SolrTestCaseJ4 {
     System.setProperty("solr.test.sys.prop2", "proptwo");
 
     SolrRequestParsers.DEFAULT.setAddRequestHeadersToContext(true);
-    String solrXml = MiniSolrCloudCluster.DEFAULT_CLOUD_SOLR_XML.replace("</solr>",
-        " <str name=\"collectionsHandler\">" + ImpersonatorCollectionsHandler.class.getName() + "</str>\n" +
-            "</solr>");
+    System.setProperty("collectionsHandler", ImpersonatorCollectionsHandler.class.getName());
 
-    miniCluster = new MiniSolrCloudCluster(NUM_SERVERS, createTempDir(), solrXml, buildJettyConfig("/solr"));
+    miniCluster = new MiniSolrCloudCluster(NUM_SERVERS, createTempDir(), buildJettyConfig("/solr"));
     JettySolrRunner runner = miniCluster.getJettySolrRunners().get(0);
     solrClient = new HttpSolrClient.Builder(runner.getBaseUrl().toString()).build();
   }
@@ -168,6 +166,8 @@ public class TestSolrCloudWithSecureImpersonation extends SolrTestCaseJ4 {
     }
     System.clearProperty("solr.test.sys.prop1");
     System.clearProperty("solr.test.sys.prop2");
+    System.clearProperty("collectionsHandler");
+
     SolrRequestParsers.DEFAULT.setAddRequestHeadersToContext(false);
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a1a8b286/solr/core/src/test/org/apache/solr/security/hadoop/ImpersonationUtil.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/security/hadoop/ImpersonationUtil.java b/solr/core/src/test/org/apache/solr/security/hadoop/ImpersonationUtil.java
new file mode 100644
index 0000000..00c2b71
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/security/hadoop/ImpersonationUtil.java
@@ -0,0 +1,73 @@
+/*
+ * 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.solr.security.hadoop;
+
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.authentication.client.PseudoAuthenticator;
+import org.apache.lucene.util.Constants;
+import org.apache.solr.client.solrj.SolrRequest;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.common.params.SolrParams;
+import org.apache.solr.security.HadoopAuthPlugin;
+import org.apache.solr.security.KerberosPlugin;
+
+/**
+ * This class implements utility functions required to test the secure impersonation feature for {@linkplain HadoopAuthPlugin}
+ */
+public class ImpersonationUtil {
+
+  static String getUsersFirstGroup() throws Exception {
+    String group = "*"; // accept any group if a group can't be found
+    if (!Constants.WINDOWS) { // does not work on Windows!
+      org.apache.hadoop.security.Groups hGroups =
+          new org.apache.hadoop.security.Groups(new Configuration());
+      try {
+        List<String> g = hGroups.getGroups(System.getProperty("user.name"));
+        if (g != null && g.size() > 0) {
+          group = g.get(0);
+        }
+      } catch (NullPointerException npe) {
+        // if user/group doesn't exist on test box
+      }
+    }
+    return group;
+  }
+
+  static SolrRequest getProxyRequest(String user, String doAs) {
+    return new CollectionAdminRequest.List() {
+      @Override
+      public SolrParams getParams() {
+        ModifiableSolrParams params = new ModifiableSolrParams(super.getParams());
+        params.set(PseudoAuthenticator.USER_NAME, user);
+        params.set(KerberosPlugin.IMPERSONATOR_DO_AS_HTTP_PARAM, doAs);
+        return params;
+      }
+    };
+  }
+
+  static String getExpectedGroupExMsg(String user, String doAs) {
+    return "User: " + user + " is not allowed to impersonate " + doAs;
+  }
+
+  static String getExpectedHostExMsg(String user) {
+    return "Unauthorized connection for super-user: " + user;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a1a8b286/solr/core/src/test/org/apache/solr/security/hadoop/ImpersonatorCollectionsHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/security/hadoop/ImpersonatorCollectionsHandler.java b/solr/core/src/test/org/apache/solr/security/hadoop/ImpersonatorCollectionsHandler.java
new file mode 100644
index 0000000..2a2a469
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/security/hadoop/ImpersonatorCollectionsHandler.java
@@ -0,0 +1,60 @@
+/*
+ * 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.solr.security.hadoop;
+
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import javax.servlet.http.HttpServletRequest;
+
+import org.apache.hadoop.security.authentication.client.PseudoAuthenticator;
+import org.apache.solr.core.CoreContainer;
+import org.apache.solr.handler.admin.CollectionsHandler;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.response.SolrQueryResponse;
+import org.apache.solr.security.HadoopAuthPlugin;
+import org.apache.solr.security.KerberosPlugin;
+import org.junit.Assert;
+
+/**
+ * This class extends {@linkplain CollectionsHandler} and implements extra validations
+ * for verifying proxy users support in {@linkplain HadoopAuthPlugin}
+ */
+public class ImpersonatorCollectionsHandler extends CollectionsHandler {
+  static AtomicBoolean called = new AtomicBoolean(false);
+
+  public ImpersonatorCollectionsHandler() {
+    super();
+  }
+
+  public ImpersonatorCollectionsHandler(final CoreContainer coreContainer) {
+    super(coreContainer);
+  }
+
+  @Override
+  public void handleRequestBody(SolrQueryRequest req, SolrQueryResponse rsp) throws Exception {
+    called.set(true);
+    super.handleRequestBody(req, rsp);
+    String doAs = req.getParams().get(KerberosPlugin.IMPERSONATOR_DO_AS_HTTP_PARAM);
+    if (doAs != null) {
+      HttpServletRequest httpRequest = (HttpServletRequest)req.getContext().get("httpRequest");
+      Assert.assertNotNull(httpRequest);
+      String user = req.getParams().get(PseudoAuthenticator.USER_NAME);
+      Assert.assertNotNull(user);
+      Assert.assertEquals(user, httpRequest.getAttribute(KerberosPlugin.IMPERSONATOR_USER_NAME));
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a1a8b286/solr/core/src/test/org/apache/solr/security/hadoop/TestDelegationWithHadoopAuth.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/security/hadoop/TestDelegationWithHadoopAuth.java b/solr/core/src/test/org/apache/solr/security/hadoop/TestDelegationWithHadoopAuth.java
new file mode 100644
index 0000000..37c6b8c
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/security/hadoop/TestDelegationWithHadoopAuth.java
@@ -0,0 +1,400 @@
+/*
+ * 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.solr.security.hadoop;
+
+import java.util.HashSet;
+import java.util.Set;
+
+import org.apache.hadoop.security.authentication.client.PseudoAuthenticator;
+import org.apache.hadoop.util.Time;
+import org.apache.http.HttpStatus;
+import org.apache.lucene.util.Constants;
+import org.apache.solr.client.solrj.SolrClient;
+import org.apache.solr.client.solrj.SolrRequest;
+import org.apache.solr.client.solrj.embedded.JettySolrRunner;
+import org.apache.solr.client.solrj.impl.CloudSolrClient;
+import org.apache.solr.client.solrj.impl.HttpSolrClient;
+import org.apache.solr.client.solrj.impl.LBHttpSolrClient;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.client.solrj.request.DelegationTokenRequest;
+import org.apache.solr.client.solrj.response.DelegationTokenResponse;
+import org.apache.solr.cloud.SolrCloudTestCase;
+import org.apache.solr.common.SolrException.ErrorCode;
+import org.apache.solr.common.cloud.SolrZkClient;
+import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.common.params.SolrParams;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import junit.framework.Assert;
+
+public class TestDelegationWithHadoopAuth extends SolrCloudTestCase {
+  protected static final int NUM_SERVERS = 2;
+  protected static final String USER_1 = "foo";
+  protected static final String USER_2 = "bar";
+  private static HttpSolrClient primarySolrClient, secondarySolrClient;
+
+  @BeforeClass
+  public static void setupClass() throws Exception {
+    assumeFalse("Hadoop does not work on Windows", Constants.WINDOWS);
+
+    configureCluster(NUM_SERVERS)// nodes
+        .withSecurityJson(TEST_PATH().resolve("security").resolve("hadoop_simple_auth_with_delegation.json"))
+        .configure();
+
+    JettySolrRunner runnerPrimary = cluster.getJettySolrRunners().get(0);
+    primarySolrClient =
+        new HttpSolrClient.Builder(runnerPrimary.getBaseUrl().toString())
+            .build();
+    JettySolrRunner runnerSecondary = cluster.getJettySolrRunners().get(1);
+    secondarySolrClient =
+        new HttpSolrClient.Builder(runnerSecondary.getBaseUrl().toString())
+            .build();
+  }
+
+  @AfterClass
+  public static void tearDownClass() throws Exception {
+    if (primarySolrClient != null) {
+      primarySolrClient.close();
+      primarySolrClient = null;
+    }
+
+    if (secondarySolrClient != null) {
+      secondarySolrClient.close();
+      secondarySolrClient = null;
+    }
+  }
+
+  private String getDelegationToken(final String renewer, final String user, HttpSolrClient solrClient) throws Exception {
+    DelegationTokenRequest.Get get = new DelegationTokenRequest.Get(renewer) {
+      @Override
+      public SolrParams getParams() {
+        ModifiableSolrParams params = new ModifiableSolrParams(super.getParams());
+        params.set(PseudoAuthenticator.USER_NAME, user);
+        return params;
+      }
+    };
+    DelegationTokenResponse.Get getResponse = get.process(solrClient);
+    return getResponse.getDelegationToken();
+  }
+
+  private long renewDelegationToken(final String token, final int expectedStatusCode,
+      final String user, HttpSolrClient client) throws Exception {
+    DelegationTokenRequest.Renew renew = new DelegationTokenRequest.Renew(token) {
+      @Override
+      public SolrParams getParams() {
+        ModifiableSolrParams params = new ModifiableSolrParams(super.getParams());
+        params.set(PseudoAuthenticator.USER_NAME, user);
+        return params;
+      }
+
+      @Override
+      public Set<String> getQueryParams() {
+        Set<String> queryParams = super.getQueryParams();
+        queryParams.add(PseudoAuthenticator.USER_NAME);
+        return queryParams;
+      }
+    };
+    try {
+      DelegationTokenResponse.Renew renewResponse = renew.process(client);
+      assertEquals(HttpStatus.SC_OK, expectedStatusCode);
+      return renewResponse.getExpirationTime();
+    } catch (HttpSolrClient.RemoteSolrException ex) {
+      assertEquals(expectedStatusCode, ex.code());
+      return -1;
+    }
+  }
+
+  private void cancelDelegationToken(String token, int expectedStatusCode, HttpSolrClient client)
+  throws Exception {
+    DelegationTokenRequest.Cancel cancel = new DelegationTokenRequest.Cancel(token);
+    try {
+      cancel.process(client);
+      assertEquals(HttpStatus.SC_OK, expectedStatusCode);
+    } catch (HttpSolrClient.RemoteSolrException ex) {
+      assertEquals(expectedStatusCode, ex.code());
+    }
+  }
+
+  private void doSolrRequest(String token, int expectedStatusCode, HttpSolrClient client)
+  throws Exception {
+    doSolrRequest(token, expectedStatusCode, client, 1);
+  }
+
+  private void doSolrRequest(String token, int expectedStatusCode, HttpSolrClient client, int trials)
+  throws Exception {
+    int lastStatusCode = 0;
+    for (int i = 0; i < trials; ++i) {
+      lastStatusCode = getStatusCode(token, null, null, client);
+      if (lastStatusCode == expectedStatusCode) {
+        return;
+      }
+      Thread.sleep(1000);
+    }
+    assertEquals("Did not receieve excepted status code", expectedStatusCode, lastStatusCode);
+  }
+
+  private SolrRequest getAdminRequest(final SolrParams params) {
+    return new CollectionAdminRequest.List() {
+      @Override
+      public SolrParams getParams() {
+        ModifiableSolrParams p = new ModifiableSolrParams(super.getParams());
+        p.add(params);
+        return p;
+      }
+    };
+  }
+
+  private int getStatusCode(String token, final String user, final String op, HttpSolrClient client)
+  throws Exception {
+    SolrClient delegationTokenClient;
+    if (random().nextBoolean()) delegationTokenClient = new HttpSolrClient.Builder(client.getBaseURL().toString())
+        .withKerberosDelegationToken(token)
+        .withResponseParser(client.getParser())
+        .build();
+    else delegationTokenClient = new CloudSolrClient.Builder()
+        .withZkHost((cluster.getZkServer().getZkAddress()))
+        .withLBHttpSolrClientBuilder(new LBHttpSolrClient.Builder()
+            .withResponseParser(client.getParser())
+            .withHttpSolrClientBuilder(
+                new HttpSolrClient.Builder()
+                    .withKerberosDelegationToken(token)
+            ))
+        .build();
+    try {
+      ModifiableSolrParams p = new ModifiableSolrParams();
+      if (user != null) p.set(PseudoAuthenticator.USER_NAME, user);
+      if (op != null) p.set("op", op);
+      SolrRequest req = getAdminRequest(p);
+      if (user != null || op != null) {
+        Set<String> queryParams = new HashSet<>();
+        if (user != null) queryParams.add(PseudoAuthenticator.USER_NAME);
+        if (op != null) queryParams.add("op");
+        req.setQueryParams(queryParams);
+      }
+      try {
+        delegationTokenClient.request(req, null);
+        return HttpStatus.SC_OK;
+      } catch (HttpSolrClient.RemoteSolrException re) {
+        return re.code();
+      }
+    } finally {
+      delegationTokenClient.close();
+    }
+  }
+
+  private void doSolrRequest(SolrClient client, SolrRequest request,
+      int expectedStatusCode) throws Exception {
+    try {
+      client.request(request);
+      assertEquals(HttpStatus.SC_OK, expectedStatusCode);
+    } catch (HttpSolrClient.RemoteSolrException ex) {
+      assertEquals(expectedStatusCode, ex.code());
+    }
+  }
+
+  private void verifyTokenValid(String token) throws Exception {
+     // pass with token
+    doSolrRequest(token, HttpStatus.SC_OK, primarySolrClient);
+
+    // fail without token
+    doSolrRequest(null, ErrorCode.UNAUTHORIZED.code, primarySolrClient);
+
+    // pass with token on other server
+    doSolrRequest(token, HttpStatus.SC_OK, secondarySolrClient);
+
+    // fail without token on other server
+    doSolrRequest(null, ErrorCode.UNAUTHORIZED.code, secondarySolrClient);
+  }
+
+  /**
+   * Test basic Delegation Token get/verify
+   */
+  @Test
+  public void testDelegationTokenVerify() throws Exception {
+    // Get token
+    String token = getDelegationToken(null, USER_1, primarySolrClient);
+    assertNotNull(token);
+    verifyTokenValid(token);
+  }
+
+  private void verifyTokenCancelled(String token, HttpSolrClient client) throws Exception {
+    // fail with token on both servers.  If cancelToOtherURL is true,
+    // the request went to other url, so FORBIDDEN should be returned immediately.
+    // The cancelled token may take awhile to propogate to the standard url (via ZK).
+    // This is of course the opposite if cancelToOtherURL is false.
+    doSolrRequest(token, ErrorCode.FORBIDDEN.code, client, 10);
+
+    // fail without token on both servers
+    doSolrRequest(null, ErrorCode.UNAUTHORIZED.code, primarySolrClient);
+    doSolrRequest(null, ErrorCode.UNAUTHORIZED.code, secondarySolrClient);
+  }
+
+  @Test
+  public void testDelegationTokenCancel() throws Exception {
+    {
+      // Get token
+      String token = getDelegationToken(null, USER_1, primarySolrClient);
+      assertNotNull(token);
+
+      // cancel token, note don't need to be authenticated to cancel (no user specified)
+      cancelDelegationToken(token, HttpStatus.SC_OK, primarySolrClient);
+      verifyTokenCancelled(token, primarySolrClient);
+    }
+
+    {
+      // cancel token on different server from where we got it
+      String token = getDelegationToken(null, USER_1, primarySolrClient);
+      assertNotNull(token);
+
+      cancelDelegationToken(token, HttpStatus.SC_OK, secondarySolrClient);
+      verifyTokenCancelled(token, secondarySolrClient);
+    }
+  }
+
+  @Test
+  public void testDelegationTokenCancelFail() throws Exception {
+    // cancel a bogus token
+    cancelDelegationToken("BOGUS", ErrorCode.NOT_FOUND.code, primarySolrClient);
+
+    {
+      // cancel twice, first on same server
+      String token = getDelegationToken(null, USER_1, primarySolrClient);
+      assertNotNull(token);
+      cancelDelegationToken(token, HttpStatus.SC_OK, primarySolrClient);
+      cancelDelegationToken(token, ErrorCode.NOT_FOUND.code, secondarySolrClient);
+      cancelDelegationToken(token, ErrorCode.NOT_FOUND.code, primarySolrClient);
+    }
+
+    {
+      // cancel twice, first on other server
+      String token = getDelegationToken(null, USER_1, primarySolrClient);
+      assertNotNull(token);
+      cancelDelegationToken(token, HttpStatus.SC_OK, secondarySolrClient);
+      cancelDelegationToken(token, ErrorCode.NOT_FOUND.code, secondarySolrClient);
+      cancelDelegationToken(token, ErrorCode.NOT_FOUND.code, primarySolrClient);
+    }
+  }
+
+  private void verifyDelegationTokenRenew(String renewer, String user)
+  throws Exception {
+    {
+      // renew on same server
+      String token = getDelegationToken(renewer, user, primarySolrClient);
+      assertNotNull(token);
+      long now = Time.now();
+      assertTrue(renewDelegationToken(token, HttpStatus.SC_OK, user, primarySolrClient) > now);
+      verifyTokenValid(token);
+    }
+
+    {
+      // renew on different server
+      String token = getDelegationToken(renewer, user, primarySolrClient);
+      assertNotNull(token);
+      long now = Time.now();
+      assertTrue(renewDelegationToken(token, HttpStatus.SC_OK, user, secondarySolrClient) > now);
+      verifyTokenValid(token);
+    }
+  }
+
+  @Test
+  public void testDelegationTokenRenew() throws Exception {
+    // test with specifying renewer
+    verifyDelegationTokenRenew(USER_1, USER_1);
+
+    // test without specifying renewer
+    verifyDelegationTokenRenew(null, USER_1);
+  }
+
+  @Test
+  public void testDelegationTokenRenewFail() throws Exception {
+    // don't set renewer and try to renew as an a different user
+    String token = getDelegationToken(null, USER_1, primarySolrClient);
+    assertNotNull(token);
+    renewDelegationToken(token, ErrorCode.FORBIDDEN.code, USER_2, primarySolrClient);
+    renewDelegationToken(token, ErrorCode.FORBIDDEN.code, USER_2, secondarySolrClient);
+
+    // set renewer and try to renew as different user
+    token = getDelegationToken("renewUser", USER_1, primarySolrClient);
+    assertNotNull(token);
+    renewDelegationToken(token, ErrorCode.FORBIDDEN.code, "notRenewUser", primarySolrClient);
+    renewDelegationToken(token, ErrorCode.FORBIDDEN.code, "notRenewUser", secondarySolrClient);
+  }
+
+  /**
+   * Test that a non-delegation-token "op" http param is handled correctly
+   */
+  @Test
+  public void testDelegationOtherOp() throws Exception {
+    assertEquals(HttpStatus.SC_OK, getStatusCode(null, USER_1, "someSolrOperation", primarySolrClient));
+  }
+
+  @Test
+  public void testZNodePaths() throws Exception {
+    getDelegationToken(null, USER_1, primarySolrClient);
+    SolrZkClient zkClient = new SolrZkClient(cluster.getZkServer().getZkAddress(), 1000);
+    try {
+      assertTrue(zkClient.exists("/security/zkdtsm", true));
+      assertTrue(zkClient.exists("/security/token", true));
+    } finally {
+      zkClient.close();
+    }
+  }
+
+  /**
+   * Test HttpSolrServer's delegation token support
+   */
+  @Test
+  public void testDelegationTokenSolrClient() throws Exception {
+    // Get token
+    String token = getDelegationToken(null, USER_1, primarySolrClient);
+    assertNotNull(token);
+
+    SolrRequest request = getAdminRequest(new ModifiableSolrParams());
+
+    // test without token
+    HttpSolrClient ss =
+        new HttpSolrClient.Builder(primarySolrClient.getBaseURL().toString())
+            .withResponseParser(primarySolrClient.getParser())
+            .build();
+    try {
+      doSolrRequest(ss, request, ErrorCode.UNAUTHORIZED.code);
+    } finally {
+      ss.close();
+    }
+
+    ss = new HttpSolrClient.Builder(primarySolrClient.getBaseURL().toString())
+        .withDelegationToken(token)
+        .withResponseParser(primarySolrClient.getParser())
+        .build();
+    try {
+      // test with token via property
+      doSolrRequest(ss, request, HttpStatus.SC_OK);
+
+      // test with param -- should throw an exception
+      ModifiableSolrParams tokenParam = new ModifiableSolrParams();
+      tokenParam.set("delegation", "invalidToken");
+      try {
+        doSolrRequest(ss, getAdminRequest(tokenParam), ErrorCode.FORBIDDEN.code);
+        Assert.fail("Expected exception");
+      } catch (IllegalArgumentException ex) {}
+    } finally {
+      ss.close();
+    }
+  }
+}