You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by no...@apache.org on 2015/08/06 21:15:58 UTC

svn commit: r1694556 - in /lucene/dev/branches/branch_5x: ./ solr/ solr/core/ solr/core/src/java/org/apache/solr/cloud/ solr/core/src/java/org/apache/solr/core/ solr/core/src/java/org/apache/solr/handler/ solr/core/src/java/org/apache/solr/handler/admi...

Author: noble
Date: Thu Aug  6 19:15:57 2015
New Revision: 1694556

URL: http://svn.apache.org/r1694556
Log:
SOLR-7757: Improved security framework where security components can be edited/reloaded, Solr now watches /security.json. Components can choose to make their configs editable

Added:
    lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/handler/admin/SecurityConfHandler.java
      - copied unchanged from r1694552, lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/admin/SecurityConfHandler.java
    lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/security/ConfigEditablePlugin.java
      - copied unchanged from r1694552, lucene/dev/trunk/solr/core/src/java/org/apache/solr/security/ConfigEditablePlugin.java
    lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/security/SecurityPluginHolder.java
      - copied unchanged from r1694552, lucene/dev/trunk/solr/core/src/java/org/apache/solr/security/SecurityPluginHolder.java
    lucene/dev/branches/branch_5x/solr/solrj/src/java/org/apache/solr/common/Callable.java
      - copied unchanged from r1694552, lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/common/Callable.java
Modified:
    lucene/dev/branches/branch_5x/   (props changed)
    lucene/dev/branches/branch_5x/solr/   (props changed)
    lucene/dev/branches/branch_5x/solr/CHANGES.txt   (contents, props changed)
    lucene/dev/branches/branch_5x/solr/core/   (props changed)
    lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/cloud/ZkController.java
    lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/CoreContainer.java
    lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/handler/SolrConfigHandler.java
    lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/security/AuthenticationPlugin.java
    lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/security/AuthorizationContext.java
    lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/security/AuthorizationResponse.java
    lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java
    lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/util/CommandOperation.java
    lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/cloud/TestAuthenticationFramework.java
    lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/security/PKIAuthenticationIntegrationTest.java
    lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/security/TestAuthorizationFramework.java
    lucene/dev/branches/branch_5x/solr/solrj/   (props changed)
    lucene/dev/branches/branch_5x/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrClient.java
    lucene/dev/branches/branch_5x/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java
    lucene/dev/branches/branch_5x/solr/solrj/src/java/org/apache/solr/common/params/CollectionParams.java
    lucene/dev/branches/branch_5x/solr/solrj/src/java/org/apache/solr/common/util/Utils.java
    lucene/dev/branches/branch_5x/solr/test-framework/   (props changed)
    lucene/dev/branches/branch_5x/solr/test-framework/src/java/org/apache/solr/cloud/AbstractZkTestCase.java

Modified: lucene/dev/branches/branch_5x/solr/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/CHANGES.txt?rev=1694556&r1=1694555&r2=1694556&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/CHANGES.txt (original)
+++ lucene/dev/branches/branch_5x/solr/CHANGES.txt Thu Aug  6 19:15:57 2015
@@ -127,6 +127,10 @@ New Features
 
 * SOLR-7220: Nested C-style comments in queries. (yonik)
 
+* SOLR-7757: Improved security framework where security components can be edited/reloaded, Solr
+  now watches /security.json. Components can choose to make their config editable
+  (Noble Paul, Anshum Gupta, Ishan Chattopadhyaya)
+
 Bug Fixes
 ----------------------
 

