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/05 13:38:07 UTC

svn commit: r1636862 [1/2] - in /lucene/dev/trunk/solr: ./ 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/apache/solr/rest/ ...

Author: noble
Date: Wed Nov  5 12:38:06 2014
New Revision: 1636862

URL: http://svn.apache.org/r1636862
Log:
SOLR-6533,SOLR-6654

Added:
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/ConfigOverlay.java   (with props)
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/MapSerializable.java   (with props)
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/SolrConfigHandler.java   (with props)
    lucene/dev/trunk/solr/core/src/test/org/apache/solr/core/TestConfigOverlay.java   (with props)
    lucene/dev/trunk/solr/core/src/test/org/apache/solr/core/TestSolrConfigHandler.java   (with props)
    lucene/dev/trunk/solr/core/src/test/org/apache/solr/handler/TestSolrConfigHandlerConcurrent.java   (with props)
Modified:
    lucene/dev/trunk/solr/CHANGES.txt
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/ZkController.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/ZkSolrResourceLoader.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/Config.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/CoreContainer.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/PluginInfo.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/SolrConfig.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/SolrCore.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/SolrResourceLoader.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminHandler.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/rest/SolrConfigRestApi.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/CacheConfig.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/SolrIndexConfig.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/util/CommandOperation.java
    lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/solrconfig-managed-schema.xml
    lucene/dev/trunk/solr/core/src/test/org/apache/solr/rest/TestRestManager.java
    lucene/dev/trunk/solr/core/src/test/org/apache/solr/rest/schema/TestBulkSchemaAPI.java
    lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/common/cloud/SolrZkClient.java
    lucene/dev/trunk/solr/test-framework/src/java/org/apache/solr/util/TestHarness.java

Modified: lucene/dev/trunk/solr/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/CHANGES.txt?rev=1636862&r1=1636861&r2=1636862&view=diff
==============================================================================
--- lucene/dev/trunk/solr/CHANGES.txt (original)
+++ lucene/dev/trunk/solr/CHANGES.txt Wed Nov  5 12:38:06 2014
@@ -205,6 +205,8 @@ New Features
   facet.pivot which refers to a 'tag' local param in one or more stats.field params.
   (hossman, Vitaliy Zhovtyuk)
 
+SOLR-6533: Support editing common solrconfig.xml values (Noble Paul)
+
 Bug Fixes
 ----------------------
 

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/ZkController.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/ZkController.java?rev=1636862&r1=1636861&r2=1636862&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/ZkController.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/ZkController.java Wed Nov  5 12:38:06 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;
@@ -42,11 +41,10 @@ import java.util.Set;
 import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicInteger;
 
 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 +70,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 +83,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,11 +200,10 @@ 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) 
-      throws InterruptedException, TimeoutException, IOException
-  {
+                      String localHostContext, int leaderVoteWait, int leaderConflictResolveWait, boolean genericCoreNodeNames, final CurrentCoreDescriptorProvider registerOnReconnect)
+      throws InterruptedException, TimeoutException, IOException {
 
     if (cc == null) throw new IllegalArgumentException("CoreContainer cannot be null.");
     this.cc = cc;
@@ -214,40 +213,41 @@ 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();
     ZkACLProvider zkACLProvider = null;
     if (zkACLProviderClass != null && zkACLProviderClass.trim().length() > 0) {
-      zkACLProvider  = cc.getResourceLoader().newInstance(zkACLProviderClass, ZkACLProvider.class);
+      zkACLProvider = cc.getResourceLoader().newInstance(zkACLProviderClass, ZkACLProvider.class);
     } else {
       zkACLProvider = new DefaultZkACLProvider();
     }
-    
+
     String zkCredentialProviderClass = cc.getConfig().getZkCredentialProviderClass();
     if (zkCredentialProviderClass != null && zkCredentialProviderClass.trim().length() > 0) {
       strat.setZkCredentialsToAddAutomatically(cc.getResourceLoader().newInstance(zkCredentialProviderClass, ZkCredentialsProvider.class));
     } 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
@@ -314,7 +314,7 @@ public final class ZkController {
                     listener.command();
                   } catch (Exception exc) {
                     // not much we can do here other than warn in the log
-                    log.warn("Error when notifying OnReconnect listener "+listener+" after session re-connected.", exc);
+                    log.warn("Error when notifying OnReconnect listener " + listener + " after session re-connected.", exc);
                   }
                 }
               }
