You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ja...@apache.org on 2017/02/22 21:37:35 UTC

lucene-solr:branch_6x: SOLR-9481: Authentication and Authorization plugins now support standalone mode

Repository: lucene-solr
Updated Branches:
  refs/heads/branch_6x 5d76917cf -> b1ac6ddcf


SOLR-9481: Authentication and Authorization plugins now support standalone mode


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

Branch: refs/heads/branch_6x
Commit: b1ac6ddcf2f1027806f04a6af0e5a51f01334113
Parents: 5d76917
Author: Jan H�ydahl <ja...@apache.org>
Authored: Wed Feb 22 22:35:51 2017 +0100
Committer: Jan H�ydahl <ja...@apache.org>
Committed: Wed Feb 22 22:35:51 2017 +0100

----------------------------------------------------------------------
 solr/CHANGES.txt                                |   4 +
 .../org/apache/solr/core/CoreContainer.java     |  32 +--
 .../solr/handler/admin/SecurityConfHandler.java | 148 +++++++++----
 .../handler/admin/SecurityConfHandlerLocal.java | 104 +++++++++
 .../handler/admin/SecurityConfHandlerZk.java    |  92 ++++++++
 .../apache/solr/security/BasicAuthPlugin.java   |  17 +-
 .../security/Sha256AuthenticationProvider.java  |   2 +-
 .../SecurityConfHandlerLocalForTesting.java     |  39 ++++
 .../handler/admin/SecurityConfHandlerTest.java  |  66 +++---
 .../solr/security/BasicAuthStandaloneTest.java  | 221 +++++++++++++++++++
 .../solr/client/solrj/impl/HttpClientUtil.java  |   8 +-
 11 files changed, 637 insertions(+), 96 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b1ac6ddc/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 712f959..757fb72 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -74,6 +74,10 @@ New Features
   field must both be stored=false, indexed=false, docValues=true. (Ishan Chattopadhyaya, hossman, noble,
   shalin, yonik)
 
+* SOLR-9481: Authentication and Authorization plugins now work in standalone mode if security.json is placed in
+  SOLR_HOME on every node. Editing config through API is supported but affects only that one node.
+  This feature should be considered experimental for this release and may not work with SSL yet. (janhoy)
+
 Bug Fixes
 ----------------------
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b1ac6ddc/solr/core/src/java/org/apache/solr/core/CoreContainer.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/CoreContainer.java b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
index c92936c..a160612 100644
--- a/solr/core/src/java/org/apache/solr/core/CoreContainer.java
+++ b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
@@ -39,7 +39,6 @@ import java.util.concurrent.Future;
 import com.codahale.metrics.Gauge;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Maps;
-
 import org.apache.lucene.index.CorruptIndexException;
 import org.apache.lucene.store.Directory;
 import org.apache.solr.client.solrj.impl.HttpClientConfigurer;
@@ -52,7 +51,6 @@ import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;
 import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.cloud.Replica.State;
-import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.util.ExecutorUtil;
 import org.apache.solr.common.util.IOUtils;
 import org.apache.solr.common.util.Utils;
@@ -67,6 +65,8 @@ import org.apache.solr.handler.admin.CoreAdminHandler;
 import org.apache.solr.handler.admin.InfoHandler;
 import org.apache.solr.handler.admin.MetricsHandler;
 import org.apache.solr.handler.admin.SecurityConfHandler;
+import org.apache.solr.handler.admin.SecurityConfHandlerLocal;
+import org.apache.solr.handler.admin.SecurityConfHandlerZk;
 import org.apache.solr.handler.admin.ZookeeperInfoHandler;
 import org.apache.solr.handler.component.HttpShardHandlerFactory;
 import org.apache.solr.handler.component.ShardHandlerFactory;
@@ -90,7 +90,6 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import static java.util.Objects.requireNonNull;
-import static java.util.Collections.EMPTY_MAP;
 import static org.apache.solr.common.params.CommonParams.AUTHC_PATH;
 import static org.apache.solr.common.params.CommonParams.AUTHZ_PATH;
 import static org.apache.solr.common.params.CommonParams.COLLECTIONS_HANDLER_PATH;
