You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by gc...@apache.org on 2016/07/28 17:00:31 UTC

[2/2] lucene-solr:master: SOLR-9200: Add Delegation Token Support to Solr

SOLR-9200: Add Delegation Token Support to Solr


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

Branch: refs/heads/master
Commit: 7bf019a9c2779368133850f141174febede6ad96
Parents: 58f0fbd
Author: Gregory Chanan <gc...@cloudera.com>
Authored: Fri Jun 17 16:49:48 2016 -0700
Committer: Gregory Chanan <gc...@cloudera.com>
Committed: Thu Jul 28 12:59:05 2016 -0400

----------------------------------------------------------------------
 lucene/ivy-versions.properties                  |   1 +
 solr/CHANGES.txt                                |   3 +
 solr/core/ivy.xml                               |   4 +
 .../solr/security/AuthenticationPlugin.java     |  29 +-
 .../apache/solr/security/BasicAuthPlugin.java   |   5 +-
 .../security/DelegationTokenKerberosFilter.java | 171 ++++++++
 .../apache/solr/security/KerberosFilter.java    |  14 +
 .../apache/solr/security/KerberosPlugin.java    | 195 ++++++++-
 .../solr/security/PKIAuthenticationPlugin.java  |  13 +-
 .../solr/security/PrintWriterWrapper.java       | 215 ++++++++++
 .../apache/solr/servlet/SolrDispatchFilter.java |  12 +-
 .../apache/solr/cloud/KerberosTestServices.java | 228 +++++++++++
 .../org/apache/solr/cloud/KerberosTestUtil.java | 147 -------
 ...utOfBoxZkACLAndCredentialsProvidersTest.java |   7 +-
 ...rriddenZkACLAndCredentialsProvidersTest.java |  71 ++--
 .../solr/cloud/SaslZkACLProviderTest.java       |  39 +-
 .../solr/cloud/TestAuthenticationFramework.java |  10 +-
 .../cloud/TestMiniSolrCloudClusterKerberos.java |  29 +-
 .../TestSolrCloudWithDelegationTokens.java      | 405 +++++++++++++++++++
 .../cloud/TestSolrCloudWithKerberosAlt.java     |  37 +-
 ...MParamsZkACLAndCredentialsProvidersTest.java |  25 +-
 ...ramDelegationTokenAuthenticationHandler.java | 109 +++++
 .../solr/security/MockAuthenticationPlugin.java |  32 +-
 solr/licenses/curator-recipes-2.8.0.jar.sha1    |   1 +
 solr/licenses/curator-recipes-LICENSE-ASL.txt   | 202 +++++++++
 solr/licenses/curator-recipes-NOTICE.txt        |   5 +
 solr/solrj/ivy.xml                              |   5 +
 .../solr/client/solrj/impl/HttpSolrClient.java  |  57 ++-
 .../solrj/impl/Krb5HttpClientBuilder.java       |  18 +-
 .../solrj/request/DelegationTokenRequest.java   | 152 +++++++
 .../solrj/response/DelegationTokenResponse.java | 108 +++++
 .../solr/common/cloud/SaslZkACLProvider.java    |  21 +-
 .../cloud/SecurityAwareZkACLProvider.java       |  79 ++++
 .../apache/solr/common/cloud/SolrZkClient.java  |   2 +-
 ...ParamsAllAndReadonlyDigestZkACLProvider.java |  52 ++-
 .../cloud/ZkClientConnectionStrategy.java       |   4 +-
 .../request/TestDelegationTokenRequest.java     |  70 ++++
 .../response/TestDelegationTokenResponse.java   | 138 +++++++
 38 files changed, 2376 insertions(+), 339 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7bf019a9/lucene/ivy-versions.properties
----------------------------------------------------------------------
diff --git a/lucene/ivy-versions.properties b/lucene/ivy-versions.properties
index 510befa..bc46ee6 100644
--- a/lucene/ivy-versions.properties
+++ b/lucene/ivy-versions.properties
@@ -106,6 +106,7 @@ io.netty.netty-all.version = 4.0.36.Final
 org.apache.curator.version = 2.8.0
 /org.apache.curator/curator-client = ${org.apache.curator.version}
 /org.apache.curator/curator-framework = ${org.apache.curator.version}
+/org.apache.curator/curator-recipes = ${org.apache.curator.version}
 
 /org.apache.derby/derby = 10.9.1.0
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7bf019a9/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index a728c8a..6707e1a 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -100,6 +100,9 @@ New Features
 * SOLR-9275: XML QueryParser support (defType=xmlparser) now extensible via configuration.
   (Christine Poerschke)
 
+* SOLR-9200: Add Delegation Token Support to Solr.
+  (Gregory Chanan)
+
 Bug Fixes
 ----------------------
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7bf019a9/solr/core/ivy.xml
----------------------------------------------------------------------
diff --git a/solr/core/ivy.xml b/solr/core/ivy.xml
index 5dad49b..08272ad 100644
--- a/solr/core/ivy.xml
+++ b/solr/core/ivy.xml
@@ -134,6 +134,10 @@
     <dependency org="antlr" name="antlr" rev="${/antlr/antlr}" conf="test.MiniKdc"/>
     <dependency org="net.sf.ehcache" name="ehcache-core" rev="${/net.sf.ehcache/ehcache-core}" conf="test.MiniKdc"/>
 
+    <dependency org="org.apache.curator" name="curator-framework" rev="${/org.apache.curator/curator-framework}" conf="compile"/>
+    <dependency org="org.apache.curator" name="curator-client" rev="${/org.apache.curator/curator-client}" conf="compile"/>
+    <dependency org="org.apache.curator" name="curator-recipes" rev="${/org.apache.curator/curator-recipes}" conf="compile"/>
+
     <!-- StatsComponents percentiles Dependencies-->
     <dependency org="com.tdunning" name="t-digest" rev="${/com.tdunning/t-digest}" conf="compile->*"/>
     <!-- SQL Parser -->

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7bf019a9/solr/core/src/java/org/apache/solr/security/AuthenticationPlugin.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/security/AuthenticationPlugin.java b/solr/core/src/java/org/apache/solr/security/AuthenticationPlugin.java
index 105f307..d8f2ef2 100644
--- a/solr/core/src/java/org/apache/solr/security/AuthenticationPlugin.java
+++ b/solr/core/src/java/org/apache/solr/security/AuthenticationPlugin.java
@@ -17,18 +17,11 @@
 package org.apache.solr.security;
 
 import javax.servlet.FilterChain;
-import javax.servlet.ServletException;
 import javax.servlet.ServletRequest;
 import javax.servlet.ServletResponse;
-import javax.servlet.http.HttpServletRequest;
-import javax.servlet.http.HttpServletRequestWrapper;
 import java.io.Closeable;
-import java.io.IOException;
-import java.security.Principal;
 import java.util.Map;
 
-import org.apache.http.auth.BasicUserPrincipal;
-
 /**
  * 
  * @lucene.experimental
@@ -42,32 +35,20 @@ public abstract class AuthenticationPlugin implements Closeable {
    * @param pluginConfig Config parameters, possibly from a ZK source
    */
   public abstract void init(Map<String, Object> pluginConfig);
-
-  protected void forward(String user, ServletRequest  req, ServletResponse rsp,
-                                    FilterChain chain) throws IOException, ServletException {
-    if(user != null) {
-      final Principal p = new BasicUserPrincipal(user);
-      req = new HttpServletRequestWrapper((HttpServletRequest) req) {
-        @Override
-        public Principal getUserPrincipal() {
-          return p;
-        }
-      };
-    }
-    chain.doFilter(req,rsp);
-  }
  
   /**
-   * This method must authenticate the request. Upon a successful authentication, this 
+   * This method attempts to authenticate the request. Upon a successful authentication, this
    * must call the next filter in the filter chain and set the user principal of the request,
    * or else, upon an error or an authentication failure, throw an exception.
-   * 
+   *
    * @param request the http request
    * @param response the http response
    * @param filterChain the servlet filter chain
+   * @return false if the request not be processed by Solr (not continue), i.e.
+   * the response and status code have already been sent.
    * @throws Exception any exception thrown during the authentication, e.g. PrivilegedActionException
    */
-  public abstract void doAuthenticate(ServletRequest request, ServletResponse response,
+  public abstract boolean doAuthenticate(ServletRequest request, ServletResponse response,
       FilterChain filterChain) throws Exception;
 
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7bf019a9/solr/core/src/java/org/apache/solr/security/BasicAuthPlugin.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/security/BasicAuthPlugin.java b/solr/core/src/java/org/apache/solr/security/BasicAuthPlugin.java
index 03c75c6..e3f53a2 100644
--- a/solr/core/src/java/org/apache/solr/security/BasicAuthPlugin.java
+++ b/solr/core/src/java/org/apache/solr/security/BasicAuthPlugin.java
@@ -99,7 +99,7 @@ public class BasicAuthPlugin extends AuthenticationPlugin implements ConfigEdita
   }
 
   @Override