Modified: lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/cloud/ZkController.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/cloud/ZkController.java?rev=1694556&r1=1694555&r2=1694556&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/cloud/ZkController.java (original)
+++ lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/cloud/ZkController.java Thu Aug  6 19:15:57 2015
@@ -382,7 +382,12 @@ public final class ZkController {
     this.overseerFailureMap = Overseer.getFailureMap(zkClient);
     cmdExecutor = new ZkCmdExecutor(clientTimeout);
     leaderElector = new LeaderElector(zkClient);
-    zkStateReader = new ZkStateReader(zkClient);
+    zkStateReader = new ZkStateReader(zkClient, new Runnable() {
+      @Override
+      public void run() {
+        if(cc!=null) cc.securityNodeChanged();
+      }
+    });
 
     this.baseURL = zkStateReader.getBaseUrlForNodeName(this.nodeName);
 
@@ -629,6 +634,7 @@ public final class ZkController {
     cmdExecutor.ensureExists(ZkStateReader.COLLECTIONS_ZKNODE, zkClient);
     cmdExecutor.ensureExists(ZkStateReader.ALIASES, zkClient);
     cmdExecutor.ensureExists(ZkStateReader.CLUSTER_STATE, zkClient);
+    cmdExecutor.ensureExists(ZkStateReader.SOLR_SECURITY_CONF_PATH,"{}".getBytes(StandardCharsets.UTF_8),CreateMode.PERSISTENT, zkClient);
   }
 
   private void init(CurrentCoreDescriptorProvider registerOnReconnect) {

Modified: lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/CoreContainer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/CoreContainer.java?rev=1694556&r1=1694555&r2=1694556&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/CoreContainer.java (original)
+++ lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/CoreContainer.java Thu Aug  6 19:15:57 2015
@@ -30,6 +30,7 @@ import java.util.concurrent.ConcurrentHa
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
 
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Maps;
@@ -39,12 +40,15 @@ import org.apache.solr.client.solrj.impl
 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.ZkStateReader;
 import org.apache.solr.common.util.ExecutorUtil;
 import org.apache.solr.common.util.IOUtils;
+import org.apache.solr.common.util.Utils;
 import org.apache.solr.handler.RequestHandlerBase;
 import org.apache.solr.handler.admin.CollectionsHandler;
 import org.apache.solr.handler.admin.CoreAdminHandler;
 import org.apache.solr.handler.admin.InfoHandler;
+import org.apache.solr.handler.admin.SecurityConfHandler;
 import org.apache.solr.handler.component.HttpShardHandlerFactory;
 import org.apache.solr.handler.component.ShardHandlerFactory;
 import org.apache.solr.logging.LogWatcher;
@@ -54,6 +58,7 @@ import org.apache.solr.security.Authoriz
 import org.apache.solr.security.AuthenticationPlugin;
 import org.apache.solr.security.HttpClientInterceptorPlugin;
 import org.apache.solr.security.PKIAuthenticationPlugin;
+import org.apache.solr.security.SecurityPluginHolder;
 import org.apache.solr.update.UpdateShardHandler;
 import org.apache.solr.util.DefaultSolrThreadFactory;
 import org.apache.solr.util.FileUtils;
@@ -62,6 +67,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import static com.google.common.base.Preconditions.checkNotNull;
+import static java.util.Collections.EMPTY_MAP;
 import static org.apache.solr.security.AuthenticationPlugin.AUTHENTICATION_PLUGIN_PROP;
 
 
@@ -75,10 +81,6 @@ public class CoreContainer {
 
   final SolrCores solrCores = new SolrCores(this);
 
-  protected AuthorizationPlugin authorizationPlugin;
-
-  protected AuthenticationPlugin authenticationPlugin;
-
   public static class CoreLoadFailure {
 
     public final CoreDescriptor cd;
@@ -132,6 +134,12 @@ public class CoreContainer {
 
   private boolean asyncSolrCoreLoad;
 
+  protected SecurityConfHandler securityConfHandler;
+
+  private SecurityPluginHolder<AuthorizationPlugin> authorizationPlugin;
+
+  private SecurityPluginHolder<AuthenticationPlugin> authenticationPlugin;
+
   public ExecutorService getCoreZkRegisterExecutorService() {
     return zkSys.getCoreZkRegisterExecutorService();
   }
@@ -211,42 +219,45 @@ public class CoreContainer {
     this.asyncSolrCoreLoad = asyncSolrCoreLoad;
   }
 
-  private void intializeAuthorizationPlugin() {
+  private synchronized void initializeAuthorizationPlugin(Map<String, Object> authorizationConf) {
+    authorizationConf = Utils.getDeepCopy(authorizationConf, 4);
     //Initialize the Authorization module
-    Map securityProps = getZkController().getZkStateReader().getSecurityProps();
-    if(securityProps != null) {
-      Map authorizationConf = (Map) securityProps.get("authorization");
-      if(authorizationConf == null) return;
+    SecurityPluginHolder<AuthorizationPlugin> old = authorizationPlugin;
+    SecurityPluginHolder<AuthorizationPlugin> authorizationPlugin = null;
+    if (authorizationConf != null) {
       String klas = (String) authorizationConf.get("class");
-      if(klas == null){
+      if (klas == null) {
         throw new SolrException(ErrorCode.SERVER_ERROR, "class is required for authorization plugin");
       }
+      if (old != null && old.getZnodeVersion() == readVersion(authorizationConf)) {
+        return;
+      }
       log.info("Initializing authorization plugin: " + klas);
-      authorizationPlugin = getResourceLoader().newInstance((String) klas,
-          AuthorizationPlugin.class);
+      authorizationPlugin = new SecurityPluginHolder<>(readVersion(authorizationConf),
+          getResourceLoader().newInstance(klas, AuthorizationPlugin.class));
 
       // Read and pass the authorization context to the plugin
-      authorizationPlugin.init(authorizationConf);
+      authorizationPlugin.plugin.init(authorizationConf);
     } else {
       log.info("Security conf doesn't exist. Skipping setup for authorization module.");
     }
+    this.authorizationPlugin = authorizationPlugin;
+    if (old != null) {
+      try {
+        old.plugin.close();
+      } catch (Exception e) {
+      }
+    }
   }
 
-  private void initializeAuthenticationPlugin() {
+  private synchronized void initializeAuthenticationPlugin(Map<String, Object> authenticationConfig) {
+    authenticationConfig = Utils.getDeepCopy(authenticationConfig, 4);
     String pluginClassName = null;
-    Map<String, Object> authenticationConfig = null;
-
-    if (isZooKeeperAware()) {
-      Map securityProps = getZkController().getZkStateReader().getSecurityProps();
-      if (securityProps != null) {
-        authenticationConfig = (Map<String, Object>) securityProps.get("authentication");
-        if (authenticationConfig!=null) {
-          if (authenticationConfig.containsKey("class")) {
-            pluginClassName = String.valueOf(authenticationConfig.get("class"));
-          } else {
-            throw new SolrException(ErrorCode.SERVER_ERROR, "No 'class' specified for authentication in ZK.");
-          }
-        }
+    if (authenticationConfig != null) {
+      if (authenticationConfig.containsKey("class")) {
+        pluginClassName = String.valueOf(authenticationConfig.get("class"));
+      } else {
+        throw new SolrException(ErrorCode.SERVER_ERROR, "No 'class' specified for authentication in ZK.");
       }
     }
 
@@ -259,15 +270,23 @@ public class CoreContainer {
     } else {
       log.info("No authentication plugin used.");
     }
+    SecurityPluginHolder<AuthenticationPlugin> old = authenticationPlugin;
+    SecurityPluginHolder<AuthenticationPlugin> authenticationPlugin = null;
 
     // Initialize the filter
     if (pluginClassName != null) {
-      authenticationPlugin = getResourceLoader().newInstance(pluginClassName, AuthenticationPlugin.class);
+      authenticationPlugin = new SecurityPluginHolder<>(readVersion(authenticationConfig),
+          getResourceLoader().newInstance(pluginClassName, AuthenticationPlugin.class));
     }
     if (authenticationPlugin != null) {
-      authenticationPlugin.init(authenticationConfig);
-      addHttpConfigurer(authenticationPlugin);
+      authenticationPlugin.plugin.init(authenticationConfig);
+      addHttpConfigurer(authenticationPlugin.plugin);
     }
+    this.authenticationPlugin = authenticationPlugin;
+    try {
+      if (old != null) old.plugin.close();
+    } catch (Exception e) {/*do nothing*/ }
+
   }
 
   private void addHttpConfigurer(Object authcPlugin) {
@@ -293,6 +312,14 @@ public class CoreContainer {
     }
   }
 
+  private static int readVersion(Map<String, Object> conf) {
+    if (conf == null) return -1;
+    Map meta = (Map) conf.get("");
+    if (meta == null) return -1;
+    Number v = (Number) meta.get("v");
+    return v == null ? -1 : v.intValue();
+  }
+
   /**
    * This method allows subclasses to construct a CoreContainer
    * without any default init behavior.
@@ -367,18 +394,19 @@ public class CoreContainer {
     zkSys.initZooKeeper(this, solrHome, cfg.getCloudConfig());
     if(isZooKeeperAware())  pkiAuthenticationPlugin = new PKIAuthenticationPlugin(this, zkSys.getZkController().getNodeName());
 
-    initializeAuthenticationPlugin();
-
-    if (isZooKeeperAware()) {
-      intializeAuthorizationPlugin();
-    }
+    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 = new SecurityConfHandler(this);
     collectionsHandler = createHandler(cfg.getCollectionsHandlerClass(), CollectionsHandler.class);
     containerHandlers.put(COLLECTIONS_HANDLER_PATH, collectionsHandler);
     infoHandler        = createHandler(cfg.getInfoHandlerClass(), InfoHandler.class);
     containerHandlers.put(INFO_HANDLER_PATH, infoHandler);
     coreAdminHandler   = createHandler(cfg.getCoreAdminHandlerClass(), CoreAdminHandler.class);
     containerHandlers.put(CORES_HANDLER_PATH, coreAdminHandler);
+    containerHandlers.put("/admin/authorization", securityConfHandler);
+    containerHandlers.put("/admin/authentication", securityConfHandler);
     if(pkiAuthenticationPlugin != null)
       containerHandlers.put(PKIAuthenticationPlugin.PATH, pkiAuthenticationPlugin.getRequestHandler());
 
@@ -466,6 +494,13 @@ 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"));
+  }
+
   private static void checkForDuplicateCoreNames(List<CoreDescriptor> cds) {
     Map<String, String> addedCores = Maps.newHashMap();
     for (CoreDescriptor cd : cds) {
@@ -546,20 +581,20 @@ public class CoreContainer {
         }
       }
     }
-    
+
     // It should be safe to close the authorization plugin at this point.
     try {
       if(authorizationPlugin != null) {
-        authorizationPlugin.close();
+        authorizationPlugin.plugin.close();
       }
     } catch (IOException e) {
       log.warn("Exception while closing authorization plugin.", e);
     }
-    
+
     // It should be safe to close the authentication plugin at this point.
     try {
       if(authenticationPlugin != null) {
-        authenticationPlugin.close();
+        authenticationPlugin.plugin.close();
         authenticationPlugin = null;
       }
     } catch (Exception e) {
@@ -1079,11 +1114,11 @@ public class CoreContainer {
   }
 
   public AuthorizationPlugin getAuthorizationPlugin() {
-    return authorizationPlugin;
+    return authorizationPlugin == null ? null : authorizationPlugin.plugin;
   }
 
   public AuthenticationPlugin getAuthenticationPlugin() {
-    return authenticationPlugin;
+    return authenticationPlugin == null ? null : authenticationPlugin.plugin;
   }
 
 }

Modified: lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/handler/SolrConfigHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/handler/SolrConfigHandler.java?rev=1694556&r1=1694555&r2=1694556&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/handler/SolrConfigHandler.java (original)
+++ lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/handler/SolrConfigHandler.java Thu Aug  6 19:15:57 2015
@@ -245,20 +245,8 @@ public class SolrConfigHandler extends R
 
 
     private void handlePOST() throws IOException {
-      Iterable<ContentStream> streams = req.getContentStreams();
-      if (streams == null) {
-        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "missing content stream");
-      }
-      ArrayList<CommandOperation> ops = new ArrayList<>();
-
-      for (ContentStream stream : streams)
-        ops.addAll(CommandOperation.parse(stream.getReader()));
-      List<Map> errList = CommandOperation.captureErrors(ops);
-      if (!errList.isEmpty()) {
-        resp.add(CommandOperation.ERR_MSGS, errList);
-        return;
-      }
-
+      List<CommandOperation> ops = CommandOperation.readCommands(req.getContentStreams(), resp);
+      if (ops == null) return;
       try {
         for (; ; ) {
           ArrayList<CommandOperation> opsCopy = new ArrayList<>(ops.size());
@@ -406,7 +394,7 @@ public class SolrConfigHandler extends R
                   overlay = updateNamedPlugin(info, op, overlay, prefix.equals("create") || prefix.equals("add"));
                 }
               } else {
-                op.addError(formatString("Unknown operation ''{0}'' ", op.name));
+                op.unknownOperation();
               }
             }
           }
@@ -592,7 +580,7 @@ public class SolrConfigHandler extends R
     return null;
   }
 
-  static void setWt(SolrQueryRequest req, String wt) {
+  public static void setWt(SolrQueryRequest req, String wt) {
     SolrParams params = req.getParams();
     if (params.get(CommonParams.WT) != null) return;//wt is set by user
     Map<String, String> map = new HashMap<>(1);

Modified: lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/security/AuthenticationPlugin.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/security/AuthenticationPlugin.java?rev=1694556&r1=1694555&r2=1694556&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/security/AuthenticationPlugin.java (original)
+++ lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/security/AuthenticationPlugin.java Thu Aug  6 19:15:57 2015
@@ -76,4 +76,10 @@ public abstract class AuthenticationPlug
       FilterChain filterChain) throws Exception;
 
 
+  /**
+   * Cleanup any per request  data
+   */
+  public void closeRequest() {
+  }
+
 }

Modified: lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/security/AuthorizationContext.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/security/AuthorizationContext.java?rev=1694556&r1=1694555&r2=1694556&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/security/AuthorizationContext.java (original)
+++ lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/security/AuthorizationContext.java Thu Aug  6 19:15:57 2015
@@ -54,6 +54,8 @@ public abstract class AuthorizationConte
   
   public abstract String getResource();
 
+  public abstract String getHttpMethod();
+
   public enum RequestType {READ, WRITE, ADMIN, UNKNOWN}
 
 }
\ No newline at end of file

Modified: lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/security/AuthorizationResponse.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/security/AuthorizationResponse.java?rev=1694556&r1=1694555&r2=1694556&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/security/AuthorizationResponse.java (original)
+++ lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/security/AuthorizationResponse.java Thu Aug  6 19:15:57 2015
@@ -21,6 +21,9 @@ package org.apache.solr.security;
    be used to return ACLs and other information from the authorization plugin.
  */
 public class AuthorizationResponse {
+  public static final AuthorizationResponse OK = new AuthorizationResponse(200);
+  public static final AuthorizationResponse FORBIDDEN = new AuthorizationResponse(403);
+  public static final AuthorizationResponse PROMPT = new AuthorizationResponse(401);
   public final int statusCode;
   String message;
 

Modified: lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java?rev=1694556&r1=1694555&r2=1694556&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java (original)
+++ lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java Thu Aug  6 19:15:57 2015
@@ -100,6 +100,7 @@ import org.apache.solr.request.SolrReque
 import org.apache.solr.response.QueryResponseWriter;
 import org.apache.solr.response.QueryResponseWriterUtil;
 import org.apache.solr.response.SolrQueryResponse;
+import org.apache.solr.security.AuthenticationPlugin;
 import org.apache.solr.security.AuthorizationContext;
 import org.apache.solr.security.AuthorizationContext.CollectionRequest;
 import org.apache.solr.security.AuthorizationContext.RequestType;
@@ -422,6 +423,13 @@ public class HttpSolrCall {
         AuthorizationContext context = getAuthCtx();
         log.info(context.toString());
         AuthorizationResponse authResponse = cores.getAuthorizationPlugin().authorize(context);
+        if (authResponse.statusCode == AuthorizationResponse.PROMPT.statusCode) {
+          Map<String, String> headers = (Map) getReq().getAttribute(AuthenticationPlugin.class.getName());
+          if (headers != null) {
+            for (Map.Entry<String, String> e : headers.entrySet()) response.setHeader(e.getKey(), e.getValue());
+          }
+          log.debug("USER_REQUIRED "+req.getHeader("Authorization")+" "+ req.getUserPrincipal());
+        }
         if (!(authResponse.statusCode == HttpStatus.SC_ACCEPTED) && !(authResponse.statusCode == HttpStatus.SC_OK)) {
           sendError(authResponse.statusCode,
               "Unauthorized request, Response code: " + authResponse.statusCode);
@@ -506,6 +514,8 @@ public class HttpSolrCall {
       } finally {
         SolrRequestInfo.clearRequestInfo();
       }
+      AuthenticationPlugin authcPlugin = cores.getAuthenticationPlugin();
+      if (authcPlugin != null) authcPlugin.closeRequest();
     }
   }
 
@@ -980,7 +990,12 @@ public class HttpSolrCall {
       public String getResource() {
         return path;
       }
-      
+
+      @Override
+      public String getHttpMethod() {
+        return getReq().getMethod();
+      }
+
       @Override
       public String toString() {
         StringBuilder response = new StringBuilder("userPrincipal: [").append(getUserPrincipal()).append("]")

Modified: lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/util/CommandOperation.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/util/CommandOperation.java?rev=1694556&r1=1694555&r2=1694556&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/util/CommandOperation.java (original)
+++ lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/util/CommandOperation.java Thu Aug  6 19:15:57 2015
@@ -27,22 +27,26 @@ import java.util.List;
 import java.util.Map;
 
 import org.apache.lucene.util.IOUtils;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.util.ContentStream;
 import org.apache.solr.common.util.StrUtils;
 import org.apache.solr.common.util.Utils;
+import org.apache.solr.response.SolrQueryResponse;
 import org.noggit.JSONParser;
 import org.noggit.ObjectBuilder;
 
 import static java.util.Collections.emptyMap;
 import static java.util.Collections.singletonList;
 import static java.util.Collections.singletonMap;
-import static org.apache.solr.common.util.Utils.makeMap;
+import static org.apache.solr.common.util.StrUtils.formatString;
+import static org.apache.solr.common.util.Utils.toJSON;
 
 public class CommandOperation {
   public final String name;
   private Object commandData;//this is most often a map
   private List<String> errors = new ArrayList<>();
 
-  CommandOperation(String operationName, Object metaData) {
+  public CommandOperation(String operationName, Object metaData) {
     commandData = metaData;
     this.name = operationName;
   }
@@ -98,6 +102,10 @@ public class CommandOperation {
 
   }
 
+  public void unknownOperation() {
+    addError(formatString("Unknown operation ''{0}'' ", name));
+  }
+
   static final String REQD = "''{0}'' is a required field";
 
 
@@ -147,7 +155,7 @@ public class CommandOperation {
   }
 
   private Map errorDetails() {
-    return makeMap(name, commandData, ERR_MSGS, errors);
+    return Utils.makeMap(name, commandData, ERR_MSGS, errors);
   }
 
   public boolean hasError() {
@@ -215,7 +223,12 @@ public class CommandOperation {
       if (val instanceof List) {
         List list = (List) val;
         for (Object o : list) {
-          operations.add(new CommandOperation(String.valueOf(key), o));
+          if (!(o instanceof Map)) {
+            operations.add(new CommandOperation(String.valueOf(key), list));
+            break;
+          } else {
+            operations.add(new CommandOperation(String.valueOf(key), o));
+          }
         }
       } else {
         operations.add(new CommandOperation(String.valueOf(key), val));
@@ -243,10 +256,35 @@ public class CommandOperation {
   @Override
   public String toString() {
     try {
-      return new String(Utils.toJSON(singletonMap(name, commandData)), IOUtils.UTF_8);
+      return new String(toJSON(singletonMap(name, commandData)), IOUtils.UTF_8);
     } catch (UnsupportedEncodingException e) {
       //should not happen
       return "";
     }
   }
+
+  public static List<CommandOperation> readCommands(Iterable<ContentStream> streams, SolrQueryResponse resp)
+      throws IOException {
+    if (streams == null) {
+      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "missing content stream");
+    }
+    ArrayList<CommandOperation> ops = new ArrayList<>();
+
+    for (ContentStream stream : streams)
+      ops.addAll(parse(stream.getReader()));
+    List<Map> errList = CommandOperation.captureErrors(ops);
+    if (!errList.isEmpty()) {
+      resp.add(CommandOperation.ERR_MSGS, errList);
+      return null;
+    }
+    return ops;
+  }
+
+  public static List<CommandOperation> clone(List<CommandOperation> ops) {
+    List<CommandOperation> opsCopy = new ArrayList<>(ops.size());
+    for (CommandOperation op : ops) opsCopy.add(op.getCopy());
+    return opsCopy;
+  }
+
+
 }

Modified: lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/cloud/TestAuthenticationFramework.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/cloud/TestAuthenticationFramework.java?rev=1694556&r1=1694555&r2=1694556&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/cloud/TestAuthenticationFramework.java (original)
+++ lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/cloud/TestAuthenticationFramework.java Thu Aug  6 19:15:57 2015
@@ -60,7 +60,7 @@ public class TestAuthenticationFramework
   
   static String requestUsername = MockAuthenticationPlugin.expectedUsername;
   static String requestPassword = MockAuthenticationPlugin.expectedPassword;
-  
+
   @Rule
   public TestRule solrTestRules = RuleChain
       .outerRule(new SystemPropertiesRestoreRule());
@@ -78,23 +78,22 @@ public class TestAuthenticationFramework
   
   private void setupAuthenticationPlugin() throws Exception {
     System.setProperty("authenticationPlugin", "org.apache.solr.cloud.TestAuthenticationFramework$MockAuthenticationPlugin");
+    MockAuthenticationPlugin.expectedUsername = null;
+    MockAuthenticationPlugin.expectedPassword = null;
+
   }
   
   @Test
   @Override
   public void testBasics() throws Exception {
-    // save original username/password
-    final String originalRequestUsername = requestUsername;
-    final String originalRequestPassword = requestPassword;
 
-    requestUsername = MockAuthenticationPlugin.expectedUsername;
-    requestPassword = MockAuthenticationPlugin.expectedPassword;
-    
+    final String collectionName = "testAuthenticationFrameworkCollection";
+
     // Should pass
     testCollectionCreateSearchDelete();
-    
-    requestUsername = MockAuthenticationPlugin.expectedUsername;
-    requestPassword = "junkpassword";
+
+    MockAuthenticationPlugin.expectedUsername = "solr";
+    MockAuthenticationPlugin.expectedPassword = "s0lrRocks";
     
     // Should fail with 401
     try {
@@ -105,9 +104,8 @@ public class TestAuthenticationFramework
         fail("Should've returned a 401 error");
       }
     } finally {
-      // restore original username/password
-      requestUsername = originalRequestUsername;
-      requestPassword = originalRequestPassword;        
+      MockAuthenticationPlugin.expectedUsername = null;
+      MockAuthenticationPlugin.expectedPassword = null;
     }
   }
 
@@ -120,8 +118,8 @@ public class TestAuthenticationFramework
   public static class MockAuthenticationPlugin extends AuthenticationPlugin implements HttpClientInterceptorPlugin {
     private static Logger log = LoggerFactory.getLogger(MockAuthenticationPlugin.class);
 
-    public static String expectedUsername = "solr";
-    public static String expectedPassword = "s0lrRocks";
+    public static String expectedUsername;
+    public static String expectedPassword;
 
     @Override
     public void init(Map<String,Object> pluginConfig) {}
@@ -129,6 +127,10 @@ public class TestAuthenticationFramework
     @Override
     public void doAuthenticate(ServletRequest request, ServletResponse response, FilterChain filterChain)
         throws Exception {
+      if (expectedUsername == null) {
+        filterChain.doFilter(request, response);
+        return;
+      }
       HttpServletRequest httpRequest = (HttpServletRequest)request;
       String username = httpRequest.getHeader("username");
       String password = httpRequest.getHeader("password");

Modified: lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/security/PKIAuthenticationIntegrationTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/security/PKIAuthenticationIntegrationTest.java?rev=1694556&r1=1694555&r2=1694556&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/security/PKIAuthenticationIntegrationTest.java (original)
+++ lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/security/PKIAuthenticationIntegrationTest.java Thu Aug  6 19:15:57 2015
@@ -44,21 +44,19 @@ public class PKIAuthenticationIntegratio
 
   static final int TIMEOUT = 10000;
 
-  public void distribSetUp() throws Exception {
-    super.distribSetUp();
+  @Test
+  public void testPkiAuth() throws Exception {
+    waitForThingsToLevelOut(10);
 
     byte[] bytes = Utils.toJSON(makeMap("authorization", singletonMap("class", MockAuthorizationPlugin.class.getName()),
         "authentication", singletonMap("class", MockAuthenticationPlugin.class.getName())));
 
     try (ZkStateReader zkStateReader = new ZkStateReader(zkServer.getZkAddress(),
         TIMEOUT, TIMEOUT)) {
-      zkStateReader.getZkClient().create(ZkStateReader.SOLR_SECURITY_CONF_PATH, bytes, CreateMode.PERSISTENT, true);
+      zkStateReader.getZkClient().setData(ZkStateReader.SOLR_SECURITY_CONF_PATH, bytes, true);
     }
-  }
-
-  @Test
-  public void testPkiAuth() throws Exception {
-    waitForThingsToLevelOut(10);
+    String baseUrl = jettys.get(0).getBaseUrl().toString();
+    TestAuthorizationFramework.verifySecurityStatus(cloudClient.getLbClient().getHttpClient(), baseUrl + "/admin/authorization", "authorization/class", MockAuthorizationPlugin.class.getName(), 20);
     log.info("Starting test");
     ModifiableSolrParams params = new ModifiableSolrParams();
     params.add("q", "*:*");
@@ -93,12 +91,6 @@ public class PKIAuthenticationIntegratio
       }
     };
     QueryRequest query = new QueryRequest(params);
-    LocalSolrQueryRequest lsqr = new LocalSolrQueryRequest(null, new ModifiableSolrParams()) {
-      @Override
-      public Principal getUserPrincipal() {
-        return null;
-      }
-    };
     query.process(cloudClient);
     log.info("count :{}", count);
     assertTrue(count.get() > 2);

Modified: lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/security/TestAuthorizationFramework.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/security/TestAuthorizationFramework.java?rev=1694556&r1=1694555&r2=1694556&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/security/TestAuthorizationFramework.java (original)
+++ lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/security/TestAuthorizationFramework.java Thu Aug  6 19:15:57 2015
@@ -17,12 +17,21 @@ package org.apache.solr.security;
  * limitations under the License.
  */
 
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.function.Predicate;
+
 import org.apache.commons.io.Charsets;
+import org.apache.http.client.HttpClient;
+import org.apache.http.client.methods.HttpGet;
+import org.apache.http.util.EntityUtils;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.solr.cloud.AbstractFullDistribZkTestBase;
 import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.params.ModifiableSolrParams;
-import org.apache.zookeeper.CreateMode;
+import org.apache.solr.common.util.StrUtils;
+import org.apache.solr.common.util.Utils;
 import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -33,27 +42,26 @@ public class TestAuthorizationFramework
 
   static final int TIMEOUT = 10000;
 
-  public void distribSetUp() throws Exception {
-    super.distribSetUp();
-    try (ZkStateReader zkStateReader = new ZkStateReader(zkServer.getZkAddress(),
-        TIMEOUT, TIMEOUT)) {
-      zkStateReader.getZkClient().create(ZkStateReader.SOLR_SECURITY_CONF_PATH,
-          "{\"authorization\":{\"class\":\"org.apache.solr.security.MockAuthorizationPlugin\"}}".getBytes(Charsets.UTF_8),
-          CreateMode.PERSISTENT, true);
-    }
-  }
 
   @Test
   public void authorizationFrameworkTest() throws Exception {
     MockAuthorizationPlugin.denyUsers.add("user1");
     MockAuthorizationPlugin.denyUsers.add("user1");
     waitForThingsToLevelOut(10);
+    try (ZkStateReader zkStateReader = new ZkStateReader(zkServer.getZkAddress(),
+        TIMEOUT, TIMEOUT)) {
+      zkStateReader.getZkClient().setData(ZkStateReader.SOLR_SECURITY_CONF_PATH,
+          "{\"authorization\":{\"class\":\"org.apache.solr.security.MockAuthorizationPlugin\"}}".getBytes(Charsets.UTF_8),
+          true);
+    }
+    String baseUrl = jettys.get(0).getBaseUrl().toString();
+    verifySecurityStatus(cloudClient.getLbClient().getHttpClient(), baseUrl + "/admin/authorization", "authorization/class", MockAuthorizationPlugin.class.getName(), 20);
     log.info("Starting test");
     ModifiableSolrParams params = new ModifiableSolrParams();
     params.add("q", "*:*");
     // This should work fine.
     cloudClient.query(params);
-    
+
     // This user is blacklisted in the mock. The request should return a 403.
     params.add("uname", "user1");
     try {
@@ -69,4 +77,30 @@ public class TestAuthorizationFramework
     MockAuthorizationPlugin.denyUsers.clear();
 
   }
+
+  public static void verifySecurityStatus(HttpClient cl, String url, String objPath, Object expected, int count) throws Exception {
+    boolean success = false;
+    String s = null;
+    List<String> hierarchy = StrUtils.splitSmart(objPath, '/');
+    for (int i = 0; i < count; i++) {
+      HttpGet get = new HttpGet(url);
+      s = EntityUtils.toString(cl.execute(get).getEntity());
+      Map m = (Map) Utils.fromJSONString(s);
+
+      Object actual = Utils.getObjectByPath(m, true, hierarchy);
+      if (expected instanceof Predicate) {
+        Predicate predicate = (Predicate) expected;
+        if (predicate.test(actual)) {
+          success = true;
+          break;
+        }
+      } else if (Objects.equals(String.valueOf(actual), expected)) {
+        success = true;
+        break;
+      }
+      Thread.sleep(50);
+    }
+    assertTrue("No match for " + objPath + " = " + expected + ", full response = " + s, success);
+
+  }
 }

Modified: lucene/dev/branches/branch_5x/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrClient.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrClient.java?rev=1694556&r1=1694555&r2=1694556&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrClient.java (original)
+++ lucene/dev/branches/branch_5x/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrClient.java Thu Aug  6 19:15:57 2015
@@ -978,8 +978,7 @@ public class CloudSolrClient extends Sol
       reqParams = new ModifiableSolrParams();
     }
     List<String> theUrlList = new ArrayList<>();
-    if (request.getPath().equals("/admin/collections")
-        || request.getPath().equals("/admin/cores")) {
+    if (request.getPath().startsWith("/admin/")) {
       Set<String> liveNodes = clusterState.getLiveNodes();
       for (String liveNode : liveNodes) {
         theUrlList.add(zkStateReader.getBaseUrlForNodeName(liveNode));

Modified: lucene/dev/branches/branch_5x/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java?rev=1694556&r1=1694555&r2=1694556&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java (original)
+++ lucene/dev/branches/branch_5x/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java Thu Aug  6 19:15:57 2015
@@ -34,6 +34,8 @@ import java.util.concurrent.ScheduledExe
 import java.util.concurrent.ThreadFactory;
 import java.util.concurrent.TimeUnit;
 
+import javafx.util.Pair;
+import org.apache.solr.common.Callable;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;
 import org.apache.solr.common.util.Utils;
@@ -47,7 +49,9 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import static java.util.Arrays.asList;
+import static java.util.Collections.EMPTY_MAP;
 import static java.util.Collections.unmodifiableSet;
+import static org.apache.solr.common.util.Utils.fromJSON;
 
 public class ZkStateReader implements Closeable {
   private static Logger log = LoggerFactory.getLogger(ZkStateReader.class);
@@ -111,6 +115,10 @@ public class ZkStateReader implements Cl
 
   private final ZkConfigManager configManager;
 
+  private ConfigData securityData;
+
+  private final Runnable securityNodeListener;
+
   public static final Set<String> KNOWN_CLUSTER_PROPS = unmodifiableSet(new HashSet<>(asList(
       LEGACY_CLOUD,
       URL_SCHEME,
@@ -184,12 +192,18 @@ public class ZkStateReader implements Cl
   private volatile boolean closed = false;
 
   public ZkStateReader(SolrZkClient zkClient) {
+    this(zkClient, null);
+  }
+
+  public ZkStateReader(SolrZkClient zkClient, Runnable securityNodeListener) {
     this.zkClient = zkClient;
     this.cmdExecutor = new ZkCmdExecutor(zkClient.getZkClientTimeout());
     this.configManager = new ZkConfigManager(zkClient);
     this.closeClient = false;
+    this.securityNodeListener = securityNodeListener;
   }
 
+
   public ZkStateReader(String zkServerAddress, int zkClientTimeout, int zkClientConnectTimeout) {
     this.zkClient = new SolrZkClient(zkServerAddress, zkClientTimeout, zkClientConnectTimeout,
         // on reconnect, reload cloud info
@@ -214,6 +228,7 @@ public class ZkStateReader implements Cl
     this.cmdExecutor = new ZkCmdExecutor(zkClientTimeout);
     this.configManager = new ZkConfigManager(zkClient);
     this.closeClient = true;
+    this.securityNodeListener = null;
   }
 
   public ZkConfigManager getConfigManager() {
@@ -409,8 +424,68 @@ public class ZkStateReader implements Cl
         addZkWatch(watchedCollection);
       }
     }
+    if (securityNodeListener != null) {
+      addSecuritynodeWatcher(SOLR_SECURITY_CONF_PATH, new Callable<Pair<byte[], Stat>>() {
+        @Override
+        public void call(Pair<byte[], Stat> pair) {
+          ConfigData cd = new ConfigData();
+          cd.data = pair.getKey() == null || pair.getKey().length == 0 ? EMPTY_MAP : Utils.getDeepCopy((Map) fromJSON(pair.getKey()), 4, false);
+          cd.version = pair.getValue() == null ? -1 : pair.getValue().getVersion();
+          securityData = cd;
+          securityNodeListener.run();
+
+        }
+      });
+    }
   }
 
+  private void addSecuritynodeWatcher(final String path, final Callable<Pair<byte[], Stat>> callback)
+      throws KeeperException, InterruptedException {
+    zkClient.exists(SOLR_SECURITY_CONF_PATH,
+        new Watcher() {
+
+          @Override
+          public void process(WatchedEvent event) {
+            // session events are not change events,
+            // and do not remove the watcher
+            if (EventType.None.equals(event.getType())) {
+              return;
+            }
+            try {
+              synchronized (ZkStateReader.this.getUpdateLock()) {
+                log.info("Updating {} ... ", path);
+
+                // remake watch
+                final Watcher thisWatch = this;
+                Stat stat = new Stat();
+                byte[] data = getZkClient().getData(path, thisWatch, stat, true);
+                try {
+                  callback.call(new Pair<>(data, stat));
+                } catch (Exception e) {
+                  if (e instanceof KeeperException) throw (KeeperException) e;
+                  if (e instanceof InterruptedException) throw (InterruptedException) e;
+                  log.error("Error running collections node listener", e);
+                }
+              }
+            } catch (KeeperException e) {
+              if (e.code() == KeeperException.Code.SESSIONEXPIRED
+                  || e.code() == KeeperException.Code.CONNECTIONLOSS) {
+                log.warn("ZooKeeper watch triggered, but Solr cannot talk to ZK");
+                return;
+              }
+              log.error("", e);
+              throw new ZooKeeperException(
+                  ErrorCode.SERVER_ERROR, "", e);
+            } catch (InterruptedException e) {
+              // Restore the interrupted status
+              Thread.currentThread().interrupt();
+              log.warn("", e);
+              return;
+            }
+          }
+
+        }, true);
+  }
   private ClusterState constructState(Set<String> ln, Watcher watcher) throws KeeperException, InterruptedException {
     Stat stat = new Stat();
     byte[] data = zkClient.getData(CLUSTER_STATE, watcher, stat, true);
@@ -715,11 +790,19 @@ public class ZkStateReader implements Cl
    * Returns the content of /security.json from ZooKeeper as a Map
    * If the files doesn't exist, it returns null.
    */
-  public Map getSecurityProps() {
+  public ConfigData getSecurityProps(boolean getFresh) {
+    if (!getFresh) {
+      if (securityData == null) return new ConfigData(EMPTY_MAP, -1);
+      return new ConfigData(securityData.data, securityData.version);
+    }
     try {
+      Stat stat = new Stat();
       if(getZkClient().exists(SOLR_SECURITY_CONF_PATH, true)) {
-        return (Map) Utils.fromJSON(getZkClient()
-            .getData(ZkStateReader.SOLR_SECURITY_CONF_PATH, null, new Stat(), true)) ;
+        byte[] data = getZkClient()
+            .getData(ZkStateReader.SOLR_SECURITY_CONF_PATH, null, stat, true);
+        return data != null && data.length > 0 ?
+            new ConfigData((Map<String, Object>) Utils.fromJSON(data), stat.getVersion()) :
+            null;
       }
     } catch (KeeperException | InterruptedException e) {
       throw new SolrException(ErrorCode.SERVER_ERROR,"Error reading security properties",e) ;
@@ -883,4 +966,18 @@ public class ZkStateReader implements Cl
     }
   }
 
+  public static class ConfigData {
+    public Map<String, Object> data;
+    public int version;
+
+    public ConfigData() {
+    }
+
+    public ConfigData(Map<String, Object> data, int version) {
+      this.data = data;
+      this.version = version;
+
+    }
+  }
+
 }

Modified: lucene/dev/branches/branch_5x/solr/solrj/src/java/org/apache/solr/common/params/CollectionParams.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/solrj/src/java/org/apache/solr/common/params/CollectionParams.java?rev=1694556&r1=1694555&r2=1694556&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/solrj/src/java/org/apache/solr/common/params/CollectionParams.java (original)
+++ lucene/dev/branches/branch_5x/solr/solrj/src/java/org/apache/solr/common/params/CollectionParams.java Thu Aug  6 19:15:57 2015
@@ -19,8 +19,6 @@ package org.apache.solr.common.params;
 
 import java.util.Locale;
 
-import org.apache.solr.common.SolrException;
-
 public interface CollectionParams 
 {
   /** What action **/
@@ -30,30 +28,36 @@ public interface CollectionParams
 
 
   public enum CollectionAction {
-    CREATE,
-    DELETE,
-    RELOAD,
-    SYNCSHARD,
-    CREATEALIAS,
-    DELETEALIAS,
-    SPLITSHARD,
-    DELETESHARD,
-    CREATESHARD,
-    DELETEREPLICA,
-    MIGRATE,
-    ADDROLE,
-    REMOVEROLE,
-    CLUSTERPROP,
-    REQUESTSTATUS,
-    ADDREPLICA,
-    OVERSEERSTATUS,
-    LIST,
-    CLUSTERSTATUS,
-    ADDREPLICAPROP,
-    DELETEREPLICAPROP,
-    BALANCESHARDUNIQUE,
-    REBALANCELEADERS,
-    MODIFYCOLLECTION;
+    CREATE(true),
+    DELETE(true),
+    RELOAD(true),
+    SYNCSHARD(true),
+    CREATEALIAS(true),
+    DELETEALIAS(true),
+    SPLITSHARD(true),
+    DELETESHARD(true),
+    CREATESHARD(true),
+    DELETEREPLICA(true),
+    MIGRATE(true),
+    ADDROLE(true),
+    REMOVEROLE(true),
+    CLUSTERPROP(true),
+    REQUESTSTATUS(false),
+    ADDREPLICA(true),
+    OVERSEERSTATUS(false),
+    LIST(false),
+    CLUSTERSTATUS(false),
+    ADDREPLICAPROP(true),
+    DELETEREPLICAPROP(true),
+    BALANCESHARDUNIQUE(true),
+    REBALANCELEADERS(true),
+    MODIFYCOLLECTION(true);
+    
+    public final boolean isWrite;
+
+    CollectionAction(boolean isWrite) {
+      this.isWrite = isWrite;
+    }
 
     public static CollectionAction get(String p) {
       if( p != null ) {

Modified: lucene/dev/branches/branch_5x/solr/solrj/src/java/org/apache/solr/common/util/Utils.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/solrj/src/java/org/apache/solr/common/util/Utils.java?rev=1694556&r1=1694555&r2=1694556&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/solrj/src/java/org/apache/solr/common/util/Utils.java (original)
+++ lucene/dev/branches/branch_5x/solr/solrj/src/java/org/apache/solr/common/util/Utils.java Thu Aug  6 19:15:57 2015
@@ -19,41 +19,58 @@ package org.apache.solr.common.util;
 
 
 import java.io.IOException;
+import java.io.StringReader;
 import java.nio.charset.StandardCharsets;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
 import java.util.HashSet;
 import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
+import org.apache.solr.common.SolrException;
 import org.noggit.CharArr;
 import org.noggit.JSONParser;
 import org.noggit.JSONWriter;
 import org.noggit.ObjectBuilder;
 
+import static java.util.Collections.unmodifiableList;
+import static java.util.Collections.unmodifiableSet;
+
 public class Utils {
   public static Map getDeepCopy(Map map, int maxDepth) {
-    Map copy = new LinkedHashMap<>();
+    return getDeepCopy(map, maxDepth, true);
+  }
+
+  public static Map getDeepCopy(Map map, int maxDepth, boolean mutable) {
+    if(map == null) return null;
+    if (maxDepth < 1) return map;
+    Map copy = new LinkedHashMap();
     for (Object o : map.entrySet()) {
       Map.Entry e = (Map.Entry) o;
       Object v = e.getValue();
-      if (v instanceof Map && maxDepth > 0) {
-        v = getDeepCopy((Map) v, maxDepth - 1);
-      } else if (v instanceof Set) {
-        v = new HashSet((Set) v);
-      } else if (v instanceof List) {
-        v = new ArrayList((List) v);
-      }
+      if (v instanceof Map) v = getDeepCopy((Map) v, maxDepth - 1, mutable);
+      else if (v instanceof Collection) v = getDeepCopy((Collection) v, maxDepth - 1, mutable);
       copy.put(e.getKey(), v);
     }
-    return copy;
+    return mutable ? copy : Collections.unmodifiableMap(copy);
+  }
+
+  public static Collection getDeepCopy(Collection c, int maxDepth, boolean mutable) {
+    if (c == null || maxDepth < 1) return c;
+    Collection result = c instanceof Set ? new HashSet() : new ArrayList();
+    for (Object o : c) {
+      if (o instanceof Map) {
+        o = getDeepCopy((Map) o, maxDepth - 1, mutable);
+      }
+      result.add(o);
+    }
+    return mutable ? result : result instanceof Set ? unmodifiableSet((Set) result) : unmodifiableList((List) result);
   }
 
-  //
-  // convenience methods... should these go somewhere else?
-  //
   public static byte[] toJSON(Object o) {
     CharArr out = new CharArr();
     new JSONWriter(out, 2).write(o); // indentation by default
@@ -84,17 +101,26 @@ public class Utils {
     }
   }
 
-  public static Map<String,Object> makeMap(Object... keyVals) {
+  public static Map<String, Object> makeMap(Object... keyVals) {
     if ((keyVals.length & 0x01) != 0) {
       throw new IllegalArgumentException("arguments should be key,value");
     }
-    Map<String,Object> propMap = new LinkedHashMap<>(keyVals.length>>1);
-    for (int i = 0; i < keyVals.length; i+=2) {
-      propMap.put(keyVals[i].toString(), keyVals[i+1]);
+    Map<String, Object> propMap = new LinkedHashMap<>(keyVals.length >> 1);
+    for (int i = 0; i < keyVals.length; i += 2) {
+      propMap.put(keyVals[i].toString(), keyVals[i + 1]);
     }
     return propMap;
   }
 
+  public static Object fromJSONString(String json)  {
+    try {
+      return new ObjectBuilder(new JSONParser(new StringReader(
+          json))).getObject();
+    } catch (IOException e) {
+      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Parse error", e);
+    }
+  }
+
   public static Object getObjectByPath(Map root, boolean onlyPrimitive, List<String> hierarchy) {
     Map obj = root;
     for (int i = 0; i < hierarchy.size(); i++) {

Modified: lucene/dev/branches/branch_5x/solr/test-framework/src/java/org/apache/solr/cloud/AbstractZkTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/test-framework/src/java/org/apache/solr/cloud/AbstractZkTestCase.java?rev=1694556&r1=1694555&r2=1694556&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/test-framework/src/java/org/apache/solr/cloud/AbstractZkTestCase.java (original)
+++ lucene/dev/branches/branch_5x/solr/test-framework/src/java/org/apache/solr/cloud/AbstractZkTestCase.java Thu Aug  6 19:15:57 2015
@@ -37,7 +37,7 @@ import java.util.Map;
 public abstract class AbstractZkTestCase extends SolrTestCaseJ4 {
   private static final String ZOOKEEPER_FORCE_SYNC = "zookeeper.forceSync";
   
-  static final int TIMEOUT = 10000;
+  public static final int TIMEOUT = 10000;
 
   private static final boolean DEBUG = false;