@@ -316,10 +315,10 @@ public class CoreContainer {
     }
 
     if (pluginClassName != null) {
-      log.info("Authentication plugin class obtained from ZK: "+pluginClassName);
+      log.debug("Authentication plugin class obtained from security.json: "+pluginClassName);
     } else if (System.getProperty(AUTHENTICATION_PLUGIN_PROP) != null) {
       pluginClassName = System.getProperty(AUTHENTICATION_PLUGIN_PROP);
-      log.info("Authentication plugin class obtained from system property '" +
+      log.debug("Authentication plugin class obtained from system property '" +
           AUTHENTICATION_PLUGIN_PROP + "': " + pluginClassName);
     } else {
       log.debug("No authentication plugin used.");
@@ -329,6 +328,7 @@ public class CoreContainer {
 
     // Initialize the plugin
     if (pluginClassName != null) {
+      log.info("Initializing authentication plugin: " + pluginClassName);
       authenticationPlugin = new SecurityPluginHolder<>(readVersion(authenticationConfig),
           getResourceLoader().newInstance(pluginClassName,
               AuthenticationPlugin.class,
@@ -480,10 +480,8 @@ public class CoreContainer {
 
     MDCLoggingContext.setNode(this);
 
-    ZkStateReader.ConfigData securityConfig = isZooKeeperAware() ? getZkController().getZkStateReader().getSecurityProps(false) : new ZkStateReader.ConfigData(EMPTY_MAP, -1);
-    initializeAuthorizationPlugin((Map<String, Object>) securityConfig.data.get("authorization"));
-    initializeAuthenticationPlugin((Map<String, Object>) securityConfig.data.get("authentication"));
-
+    securityConfHandler = isZooKeeperAware() ? new SecurityConfHandlerZk(this) : new SecurityConfHandlerLocal(this);
+    reloadSecurityProperties();
     this.backupRepoFactory = new BackupRepositoryFactory(cfg.getBackupRepositoryPlugins());
 
     createHandler(ZK_PATH, ZookeeperInfoHandler.class.getName(), ZookeeperInfoHandler.class);
@@ -493,7 +491,6 @@ public class CoreContainer {
     configSetsHandler = createHandler(CONFIGSETS_HANDLER_PATH, cfg.getConfigSetsHandlerClass(), ConfigSetsHandler.class);
     metricsHandler = createHandler(METRICS_PATH, MetricsHandler.class.getName(), MetricsHandler.class);
 
-    securityConfHandler = new SecurityConfHandler(this);
     containerHandlers.put(AUTHZ_PATH, securityConfHandler);
     securityConfHandler.initializeMetrics(metricManager, SolrInfoMBean.Group.node.toString(), AUTHZ_PATH);
     containerHandlers.put(AUTHC_PATH, securityConfHandler);
@@ -602,10 +599,17 @@ public class CoreContainer {
   }
 
   public void securityNodeChanged() {
-    log.info("Security node changed");
-    ZkStateReader.ConfigData securityConfig = getZkController().getZkStateReader().getSecurityProps(false);
-    initializeAuthorizationPlugin((Map<String, Object>) securityConfig.data.get("authorization"));
-    initializeAuthenticationPlugin((Map<String, Object>) securityConfig.data.get("authentication"));
+    log.info("Security node changed, reloading security.json");
+    reloadSecurityProperties();
+  }
+
+  /**
+   * Make sure securityConfHandler is initialized
+   */
+  private void reloadSecurityProperties() {
+    SecurityConfHandler.SecurityConfig securityConfig = securityConfHandler.getSecurityConfig(false);
+    initializeAuthorizationPlugin((Map<String, Object>) securityConfig.getData().get("authorization"));
+    initializeAuthenticationPlugin((Map<String, Object>) securityConfig.getData().get("authentication"));
   }
 
   private static void checkForDuplicateCoreNames(List<CoreDescriptor> cds) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b1ac6ddc/solr/core/src/java/org/apache/solr/handler/admin/SecurityConfHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/SecurityConfHandler.java b/solr/core/src/java/org/apache/solr/handler/admin/SecurityConfHandler.java
index 9f09de8..22fa9ce 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/SecurityConfHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/SecurityConfHandler.java
@@ -17,6 +17,8 @@
 package org.apache.solr.handler.admin;
 
 import java.io.IOException;
+import java.io.InputStream;
+import java.lang.invoke.MethodHandles;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
@@ -31,7 +33,6 @@ import org.apache.solr.api.ApiBag;
 import org.apache.solr.api.ApiBag.ReqHandlerToApi;
 import org.apache.solr.api.SpecProvider;
 import org.apache.solr.common.SolrException;
-import org.apache.solr.common.cloud.ZkStateReader.ConfigData;
 import org.apache.solr.common.params.CommonParams;
 import org.apache.solr.common.util.Utils;
 import org.apache.solr.core.CoreContainer;
@@ -46,10 +47,14 @@ import org.apache.solr.security.ConfigEditablePlugin;
 import org.apache.solr.security.PermissionNameProvider;
 import org.apache.solr.util.CommandOperation;
 import org.apache.solr.util.JsonSchemaValidator;
-import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
-public class SecurityConfHandler extends RequestHandlerBase implements PermissionNameProvider {
-  private CoreContainer cores;
+import static org.apache.solr.common.SolrException.ErrorCode.SERVER_ERROR;
+
+public abstract class SecurityConfHandler extends RequestHandlerBase implements PermissionNameProvider {
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+  protected CoreContainer cores;
 
   public SecurityConfHandler(CoreContainer coreContainer) {
     this.cores = coreContainer;
@@ -101,11 +106,12 @@ public class SecurityConfHandler extends RequestHandlerBase implements Permissio
     if (ops == null) {
       throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "No commands");
     }
-    for (; ; ) {
-      ConfigData data = getSecurityProps(true);
-      Map<String, Object> latestConf = (Map<String, Object>) data.data.get(key);
+    for (int count = 1; count <= 3 ; count++ ) {
+      SecurityConfig securityConfig = getSecurityConfig(true);
+      Map<String, Object> data = securityConfig.getData();
+      Map<String, Object> latestConf = (Map<String, Object>) data.get(key);
       if (latestConf == null) {
-        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "No configuration present for " + key);
+        throw new SolrException(SERVER_ERROR, "No configuration present for " + key);
       }
       List<CommandOperation> commandsCopy = CommandOperation.clone(ops);
       Map<String, Object> out = configEditablePlugin.edit(Utils.getDeepCopy(latestConf, 4) , commandsCopy);
@@ -115,20 +121,31 @@ public class SecurityConfHandler extends RequestHandlerBase implements Permissio
           rsp.add(CommandOperation.ERR_MSGS, errs);
           return;
         }
-        //no edits
+        log.debug("No edits made");
         return;
       } else {
         if(!Objects.equals(latestConf.get("class") , out.get("class"))){
-          throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "class cannot be modified");
+          throw new SolrException(SERVER_ERROR, "class cannot be modified");
         }
         Map meta = getMapValue(out, "");
-        meta.put("v", data.version+1);//encode the expected zkversion
-        data.data.put(key, out);
-        if(persistConf("/security.json", Utils.toJSON(data.data), data.version)) return;
+        meta.put("v", securityConfig.getVersion()+1);//encode the expected zkversion
+        data.put(key, out);
+        
+        if(persistConf(securityConfig)) {
+          securityConfEdited();
+          return;
+        }
       }
+      log.debug("Security edit operation failed {} time(s)" + count);
     }
+    throw new SolrException(SERVER_ERROR, "Failed to persist security config after 3 attempts. Giving up");
   }
 
+  /**
+   * Hook where you can do stuff after a config has been edited. Defaults to NOP
+   */
+  protected void securityConfEdited() {}
+
   Object getPlugin(String key) {
     Object plugin = null;
     if ("authentication".equals(key)) plugin = cores.getAuthenticationPlugin();
@@ -136,38 +153,14 @@ public class SecurityConfHandler extends RequestHandlerBase implements Permissio
     return plugin;
   }
 
-  ConfigData getSecurityProps(boolean getFresh) {
-    return cores.getZkController().getZkStateReader().getSecurityProps(getFresh);
-  }
-
-  boolean persistConf(String path,  byte[] buf, int version) {
-    try {
-      cores.getZkController().getZkClient().setData(path,buf,version, true);
-      return true;
-    } catch (KeeperException.BadVersionException bdve){
-      return false;
-    } catch (Exception e) {
-      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, " Unable to persist conf",e);
-    }
-  }
-
-
-  private void getConf(SolrQueryResponse rsp, String key) {
-    ConfigData map = cores.getZkController().getZkStateReader().getSecurityProps(false);
-    Object o = map == null ? null : map.data.get(key);
-    if (o == null) {
-      rsp.add(CommandOperation.ERR_MSGS, Collections.singletonList("No " + key + " configured"));
-    } else {
-      rsp.add(key+".enabled", getPlugin(key)!=null);
-      rsp.add(key, o);
-    }
-  }
+  protected abstract void getConf(SolrQueryResponse rsp, String key);
 
   public static Map<String, Object> getMapValue(Map<String, Object> lookupMap, String key) {
     Map<String, Object> m = (Map<String, Object>) lookupMap.get(key);
     if (m == null) lookupMap.put(key, m = new LinkedHashMap<>());
     return m;
   }
+
   public static List getListValue(Map<String, Object> lookupMap, String key) {
     List l = (List) lookupMap.get(key);
     if (l == null) lookupMap.put(key, l= new ArrayList());
@@ -245,5 +238,82 @@ public class SecurityConfHandler extends RequestHandlerBase implements Permissio
   public Boolean registerV2() {
     return Boolean.TRUE;
   }
+  
+  /**
+   * Gets security.json from source
+   */
+  public abstract SecurityConfig getSecurityConfig(boolean getFresh);
+
+  /**
+   * Persist security.json to the source, optionally with a version
+   */
+  protected abstract boolean persistConf(SecurityConfig securityConfig) throws IOException;
+
+  /**
+   * Object to hold security.json as nested <code>Map&lt;String,Object&gt;</code> and optionally its version.
+   * The version property is optional and defaults to -1 if not initialized.
+   * The data object defaults to EMPTY_MAP if not set
+   */
+  public static class SecurityConfig {
+    private Map<String, Object> data = Collections.EMPTY_MAP;
+    private int version = -1;
+
+    public SecurityConfig() {}
+
+    /**
+     * Sets the data as a Map
+     * @param data a Map
+     * @return SecurityConf object (builder pattern)
+     */
+    public SecurityConfig setData(Map<String, Object> data) {
+      this.data = data;
+      return this;
+    }
+
+    /**
+     * Sets the data as an Object, but the object needs to be of type Map
+     * @param data an Object of type Map&lt;String,Object&gt;
+     * @return SecurityConf object (builder pattern)
+     */
+    public SecurityConfig setData(Object data) {
+      if (data instanceof Map) {
+        this.data = (Map<String, Object>) data;
+        return this;
+      } else {
+        throw new SolrException(SERVER_ERROR, "Illegal format when parsing security.json, not object");
+      }
+    }
+
+    /**
+     * Sets version
+     * @param version integer for version. Depends on underlying storage
+     * @return SecurityConf object (builder pattern)
+     */
+    public SecurityConfig setVersion(int version) {
+      this.version = version;
+      return this;
+    }
+
+    public Map<String, Object> getData() {
+      return data;
+    }
+
+    public int getVersion() {
+      return version;
+    }
+
+    /**
+     * Set data from input stream
+     * @param securityJsonInputStream an input stream for security.json
+     * @return this (builder pattern)
+     */
+    public SecurityConfig setData(InputStream securityJsonInputStream) {
+      return setData(Utils.fromJSON(securityJsonInputStream));
+    }
+
+    public String toString() {
+      return "SecurityConfig: version=" + version + ", data=" + Utils.toJSONString(data);
+    } 
+  }
 }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b1ac6ddc/solr/core/src/java/org/apache/solr/handler/admin/SecurityConfHandlerLocal.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/SecurityConfHandlerLocal.java b/solr/core/src/java/org/apache/solr/handler/admin/SecurityConfHandlerLocal.java
new file mode 100644
index 0000000..852d501
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/handler/admin/SecurityConfHandlerLocal.java
@@ -0,0 +1,104 @@
+/*
+ * 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.handler.admin;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.lang.invoke.MethodHandles;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Collections;
+
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.util.Utils;
+import org.apache.solr.core.CoreContainer;
+import org.apache.solr.response.SolrQueryResponse;
+import org.apache.solr.util.CommandOperation;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Security Configuration Handler which works on standalone local files
+ */
+public class SecurityConfHandlerLocal extends SecurityConfHandler {
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+  protected Path securityJsonPath;
+  
+  public SecurityConfHandlerLocal(CoreContainer coreContainer) {
+    super(coreContainer);
+    securityJsonPath = Paths.get(coreContainer.getSolrHome()).resolve("security.json");
+  }
+
+  /**
+   * Fetches security props from SOLR_HOME
+   * @param getFresh NOP
+   * @return SecurityConfig whose data property either contains security.json, or an empty map if not found
+   */
+  @Override
+  public SecurityConfig getSecurityConfig(boolean getFresh) {
+    if (Files.exists(securityJsonPath)) {
+      try (InputStream securityJsonIs = Files.newInputStream(securityJsonPath)) {
+        return new SecurityConfig().setData(securityJsonIs);
+      } catch (Exception e) { 
+        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Failed opening existing security.json file: " + securityJsonPath, e);
+      }
+    }
+    return new SecurityConfig();
+  }
+
+  @Override
+  protected void getConf(SolrQueryResponse rsp, String key) {
+    SecurityConfig props = getSecurityConfig(false);
+    Object o = props.getData().get(key);
+    if (o == null) {
+      rsp.add(CommandOperation.ERR_MSGS, Collections.singletonList("No " + key + " configured"));
+    } else {
+      rsp.add(key+".enabled", getPlugin(key)!=null);
+      rsp.add(key, o);
+    }
+  }
+  
+  @Override
+  protected boolean persistConf(SecurityConfig securityConfig) throws IOException {
+    if (securityConfig == null || securityConfig.getData().isEmpty()) {
+      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, 
+          "Failed persisting security.json to SOLR_HOME. Object was empty.");
+    }
+    try(OutputStream securityJsonOs = Files.newOutputStream(securityJsonPath)) {
+      securityJsonOs.write(Utils.toJSON(securityConfig.getData()));
+      log.debug("Persisted security.json to {}", securityJsonPath);
+      return true;
+    } catch (Exception e) {
+      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, 
+          "Failed persisting security.json to " + securityJsonPath, e);
+    }
+  }
+
+  @Override
+  public String getDescription() {
+    return "Edit or read security configuration locally in SOLR_HOME";
+  }
+
+  @Override
+  protected void securityConfEdited() {
+    // Need to call explicitly since we will not get notified of changes to local security.json
+    cores.securityNodeChanged();
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b1ac6ddc/solr/core/src/java/org/apache/solr/handler/admin/SecurityConfHandlerZk.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/SecurityConfHandlerZk.java b/solr/core/src/java/org/apache/solr/handler/admin/SecurityConfHandlerZk.java
new file mode 100644
index 0000000..8323b8a
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/handler/admin/SecurityConfHandlerZk.java
@@ -0,0 +1,92 @@
+/*
+ * 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.handler.admin;
+
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.util.Collections;
+
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.common.util.Utils;
+import org.apache.solr.core.CoreContainer;
+import org.apache.solr.response.SolrQueryResponse;
+import org.apache.solr.util.CommandOperation;
+import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.solr.common.SolrException.ErrorCode.SERVER_ERROR;
+import static org.apache.solr.common.cloud.ZkStateReader.SOLR_SECURITY_CONF_PATH;
+
+/**
+ * Security Configuration Handler which works with Zookeeper
+ */
+public class SecurityConfHandlerZk extends SecurityConfHandler {
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+  public SecurityConfHandlerZk(CoreContainer coreContainer) {
+    super(coreContainer);
+  }
+
+  /**
+   * Fetches security props from Zookeeper and adds version
+   * @param getFresh refresh from ZK
+   * @return SecurityConfig whose data property either contains security.json, or an empty map if not found
+   */
+  @Override
+  public SecurityConfig getSecurityConfig(boolean getFresh) {
+    ZkStateReader.ConfigData configDataFromZk = cores.getZkController().getZkStateReader().getSecurityProps(getFresh);
+    return configDataFromZk == null ? 
+        new SecurityConfig() :
+        new SecurityConfig().setData(configDataFromZk.data).setVersion(configDataFromZk.version);
+  }
+
+  @Override
+  protected void getConf(SolrQueryResponse rsp, String key) {
+    ZkStateReader.ConfigData map = cores.getZkController().getZkStateReader().getSecurityProps(false);
+    Object o = map == null ? null : map.data.get(key);
+    if (o == null) {
+      rsp.add(CommandOperation.ERR_MSGS, Collections.singletonList("No " + key + " configured"));
+    } else {
+      rsp.add(key+".enabled", getPlugin(key)!=null);
+      rsp.add(key, o);
+    }
+  }
+  
+  @Override
+  protected boolean persistConf(SecurityConfig securityConfig) throws IOException {
+    try {
+      cores.getZkController().getZkClient().setData(SOLR_SECURITY_CONF_PATH, 
+          Utils.toJSON(securityConfig.getData()), 
+          securityConfig.getVersion(), true);
+      log.debug("Persisted security.json to {}", SOLR_SECURITY_CONF_PATH);
+      return true;
+    } catch (KeeperException.BadVersionException bdve){
+      log.warn("Failed persisting security.json to {}", SOLR_SECURITY_CONF_PATH, bdve);
+      return false;
+    } catch (Exception e) {
+      throw new SolrException(SERVER_ERROR, "Unable to persist security.json", e);
+    }
+  }
+  
+  @Override
+  public String getDescription() {
+    return "Edit or read security configuration from Zookeeper";
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b1ac6ddc/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 833c345..f1665c7 100644
--- a/solr/core/src/java/org/apache/solr/security/BasicAuthPlugin.java
+++ b/solr/core/src/java/org/apache/solr/security/BasicAuthPlugin.java
@@ -45,12 +45,12 @@ import org.slf4j.LoggerFactory;
 
 public class BasicAuthPlugin extends AuthenticationPlugin implements ConfigEditablePlugin , SpecProvider {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-  private AuthenticationProvider zkAuthentication;
+  private AuthenticationProvider authenticationProvider;
   private final static ThreadLocal<Header> authHeader = new ThreadLocal<>();
   private boolean blockUnknown = false;
 
   public boolean authenticate(String username, String pwd) {
-    return zkAuthentication.authenticate(username, pwd);
+    return authenticationProvider.authenticate(username, pwd);
   }
 
   @Override
@@ -63,7 +63,7 @@ public class BasicAuthPlugin extends AuthenticationPlugin implements ConfigEdita
         log.error(e.getMessage());
       }
     }
-    zkAuthentication = getAuthenticationProvider(pluginConfig);
+    authenticationProvider = getAuthenticationProvider(pluginConfig);
   }
 
   @Override
@@ -81,8 +81,8 @@ public class BasicAuthPlugin extends AuthenticationPlugin implements ConfigEdita
       }
     }
     if (!CommandOperation.captureErrors(commands).isEmpty()) return null;
-    if (zkAuthentication instanceof ConfigEditablePlugin) {
-      ConfigEditablePlugin editablePlugin = (ConfigEditablePlugin) zkAuthentication;
+    if (authenticationProvider instanceof ConfigEditablePlugin) {
+      ConfigEditablePlugin editablePlugin = (ConfigEditablePlugin) authenticationProvider;
       return editablePlugin.edit(latestConf, commands);
     }
     throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "This cannot be edited");
@@ -95,7 +95,7 @@ public class BasicAuthPlugin extends AuthenticationPlugin implements ConfigEdita
   }
 
   private void authenticationFailure(HttpServletResponse response, String message) throws IOException {
-    for (Map.Entry<String, String> entry : zkAuthentication.getPromptHeaders().entrySet()) {
+    for (Map.Entry<String, String> entry : authenticationProvider.getPromptHeaders().entrySet()) {
       response.setHeader(entry.getKey(), entry.getValue());
     }
     response.sendError(401, message);
@@ -121,6 +121,7 @@ public class BasicAuthPlugin extends AuthenticationPlugin implements ConfigEdita
               final String username = credentials.substring(0, p).trim();
               String pwd = credentials.substring(p + 1).trim();
               if (!authenticate(username, pwd)) {
+                log.debug("Bad auth credentials supplied in Authorization header");
                 authenticationFailure(response, "Bad credentials");
               } else {
                 HttpServletRequestWrapper wrapper = new HttpServletRequestWrapper(request) {
@@ -145,7 +146,7 @@ public class BasicAuthPlugin extends AuthenticationPlugin implements ConfigEdita
       if (blockUnknown) {
         authenticationFailure(response, "require authentication");
       } else {
-        request.setAttribute(AuthenticationPlugin.class.getName(), zkAuthentication.getPromptHeaders());
+        request.setAttribute(AuthenticationPlugin.class.getName(), authenticationProvider.getPromptHeaders());
         filterChain.doFilter(request, response);
         return true;
       }
@@ -173,7 +174,7 @@ public class BasicAuthPlugin extends AuthenticationPlugin implements ConfigEdita
 
   @Override
   public ValidatingJsonMap getSpec() {
-    return zkAuthentication.getSpec();
+    return authenticationProvider.getSpec();
   }
   public boolean getBlockUnknown(){
     return blockUnknown;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b1ac6ddc/solr/core/src/java/org/apache/solr/security/Sha256AuthenticationProvider.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/security/Sha256AuthenticationProvider.java b/solr/core/src/java/org/apache/solr/security/Sha256AuthenticationProvider.java
index 7ce6b8e..0cc58cd 100644
--- a/solr/core/src/java/org/apache/solr/security/Sha256AuthenticationProvider.java
+++ b/solr/core/src/java/org/apache/solr/security/Sha256AuthenticationProvider.java
@@ -67,7 +67,7 @@ public class Sha256AuthenticationProvider implements ConfigEditablePlugin,  Basi
     credentials = new LinkedHashMap<>();
     Map<String,String> users = (Map<String,String>) pluginConfig.get("credentials");
     if (users == null) {
-      log.warn("No users configured yet");
+      log.debug("No users configured yet");
       return;
     }
     for (Map.Entry<String, String> e : users.entrySet()) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b1ac6ddc/solr/core/src/test/org/apache/solr/handler/admin/SecurityConfHandlerLocalForTesting.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/handler/admin/SecurityConfHandlerLocalForTesting.java b/solr/core/src/test/org/apache/solr/handler/admin/SecurityConfHandlerLocalForTesting.java
new file mode 100644
index 0000000..e56dd5b
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/handler/admin/SecurityConfHandlerLocalForTesting.java
@@ -0,0 +1,39 @@
+/*
+ * 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.handler.admin;
+
+import java.io.IOException;
+
+import org.apache.solr.core.CoreContainer;
+
+/**
+ * Wrapper for use in tests
+ */
+public class SecurityConfHandlerLocalForTesting extends SecurityConfHandlerLocal {
+  public SecurityConfHandlerLocalForTesting(CoreContainer coreContainer) {
+    super(coreContainer);
+  }
+
+  public boolean persistConf(SecurityConfig securityConfig) throws IOException {
+    return super.persistConf(securityConfig);
+  }
+  
+  public void securityConfEdited() {
+    super.securityConfEdited();
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b1ac6ddc/solr/core/src/test/org/apache/solr/handler/admin/SecurityConfHandlerTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/handler/admin/SecurityConfHandlerTest.java b/solr/core/src/test/org/apache/solr/handler/admin/SecurityConfHandlerTest.java
index 5e3d407..54c8587 100644
--- a/solr/core/src/test/org/apache/solr/handler/admin/SecurityConfHandlerTest.java
+++ b/solr/core/src/test/org/apache/solr/handler/admin/SecurityConfHandlerTest.java
@@ -23,7 +23,6 @@ import java.util.List;
 import java.util.Map;
 
 import org.apache.solr.SolrTestCaseJ4;
-import org.apache.solr.common.cloud.ZkStateReader.ConfigData;
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.util.ContentStreamBase;
 import org.apache.solr.common.util.Utils;
@@ -34,6 +33,7 @@ import org.apache.solr.security.RuleBasedAuthorizationPlugin;
 import org.apache.solr.util.CommandOperation;
 
 import static org.apache.solr.common.util.Utils.makeMap;
+import static org.apache.solr.handler.admin.SecurityConfHandler.SecurityConfig;
 
 public class SecurityConfHandlerTest extends SolrTestCaseJ4 {
 
@@ -51,8 +51,8 @@ public class SecurityConfHandlerTest extends SolrTestCaseJ4 {
     handler.handleRequestBody(req,new SolrQueryResponse());
 
     BasicAuthPlugin basicAuth = new BasicAuthPlugin();
-    ConfigData securityCfg = (ConfigData) handler.m.get("/security.json");
-    basicAuth.init((Map<String, Object>) securityCfg.data.get("authentication"));
+    SecurityConfig securityCfg = handler.m.get("/security.json");
+    basicAuth.init((Map<String, Object>) securityCfg.getData().get("authentication"));
     assertTrue(basicAuth.authenticate("tom", "TomIsUberCool"));
 
     command = "{\n" +
@@ -62,9 +62,9 @@ public class SecurityConfHandlerTest extends SolrTestCaseJ4 {
     o = new ContentStreamBase.ByteArrayStream(command.getBytes(StandardCharsets.UTF_8),"");
     req.setContentStreams(Collections.singletonList(o));
     handler.handleRequestBody(req,new SolrQueryResponse());
-    securityCfg = (ConfigData) handler.m.get("/security.json");
-    assertEquals(3, securityCfg.version);
-    Map result = (Map) securityCfg.data.get("authentication");
+    securityCfg = handler.m.get("/security.json");
+    assertEquals(3, securityCfg.getVersion());
+    Map result = (Map) securityCfg.getData().get("authentication");
     result = (Map) result.get("credentials");
     assertTrue(result.isEmpty());
 
@@ -86,7 +86,7 @@ public class SecurityConfHandlerTest extends SolrTestCaseJ4 {
     SolrQueryResponse rsp = new SolrQueryResponse();
     handler.handleRequestBody(req, rsp);
     assertNull(rsp.getValues().get(CommandOperation.ERR_MSGS));
-    Map authzconf = (Map) ((ConfigData) handler.m.get("/security.json")).data.get("authorization");
+    Map authzconf = (Map) handler.m.get("/security.json").getData().get("authorization");
     Map userRoles = (Map) authzconf.get("user-role");
     List tomRoles = (List) userRoles.get("tom");
     assertTrue(tomRoles.contains("admin"));
@@ -108,7 +108,7 @@ public class SecurityConfHandlerTest extends SolrTestCaseJ4 {
     req.setContentStreams(Collections.singletonList(o));
     rsp = new SolrQueryResponse();
     handler.handleRequestBody(req, rsp);
-    authzconf = (Map) ((ConfigData) handler.m.get("/security.json")).data.get("authorization");
+    authzconf = (Map) handler.m.get("/security.json").getData().get("authorization");
     permissions = (List<Map>) authzconf.get("permissions");
 
     Map p = permissions.get(1);
@@ -128,7 +128,7 @@ public class SecurityConfHandlerTest extends SolrTestCaseJ4 {
     req.setContentStreams(Collections.singletonList(o));
     rsp = new SolrQueryResponse();
     handler.handleRequestBody(req, rsp);
-    authzconf = (Map) ((ConfigData) handler.m.get("/security.json")).data.get("authorization");
+    authzconf = (Map) handler.m.get("/security.json").getData().get("authorization");
     permissions = (List<Map>) authzconf.get("permissions");
 
     p = permissions.get(0);
@@ -151,7 +151,7 @@ public class SecurityConfHandlerTest extends SolrTestCaseJ4 {
     rsp = new SolrQueryResponse();
     handler.handleRequestBody(req, rsp);
     assertNull(rsp.getValues().get(CommandOperation.ERR_MSGS));
-    authzconf = (Map) ((ConfigData) handler.m.get("/security.json")).data.get("authorization");
+    authzconf = (Map) handler.m.get("/security.json").getData().get("authorization");
     userRoles = (Map) authzconf.get("user-role");
     assertEquals(0, userRoles.size());
     permissions = (List<Map>) authzconf.get("permissions");
@@ -178,25 +178,26 @@ public class SecurityConfHandlerTest extends SolrTestCaseJ4 {
 
 
   public static class MockSecurityHandler extends SecurityConfHandler {
-    private Map<String, Object> m;
+    private Map<String, SecurityConfig> m;
     final BasicAuthPlugin basicAuthPlugin = new BasicAuthPlugin();
     final RuleBasedAuthorizationPlugin rulesBasedAuthorizationPlugin = new RuleBasedAuthorizationPlugin();
 
 
-     public MockSecurityHandler() {
+    public MockSecurityHandler() {
       super(null);
       m = new HashMap<>();
-      ConfigData data = new ConfigData(makeMap("authentication", makeMap("class", "solr."+ BasicAuthPlugin.class.getSimpleName())), 1);
-      data.data.put("authorization", makeMap("class", "solr."+RuleBasedAuthorizationPlugin.class.getSimpleName()));
-      m.put("/security.json", data);
-
+      SecurityConfig sp = new SecurityConfig();
+      sp.setData(makeMap("authentication", makeMap("class", "solr."+ BasicAuthPlugin.class.getSimpleName())));
+      sp.setVersion(1);
+      sp.getData().put("authorization", makeMap("class", "solr."+RuleBasedAuthorizationPlugin.class.getSimpleName()));
+      m.put("/security.json", sp);
 
       basicAuthPlugin.init(new HashMap<>());
 
       rulesBasedAuthorizationPlugin.init(new HashMap<>());
     }
 
-    public Map<String, Object> getM() {
+    public Map<String, SecurityConfig> getM() {
       return m;
     }
 
@@ -212,24 +213,25 @@ public class SecurityConfHandlerTest extends SolrTestCaseJ4 {
     }
 
     @Override
-    ConfigData getSecurityProps(boolean getFresh) {
-      return (ConfigData) m.get("/security.json");
+    protected void getConf(SolrQueryResponse rsp, String key) {
+      // NOP
+    }
+
+    @Override
+    public SecurityConfig getSecurityConfig(boolean getFresh) {
+      return m.get("/security.json");
     }
 
     @Override
-    boolean persistConf(String key, byte[] buf, int version) {
-      Object data = m.get(key);
-      if (data instanceof ConfigData) {
-        ConfigData configData = (ConfigData) data;
-        if (configData.version == version) {
-          ConfigData result = new ConfigData((Map<String, Object>) Utils.fromJSON(buf), version + 1);
-          m.put(key, result);
-          return true;
-        } else {
-          return false;
-        }
+    protected boolean persistConf(SecurityConfig props) {
+      SecurityConfig fromMap = m.get("/security.json");
+      if (fromMap.getVersion() == props.getVersion()) {
+        props.setVersion(props.getVersion()+1);
+        m.put("/security.json", props);
+        return true;
+      } else {
+        return false;
       }
-      throw new RuntimeException();
     }
 
 
@@ -254,7 +256,7 @@ public class SecurityConfHandlerTest extends SolrTestCaseJ4 {
       req.setContentStreams(Collections.singletonList(o));
       SolrQueryResponse rsp = new SolrQueryResponse();
       handleRequestBody(req, rsp);
-      Map<String, Object> data = ((ConfigData) m.get("/security.json")).data;
+      Map<String, Object> data = m.get("/security.json").getData();
       ((Map)data.get("authentication")).remove("");
       ((Map)data.get("authorization")).remove("");
       return Utils.toJSONString (data);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b1ac6ddc/solr/core/src/test/org/apache/solr/security/BasicAuthStandaloneTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/security/BasicAuthStandaloneTest.java b/solr/core/src/test/org/apache/solr/security/BasicAuthStandaloneTest.java
new file mode 100644
index 0000000..7d6c436
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/security/BasicAuthStandaloneTest.java
@@ -0,0 +1,221 @@
+/*
+ * 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.lang.invoke.MethodHandles;
+import java.nio.charset.Charset;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Collections;
+import java.util.Properties;
+
+import org.apache.http.HttpResponse;
+import org.apache.http.client.HttpClient;
+import org.apache.http.client.methods.HttpPost;
+import org.apache.http.entity.ByteArrayEntity;
+import org.apache.http.message.AbstractHttpMessage;
+import org.apache.http.message.BasicHeader;
+import org.apache.solr.client.solrj.SolrRequest;
+import org.apache.solr.client.solrj.embedded.JettySolrRunner;
+import org.apache.solr.client.solrj.impl.HttpClientUtil;
+import org.apache.solr.client.solrj.impl.HttpSolrClient;
+import org.apache.solr.client.solrj.request.GenericSolrRequest;
+import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.common.util.Base64;
+import org.apache.solr.common.util.ContentStreamBase;
+import org.apache.solr.common.util.Utils;
+import org.apache.solr.handler.admin.SecurityConfHandler;
+import org.apache.solr.handler.admin.SecurityConfHandlerLocalForTesting;
+import org.apache.solr.util.AbstractSolrTestCase;
+import org.apache.solr.util.LogLevel;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.solr.security.BasicAuthIntegrationTest.NOT_NULL_PREDICATE;
+import static org.apache.solr.security.BasicAuthIntegrationTest.STD_CONF;
+import static org.apache.solr.security.BasicAuthIntegrationTest.verifySecurityStatus;
+
+public class BasicAuthStandaloneTest extends AbstractSolrTestCase {
+
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+  private Path ROOT_DIR = Paths.get(getSolrHome());
+  private Path CONF_DIR = ROOT_DIR.resolve("configsets").resolve("configset-2").resolve("conf");
+
+  SecurityConfHandlerLocalForTesting securityConfHandler;
+  SolrInstance instance = null;
+  JettySolrRunner jetty;
+      
+  @Before
+  @Override
+  public void setUp() throws Exception
+  {
+    super.setUp();
+    instance = new SolrInstance("inst", null);
+    instance.setUp();
+    jetty = createJetty(instance);
+    securityConfHandler = new SecurityConfHandlerLocalForTesting(jetty.getCoreContainer());
+    HttpClientUtil.clearRequestInterceptors(); // Clear out any old Authorization headers
+  }
+
+  @Override
+  @After
+  public void tearDown() throws Exception {
+    jetty.stop();
+    super.tearDown();
+  }
+
+  @Test
+  @LogLevel("org.apache.solr=DEBUG")
+  public void testBasicAuth() throws Exception {
+
+    String authcPrefix = "/admin/authentication";
+
+    HttpClient cl = null;
+    HttpSolrClient httpSolrClient = null;
+    try {
+      cl = HttpClientUtil.createClient(null);
+      String baseUrl = buildUrl(jetty.getLocalPort(), "/solr"); 
+      httpSolrClient = getHttpSolrClient(baseUrl);
+      
+      verifySecurityStatus(cl, baseUrl + authcPrefix, "/errorMessages", null, 20);
+
+      // Write security.json locally. Should cause security to be initialized
+      securityConfHandler.persistConf(new SecurityConfHandler.SecurityConfig()
+          .setData(Utils.fromJSONString(STD_CONF.replaceAll("'", "\""))));
+      securityConfHandler.securityConfEdited();
+      verifySecurityStatus(cl, baseUrl + authcPrefix, "authentication/class", "solr.BasicAuthPlugin", 20);
+
+      String command = "{\n" +
+          "'set-user': {'harry':'HarryIsCool'}\n" +
+          "}";
+
+      GenericSolrRequest genericReq = new GenericSolrRequest(SolrRequest.METHOD.POST, authcPrefix, new ModifiableSolrParams());
+      genericReq.setContentStreams(Collections.singletonList(new ContentStreamBase.ByteArrayStream(command.getBytes(UTF_8), "")));
+
+      HttpSolrClient finalHttpSolrClient = httpSolrClient;
+      HttpSolrClient.RemoteSolrException exp = expectThrows(HttpSolrClient.RemoteSolrException.class, () -> {
+        finalHttpSolrClient.request(genericReq);
+      });
+      assertEquals(401, exp.code());
+
+      command = "{\n" +
+          "'set-user': {'harry':'HarryIsUberCool'}\n" +
+          "}";
+
+      HttpPost httpPost = new HttpPost(baseUrl + authcPrefix);
+      setBasicAuthHeader(httpPost, "solr", "SolrRocks");
+      httpPost.setEntity(new ByteArrayEntity(command.getBytes(UTF_8)));
+      httpPost.addHeader("Content-Type", "application/json; charset=UTF-8");
+      verifySecurityStatus(cl, baseUrl + authcPrefix, "authentication.enabled", "true", 20);
+      HttpResponse r = cl.execute(httpPost);
+      int statusCode = r.getStatusLine().getStatusCode();
+      Utils.consumeFully(r.getEntity());
+      assertEquals("proper_cred sent, but access denied", 200, statusCode);
+
+      verifySecurityStatus(cl, baseUrl + authcPrefix, "authentication/credentials/harry", NOT_NULL_PREDICATE, 20);
+
+      // Read file from SOLR_HOME and verify that it contains our new user
+      assertTrue(new String(Utils.toJSON(securityConfHandler.getSecurityConfig(false).getData()), 
+          Charset.forName("UTF-8")).contains("harry"));
+    } finally {
+      if (cl != null) {
+        HttpClientUtil.close(cl);
+        httpSolrClient.close();
+      }
+    }
+  }
+
+  public static void setBasicAuthHeader(AbstractHttpMessage httpMsg, String user, String pwd) {
+    String userPass = user + ":" + pwd;
+    String encoded = Base64.byteArrayToBase64(userPass.getBytes(UTF_8));
+    httpMsg.setHeader(new BasicHeader("Authorization", "Basic " + encoded));
+    log.info("Added Basic Auth security Header {}",encoded );
+  }
+
+  private JettySolrRunner createJetty(SolrInstance instance) throws Exception {
+    Properties nodeProperties = new Properties();
+    nodeProperties.setProperty("solr.data.dir", instance.getDataDir().toString());
+    JettySolrRunner jetty = new JettySolrRunner(instance.getHomeDir().toString(), nodeProperties, buildJettyConfig("/solr"));
+    jetty.start();
+    return jetty;
+  }
+  
+  
+  private class SolrInstance {
+    String name;
+    Integer port;
+    Path homeDir;
+    Path confDir;
+    Path dataDir;
+    
+    /**
+     * if masterPort is null, this instance is a master -- otherwise this instance is a slave, and assumes the master is
+     * on localhost at the specified port.
+     */
+    public SolrInstance(String name, Integer port) {
+      this.name = name;
+      this.port = port;
+    }
+
+    public Path getHomeDir() {
+      return homeDir;
+    }
+
+    public Path getSchemaFile() {
+      return CONF_DIR.resolve("schema.xml");
+    }
+
+    public Path getConfDir() {
+      return confDir;
+    }
+
+    public Path getDataDir() {
+      return dataDir;
+    }
+
+    public Path getSolrConfigFile() {
+      return CONF_DIR.resolve("solrconfig.xml");
+    }
+
+    public Path getSolrXmlFile() {
+      return ROOT_DIR.resolve("solr.xml");
+    }
+
+
+    public void setUp() throws Exception {
+      homeDir = createTempDir(name).toAbsolutePath();
+      dataDir = homeDir.resolve("collection1").resolve("data");
+      confDir = homeDir.resolve("collection1").resolve("conf");
+
+      Files.createDirectories(homeDir);
+      Files.createDirectories(dataDir);
+      Files.createDirectories(confDir);
+
+      Files.copy(getSolrXmlFile(), homeDir.resolve("solr.xml"));
+      Files.copy(getSolrConfigFile(), confDir.resolve("solrconfig.xml"));
+      Files.copy(getSchemaFile(), confDir.resolve("schema.xml"));
+
+      Files.createFile(homeDir.resolve("collection1").resolve("core.properties"));
+    }
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b1ac6ddc/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpClientUtil.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpClientUtil.java b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpClientUtil.java
index 2a66ad8..7726125 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpClientUtil.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpClientUtil.java
@@ -16,8 +16,6 @@
  */
 package org.apache.solr.client.solrj.impl;
 
-import static java.util.Collections.singletonList;
-
 import java.io.IOException;
 import java.io.InputStream;
 import java.lang.invoke.MethodHandles;
@@ -63,6 +61,8 @@ import org.apache.solr.common.params.SolrParams;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import static java.util.Collections.singletonList;
+
 /**
  * Utility class for creating/configuring httpclient instances. 
  */
@@ -224,6 +224,10 @@ public class HttpClientUtil {
     interceptors.remove(interceptor);
   }
 
+  public static void clearRequestInterceptors() {
+    interceptors.clear();
+  }
+
   /**
    * Control HTTP payload compression.
    *