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 2014/11/24 17:57:02 UTC

svn commit: r1641420 [1/2] - in /lucene/dev/branches/branch_5x/solr: ./ core/ core/src/java/org/apache/solr/cloud/ core/src/java/org/apache/solr/core/ core/src/java/org/apache/solr/handler/ core/src/java/org/apache/solr/handler/admin/ core/src/java/org...

Author: noble
Date: Mon Nov 24 16:57:01 2014
New Revision: 1641420

URL: http://svn.apache.org/r1641420
Log:
SOLR-6533 , SOLR-6715 , SOLR-6654 Backporting all the changes from trunk

Added:
    lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/ConfigOverlay.java
      - copied, changed from r1636862, lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/ConfigOverlay.java
    lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/MapSerializable.java
      - copied unchanged from r1636862, lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/MapSerializable.java
    lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/handler/SolrConfigHandler.java
      - copied, changed from r1636862, lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/SolrConfigHandler.java
    lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/core/TestConfigOverlay.java
      - copied unchanged from r1636862, lucene/dev/trunk/solr/core/src/test/org/apache/solr/core/TestConfigOverlay.java
    lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/core/TestSolrConfigHandler.java
      - copied unchanged from r1636862, lucene/dev/trunk/solr/core/src/test/org/apache/solr/core/TestSolrConfigHandler.java
    lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/handler/TestConfigReload.java
      - copied unchanged from r1640857, lucene/dev/trunk/solr/core/src/test/org/apache/solr/handler/TestConfigReload.java
    lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/handler/TestSolrConfigHandlerConcurrent.java
      - copied, changed from r1636862, lucene/dev/trunk/solr/core/src/test/org/apache/solr/handler/TestSolrConfigHandlerConcurrent.java
Modified:
    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/CloudUtil.java
    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/cloud/ZkSolrResourceLoader.java
    lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/Config.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/core/PluginInfo.java
    lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/SolrConfig.java
    lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/SolrCore.java
    lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/SolrResourceLoader.java
    lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminHandler.java
    lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/handler/admin/EditFileRequestHandler.java
    lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/handler/admin/ShowFileRequestHandler.java
    lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/rest/SolrConfigRestApi.java
    lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/schema/ManagedIndexSchema.java
    lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/schema/ManagedIndexSchemaFactory.java
    lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/schema/ZkIndexSchemaReader.java
    lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/search/CacheConfig.java
    lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
    lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/update/SolrIndexConfig.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-files/solr/collection1/conf/solrconfig-managed-schema.xml
    lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/rest/TestRestManager.java
    lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/rest/schema/TestBulkSchemaAPI.java
    lucene/dev/branches/branch_5x/solr/solrj/   (props changed)
    lucene/dev/branches/branch_5x/solr/solrj/src/java/org/apache/solr/common/cloud/SolrZkClient.java
    lucene/dev/branches/branch_5x/solr/test-framework/   (props changed)
    lucene/dev/branches/branch_5x/solr/test-framework/src/java/org/apache/solr/util/TestHarness.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=1641420&r1=1641419&r2=1641420&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/CHANGES.txt (original)
+++ lucene/dev/branches/branch_5x/solr/CHANGES.txt Mon Nov 24 16:57:01 2014
@@ -178,6 +178,8 @@ New Features
   facet.pivot which refers to a 'tag' local param in one or more stats.field params.
   (hossman, Vitaliy Zhovtyuk, Steve Molloy)
 
+* SOLR-6533: Support editing common solrconfig.xml values (Noble Paul)
+
 Bug Fixes
 ----------------------
 
@@ -365,23 +367,8 @@ Other Changes
   References to zkCredentialProvider in System properties or configurations should be
   changed to zkCredentialsProvider.  (Gregory Chanan)
 
-* SOLR-6697: bin/solr start scripts allow setting SOLR_OPTS in solr.in.* (janhoy)
-
-* SOLR-6739: Admin UI - Sort list of command line args (steffkes)
-
-* SOLR-6740: Admin UI - improve Files View (steffkes)
-
-* SOLR-6570: Run SolrZkClient session watch asynchronously. 
- (Ramkumar Aiyengar via Mark Miller)
- 
-* SOLR-6747: Add an optional caching option as a workaround for SOLR-6586.
-  (Mark Miller, Gregory Chanan)
-
-* SOLR-6459: Normalize logging of operations in Overseer and log current queue size.
-  (Ramkumar Aiyengar via Mark Miller)
-
-* SOLR-6754: ZkController.publish doesn't use the updateLastState parameter.
-  (shalin)
+* SOLR-6715: ZkSolrResourceLoader constructors accept a parameter called 'collection'
+  but it should be 'configName'. (shalin)
 
 ==================  4.10.3 ==================
 

Modified: lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/cloud/CloudUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/cloud/CloudUtil.java?rev=1641420&r1=1641419&r2=1641420&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/cloud/CloudUtil.java (original)
+++ lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/cloud/CloudUtil.java Mon Nov 24 16:57:01 2014
@@ -88,7 +88,7 @@ public class CloudUtil {
 
   /**
    * Returns a displayable unified path to the given resource. For non-solrCloud that will be the
-   * same as getConfigDir, but for Cloud it will be getCollectionZkPath ending in a /
+   * same as getConfigDir, but for Cloud it will be getConfigSetZkPath ending in a /
    * <p/>
    * <b>Note:</b> Do not use this to generate a valid file path, but for debug printing etc
    * @param loader Resource loader instance
@@ -96,7 +96,7 @@ public class CloudUtil {
    */
   public static String unifiedResourcePath(SolrResourceLoader loader) {
     return (loader instanceof ZkSolrResourceLoader) ?
-            ((ZkSolrResourceLoader) loader).getCollectionZkPath() + "/" :
+            ((ZkSolrResourceLoader) loader).getConfigSetZkPath() + "/" :
             loader.getConfigDir();
   }
 }

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=1641420&r1=1641419&r2=1641420&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 Mon Nov 24 16:57:01 2014
@@ -20,18 +20,17 @@ package org.apache.solr.cloud;
 import java.io.File;
 import java.io.IOException;
 import java.io.UnsupportedEncodingException;
-import java.net.ConnectException;
 import java.net.InetAddress;
 import java.net.NetworkInterface;
-import java.net.SocketException;
-import java.net.SocketTimeoutException;
 import java.net.URLEncoder;
 import java.net.UnknownHostException;
+import java.text.MessageFormat;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.Enumeration;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.Iterator;
 import java.util.LinkedHashMap;
 import java.util.List;
@@ -45,8 +44,6 @@ import java.util.concurrent.TimeoutExcep
 
 import org.apache.commons.io.FileUtils;
 import org.apache.commons.lang.StringUtils;
-import org.apache.http.NoHttpResponseException;
-import org.apache.http.conn.ConnectTimeoutException;
 import org.apache.solr.client.solrj.impl.HttpSolrServer;
 import org.apache.solr.client.solrj.request.CoreAdminRequest.WaitForState;
 import org.apache.solr.common.SolrException;
@@ -72,9 +69,11 @@ import org.apache.solr.common.cloud.ZooK
 import org.apache.solr.common.params.CollectionParams;
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.URLUtil;
+import org.apache.solr.core.CloseHook;
 import org.apache.solr.core.CoreContainer;
 import org.apache.solr.core.CoreDescriptor;
 import org.apache.solr.core.SolrCore;
+import org.apache.solr.core.SolrResourceLoader;
 import org.apache.solr.handler.component.ShardHandler;
 import org.apache.solr.update.UpdateLog;
 import org.apache.solr.update.UpdateShardHandler;
@@ -83,9 +82,9 @@ import org.apache.zookeeper.KeeperExcept
 import org.apache.zookeeper.KeeperException.ConnectionLossException;
 import org.apache.zookeeper.KeeperException.NoNodeException;
 import org.apache.zookeeper.KeeperException.SessionExpiredException;
+import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.Watcher;
 import org.apache.zookeeper.data.Stat;
-import org.noggit.JSONParser;
-import org.noggit.ObjectBuilder;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -200,9 +199,9 @@ public final class ZkController {
 
   // keeps track of a list of objects that need to know a new ZooKeeper session was created after expiration occurred
   private List<OnReconnect> reconnectListeners = new ArrayList<OnReconnect>();
-  
+
   public ZkController(final CoreContainer cc, String zkServerAddress, int zkClientTimeout, int zkClientConnectTimeout, String localHost, String locaHostPort,
-        String localHostContext, int leaderVoteWait, int leaderConflictResolveWait, boolean genericCoreNodeNames, final CurrentCoreDescriptorProvider registerOnReconnect) 
+                      String localHostContext, int leaderVoteWait, int leaderConflictResolveWait, boolean genericCoreNodeNames, final CurrentCoreDescriptorProvider registerOnReconnect)
       throws InterruptedException, TimeoutException, IOException
   {
 
@@ -214,18 +213,18 @@ public final class ZkController {
     // solr.xml to indicate the root context, instead of hostContext="" 
     // which means the default of "solr"
     localHostContext = trimLeadingAndTrailingSlashes(localHostContext);
-    
+
     this.zkServerAddress = zkServerAddress;
     this.localHostPort = locaHostPort;
     this.localHostContext = localHostContext;
     this.hostName = normalizeHostName(localHost);
-    this.nodeName = generateNodeName(this.hostName, 
-                                     this.localHostPort, 
-                                     this.localHostContext);
+    this.nodeName = generateNodeName(this.hostName,
+        this.localHostPort,
+        this.localHostContext);
 
     this.leaderVoteWait = leaderVoteWait;
     this.leaderConflictResolveWait = leaderConflictResolveWait;
-    
+
     this.clientTimeout = zkClientTimeout;
     DefaultConnectionStrategy strat = new DefaultConnectionStrategy();
     String zkACLProviderClass = cc.getConfig().getZkACLProviderClass();
@@ -242,12 +241,13 @@ public final class ZkController {
     } else {
       strat.setZkCredentialsToAddAutomatically(new DefaultZkCredentialsProvider());
     }
-    
+    addOnReconnectListener(getConfigDirListener());
+
     zkClient = new SolrZkClient(zkServerAddress, zkClientTimeout,
         zkClientConnectTimeout, strat,
         // on reconnect, reload cloud info
         new OnReconnect() {
-          
+
           @Override
           public void command() {
             log.info("ZooKeeper session re-connected ... refreshing core states after session expiration.");
@@ -260,32 +260,32 @@ public final class ZkController {
               // he is involved in the sync, and he certainly may not be
               // ExecutorUtil.shutdownAndAwaitTermination(cc.getCmdDistribExecutor());
               // we need to create all of our lost watches
-              
+
               // seems we dont need to do this again...
               // Overseer.createClientNodes(zkClient, getNodeName());
-              
+
               cc.cancelCoreRecoveries();
-              
+
               registerAllCoresAsDown(registerOnReconnect, false);
-              
+
               if (!zkRunOnly) {
                 ElectionContext context = new OverseerElectionContext(zkClient,
                     overseer, getNodeName());
-                
+
                 ElectionContext prevContext = overseerElector.getContext();
                 if (prevContext != null) {
                   prevContext.cancelElection();
                 }
-                
+
                 overseerElector.setup(context);
                 overseerElector.joinElection(context, true);
               }
-              
+
               zkStateReader.createClusterStateWatchersAndUpdate();
-              
+
               // we have to register as live first to pick up docs in the buffer
               createEphemeralLiveNode();
-              
+
               List<CoreDescriptor> descriptors = registerOnReconnect
                   .getCurrentDescriptors();
               // re register all descriptors
@@ -330,20 +330,20 @@ public final class ZkController {
                   SolrException.ErrorCode.SERVER_ERROR, "", e);
             }
           }
-          
+
         }, new BeforeReconnect() {
-          
-          @Override
-          public void command() {
-            try {
-              ZkController.this.overseer.close();
-            } catch (Exception e) {
-              log.error("Error trying to stop any Overseer threads", e);
-            }
-            markAllAsNotLeader(registerOnReconnect);
-          }
-        }, zkACLProvider);
-    
+
+      @Override
+      public void command() {
+        try {
+          ZkController.this.overseer.close();
+        } catch (Exception e) {
+          log.error("Error trying to stop any Overseer threads", e);
+        }
+        markAllAsNotLeader(registerOnReconnect);
+      }
+    }, zkACLProvider);
+
     this.overseerJobQueue = Overseer.getInQueue(zkClient);
     this.overseerCollectionQueue = Overseer.getCollectionQueue(zkClient);
     this.overseerRunningMap = Overseer.getRunningMap(zkClient);
@@ -352,9 +352,9 @@ public final class ZkController {
     cmdExecutor = new ZkCmdExecutor(zkClientTimeout);
     leaderElector = new LeaderElector(zkClient);
     zkStateReader = new ZkStateReader(zkClient);
-    
+
     this.baseURL = zkStateReader.getBaseUrlForNodeName(this.nodeName);
-    
+
     init(registerOnReconnect);
   }
 
@@ -1179,7 +1179,7 @@ public final class ZkController {
     return true;
   }
 
-  public void unregister(String coreName, CoreDescriptor cd)
+  public void unregister(String coreName, CoreDescriptor cd, String configLocation)
       throws InterruptedException, KeeperException {
     final String coreNodeName = cd.getCloudDescriptor().getCoreNodeName();
     final String collection = cd.getCloudDescriptor().getCollectionName();
@@ -1200,6 +1200,10 @@ public final class ZkController {
     boolean removeWatch = true;
     // if there is no SolrCore which is a member of this collection, remove the watch
     for (SolrCore solrCore : cc.getCores()) {
+      if (((ZkSolrResourceLoader)solrCore.getResourceLoader()).getConfigSetZkPath().equals(configLocation))
+        configLocation = null; //if a core uses this config dir , then set it to null
+
+
       CloudDescriptor cloudDesc = solrCore.getCoreDescriptor()
           .getCloudDescriptor();
       if (cloudDesc != null
@@ -1216,6 +1220,13 @@ public final class ZkController {
         ZkStateReader.COLLECTION_PROP, cloudDescriptor.getCollectionName(),
         ZkStateReader.CORE_NODE_NAME_PROP, coreNodeName);
     overseerJobQueue.offer(ZkStateReader.toJSON(m));
+
+    if(configLocation != null) {
+      synchronized (confDirectoryListeners) {
+        log.info("This conf directory is no more watched {0}",configLocation);
+        confDirectoryListeners.remove(configLocation);
+      }
+    }
   }
   
   public void createCollection(String collection) throws KeeperException,
@@ -2098,4 +2109,182 @@ public final class ZkController {
       }
     }
   }
+
+  /**
+   * Persists a config file to ZooKeeper using optimistic concurrency.
+   *
+   * @return true on success
+   */
+  public static boolean persistConfigResourceToZooKeeper( SolrResourceLoader loader, int znodeVersion ,
+                                                          String resourceName, byte[] content,
+                                                          boolean createIfNotExists) {
+    final ZkSolrResourceLoader zkLoader = (ZkSolrResourceLoader)loader;
+    final ZkController zkController = zkLoader.getZkController();
+    final SolrZkClient zkClient = zkController.getZkClient();
+    final String resourceLocation = zkLoader.getConfigSetZkPath() + "/" + resourceName;
+    String errMsg = "Failed to persist resource at {0} - version mismatch {1}";
+    try {
+      try {
+        zkClient.setData(resourceLocation , content,znodeVersion, true);
+        zkClient.setData(zkLoader.getConfigSetZkPath(),new byte[]{0},true);
+      } catch (NoNodeException e) {
+        if(createIfNotExists){
+          try {
+            zkClient.create(resourceLocation,content, CreateMode.PERSISTENT,true);
+            zkClient.setData(zkLoader.getConfigSetZkPath(), new byte[]{0}, true);
+          } catch (KeeperException.NodeExistsException nee) {
+            try {
+              Stat stat = zkClient.exists(resourceLocation, null, true);
+              log.info("failed to set data version in zk is {} and expected version is {} ", stat.getVersion(),znodeVersion);
+            } catch (Exception e1) {
+              log.warn("could not get stat");
+            }
+
+            log.info(MessageFormat.format(errMsg,resourceLocation,znodeVersion));
+            throw new ResourceModifiedInZkException(ErrorCode.CONFLICT, MessageFormat.format(errMsg,resourceLocation,znodeVersion) + ", retry.");
+          }
+        }
+      }
+
+    } catch (KeeperException.BadVersionException bve){
+      log.info(MessageFormat.format(errMsg,resourceLocation));
+      throw new ResourceModifiedInZkException(ErrorCode.CONFLICT, MessageFormat.format(errMsg,resourceLocation,znodeVersion) + ", retry.");
+    }catch (ResourceModifiedInZkException e){
+      throw e;
+    } catch (Exception e) {
+      if (e instanceof InterruptedException) {
+        Thread.currentThread().interrupt(); // Restore the interrupted status
+      }
+      final String msg = "Error persisting resource at " + resourceLocation;
+      log.error(msg, e);
+      throw new SolrException(ErrorCode.SERVER_ERROR, msg, e);
+    }
+    return true;
+  }
+
+  public static  class ResourceModifiedInZkException extends SolrException {
+    public ResourceModifiedInZkException(ErrorCode code, String msg) {
+      super(code, msg);
+    }
+  }
+
+  public void unRegisterConfListener(Runnable listener) {
+    if(listener == null) return;
+    synchronized (confDirectoryListeners){
+      for (Set<Runnable> listeners : confDirectoryListeners.values()) {
+        if(listeners != null) {
+          if(listeners.remove(listener)) {
+            log.info(" a listener was removed because of core close");
+          }
+        }
+      }
+    }
+
+  }
+
+  /**This will give a callback to the listener whenever a child is modified in the
+   * conf directory. It is the responsibility of the listener to check if the individual
+   * item of interest has been modified.  When the last core which was interested in
+   * this conf directory is gone the listeners will be removed automatically.
+   */
+  public void registerConfListenerForCore(String confDir,SolrCore core, final Runnable listener){
+    if(listener==null) throw new NullPointerException("listener cannot be null");
+    synchronized (confDirectoryListeners){
+      if(confDirectoryListeners.containsKey(confDir)){
+        confDirectoryListeners.get(confDir).add(listener);
+        core.addCloseHook(new CloseHook() {
+          @Override
+          public void preClose(SolrCore core) {
+            unRegisterConfListener(listener);
+          }
+
+          @Override
+          public void postClose(SolrCore core) { }
+        });
+      } else {
+        throw new SolrException(ErrorCode.SERVER_ERROR,"This conf directory is not valid");
+      }
+    }
+  }
+
+  private final Map<String , Set<Runnable>> confDirectoryListeners =  new HashMap<>();
+
+  void watchZKConfDir(final String zkDir) {
+    log.info("watch zkdir " + zkDir);
+    if (!confDirectoryListeners.containsKey(zkDir)) {
+      confDirectoryListeners.put(zkDir,  new HashSet<Runnable>());
+      setConfWatcher(zkDir, new WatcherImpl(zkDir));
+
+    }
+
+
+  }
+  private class WatcherImpl implements Watcher{
+    private final String zkDir ;
+
+    private WatcherImpl(String dir) {
+      this.zkDir = dir;
+    }
+
+    @Override
+      public void process(WatchedEvent event) {
+        try {
+
+          synchronized (confDirectoryListeners) {
+            // if this is not among directories to be watched then don't set the watcher anymore
+            if( !confDirectoryListeners.containsKey(zkDir)) {
+              log.info("Watcher on {} is removed ", zkDir);
+              return;
+            }
+            final Set<Runnable> listeners = confDirectoryListeners.get(zkDir);
+            if (listeners != null && !listeners.isEmpty()) {
+              new Thread() {
+                //run these in a separate thread because this can be long running
+                public void run() {
+                  for (final Runnable listener : listeners)
+                    try {
+                      listener.run();
+                    } catch (Exception e) {
+                      log.warn("listener throws error", e);
+                    }
+                }
+              }.start();
+            }
+
+          }
+
+        } finally {
+          if (Event.EventType.None.equals(event.getType())) {
+            log.info("A node got unwatched for {}", zkDir);
+            return;
+          } else {
+            setConfWatcher(zkDir,this);
+          }
+        }
+      }
+    }
+
+  private void setConfWatcher(String zkDir, Watcher watcher) {
+    try {
+      zkClient.exists(zkDir,watcher,true);
+    } catch (KeeperException e) {
+      log.error("failed to set watcher for conf dir {} ", zkDir);
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      log.error("failed to set watcher for conf dir {} ", zkDir);
+    }
+  }
+
+  public OnReconnect getConfigDirListener() {
+    return new OnReconnect() {
+      @Override
+      public void command() {
+        synchronized (confDirectoryListeners){
+          for (String s : confDirectoryListeners.keySet()) {
+            watchZKConfDir(s);
+          }
+        }
+      }
+    };
+  }
 }