-  public void doAuthenticate(ServletRequest servletRequest, ServletResponse servletResponse, FilterChain filterChain) throws Exception {
+  public boolean doAuthenticate(ServletRequest servletRequest, ServletResponse servletResponse, FilterChain filterChain) throws Exception {
 
     HttpServletRequest request = (HttpServletRequest) servletRequest;
     HttpServletResponse response = (HttpServletResponse) servletResponse;
@@ -127,6 +127,7 @@ public class BasicAuthPlugin extends AuthenticationPlugin implements ConfigEdita
                   }
                 };
                 filterChain.doFilter(wrapper, response);
+                return true;
               }
 
             } else {
@@ -143,8 +144,10 @@ public class BasicAuthPlugin extends AuthenticationPlugin implements ConfigEdita
       } else {
         request.setAttribute(AuthenticationPlugin.class.getName(), zkAuthentication.getPromptHeaders());
         filterChain.doFilter(request, response);
+        return true;
       }
     }
+    return false;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7bf019a9/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
new file mode 100644
index 0000000..7dbb1ad
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/security/DelegationTokenKerberosFilter.java
@@ -0,0 +1,171 @@
+/*
+ * 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.lang.invoke.MethodHandles;
+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.authentication.server.AuthenticationHandler;
+import org.apache.hadoop.security.token.delegation.web.DelegationTokenAuthenticationFilter;
+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;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class DelegationTokenKerberosFilter extends DelegationTokenAuthenticationFilter {
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  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(KerberosPlugin.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;
+      }
+    };
+    super.doFilter(requestNonNullQueryString, response, filterChain);
+  }
+
+  @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(KerberosPlugin.RequestContinuesRecorderAuthenticationHandler.class.getName(), filterConfig);
+    KerberosPlugin.RequestContinuesRecorderAuthenticationHandler newAuthHandler =
+        (KerberosPlugin.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.substring(zkHost.indexOf("/"));
+    zkChroot = zkChroot.startsWith("/") ? zkChroot.substring(1) : zkChroot;
+    String zkNamespace = zkChroot + SecurityAwareZkACLProvider.SECURITY_ZNODE_PATH;
+    String zkConnectionString = zkHost.substring(0, zkHost.indexOf("/"));
+    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;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7bf019a9/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 ee23488..9c53050 100644
--- a/solr/core/src/java/org/apache/solr/security/KerberosFilter.java
+++ b/solr/core/src/java/org/apache/solr/security/KerberosFilter.java
@@ -26,6 +26,7 @@ import javax.servlet.ServletResponse;
 import javax.servlet.http.HttpServletRequest;
 import javax.servlet.http.HttpServletResponse;
 import org.apache.hadoop.security.authentication.server.AuthenticationFilter;
+import org.apache.hadoop.security.authentication.server.AuthenticationHandler;
 
 public class KerberosFilter extends AuthenticationFilter {
   
@@ -35,6 +36,19 @@ public class KerberosFilter extends AuthenticationFilter {
   }
 
   @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(
+        KerberosPlugin.RequestContinuesRecorderAuthenticationHandler.class.getName(), filterConfig);
+    KerberosPlugin.RequestContinuesRecorderAuthenticationHandler newAuthHandler =
+        (KerberosPlugin.RequestContinuesRecorderAuthenticationHandler)getAuthenticationHandler();
+    newAuthHandler.setAuthHandler(authHandler);
+  }
+
+  @Override
   protected void doFilter(FilterChain filterChain, HttpServletRequest request,
       HttpServletResponse response) throws IOException, ServletException {
     super.doFilter(filterChain, request, response);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7bf019a9/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 7a83ab5..1cd476f 100644
--- a/solr/core/src/java/org/apache/solr/security/KerberosPlugin.java
+++ b/solr/core/src/java/org/apache/solr/security/KerberosPlugin.java
@@ -16,14 +16,18 @@
  */
 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;
@@ -41,12 +45,22 @@ 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;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;
+import org.apache.solr.common.cloud.SecurityAwareZkACLProvider;
+import org.apache.solr.common.util.SuppressForbidden;
 import org.apache.solr.core.CoreContainer;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -55,7 +69,7 @@ public class KerberosPlugin extends AuthenticationPlugin implements HttpClientBu
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
   Krb5HttpClientBuilder kerberosBuilder = new Krb5HttpClientBuilder();
-  Filter kerberosFilter = new KerberosFilter();
+  Filter kerberosFilter;
   
   public static final String NAME_RULES_PARAM = "solr.kerberos.name.rules";
   public static final String COOKIE_DOMAIN_PARAM = "solr.kerberos.cookie.domain";
@@ -64,6 +78,26 @@ public class KerberosPlugin extends AuthenticationPlugin implements HttpClientBu
   public static final String KEYTAB_PARAM = "solr.kerberos.keytab";
   public static final String TOKEN_VALID_PARAM = "solr.kerberos.token.valid";
   public static final String COOKIE_PORT_AWARE_PARAM = "solr.kerberos.cookie.portaware";
+  public static final String DELEGATION_TOKEN_ENABLED = "solr.kerberos.delegation.token.enabled";
+  public static final String DELEGATION_TOKEN_KIND = "solr.kerberos.delegation.token.kind";
+  public static final String DELEGATION_TOKEN_VALIDITY = "solr.kerberos.delegation.token.validity";
+  public static final String DELEGATION_TOKEN_SECRET_PROVIDER = "solr.kerberos.delegation.token.signer.secret.provider";
+  public static final String DELEGATION_TOKEN_SECRET_PROVIDER_ZK_PATH =
+      "solr.kerberos.delegation.token.signer.secret.provider.zookeper.path";
+  public static final String DELEGATION_TOKEN_SECRET_MANAGER_ZNODE_WORKING_PATH =
+      "solr.kerberos.delegation.token.secret.manager.znode.working.path";
+  public static final String DELEGATION_TOKEN_TYPE_DEFAULT = "solr-dt";
+  
+  // 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";
+
+  // allows test to specify an alternate auth handler
+  @VisibleForTesting
+  public static final String AUTH_HANDLER_PARAM = "solr.kerberos.auth.handler";
+
   private final CoreContainer coreContainer;
 
   public KerberosPlugin(CoreContainer coreContainer) {
@@ -74,12 +108,47 @@ public class KerberosPlugin extends AuthenticationPlugin implements HttpClientBu
   public void init(Map<String, Object> pluginConfig) {
     try {
       Map<String, String> params = new HashMap();
-      params.put("type", "kerberos");
+      putParam(params, "type", AUTH_HANDLER_PARAM, "kerberos");
       putParam(params, "kerberos.name.rules", NAME_RULES_PARAM, "DEFAULT");
       putParam(params, "token.valid", TOKEN_VALID_PARAM, "30");
       putParam(params, "cookie.path", COOKIE_PATH_PARAM, "/");
-      putParam(params, "kerberos.principal", PRINCIPAL_PARAM, null);
-      putParam(params, "kerberos.keytab", KEYTAB_PARAM, null);
+      if ("kerberos".equals(params.get("type"))) {
+        putParam(params, "kerberos.principal", PRINCIPAL_PARAM, null);
+        putParam(params, "kerberos.keytab", KEYTAB_PARAM, null);
+      } else {
+        // allow tests which specify AUTH_HANDLER_PARAM to avoid specifying kerberos principal/keytab
+        putParamOptional(params, "kerberos.principal", PRINCIPAL_PARAM);
+        putParamOptional(params, "kerberos.keytab", KEYTAB_PARAM);
+      }
+
+      String delegationTokenStr = System.getProperty(DELEGATION_TOKEN_ENABLED, null);
+      boolean delegationTokenEnabled =
+          (delegationTokenStr == null) ? false : Boolean.parseBoolean(delegationTokenStr);
+      ZkController controller = coreContainer.getZkController();
+
+      if (delegationTokenEnabled) {
+        putParam(params, "delegation-token.token-kind", DELEGATION_TOKEN_KIND, DELEGATION_TOKEN_TYPE_DEFAULT);
+        if (coreContainer.isZooKeeperAware()) {
+          putParam(params, "signer.secret.provider", DELEGATION_TOKEN_SECRET_PROVIDER, "zookeeper");
+          if ("zookeeper".equals(params.get("signer.secret.provider"))) {
+            String zkHost = controller.getZkServerAddress();
+            putParam(params, "token.validity", DELEGATION_TOKEN_VALIDITY, "36000");
+            params.put("zk-dt-secret-manager.enable", "true");
+            // Note - Curator complains if the znodeWorkingPath starts with /
+            String chrootPath = zkHost.substring(zkHost.indexOf("/"));
+            String relativePath = chrootPath.startsWith("/") ? chrootPath.substring(1) : chrootPath;
+            putParam(params, "zk-dt-secret-manager.znodeWorkingPath",
+                DELEGATION_TOKEN_SECRET_MANAGER_ZNODE_WORKING_PATH,
+                relativePath + SecurityAwareZkACLProvider.SECURITY_ZNODE_PATH + "/zkdtsm");
+            putParam(params, "signer.secret.provider.zookeeper.path",
+                DELEGATION_TOKEN_SECRET_PROVIDER_ZK_PATH, "/token");
+            // ensure krb5 is setup properly before running curator
+            getHttpClientBuilder(SolrHttpClientBuilder.create());
+          }
+        } else {
+          log.info("CoreContainer is not ZooKeeperAware, not setting ZK-related delegation token properties");
+        }
+      }
 
       // Special handling for the "cookie.domain" based on whether port should be
       // appended to the domain. Useful for situations where multiple solr nodes are
@@ -94,16 +163,27 @@ public class KerberosPlugin extends AuthenticationPlugin implements HttpClientBu
         if (host==null) {
           throw new SolrException(ErrorCode.SERVER_ERROR, "Missing required parameter '"+COOKIE_DOMAIN_PARAM+"'.");
         }
-        int port = coreContainer.getZkController().getHostPort();
+        int port = controller.getHostPort();
         params.put("cookie.domain", host + ":" + port);
       }
-      
+
+      final ServletContext servletContext = new AttributeOnlyServletContext();
+      if (delegationTokenEnabled) {
+        kerberosFilter = new DelegationTokenKerberosFilter();
+        // pass an attribute-enabled context in order to pass the zkClient
+        // and because the filter may pass a curator instance.
+        if (controller != null) {
+          servletContext.setAttribute(DELEGATION_TOKEN_ZK_CLIENT, controller.getZkClient());
+        }
+      } else {
+        kerberosFilter = new KerberosFilter();
+      }
       log.info("Params: "+params);
 
       FilterConfig conf = new FilterConfig() {
         @Override
         public ServletContext getServletContext() {
-          return noContext;
+          return servletContext;
         }
 
         @Override
@@ -136,11 +216,43 @@ public class KerberosPlugin extends AuthenticationPlugin implements HttpClientBu
     params.put(internalParamName, value);
   }
 
+  private void putParamOptional(Map<String, String> params, String internalParamName, String externalParamName) {
+    String value = System.getProperty(externalParamName);
+    if (value!=null) {
+      params.put(internalParamName, value);
+    }
+  }
+
   @Override
-  public void doAuthenticate(ServletRequest req, ServletResponse rsp,
+  public boolean doAuthenticate(ServletRequest req, ServletResponse rsp,
       FilterChain chain) throws Exception {
     log.debug("Request to authenticate using kerberos: "+req);
-    kerberosFilter.doFilter(req, rsp, chain);    
+
+    final HttpServletResponse frsp = (HttpServletResponse)rsp;
+
+    // kerberosFilter may close the stream and write to closed streams,
+    // see HADOOP-13346.  To work around, pass a PrintWriter that ignores
+    // closes
+    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;
+      }
+    };
+    kerberosFilter.doFilter(req, rspCloseShield, chain);
+    String requestContinuesAttr = (String)req.getAttribute(REQUEST_CONTINUES_ATTR);
+    if (requestContinuesAttr == null) {
+      log.warn("Could not find " + REQUEST_CONTINUES_ATTR);
+      return false;
+    } else {
+      return Boolean.parseBoolean(requestContinuesAttr);
+    }
   }
 
   @Override
@@ -153,8 +265,9 @@ public class KerberosPlugin extends AuthenticationPlugin implements HttpClientBu
     kerberosBuilder.close();
   }
 