@@ -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()).getCollectionZkPath().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 (confDirectoryWatchers) {
+        log.info("This conf directory is no more watched {0}",configLocation);
+        confDirectoryWatchers.remove(configLocation);
+      }
+    }
   }
   
   public void createCollection(String collection) throws KeeperException,
@@ -2098,4 +2109,159 @@ 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.getCollectionZkPath() + "/" + resourceName;
+    String errMsg = "Failed to persist resource at {0} - version mismatch";
+    try {
+      try {
+        zkClient.setData(resourceLocation , content,znodeVersion, true);
+      } catch (NoNodeException e) {
+        if(createIfNotExists){
+          try {
+            zkClient.create(resourceLocation,content, CreateMode.PERSISTENT,true);
+          } catch (KeeperException.NodeExistsException nee) {
+            log.info(MessageFormat.format(errMsg,resourceLocation));
+            throw new ResourceModifiedInZkException(ErrorCode.CONFLICT, MessageFormat.format(errMsg,resourceLocation) + ", retry.");
+          }
+        }
+      }
+
+    } catch (KeeperException.BadVersionException bve){
+      log.info(MessageFormat.format(errMsg,resourceLocation));
+      throw new ResourceModifiedInZkException(ErrorCode.CONFLICT, MessageFormat.format(errMsg,resourceLocation) + ", retry.");
+    } 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 (confDirectoryWatchers){
+      for (Set<Runnable> runnables : confDirectoryWatchers.values()) {
+        if(runnables != null) runnables.remove(listener);
+      }
+    }
+
+  }
+
+  /**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 (confDirectoryWatchers){
+      if(confDirectoryWatchers.containsKey(confDir)){
+        confDirectoryWatchers.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 Map<String , Set<Runnable>> confDirectoryWatchers =  new HashMap<>();
+  void watchZKConfDir(final String zkDir)  {
+
+      if(!confDirectoryWatchers.containsKey(zkDir)){
+        confDirectoryWatchers.put(zkDir,new HashSet<Runnable>());
+      }else{
+        //it's already watched
+        return;
+      }
+
+      Watcher watcher = new Watcher() {
+        @Override
+        public void process(WatchedEvent event) {
+          try {
+            synchronized (confDirectoryWatchers) {
+              // if this is not among directories to be watched then don't set the watcher anymore
+              if(!confDirectoryWatchers.containsKey(zkDir)) return;
+            }
+
+            if (event.getType() == Event.EventType.NodeChildrenChanged) {
+              synchronized (confDirectoryWatchers) {
+                final Set<Runnable> listeners = confDirectoryWatchers.get(zkDir);
+                if (listeners != null) {
+                  new Thread() {
+                    @Override
+                    public synchronized void run() {
+                    //running in a separate thread so that the zk event thread is not
+                    // unnecessarily held up
+                      for (Runnable listener : listeners) listener.run();
+                    }
+                  }.start();
+                }
+              }
+
+            }
+          } finally {
+            if (Event.EventType.None.equals(event.getType())) {
+              return;
+            } else {
+              setConfWatcher(zkDir,this);
+            }
+          }
+        }
+      };
+
+     setConfWatcher(zkDir,watcher);
+    }
+
+  private void setConfWatcher(String zkDir, Watcher watcher) {
+    try {
+      zkClient.getChildren(zkDir,watcher,true);
+    } catch (KeeperException e) {
+      log.error("failed to set watcher for conf dir {} ", zkDir);
+    } catch (InterruptedException e) {
+      Thread.interrupted();
+      log.error("failed to set watcher for conf dir {} ", zkDir);
+    }
+  }
+
+  public OnReconnect getConfigDirListener() {
+    return new OnReconnect() {
+      @Override
+      public void command() {
+        synchronized (confDirectoryWatchers){
+          for (String s : confDirectoryWatchers.keySet()) {
+            watchZKConfDir(s);
+          }
+
+        }
+      }
+    };
+  }
 }

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/ZkSolrResourceLoader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/ZkSolrResourceLoader.java?rev=1636862&r1=1636861&r2=1636862&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/ZkSolrResourceLoader.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/ZkSolrResourceLoader.java Wed Nov  5 12:38:06 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.
@@ -46,6 +47,7 @@ public class ZkSolrResourceLoader extend
     super(instanceDir);
     this.zkController = zooKeeperController;
     collectionZkPath = ZkController.CONFIGS_ZKNODE + "/" + collection;
+    zkController.watchZKConfDir(collectionZkPath);
   }
 
   /**
@@ -61,6 +63,7 @@ public class ZkSolrResourceLoader extend
     super(instanceDir, parent, coreProperties);
     this.zkController = zooKeeperController;
     collectionZkPath = ZkController.CONFIGS_ZKNODE + "/" + collection;
+    zkController.watchZKConfDir(collectionZkPath);
   }
 
   /**
@@ -78,8 +81,9 @@ public class ZkSolrResourceLoader extend
     String file = collectionZkPath + "/" + 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);
@@ -98,6 +102,24 @@ public class ZkSolrResourceLoader extend
     return is;
   }
 
+  public static class ZkByteArrayInputStream extends ByteArrayInputStream{
+
+    private final Stat stat;
+    public ZkByteArrayInputStream(byte[] buf, Stat stat) {
+      super(buf);
+      this.stat = stat;
+
+    }
+
+    public ZkByteArrayInputStream(byte[] buf, int offset, int length, Stat stat) {
+      super(buf, offset, length);
+      this.stat = stat;
+    }
+    public Stat getStat(){
+      return stat;
+    }
+  }
+
   @Override
   public String getConfigDir() {
     throw new ZooKeeperException(

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/Config.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/Config.java?rev=1636862&r1=1636861&r2=1636862&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/Config.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/Config.java Wed Nov  5 12:38:06 2014
@@ -52,6 +52,7 @@ 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;
@@ -102,7 +103,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 );
@@ -138,8 +139,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 +153,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 +212,7 @@ public class Config {
   }
   
   public void substituteProperties() {
-    DOMUtil.substituteProperties(doc, loader.getCoreProperties());
+    DOMUtil.substituteProperties(doc, getSubstituteProperties());
   }
 
 

Added: lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/ConfigOverlay.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/ConfigOverlay.java?rev=1636862&view=auto
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/ConfigOverlay.java (added)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/ConfigOverlay.java Wed Nov  5 12:38:06 2014
@@ -0,0 +1,257 @@
+package org.apache.solr.core;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+import java.io.StringReader;
+import java.text.MessageFormat;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.common.util.StrUtils;
+import org.noggit.CharArr;
+import org.noggit.JSONParser;
+import org.noggit.JSONWriter;
+import org.noggit.ObjectBuilder;
+
+public class ConfigOverlay {
+  private final int znodeVersion ;
+  private Map<String, Object> data;
+  private Map<String,Object> props;
+  private Map<String,Object> userProps;
+
+  public ConfigOverlay(Map<String,Object> jsonObj, int znodeVersion){
+    if(jsonObj == null) jsonObj= Collections.EMPTY_MAP;
+    this.znodeVersion = znodeVersion;
+    data = Collections.unmodifiableMap(jsonObj);
+    props = (Map<String, Object>) data.get("props");
+    if(props == null) props= Collections.EMPTY_MAP;
+    userProps = (Map<String, Object>) data.get("userProps");
+    if(userProps == null) userProps= Collections.EMPTY_MAP;
+
+  }
+  public Object getXPathProperty(String xpath){
+    return getXPathProperty(xpath,true);
+  }
+
+  public Object getXPathProperty(String xpath, boolean onlyPrimitive) {
+    List<String> hierarchy = checkEditable(xpath, true, false);
+    if(hierarchy == null) return null;
+    return getObjectByPath(props, onlyPrimitive, hierarchy);
+  }
+
+  public static Object getObjectByPath(Map root, boolean onlyPrimitive, List<String> hierarchy) {
+    Map obj = root;
+    for (int i = 0; i < hierarchy.size(); i++) {
+      String s = hierarchy.get(i);
+      if(i < hierarchy.size()-1){
+        obj = (Map) obj.get(s);
+        if(obj == null) return null;
+      } else {
+        Object val = obj.get(s);
+        if (onlyPrimitive && val instanceof Map) {
+          return null;
+        }
+        return val;
+      }
+    }
+
+    return false;
+  }
+
+  public ConfigOverlay setUserProperty(String key, Object val){
+    Map copy = new LinkedHashMap(userProps);
+    copy.put(key,val);
+    Map<String, Object> jsonObj = new LinkedHashMap<>(this.data);
+    jsonObj.put("userProps", copy);
+    return new ConfigOverlay(jsonObj, znodeVersion);
+  }
+  public ConfigOverlay unsetUserProperty(String key){
+    if(!userProps.containsKey(key)) return this;
+    Map copy = new LinkedHashMap(userProps);
+    copy.remove(key);
+    Map<String, Object> jsonObj = new LinkedHashMap<>(this.data);
+    jsonObj.put("userProps", copy);
+    return new ConfigOverlay(jsonObj, znodeVersion);
+  }
+
+  public ConfigOverlay setProperty(String name, Object val) {
+    List<String> hierarchy  = checkEditable(name,false, true);
+    Map deepCopy = getDeepCopy(props);
+    Map obj = deepCopy;
+    for (int i = 0; i < hierarchy.size(); i++) {
+      String s = hierarchy.get(i);
+      if (i < hierarchy.size()-1) {
+        if(obj.get(s) == null || (!(obj.get(s) instanceof Map))) {
+          obj.put(s, new LinkedHashMap<>());
+        }
+        obj = (Map) obj.get(s);
+      } else {
+        obj.put(s,val);
+      }
+    }
+
+    Map<String, Object> jsonObj = new LinkedHashMap<>(this.data);
+    jsonObj.put("props", deepCopy);
+
+    return new ConfigOverlay(jsonObj, znodeVersion);
+  }
+
+
+
+  private Map getDeepCopy(Map map) {
+    return (Map) ZkStateReader.fromJSON(ZkStateReader.toJSON(map));
+  }
+
+  public static final String NOT_EDITABLE = "''{0}'' is not an editable property";
+
+  private List<String> checkEditable(String propName, boolean isXPath, boolean failOnError) {
+    LinkedList<String> hierarchy = new LinkedList<>();
+    if(!isEditableProp(propName, isXPath,hierarchy)) {
+      if(failOnError) throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, MessageFormat.format( NOT_EDITABLE,propName));
+      else return null;
+    }
+    return hierarchy;
+
+  }
+
+  public ConfigOverlay unsetProperty(String name) {
+    List<String> hierarchy  = checkEditable(name,false, true);
+    Map deepCopy = getDeepCopy(props);
+    Map obj = deepCopy;
+    for (int i = 0; i < hierarchy.size(); i++) {
+      String s = hierarchy.get(i);
+      if (i < hierarchy.size()-1) {
+        if(obj.get(s) == null || (!(obj.get(s) instanceof Map))) {
+          return this;
+        }
+        obj = (Map) obj.get(s);
+      } else {
+        obj.remove(s);
+      }
+    }
+
+    Map<String, Object> jsonObj = new LinkedHashMap<>(this.data);
+    jsonObj.put("props", deepCopy);
+
+    return new ConfigOverlay(jsonObj, znodeVersion);
+  }
+
+  public byte[] toByteArray() {
+    return ZkStateReader.toJSON(data);
+  }
+
+
+  public int getZnodeVersion(){
+    return znodeVersion;
+  }
+
+  @Override
+  public String toString() {
+    CharArr out = new CharArr();
+    try {
+      new JSONWriter(out, 2).write(data);
+    } catch (Exception e) {
+      throw new RuntimeException(e);
+    }
+    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";
+
+  private static final Long XML_ATTR = 0L;
+  private static final Long XML_NODE = 1L;
+
+  private static Map editable_prop_map ;
+  public static final String MAPPING = "{ updateHandler : {" +
+      "                 autoCommit : { maxDocs:1, maxTime:1, openSearcher:1 }," +
+      "                 autoSoftCommit : { maxDocs:1, maxTime :1}," +
+      "                 commitWithin : {softCommit:1}," +
+      "                 commitIntervalLowerBound:1," +
+      "                 indexWriter : {closeWaitsForMerges:1}" +
+      "                 }," +
+      " query : {" +
+      "          filterCache : {class:0, size:0, initialSize:0 , autowarmCount:0 , regenerator:0}," +
+      "          queryResultCache :{class:0, size:0, initialSize:0,autowarmCount:0,regenerator:0}," +
+      "          documentCache :{class:0, size:0, initialSize:0 ,autowarmCount:0,regenerator:0}," +
+      "          fieldValueCache :{class:0, size:0, initialSize:0 ,autowarmCount:0,regenerator:0}" +
+      "}}";
+  static{
+    try {
+      editable_prop_map =  (Map)new ObjectBuilder(new JSONParser(new StringReader(
+          MAPPING))).getObject();
+    } catch (IOException e) {
+      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "error parsing mapping ", e);
+    }
+  }
+
+
+  public static boolean isEditableProp(String path, boolean isXpath, List<String> hierarchy) {
+    List<String> parts = StrUtils.splitSmart(path, isXpath? '/':'.');
+    Object obj = editable_prop_map;
+    for (int i = 0; i < parts.size(); i++) {
+      String part = parts.get(i);
+      boolean isAttr = isXpath && part.startsWith("@");
+      if(isAttr){
+        part = part.substring(1);
+      }
+      if(hierarchy != null) hierarchy.add(part);
+      if(obj ==null) return false;
+      if(i == parts.size()-1) {
+        if (obj instanceof Map) {
+          Map map = (Map) obj;
+          if(isXpath && isAttr){
+            return XML_ATTR.equals(map.get(part));
+          } else {
+             return XML_ATTR.equals( map.get(part)) || XML_NODE.equals(map.get(part));
+          }
+        }
+        return false;
+      }
+      obj = ((Map) obj).get(part);
+    }
+    return false;
+  }
+
+
+  public Map<String, String> getEditableSubProperties(String xpath) {
+    Object o = getObjectByPath(props,false,StrUtils.splitSmart(xpath,'/'));
+    if (o instanceof Map) {
+      return  (Map) o;
+    } else {
+      return null;
+    }
+  }
+
+  public Map<String, Object> getUserProps() {
+    return userProps;
+  }
+}

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/CoreContainer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/CoreContainer.java?rev=1636862&r1=1636861&r2=1636862&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/CoreContainer.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/CoreContainer.java Wed Nov  5 12:38:06 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 collectionZkPath =  core.getResourceLoader() instanceof ZkSolrResourceLoader ?  ((ZkSolrResourceLoader)core.getResourceLoader()).getCollectionZkPath() : 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, collectionZkPath);
       } catch (InterruptedException e) {
         Thread.currentThread().interrupt();
         throw new SolrException(ErrorCode.SERVER_ERROR, "Interrupted while unregistering core [" + name + "] from cloud state");

Added: lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/MapSerializable.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/MapSerializable.java?rev=1636862&view=auto
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/MapSerializable.java (added)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/MapSerializable.java Wed Nov  5 12:38:06 2014
@@ -0,0 +1,25 @@
+package org.apache.solr.core;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+import java.util.Map;
+
+public interface MapSerializable {
+  public Map<String, Object> toMap();
+}

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/PluginInfo.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/PluginInfo.java?rev=1636862&r1=1636861&r2=1636862&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/PluginInfo.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/PluginInfo.java Wed Nov  5 12:38:06 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;
@@ -92,6 +92,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/trunk/solr/core/src/java/org/apache/solr/core/SolrConfig.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/SolrConfig.java?rev=1636862&r1=1636861&r2=1636862&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/SolrConfig.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/SolrConfig.java Wed Nov  5 12:38:06 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;
@@ -242,48 +251,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( 
@@ -302,7 +270,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<>();
@@ -317,6 +284,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);
+    }
+  }
+
+  private 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;
@@ -333,20 +365,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) {
@@ -411,7 +442,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;
@@ -434,9 +465,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
@@ -445,7 +485,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;
 
@@ -505,7 +553,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;
@@ -531,7 +579,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; }
@@ -619,5 +689,99 @@ 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();
+    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/trunk/solr/core/src/java/org/apache/solr/core/SolrCore.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/SolrCore.java?rev=1636862&r1=1636861&r2=1636862&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/SolrCore.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/SolrCore.java Wed Nov  5 12:38:06 2014
@@ -40,6 +40,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/trunk/solr/core/src/java/org/apache/solr/core/SolrResourceLoader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/SolrResourceLoader.java?rev=1636862&r1=1636861&r2=1636862&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/SolrResourceLoader.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/SolrResourceLoader.java Wed Nov  5 12:38:06 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;
@@ -819,4 +821,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);
+      }
+    }
+  }
 }

Added: lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/SolrConfigHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/SolrConfigHandler.java?rev=1636862&view=auto
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/SolrConfigHandler.java (added)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/SolrConfigHandler.java Wed Nov  5 12:38:06 2014
@@ -0,0 +1,345 @@
+package org.apache.solr.handler;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+import java.io.IOException;
+import java.net.URL;
+import java.text.MessageFormat;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+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.ZkNodeProps;
+import org.apache.solr.common.params.CollectionParams;
+import org.apache.solr.common.params.CommonParams;
+import org.apache.solr.common.params.CoreAdminParams;
+import org.apache.solr.common.params.MapSolrParams;
+import org.apache.solr.common.params.SolrParams;
+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.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;
+import org.apache.solr.request.SolrRequestHandler;
+import org.apache.solr.response.SolrQueryResponse;
+import org.apache.solr.schema.SchemaManager;
+import org.apache.solr.util.CommandOperation;
+import org.apache.solr.util.plugin.SolrCoreAware;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.data.Stat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static java.text.MessageFormat.format;
+import static java.util.Collections.singletonList;
+import static java.util.Collections.singletonMap;
+import static org.apache.solr.common.cloud.ZkNodeProps.makeMap;
+import static org.apache.solr.core.ConfigOverlay.NOT_EDITABLE;
+import static org.apache.solr.core.PluginInfo.DEFAULTS;
+
+public class SolrConfigHandler extends RequestHandlerBase implements SolrCoreAware{
+  public static final Logger log = LoggerFactory.getLogger(SolrConfigHandler.class);
+  public static final boolean configEditing_disabled = Boolean.getBoolean("disable.configEdit");
+
+  @Override
+  public void handleRequestBody(SolrQueryRequest req, SolrQueryResponse rsp) throws Exception {
+
+    setWt(req, "json");
+    String httpMethod = (String) req.getContext().get("httpMethod");
+    Command command = new Command(req, rsp, httpMethod);
+    if("POST".equals(httpMethod)){
+      if(configEditing_disabled) throw new SolrException(SolrException.ErrorCode.FORBIDDEN," solrconfig editing is not enabled");
+      command.handlePOST();
+    }  else {
+      command.handleGET();
+    }
+  }
+
+
+
+  @Override
+  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());
+            }
+          } 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 class Command{
+    private final SolrQueryRequest req;
+    private final SolrQueryResponse resp;
+    private final String method;
+
+    private Command(SolrQueryRequest req, SolrQueryResponse resp, String httpMethod) {
+      this.req = req;
+      this.resp = resp;
+      this.method = httpMethod;
+    }
+
+    private void handleGET() {
+      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;
+      } else {
+        List<String> parts =StrUtils.splitSmart(path, '/');
+        if(parts.get(0).isEmpty()) parts.remove(0);
+        if(parts.size() == 1) {
+          resp.add("solrConfig", req.getCore().getSolrConfig().toMap());
+        } else{
+          Map<String, Object> m = req.getCore().getSolrConfig().toMap();
+          resp.add("solrConfig", ZkNodeProps.makeMap(parts.get(1),m.get(parts.get(1))));
+        }
+      }
+    }
+
+
+    private void handlePOST() throws IOException {
+    Iterable<ContentStream> streams = req.getContentStreams();
+    if(streams == null ){
+      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "missing content stream");
+    }
+      try {
+        for (ContentStream stream : streams) {
+          runCommandsTillSuccess(stream);
+        }
+      } catch (Exception e) {
+        resp.setException(e);
+        resp.add(CommandOperation.ERR_MSGS, singletonList(SchemaManager.getErrorStr(e)));
+      }
+
+    }
+
+    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());
+    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);
+      }
+    }
+    List errs = CommandOperation.captureErrors(ops);
+    if (!errs.isEmpty()) {
+      resp.add(CommandOperation.ERR_MSGS,errs);
+      return;
+    }
+
+    SolrResourceLoader loader = req.getCore().getResourceLoader();
+    if (loader instanceof ZkSolrResourceLoader) {
+      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());
+    }
+
+  }
+
+    private ConfigOverlay applySetUserProp(CommandOperation op, ConfigOverlay overlay) {
+      Map<String, Object> m = op.getDataMap();
+      if(op.hasError()) return overlay;
+      for (Map.Entry<String, Object> e : m.entrySet()) {
+        String name = e.getKey();
+        Object val = e.getValue();
+        overlay = overlay.setUserProperty(name, val);
+      }
+      return overlay;
+    }
+
+    private ConfigOverlay applyUnsetUserProp(CommandOperation op, ConfigOverlay overlay) {
+      List<String> name = op.getStrs(CommandOperation.ROOT_OBJ);
+      if(op.hasError()) return overlay;
+      for (String o : name) {
+        if(!overlay.getUserProps().containsKey(o)) {
+          op.addError(format("No such property ''{0}''", name));
+        } else {
+          overlay = overlay.unsetUserProperty(o);
+        }
+      }
+      return overlay;
+    }
+
+
+
+    private ConfigOverlay applyUnset(CommandOperation op, ConfigOverlay overlay) {
+      List<String> name = op.getStrs(CommandOperation.ROOT_OBJ);
+      if(op.hasError()) return overlay;
+
+      for (String o : name) {
+        if(!ConfigOverlay.isEditableProp(o, false, null)) {
+          op.addError(format(NOT_EDITABLE, name));
+        } else {
+          overlay = overlay.unsetProperty(o);
+        }
+      }
+      return overlay;
+    }
+
+    private ConfigOverlay applySetProp(CommandOperation op, ConfigOverlay overlay) {
+      Map<String, Object> m = op.getDataMap();
+      if(op.hasError()) return overlay;
+      for (Map.Entry<String, Object> e : m.entrySet()) {
+        String name = e.getKey();
+        Object val = e.getValue();
+        if(!ConfigOverlay.isEditableProp(name, false, null)) {
+          op.addError(format(NOT_EDITABLE, name));
+          continue;
+        }
+        overlay = overlay.setProperty(name, val);
+      }
+      return overlay;
+    }
+
+  }
+
+  static void setWt(SolrQueryRequest req, String wt){
+    SolrParams params = req.getParams();
+    if( params.get(CommonParams.WT) != null ) return;//wt is set by user
+    Map<String,String> map = new HashMap<>(1);
+    map.put(CommonParams.WT, wt);
+    map.put("indent", "true");
+    req.setParams(SolrParams.wrapDefaults(params, new MapSolrParams(map)));
+  }
+
+
+  public static void addImplicits(List<PluginInfo> infoList){
+    Map m = makeMap("name", "/config", "class", SolrConfigHandler.class.getName());
+    infoList.add(new PluginInfo(SolrRequestHandler.TYPE, m, new NamedList<>(singletonMap(DEFAULTS, new NamedList())), null));
+  }
+
+
+
+  @Override
+  public SolrRequestHandler getSubHandler(String path) {
+    if(subPaths.contains(path)) return this;
+    return null;
+  }
+
+
+  private static Set<String> subPaths =  new HashSet<>(Arrays.asList("/overlay",
+      "/query","/jmx","/requestDispatcher"));
+  static {
+    for (SolrConfig.SolrPluginInfo solrPluginInfo : SolrConfig.plugins) subPaths.add("/"+solrPluginInfo.tag.replaceAll("/",""));
+
+  }
+
+  //////////////////////// SolrInfoMBeans methods //////////////////////
+
+
+  @Override
+  public String getDescription() {
+    return "Edit solrconfig.xml";
+  }
+
+
+  @Override
+  public String getVersion() {
+    return getClass().getPackage().getSpecificationVersion();
+  }
+
+  @Override
+  public Category getCategory() {
+    return Category.OTHER;
+  }
+
+
+
+  public static final String SET_PROPERTY = "set-property";
+  public static final String UNSET_PROPERTY = "unset-property";
+  public static final String SET_USER_PROPERTY = "set-user-property";
+  public static final String UNSET_USER_PROPERTY = "unset-user-property";
+
+
+
+}

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminHandler.java?rev=1636862&r1=1636861&r2=1636862&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminHandler.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminHandler.java Wed Nov  5 12:38:06 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/trunk/solr/core/src/java/org/apache/solr/rest/SolrConfigRestApi.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/rest/SolrConfigRestApi.java?rev=1636862&r1=1636861&r2=1636862&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/rest/SolrConfigRestApi.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/rest/SolrConfigRestApi.java Wed Nov  5 12:38:06 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/trunk/solr/core/src/java/org/apache/solr/search/CacheConfig.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/CacheConfig.java?rev=1636862&r1=1636861&r2=1636862&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/CacheConfig.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/CacheConfig.java Wed Nov  5 12:38:06 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/trunk/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java?rev=1636862&r1=1636861&r2=1636862&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java Wed Nov  5 12:38:06 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/trunk/solr/core/src/java/org/apache/solr/update/SolrIndexConfig.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/SolrIndexConfig.java?rev=1636862&r1=1636861&r2=1636862&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/SolrIndexConfig.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/SolrIndexConfig.java Wed Nov  5 12:38:06 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/trunk/solr/core/src/java/org/apache/solr/util/CommandOperation.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/util/CommandOperation.java?rev=1636862&r1=1636861&r2=1636862&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/util/CommandOperation.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/util/CommandOperation.java Wed Nov  5 12:38:06 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 {

Modified: lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/solrconfig-managed-schema.xml
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/solrconfig-managed-schema.xml?rev=1636862&r1=1636861&r2=1636862&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/solrconfig-managed-schema.xml (original)
+++ lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/solrconfig-managed-schema.xml Wed Nov  5 12:38:06 2014
@@ -48,4 +48,11 @@
   <requestHandler name="/update" class="solr.UpdateRequestHandler"/>
   <requestHandler name="/replication" class="solr.ReplicationHandler" startup="lazy" />
 
+  <requestHandler name="/dump" class="DumpRequestHandler" initParams="a">
+    <lst name="defaults">
+      <str name="a">${my.custom.variable.a:A}</str>
+      <str name="b">${my.custom.variable.b:B}</str>
+    </lst>
+  </requestHandler>
+
 </config>

Added: lucene/dev/trunk/solr/core/src/test/org/apache/solr/core/TestConfigOverlay.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/core/TestConfigOverlay.java?rev=1636862&view=auto
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/core/TestConfigOverlay.java (added)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/core/TestConfigOverlay.java Wed Nov  5 12:38:06 2014
@@ -0,0 +1,65 @@
+package org.apache.solr.core;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.util.Collections;
+import java.util.Map;
+
+import org.apache.lucene.util.LuceneTestCase;
+
+import static org.apache.solr.core.ConfigOverlay.isEditableProp;
+
+public class TestConfigOverlay extends LuceneTestCase {
+
+  public void testPaths() {
+    assertTrue(isEditableProp("updateHandler/autoCommit/maxDocs", true, null));
+    assertTrue(isEditableProp("updateHandler/autoCommit/maxTime", true, null));
+    assertTrue(isEditableProp("updateHandler/autoCommit/openSearcher", true, null));
+    assertTrue(isEditableProp("updateHandler/autoCommit/maxDocs", true, null));
+    assertTrue(isEditableProp("updateHandler/autoCommit/maxTime", true, null));
+    assertTrue(isEditableProp("updateHandler/commitWithin/softCommit", true, null));
+    assertTrue(isEditableProp("updateHandler/indexWriter/closeWaitsForMerges", true, null));
+
+    assertTrue(isEditableProp("updateHandler.autoCommit.maxDocs", false, null));
+    assertTrue(isEditableProp("updateHandler.autoCommit.maxTime", false, null));
+    assertTrue(isEditableProp("updateHandler.autoCommit.openSearcher", false, null));
+    assertTrue(isEditableProp("updateHandler.autoCommit.maxDocs", false, null));
+    assertTrue(isEditableProp("updateHandler.autoCommit.maxTime", false, null));
+    assertTrue(isEditableProp("updateHandler.commitWithin.softCommit", false, null));
+    assertTrue(isEditableProp("updateHandler.indexWriter.closeWaitsForMerges", false, null));
+
+    assertTrue(isEditableProp("updateHandler/commitIntervalLowerBound", true, null));
+    assertFalse(isEditableProp("updateHandler/commitIntervalLowerBound1", true, null));
+    assertTrue(isEditableProp("query.filterCache.initialSize", false, null));
+    assertFalse(isEditableProp("query.filterCache", false, null));
+    assertTrue(isEditableProp("query/filterCache/@initialSize", true, null));
+    assertFalse(isEditableProp("query/filterCache/@initialSize1", true, null));
+  }
+
+  public void testSetProperty(){
+    ConfigOverlay overlay = new ConfigOverlay(Collections.EMPTY_MAP,0);
+    overlay = overlay.setProperty("query.filterCache.initialSize",100);
+    assertEquals(100, overlay.getXPathProperty("query/filterCache/@initialSize"));
+    Map<String, String> map = overlay.getEditableSubProperties("query/filterCache");
+    assertNotNull(map);
+    assertEquals(1,map.size());
+    assertEquals(100,map.get("initialSize"));
+  }
+
+
+}