Modified: lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/cloud/ZkSolrResourceLoader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/cloud/ZkSolrResourceLoader.java?rev=1641420&r1=1641419&r2=1641420&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/cloud/ZkSolrResourceLoader.java (original)
+++ lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/cloud/ZkSolrResourceLoader.java Mon Nov 24 16:57:01 2014
@@ -30,6 +30,7 @@ import org.apache.solr.common.cloud.ZooK
 import org.apache.solr.core.SolrResourceLoader;
 import org.apache.solr.schema.ZkIndexSchemaReader;
 import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.data.Stat;
 
 /**
  * ResourceLoader that works with ZooKeeper.
@@ -37,15 +38,16 @@ import org.apache.zookeeper.KeeperExcept
  */
 public class ZkSolrResourceLoader extends SolrResourceLoader {
 
-  private final String collectionZkPath;
+  private final String configSetZkPath;
   private ZkController zkController;
   private ZkIndexSchemaReader zkIndexSchemaReader;
 
-  public ZkSolrResourceLoader(String instanceDir, String collection,
+  public ZkSolrResourceLoader(String instanceDir, String configSet,
       ZkController zooKeeperController) {
     super(instanceDir);
     this.zkController = zooKeeperController;
-    collectionZkPath = ZkController.CONFIGS_ZKNODE + "/" + collection;
+    configSetZkPath = ZkController.CONFIGS_ZKNODE + "/" + configSet;
+    zkController.watchZKConfDir(configSetZkPath);
   }
 
   /**
@@ -56,11 +58,12 @@ public class ZkSolrResourceLoader extend
    * the "lib/" directory in the specified instance directory.
    * <p>
    */
-  public ZkSolrResourceLoader(String instanceDir, String collection, ClassLoader parent,
+  public ZkSolrResourceLoader(String instanceDir, String configSet, ClassLoader parent,
       Properties coreProperties, ZkController zooKeeperController) {
     super(instanceDir, parent, coreProperties);
     this.zkController = zooKeeperController;
-    collectionZkPath = ZkController.CONFIGS_ZKNODE + "/" + collection;
+    configSetZkPath = ZkController.CONFIGS_ZKNODE + "/" + configSet;
+    zkController.watchZKConfDir(configSetZkPath);
   }
 
   /**
@@ -75,11 +78,12 @@ public class ZkSolrResourceLoader extend
   @Override
   public InputStream openResource(String resource) throws IOException {
     InputStream is = null;
-    String file = collectionZkPath + "/" + resource;
+    String file = configSetZkPath + "/" + resource;
     try {
       if (zkController.pathExists(file)) {
-        byte[] bytes = zkController.getZkClient().getData(file, null, null, true);
-        return new ByteArrayInputStream(bytes);
+        Stat stat = new Stat();
+        byte[] bytes = zkController.getZkClient().getData(file, null, stat, true);
+        return new ZkByteArrayInputStream(bytes, stat);
       }
     } catch (Exception e) {
       throw new IOException("Error opening " + file, e);
@@ -92,12 +96,26 @@ public class ZkSolrResourceLoader extend
     }
     if (is == null) {
       throw new IOException("Can't find resource '" + resource
-          + "' in classpath or '" + collectionZkPath + "', cwd="
+          + "' in classpath or '" + configSetZkPath + "', cwd="
           + System.getProperty("user.dir"));
     }
     return is;
   }
 
+  public static class ZkByteArrayInputStream extends ByteArrayInputStream{
+
+    private final Stat stat;
+    public ZkByteArrayInputStream(byte[] buf, Stat stat) {
+      super(buf);
+      this.stat = stat;
+
+    }
+
+    public Stat getStat(){
+      return stat;
+    }
+  }
+
   @Override
   public String getConfigDir() {
     throw new ZooKeeperException(
@@ -109,7 +127,7 @@ public class ZkSolrResourceLoader extend
   public String[] listConfigDir() {
     List<String> list;
     try {
-      list = zkController.getZkClient().getChildren(collectionZkPath, null, true);
+      list = zkController.getZkClient().getChildren(configSetZkPath, null, true);
     } catch (InterruptedException e) {
       // Restore the interrupted status
       Thread.currentThread().interrupt();
@@ -124,8 +142,8 @@ public class ZkSolrResourceLoader extend
     return list.toArray(new String[0]);
   }
 
-  public String getCollectionZkPath() {
-    return collectionZkPath;
+  public String getConfigSetZkPath() {
+    return configSetZkPath;
   }
   
   public ZkController getZkController() {

Modified: lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/Config.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/Config.java?rev=1641420&r1=1641419&r2=1641420&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/Config.java (original)
+++ lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/Config.java Mon Nov 24 16:57:01 2014
@@ -18,6 +18,7 @@
 package org.apache.solr.core;
 
 import org.apache.lucene.util.Version;
+import org.apache.solr.cloud.ZkSolrResourceLoader;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.util.DOMUtil;
 import org.apache.solr.util.SystemIdResolver;
@@ -48,10 +49,12 @@ import javax.xml.xpath.XPathExpressionEx
 import javax.xml.xpath.XPathFactory;
 
 import java.io.IOException;
+import java.io.InputStream;
 import java.text.ParseException;
 import java.util.Arrays;
 import java.util.HashSet;
 import java.util.Map;
+import java.util.Properties;
 import java.util.Set;
 import java.util.SortedMap;
 import java.util.SortedSet;
@@ -73,6 +76,7 @@ public class Config {
   private final String prefix;
   private final String name;
   private final SolrResourceLoader loader;
+  private int zkVersion = -1;
 
   /**
    * Builds a config from a resource name with no xpath prefix.
@@ -102,7 +106,7 @@ public class Config {
    * @param is the resource as a SAX InputSource
    * @param prefix an optional prefix that will be preprended to all non-absolute xpath expressions
    */
-  public Config(SolrResourceLoader loader, String name, InputSource is, String prefix, boolean subProps) throws ParserConfigurationException, IOException, SAXException 
+  public Config(SolrResourceLoader loader, String name, InputSource is, String prefix, boolean substituteProps) throws ParserConfigurationException, IOException, SAXException
   {
     if( loader == null ) {
       loader = new SolrResourceLoader( null );
@@ -112,9 +116,14 @@ public class Config {
     this.prefix = (prefix != null && !prefix.endsWith("/"))? prefix + '/' : prefix;
     try {
       javax.xml.parsers.DocumentBuilderFactory dbf = DocumentBuilderFactory.newInstance();
-      
+
       if (is == null) {
-        is = new InputSource(loader.openConfig(name));
+        InputStream in = loader.openConfig(name);
+        if (in instanceof ZkSolrResourceLoader.ZkByteArrayInputStream) {
+          zkVersion = ((ZkSolrResourceLoader.ZkByteArrayInputStream) in).getStat().getVersion();
+          log.info("loaded config {} with version {} ",name,zkVersion);
+        }
+        is = new InputSource(in);
         is.setSystemId(SystemIdResolver.createSystemIdFromResourceName(name));
       }
 
@@ -138,8 +147,8 @@ public class Config {
         // some XML parsers are broken and don't close the byte stream (but they should according to spec)
         IOUtils.closeQuietly(is.getByteStream());
       }
-      if (subProps) {
-        DOMUtil.substituteProperties(doc, loader.getCoreProperties());
+      if (substituteProps) {
+        DOMUtil.substituteProperties(doc, getSubstituteProperties());
       }
     } catch (ParserConfigurationException e)  {
       SolrException.log(log, "Exception during parsing file: " + name, e);
@@ -152,7 +161,11 @@ public class Config {
       throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
     }
   }
-  
+
+  protected Properties getSubstituteProperties() {
+    return loader.getCoreProperties();
+  }
+
   public Config(SolrResourceLoader loader, String name, Document doc) {
     this.prefix = null;
     this.doc = doc;
@@ -207,7 +220,7 @@ public class Config {
   }
   
   public void substituteProperties() {
-    DOMUtil.substituteProperties(doc, loader.getCoreProperties());
+    DOMUtil.substituteProperties(doc, getSubstituteProperties());
   }
 
 
@@ -459,6 +472,12 @@ public class Config {
     return version;
   }
 
+  /**If this config is loaded from zk the version is relevant other wise -1 is returned
+   */
+  public int getZnodeVersion(){
+    return zkVersion;
+  }
+
   public Config getOriginalConfig() {
     return new Config(loader, null, origDoc);
   }

Copied: lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/ConfigOverlay.java (from r1636862, lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/ConfigOverlay.java)
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/ConfigOverlay.java?p2=lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/ConfigOverlay.java&p1=lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/ConfigOverlay.java&r1=1636862&r2=1641420&rev=1641420&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/ConfigOverlay.java (original)
+++ lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/ConfigOverlay.java Mon Nov 24 16:57:01 2014
@@ -34,7 +34,7 @@ import org.noggit.JSONParser;
 import org.noggit.JSONWriter;
 import org.noggit.ObjectBuilder;
 
-public class ConfigOverlay {
+public class ConfigOverlay implements MapSerializable{
   private final int znodeVersion ;
   private Map<String, Object> data;
   private Map<String,Object> props;
@@ -177,13 +177,6 @@ public class ConfigOverlay {
     return out.toString();
   }
 
-  public  Map toOutputFormat() {
-    Map result = new LinkedHashMap();
-    result.put("version",znodeVersion);
-    result.putAll(data);
-    return result;
-  }
-
 
   public static final String RESOURCE_NAME = "configoverlay.json";
 
@@ -254,4 +247,12 @@ public class ConfigOverlay {
   public Map<String, Object> getUserProps() {
     return userProps;
   }
+
+  @Override
+  public Map<String, Object> toMap() {
+    Map result = new LinkedHashMap();
+    result.put("znodeVersion",znodeVersion);
+    result.putAll(data);
+    return result;
+  }
 }

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=1641420&r1=1641419&r2=1641420&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 Mon Nov 24 16:57:01 2014
@@ -34,6 +34,7 @@ import java.util.concurrent.ExecutorServ
 import java.util.concurrent.Executors;
 
 import org.apache.solr.cloud.ZkController;
+import org.apache.solr.cloud.ZkSolrResourceLoader;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;
 import org.apache.solr.common.util.ExecutorUtil;
@@ -683,6 +684,7 @@ public class CoreContainer {
       // cancel recovery in cloud mode
       core.getSolrCoreState().cancelRecovery();
     }
+    String configSetZkPath =  core.getResourceLoader() instanceof ZkSolrResourceLoader ?  ((ZkSolrResourceLoader)core.getResourceLoader()).getConfigSetZkPath() : null;
 
     core.unloadOnClose(deleteIndexDir, deleteDataDir, deleteInstanceDir);
     if (close)
@@ -690,7 +692,7 @@ public class CoreContainer {
 
     if (zkSys.getZkController() != null) {
       try {
-        zkSys.getZkController().unregister(name, cd);
+        zkSys.getZkController().unregister(name, cd, configSetZkPath);
       } catch (InterruptedException e) {
         Thread.currentThread().interrupt();
         throw new SolrException(ErrorCode.SERVER_ERROR, "Interrupted while unregistering core [" + name + "] from cloud state");

Modified: lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/PluginInfo.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/PluginInfo.java?rev=1641420&r1=1641419&r2=1641420&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/PluginInfo.java (original)
+++ lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/PluginInfo.java Mon Nov 24 16:57:01 2014
@@ -29,7 +29,7 @@ import static java.util.Collections.unmo
  * An Object which represents a Plugin of any type 
  *
  */
-public class PluginInfo {
+public class PluginInfo implements MapSerializable{
   public final String name, className, type;
   public final NamedList initArgs;
   public final Map<String, String> attributes;
@@ -95,6 +95,28 @@ public class PluginInfo {
     List<PluginInfo> l = getChildren(type);
     return  l.isEmpty() ? null:l.get(0);
   }
+ public Map<String,Object> toMap(){
+    LinkedHashMap m = new LinkedHashMap(attributes);
+    if(initArgs!=null ) m.putAll(initArgs.asMap(3));
+    if(children != null){
+      for (PluginInfo child : children) {
+        Object old = m.get(child.name);
+        if(old == null){
+          m.put(child.name, child.toMap());
+        } else if (old instanceof List) {
+          List list = (List) old;
+          list.add(child.toMap());
+        }  else {
+          ArrayList l = new ArrayList();
+          l.add(old);
+          l.add(child.toMap());
+          m.put(child.name,l);
+        }
+      }
+
+    }
+    return m;
+  }
 
   /**Filter children by type
    * @param type The type name. must not be null

Modified: lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/SolrConfig.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/SolrConfig.java?rev=1641420&r1=1641419&r2=1641420&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/SolrConfig.java (original)
+++ lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/SolrConfig.java Mon Nov 24 16:57:01 2014
@@ -18,12 +18,14 @@
 package org.apache.solr.core;
 
 
+import com.google.common.collect.ImmutableList;
 import org.apache.lucene.index.IndexDeletionPolicy;
 import org.apache.lucene.search.BooleanQuery;
 import org.apache.lucene.util.Version;
 import org.apache.solr.cloud.ZkSolrResourceLoader;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;
+import org.apache.solr.common.cloud.ZkNodeProps;
 import org.apache.solr.handler.component.SearchComponent;
 import org.apache.solr.request.SolrRequestHandler;
 import org.apache.solr.response.QueryResponseWriter;
@@ -42,6 +44,8 @@ import org.apache.solr.update.processor.
 import org.apache.solr.util.DOMUtil;
 import org.apache.solr.util.FileUtils;
 import org.apache.solr.util.RegexFileFilter;
+import org.noggit.JSONParser;
+import org.noggit.ObjectBuilder;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.w3c.dom.Node;
@@ -54,6 +58,8 @@ import javax.xml.xpath.XPathConstants;
 import java.io.File;
 import java.io.FileFilter;
 import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.EnumSet;
@@ -62,6 +68,8 @@ import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Locale;
 import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
@@ -76,7 +84,7 @@ import static org.apache.solr.core.SolrC
  * configuration data for a a Solr instance -- typically found in
  * "solrconfig.xml".
  */
-public class SolrConfig extends Config {
+public class SolrConfig extends Config implements MapSerializable{
 
   public static final Logger log = LoggerFactory.getLogger(SolrConfig.class);
   
@@ -165,6 +173,7 @@ public class SolrConfig extends Config {
   public SolrConfig(SolrResourceLoader loader, String name, InputSource is)
   throws ParserConfigurationException, IOException, SAXException {
     super(loader, name, is, "/config/");
+    getOverlay();//just in case it is not initialized
     initLibs();
     luceneMatchVersion = getLuceneVersion("luceneMatchVersion");
     String indexConfigPrefix;
@@ -254,48 +263,7 @@ public class SolrConfig extends Config {
     }
      maxWarmingSearchers = getInt("query/maxWarmingSearchers",Integer.MAX_VALUE);
      slowQueryThresholdMillis = getInt("query/slowQueryThresholdMillis", -1);
-
-     loadPluginInfo(SolrRequestHandler.class,"requestHandler",
-                    REQUIRE_NAME, REQUIRE_CLASS, MULTI_OK);
-     loadPluginInfo(QParserPlugin.class,"queryParser",
-                    REQUIRE_NAME, REQUIRE_CLASS, MULTI_OK);
-     loadPluginInfo(QueryResponseWriter.class,"queryResponseWriter",
-                    REQUIRE_NAME, REQUIRE_CLASS, MULTI_OK);
-     loadPluginInfo(ValueSourceParser.class,"valueSourceParser",
-                    REQUIRE_NAME, REQUIRE_CLASS, MULTI_OK);
-     loadPluginInfo(TransformerFactory.class,"transformer",
-                    REQUIRE_NAME, REQUIRE_CLASS, MULTI_OK);
-     loadPluginInfo(SearchComponent.class,"searchComponent",
-                    REQUIRE_NAME, REQUIRE_CLASS, MULTI_OK);
-
-     // TODO: WTF is up with queryConverter???
-     // it aparently *only* works as a singleton? - SOLR-4304
-     // and even then -- only if there is a single SpellCheckComponent
-     // because of queryConverter.setIndexAnalyzer
-     loadPluginInfo(QueryConverter.class,"queryConverter",
-                    REQUIRE_NAME, REQUIRE_CLASS);
-
-     // this is hackish, since it picks up all SolrEventListeners,
-     // regardless of when/how/why they are used (or even if they are
-     // declared outside of the appropriate context) but there's no nice
-     // way around that in the PluginInfo framework
-     loadPluginInfo(SolrEventListener.class, "//listener",
-                    REQUIRE_CLASS, MULTI_OK);
-
-     loadPluginInfo(DirectoryFactory.class,"directoryFactory",
-                    REQUIRE_CLASS);
-     loadPluginInfo(IndexDeletionPolicy.class,indexConfigPrefix+"/deletionPolicy",
-                    REQUIRE_CLASS);
-     loadPluginInfo(CodecFactory.class,"codecFactory",
-                    REQUIRE_CLASS);
-     loadPluginInfo(IndexReaderFactory.class,"indexReaderFactory",
-                    REQUIRE_CLASS);
-     loadPluginInfo(UpdateRequestProcessorChain.class,"updateRequestProcessorChain",
-                    MULTI_OK);
-     loadPluginInfo(UpdateLog.class,"updateHandler/updateLog");
-     loadPluginInfo(IndexSchemaFactory.class,"schemaFactory",
-                    REQUIRE_CLASS);
-     loadPluginInfo(RestManager.class, "restManager");
+    for (SolrPluginInfo plugin : plugins) loadPluginInfo(plugin);
      updateHandlerInfo = loadUpdatehandlerInfo();
      
      multipartUploadLimitKB = getInt( 
@@ -314,7 +282,6 @@ public class SolrConfig extends Config {
      addHttpRequestToContext = getBool( 
          "requestDispatcher/requestParsers/@addHttpRequestToContext", false );
 
-    loadPluginInfo(InitParams.class, InitParams.TYPE, MULTI_OK);
     List<PluginInfo> argsInfos =  pluginStore.get(InitParams.class.getName()) ;
     if(argsInfos!=null){
       Map<String,InitParams> argsMap = new HashMap<>();
@@ -329,6 +296,71 @@ public class SolrConfig extends Config {
     solrRequestParsers = new SolrRequestParsers(this);
     Config.log.info("Loaded SolrConfig: " + name);
   }
+
+  public static List<SolrPluginInfo> plugins = ImmutableList.<SolrPluginInfo>builder()
+      .add(new SolrPluginInfo(SolrRequestHandler.class, "requestHandler", REQUIRE_NAME, REQUIRE_CLASS, MULTI_OK))
+      .add(new SolrPluginInfo(QParserPlugin.class, "queryParser", REQUIRE_NAME, REQUIRE_CLASS, MULTI_OK))
+      .add(new SolrPluginInfo(QueryResponseWriter.class, "queryResponseWriter", REQUIRE_NAME, REQUIRE_CLASS, MULTI_OK))
+      .add(new SolrPluginInfo(ValueSourceParser.class, "valueSourceParser", REQUIRE_NAME, REQUIRE_CLASS, MULTI_OK))
+      .add(new SolrPluginInfo(TransformerFactory.class, "transformer", REQUIRE_NAME, REQUIRE_CLASS, MULTI_OK))
+      .add(new SolrPluginInfo(SearchComponent.class, "searchComponent", REQUIRE_NAME, REQUIRE_CLASS, MULTI_OK))
+      // TODO: WTF is up with queryConverter???
+      // it aparently *only* works as a singleton? - SOLR-4304
+      // and even then -- only if there is a single SpellCheckComponent
+      // because of queryConverter.setIndexAnalyzer
+      .add(new SolrPluginInfo(QueryConverter.class, "queryConverter", REQUIRE_NAME, REQUIRE_CLASS))
+      // this is hackish, since it picks up all SolrEventListeners,
+      // regardless of when/how/why they are used (or even if they are
+      // declared outside of the appropriate context) but there's no nice
+      // way around that in the PluginInfo framework
+      .add(new SolrPluginInfo(SolrEventListener.class, "//listener", REQUIRE_CLASS, MULTI_OK))
+      .add(new SolrPluginInfo(DirectoryFactory.class, "directoryFactory", REQUIRE_CLASS))
+      .add(new SolrPluginInfo(IndexDeletionPolicy.class, "indexConfig/deletionPolicy", REQUIRE_CLASS))
+      .add(new SolrPluginInfo(CodecFactory.class, "codecFactory", REQUIRE_CLASS))
+      .add(new SolrPluginInfo(IndexReaderFactory.class, "indexReaderFactory", REQUIRE_CLASS))
+      .add(new SolrPluginInfo(UpdateRequestProcessorChain.class,"updateRequestProcessorChain", MULTI_OK))
+      .add(new SolrPluginInfo(UpdateLog.class,"updateHandler/updateLog"))
+      .add(new SolrPluginInfo(IndexSchemaFactory.class, "schemaFactory", REQUIRE_CLASS))
+      .add(new SolrPluginInfo(RestManager.class, "restManager"))
+      .add(new SolrPluginInfo(InitParams.class, InitParams.TYPE, MULTI_OK))
+      .build();
+
+  public static class SolrPluginInfo{
+
+    public final Class clazz;
+    public final String tag;
+    public final Set<PluginOpts> options;
+
+
+    private SolrPluginInfo(Class clz, String tag, PluginOpts... opts) {
+      this.clazz = clz;
+      this.tag = tag;
+      this.options=  opts == null? Collections.EMPTY_SET :  EnumSet.of(NOOP, opts);
+    }
+  }
+
+  public static  ConfigOverlay getConfigOverlay(SolrResourceLoader loader) {
+    InputStream in = null;
+    try {
+      in = loader.openResource(ConfigOverlay.RESOURCE_NAME);
+    } catch (IOException e) {
+      //no problem no overlay.json file
+      return new ConfigOverlay(Collections.EMPTY_MAP,0);
+    }
+
+    try {
+      int version = 0; //will be always 0 for file based resourceloader
+      if (in instanceof ZkSolrResourceLoader.ZkByteArrayInputStream) {
+        version = ((ZkSolrResourceLoader.ZkByteArrayInputStream) in).getStat().getVersion();
+      }
+      Map m = (Map) ObjectBuilder.getVal(new JSONParser(new InputStreamReader(in)));
+      return new ConfigOverlay(m,version);
+    } catch (Exception e) {
+      throw new SolrException(ErrorCode.SERVER_ERROR,"Error reading config overlay",e);
+    }
+
+  }
+
   private Map<String,InitParams> initParams = Collections.emptyMap();
   public Map<String, InitParams> getInitParams() {
     return initParams;
@@ -345,20 +377,19 @@ public class SolrConfig extends Config {
             getBool("updateHandler/commitWithin/softCommit",true));
   }
 
-  private void loadPluginInfo(Class clazz, String tag, PluginOpts... opts) {
-    EnumSet<PluginOpts> options = EnumSet.<PluginOpts>of(NOOP, opts);
-    boolean requireName = options.contains(REQUIRE_NAME);
-    boolean requireClass = options.contains(REQUIRE_CLASS);
+  private void loadPluginInfo(SolrPluginInfo pluginInfo) {
+    boolean requireName = pluginInfo.options.contains(REQUIRE_NAME);
+    boolean requireClass = pluginInfo.options.contains(REQUIRE_CLASS);
 
-    List<PluginInfo> result = readPluginInfos(tag, requireName, requireClass);
+    List<PluginInfo> result = readPluginInfos(pluginInfo.tag, requireName, requireClass);
 
-    if (1 < result.size() && ! options.contains(MULTI_OK)) {
+    if (1 < result.size() && ! pluginInfo.options.contains(MULTI_OK)) {
         throw new SolrException
           (SolrException.ErrorCode.SERVER_ERROR,
            "Found " + result.size() + " configuration sections when at most "
-           + "1 is allowed matching expression: " + tag);
+           + "1 is allowed matching expression: " + pluginInfo.tag);
     }
-    if(!result.isEmpty()) pluginStore.put(clazz.getName(),result);
+    if(!result.isEmpty()) pluginStore.put(pluginInfo.clazz.getName(),result);
   }
 
   public List<PluginInfo> readPluginInfos(String tag, boolean requireName, boolean requireClass) {
@@ -423,7 +454,7 @@ public class SolrConfig extends Config {
     return httpCachingConfig;
   }
 
-  public static class JmxConfiguration {
+  public static class JmxConfiguration implements MapSerializable{
     public boolean enabled = false;
     public String agentId;
     public String serviceUrl;
@@ -446,9 +477,18 @@ public class SolrConfig extends Config {
       }
       
     }
+
+    @Override
+    public Map<String, Object> toMap() {
+      LinkedHashMap map = new LinkedHashMap();
+      map.put("agentId",agentId);
+      map.put("serviceUrl",serviceUrl);
+      map.put("rootName",rootName);
+      return map;
+    }
   }
 
-  public static class HttpCachingConfig {
+  public static class HttpCachingConfig implements MapSerializable{
 
     /** config xpath prefix for getting HTTP Caching options */
     private final static String CACHE_PRE
@@ -457,7 +497,15 @@ public class SolrConfig extends Config {
     /** For extracting Expires "ttl" from <cacheControl> config */
     private final static Pattern MAX_AGE
       = Pattern.compile("\\bmax-age=(\\d+)");
-    
+
+    @Override
+    public Map<String, Object> toMap() {
+      return ZkNodeProps.makeMap("never304",never304,
+          "etagSeed",etagSeed,
+          "lastModFrom",lastModFrom.name().toLowerCase(Locale.ROOT),
+          "cacheControl",cacheControlHeader);
+    }
+
     public static enum LastModFrom {
       OPENTIME, DIRLASTMOD, BOGUS;
 
@@ -517,7 +565,7 @@ public class SolrConfig extends Config {
     public LastModFrom getLastModFrom() { return lastModFrom; }
   }
 
-  public static class UpdateHandlerInfo{
+  public static class UpdateHandlerInfo implements MapSerializable{
     public final String className;
     public final int autoCommmitMaxDocs,autoCommmitMaxTime,commitIntervalLowerBound,
         autoSoftCommmitMaxDocs,autoSoftCommmitMaxTime;
@@ -543,7 +591,29 @@ public class SolrConfig extends Config {
       this.autoSoftCommmitMaxTime = autoSoftCommmitMaxTime;
       
       this.commitWithinSoftCommit = commitWithinSoftCommit;
-    } 
+    }
+
+
+
+    @Override
+    public Map<String, Object> toMap() {
+      LinkedHashMap result = new LinkedHashMap();
+      result.put("class",className);
+      result.put("autoCommmitMaxDocs",autoCommmitMaxDocs);
+      result.put("indexWriterCloseWaitsForMerges",indexWriterCloseWaitsForMerges);
+      result.put("openSearcher",openSearcher);
+      result.put("commitIntervalLowerBound",commitIntervalLowerBound);
+      result.put("commitWithinSoftCommit",commitWithinSoftCommit);
+      result.put("autoCommit", ZkNodeProps.makeMap(
+          "maxDocs", autoCommmitMaxDocs,
+          "maxTime",autoCommmitMaxTime,
+          "commitIntervalLowerBound", commitIntervalLowerBound
+      ));
+      result.put("autoSoftCommit" ,
+          ZkNodeProps.makeMap("maxDocs", autoSoftCommmitMaxDocs,
+              "maxTime",autoSoftCommmitMaxTime));
+      return result;
+    }
   }
 
 //  public Map<String, List<PluginInfo>> getUpdateProcessorChainInfo() { return updateProcessorChainInfo; }
@@ -631,5 +701,100 @@ public class SolrConfig extends Config {
     return enableRemoteStreams;
   }
 
+  @Override
+  public int getInt(String path) {
+    return getInt(path, 0);
+  }
+
+  @Override
+  public int getInt(String path, int def) {
+    Object v = overlay.getXPathProperty(path);
+
+    Object val = overlay.getXPathProperty(path);
+    if (val != null) return Integer.parseInt(val.toString());
+    return super.getInt(path, def);
+  }
+  @Override
+  public boolean getBool(String path, boolean def) {
+    Object val = overlay.getXPathProperty(path);
+    if (val != null) return Boolean.parseBoolean(val.toString());
+    return super.getBool(path, def);
+  }
+  @Override
+  public Map<String, Object> toMap() {
+    LinkedHashMap result = new LinkedHashMap();
+    if(getZnodeVersion() > -1) result.put("znodeVersion",getZnodeVersion());
+    result.put("luceneMatchVersion",luceneMatchVersion);
+    result.put("updateHandler", getUpdateHandlerInfo().toMap());
+    Map m = new LinkedHashMap();
+    result.put("query", m);
+    m.put("useFilterForSortedQuery", useFilterForSortedQuery);
+    m.put("queryResultWindowSize", queryResultWindowSize);
+    m.put("queryResultMaxDocsCached", queryResultMaxDocsCached);
+    m.put("enableLazyFieldLoading", enableLazyFieldLoading);
+    m.put("maxBooleanClauses", booleanQueryMaxClauseCount);
+
+    for (SolrPluginInfo plugin : plugins) {
+      List<PluginInfo> infos = getPluginInfos(plugin.clazz.getName());
+      if(infos == null || infos.isEmpty()) continue;
+      String tag = plugin.tag;
+      tag = tag.replace("/","");
+      if(plugin.options.contains(PluginOpts.REQUIRE_NAME)){
+        LinkedHashMap items = new LinkedHashMap();
+        for (PluginInfo info : infos) items.put(info.name, info.toMap());
+        result.put(tag,items);
+      } else {
+        if(plugin.options.contains(MULTI_OK)){
+          ArrayList<Map> l = new ArrayList<>();
+          for (PluginInfo info : infos) l.add(info.toMap());
+          result.put(tag,l);
+        } else {
+          result.put(tag, infos.get(0).toMap());
+        }
+
+      }
+
+    }
+
+
+    addCacheConfig(m,filterCacheConfig,queryResultCacheConfig,documentCacheConfig,fieldValueCacheConfig);
+    if(jmxConfig != null) result.put("jmx",jmxConfig.toMap());
+    m = new LinkedHashMap();
+    result.put("requestDispatcher", m);
+    m.put("handleSelect",handleSelect);
+    if(httpCachingConfig!=null) m.put("httpCaching", httpCachingConfig.toMap());
+    m.put("requestParsers", ZkNodeProps.makeMap("multipartUploadLimitKB",multipartUploadLimitKB,
+        "formUploadLimitKB",formUploadLimitKB,
+        "addHttpRequestToContext",addHttpRequestToContext));
+    if(indexConfig != null) result.put("indexConfig",indexConfig.toMap());
+
+    //TODO there is more to add
+
+    return result;
+  }
+
+  private void addCacheConfig(Map queryMap, CacheConfig... cache) {
+    if(cache==null)return;
+    for (CacheConfig config : cache) if(config !=null) queryMap.put(config.getNodeName(),config.toMap());
+
+  }
+
+  @Override
+  protected Properties getSubstituteProperties() {
+    Map<String, Object> p = getOverlay().getUserProps();
+    if(p==null || p.isEmpty()) return super.getSubstituteProperties();
+    Properties result = new Properties(super.getSubstituteProperties());
+    result.putAll(p);
+    return result;
+  }
+  private ConfigOverlay overlay;
+
+  public ConfigOverlay getOverlay() {
+    if(overlay ==null) {
+      overlay = getConfigOverlay(getResourceLoader());
+    }
+    return overlay;
+  }
+
 
 }

Modified: lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/SolrCore.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/SolrCore.java?rev=1641420&r1=1641419&r2=1641420&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/SolrCore.java (original)
+++ lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/SolrCore.java Mon Nov 24 16:57:01 2014
@@ -77,6 +77,7 @@ import org.apache.solr.common.util.Simpl
 import org.apache.solr.core.DirectoryFactory.DirContext;
 import org.apache.solr.handler.RequestHandlerBase;
 import org.apache.solr.handler.SnapPuller;
+import org.apache.solr.handler.SolrConfigHandler;
 import org.apache.solr.handler.UpdateRequestHandler;
 import org.apache.solr.handler.admin.ShowFileRequestHandler;
 import org.apache.solr.handler.component.DebugComponent;
@@ -808,6 +809,8 @@ public final class SolrCore implements S
       reqHandlers = new RequestHandlers(this);
       List<PluginInfo> implicitReqHandlerInfo = new ArrayList<>();
       UpdateRequestHandler.addImplicits(implicitReqHandlerInfo);
+      SolrConfigHandler.addImplicits(implicitReqHandlerInfo);
+
       reqHandlers.initHandlersFromConfig(solrConfig, implicitReqHandlerInfo);
 
       // Handle things that should eventually go away

Modified: lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/SolrResourceLoader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/SolrResourceLoader.java?rev=1641420&r1=1641419&r2=1641420&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/SolrResourceLoader.java (original)
+++ lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/SolrResourceLoader.java Mon Nov 24 16:57:01 2014
@@ -53,8 +53,10 @@ import java.io.Closeable;
 import java.io.File;
 import java.io.FileFilter;
 import java.io.FileInputStream;
+import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.InputStream;
+import java.io.OutputStreamWriter;
 import java.lang.reflect.Constructor;
 import java.net.MalformedURLException;
 import java.net.URI;
@@ -823,4 +825,36 @@ public class SolrResourceLoader implemen
   public List<SolrInfoMBean> getInfoMBeans(){
     return Collections.unmodifiableList(infoMBeans);
   }
+
+
+  public static void persistConfLocally(SolrResourceLoader loader, String resourceName, byte[] content) {
+    // Persist locally
+    File managedSchemaFile = new File(loader.getConfigDir(), resourceName);
+    OutputStreamWriter writer = null;
+    try {
+      File parentDir = managedSchemaFile.getParentFile();
+      if ( ! parentDir.isDirectory()) {
+        if ( ! parentDir.mkdirs()) {
+          final String msg = "Can't create managed schema directory " + parentDir.getAbsolutePath();
+          log.error(msg);
+          throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, msg);
+        }
+      }
+      final FileOutputStream out = new FileOutputStream(managedSchemaFile);
+      out.write(content);
+      log.info("Upgraded to managed schema at " + managedSchemaFile.getPath());
+    } catch (IOException e) {
+      final String msg = "Error persisting managed schema " + managedSchemaFile;
+      log.error(msg, e);
+      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, msg, e);
+    } finally {
+      org.apache.commons.io.IOUtils.closeQuietly(writer);
+      try {
+        FileUtils.sync(managedSchemaFile);
+      } catch (IOException e) {
+        final String msg = "Error syncing the managed schema file " + managedSchemaFile;
+        log.error(msg, e);
+      }
+    }
+  }
 }

Copied: lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/handler/SolrConfigHandler.java (from r1636862, lucene/dev/trunk/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?p2=lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/handler/SolrConfigHandler.java&p1=lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/SolrConfigHandler.java&r1=1636862&r2=1641420&rev=1641420&view=diff
==============================================================================
--- lucene/dev/trunk/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 Mon Nov 24 16:57:01 2014
@@ -19,8 +19,8 @@ package org.apache.solr.handler;
 
 
 import java.io.IOException;
-import java.net.URL;
 import java.text.MessageFormat;
+import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -28,13 +28,10 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableSet;
-import org.apache.lucene.analysis.util.ResourceLoader;
-import org.apache.lucene.analysis.util.ResourceLoaderAware;
 import org.apache.solr.cloud.ZkController;
 import org.apache.solr.cloud.ZkSolrResourceLoader;
 import org.apache.solr.common.SolrException;
+import org.apache.solr.common.cloud.SolrZkClient;
 import org.apache.solr.common.cloud.ZkNodeProps;
 import org.apache.solr.common.params.CollectionParams;
 import org.apache.solr.common.params.CommonParams;
@@ -44,12 +41,11 @@ import org.apache.solr.common.params.Sol
 import org.apache.solr.common.util.ContentStream;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.StrUtils;
-import org.apache.solr.core.CloseHook;
 import org.apache.solr.core.ConfigOverlay;
+import org.apache.solr.core.CoreContainer;
 import org.apache.solr.core.PluginInfo;
 import org.apache.solr.core.SolrConfig;
 import org.apache.solr.core.SolrCore;
-import org.apache.solr.core.SolrInfoMBean;
 import org.apache.solr.core.SolrResourceLoader;
 import org.apache.solr.request.LocalSolrQueryRequest;
 import org.apache.solr.request.SolrQueryRequest;
@@ -94,30 +90,60 @@ public class SolrConfigHandler extends R
   public void inform(final SolrCore core) {
     if( ! (core.getResourceLoader() instanceof  ZkSolrResourceLoader)) return;
     final ZkSolrResourceLoader zkSolrResourceLoader = (ZkSolrResourceLoader) core.getResourceLoader();
-    if(zkSolrResourceLoader != null){
-      Runnable listener = new Runnable() {
-        @Override
-        public void run() {
-          try {
-            if(core.isClosed()) return;
-            Stat stat = zkSolrResourceLoader.getZkController().getZkClient().exists((zkSolrResourceLoader).getCollectionZkPath() + "/" + ConfigOverlay.RESOURCE_NAME, null, true);
-            if(stat == null) return;
-            if (stat.getVersion() >  core.getSolrConfig().getOverlay().getZnodeVersion()) {
-              core.getCoreDescriptor().getCoreContainer().reload(core.getName());
+    if(zkSolrResourceLoader != null)
+      zkSolrResourceLoader.getZkController().registerConfListenerForCore(
+          zkSolrResourceLoader.getConfigSetZkPath(),
+          core,
+          getListener(core, zkSolrResourceLoader));
+
+  }
+
+  private static Runnable getListener(SolrCore core, ZkSolrResourceLoader zkSolrResourceLoader) {
+    final String coreName = core.getName();
+    final CoreContainer cc = core.getCoreDescriptor().getCoreContainer();
+    final String overlayPath = (zkSolrResourceLoader).getConfigSetZkPath() + "/" + ConfigOverlay.RESOURCE_NAME;
+    final String solrConfigPath = (zkSolrResourceLoader).getConfigSetZkPath() + "/" + core.getSolrConfig().getName();
+    return new Runnable() {
+          @Override
+          public void run() {
+            log.info("config update_listener called");
+            SolrZkClient zkClient = cc.getZkController().getZkClient();
+            int solrConfigversion,overlayVersion;
+            try (SolrCore core = cc.getCore(coreName))  {
+              if (core.isClosed()) return;
+               solrConfigversion = core.getSolrConfig().getOverlay().getZnodeVersion();
+               overlayVersion = core.getSolrConfig().getZnodeVersion();
+            }
+
+            if (checkStale(zkClient, overlayPath, solrConfigversion) ||
+                checkStale(zkClient, solrConfigPath, overlayVersion)) {
+              log.info("core reload");
+              cc.reload(coreName);
             }
-          } catch (KeeperException.NoNodeException nne){
-            //no problem
-          } catch (KeeperException e) {
-            log.error("error refreshing solrconfig ", e);
-          } catch (InterruptedException e) {
-            Thread.currentThread().isInterrupted();
           }
-        }
-      };
+        };
+  }
 
-      zkSolrResourceLoader.getZkController().registerConfListenerForCore(zkSolrResourceLoader.getCollectionZkPath(), core,listener);
+  private static boolean checkStale(SolrZkClient zkClient,  String zkPath, int currentVersion)  {
+    try {
+      Stat stat = zkClient.exists(zkPath, null, true);
+      if(stat == null){
+        if(currentVersion>0) return true;
+        return false;
+      }
+      if (stat.getVersion() >  currentVersion) {
+        log.info(zkPath+" is stale will need an update from {} to {}", currentVersion,stat.getVersion());
+        return true;
+      }
+      return false;
+    } catch (KeeperException.NoNodeException nne){
+      //no problem
+    } catch (KeeperException e) {
+      log.error("error refreshing solrconfig ", e);
+    } catch (InterruptedException e) {
+      Thread.currentThread().isInterrupted();
     }
-
+    return false;
   }
 
 
@@ -136,8 +162,7 @@ public class SolrConfigHandler extends R
       String path = (String) req.getContext().get("path");
       if(path == null) path="/config";
       if("/config/overlay".equals(path)){
-        resp.add("overlay", req.getCore().getSolrConfig().getOverlay().toOutputFormat());
-        return;
+        resp.add("overlay", req.getCore().getSolrConfig().getOverlay().toMap());
       } else {
         List<String> parts =StrUtils.splitSmart(path, '/');
         if(parts.get(0).isEmpty()) parts.remove(0);
@@ -152,13 +177,32 @@ 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");
-    }
+      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;
+      }
+
       try {
-        for (ContentStream stream : streams) {
-          runCommandsTillSuccess(stream);
+        for (;;) {
+          ArrayList<CommandOperation> opsCopy = new ArrayList<>(ops.size());
+          ConfigOverlay overlay = SolrConfig.getConfigOverlay(req.getCore().getResourceLoader());
+          for (CommandOperation op : ops) opsCopy.add(op.getCopy());
+          try {
+            handleCommands(opsCopy, overlay);
+            break;
+          } catch (ZkController.ResourceModifiedInZkException e) {
+            //retry
+            log.info("Race condition, the node is modified in ZK by someone else " +e.getMessage());
+          }
         }
       } catch (Exception e) {
         resp.setException(e);
@@ -167,30 +211,21 @@ public class SolrConfigHandler extends R
 
     }
 
-    private void runCommandsTillSuccess(ContentStream stream) throws IOException {
-      for (;;) {
-        try {
-          handleCommands(stream);
-          break;
-        } catch (ZkController.ResourceModifiedInZkException e) {
-          log.info(e.getMessage());
-
-        }
-      }
-    }
-
-    private void handleCommands( ContentStream stream) throws IOException {
-    ConfigOverlay overlay = req.getCore().getSolrConfig().getOverlay();
-    List<CommandOperation> ops = CommandOperation.parse(stream.getReader());
+    private void handleCommands(List<CommandOperation> ops, ConfigOverlay overlay ) throws IOException {
     for (CommandOperation op : ops) {
-      if(SET_PROPERTY.equals( op.name) ){
-        overlay = applySetProp(op, overlay);
-      }else if(UNSET_PROPERTY.equals(op.name)){
-        overlay = applyUnset(op,overlay);
-      }else if(SET_USER_PROPERTY.equals(op.name)){
-        overlay = applySetUserProp(op ,overlay);
-      }else if(UNSET_USER_PROPERTY.equals(op.name)){
-        overlay = applyUnsetUserProp(op, overlay);
+      switch (op.name) {
+        case SET_PROPERTY:
+          overlay = applySetProp(op, overlay);
+          break;
+        case UNSET_PROPERTY:
+          overlay = applyUnset(op, overlay);
+          break;
+        case SET_USER_PROPERTY:
+          overlay = applySetUserProp(op, overlay);
+          break;
+        case UNSET_USER_PROPERTY:
+          overlay = applyUnsetUserProp(op, overlay);
+          break;
       }
     }
     List errs = CommandOperation.captureErrors(ops);
@@ -204,21 +239,6 @@ public class SolrConfigHandler extends R
       ZkController.persistConfigResourceToZooKeeper(loader,overlay.getZnodeVersion(),
           ConfigOverlay.RESOURCE_NAME,overlay.toByteArray(),true);
 
-      String collectionName = req.getCore().getCoreDescriptor().getCloudDescriptor().getCollectionName();
-      Map map = ZkNodeProps.makeMap(CoreAdminParams.ACTION, CollectionParams.CollectionAction.RELOAD.toString() ,
-          CollectionParams.NAME, collectionName);
-
-      SolrQueryRequest  solrQueryRequest = new LocalSolrQueryRequest(req.getCore(), new MapSolrParams(map));
-      SolrQueryResponse tmpResp = new SolrQueryResponse();
-      try {
-        //doing a collection reload
-        req.getCore().getCoreDescriptor().getCoreContainer().getCollectionsHandler().handleRequestBody(solrQueryRequest,tmpResp);
-      } catch (Exception e) {
-        String msg = MessageFormat.format("Unable to reload collection {0}", collectionName);
-        log.error(msg);
-        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, msg);
-      }
-
     } else {
       SolrResourceLoader.persistConfLocally(loader, ConfigOverlay.RESOURCE_NAME, overlay.toByteArray());
       req.getCore().getCoreDescriptor().getCoreContainer().reload(req.getCore().getName());

Modified: lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminHandler.java?rev=1641420&r1=1641419&r2=1641420&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminHandler.java (original)
+++ lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminHandler.java Mon Nov 24 16:57:01 2014
@@ -587,7 +587,7 @@ public class CoreAdminHandler extends Re
     catch (Exception ex) {
       if (coreContainer.isZooKeeperAware() && dcore != null && !preExisitingZkEntry) {
         try {
-          coreContainer.getZkController().unregister(dcore.getName(), dcore);
+          coreContainer.getZkController().unregister(dcore.getName(), dcore,null);
         } catch (InterruptedException e) {
           Thread.currentThread().interrupt();
           SolrException.log(log, null, e);

Modified: lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/handler/admin/EditFileRequestHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/handler/admin/EditFileRequestHandler.java?rev=1641420&r1=1641419&r2=1641420&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/handler/admin/EditFileRequestHandler.java (original)
+++ lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/handler/admin/EditFileRequestHandler.java Mon Nov 24 16:57:01 2014
@@ -282,7 +282,7 @@ public class EditFileRequestHandler exte
 
       if (coreContainer.isZooKeeperAware()) {
         try {
-          String confPath = ((ZkSolrResourceLoader) core.getResourceLoader()).getCollectionZkPath();
+          String confPath = ((ZkSolrResourceLoader) core.getResourceLoader()).getConfigSetZkPath();
 
           ZkController.downloadConfigDir(coreContainer.getZkController().getZkClient(), confPath,
               new File(coll, "conf"));

Modified: lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/handler/admin/ShowFileRequestHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/handler/admin/ShowFileRequestHandler.java?rev=1641420&r1=1641419&r2=1641420&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/handler/admin/ShowFileRequestHandler.java (original)
+++ lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/handler/admin/ShowFileRequestHandler.java Mon Nov 24 16:57:01 2014
@@ -304,7 +304,7 @@ public class ShowFileRequestHandler exte
 
     final ZkSolrResourceLoader loader = (ZkSolrResourceLoader) core
         .getResourceLoader();
-    String confPath = loader.getCollectionZkPath();
+    String confPath = loader.getConfigSetZkPath();
 
     String fname = req.getParams().get("file", null);
     if (fname == null) {

Modified: lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/rest/SolrConfigRestApi.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/rest/SolrConfigRestApi.java?rev=1641420&r1=1641419&r2=1641420&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/rest/SolrConfigRestApi.java (original)
+++ lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/rest/SolrConfigRestApi.java Mon Nov 24 16:57:01 2014
@@ -58,7 +58,7 @@ public class SolrConfigRestApi extends A
    */
   @Override
   public synchronized Restlet createInboundRoot() {
-
+/*
     log.info("createInboundRoot started for /config");
     
     router.attachDefault(RestManager.ManagedEndpoint.class);
@@ -70,6 +70,7 @@ public class SolrConfigRestApi extends A
 
     log.info("createInboundRoot complete for /config");
 
-    return router;
+    return router;*/
+    return null;
   }  
 }

Modified: lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/schema/ManagedIndexSchema.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/schema/ManagedIndexSchema.java?rev=1641420&r1=1641419&r2=1641420&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/schema/ManagedIndexSchema.java (original)
+++ lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/schema/ManagedIndexSchema.java Mon Nov 24 16:57:01 2014
@@ -165,7 +165,7 @@ public final class ManagedIndexSchema ex
     final ZkSolrResourceLoader zkLoader = (ZkSolrResourceLoader)loader;
     final ZkController zkController = zkLoader.getZkController();
     final SolrZkClient zkClient = zkController.getZkClient();
-    final String managedSchemaPath = zkLoader.getCollectionZkPath() + "/" + managedSchemaResourceName;
+    final String managedSchemaPath = zkLoader.getConfigSetZkPath() + "/" + managedSchemaResourceName;
     boolean success = true;
     boolean schemaChangedInZk = false;
     try {

Modified: lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/schema/ManagedIndexSchemaFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/schema/ManagedIndexSchemaFactory.java?rev=1641420&r1=1641419&r2=1641420&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/schema/ManagedIndexSchemaFactory.java (original)
+++ lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/schema/ManagedIndexSchemaFactory.java Mon Nov 24 16:57:01 2014
@@ -119,7 +119,7 @@ public class ManagedIndexSchemaFactory e
     } else { // ZooKeeper
       final ZkSolrResourceLoader zkLoader = (ZkSolrResourceLoader)loader;
       final SolrZkClient zkClient = zkLoader.getZkController().getZkClient();
-      final String managedSchemaPath = zkLoader.getCollectionZkPath() + "/" + managedSchemaResourceName;
+      final String managedSchemaPath = zkLoader.getConfigSetZkPath() + "/" + managedSchemaResourceName;
       Stat stat = new Stat();
       try {
         // Attempt to load the managed schema
@@ -224,7 +224,7 @@ public class ManagedIndexSchemaFactory e
       SolrResourceLoader loader = config.getResourceLoader();
       if (loader instanceof ZkSolrResourceLoader) {
         ZkSolrResourceLoader zkLoader = (ZkSolrResourceLoader)loader;
-        String nonManagedSchemaPath = zkLoader.getCollectionZkPath() + "/" + resourceName;
+        String nonManagedSchemaPath = zkLoader.getConfigSetZkPath() + "/" + resourceName;
         try {
           exists = zkLoader.getZkController().pathExists(nonManagedSchemaPath);
         } catch (InterruptedException e) {
@@ -349,7 +349,7 @@ public class ManagedIndexSchemaFactory e
     } else {
       // Rename the non-managed schema znode in ZooKeeper
       ZkSolrResourceLoader zkLoader = (ZkSolrResourceLoader)loader;
-      final String nonManagedSchemaPath = zkLoader.getCollectionZkPath() + "/" + resourceName;
+      final String nonManagedSchemaPath = zkLoader.getConfigSetZkPath() + "/" + resourceName;
       try {
         ZkController zkController = zkLoader.getZkController();
         ZkCmdExecutor zkCmdExecutor = new ZkCmdExecutor(zkController.getClientTimeout());

Modified: lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/schema/ZkIndexSchemaReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/schema/ZkIndexSchemaReader.java?rev=1641420&r1=1641419&r2=1641420&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/schema/ZkIndexSchemaReader.java (original)
+++ lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/schema/ZkIndexSchemaReader.java Mon Nov 24 16:57:01 2014
@@ -43,7 +43,7 @@ public class ZkIndexSchemaReader impleme
     this.managedIndexSchemaFactory = managedIndexSchemaFactory;
     ZkSolrResourceLoader zkLoader = (ZkSolrResourceLoader)managedIndexSchemaFactory.getResourceLoader();
     this.zkClient = zkLoader.getZkController().getZkClient();
-    managedSchemaPath = zkLoader.getCollectionZkPath() + "/" + managedIndexSchemaFactory.getManagedSchemaResourceName();
+    managedSchemaPath = zkLoader.getConfigSetZkPath() + "/" + managedIndexSchemaFactory.getManagedSchemaResourceName();
     createSchemaWatcher();
     zkLoader.getZkController().addOnReconnectListener(this);
   }

Modified: lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/search/CacheConfig.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/search/CacheConfig.java?rev=1641420&r1=1641419&r2=1641420&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/search/CacheConfig.java (original)
+++ lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/search/CacheConfig.java Mon Nov 24 16:57:01 2014
@@ -17,9 +17,14 @@
 
 package org.apache.solr.search;
 
+import org.apache.solr.common.util.StrUtils;
+import org.apache.solr.core.MapSerializable;
 import org.w3c.dom.Node;
 import org.w3c.dom.NodeList;
 
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
 import java.util.Map;
 
 import org.apache.solr.common.SolrException;
@@ -36,7 +41,7 @@ import javax.xml.xpath.XPathConstants;
  *
  *
  */
-public class CacheConfig {
+public class CacheConfig implements MapSerializable{
   private String nodeName;
 
   private Class<? extends SolrCache> clazz;
@@ -70,7 +75,7 @@ public class CacheConfig {
     if (nodes==null || nodes.getLength()==0) return null;
     CacheConfig[] configs = new CacheConfig[nodes.getLength()];
     for (int i=0; i<nodes.getLength(); i++) {
-      configs[i] = getConfig(solrConfig, nodes.item(i));
+      configs[i] = getConfig(solrConfig, nodes.item(i).getNodeName(), DOMUtil.toMap(nodes.item(i).getAttributes()), configPath);
     }
     return configs;
   }
@@ -78,15 +83,29 @@ public class CacheConfig {
 
   public static CacheConfig getConfig(SolrConfig solrConfig, String xpath) {
     Node node = solrConfig.getNode(xpath, false);
-    return getConfig(solrConfig, node);
+    if(node == null) {
+      Map<String, String> m = solrConfig.getOverlay().getEditableSubProperties(xpath);
+      if(m==null) return null;
+      List<String> parts = StrUtils.splitSmart(xpath, '/');
+      return getConfig(solrConfig,parts.get(parts.size()-1) , Collections.EMPTY_MAP,xpath);
+    }
+    return getConfig(solrConfig, node.getNodeName(),DOMUtil.toMap(node.getAttributes()), xpath);
   }
 
 
-  public static CacheConfig getConfig(SolrConfig solrConfig, Node node) {
-    if (node==null) return null;
+  public static CacheConfig getConfig(SolrConfig solrConfig, String nodeName, Map<String,String> attrs, String xpath) {
     CacheConfig config = new CacheConfig();
-    config.nodeName = node.getNodeName();
-    config.args = DOMUtil.toMap(node.getAttributes());
+    config.nodeName = nodeName;
+    config.args = attrs;
+
+    Map<String, String> map = solrConfig.getOverlay().getEditableSubProperties(xpath);
+    if(map != null){
+      HashMap<String, String> mapCopy = new HashMap<>(config.args);
+      for (Map.Entry<String, String> e : map.entrySet()) {
+        mapCopy.put(e.getKey(),String.valueOf(e.getValue()));
+      }
+      config.args = mapCopy;
+    }
     String nameAttr = config.args.get("name");  // OPTIONAL
     if (nameAttr==null) {
       config.args.put("name",config.nodeName);
@@ -94,6 +113,7 @@ public class CacheConfig {
 
     SolrResourceLoader loader = solrConfig.getResourceLoader();
     config.cacheImpl = config.args.get("class");
+    if(config.cacheImpl == null) config.cacheImpl = "solr.LRUCache";
     config.regenImpl = config.args.get("regenerator");
     config.clazz = loader.findClass(config.cacheImpl, SolrCache.class);
     if (config.regenImpl != null) {
@@ -116,4 +136,15 @@ public class CacheConfig {
     }
   }
 
+  @Override
+  public Map<String, Object> toMap() {
+    Map result = Collections.unmodifiableMap(args);
+    return result;
+  }
+
+  public String getNodeName() {
+    return nodeName;
+  }
+
+
 }

Modified: lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java?rev=1641420&r1=1641419&r2=1641420&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java (original)
+++ lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java Mon Nov 24 16:57:01 2014
@@ -344,7 +344,7 @@ public class SolrDispatchFilter extends 
 
           // Handle /schema/* and /config/* paths via Restlet
           if( path.equals("/schema") || path.startsWith("/schema/")
-              || path.equals("/config") || path.startsWith("/config/")) {
+              /*|| path.equals("/config") || path.startsWith("/config/")*/) {
             solrReq = parser.parse(core, path, req);
             SolrRequestInfo.setRequestInfo(new SolrRequestInfo(solrReq, new SolrQueryResponse()));
             if( path.equals(req.getServletPath()) ) {

Modified: lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/update/SolrIndexConfig.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/update/SolrIndexConfig.java?rev=1641420&r1=1641419&r2=1641420&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/update/SolrIndexConfig.java (original)
+++ lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/update/SolrIndexConfig.java Mon Nov 24 16:57:01 2014
@@ -24,7 +24,9 @@ import org.apache.lucene.util.PrintStrea
 import org.apache.lucene.util.Version;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;
+import org.apache.solr.common.cloud.ZkNodeProps;
 import org.apache.solr.common.util.NamedList;
+import org.apache.solr.core.MapSerializable;
 import org.apache.solr.core.SolrConfig;
 import org.apache.solr.core.PluginInfo;
 import org.apache.solr.schema.IndexSchema;
@@ -36,12 +38,13 @@ import java.io.File;
 import java.io.FileOutputStream;
 import java.io.PrintStream;
 import java.util.List;
+import java.util.Map;
 
 /**
  * This config object encapsulates IndexWriter config params,
  * defined in the &lt;indexConfig&gt; section of solrconfig.xml
  */
-public class SolrIndexConfig {
+public class SolrIndexConfig implements MapSerializable {
   public static final Logger log = LoggerFactory.getLogger(SolrIndexConfig.class);
   
   final String defaultMergePolicyClassName;
@@ -173,6 +176,19 @@ public class SolrIndexConfig {
 
     checkIntegrityAtMerge = solrConfig.getBool(prefix + "/checkIntegrityAtMerge", def.checkIntegrityAtMerge);
   }
+  @Override
+  public Map<String, Object> toMap() {
+    Map<String, Object> m = ZkNodeProps.makeMap("maxBufferedDocs", maxBufferedDocs,
+        "maxMergeDocs", maxMergeDocs,
+        "maxIndexingThreads", maxIndexingThreads,
+        "mergeFactor", mergeFactor,
+        "ramBufferSizeMB", ramBufferSizeMB,
+        "writeLockTimeout", writeLockTimeout,
+        "lockType", lockType);
+    if(mergeSchedulerInfo != null) m.put("mergeScheduler",mergeSchedulerInfo.toMap());
+    if(mergePolicyInfo != null) m.put("mergeScheduler",mergePolicyInfo.toMap());
+    return m;
+  }
 
   /*
    * Assert that assertCondition is true.

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=1641420&r1=1641419&r2=1641420&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 Mon Nov 24 16:57:01 2014
@@ -56,13 +56,13 @@ public  class CommandOperation {
     if (commandData instanceof Map) {
       return (Map) commandData;
     }
-    addError(MessageFormat.format("The command {0} should have the values as a json object {key:val} format", name));
+    addError(MessageFormat.format("The command ''{0}'' should have the values as a json object {key:val} format", name));
     return Collections.EMPTY_MAP;
   }
 
   private Object getRootPrimitive(){
     if (commandData instanceof Map) {
-      errors.add(MessageFormat.format("The value has to be a string for command : {1}",name));
+      errors.add(MessageFormat.format("The value has to be a string for command : ''{0}'' ",name));
       return null;
     }
     return commandData;
@@ -99,7 +99,12 @@ public  class CommandOperation {
    * single value collection is returned
    */
   public List<String> getStrs(String key, List<String> def){
-    Object v = getMapVal(key);
+    Object v = null;
+    if(ROOT_OBJ.equals(key)) {
+      v = getRootPrimitive();
+    } else {
+      v = getMapVal(key);
+    }
     if(v == null){
       return def;
     } else {
@@ -205,5 +210,8 @@ public  class CommandOperation {
     }
 
   }
+  public CommandOperation getCopy(){
+    return new CommandOperation(name,commandData);
+  }
 
 }



Re: svn commit: r1641420 [1/2] - in /lucene/dev/branches/branch_5x/solr: ./ core/ core/src/java/org/apache/solr/cloud/ core/src/java/org/apache/solr/core/ core/src/java/org/apache/solr/handler/ core/src/java/org/apache/solr/handler/admin/ core/src/java/org...

Posted by Mark Miller <ma...@gmail.com>.
Careful, you wiped out a bunch of CHANGES entries from 5x.

- Mark

http://about.me/markrmiller

> On Nov 24, 2014, at 11:57 AM, noble@apache.org wrote:
> 
> Author: noble
> Date: Mon Nov 24 16:57:01 2014
> New Revision: 1641420
> 
> URL: http://svn.apache.org/r1641420
> Log:
> SOLR-6533 , SOLR-6715 , SOLR-6654 Backporting all the changes from trunk
> 
> Added:
>    lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/ConfigOverlay.java
>      - copied, changed from r1636862, lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/ConfigOverlay.java
>    lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/MapSerializable.java
>      - copied unchanged from r1636862, lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/MapSerializable.java
>    lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/handler/SolrConfigHandler.java
>      - copied, changed from r1636862, lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/SolrConfigHandler.java
>    lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/core/TestConfigOverlay.java
>      - copied unchanged from r1636862, lucene/dev/trunk/solr/core/src/test/org/apache/solr/core/TestConfigOverlay.java
>    lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/core/TestSolrConfigHandler.java
>      - copied unchanged from r1636862, lucene/dev/trunk/solr/core/src/test/org/apache/solr/core/TestSolrConfigHandler.java
>    lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/handler/TestConfigReload.java
>      - copied unchanged from r1640857, lucene/dev/trunk/solr/core/src/test/org/apache/solr/handler/TestConfigReload.java
>    lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/handler/TestSolrConfigHandlerConcurrent.java
>      - copied, changed from r1636862, lucene/dev/trunk/solr/core/src/test/org/apache/solr/handler/TestSolrConfigHandlerConcurrent.java
> Modified:
>    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/CloudUtil.java
>    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/cloud/ZkSolrResourceLoader.java
>    lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/Config.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/core/PluginInfo.java
>    lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/SolrConfig.java
>    lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/SolrCore.java
>    lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/SolrResourceLoader.java
>    lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminHandler.java
>    lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/handler/admin/EditFileRequestHandler.java
>    lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/handler/admin/ShowFileRequestHandler.java
>    lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/rest/SolrConfigRestApi.java
>    lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/schema/ManagedIndexSchema.java
>    lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/schema/ManagedIndexSchemaFactory.java
>    lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/schema/ZkIndexSchemaReader.java
>    lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/search/CacheConfig.java
>    lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
>    lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/update/SolrIndexConfig.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-files/solr/collection1/conf/solrconfig-managed-schema.xml
>    lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/rest/TestRestManager.java
>    lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/rest/schema/TestBulkSchemaAPI.java
>    lucene/dev/branches/branch_5x/solr/solrj/   (props changed)
>    lucene/dev/branches/branch_5x/solr/solrj/src/java/org/apache/solr/common/cloud/SolrZkClient.java
>    lucene/dev/branches/branch_5x/solr/test-framework/   (props changed)
>    lucene/dev/branches/branch_5x/solr/test-framework/src/java/org/apache/solr/util/TestHarness.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=1641420&r1=1641419&r2=1641420&view=diff
> ==============================================================================
> --- lucene/dev/branches/branch_5x/solr/CHANGES.txt (original)
> +++ lucene/dev/branches/branch_5x/solr/CHANGES.txt Mon Nov 24 16:57:01 2014
> @@ -178,6 +178,8 @@ New Features
>   facet.pivot which refers to a 'tag' local param in one or more stats.field params.
>   (hossman, Vitaliy Zhovtyuk, Steve Molloy)
> 
> +* SOLR-6533: Support editing common solrconfig.xml values (Noble Paul)
> +
> Bug Fixes
> ----------------------
> 
> @@ -365,23 +367,8 @@ Other Changes
>   References to zkCredentialProvider in System properties or configurations should be
>   changed to zkCredentialsProvider.  (Gregory Chanan)
> 
> -* SOLR-6697: bin/solr start scripts allow setting SOLR_OPTS in solr.in.* (janhoy)
> -
> -* SOLR-6739: Admin UI - Sort list of command line args (steffkes)
> -
> -* SOLR-6740: Admin UI - improve Files View (steffkes)
> -
> -* SOLR-6570: Run SolrZkClient session watch asynchronously. 
> - (Ramkumar Aiyengar via Mark Miller)
> - 
> -* SOLR-6747: Add an optional caching option as a workaround for SOLR-6586.
> -  (Mark Miller, Gregory Chanan)
> -
> -* SOLR-6459: Normalize logging of operations in Overseer and log current queue size.
> -  (Ramkumar Aiyengar via Mark Miller)
> -
> -* SOLR-6754: ZkController.publish doesn't use the updateLastState parameter.
> -  (shalin)
> +* SOLR-6715: ZkSolrResourceLoader constructors accept a parameter called 'collection'
> +  but it should be 'configName'. (shalin)
> 
> ==================  4.10.3 ==================
> 
> 
> Modified: lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/cloud/CloudUtil.java
> URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/cloud/CloudUtil.java?rev=1641420&r1=1641419&r2=1641420&view=diff
> ==============================================================================
> --- lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/cloud/CloudUtil.java (original)
> +++ lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/cloud/CloudUtil.java Mon Nov 24 16:57:01 2014
> @@ -88,7 +88,7 @@ public class CloudUtil {
> 
>   /**
>    * Returns a displayable unified path to the given resource. For non-solrCloud that will be the
> -   * same as getConfigDir, but for Cloud it will be getCollectionZkPath ending in a /
> +   * same as getConfigDir, but for Cloud it will be getConfigSetZkPath ending in a /
>    * <p/>
>    * <b>Note:</b> Do not use this to generate a valid file path, but for debug printing etc
>    * @param loader Resource loader instance
> @@ -96,7 +96,7 @@ public class CloudUtil {
>    */
>   public static String unifiedResourcePath(SolrResourceLoader loader) {
>     return (loader instanceof ZkSolrResourceLoader) ?
> -            ((ZkSolrResourceLoader) loader).getCollectionZkPath() + "/" :
> +            ((ZkSolrResourceLoader) loader).getConfigSetZkPath() + "/" :
>             loader.getConfigDir();
>   }
> }
> 
> 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=1641420&r1=1641419&r2=1641420&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 Mon Nov 24 16:57:01 2014
> @@ -20,18 +20,17 @@ package org.apache.solr.cloud;
> import java.io.File;
> import java.io.IOException;
> import java.io.UnsupportedEncodingException;
> -import java.net.ConnectException;
> import java.net.InetAddress;
> import java.net.NetworkInterface;
> -import java.net.SocketException;
> -import java.net.SocketTimeoutException;
> import java.net.URLEncoder;
> import java.net.UnknownHostException;
> +import java.text.MessageFormat;
> import java.util.ArrayList;
> import java.util.Collection;
> import java.util.Collections;
> import java.util.Enumeration;
> import java.util.HashMap;
> +import java.util.HashSet;
> import java.util.Iterator;
> import java.util.LinkedHashMap;
> import java.util.List;
> @@ -45,8 +44,6 @@ import java.util.concurrent.TimeoutExcep
> 
> import org.apache.commons.io.FileUtils;
> import org.apache.commons.lang.StringUtils;
> -import org.apache.http.NoHttpResponseException;
> -import org.apache.http.conn.ConnectTimeoutException;
> import org.apache.solr.client.solrj.impl.HttpSolrServer;
> import org.apache.solr.client.solrj.request.CoreAdminRequest.WaitForState;
> import org.apache.solr.common.SolrException;
> @@ -72,9 +69,11 @@ import org.apache.solr.common.cloud.ZooK
> import org.apache.solr.common.params.CollectionParams;
> import org.apache.solr.common.params.SolrParams;
> import org.apache.solr.common.util.URLUtil;
> +import org.apache.solr.core.CloseHook;
> import org.apache.solr.core.CoreContainer;
> import org.apache.solr.core.CoreDescriptor;
> import org.apache.solr.core.SolrCore;
> +import org.apache.solr.core.SolrResourceLoader;
> import org.apache.solr.handler.component.ShardHandler;
> import org.apache.solr.update.UpdateLog;
> import org.apache.solr.update.UpdateShardHandler;
> @@ -83,9 +82,9 @@ import org.apache.zookeeper.KeeperExcept
> import org.apache.zookeeper.KeeperException.ConnectionLossException;
> import org.apache.zookeeper.KeeperException.NoNodeException;
> import org.apache.zookeeper.KeeperException.SessionExpiredException;
> +import org.apache.zookeeper.WatchedEvent;
> +import org.apache.zookeeper.Watcher;
> import org.apache.zookeeper.data.Stat;
> -import org.noggit.JSONParser;
> -import org.noggit.ObjectBuilder;
> import org.slf4j.Logger;
> import org.slf4j.LoggerFactory;
> 
> @@ -200,9 +199,9 @@ public final class ZkController {
> 
>   // keeps track of a list of objects that need to know a new ZooKeeper session was created after expiration occurred
>   private List<OnReconnect> reconnectListeners = new ArrayList<OnReconnect>();
> -  
> +
>   public ZkController(final CoreContainer cc, String zkServerAddress, int zkClientTimeout, int zkClientConnectTimeout, String localHost, String locaHostPort,
> -        String localHostContext, int leaderVoteWait, int leaderConflictResolveWait, boolean genericCoreNodeNames, final CurrentCoreDescriptorProvider registerOnReconnect) 
> +                      String localHostContext, int leaderVoteWait, int leaderConflictResolveWait, boolean genericCoreNodeNames, final CurrentCoreDescriptorProvider registerOnReconnect)
>       throws InterruptedException, TimeoutException, IOException
>   {
> 
> @@ -214,18 +213,18 @@ public final class ZkController {
>     // solr.xml to indicate the root context, instead of hostContext="" 
>     // which means the default of "solr"
>     localHostContext = trimLeadingAndTrailingSlashes(localHostContext);
> -    
> +
>     this.zkServerAddress = zkServerAddress;
>     this.localHostPort = locaHostPort;
>     this.localHostContext = localHostContext;
>     this.hostName = normalizeHostName(localHost);
> -    this.nodeName = generateNodeName(this.hostName, 
> -                                     this.localHostPort, 
> -                                     this.localHostContext);
> +    this.nodeName = generateNodeName(this.hostName,
> +        this.localHostPort,
> +        this.localHostContext);
> 
>     this.leaderVoteWait = leaderVoteWait;
>     this.leaderConflictResolveWait = leaderConflictResolveWait;
> -    
> +
>     this.clientTimeout = zkClientTimeout;
>     DefaultConnectionStrategy strat = new DefaultConnectionStrategy();
>     String zkACLProviderClass = cc.getConfig().getZkACLProviderClass();
> @@ -242,12 +241,13 @@ public final class ZkController {
>     } else {
>       strat.setZkCredentialsToAddAutomatically(new DefaultZkCredentialsProvider());
>     }
> -    
> +    addOnReconnectListener(getConfigDirListener());
> +
>     zkClient = new SolrZkClient(zkServerAddress, zkClientTimeout,
>         zkClientConnectTimeout, strat,
>         // on reconnect, reload cloud info
>         new OnReconnect() {
> -          
> +
>           @Override
>           public void command() {
>             log.info("ZooKeeper session re-connected ... refreshing core states after session expiration.");
> @@ -260,32 +260,32 @@ public final class ZkController {
>               // he is involved in the sync, and he certainly may not be
>               // ExecutorUtil.shutdownAndAwaitTermination(cc.getCmdDistribExecutor());
>               // we need to create all of our lost watches
> -              
> +
>               // seems we dont need to do this again...
>               // Overseer.createClientNodes(zkClient, getNodeName());
> -              
> +
>               cc.cancelCoreRecoveries();
> -              
> +
>               registerAllCoresAsDown(registerOnReconnect, false);
> -              
> +
>               if (!zkRunOnly) {
>                 ElectionContext context = new OverseerElectionContext(zkClient,
>                     overseer, getNodeName());
> -                
> +
>                 ElectionContext prevContext = overseerElector.getContext();
>                 if (prevContext != null) {
>                   prevContext.cancelElection();
>                 }
> -                
> +
>                 overseerElector.setup(context);
>                 overseerElector.joinElection(context, true);
>               }
> -              
> +
>               zkStateReader.createClusterStateWatchersAndUpdate();
> -              
> +
>               // we have to register as live first to pick up docs in the buffer
>               createEphemeralLiveNode();
> -              
> +
>               List<CoreDescriptor> descriptors = registerOnReconnect
>                   .getCurrentDescriptors();
>               // re register all descriptors
> @@ -330,20 +330,20 @@ public final class ZkController {
>                   SolrException.ErrorCode.SERVER_ERROR, "", e);
>             }
>           }
> -          
> +
>         }, new BeforeReconnect() {
> -          
> -          @Override
> -          public void command() {
> -            try {
> -              ZkController.this.overseer.close();
> -            } catch (Exception e) {
> -              log.error("Error trying to stop any Overseer threads", e);
> -            }
> -            markAllAsNotLeader(registerOnReconnect);
> -          }
> -        }, zkACLProvider);
> -    
> +
> +      @Override
> +      public void command() {
> +        try {
> +          ZkController.this.overseer.close();
> +        } catch (Exception e) {
> +          log.error("Error trying to stop any Overseer threads", e);
> +        }
> +        markAllAsNotLeader(registerOnReconnect);
> +      }
> +    }, zkACLProvider);
> +
>     this.overseerJobQueue = Overseer.getInQueue(zkClient);
>     this.overseerCollectionQueue = Overseer.getCollectionQueue(zkClient);
>     this.overseerRunningMap = Overseer.getRunningMap(zkClient);
> @@ -352,9 +352,9 @@ public final class ZkController {
>     cmdExecutor = new ZkCmdExecutor(zkClientTimeout);
>     leaderElector = new LeaderElector(zkClient);
>     zkStateReader = new ZkStateReader(zkClient);
> -    
> +
>     this.baseURL = zkStateReader.getBaseUrlForNodeName(this.nodeName);
> -    
> +
>     init(registerOnReconnect);
>   }
> 
> @@ -1179,7 +1179,7 @@ public final class ZkController {
>     return true;
>   }
> 
> -  public void unregister(String coreName, CoreDescriptor cd)
> +  public void unregister(String coreName, CoreDescriptor cd, String configLocation)
>       throws InterruptedException, KeeperException {
>     final String coreNodeName = cd.getCloudDescriptor().getCoreNodeName();
>     final String collection = cd.getCloudDescriptor().getCollectionName();
> @@ -1200,6 +1200,10 @@ public final class ZkController {
>     boolean removeWatch = true;
>     // if there is no SolrCore which is a member of this collection, remove the watch
>     for (SolrCore solrCore : cc.getCores()) {
> +      if (((ZkSolrResourceLoader)solrCore.getResourceLoader()).getConfigSetZkPath().equals(configLocation))
> +        configLocation = null; //if a core uses this config dir , then set it to null
> +
> +
>       CloudDescriptor cloudDesc = solrCore.getCoreDescriptor()
>           .getCloudDescriptor();
>       if (cloudDesc != null
> @@ -1216,6 +1220,13 @@ public final class ZkController {
>         ZkStateReader.COLLECTION_PROP, cloudDescriptor.getCollectionName(),
>         ZkStateReader.CORE_NODE_NAME_PROP, coreNodeName);
>     overseerJobQueue.offer(ZkStateReader.toJSON(m));
> +
> +    if(configLocation != null) {
> +      synchronized (confDirectoryListeners) {
> +        log.info("This conf directory is no more watched {0}",configLocation);
> +        confDirectoryListeners.remove(configLocation);
> +      }
> +    }
>   }
> 
>   public void createCollection(String collection) throws KeeperException,
> @@ -2098,4 +2109,182 @@ public final class ZkController {
>       }
>     }
>   }
> +
> +  /**
> +   * Persists a config file to ZooKeeper using optimistic concurrency.
> +   *
> +   * @return true on success
> +   */
> +  public static boolean persistConfigResourceToZooKeeper( SolrResourceLoader loader, int znodeVersion ,
> +                                                          String resourceName, byte[] content,
> +                                                          boolean createIfNotExists) {
> +    final ZkSolrResourceLoader zkLoader = (ZkSolrResourceLoader)loader;
> +    final ZkController zkController = zkLoader.getZkController();
> +    final SolrZkClient zkClient = zkController.getZkClient();
> +    final String resourceLocation = zkLoader.getConfigSetZkPath() + "/" + resourceName;
> +    String errMsg = "Failed to persist resource at {0} - version mismatch {1}";
> +    try {
> +      try {
> +        zkClient.setData(resourceLocation , content,znodeVersion, true);
> +        zkClient.setData(zkLoader.getConfigSetZkPath(),new byte[]{0},true);
> +      } catch (NoNodeException e) {
> +        if(createIfNotExists){
> +          try {
> +            zkClient.create(resourceLocation,content, CreateMode.PERSISTENT,true);
> +            zkClient.setData(zkLoader.getConfigSetZkPath(), new byte[]{0}, true);
> +          } catch (KeeperException.NodeExistsException nee) {
> +            try {
> +              Stat stat = zkClient.exists(resourceLocation, null, true);
> +              log.info("failed to set data version in zk is {} and expected version is {} ", stat.getVersion(),znodeVersion);
> +            } catch (Exception e1) {
> +              log.warn("could not get stat");
> +            }
> +
> +            log.info(MessageFormat.format(errMsg,resourceLocation,znodeVersion));
> +            throw new ResourceModifiedInZkException(ErrorCode.CONFLICT, MessageFormat.format(errMsg,resourceLocation,znodeVersion) + ", retry.");
> +          }
> +        }
> +      }
> +
> +    } catch (KeeperException.BadVersionException bve){
> +      log.info(MessageFormat.format(errMsg,resourceLocation));
> +      throw new ResourceModifiedInZkException(ErrorCode.CONFLICT, MessageFormat.format(errMsg,resourceLocation,znodeVersion) + ", retry.");
> +    }catch (ResourceModifiedInZkException e){
> +      throw e;
> +    } catch (Exception e) {
> +      if (e instanceof InterruptedException) {
> +        Thread.currentThread().interrupt(); // Restore the interrupted status
> +      }
> +      final String msg = "Error persisting resource at " + resourceLocation;
> +      log.error(msg, e);
> +      throw new SolrException(ErrorCode.SERVER_ERROR, msg, e);
> +    }
> +    return true;
> +  }
> +
> +  public static  class ResourceModifiedInZkException extends SolrException {
> +    public ResourceModifiedInZkException(ErrorCode code, String msg) {
> +      super(code, msg);
> +    }
> +  }
> +
> +  public void unRegisterConfListener(Runnable listener) {
> +    if(listener == null) return;
> +    synchronized (confDirectoryListeners){
> +      for (Set<Runnable> listeners : confDirectoryListeners.values()) {
> +        if(listeners != null) {
> +          if(listeners.remove(listener)) {
> +            log.info(" a listener was removed because of core close");
> +          }
> +        }
> +      }
> +    }
> +
> +  }
> +
> +  /**This will give a callback to the listener whenever a child is modified in the
> +   * conf directory. It is the responsibility of the listener to check if the individual
> +   * item of interest has been modified.  When the last core which was interested in
> +   * this conf directory is gone the listeners will be removed automatically.
> +   */
> +  public void registerConfListenerForCore(String confDir,SolrCore core, final Runnable listener){
> +    if(listener==null) throw new NullPointerException("listener cannot be null");
> +    synchronized (confDirectoryListeners){
> +      if(confDirectoryListeners.containsKey(confDir)){
> +        confDirectoryListeners.get(confDir).add(listener);
> +        core.addCloseHook(new CloseHook() {
> +          @Override
> +          public void preClose(SolrCore core) {
> +            unRegisterConfListener(listener);
> +          }
> +
> +          @Override
> +          public void postClose(SolrCore core) { }
> +        });
> +      } else {
> +        throw new SolrException(ErrorCode.SERVER_ERROR,"This conf directory is not valid");
> +      }
> +    }
> +  }
> +
> +  private final Map<String , Set<Runnable>> confDirectoryListeners =  new HashMap<>();
> +
> +  void watchZKConfDir(final String zkDir) {
> +    log.info("watch zkdir " + zkDir);
> +    if (!confDirectoryListeners.containsKey(zkDir)) {
> +      confDirectoryListeners.put(zkDir,  new HashSet<Runnable>());
> +      setConfWatcher(zkDir, new WatcherImpl(zkDir));
> +
> +    }
> +
> +
> +  }
> +  private class WatcherImpl implements Watcher{
> +    private final String zkDir ;
> +
> +    private WatcherImpl(String dir) {
> +      this.zkDir = dir;
> +    }
> +
> +    @Override
> +      public void process(WatchedEvent event) {
> +        try {
> +
> +          synchronized (confDirectoryListeners) {
> +            // if this is not among directories to be watched then don't set the watcher anymore
> +            if( !confDirectoryListeners.containsKey(zkDir)) {
> +              log.info("Watcher on {} is removed ", zkDir);
> +              return;
> +            }
> +            final Set<Runnable> listeners = confDirectoryListeners.get(zkDir);
> +            if (listeners != null && !listeners.isEmpty()) {
> +              new Thread() {
> +                //run these in a separate thread because this can be long running
> +                public void run() {
> +                  for (final Runnable listener : listeners)
> +                    try {
> +                      listener.run();
> +                    } catch (Exception e) {
> +                      log.warn("listener throws error", e);
> +                    }
> +                }
> +              }.start();
> +            }
> +
> +          }
> +
> +        } finally {
> +          if (Event.EventType.None.equals(event.getType())) {
> +            log.info("A node got unwatched for {}", zkDir);
> +            return;
> +          } else {
> +            setConfWatcher(zkDir,this);
> +          }
> +        }
> +      }
> +    }
> +
> +  private void setConfWatcher(String zkDir, Watcher watcher) {
> +    try {
> +      zkClient.exists(zkDir,watcher,true);
> +    } catch (KeeperException e) {
> +      log.error("failed to set watcher for conf dir {} ", zkDir);
> +    } catch (InterruptedException e) {
> +      Thread.currentThread().interrupt();
> +      log.error("failed to set watcher for conf dir {} ", zkDir);
> +    }
> +  }
> +
> +  public OnReconnect getConfigDirListener() {
> +    return new OnReconnect() {
> +      @Override
> +      public void command() {
> +        synchronized (confDirectoryListeners){
> +          for (String s : confDirectoryListeners.keySet()) {
> +            watchZKConfDir(s);
> +          }
> +        }
> +      }
> +    };
> +  }
> }
> 
> Modified: lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/cloud/ZkSolrResourceLoader.java
> URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/cloud/ZkSolrResourceLoader.java?rev=1641420&r1=1641419&r2=1641420&view=diff
> ==============================================================================
> --- lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/cloud/ZkSolrResourceLoader.java (original)
> +++ lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/cloud/ZkSolrResourceLoader.java Mon Nov 24 16:57:01 2014
> @@ -30,6 +30,7 @@ import org.apache.solr.common.cloud.ZooK
> import org.apache.solr.core.SolrResourceLoader;
> import org.apache.solr.schema.ZkIndexSchemaReader;
> import org.apache.zookeeper.KeeperException;
> +import org.apache.zookeeper.data.Stat;
> 
> /**
>  * ResourceLoader that works with ZooKeeper.
> @@ -37,15 +38,16 @@ import org.apache.zookeeper.KeeperExcept
>  */
> public class ZkSolrResourceLoader extends SolrResourceLoader {
> 
> -  private final String collectionZkPath;
> +  private final String configSetZkPath;
>   private ZkController zkController;
>   private ZkIndexSchemaReader zkIndexSchemaReader;
> 
> -  public ZkSolrResourceLoader(String instanceDir, String collection,
> +  public ZkSolrResourceLoader(String instanceDir, String configSet,
>       ZkController zooKeeperController) {
>     super(instanceDir);
>     this.zkController = zooKeeperController;
> -    collectionZkPath = ZkController.CONFIGS_ZKNODE + "/" + collection;
> +    configSetZkPath = ZkController.CONFIGS_ZKNODE + "/" + configSet;
> +    zkController.watchZKConfDir(configSetZkPath);
>   }
> 
>   /**
> @@ -56,11 +58,12 @@ public class ZkSolrResourceLoader extend
>    * the "lib/" directory in the specified instance directory.
>    * <p>
>    */
> -  public ZkSolrResourceLoader(String instanceDir, String collection, ClassLoader parent,
> +  public ZkSolrResourceLoader(String instanceDir, String configSet, ClassLoader parent,
>       Properties coreProperties, ZkController zooKeeperController) {
>     super(instanceDir, parent, coreProperties);
>     this.zkController = zooKeeperController;
> -    collectionZkPath = ZkController.CONFIGS_ZKNODE + "/" + collection;
> +    configSetZkPath = ZkController.CONFIGS_ZKNODE + "/" + configSet;
> +    zkController.watchZKConfDir(configSetZkPath);
>   }
> 
>   /**
> @@ -75,11 +78,12 @@ public class ZkSolrResourceLoader extend
>   @Override
>   public InputStream openResource(String resource) throws IOException {
>     InputStream is = null;
> -    String file = collectionZkPath + "/" + resource;
> +    String file = configSetZkPath + "/" + resource;
>     try {
>       if (zkController.pathExists(file)) {
> -        byte[] bytes = zkController.getZkClient().getData(file, null, null, true);
> -        return new ByteArrayInputStream(bytes);
> +        Stat stat = new Stat();
> +        byte[] bytes = zkController.getZkClient().getData(file, null, stat, true);
> +        return new ZkByteArrayInputStream(bytes, stat);
>       }
>     } catch (Exception e) {
>       throw new IOException("Error opening " + file, e);
> @@ -92,12 +96,26 @@ public class ZkSolrResourceLoader extend
>     }
>     if (is == null) {
>       throw new IOException("Can't find resource '" + resource
> -          + "' in classpath or '" + collectionZkPath + "', cwd="
> +          + "' in classpath or '" + configSetZkPath + "', cwd="
>           + System.getProperty("user.dir"));
>     }
>     return is;
>   }
> 
> +  public static class ZkByteArrayInputStream extends ByteArrayInputStream{
> +
> +    private final Stat stat;
> +    public ZkByteArrayInputStream(byte[] buf, Stat stat) {
> +      super(buf);
> +      this.stat = stat;
> +
> +    }
> +
> +    public Stat getStat(){
> +      return stat;
> +    }
> +  }
> +
>   @Override
>   public String getConfigDir() {
>     throw new ZooKeeperException(
> @@ -109,7 +127,7 @@ public class ZkSolrResourceLoader extend
>   public String[] listConfigDir() {
>     List<String> list;
>     try {
> -      list = zkController.getZkClient().getChildren(collectionZkPath, null, true);
> +      list = zkController.getZkClient().getChildren(configSetZkPath, null, true);
>     } catch (InterruptedException e) {
>       // Restore the interrupted status
>       Thread.currentThread().interrupt();
> @@ -124,8 +142,8 @@ public class ZkSolrResourceLoader extend
>     return list.toArray(new String[0]);
>   }
> 
> -  public String getCollectionZkPath() {
> -    return collectionZkPath;
> +  public String getConfigSetZkPath() {
> +    return configSetZkPath;
>   }
> 
>   public ZkController getZkController() {
> 
> Modified: lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/Config.java
> URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/Config.java?rev=1641420&r1=1641419&r2=1641420&view=diff
> ==============================================================================
> --- lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/Config.java (original)
> +++ lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/Config.java Mon Nov 24 16:57:01 2014
> @@ -18,6 +18,7 @@
> package org.apache.solr.core;
> 
> import org.apache.lucene.util.Version;
> +import org.apache.solr.cloud.ZkSolrResourceLoader;
> import org.apache.solr.common.SolrException;
> import org.apache.solr.util.DOMUtil;
> import org.apache.solr.util.SystemIdResolver;
> @@ -48,10 +49,12 @@ import javax.xml.xpath.XPathExpressionEx
> import javax.xml.xpath.XPathFactory;
> 
> import java.io.IOException;
> +import java.io.InputStream;
> import java.text.ParseException;
> import java.util.Arrays;
> import java.util.HashSet;
> import java.util.Map;
> +import java.util.Properties;
> import java.util.Set;
> import java.util.SortedMap;
> import java.util.SortedSet;
> @@ -73,6 +76,7 @@ public class Config {
>   private final String prefix;
>   private final String name;
>   private final SolrResourceLoader loader;
> +  private int zkVersion = -1;
> 
>   /**
>    * Builds a config from a resource name with no xpath prefix.
> @@ -102,7 +106,7 @@ public class Config {
>    * @param is the resource as a SAX InputSource
>    * @param prefix an optional prefix that will be preprended to all non-absolute xpath expressions
>    */
> -  public Config(SolrResourceLoader loader, String name, InputSource is, String prefix, boolean subProps) throws ParserConfigurationException, IOException, SAXException 
> +  public Config(SolrResourceLoader loader, String name, InputSource is, String prefix, boolean substituteProps) throws ParserConfigurationException, IOException, SAXException
>   {
>     if( loader == null ) {
>       loader = new SolrResourceLoader( null );
> @@ -112,9 +116,14 @@ public class Config {
>     this.prefix = (prefix != null && !prefix.endsWith("/"))? prefix + '/' : prefix;
>     try {
>       javax.xml.parsers.DocumentBuilderFactory dbf = DocumentBuilderFactory.newInstance();
> -      
> +
>       if (is == null) {
> -        is = new InputSource(loader.openConfig(name));
> +        InputStream in = loader.openConfig(name);
> +        if (in instanceof ZkSolrResourceLoader.ZkByteArrayInputStream) {
> +          zkVersion = ((ZkSolrResourceLoader.ZkByteArrayInputStream) in).getStat().getVersion();
> +          log.info("loaded config {} with version {} ",name,zkVersion);
> +        }
> +        is = new InputSource(in);
>         is.setSystemId(SystemIdResolver.createSystemIdFromResourceName(name));
>       }
> 
> @@ -138,8 +147,8 @@ public class Config {
>         // some XML parsers are broken and don't close the byte stream (but they should according to spec)
>         IOUtils.closeQuietly(is.getByteStream());
>       }
> -      if (subProps) {
> -        DOMUtil.substituteProperties(doc, loader.getCoreProperties());
> +      if (substituteProps) {
> +        DOMUtil.substituteProperties(doc, getSubstituteProperties());
>       }
>     } catch (ParserConfigurationException e)  {
>       SolrException.log(log, "Exception during parsing file: " + name, e);
> @@ -152,7 +161,11 @@ public class Config {
>       throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
>     }
>   }
> -  
> +
> +  protected Properties getSubstituteProperties() {
> +    return loader.getCoreProperties();
> +  }
> +
>   public Config(SolrResourceLoader loader, String name, Document doc) {
>     this.prefix = null;
>     this.doc = doc;
> @@ -207,7 +220,7 @@ public class Config {
>   }
> 
>   public void substituteProperties() {
> -    DOMUtil.substituteProperties(doc, loader.getCoreProperties());
> +    DOMUtil.substituteProperties(doc, getSubstituteProperties());
>   }
> 
> 
> @@ -459,6 +472,12 @@ public class Config {
>     return version;
>   }
> 
> +  /**If this config is loaded from zk the version is relevant other wise -1 is returned
> +   */
> +  public int getZnodeVersion(){
> +    return zkVersion;
> +  }
> +
>   public Config getOriginalConfig() {
>     return new Config(loader, null, origDoc);
>   }
> 
> Copied: lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/ConfigOverlay.java (from r1636862, lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/ConfigOverlay.java)
> URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/ConfigOverlay.java?p2=lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/ConfigOverlay.java&p1=lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/ConfigOverlay.java&r1=1636862&r2=1641420&rev=1641420&view=diff
> ==============================================================================
> --- lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/ConfigOverlay.java (original)
> +++ lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/ConfigOverlay.java Mon Nov 24 16:57:01 2014
> @@ -34,7 +34,7 @@ import org.noggit.JSONParser;
> import org.noggit.JSONWriter;
> import org.noggit.ObjectBuilder;
> 
> -public class ConfigOverlay {
> +public class ConfigOverlay implements MapSerializable{
>   private final int znodeVersion ;
>   private Map<String, Object> data;
>   private Map<String,Object> props;
> @@ -177,13 +177,6 @@ public class ConfigOverlay {
>     return out.toString();
>   }
> 
> -  public  Map toOutputFormat() {
> -    Map result = new LinkedHashMap();
> -    result.put("version",znodeVersion);
> -    result.putAll(data);
> -    return result;
> -  }
> -
> 
>   public static final String RESOURCE_NAME = "configoverlay.json";
> 
> @@ -254,4 +247,12 @@ public class ConfigOverlay {
>   public Map<String, Object> getUserProps() {
>     return userProps;
>   }
> +
> +  @Override
> +  public Map<String, Object> toMap() {
> +    Map result = new LinkedHashMap();
> +    result.put("znodeVersion",znodeVersion);
> +    result.putAll(data);
> +    return result;
> +  }
> }
> 
> 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=1641420&r1=1641419&r2=1641420&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 Mon Nov 24 16:57:01 2014
> @@ -34,6 +34,7 @@ import java.util.concurrent.ExecutorServ
> import java.util.concurrent.Executors;
> 
> import org.apache.solr.cloud.ZkController;
> +import org.apache.solr.cloud.ZkSolrResourceLoader;
> import org.apache.solr.common.SolrException;
> import org.apache.solr.common.SolrException.ErrorCode;
> import org.apache.solr.common.util.ExecutorUtil;
> @@ -683,6 +684,7 @@ public class CoreContainer {
>       // cancel recovery in cloud mode
>       core.getSolrCoreState().cancelRecovery();
>     }
> +    String configSetZkPath =  core.getResourceLoader() instanceof ZkSolrResourceLoader ?  ((ZkSolrResourceLoader)core.getResourceLoader()).getConfigSetZkPath() : null;
> 
>     core.unloadOnClose(deleteIndexDir, deleteDataDir, deleteInstanceDir);
>     if (close)
> @@ -690,7 +692,7 @@ public class CoreContainer {
> 
>     if (zkSys.getZkController() != null) {
>       try {
> -        zkSys.getZkController().unregister(name, cd);
> +        zkSys.getZkController().unregister(name, cd, configSetZkPath);
>       } catch (InterruptedException e) {
>         Thread.currentThread().interrupt();
>         throw new SolrException(ErrorCode.SERVER_ERROR, "Interrupted while unregistering core [" + name + "] from cloud state");
> 
> Modified: lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/PluginInfo.java
> URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/PluginInfo.java?rev=1641420&r1=1641419&r2=1641420&view=diff
> ==============================================================================
> --- lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/PluginInfo.java (original)
> +++ lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/PluginInfo.java Mon Nov 24 16:57:01 2014
> @@ -29,7 +29,7 @@ import static java.util.Collections.unmo
>  * An Object which represents a Plugin of any type 
>  *
>  */
> -public class PluginInfo {
> +public class PluginInfo implements MapSerializable{
>   public final String name, className, type;
>   public final NamedList initArgs;
>   public final Map<String, String> attributes;
> @@ -95,6 +95,28 @@ public class PluginInfo {
>     List<PluginInfo> l = getChildren(type);
>     return  l.isEmpty() ? null:l.get(0);
>   }
> + public Map<String,Object> toMap(){
> +    LinkedHashMap m = new LinkedHashMap(attributes);
> +    if(initArgs!=null ) m.putAll(initArgs.asMap(3));
> +    if(children != null){
> +      for (PluginInfo child : children) {
> +        Object old = m.get(child.name);
> +        if(old == null){
> +          m.put(child.name, child.toMap());
> +        } else if (old instanceof List) {
> +          List list = (List) old;
> +          list.add(child.toMap());
> +        }  else {
> +          ArrayList l = new ArrayList();
> +          l.add(old);
> +          l.add(child.toMap());
> +          m.put(child.name,l);
> +        }
> +      }
> +
> +    }
> +    return m;
> +  }
> 
>   /**Filter children by type
>    * @param type The type name. must not be null
> 
> Modified: lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/SolrConfig.java
> URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/SolrConfig.java?rev=1641420&r1=1641419&r2=1641420&view=diff
> ==============================================================================
> --- lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/SolrConfig.java (original)
> +++ lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/SolrConfig.java Mon Nov 24 16:57:01 2014
> @@ -18,12 +18,14 @@
> package org.apache.solr.core;
> 
> 
> +import com.google.common.collect.ImmutableList;
> import org.apache.lucene.index.IndexDeletionPolicy;
> import org.apache.lucene.search.BooleanQuery;
> import org.apache.lucene.util.Version;
> import org.apache.solr.cloud.ZkSolrResourceLoader;
> import org.apache.solr.common.SolrException;
> import org.apache.solr.common.SolrException.ErrorCode;
> +import org.apache.solr.common.cloud.ZkNodeProps;
> import org.apache.solr.handler.component.SearchComponent;
> import org.apache.solr.request.SolrRequestHandler;
> import org.apache.solr.response.QueryResponseWriter;
> @@ -42,6 +44,8 @@ import org.apache.solr.update.processor.
> import org.apache.solr.util.DOMUtil;
> import org.apache.solr.util.FileUtils;
> import org.apache.solr.util.RegexFileFilter;
> +import org.noggit.JSONParser;
> +import org.noggit.ObjectBuilder;
> import org.slf4j.Logger;
> import org.slf4j.LoggerFactory;
> import org.w3c.dom.Node;
> @@ -54,6 +58,8 @@ import javax.xml.xpath.XPathConstants;
> import java.io.File;
> import java.io.FileFilter;
> import java.io.IOException;
> +import java.io.InputStream;
> +import java.io.InputStreamReader;
> import java.util.ArrayList;
> import java.util.Collections;
> import java.util.EnumSet;
> @@ -62,6 +68,8 @@ import java.util.LinkedHashMap;
> import java.util.List;
> import java.util.Locale;
> import java.util.Map;
> +import java.util.Properties;
> +import java.util.Set;
> import java.util.regex.Matcher;
> import java.util.regex.Pattern;
> 
> @@ -76,7 +84,7 @@ import static org.apache.solr.core.SolrC
>  * configuration data for a a Solr instance -- typically found in
>  * "solrconfig.xml".
>  */
> -public class SolrConfig extends Config {
> +public class SolrConfig extends Config implements MapSerializable{
> 
>   public static final Logger log = LoggerFactory.getLogger(SolrConfig.class);
> 
> @@ -165,6 +173,7 @@ public class SolrConfig extends Config {
>   public SolrConfig(SolrResourceLoader loader, String name, InputSource is)
>   throws ParserConfigurationException, IOException, SAXException {
>     super(loader, name, is, "/config/");
> +    getOverlay();//just in case it is not initialized
>     initLibs();
>     luceneMatchVersion = getLuceneVersion("luceneMatchVersion");
>     String indexConfigPrefix;
> @@ -254,48 +263,7 @@ public class SolrConfig extends Config {
>     }
>      maxWarmingSearchers = getInt("query/maxWarmingSearchers",Integer.MAX_VALUE);
>      slowQueryThresholdMillis = getInt("query/slowQueryThresholdMillis", -1);
> -
> -     loadPluginInfo(SolrRequestHandler.class,"requestHandler",
> -                    REQUIRE_NAME, REQUIRE_CLASS, MULTI_OK);
> -     loadPluginInfo(QParserPlugin.class,"queryParser",
> -                    REQUIRE_NAME, REQUIRE_CLASS, MULTI_OK);
> -     loadPluginInfo(QueryResponseWriter.class,"queryResponseWriter",
> -                    REQUIRE_NAME, REQUIRE_CLASS, MULTI_OK);
> -     loadPluginInfo(ValueSourceParser.class,"valueSourceParser",
> -                    REQUIRE_NAME, REQUIRE_CLASS, MULTI_OK);
> -     loadPluginInfo(TransformerFactory.class,"transformer",
> -                    REQUIRE_NAME, REQUIRE_CLASS, MULTI_OK);
> -     loadPluginInfo(SearchComponent.class,"searchComponent",
> -                    REQUIRE_NAME, REQUIRE_CLASS, MULTI_OK);
> -
> -     // TODO: WTF is up with queryConverter???
> -     // it aparently *only* works as a singleton? - SOLR-4304
> -     // and even then -- only if there is a single SpellCheckComponent
> -     // because of queryConverter.setIndexAnalyzer
> -     loadPluginInfo(QueryConverter.class,"queryConverter",
> -                    REQUIRE_NAME, REQUIRE_CLASS);
> -
> -     // this is hackish, since it picks up all SolrEventListeners,
> -     // regardless of when/how/why they are used (or even if they are
> -     // declared outside of the appropriate context) but there's no nice
> -     // way around that in the PluginInfo framework
> -     loadPluginInfo(SolrEventListener.class, "//listener",
> -                    REQUIRE_CLASS, MULTI_OK);
> -
> -     loadPluginInfo(DirectoryFactory.class,"directoryFactory",
> -                    REQUIRE_CLASS);
> -     loadPluginInfo(IndexDeletionPolicy.class,indexConfigPrefix+"/deletionPolicy",
> -                    REQUIRE_CLASS);
> -     loadPluginInfo(CodecFactory.class,"codecFactory",
> -                    REQUIRE_CLASS);
> -     loadPluginInfo(IndexReaderFactory.class,"indexReaderFactory",
> -                    REQUIRE_CLASS);
> -     loadPluginInfo(UpdateRequestProcessorChain.class,"updateRequestProcessorChain",
> -                    MULTI_OK);
> -     loadPluginInfo(UpdateLog.class,"updateHandler/updateLog");
> -     loadPluginInfo(IndexSchemaFactory.class,"schemaFactory",
> -                    REQUIRE_CLASS);
> -     loadPluginInfo(RestManager.class, "restManager");
> +    for (SolrPluginInfo plugin : plugins) loadPluginInfo(plugin);
>      updateHandlerInfo = loadUpdatehandlerInfo();
> 
>      multipartUploadLimitKB = getInt( 
> @@ -314,7 +282,6 @@ public class SolrConfig extends Config {
>      addHttpRequestToContext = getBool( 
>          "requestDispatcher/requestParsers/@addHttpRequestToContext", false );
> 
> -    loadPluginInfo(InitParams.class, InitParams.TYPE, MULTI_OK);
>     List<PluginInfo> argsInfos =  pluginStore.get(InitParams.class.getName()) ;
>     if(argsInfos!=null){
>       Map<String,InitParams> argsMap = new HashMap<>();
> @@ -329,6 +296,71 @@ public class SolrConfig extends Config {
>     solrRequestParsers = new SolrRequestParsers(this);
>     Config.log.info("Loaded SolrConfig: " + name);
>   }
> +
> +  public static List<SolrPluginInfo> plugins = ImmutableList.<SolrPluginInfo>builder()
> +      .add(new SolrPluginInfo(SolrRequestHandler.class, "requestHandler", REQUIRE_NAME, REQUIRE_CLASS, MULTI_OK))
> +      .add(new SolrPluginInfo(QParserPlugin.class, "queryParser", REQUIRE_NAME, REQUIRE_CLASS, MULTI_OK))
> +      .add(new SolrPluginInfo(QueryResponseWriter.class, "queryResponseWriter", REQUIRE_NAME, REQUIRE_CLASS, MULTI_OK))
> +      .add(new SolrPluginInfo(ValueSourceParser.class, "valueSourceParser", REQUIRE_NAME, REQUIRE_CLASS, MULTI_OK))
> +      .add(new SolrPluginInfo(TransformerFactory.class, "transformer", REQUIRE_NAME, REQUIRE_CLASS, MULTI_OK))
> +      .add(new SolrPluginInfo(SearchComponent.class, "searchComponent", REQUIRE_NAME, REQUIRE_CLASS, MULTI_OK))
> +      // TODO: WTF is up with queryConverter???
> +      // it aparently *only* works as a singleton? - SOLR-4304
> +      // and even then -- only if there is a single SpellCheckComponent
> +      // because of queryConverter.setIndexAnalyzer
> +      .add(new SolrPluginInfo(QueryConverter.class, "queryConverter", REQUIRE_NAME, REQUIRE_CLASS))
> +      // this is hackish, since it picks up all SolrEventListeners,
> +      // regardless of when/how/why they are used (or even if they are
> +      // declared outside of the appropriate context) but there's no nice
> +      // way around that in the PluginInfo framework
> +      .add(new SolrPluginInfo(SolrEventListener.class, "//listener", REQUIRE_CLASS, MULTI_OK))
> +      .add(new SolrPluginInfo(DirectoryFactory.class, "directoryFactory", REQUIRE_CLASS))
> +      .add(new SolrPluginInfo(IndexDeletionPolicy.class, "indexConfig/deletionPolicy", REQUIRE_CLASS))
> +      .add(new SolrPluginInfo(CodecFactory.class, "codecFactory", REQUIRE_CLASS))
> +      .add(new SolrPluginInfo(IndexReaderFactory.class, "indexReaderFactory", REQUIRE_CLASS))
> +      .add(new SolrPluginInfo(UpdateRequestProcessorChain.class,"updateRequestProcessorChain", MULTI_OK))
> +      .add(new SolrPluginInfo(UpdateLog.class,"updateHandler/updateLog"))
> +      .add(new SolrPluginInfo(IndexSchemaFactory.class, "schemaFactory", REQUIRE_CLASS))
> +      .add(new SolrPluginInfo(RestManager.class, "restManager"))
> +      .add(new SolrPluginInfo(InitParams.class, InitParams.TYPE, MULTI_OK))
> +      .build();
> +
> +  public static class SolrPluginInfo{
> +
> +    public final Class clazz;
> +    public final String tag;
> +    public final Set<PluginOpts> options;
> +
> +
> +    private SolrPluginInfo(Class clz, String tag, PluginOpts... opts) {
> +      this.clazz = clz;
> +      this.tag = tag;
> +      this.options=  opts == null? Collections.EMPTY_SET :  EnumSet.of(NOOP, opts);
> +    }
> +  }
> +
> +  public static  ConfigOverlay getConfigOverlay(SolrResourceLoader loader) {
> +    InputStream in = null;
> +    try {
> +      in = loader.openResource(ConfigOverlay.RESOURCE_NAME);
> +    } catch (IOException e) {
> +      //no problem no overlay.json file
> +      return new ConfigOverlay(Collections.EMPTY_MAP,0);
> +    }
> +
> +    try {
> +      int version = 0; //will be always 0 for file based resourceloader
> +      if (in instanceof ZkSolrResourceLoader.ZkByteArrayInputStream) {
> +        version = ((ZkSolrResourceLoader.ZkByteArrayInputStream) in).getStat().getVersion();
> +      }
> +      Map m = (Map) ObjectBuilder.getVal(new JSONParser(new InputStreamReader(in)));
> +      return new ConfigOverlay(m,version);
> +    } catch (Exception e) {
> +      throw new SolrException(ErrorCode.SERVER_ERROR,"Error reading config overlay",e);
> +    }
> +
> +  }
> +
>   private Map<String,InitParams> initParams = Collections.emptyMap();
>   public Map<String, InitParams> getInitParams() {
>     return initParams;
> @@ -345,20 +377,19 @@ public class SolrConfig extends Config {
>             getBool("updateHandler/commitWithin/softCommit",true));
>   }
> 
> -  private void loadPluginInfo(Class clazz, String tag, PluginOpts... opts) {
> -    EnumSet<PluginOpts> options = EnumSet.<PluginOpts>of(NOOP, opts);
> -    boolean requireName = options.contains(REQUIRE_NAME);
> -    boolean requireClass = options.contains(REQUIRE_CLASS);
> +  private void loadPluginInfo(SolrPluginInfo pluginInfo) {
> +    boolean requireName = pluginInfo.options.contains(REQUIRE_NAME);
> +    boolean requireClass = pluginInfo.options.contains(REQUIRE_CLASS);
> 
> -    List<PluginInfo> result = readPluginInfos(tag, requireName, requireClass);
> +    List<PluginInfo> result = readPluginInfos(pluginInfo.tag, requireName, requireClass);
> 
> -    if (1 < result.size() && ! options.contains(MULTI_OK)) {
> +    if (1 < result.size() && ! pluginInfo.options.contains(MULTI_OK)) {
>         throw new SolrException
>           (SolrException.ErrorCode.SERVER_ERROR,
>            "Found " + result.size() + " configuration sections when at most "
> -           + "1 is allowed matching expression: " + tag);
> +           + "1 is allowed matching expression: " + pluginInfo.tag);
>     }
> -    if(!result.isEmpty()) pluginStore.put(clazz.getName(),result);
> +    if(!result.isEmpty()) pluginStore.put(pluginInfo.clazz.getName(),result);
>   }
> 
>   public List<PluginInfo> readPluginInfos(String tag, boolean requireName, boolean requireClass) {
> @@ -423,7 +454,7 @@ public class SolrConfig extends Config {
>     return httpCachingConfig;
>   }
> 
> -  public static class JmxConfiguration {
> +  public static class JmxConfiguration implements MapSerializable{
>     public boolean enabled = false;
>     public String agentId;
>     public String serviceUrl;
> @@ -446,9 +477,18 @@ public class SolrConfig extends Config {
>       }
> 
>     }
> +
> +    @Override
> +    public Map<String, Object> toMap() {
> +      LinkedHashMap map = new LinkedHashMap();
> +      map.put("agentId",agentId);
> +      map.put("serviceUrl",serviceUrl);
> +      map.put("rootName",rootName);
> +      return map;
> +    }
>   }
> 
> -  public static class HttpCachingConfig {
> +  public static class HttpCachingConfig implements MapSerializable{
> 
>     /** config xpath prefix for getting HTTP Caching options */
>     private final static String CACHE_PRE
> @@ -457,7 +497,15 @@ public class SolrConfig extends Config {
>     /** For extracting Expires "ttl" from <cacheControl> config */
>     private final static Pattern MAX_AGE
>       = Pattern.compile("\\bmax-age=(\\d+)");
> -    
> +
> +    @Override
> +    public Map<String, Object> toMap() {
> +      return ZkNodeProps.makeMap("never304",never304,
> +          "etagSeed",etagSeed,
> +          "lastModFrom",lastModFrom.name().toLowerCase(Locale.ROOT),
> +          "cacheControl",cacheControlHeader);
> +    }
> +
>     public static enum LastModFrom {
>       OPENTIME, DIRLASTMOD, BOGUS;
> 
> @@ -517,7 +565,7 @@ public class SolrConfig extends Config {
>     public LastModFrom getLastModFrom() { return lastModFrom; }
>   }
> 
> -  public static class UpdateHandlerInfo{
> +  public static class UpdateHandlerInfo implements MapSerializable{
>     public final String className;
>     public final int autoCommmitMaxDocs,autoCommmitMaxTime,commitIntervalLowerBound,
>         autoSoftCommmitMaxDocs,autoSoftCommmitMaxTime;
> @@ -543,7 +591,29 @@ public class SolrConfig extends Config {
>       this.autoSoftCommmitMaxTime = autoSoftCommmitMaxTime;
> 
>       this.commitWithinSoftCommit = commitWithinSoftCommit;
> -    } 
> +    }
> +
> +
> +
> +    @Override
> +    public Map<String, Object> toMap() {
> +      LinkedHashMap result = new LinkedHashMap();
> +      result.put("class",className);
> +      result.put("autoCommmitMaxDocs",autoCommmitMaxDocs);
> +      result.put("indexWriterCloseWaitsForMerges",indexWriterCloseWaitsForMerges);
> +      result.put("openSearcher",openSearcher);
> +      result.put("commitIntervalLowerBound",commitIntervalLowerBound);
> +      result.put("commitWithinSoftCommit",commitWithinSoftCommit);
> +      result.put("autoCommit", ZkNodeProps.makeMap(
> +          "maxDocs", autoCommmitMaxDocs,
> +          "maxTime",autoCommmitMaxTime,
> +          "commitIntervalLowerBound", commitIntervalLowerBound
> +      ));
> +      result.put("autoSoftCommit" ,
> +          ZkNodeProps.makeMap("maxDocs", autoSoftCommmitMaxDocs,
> +              "maxTime",autoSoftCommmitMaxTime));
> +      return result;
> +    }
>   }
> 
> //  public Map<String, List<PluginInfo>> getUpdateProcessorChainInfo() { return updateProcessorChainInfo; }
> @@ -631,5 +701,100 @@ public class SolrConfig extends Config {
>     return enableRemoteStreams;
>   }
> 
> +  @Override
> +  public int getInt(String path) {
> +    return getInt(path, 0);
> +  }
> +
> +  @Override
> +  public int getInt(String path, int def) {
> +    Object v = overlay.getXPathProperty(path);
> +
> +    Object val = overlay.getXPathProperty(path);
> +    if (val != null) return Integer.parseInt(val.toString());
> +    return super.getInt(path, def);
> +  }
> +  @Override
> +  public boolean getBool(String path, boolean def) {
> +    Object val = overlay.getXPathProperty(path);
> +    if (val != null) return Boolean.parseBoolean(val.toString());
> +    return super.getBool(path, def);
> +  }
> +  @Override
> +  public Map<String, Object> toMap() {
> +    LinkedHashMap result = new LinkedHashMap();
> +    if(getZnodeVersion() > -1) result.put("znodeVersion",getZnodeVersion());
> +    result.put("luceneMatchVersion",luceneMatchVersion);
> +    result.put("updateHandler", getUpdateHandlerInfo().toMap());
> +    Map m = new LinkedHashMap();
> +    result.put("query", m);
> +    m.put("useFilterForSortedQuery", useFilterForSortedQuery);
> +    m.put("queryResultWindowSize", queryResultWindowSize);
> +    m.put("queryResultMaxDocsCached", queryResultMaxDocsCached);
> +    m.put("enableLazyFieldLoading", enableLazyFieldLoading);
> +    m.put("maxBooleanClauses", booleanQueryMaxClauseCount);
> +
> +    for (SolrPluginInfo plugin : plugins) {
> +      List<PluginInfo> infos = getPluginInfos(plugin.clazz.getName());
> +      if(infos == null || infos.isEmpty()) continue;
> +      String tag = plugin.tag;
> +      tag = tag.replace("/","");
> +      if(plugin.options.contains(PluginOpts.REQUIRE_NAME)){
> +        LinkedHashMap items = new LinkedHashMap();
> +        for (PluginInfo info : infos) items.put(info.name, info.toMap());
> +        result.put(tag,items);
> +      } else {
> +        if(plugin.options.contains(MULTI_OK)){
> +          ArrayList<Map> l = new ArrayList<>();
> +          for (PluginInfo info : infos) l.add(info.toMap());
> +          result.put(tag,l);
> +        } else {
> +          result.put(tag, infos.get(0).toMap());
> +        }
> +
> +      }
> +
> +    }
> +
> +
> +    addCacheConfig(m,filterCacheConfig,queryResultCacheConfig,documentCacheConfig,fieldValueCacheConfig);
> +    if(jmxConfig != null) result.put("jmx",jmxConfig.toMap());
> +    m = new LinkedHashMap();
> +    result.put("requestDispatcher", m);
> +    m.put("handleSelect",handleSelect);
> +    if(httpCachingConfig!=null) m.put("httpCaching", httpCachingConfig.toMap());
> +    m.put("requestParsers", ZkNodeProps.makeMap("multipartUploadLimitKB",multipartUploadLimitKB,
> +        "formUploadLimitKB",formUploadLimitKB,
> +        "addHttpRequestToContext",addHttpRequestToContext));
> +    if(indexConfig != null) result.put("indexConfig",indexConfig.toMap());
> +
> +    //TODO there is more to add
> +
> +    return result;
> +  }
> +
> +  private void addCacheConfig(Map queryMap, CacheConfig... cache) {
> +    if(cache==null)return;
> +    for (CacheConfig config : cache) if(config !=null) queryMap.put(config.getNodeName(),config.toMap());
> +
> +  }
> +
> +  @Override
> +  protected Properties getSubstituteProperties() {
> +    Map<String, Object> p = getOverlay().getUserProps();
> +    if(p==null || p.isEmpty()) return super.getSubstituteProperties();
> +    Properties result = new Properties(super.getSubstituteProperties());
> +    result.putAll(p);
> +    return result;
> +  }
> +  private ConfigOverlay overlay;
> +
> +  public ConfigOverlay getOverlay() {
> +    if(overlay ==null) {
> +      overlay = getConfigOverlay(getResourceLoader());
> +    }
> +    return overlay;
> +  }
> +
> 
> }
> 
> Modified: lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/SolrCore.java
> URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/SolrCore.java?rev=1641420&r1=1641419&r2=1641420&view=diff
> ==============================================================================
> --- lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/SolrCore.java (original)
> +++ lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/SolrCore.java Mon Nov 24 16:57:01 2014
> @@ -77,6 +77,7 @@ import org.apache.solr.common.util.Simpl
> import org.apache.solr.core.DirectoryFactory.DirContext;
> import org.apache.solr.handler.RequestHandlerBase;
> import org.apache.solr.handler.SnapPuller;
> +import org.apache.solr.handler.SolrConfigHandler;
> import org.apache.solr.handler.UpdateRequestHandler;
> import org.apache.solr.handler.admin.ShowFileRequestHandler;
> import org.apache.solr.handler.component.DebugComponent;
> @@ -808,6 +809,8 @@ public final class SolrCore implements S
>       reqHandlers = new RequestHandlers(this);
>       List<PluginInfo> implicitReqHandlerInfo = new ArrayList<>();
>       UpdateRequestHandler.addImplicits(implicitReqHandlerInfo);
> +      SolrConfigHandler.addImplicits(implicitReqHandlerInfo);
> +
>       reqHandlers.initHandlersFromConfig(solrConfig, implicitReqHandlerInfo);
> 
>       // Handle things that should eventually go away
> 
> Modified: lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/SolrResourceLoader.java
> URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/SolrResourceLoader.java?rev=1641420&r1=1641419&r2=1641420&view=diff
> ==============================================================================
> --- lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/SolrResourceLoader.java (original)
> +++ lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/SolrResourceLoader.java Mon Nov 24 16:57:01 2014
> @@ -53,8 +53,10 @@ import java.io.Closeable;
> import java.io.File;
> import java.io.FileFilter;
> import java.io.FileInputStream;
> +import java.io.FileOutputStream;
> import java.io.IOException;
> import java.io.InputStream;
> +import java.io.OutputStreamWriter;
> import java.lang.reflect.Constructor;
> import java.net.MalformedURLException;
> import java.net.URI;
> @@ -823,4 +825,36 @@ public class SolrResourceLoader implemen
>   public List<SolrInfoMBean> getInfoMBeans(){
>     return Collections.unmodifiableList(infoMBeans);
>   }
> +
> +
> +  public static void persistConfLocally(SolrResourceLoader loader, String resourceName, byte[] content) {
> +    // Persist locally
> +    File managedSchemaFile = new File(loader.getConfigDir(), resourceName);
> +    OutputStreamWriter writer = null;
> +    try {
> +      File parentDir = managedSchemaFile.getParentFile();
> +      if ( ! parentDir.isDirectory()) {
> +        if ( ! parentDir.mkdirs()) {
> +          final String msg = "Can't create managed schema directory " + parentDir.getAbsolutePath();
> +          log.error(msg);
> +          throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, msg);
> +        }
> +      }
> +      final FileOutputStream out = new FileOutputStream(managedSchemaFile);
> +      out.write(content);
> +      log.info("Upgraded to managed schema at " + managedSchemaFile.getPath());
> +    } catch (IOException e) {
> +      final String msg = "Error persisting managed schema " + managedSchemaFile;
> +      log.error(msg, e);
> +      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, msg, e);
> +    } finally {
> +      org.apache.commons.io.IOUtils.closeQuietly(writer);
> +      try {
> +        FileUtils.sync(managedSchemaFile);
> +      } catch (IOException e) {
> +        final String msg = "Error syncing the managed schema file " + managedSchemaFile;
> +        log.error(msg, e);
> +      }
> +    }
> +  }
> }
> 
> Copied: lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/handler/SolrConfigHandler.java (from r1636862, lucene/dev/trunk/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?p2=lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/handler/SolrConfigHandler.java&p1=lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/SolrConfigHandler.java&r1=1636862&r2=1641420&rev=1641420&view=diff
> ==============================================================================
> --- lucene/dev/trunk/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 Mon Nov 24 16:57:01 2014
> @@ -19,8 +19,8 @@ package org.apache.solr.handler;
> 
> 
> import java.io.IOException;
> -import java.net.URL;
> import java.text.MessageFormat;
> +import java.util.ArrayList;
> import java.util.Arrays;
> import java.util.HashMap;
> import java.util.HashSet;
> @@ -28,13 +28,10 @@ import java.util.List;
> import java.util.Map;
> import java.util.Set;
> 
> -import com.google.common.collect.ImmutableList;
> -import com.google.common.collect.ImmutableSet;
> -import org.apache.lucene.analysis.util.ResourceLoader;
> -import org.apache.lucene.analysis.util.ResourceLoaderAware;
> import org.apache.solr.cloud.ZkController;
> import org.apache.solr.cloud.ZkSolrResourceLoader;
> import org.apache.solr.common.SolrException;
> +import org.apache.solr.common.cloud.SolrZkClient;
> import org.apache.solr.common.cloud.ZkNodeProps;
> import org.apache.solr.common.params.CollectionParams;
> import org.apache.solr.common.params.CommonParams;
> @@ -44,12 +41,11 @@ import org.apache.solr.common.params.Sol
> import org.apache.solr.common.util.ContentStream;
> import org.apache.solr.common.util.NamedList;
> import org.apache.solr.common.util.StrUtils;
> -import org.apache.solr.core.CloseHook;
> import org.apache.solr.core.ConfigOverlay;
> +import org.apache.solr.core.CoreContainer;
> import org.apache.solr.core.PluginInfo;
> import org.apache.solr.core.SolrConfig;
> import org.apache.solr.core.SolrCore;
> -import org.apache.solr.core.SolrInfoMBean;
> import org.apache.solr.core.SolrResourceLoader;
> import org.apache.solr.request.LocalSolrQueryRequest;
> import org.apache.solr.request.SolrQueryRequest;
> @@ -94,30 +90,60 @@ public class SolrConfigHandler extends R
>   public void inform(final SolrCore core) {
>     if( ! (core.getResourceLoader() instanceof  ZkSolrResourceLoader)) return;
>     final ZkSolrResourceLoader zkSolrResourceLoader = (ZkSolrResourceLoader) core.getResourceLoader();
> -    if(zkSolrResourceLoader != null){
> -      Runnable listener = new Runnable() {
> -        @Override
> -        public void run() {
> -          try {
> -            if(core.isClosed()) return;
> -            Stat stat = zkSolrResourceLoader.getZkController().getZkClient().exists((zkSolrResourceLoader).getCollectionZkPath() + "/" + ConfigOverlay.RESOURCE_NAME, null, true);
> -            if(stat == null) return;
> -            if (stat.getVersion() >  core.getSolrConfig().getOverlay().getZnodeVersion()) {
> -              core.getCoreDescriptor().getCoreContainer().reload(core.getName());
> +    if(zkSolrResourceLoader != null)
> +      zkSolrResourceLoader.getZkController().registerConfListenerForCore(
> +          zkSolrResourceLoader.getConfigSetZkPath(),
> +          core,
> +          getListener(core, zkSolrResourceLoader));
> +
> +  }
> +
> +  private static Runnable getListener(SolrCore core, ZkSolrResourceLoader zkSolrResourceLoader) {
> +    final String coreName = core.getName();
> +    final CoreContainer cc = core.getCoreDescriptor().getCoreContainer();
> +    final String overlayPath = (zkSolrResourceLoader).getConfigSetZkPath() + "/" + ConfigOverlay.RESOURCE_NAME;
> +    final String solrConfigPath = (zkSolrResourceLoader).getConfigSetZkPath() + "/" + core.getSolrConfig().getName();
> +    return new Runnable() {
> +          @Override
> +          public void run() {
> +            log.info("config update_listener called");
> +            SolrZkClient zkClient = cc.getZkController().getZkClient();
> +            int solrConfigversion,overlayVersion;
> +            try (SolrCore core = cc.getCore(coreName))  {
> +              if (core.isClosed()) return;
> +               solrConfigversion = core.getSolrConfig().getOverlay().getZnodeVersion();
> +               overlayVersion = core.getSolrConfig().getZnodeVersion();
> +            }
> +
> +            if (checkStale(zkClient, overlayPath, solrConfigversion) ||
> +                checkStale(zkClient, solrConfigPath, overlayVersion)) {
> +              log.info("core reload");
> +              cc.reload(coreName);
>             }
> -          } catch (KeeperException.NoNodeException nne){
> -            //no problem
> -          } catch (KeeperException e) {
> -            log.error("error refreshing solrconfig ", e);
> -          } catch (InterruptedException e) {
> -            Thread.currentThread().isInterrupted();
>           }
> -        }
> -      };
> +        };
> +  }
> 
> -      zkSolrResourceLoader.getZkController().registerConfListenerForCore(zkSolrResourceLoader.getCollectionZkPath(), core,listener);
> +  private static boolean checkStale(SolrZkClient zkClient,  String zkPath, int currentVersion)  {
> +    try {
> +      Stat stat = zkClient.exists(zkPath, null, true);
> +      if(stat == null){
> +        if(currentVersion>0) return true;
> +        return false;
> +      }
> +      if (stat.getVersion() >  currentVersion) {
> +        log.info(zkPath+" is stale will need an update from {} to {}", currentVersion,stat.getVersion());
> +        return true;
> +      }
> +      return false;
> +    } catch (KeeperException.NoNodeException nne){
> +      //no problem
> +    } catch (KeeperException e) {
> +      log.error("error refreshing solrconfig ", e);
> +    } catch (InterruptedException e) {
> +      Thread.currentThread().isInterrupted();
>     }
> -
> +    return false;
>   }
> 
> 
> @@ -136,8 +162,7 @@ public class SolrConfigHandler extends R
>       String path = (String) req.getContext().get("path");
>       if(path == null) path="/config";
>       if("/config/overlay".equals(path)){
> -        resp.add("overlay", req.getCore().getSolrConfig().getOverlay().toOutputFormat());
> -        return;
> +        resp.add("overlay", req.getCore().getSolrConfig().getOverlay().toMap());
>       } else {
>         List<String> parts =StrUtils.splitSmart(path, '/');
>         if(parts.get(0).isEmpty()) parts.remove(0);
> @@ -152,13 +177,32 @@ 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");
> -    }
> +      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;
> +      }
> +
>       try {
> -        for (ContentStream stream : streams) {
> -          runCommandsTillSuccess(stream);
> +        for (;;) {
> +          ArrayList<CommandOperation> opsCopy = new ArrayList<>(ops.size());
> +          ConfigOverlay overlay = SolrConfig.getConfigOverlay(req.getCore().getResourceLoader());
> +          for (CommandOperation op : ops) opsCopy.add(op.getCopy());
> +          try {
> +            handleCommands(opsCopy, overlay);
> +            break;
> +          } catch (ZkController.ResourceModifiedInZkException e) {
> +            //retry
> +            log.info("Race condition, the node is modified in ZK by someone else " +e.getMessage());
> +          }
>         }
>       } catch (Exception e) {
>         resp.setException(e);
> @@ -167,30 +211,21 @@ public class SolrConfigHandler extends R
> 
>     }
> 
> -    private void runCommandsTillSuccess(ContentStream stream) throws IOException {
> -      for (;;) {
> -        try {
> -          handleCommands(stream);
> -          break;
> -        } catch (ZkController.ResourceModifiedInZkException e) {
> -          log.info(e.getMessage());
> -
> -        }
> -      }
> -    }
> -
> -    private void handleCommands( ContentStream stream) throws IOException {
> -    ConfigOverlay overlay = req.getCore().getSolrConfig().getOverlay();
> -    List<CommandOperation> ops = CommandOperation.parse(stream.getReader());
> +    private void handleCommands(List<CommandOperation> ops, ConfigOverlay overlay ) throws IOException {
>     for (CommandOperation op : ops) {
> -      if(SET_PROPERTY.equals( op.name) ){
> -        overlay = applySetProp(op, overlay);
> -      }else if(UNSET_PROPERTY.equals(op.name)){
> -        overlay = applyUnset(op,overlay);
> -      }else if(SET_USER_PROPERTY.equals(op.name)){
> -        overlay = applySetUserProp(op ,overlay);
> -      }else if(UNSET_USER_PROPERTY.equals(op.name)){
> -        overlay = applyUnsetUserProp(op, overlay);
> +      switch (op.name) {
> +        case SET_PROPERTY:
> +          overlay = applySetProp(op, overlay);
> +          break;
> +        case UNSET_PROPERTY:
> +          overlay = applyUnset(op, overlay);
> +          break;
> +        case SET_USER_PROPERTY:
> +          overlay = applySetUserProp(op, overlay);
> +          break;
> +        case UNSET_USER_PROPERTY:
> +          overlay = applyUnsetUserProp(op, overlay);
> +          break;
>       }
>     }
>     List errs = CommandOperation.captureErrors(ops);
> @@ -204,21 +239,6 @@ public class SolrConfigHandler extends R
>       ZkController.persistConfigResourceToZooKeeper(loader,overlay.getZnodeVersion(),
>           ConfigOverlay.RESOURCE_NAME,overlay.toByteArray(),true);
> 
> -      String collectionName = req.getCore().getCoreDescriptor().getCloudDescriptor().getCollectionName();
> -      Map map = ZkNodeProps.makeMap(CoreAdminParams.ACTION, CollectionParams.CollectionAction.RELOAD.toString() ,
> -          CollectionParams.NAME, collectionName);
> -
> -      SolrQueryRequest  solrQueryRequest = new LocalSolrQueryRequest(req.getCore(), new MapSolrParams(map));
> -      SolrQueryResponse tmpResp = new SolrQueryResponse();
> -      try {
> -        //doing a collection reload
> -        req.getCore().getCoreDescriptor().getCoreContainer().getCollectionsHandler().handleRequestBody(solrQueryRequest,tmpResp);
> -      } catch (Exception e) {
> -        String msg = MessageFormat.format("Unable to reload collection {0}", collectionName);
> -        log.error(msg);
> -        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, msg);
> -      }
> -
>     } else {
>       SolrResourceLoader.persistConfLocally(loader, ConfigOverlay.RESOURCE_NAME, overlay.toByteArray());
>       req.getCore().getCoreDescriptor().getCoreContainer().reload(req.getCore().getName());
> 
> Modified: lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminHandler.java
> URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminHandler.java?rev=1641420&r1=1641419&r2=1641420&view=diff
> ==============================================================================
> --- lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminHandler.java (original)
> +++ lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminHandler.java Mon Nov 24 16:57:01 2014
> @@ -587,7 +587,7 @@ public class CoreAdminHandler extends Re
>     catch (Exception ex) {
>       if (coreContainer.isZooKeeperAware() && dcore != null && !preExisitingZkEntry) {
>         try {
> -          coreContainer.getZkController().unregister(dcore.getName(), dcore);
> +          coreContainer.getZkController().unregister(dcore.getName(), dcore,null);
>         } catch (InterruptedException e) {
>           Thread.currentThread().interrupt();
>           SolrException.log(log, null, e);
> 
> Modified: lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/handler/admin/EditFileRequestHandler.java
> URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/handler/admin/EditFileRequestHandler.java?rev=1641420&r1=1641419&r2=1641420&view=diff
> ==============================================================================
> --- lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/handler/admin/EditFileRequestHandler.java (original)
> +++ lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/handler/admin/EditFileRequestHandler.java Mon Nov 24 16:57:01 2014
> @@ -282,7 +282,7 @@ public class EditFileRequestHandler exte
> 
>       if (coreContainer.isZooKeeperAware()) {
>         try {
> -          String confPath = ((ZkSolrResourceLoader) core.getResourceLoader()).getCollectionZkPath();
> +          String confPath = ((ZkSolrResourceLoader) core.getResourceLoader()).getConfigSetZkPath();
> 
>           ZkController.downloadConfigDir(coreContainer.getZkController().getZkClient(), confPath,
>               new File(coll, "conf"));
> 
> Modified: lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/handler/admin/ShowFileRequestHandler.java
> URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/handler/admin/ShowFileRequestHandler.java?rev=1641420&r1=1641419&r2=1641420&view=diff
> ==============================================================================
> --- lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/handler/admin/ShowFileRequestHandler.java (original)
> +++ lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/handler/admin/ShowFileRequestHandler.java Mon Nov 24 16:57:01 2014
> @@ -304,7 +304,7 @@ public class ShowFileRequestHandler exte
> 
>     final ZkSolrResourceLoader loader = (ZkSolrResourceLoader) core
>         .getResourceLoader();
> -    String confPath = loader.getCollectionZkPath();
> +    String confPath = loader.getConfigSetZkPath();
> 
>     String fname = req.getParams().get("file", null);
>     if (fname == null) {
> 
> Modified: lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/rest/SolrConfigRestApi.java
> URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/rest/SolrConfigRestApi.java?rev=1641420&r1=1641419&r2=1641420&view=diff
> ==============================================================================
> --- lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/rest/SolrConfigRestApi.java (original)
> +++ lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/rest/SolrConfigRestApi.java Mon Nov 24 16:57:01 2014
> @@ -58,7 +58,7 @@ public class SolrConfigRestApi extends A
>    */
>   @Override
>   public synchronized Restlet createInboundRoot() {
> -
> +/*
>     log.info("createInboundRoot started for /config");
> 
>     router.attachDefault(RestManager.ManagedEndpoint.class);
> @@ -70,6 +70,7 @@ public class SolrConfigRestApi extends A
> 
>     log.info("createInboundRoot complete for /config");
> 
> -    return router;
> +    return router;*/
> +    return null;
>   }  
> }
> 
> Modified: lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/schema/ManagedIndexSchema.java
> URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/schema/ManagedIndexSchema.java?rev=1641420&r1=1641419&r2=1641420&view=diff
> ==============================================================================
> --- lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/schema/ManagedIndexSchema.java (original)
> +++ lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/schema/ManagedIndexSchema.java Mon Nov 24 16:57:01 2014
> @@ -165,7 +165,7 @@ public final class ManagedIndexSchema ex
>     final ZkSolrResourceLoader zkLoader = (ZkSolrResourceLoader)loader;
>     final ZkController zkController = zkLoader.getZkController();
>     final SolrZkClient zkClient = zkController.getZkClient();
> -    final String managedSchemaPath = zkLoader.getCollectionZkPath() + "/" + managedSchemaResourceName;
> +    final String managedSchemaPath = zkLoader.getConfigSetZkPath() + "/" + managedSchemaResourceName;
>     boolean success = true;
>     boolean schemaChangedInZk = false;
>     try {
> 
> Modified: lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/schema/ManagedIndexSchemaFactory.java
> URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/schema/ManagedIndexSchemaFactory.java?rev=1641420&r1=1641419&r2=1641420&view=diff
> ==============================================================================
> --- lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/schema/ManagedIndexSchemaFactory.java (original)
> +++ lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/schema/ManagedIndexSchemaFactory.java Mon Nov 24 16:57:01 2014
> @@ -119,7 +119,7 @@ public class ManagedIndexSchemaFactory e
>     } else { // ZooKeeper
>       final ZkSolrResourceLoader zkLoader = (ZkSolrResourceLoader)loader;
>       final SolrZkClient zkClient = zkLoader.getZkController().getZkClient();
> -      final String managedSchemaPath = zkLoader.getCollectionZkPath() + "/" + managedSchemaResourceName;
> +      final String managedSchemaPath = zkLoader.getConfigSetZkPath() + "/" + managedSchemaResourceName;
>       Stat stat = new Stat();
>       try {
>         // Attempt to load the managed schema
> @@ -224,7 +224,7 @@ public class ManagedIndexSchemaFactory e
>       SolrResourceLoader loader = config.getResourceLoader();
>       if (loader instanceof ZkSolrResourceLoader) {
>         ZkSolrResourceLoader zkLoader = (ZkSolrResourceLoader)loader;
> -        String nonManagedSchemaPath = zkLoader.getCollectionZkPath() + "/" + resourceName;
> +        String nonManagedSchemaPath = zkLoader.getConfigSetZkPath() + "/" + resourceName;
>         try {
>           exists = zkLoader.getZkController().pathExists(nonManagedSchemaPath);
>         } catch (InterruptedException e) {
> @@ -349,7 +349,7 @@ public class ManagedIndexSchemaFactory e
>     } else {
>       // Rename the non-managed schema znode in ZooKeeper
>       ZkSolrResourceLoader zkLoader = (ZkSolrResourceLoader)loader;
> -      final String nonManagedSchemaPath = zkLoader.getCollectionZkPath() + "/" + resourceName;
> +      final String nonManagedSchemaPath = zkLoader.getConfigSetZkPath() + "/" + resourceName;
>       try {
>         ZkController zkController = zkLoader.getZkController();
>         ZkCmdExecutor zkCmdExecutor = new ZkCmdExecutor(zkController.getClientTimeout());
> 
> Modified: lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/schema/ZkIndexSchemaReader.java
> URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/schema/ZkIndexSchemaReader.java?rev=1641420&r1=1641419&r2=1641420&view=diff
> ==============================================================================
> --- lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/schema/ZkIndexSchemaReader.java (original)
> +++ lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/schema/ZkIndexSchemaReader.java Mon Nov 24 16:57:01 2014
> @@ -43,7 +43,7 @@ public class ZkIndexSchemaReader impleme
>     this.managedIndexSchemaFactory = managedIndexSchemaFactory;
>     ZkSolrResourceLoader zkLoader = (ZkSolrResourceLoader)managedIndexSchemaFactory.getResourceLoader();
>     this.zkClient = zkLoader.getZkController().getZkClient();
> -    managedSchemaPath = zkLoader.getCollectionZkPath() + "/" + managedIndexSchemaFactory.getManagedSchemaResourceName();
> +    managedSchemaPath = zkLoader.getConfigSetZkPath() + "/" + managedIndexSchemaFactory.getManagedSchemaResourceName();
>     createSchemaWatcher();
>     zkLoader.getZkController().addOnReconnectListener(this);
>   }
> 
> Modified: lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/search/CacheConfig.java
> URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/search/CacheConfig.java?rev=1641420&r1=1641419&r2=1641420&view=diff
> ==============================================================================
> --- lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/search/CacheConfig.java (original)
> +++ lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/search/CacheConfig.java Mon Nov 24 16:57:01 2014
> @@ -17,9 +17,14 @@
> 
> package org.apache.solr.search;
> 
> +import org.apache.solr.common.util.StrUtils;
> +import org.apache.solr.core.MapSerializable;
> import org.w3c.dom.Node;
> import org.w3c.dom.NodeList;
> 
> +import java.util.Collections;
> +import java.util.HashMap;
> +import java.util.List;
> import java.util.Map;
> 
> import org.apache.solr.common.SolrException;
> @@ -36,7 +41,7 @@ import javax.xml.xpath.XPathConstants;
>  *
>  *
>  */
> -public class CacheConfig {
> +public class CacheConfig implements MapSerializable{
>   private String nodeName;
> 
>   private Class<? extends SolrCache> clazz;
> @@ -70,7 +75,7 @@ public class CacheConfig {
>     if (nodes==null || nodes.getLength()==0) return null;
>     CacheConfig[] configs = new CacheConfig[nodes.getLength()];
>     for (int i=0; i<nodes.getLength(); i++) {
> -      configs[i] = getConfig(solrConfig, nodes.item(i));
> +      configs[i] = getConfig(solrConfig, nodes.item(i).getNodeName(), DOMUtil.toMap(nodes.item(i).getAttributes()), configPath);
>     }
>     return configs;
>   }
> @@ -78,15 +83,29 @@ public class CacheConfig {
> 
>   public static CacheConfig getConfig(SolrConfig solrConfig, String xpath) {
>     Node node = solrConfig.getNode(xpath, false);
> -    return getConfig(solrConfig, node);
> +    if(node == null) {
> +      Map<String, String> m = solrConfig.getOverlay().getEditableSubProperties(xpath);
> +      if(m==null) return null;
> +      List<String> parts = StrUtils.splitSmart(xpath, '/');
> +      return getConfig(solrConfig,parts.get(parts.size()-1) , Collections.EMPTY_MAP,xpath);
> +    }
> +    return getConfig(solrConfig, node.getNodeName(),DOMUtil.toMap(node.getAttributes()), xpath);
>   }
> 
> 
> -  public static CacheConfig getConfig(SolrConfig solrConfig, Node node) {
> -    if (node==null) return null;
> +  public static CacheConfig getConfig(SolrConfig solrConfig, String nodeName, Map<String,String> attrs, String xpath) {
>     CacheConfig config = new CacheConfig();
> -    config.nodeName = node.getNodeName();
> -    config.args = DOMUtil.toMap(node.getAttributes());
> +    config.nodeName = nodeName;
> +    config.args = attrs;
> +
> +    Map<String, String> map = solrConfig.getOverlay().getEditableSubProperties(xpath);
> +    if(map != null){
> +      HashMap<String, String> mapCopy = new HashMap<>(config.args);
> +      for (Map.Entry<String, String> e : map.entrySet()) {
> +        mapCopy.put(e.getKey(),String.valueOf(e.getValue()));
> +      }
> +      config.args = mapCopy;
> +    }
>     String nameAttr = config.args.get("name");  // OPTIONAL
>     if (nameAttr==null) {
>       config.args.put("name",config.nodeName);
> @@ -94,6 +113,7 @@ public class CacheConfig {
> 
>     SolrResourceLoader loader = solrConfig.getResourceLoader();
>     config.cacheImpl = config.args.get("class");
> +    if(config.cacheImpl == null) config.cacheImpl = "solr.LRUCache";
>     config.regenImpl = config.args.get("regenerator");
>     config.clazz = loader.findClass(config.cacheImpl, SolrCache.class);
>     if (config.regenImpl != null) {
> @@ -116,4 +136,15 @@ public class CacheConfig {
>     }
>   }
> 
> +  @Override
> +  public Map<String, Object> toMap() {
> +    Map result = Collections.unmodifiableMap(args);
> +    return result;
> +  }
> +
> +  public String getNodeName() {
> +    return nodeName;
> +  }
> +
> +
> }
> 
> Modified: lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
> URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java?rev=1641420&r1=1641419&r2=1641420&view=diff
> ==============================================================================
> --- lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java (original)
> +++ lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java Mon Nov 24 16:57:01 2014
> @@ -344,7 +344,7 @@ public class SolrDispatchFilter extends 
> 
>           // Handle /schema/* and /config/* paths via Restlet
>           if( path.equals("/schema") || path.startsWith("/schema/")
> -              || path.equals("/config") || path.startsWith("/config/")) {
> +              /*|| path.equals("/config") || path.startsWith("/config/")*/) {
>             solrReq = parser.parse(core, path, req);
>             SolrRequestInfo.setRequestInfo(new SolrRequestInfo(solrReq, new SolrQueryResponse()));
>             if( path.equals(req.getServletPath()) ) {
> 
> Modified: lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/update/SolrIndexConfig.java
> URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/update/SolrIndexConfig.java?rev=1641420&r1=1641419&r2=1641420&view=diff
> ==============================================================================
> --- lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/update/SolrIndexConfig.java (original)
> +++ lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/update/SolrIndexConfig.java Mon Nov 24 16:57:01 2014
> @@ -24,7 +24,9 @@ import org.apache.lucene.util.PrintStrea
> import org.apache.lucene.util.Version;
> import org.apache.solr.common.SolrException;
> import org.apache.solr.common.SolrException.ErrorCode;
> +import org.apache.solr.common.cloud.ZkNodeProps;
> import org.apache.solr.common.util.NamedList;
> +import org.apache.solr.core.MapSerializable;
> import org.apache.solr.core.SolrConfig;
> import org.apache.solr.core.PluginInfo;
> import org.apache.solr.schema.IndexSchema;
> @@ -36,12 +38,13 @@ import java.io.File;
> import java.io.FileOutputStream;
> import java.io.PrintStream;
> import java.util.List;
> +import java.util.Map;
> 
> /**
>  * This config object encapsulates IndexWriter config params,
>  * defined in the &lt;indexConfig&gt; section of solrconfig.xml
>  */
> -public class SolrIndexConfig {
> +public class SolrIndexConfig implements MapSerializable {
>   public static final Logger log = LoggerFactory.getLogger(SolrIndexConfig.class);
> 
>   final String defaultMergePolicyClassName;
> @@ -173,6 +176,19 @@ public class SolrIndexConfig {
> 
>     checkIntegrityAtMerge = solrConfig.getBool(prefix + "/checkIntegrityAtMerge", def.checkIntegrityAtMerge);
>   }
> +  @Override
> +  public Map<String, Object> toMap() {
> +    Map<String, Object> m = ZkNodeProps.makeMap("maxBufferedDocs", maxBufferedDocs,
> +        "maxMergeDocs", maxMergeDocs,
> +        "maxIndexingThreads", maxIndexingThreads,
> +        "mergeFactor", mergeFactor,
> +        "ramBufferSizeMB", ramBufferSizeMB,
> +        "writeLockTimeout", writeLockTimeout,
> +        "lockType", lockType);
> +    if(mergeSchedulerInfo != null) m.put("mergeScheduler",mergeSchedulerInfo.toMap());
> +    if(mergePolicyInfo != null) m.put("mergeScheduler",mergePolicyInfo.toMap());
> +    return m;
> +  }
> 
>   /*
>    * Assert that assertCondition is true.
> 
> 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=1641420&r1=1641419&r2=1641420&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 Mon Nov 24 16:57:01 2014
> @@ -56,13 +56,13 @@ public  class CommandOperation {
>     if (commandData instanceof Map) {
>       return (Map) commandData;
>     }
> -    addError(MessageFormat.format("The command {0} should have the values as a json object {key:val} format", name));
> +    addError(MessageFormat.format("The command ''{0}'' should have the values as a json object {key:val} format", name));
>     return Collections.EMPTY_MAP;
>   }
> 
>   private Object getRootPrimitive(){
>     if (commandData instanceof Map) {
> -      errors.add(MessageFormat.format("The value has to be a string for command : {1}",name));
> +      errors.add(MessageFormat.format("The value has to be a string for command : ''{0}'' ",name));
>       return null;
>     }
>     return commandData;
> @@ -99,7 +99,12 @@ public  class CommandOperation {
>    * single value collection is returned
>    */
>   public List<String> getStrs(String key, List<String> def){
> -    Object v = getMapVal(key);
> +    Object v = null;
> +    if(ROOT_OBJ.equals(key)) {
> +      v = getRootPrimitive();
> +    } else {
> +      v = getMapVal(key);
> +    }
>     if(v == null){
>       return def;
>     } else {
> @@ -205,5 +210,8 @@ public  class CommandOperation {
>     }
> 
>   }
> +  public CommandOperation getCopy(){
> +    return new CommandOperation(name,commandData);
> +  }
> 
> }
> 
> 


---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org