-  protected static ServletContext noContext = new ServletContext() {
-    
+  protected static class AttributeOnlyServletContext implements ServletContext {
+    private Map<String, Object> attributes = new HashMap<String, Object>();
+
     @Override
     public void setSessionTrackingModes(Set<SessionTrackingMode> sessionTrackingModes) {}
     
@@ -162,12 +275,16 @@ public class KerberosPlugin extends AuthenticationPlugin implements HttpClientBu
     public boolean setInitParameter(String name, String value) {
       return false;
     }
-    
+
     @Override
-    public void setAttribute(String name, Object object) {}
-    
+    public void setAttribute(String name, Object object) {
+      attributes.put(name, object);
+    }
+
     @Override
-    public void removeAttribute(String name) {}
+    public void removeAttribute(String name) {
+      attributes.remove(name);
+    }
     
     @Override
     public void log(String message, Throwable throwable) {}
@@ -327,15 +444,15 @@ public class KerberosPlugin extends AuthenticationPlugin implements HttpClientBu
     public ClassLoader getClassLoader() {
       return null;
     }
-    
+
     @Override
     public Enumeration<String> getAttributeNames() {
-      return null;
+      return Collections.enumeration(attributes.keySet());
     }
-    
+
     @Override
     public Object getAttribute(String name) {
-      return null;
+      return attributes.get(name);
     }
     
     @Override
@@ -395,4 +512,44 @@ public class KerberosPlugin extends AuthenticationPlugin implements HttpClientBu
       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/7bf019a9/solr/core/src/java/org/apache/solr/security/PKIAuthenticationPlugin.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/security/PKIAuthenticationPlugin.java b/solr/core/src/java/org/apache/solr/security/PKIAuthenticationPlugin.java
index fb3665c..e5d2653 100644
--- a/solr/core/src/java/org/apache/solr/security/PKIAuthenticationPlugin.java
+++ b/solr/core/src/java/org/apache/solr/security/PKIAuthenticationPlugin.java
@@ -89,12 +89,12 @@ public class PKIAuthenticationPlugin extends AuthenticationPlugin implements Htt
 
   @SuppressForbidden(reason = "Needs currentTimeMillis to compare against time in header")
   @Override
-  public void doAuthenticate(ServletRequest request, ServletResponse response, FilterChain filterChain) throws Exception {
+  public boolean doAuthenticate(ServletRequest request, ServletResponse response, FilterChain filterChain) throws Exception {
 
     String requestURI = ((HttpServletRequest) request).getRequestURI();
     if (requestURI.endsWith(PATH)) {
       filterChain.doFilter(request, response);
-      return;
+      return true;
     }
     long receivedTime = System.currentTimeMillis();
     String header = ((HttpServletRequest) request).getHeader(HEADER);
@@ -102,14 +102,14 @@ public class PKIAuthenticationPlugin extends AuthenticationPlugin implements Htt
       //this must not happen
       log.error("No SolrAuth header present");
       filterChain.doFilter(request, response);
-      return;
+      return true;
     }
 
     List<String> authInfo = StrUtils.splitWS(header, false);
     if (authInfo.size() < 2) {
       log.error("Invalid SolrAuth Header {}", header);
       filterChain.doFilter(request, response);
-      return;
+      return true;
     }
 
     String nodeName = authInfo.get(0);
@@ -119,12 +119,12 @@ public class PKIAuthenticationPlugin extends AuthenticationPlugin implements Htt
     if (decipher == null) {
       log.error("Could not decipher a header {} . No principal set", header);
       filterChain.doFilter(request, response);
-      return;
+      return true;
     }
     if ((receivedTime - decipher.timestamp) > MAX_VALIDITY) {
       log.error("Invalid key request timestamp: {} , received timestamp: {} , TTL: {}", decipher.timestamp, receivedTime, MAX_VALIDITY);
         filterChain.doFilter(request, response);
-        return;
+        return true;
     }
 
     final Principal principal = "$".equals(decipher.userName) ?
@@ -132,6 +132,7 @@ public class PKIAuthenticationPlugin extends AuthenticationPlugin implements Htt
         new BasicUserPrincipal(decipher.userName);
 
     filterChain.doFilter(getWrapper((HttpServletRequest) request, principal), response);
+    return true;
   }
 
   private static HttpServletRequestWrapper getWrapper(final HttpServletRequest request, final Principal principal) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7bf019a9/solr/core/src/java/org/apache/solr/security/PrintWriterWrapper.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/security/PrintWriterWrapper.java b/solr/core/src/java/org/apache/solr/security/PrintWriterWrapper.java
new file mode 100644
index 0000000..a4e47b5
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/security/PrintWriterWrapper.java
@@ -0,0 +1,215 @@
+/*
+ * 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.PrintWriter;
+import java.io.StringWriter;
+import java.util.Locale;
+
+import org.apache.commons.lang.NotImplementedException;
+
+/**
+ * Wrapper for PrintWriter that delegates to constructor arg
+ */
+public class PrintWriterWrapper extends PrintWriter {
+  private PrintWriter printWriter;
+
+  public PrintWriterWrapper(PrintWriter printWriter) {
+    super(new StringWriter());
+    this.printWriter = printWriter;
+  }
+
+  @Override
+  public PrintWriter append(char c) {
+    return printWriter.append(c);
+  }
+
+  @Override
+  public PrintWriter append(CharSequence csq) {
+    return printWriter.append(csq);
+  }
+
+  @Override
+  public PrintWriter append(CharSequence csq, int start, int end) {
+    return printWriter.append(csq, start, end);
+  }
+
+  @Override
+  public boolean checkError() {
+    return printWriter.checkError();
+  }
+
+  @Override
+  protected void clearError() {
+    throw new NotImplementedException();
+  }
+
+  @Override
+  public void close() {
+    printWriter.close();
+  }
+
+  @Override
+  public void flush() {
+    printWriter.flush();
+  }
+
+  @Override
+  public PrintWriter format(Locale l, String format, Object... args) {
+    return printWriter.format(l, format, args);
+  }
+
+  @Override
+  public PrintWriter format(String format, Object... args) {
+    throw new NotImplementedException("Forbidden API");
+  }
+
+  @Override
+  public void print(boolean b) {
+    printWriter.print(b);
+  }
+
+  @Override
+  public void print(char c) {
+    printWriter.print(c);
+  }
+
+  @Override
+  public void print(char[] s) {
+    printWriter.print(s);
+  }
+
+  @Override
+  public void print(double d) {
+    printWriter.print(d);
+  }
+
+  @Override
+  public void print(float f) {
+    printWriter.print(f);
+  }
+
+  @Override
+  public void print(int i) {
+    printWriter.print(i);
+  }
+
+  @Override
+  public void print(long l) {
+    printWriter.print(l);
+  }
+
+  @Override
+  public void print(Object obj) {
+    printWriter.print(obj);
+  }
+
+  @Override
+  public void print(String s) {
+    printWriter.print(s);
+  }
+
+  @Override
+  public PrintWriter printf(Locale l, String format, Object... args) {
+    return printWriter.printf(l, format, args);
+  }
+
+  @Override
+  public PrintWriter printf(String format, Object... args) {
+    throw new NotImplementedException("Forbidden API");
+  }
+
+  @Override
+  public void println() {
+    printWriter.println();
+  }
+
+  @Override
+  public void println(boolean x) {
+    printWriter.println(x);
+  }
+
+  @Override
+  public void println(char x) {
+    printWriter.println(x);
+  }
+
+  @Override
+  public void println(char[] x) {
+    printWriter.println(x);
+  }
+
+  @Override
+  public void println(double x) {
+    printWriter.println(x);
+  }
+
+  @Override
+  public void println(float x) {
+    printWriter.println(x);
+  }
+
+  @Override
+  public void println(int x) {
+    printWriter.println(x);
+  }
+
+  @Override
+  public void println(long x) {
+    printWriter.println(x);
+  }
+
+  @Override
+  public void println(Object x) {
+    printWriter.println(x);
+  }
+
+  @Override
+  public void println(String x) {
+    printWriter.println(x);
+  }
+
+  @Override
+  protected void setError() {
+    throw new NotImplementedException();
+  }
+
+  @Override
+  public void write(char[] buf) {
+    printWriter.write(buf);
+  }
+
+  @Override
+  public void write(char[] buf, int off, int len) {
+    printWriter.write(buf, off, len);
+  }
+
+  @Override
+  public void write(int c) {
+    printWriter.write(c);
+  }
+
+  @Override
+  public void write(String s) {
+    printWriter.write(s);
+  }
+
+  @Override
+  public void write(String s, int off, int len) {
+    printWriter.write(s, off, len);
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7bf019a9/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java b/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
index 2d08935..4a680e5 100644
--- a/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
+++ b/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
@@ -296,6 +296,7 @@ public class SolrDispatchFilter extends BaseSolrFilter {
   }
 
   private boolean authenticateRequest(ServletRequest request, ServletResponse response, final AtomicReference<ServletRequest> wrappedRequest) throws IOException {
+    boolean requestContinues = false;
     final AtomicBoolean isAuthenticated = new AtomicBoolean(false);
     AuthenticationPlugin authenticationPlugin = cores.getAuthenticationPlugin();
     if (authenticationPlugin == null) {
@@ -308,7 +309,7 @@ public class SolrDispatchFilter extends BaseSolrFilter {
       try {
         log.debug("Request to authenticate: {}, domain: {}, port: {}", request, request.getLocalName(), request.getLocalPort());
         // upon successful authentication, this should call the chain's next filter.
-        authenticationPlugin.doAuthenticate(request, response, new FilterChain() {
+        requestContinues = authenticationPlugin.doAuthenticate(request, response, new FilterChain() {
           public void doFilter(ServletRequest req, ServletResponse rsp) throws IOException, ServletException {
             isAuthenticated.set(true);
             wrappedRequest.set(req);
@@ -319,8 +320,13 @@ public class SolrDispatchFilter extends BaseSolrFilter {
         throw new SolrException(ErrorCode.SERVER_ERROR, "Error during request authentication, ", e);
       }
     }
-    // failed authentication?
-    if (!isAuthenticated.get()) {
+    // requestContinues is an optional short circuit, thus we still need to check isAuthenticated.
+    // This is because the AuthenticationPlugin doesn't always have enough information to determine if
+    // it should short circuit, e.g. the Kerberos Authentication Filter will send an error and not
+    // call later filters in chain, but doesn't throw an exception.  We could force each Plugin
+    // to implement isAuthenticated to simplify the check here, but that just moves the complexity to
+    // multiple code paths.
+    if (!requestContinues || !isAuthenticated.get()) {
       response.flushBuffer();
       return false;
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7bf019a9/solr/core/src/test/org/apache/solr/cloud/KerberosTestServices.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/KerberosTestServices.java b/solr/core/src/test/org/apache/solr/cloud/KerberosTestServices.java
new file mode 100644
index 0000000..eb27218
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/cloud/KerberosTestServices.java
@@ -0,0 +1,228 @@
+/*
+ * 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.cloud;
+
+import javax.security.auth.login.AppConfigurationEntry;
+import javax.security.auth.login.Configuration;
+import java.io.File;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Properties;
+
+import com.carrotsearch.ant.tasks.junit4.dependencies.com.google.common.base.Preconditions;
+import org.apache.hadoop.minikdc.MiniKdc;
+import org.apache.solr.client.solrj.impl.Krb5HttpClientBuilder;
+
+public class KerberosTestServices {
+
+  private MiniKdc kdc;
+  private JaasConfiguration jaasConfiguration;
+  private Configuration savedConfig;
+  private Locale savedLocale;
+
+  private KerberosTestServices(MiniKdc kdc,
+                               JaasConfiguration jaasConfiguration,
+                               Configuration savedConfig,
+                               Locale savedLocale) {
+    this.kdc = kdc;
+    this.jaasConfiguration = jaasConfiguration;
+    this.savedConfig = savedConfig;
+    this.savedLocale = savedLocale;
+  }
+
+  public MiniKdc getKdc() {
+    return kdc;
+  }
+
+  public void start() throws Exception {
+    if (kdc != null) kdc.start();
+    Configuration.setConfiguration(jaasConfiguration);
+    Krb5HttpClientBuilder.regenerateJaasConfiguration();
+    if (brokenLanguagesWithMiniKdc.contains(Locale.getDefault().getLanguage())) {
+      Locale.setDefault(Locale.US);
+    }
+  }
+
+  public void stop() {
+    if (kdc != null) kdc.stop();
+    Configuration.setConfiguration(savedConfig);
+    Krb5HttpClientBuilder.regenerateJaasConfiguration();
+    Locale.setDefault(savedLocale);
+  }
+
+  public static Builder builder() {
+    return new Builder();
+  }
+
+  /**
+   * Returns a MiniKdc that can be used for creating kerberos principals
+   * and keytabs.  Caller is responsible for starting/stopping the kdc.
+   */
+  private static MiniKdc getKdc(File workDir) throws Exception {
+    Properties conf = MiniKdc.createConf();
+    return new MiniKdc(conf, workDir);
+  }
+
+  /**
+   * Programmatic version of a jaas.conf file suitable for connecting
+   * to a SASL-configured zookeeper.
+   */
+  private static class JaasConfiguration extends Configuration {
+
+    private static AppConfigurationEntry[] clientEntry;
+    private static AppConfigurationEntry[] serverEntry;
+    private String clientAppName = "Client", serverAppName = "Server";
+
+    /**
+     * Add an entry to the jaas configuration with the passed in name,
+     * principal, and keytab. The other necessary options will be set for you.
+     *
+     * @param clientPrincipal The principal of the client
+     * @param clientKeytab The location of the keytab with the clientPrincipal
+     * @param serverPrincipal The principal of the server
+     * @param serverKeytab The location of the keytab with the serverPrincipal
+     */
+    public JaasConfiguration(String clientPrincipal, File clientKeytab,
+                             String serverPrincipal, File serverKeytab) {
+      Map<String, String> clientOptions = new HashMap();
+      clientOptions.put("principal", clientPrincipal);
+      clientOptions.put("keyTab", clientKeytab.getAbsolutePath());
+      clientOptions.put("useKeyTab", "true");
+      clientOptions.put("storeKey", "true");
+      clientOptions.put("useTicketCache", "false");
+      clientOptions.put("refreshKrb5Config", "true");
+      String jaasProp = System.getProperty("solr.jaas.debug");
+      if (jaasProp != null && "true".equalsIgnoreCase(jaasProp)) {
+        clientOptions.put("debug", "true");
+      }
+      clientEntry = new AppConfigurationEntry[]{
+          new AppConfigurationEntry(getKrb5LoginModuleName(),
+              AppConfigurationEntry.LoginModuleControlFlag.REQUIRED,
+              clientOptions)};
+      if(serverPrincipal!=null && serverKeytab!=null) {
+        Map<String, String> serverOptions = new HashMap(clientOptions);
+        serverOptions.put("principal", serverPrincipal);
+        serverOptions.put("keytab", serverKeytab.getAbsolutePath());
+        serverEntry =  new AppConfigurationEntry[]{
+            new AppConfigurationEntry(getKrb5LoginModuleName(),
+                AppConfigurationEntry.LoginModuleControlFlag.REQUIRED,
+                serverOptions)};
+      }
+    }
+
+    /**
+     * Add an entry to the jaas configuration with the passed in principal and keytab,
+     * along with the app name.
+     *
+     * @param principal The principal
+     * @param keytab The keytab containing credentials for the principal
+     * @param appName The app name of the configuration
+     */
+    public JaasConfiguration(String principal, File keytab, String appName) {
+      this(principal, keytab, null, null);
+      clientAppName = appName;
+      serverAppName = null;
+    }
+
+    @Override
+    public AppConfigurationEntry[] getAppConfigurationEntry(String name) {
+      if (name.equals(clientAppName)) {
+        return clientEntry;
+      } else if (name.equals(serverAppName)) {
+        return serverEntry;
+      }
+      return null;
+    }
+
+    private String getKrb5LoginModuleName() {
+      String krb5LoginModuleName;
+      if (System.getProperty("java.vendor").contains("IBM")) {
+        krb5LoginModuleName = "com.ibm.security.auth.module.Krb5LoginModule";
+      } else {
+        krb5LoginModuleName = "com.sun.security.auth.module.Krb5LoginModule";
+      }
+      return krb5LoginModuleName;
+    }
+  }
+
+  /**
+   *  These Locales don't generate dates that are compatibile with Hadoop MiniKdc.
+   */
+  private final static List<String> brokenLanguagesWithMiniKdc =
+      Arrays.asList(
+          new Locale("th").getLanguage(),
+          new Locale("ja").getLanguage(),
+          new Locale("hi").getLanguage()
+      );
+
+  public static class Builder {
+    private File kdcWorkDir;
+    private String clientPrincipal;
+    private File clientKeytab;
+    private String serverPrincipal;
+    private File serverKeytab;
+    private String appName;
+    private Locale savedLocale;
+
+    public Builder() {
+      savedLocale = Locale.getDefault();
+    }
+
+    public Builder withKdc(File kdcWorkDir) {
+      this.kdcWorkDir = kdcWorkDir;
+      return this;
+    }
+
+    public Builder withJaasConfiguration(String clientPrincipal, File clientKeytab,
+                                         String serverPrincipal, File serverKeytab) {
+      Preconditions.checkNotNull(clientPrincipal);
+      Preconditions.checkNotNull(clientKeytab);
+      this.clientPrincipal = clientPrincipal;
+      this.clientKeytab = clientKeytab;
+      this.serverPrincipal = serverPrincipal;
+      this.serverKeytab = serverKeytab;
+      this.appName = null;
+      return this;
+    }
+
+    public Builder withJaasConfiguration(String principal, File keytab, String appName) {
+      Preconditions.checkNotNull(principal);
+      Preconditions.checkNotNull(keytab);
+      this.clientPrincipal = principal;
+      this.clientKeytab = keytab;
+      this.serverPrincipal = null;
+      this.serverKeytab = null;
+      this.appName = appName;
+      return this;
+    }
+
+    public KerberosTestServices build() throws Exception {
+      final MiniKdc kdc = kdcWorkDir != null ? getKdc(kdcWorkDir) : null;
+      final Configuration oldConfig = clientPrincipal != null ? Configuration.getConfiguration() : null;
+      JaasConfiguration jaasConfiguration = null;
+      if (clientPrincipal != null) {
+        jaasConfiguration = (appName == null) ?
+            new JaasConfiguration(clientPrincipal, clientKeytab, serverPrincipal, serverKeytab) :
+            new JaasConfiguration(clientPrincipal, clientKeytab, appName);
+      }
+      return new KerberosTestServices(kdc, jaasConfiguration, oldConfig, savedLocale);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7bf019a9/solr/core/src/test/org/apache/solr/cloud/KerberosTestUtil.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/KerberosTestUtil.java b/solr/core/src/test/org/apache/solr/cloud/KerberosTestUtil.java
deleted file mode 100644
index 7f544ef..0000000
--- a/solr/core/src/test/org/apache/solr/cloud/KerberosTestUtil.java
+++ /dev/null
@@ -1,147 +0,0 @@
-/*
- * 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.cloud;
-
-import java.io.File;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Locale;
-import java.util.Map;
-import java.util.Properties;
-
-import javax.security.auth.login.AppConfigurationEntry;
-import javax.security.auth.login.Configuration;
-
-import org.apache.hadoop.minikdc.MiniKdc;
-
-public class KerberosTestUtil {
-
-  /**
-   * Returns a MiniKdc that can be used for creating kerberos principals
-   * and keytabs.  Caller is responsible for starting/stopping the kdc.
-   */
-  public static MiniKdc getKdc(File workDir) throws Exception {
-    Properties conf = MiniKdc.createConf();
-    return new MiniKdc(conf, workDir);
-  }
-
-  /**
-   * Programmatic version of a jaas.conf file suitable for connecting
-   * to a SASL-configured zookeeper.
-   */
-  public static class JaasConfiguration extends Configuration {
-
-    private static AppConfigurationEntry[] clientEntry;
-    private static AppConfigurationEntry[] serverEntry;
-    private String clientAppName = "Client", serverAppName = "Server";
-
-    /**
-     * Add an entry to the jaas configuration with the passed in name,
-     * principal, and keytab. The other necessary options will be set for you.
-     *
-     * @param clientPrincipal The principal of the client
-     * @param clientKeytab The location of the keytab with the clientPrincipal
-     * @param serverPrincipal The principal of the server
-     * @param serverKeytab The location of the keytab with the serverPrincipal
-     */
-    public JaasConfiguration(String clientPrincipal, File clientKeytab,
-        String serverPrincipal, File serverKeytab) {
-      Map<String, String> clientOptions = new HashMap();
-      clientOptions.put("principal", clientPrincipal);
-      clientOptions.put("keyTab", clientKeytab.getAbsolutePath());
-      clientOptions.put("useKeyTab", "true");
-      clientOptions.put("storeKey", "true");
-      clientOptions.put("useTicketCache", "false");
-      clientOptions.put("refreshKrb5Config", "true");
-      String jaasProp = System.getProperty("solr.jaas.debug");
-      if (jaasProp != null && "true".equalsIgnoreCase(jaasProp)) {
-        clientOptions.put("debug", "true");
-      }
-      clientEntry = new AppConfigurationEntry[]{
-        new AppConfigurationEntry(getKrb5LoginModuleName(),
-        AppConfigurationEntry.LoginModuleControlFlag.REQUIRED,
-        clientOptions)};
-      if(serverPrincipal!=null && serverKeytab!=null) {
-        Map<String, String> serverOptions = new HashMap(clientOptions);
-        serverOptions.put("principal", serverPrincipal);
-        serverOptions.put("keytab", serverKeytab.getAbsolutePath());
-        serverEntry =  new AppConfigurationEntry[]{
-            new AppConfigurationEntry(getKrb5LoginModuleName(),
-                AppConfigurationEntry.LoginModuleControlFlag.REQUIRED,
-                serverOptions)};
-      }
-    }
-
-    /**
-     * Add an entry to the jaas configuration with the passed in principal and keytab, 
-     * along with the app name.
-     * 
-     * @param principal The principal
-     * @param keytab The keytab containing credentials for the principal
-     * @param appName The app name of the configuration
-     */
-    public JaasConfiguration(String principal, File keytab, String appName) {
-      this(principal, keytab, null, null);
-      clientAppName = appName;
-      serverAppName = null;
-    }
-
-    @Override
-    public AppConfigurationEntry[] getAppConfigurationEntry(String name) {
-      if (name.equals(clientAppName)) {
-        return clientEntry;
-      } else if (name.equals(serverAppName)) {
-        return serverEntry;
-      }
-      return null;
-    }
-
-    private String getKrb5LoginModuleName() {
-      String krb5LoginModuleName;
-      if (System.getProperty("java.vendor").contains("IBM")) {
-        krb5LoginModuleName = "com.ibm.security.auth.module.Krb5LoginModule";
-      } else {
-        krb5LoginModuleName = "com.sun.security.auth.module.Krb5LoginModule";
-      }
-      return krb5LoginModuleName;
-    }
-  }
-
-  /**
-   *  These Locales don't generate dates that are compatibile with Hadoop MiniKdc.
-   */
-  private final static List<String> brokenLanguagesWithMiniKdc =
-      Arrays.asList(
-          new Locale("th").getLanguage(), 
-          new Locale("ja").getLanguage(), 
-          new Locale("hi").getLanguage()
-          );
-  /** 
-   *returns the currently set locale, and overrides it with {@link Locale#US} if it's 
-   * currently something MiniKdc can not handle
-   *
-   * @see Locale#setDefault
-   */
-  public static final Locale overrideLocaleIfNotSpportedByMiniKdc() {
-    Locale old = Locale.getDefault();
-    if (brokenLanguagesWithMiniKdc.contains(Locale.getDefault().getLanguage())) {
-      Locale.setDefault(Locale.US);
-    }
-    return old;
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7bf019a9/solr/core/src/test/org/apache/solr/cloud/OutOfBoxZkACLAndCredentialsProvidersTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/OutOfBoxZkACLAndCredentialsProvidersTest.java b/solr/core/src/test/org/apache/solr/cloud/OutOfBoxZkACLAndCredentialsProvidersTest.java
index 51ad523..0884576 100644
--- a/solr/core/src/test/org/apache/solr/cloud/OutOfBoxZkACLAndCredentialsProvidersTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/OutOfBoxZkACLAndCredentialsProvidersTest.java
@@ -23,6 +23,7 @@ import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.common.cloud.SecurityAwareZkACLProvider;
 import org.apache.solr.common.cloud.SolrZkClient;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.ZooDefs;
@@ -77,6 +78,7 @@ public class OutOfBoxZkACLAndCredentialsProvidersTest extends SolrTestCaseJ4 {
     zkClient.makePath("/protectedMakePathNode", "content".getBytes(DATA_ENCODING), CreateMode.PERSISTENT, false);
     zkClient.create("/unprotectedCreateNode", "content".getBytes(DATA_ENCODING), CreateMode.PERSISTENT, false);
     zkClient.makePath("/unprotectedMakePathNode", "content".getBytes(DATA_ENCODING), CreateMode.PERSISTENT, false);
+    zkClient.create(SecurityAwareZkACLProvider.SECURITY_ZNODE_PATH, "content".getBytes(DATA_ENCODING), CreateMode.PERSISTENT, false);
     zkClient.close();
 
     log.info("####SETUP_END " + getTestName());
@@ -93,7 +95,9 @@ public class OutOfBoxZkACLAndCredentialsProvidersTest extends SolrTestCaseJ4 {
   public void testOutOfBoxSolrZkClient() throws Exception {
     SolrZkClient zkClient = new SolrZkClient(zkServer.getZkAddress(), AbstractZkTestCase.TIMEOUT);
     try {
-      VMParamsZkACLAndCredentialsProvidersTest.doTest(zkClient, true, true, true, true, true);
+      VMParamsZkACLAndCredentialsProvidersTest.doTest(zkClient,
+          true, true, true, true, true,
+          true, true, true, true, true);
     } finally {
       zkClient.close();
     }
@@ -110,6 +114,7 @@ public class OutOfBoxZkACLAndCredentialsProvidersTest extends SolrTestCaseJ4 {
       assertTrue(verifiedList.contains("/solr/unprotectedMakePathNode"));
       assertTrue(verifiedList.contains("/solr/protectedMakePathNode"));
       assertTrue(verifiedList.contains("/solr/protectedCreateNode"));
+      assertTrue(verifiedList.contains("/solr" + SecurityAwareZkACLProvider.SECURITY_ZNODE_PATH));
     } finally {
       zkClient.close();
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7bf019a9/solr/core/src/test/org/apache/solr/cloud/OverriddenZkACLAndCredentialsProvidersTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/OverriddenZkACLAndCredentialsProvidersTest.java b/solr/core/src/test/org/apache/solr/cloud/OverriddenZkACLAndCredentialsProvidersTest.java
index b87ab1b..56c0df9 100644
--- a/solr/core/src/test/org/apache/solr/cloud/OverriddenZkACLAndCredentialsProvidersTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/OverriddenZkACLAndCredentialsProvidersTest.java
@@ -18,18 +18,15 @@ package org.apache.solr.cloud;
 
 import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.common.StringUtils;
-import org.apache.solr.common.cloud.DefaultZkACLProvider;
 import org.apache.solr.common.cloud.DefaultZkCredentialsProvider;
+import org.apache.solr.common.cloud.SecurityAwareZkACLProvider;
 import org.apache.solr.common.cloud.SolrZkClient;
 import org.apache.solr.common.cloud.VMParamsAllAndReadonlyDigestZkACLProvider;
 import org.apache.solr.common.cloud.VMParamsSingleSetCredentialsDigestZkCredentialsProvider;
 import org.apache.solr.common.cloud.ZkACLProvider;
 import org.apache.solr.common.cloud.ZkCredentialsProvider;
 import org.apache.zookeeper.CreateMode;
-import org.apache.zookeeper.ZooDefs;
 import org.apache.zookeeper.data.ACL;
-import org.apache.zookeeper.data.Id;
-import org.apache.zookeeper.server.auth.DigestAuthenticationProvider;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -40,7 +37,6 @@ import java.io.File;
 import java.io.UnsupportedEncodingException;
 import java.lang.invoke.MethodHandles;
 import java.nio.charset.Charset;
-import java.security.NoSuchAlgorithmException;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
@@ -88,6 +84,7 @@ public class OverriddenZkACLAndCredentialsProvidersTest extends SolrTestCaseJ4 {
         "readonlyACLUsername", "readonlyACLPassword").getSolrZkClient(zkServer.getZkAddress(), AbstractZkTestCase.TIMEOUT);
     zkClient.create("/protectedCreateNode", "content".getBytes(DATA_ENCODING), CreateMode.PERSISTENT, false);
     zkClient.makePath("/protectedMakePathNode", "content".getBytes(DATA_ENCODING), CreateMode.PERSISTENT, false);
+    zkClient.create(SecurityAwareZkACLProvider.SECURITY_ZNODE_PATH, "content".getBytes(DATA_ENCODING), CreateMode.PERSISTENT, false);
     zkClient.close();
     
     zkClient = new SolrZkClientFactoryUsingCompletelyNewProviders(null, null, 
@@ -114,7 +111,9 @@ public class OverriddenZkACLAndCredentialsProvidersTest extends SolrTestCaseJ4 {
     SolrZkClient zkClient = new SolrZkClientFactoryUsingCompletelyNewProviders(null, null, 
         null, null).getSolrZkClient(zkServer.getZkAddress(), AbstractZkTestCase.TIMEOUT);
     try {
-      VMParamsZkACLAndCredentialsProvidersTest.doTest(zkClient, false, false, false, false, false);
+      VMParamsZkACLAndCredentialsProvidersTest.doTest(zkClient,
+          false, false, false, false, false,
+          false, false, false, false, false);
     } finally {
       zkClient.close();
     }
@@ -125,7 +124,9 @@ public class OverriddenZkACLAndCredentialsProvidersTest extends SolrTestCaseJ4 {
     SolrZkClient zkClient = new SolrZkClientFactoryUsingCompletelyNewProviders("connectAndAllACLUsername", "connectAndAllACLPasswordWrong", 
         null, null).getSolrZkClient(zkServer.getZkAddress(), AbstractZkTestCase.TIMEOUT);
     try {
-      VMParamsZkACLAndCredentialsProvidersTest.doTest(zkClient, false, false, false, false, false);
+      VMParamsZkACLAndCredentialsProvidersTest.doTest(zkClient,
+          false, false, false, false, false,
+          false, false, false, false, false);
     } finally {
       zkClient.close();
     }
@@ -136,7 +137,9 @@ public class OverriddenZkACLAndCredentialsProvidersTest extends SolrTestCaseJ4 {
     SolrZkClient zkClient = new SolrZkClientFactoryUsingCompletelyNewProviders("connectAndAllACLUsername", "connectAndAllACLPassword", 
         null, null).getSolrZkClient(zkServer.getZkAddress(), AbstractZkTestCase.TIMEOUT);
     try {
-      VMParamsZkACLAndCredentialsProvidersTest.doTest(zkClient, true, true, true, true, true);
+      VMParamsZkACLAndCredentialsProvidersTest.doTest(zkClient,
+          true, true, true, true, true,
+          true, true, true, true, true);
     } finally {
       zkClient.close();
     }
@@ -147,7 +150,9 @@ public class OverriddenZkACLAndCredentialsProvidersTest extends SolrTestCaseJ4 {
     SolrZkClient zkClient = new SolrZkClientFactoryUsingCompletelyNewProviders("readonlyACLUsername", "readonlyACLPassword",
         null, null).getSolrZkClient(zkServer.getZkAddress(), AbstractZkTestCase.TIMEOUT);
     try {
-      VMParamsZkACLAndCredentialsProvidersTest.doTest(zkClient, true, true, false, false, false);
+      VMParamsZkACLAndCredentialsProvidersTest.doTest(zkClient,
+          true, true, false, false, false,
+          false, false, false, false, false);
     } finally {
       zkClient.close();
     }
@@ -159,7 +164,9 @@ public class OverriddenZkACLAndCredentialsProvidersTest extends SolrTestCaseJ4 {
     
     SolrZkClient zkClient = new SolrZkClientUsingVMParamsProvidersButWithDifferentVMParamsNames(zkServer.getZkAddress(), AbstractZkTestCase.TIMEOUT);
     try {
-      VMParamsZkACLAndCredentialsProvidersTest.doTest(zkClient, false, false, false, false, false);
+      VMParamsZkACLAndCredentialsProvidersTest.doTest(zkClient,
+          false, false, false, false, false,
+          false, false, false, false, false);
     } finally {
       zkClient.close();
     }
@@ -171,7 +178,9 @@ public class OverriddenZkACLAndCredentialsProvidersTest extends SolrTestCaseJ4 {
     
     SolrZkClient zkClient = new SolrZkClientUsingVMParamsProvidersButWithDifferentVMParamsNames(zkServer.getZkAddress(), AbstractZkTestCase.TIMEOUT);
     try {
-      VMParamsZkACLAndCredentialsProvidersTest.doTest(zkClient, false, false, false, false, false);
+      VMParamsZkACLAndCredentialsProvidersTest.doTest(zkClient,
+          false, false, false, false, false,
+          false, false, false, false, false);
     } finally {
       zkClient.close();
     }
@@ -183,7 +192,9 @@ public class OverriddenZkACLAndCredentialsProvidersTest extends SolrTestCaseJ4 {
     
     SolrZkClient zkClient = new SolrZkClientUsingVMParamsProvidersButWithDifferentVMParamsNames(zkServer.getZkAddress(), AbstractZkTestCase.TIMEOUT);
     try {
-      VMParamsZkACLAndCredentialsProvidersTest.doTest(zkClient, true, true, true, true, true);
+      VMParamsZkACLAndCredentialsProvidersTest.doTest(zkClient,
+          true, true, true, true, true,
+          true, true, true, true, true);
     } finally {
       zkClient.close();
     }
@@ -195,7 +206,9 @@ public class OverriddenZkACLAndCredentialsProvidersTest extends SolrTestCaseJ4 {
     
     SolrZkClient zkClient = new SolrZkClientUsingVMParamsProvidersButWithDifferentVMParamsNames(zkServer.getZkAddress(), AbstractZkTestCase.TIMEOUT);
     try {
-      VMParamsZkACLAndCredentialsProvidersTest.doTest(zkClient, true, true, false, false, false);
+      VMParamsZkACLAndCredentialsProvidersTest.doTest(zkClient,
+          true, true, false, false, false,
+          false, false, false, false, false);
     } finally {
       zkClient.close();
     }
@@ -240,28 +253,18 @@ public class OverriddenZkACLAndCredentialsProvidersTest extends SolrTestCaseJ4 {
 
         @Override
         public ZkACLProvider createZkACLProvider() {
-          return new DefaultZkACLProvider() {
+          return new VMParamsAllAndReadonlyDigestZkACLProvider() {
             @Override
-            protected List<ACL> createGlobalACLsToAdd() {
-              try {
-                List<ACL> result = new ArrayList<ACL>();
-            
-                if (!StringUtils.isEmpty(digestUsername) && !StringUtils.isEmpty(digestPassword)) {
-                  result.add(new ACL(ZooDefs.Perms.ALL, new Id("digest", DigestAuthenticationProvider.generateDigest(digestUsername + ":" + digestPassword))));
-                }
-            
-                if (!StringUtils.isEmpty(digestReadonlyUsername) && !StringUtils.isEmpty(digestReadonlyPassword)) {
-                  result.add(new ACL(ZooDefs.Perms.READ, new Id("digest", DigestAuthenticationProvider.generateDigest(digestReadonlyUsername + ":" + digestReadonlyPassword))));
-                }
-                
-                if (result.isEmpty()) {
-                  result = ZooDefs.Ids.OPEN_ACL_UNSAFE;
-                }
-                
-                return result;
-              } catch (NoSuchAlgorithmException e) {
-                throw new RuntimeException(e);
-              }
+            protected List<ACL> createNonSecurityACLsToAdd() {
+              return createACLsToAdd(true, digestUsername, digestPassword, digestReadonlyUsername, digestReadonlyPassword);
+            }
+
+            /**
+             * @return Set of ACLs to return security-related znodes
+             */
+            @Override
+            protected List<ACL> createSecurityACLsToAdd() {
+              return createACLsToAdd(false, digestUsername, digestPassword, digestReadonlyUsername, digestReadonlyPassword);
             }
           };
         }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7bf019a9/solr/core/src/test/org/apache/solr/cloud/SaslZkACLProviderTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/SaslZkACLProviderTest.java b/solr/core/src/test/org/apache/solr/cloud/SaslZkACLProviderTest.java
index 9381c03..16b67a3 100644
--- a/solr/core/src/test/org/apache/solr/cloud/SaslZkACLProviderTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/SaslZkACLProviderTest.java
@@ -20,15 +20,12 @@ import java.io.File;
 import java.io.IOException;
 import java.lang.invoke.MethodHandles;
 import java.nio.charset.Charset;
-import java.util.Locale;
 
-import javax.security.auth.login.Configuration;
-
-import org.apache.hadoop.minikdc.MiniKdc;
 import org.apache.lucene.util.Constants;
 import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.common.cloud.DefaultZkACLProvider;
 import org.apache.solr.common.cloud.SaslZkACLProvider;
+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.util.BadZookeeperThreadsFilter;
@@ -50,8 +47,6 @@ public class SaslZkACLProviderTest extends SolrTestCaseJ4 {
 
   private static final Charset DATA_ENCODING = Charset.forName("UTF-8");
 
-  protected Locale savedLocale = null;
-
   protected ZkTestServer zkServer;
 
   @BeforeClass
@@ -71,7 +66,6 @@ public class SaslZkACLProviderTest extends SolrTestCaseJ4 {
   @Override
   public void setUp() throws Exception {
     super.setUp();
-    savedLocale = KerberosTestUtil.overrideLocaleIfNotSpportedByMiniKdc();
     log.info("####SETUP_START " + getTestName());
     createTempDir();
 
@@ -99,6 +93,7 @@ public class SaslZkACLProviderTest extends SolrTestCaseJ4 {
     try {
       zkClient.create("/protectedCreateNode", "content".getBytes(DATA_ENCODING), CreateMode.PERSISTENT, false);
       zkClient.makePath("/protectedMakePathNode", "content".getBytes(DATA_ENCODING), CreateMode.PERSISTENT, false);
+      zkClient.create(SecurityAwareZkACLProvider.SECURITY_ZNODE_PATH, "content".getBytes(DATA_ENCODING), CreateMode.PERSISTENT, false);
     } finally {
       zkClient.close();
     }
@@ -115,7 +110,6 @@ public class SaslZkACLProviderTest extends SolrTestCaseJ4 {
   @Override
   public void tearDown() throws Exception {
     zkServer.shutdown();
-    Locale.setDefault(savedLocale);
     super.tearDown();
   }
 
@@ -124,7 +118,9 @@ public class SaslZkACLProviderTest extends SolrTestCaseJ4 {
     // Test with Sasl enabled
     SolrZkClient zkClient = new SolrZkClientWithACLs(zkServer.getZkAddress(), AbstractZkTestCase.TIMEOUT);
     try {
-      VMParamsZkACLAndCredentialsProvidersTest.doTest(zkClient, true, true, true, true, true);
+      VMParamsZkACLAndCredentialsProvidersTest.doTest(zkClient,
+          true, true, true, true, true,
+          true, true, true, true, true);
      } finally {
       zkClient.close();
     }
@@ -134,7 +130,9 @@ public class SaslZkACLProviderTest extends SolrTestCaseJ4 {
     System.setProperty("zookeeper.sasl.client", "false");
     zkClient = new SolrZkClientNoACLs(zkServer.getZkAddress(), AbstractZkTestCase.TIMEOUT);
     try {
-      VMParamsZkACLAndCredentialsProvidersTest.doTest(zkClient, true, true, false, false, false);
+      VMParamsZkACLAndCredentialsProvidersTest.doTest(zkClient,
+          true, true, false, false, false,
+          false, false, false, false, false);
     } finally {
       zkClient.close();
       System.clearProperty("zookeeper.sasl.client");
@@ -176,8 +174,7 @@ public class SaslZkACLProviderTest extends SolrTestCaseJ4 {
    */
   public static class SaslZkTestServer extends ZkTestServer {
     private String kdcDir;
-    private MiniKdc kdc;
-    private Configuration conf;
+    private KerberosTestServices kerberosTestServices;
 
     public SaslZkTestServer(String zkDir, String kdcDir) {
       super(zkDir);
@@ -187,13 +184,11 @@ public class SaslZkACLProviderTest extends SolrTestCaseJ4 {
     public SaslZkTestServer(String zkDir, int port, String kdcDir) {
       super(zkDir, port);
       this.kdcDir = kdcDir;
-      conf = Configuration.getConfiguration();
     }
 
     @Override
     public void run() throws InterruptedException {
       try {
-        kdc = KerberosTestUtil.getKdc(new File(kdcDir));
         // Don't require that credentials match the entire principal string, e.g.
         // can match "solr" rather than "solr/host@DOMAIN"
         System.setProperty("zookeeper.kerberos.removeRealmFromPrincipal", "true");
@@ -202,12 +197,13 @@ public class SaslZkACLProviderTest extends SolrTestCaseJ4 {
         String zkClientPrincipal = "solr";
         String zkServerPrincipal = "zookeeper/127.0.0.1";
 
-        kdc.start();
-        // Create ZK client and server principals and load them into the Configuration
-        kdc.createPrincipal(keytabFile, zkClientPrincipal, zkServerPrincipal);
-        KerberosTestUtil.JaasConfiguration jaas = new KerberosTestUtil.JaasConfiguration(
-        zkClientPrincipal, keytabFile, zkServerPrincipal, keytabFile);
-        Configuration.setConfiguration(jaas);
+        kerberosTestServices = KerberosTestServices.builder()
+            .withKdc(new File(kdcDir))
+            .withJaasConfiguration(zkClientPrincipal, keytabFile, zkServerPrincipal, keytabFile)
+            .build();
+        kerberosTestServices.start();
+
+        kerberosTestServices.getKdc().createPrincipal(keytabFile, zkClientPrincipal, zkServerPrincipal);
       } catch (Exception ex) {
         throw new RuntimeException(ex);
       }
@@ -220,8 +216,7 @@ public class SaslZkACLProviderTest extends SolrTestCaseJ4 {
       System.clearProperty("zookeeper.authProvider.1");
       System.clearProperty("zookeeper.kerberos.removeRealmFromPrincipal");
       System.clearProperty("zookeeper.kerberos.removeHostFromPrincipal");
-      Configuration.setConfiguration(conf);
-      kdc.stop();
+      kerberosTestServices.stop();
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7bf019a9/solr/core/src/test/org/apache/solr/cloud/TestAuthenticationFramework.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestAuthenticationFramework.java b/solr/core/src/test/org/apache/solr/cloud/TestAuthenticationFramework.java
index 08db692..6b562a1 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestAuthenticationFramework.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestAuthenticationFramework.java
@@ -235,21 +235,23 @@ public class TestAuthenticationFramework extends LuceneTestCase {
     public void init(Map<String,Object> pluginConfig) {}
 
     @Override
-    public void doAuthenticate(ServletRequest request, ServletResponse response, FilterChain filterChain)
+    public boolean doAuthenticate(ServletRequest request, ServletResponse response, FilterChain filterChain)
         throws Exception {
       if (expectedUsername == null) {
         filterChain.doFilter(request, response);
-        return;
+        return true;
       }
       HttpServletRequest httpRequest = (HttpServletRequest)request;
       String username = httpRequest.getHeader("username");
       String password = httpRequest.getHeader("password");
       
       log.info("Username: "+username+", password: "+password);
-      if(MockAuthenticationPlugin.expectedUsername.equals(username) && MockAuthenticationPlugin.expectedPassword.equals(password))      
+      if(MockAuthenticationPlugin.expectedUsername.equals(username) && MockAuthenticationPlugin.expectedPassword.equals(password)) {
         filterChain.doFilter(request, response);
-      else {
+        return true;
+      } else {
         ((HttpServletResponse)response).sendError(401, "Unauthorized request");
+        return false;
       }
     }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7bf019a9/solr/core/src/test/org/apache/solr/cloud/TestMiniSolrCloudClusterKerberos.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestMiniSolrCloudClusterKerberos.java b/solr/core/src/test/org/apache/solr/cloud/TestMiniSolrCloudClusterKerberos.java
index 37439b0..d644967 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestMiniSolrCloudClusterKerberos.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestMiniSolrCloudClusterKerberos.java
@@ -16,16 +16,13 @@
  */
 package org.apache.solr.cloud;
 
-import javax.security.auth.login.Configuration;
 import java.io.File;
 import java.nio.charset.StandardCharsets;
-import java.util.Locale;
 
 import com.carrotsearch.randomizedtesting.annotations.ThreadLeakFilters;
 import com.carrotsearch.randomizedtesting.rules.SystemPropertiesRestoreRule;
 
 import org.apache.commons.io.FileUtils;
-import org.apache.hadoop.minikdc.MiniKdc;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.LuceneTestCase.SuppressSysoutChecks;
 import org.apache.solr.util.BadZookeeperThreadsFilter;
@@ -52,17 +49,14 @@ import org.junit.rules.TestRule;
 @SuppressSysoutChecks(bugUrl = "Solr logs to JUL")
 public class TestMiniSolrCloudClusterKerberos extends TestMiniSolrCloudCluster {
 
-  private final Configuration originalConfig = Configuration.getConfiguration();
-
   public TestMiniSolrCloudClusterKerberos () {
     NUM_SERVERS = 5;
     NUM_SHARDS = 2;
     REPLICATION_FACTOR = 2;
   }
   
-  private MiniKdc kdc;
+  private KerberosTestServices kerberosTestServices;
 
-  private Locale savedLocale; // in case locale is broken and we need to fill in a working locale
   @Rule
   public TestRule solrTestRules = RuleChain
       .outerRule(new SystemPropertiesRestoreRule());
@@ -74,20 +68,22 @@ public class TestMiniSolrCloudClusterKerberos extends TestMiniSolrCloudCluster {
 
   @Override
   public void setUp() throws Exception {
-    savedLocale = KerberosTestUtil.overrideLocaleIfNotSpportedByMiniKdc();
     super.setUp();
     setupMiniKdc();
   }
   
   private void setupMiniKdc() throws Exception {
     String kdcDir = createTempDir()+File.separator+"minikdc";
-    kdc = KerberosTestUtil.getKdc(new File(kdcDir));
     File keytabFile = new File(kdcDir, "keytabs");
     String principal = "HTTP/127.0.0.1";
     String zkServerPrincipal = "zookeeper/127.0.0.1";
+    KerberosTestServices kerberosTestServices = KerberosTestServices.builder()
+        .withKdc(new File(kdcDir))
+        .withJaasConfiguration(principal, keytabFile, zkServerPrincipal, keytabFile)
+        .build();
 
-    kdc.start();
-    kdc.createPrincipal(keytabFile, principal, zkServerPrincipal);
+    kerberosTestServices.start();
+    kerberosTestServices.getKdc().createPrincipal(keytabFile, principal, zkServerPrincipal);
 
     String jaas = "Client {\n"
         + " com.sun.security.auth.module.Krb5LoginModule required\n"
@@ -109,10 +105,7 @@ public class TestMiniSolrCloudClusterKerberos extends TestMiniSolrCloudCluster {
         + " debug=true\n"
         + " principal=\""+zkServerPrincipal+"\";\n" 
         + "};\n";
-    
-    Configuration conf = new KerberosTestUtil.JaasConfiguration(principal, keytabFile, zkServerPrincipal, keytabFile);
-    javax.security.auth.login.Configuration.setConfiguration(conf);
-    
+
     String jaasFilePath = kdcDir+File.separator + "jaas-client.conf";
     FileUtils.write(new File(jaasFilePath), jaas, StandardCharsets.UTF_8);
     System.setProperty("java.security.auth.login.config", jaasFilePath);
@@ -156,11 +149,7 @@ public class TestMiniSolrCloudClusterKerberos extends TestMiniSolrCloudCluster {
     System.clearProperty("kerberos.principal");
     System.clearProperty("kerberos.keytab");
     System.clearProperty("authenticationPlugin");
-    Configuration.setConfiguration(this.originalConfig);
-    if (kdc != null) {
-      kdc.stop();
-    }
-    Locale.setDefault(savedLocale);
+    kerberosTestServices.stop();
     super.tearDown();
   }
 }