You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by sh...@apache.org on 2013/05/07 13:21:14 UTC

svn commit: r1479862 [28/38] - in /lucene/dev/branches/lucene4258: ./ dev-tools/ dev-tools/idea/.idea/ dev-tools/idea/.idea/libraries/ dev-tools/maven/ dev-tools/maven/solr/ dev-tools/maven/solr/core/src/java/ dev-tools/maven/solr/solrj/src/java/ dev-t...

Modified: lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/core/CoreContainer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/core/CoreContainer.java?rev=1479862&r1=1479861&r2=1479862&view=diff
==============================================================================
--- lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/core/CoreContainer.java (original)
+++ lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/core/CoreContainer.java Tue May  7 11:20:55 2013
@@ -17,15 +17,44 @@
 
 package org.apache.solr.core;
 
+import org.apache.commons.io.IOUtils;
+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.cloud.ZooKeeperException;
+import org.apache.solr.common.util.ExecutorUtil;
+import org.apache.solr.handler.admin.CollectionsHandler;
+import org.apache.solr.handler.admin.CoreAdminHandler;
+import org.apache.solr.handler.component.HttpShardHandlerFactory;
+import org.apache.solr.handler.component.ShardHandlerFactory;
+import org.apache.solr.logging.LogWatcher;
+import org.apache.solr.logging.jul.JulWatcher;
+import org.apache.solr.schema.IndexSchema;
+import org.apache.solr.schema.IndexSchemaFactory;
+import org.apache.solr.util.DefaultSolrThreadFactory;
+import org.apache.solr.util.FileUtils;
+import org.apache.solr.util.PropertiesUtil;
+import org.apache.solr.util.plugin.PluginInfoInitialized;
+import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.w3c.dom.Document;
+import org.w3c.dom.Node;
+import org.xml.sax.InputSource;
+
+import javax.xml.transform.Transformer;
+import javax.xml.transform.TransformerException;
+import javax.xml.transform.TransformerFactory;
+import javax.xml.transform.dom.DOMResult;
+import javax.xml.transform.dom.DOMSource;
+import javax.xml.xpath.XPathExpressionException;
 import java.io.ByteArrayInputStream;
 import java.io.File;
 import java.io.FileInputStream;
-import java.io.IOException;
 import java.io.FileNotFoundException;
 import java.io.InputStream;
 import java.text.SimpleDateFormat;
-import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.Date;
@@ -37,7 +66,6 @@ import java.util.Locale;
 import java.util.Map;
 import java.util.Properties;
 import java.util.Set;
-import java.util.TreeSet;
 import java.util.concurrent.Callable;
 import java.util.concurrent.CompletionService;
 import java.util.concurrent.ConcurrentHashMap;
@@ -47,37 +75,6 @@ import java.util.concurrent.Future;
 import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
-
-import org.apache.commons.io.IOUtils;
-import org.apache.commons.lang.StringUtils;
-import org.apache.solr.cloud.CloudDescriptor;
-import org.apache.solr.cloud.CurrentCoreDescriptorProvider;
-import org.apache.solr.cloud.SolrZkServer;
-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.cloud.ZkStateReader;
-import org.apache.solr.common.cloud.ZooKeeperException;
-import org.apache.solr.common.util.ExecutorUtil;
-
-import org.apache.solr.handler.admin.CollectionsHandler;
-import org.apache.solr.handler.admin.CoreAdminHandler;
-import org.apache.solr.handler.component.HttpShardHandlerFactory;
-import org.apache.solr.handler.component.ShardHandlerFactory;
-import org.apache.solr.logging.ListenerConfig;
-import org.apache.solr.logging.LogWatcher;
-import org.apache.solr.logging.jul.JulWatcher;
-import org.apache.solr.schema.IndexSchema;
-import org.apache.solr.update.SolrCoreState;
-import org.apache.solr.util.DefaultSolrThreadFactory;
-import org.apache.solr.util.FileUtils;
-import org.apache.solr.util.PropertiesUtil;
-import org.apache.zookeeper.KeeperException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.slf4j.impl.StaticLoggerBinder;
 
 
 /**
@@ -88,8 +85,7 @@ public class CoreContainer
 {
   private static final String LEADER_VOTE_WAIT = "180000";  // 3 minutes
   private static final int CORE_LOAD_THREADS = 3;
-  private static final String DEFAULT_HOST_CONTEXT = "solr";
-  private static final String DEFAULT_HOST_PORT = "8983";
+
   private static final int DEFAULT_ZK_CLIENT_TIMEOUT = 15000;
   public static final String DEFAULT_DEFAULT_CORE_NAME = "collection1";
   private static final boolean DEFAULT_SHARE_SCHEMA = false;
@@ -97,22 +93,19 @@ public class CoreContainer
   protected static Logger log = LoggerFactory.getLogger(CoreContainer.class);
 
 
-  private final CoreMaps coreMaps = new CoreMaps(this);
+  private final SolrCores solrCores = new SolrCores(this);
 
   protected final Map<String,Exception> coreInitFailures =
     Collections.synchronizedMap(new LinkedHashMap<String,Exception>());
   
   protected boolean persistent = false;
   protected String adminPath = null;
-  protected String managementPath = null;
-  protected String hostPort;
-  protected String hostContext;
-  protected String host;
+  protected volatile String managementPath = null;
+
   protected CoreAdminHandler coreAdminHandler = null;
   protected CollectionsHandler collectionsHandler = null;
   protected File configFile = null;
   protected String libDir = null;
-  protected ClassLoader libLoader = null;
   protected SolrResourceLoader loader = null;
   protected Properties containerProperties;
   protected Map<String ,IndexSchema> indexSchemaCache;
@@ -122,18 +115,17 @@ public class CoreContainer
   protected String solrHome;
   protected String defaultCoreName = null;
 
-  private ZkController zkController;
-  private SolrZkServer zkServer;
+  protected ZkContainer zkSys = new ZkContainer();
+
   private ShardHandlerFactory shardHandlerFactory;
   protected LogWatcher logging = null;
   private String zkHost;
+  private int transientCacheSize = Integer.MAX_VALUE;
 
-  private String leaderVoteWait = LEADER_VOTE_WAIT;
-  private int distribUpdateConnTimeout = 0;
-  private int distribUpdateSoTimeout = 0;
-  protected int transientCacheSize = Integer.MAX_VALUE; // Use as a flag too, if transientCacheSize set in solr.xml this will be changed
   private int coreLoadThreads;
   private CloserThread backgroundCloser = null;
+  protected volatile ConfigSolr cfg;
+  private Config origCfg;
   
   {
     log.info("New CoreContainer " + System.identityHashCode(this));
@@ -141,7 +133,7 @@ public class CoreContainer
 
   /**
    * Deprecated
-   * @deprecated use the single arg constructure with locateSolrHome()
+   * @deprecated use the single arg constructor with locateSolrHome()
    * @see SolrResourceLoader#locateSolrHome
    */
   @Deprecated
@@ -169,152 +161,36 @@ public class CoreContainer
   public CoreContainer(String solrHome) {
     this.solrHome = solrHome;
   }
-
-  protected void initZooKeeper(String zkHost, int zkClientTimeout) {
-    // if zkHost sys property is not set, we are not using ZooKeeper
-    String zookeeperHost;
-    if(zkHost == null) {
-      zookeeperHost = System.getProperty("zkHost");
-    } else {
-      zookeeperHost = zkHost;
-    }
-
-    String zkRun = System.getProperty("zkRun");
-
-    if (zkRun == null && zookeeperHost == null)
-        return;  // not in zk mode
-
-    // zookeeper in quorum mode currently causes a failure when trying to
-    // register log4j mbeans.  See SOLR-2369
-    // TODO: remove after updating to an slf4j based zookeeper
-    System.setProperty("zookeeper.jmx.log4j.disable", "true");
-
-    if (zkRun != null) {
-      String zkDataHome = System.getProperty("zkServerDataDir", solrHome + "zoo_data");
-      String zkConfHome = System.getProperty("zkServerConfDir", solrHome);
-      zkServer = new SolrZkServer(zkRun, zookeeperHost, zkDataHome, zkConfHome, hostPort);
-      zkServer.parseConfig();
-      zkServer.start();
-      
-      // set client from server config if not already set
-      if (zookeeperHost == null) {
-        zookeeperHost = zkServer.getClientString();
-      }
-    }
-
-    int zkClientConnectTimeout = 15000;
-
-    if (zookeeperHost != null) {
-      // we are ZooKeeper enabled
-      try {
-        // If this is an ensemble, allow for a long connect time for other servers to come up
-        if (zkRun != null && zkServer.getServers().size() > 1) {
-          zkClientConnectTimeout = 24 * 60 * 60 * 1000;  // 1 day for embedded ensemble
-          log.info("Zookeeper client=" + zookeeperHost + "  Waiting for a quorum.");
-        } else {
-          log.info("Zookeeper client=" + zookeeperHost);          
-        }
-        String confDir = System.getProperty("bootstrap_confdir");
-        boolean boostrapConf = Boolean.getBoolean("bootstrap_conf");  
-        
-        if(!ZkController.checkChrootPath(zookeeperHost, (confDir!=null) || boostrapConf)) {
-          throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR,
-              "A chroot was specified in ZkHost but the znode doesn't exist. ");
-        }
-        zkController = new ZkController(this, zookeeperHost, zkClientTimeout,
-            zkClientConnectTimeout, host, hostPort, hostContext,
-            leaderVoteWait, distribUpdateConnTimeout, distribUpdateSoTimeout,
-            new CurrentCoreDescriptorProvider() {
-
-              @Override
-              public List<CoreDescriptor> getCurrentDescriptors() {
-                List<CoreDescriptor> descriptors = new ArrayList<CoreDescriptor>(
-                    getCoreNames().size());
-                for (SolrCore core : getCores()) {
-                  descriptors.add(core.getCoreDescriptor());
-                }
-                return descriptors;
-              }
-            });
-
-
-        if (zkRun != null && zkServer.getServers().size() > 1 && confDir == null && boostrapConf == false) {
-          // we are part of an ensemble and we are not uploading the config - pause to give the config time
-          // to get up
-          Thread.sleep(10000);
-        }
-        
-        if(confDir != null) {
-          File dir = new File(confDir);
-          if(!dir.isDirectory()) {
-            throw new IllegalArgumentException("bootstrap_confdir must be a directory of configuration files");
-          }
-          String confName = System.getProperty(ZkController.COLLECTION_PARAM_PREFIX+ZkController.CONFIGNAME_PROP, "configuration1");
-          zkController.uploadConfigDir(dir, confName);
-        }
-
-
-        
-        if(boostrapConf) {
-          ZkController.bootstrapConf(zkController.getZkClient(), cfg, solrHome);
-        }
-        
-      } catch (InterruptedException e) {
-        // Restore the interrupted status
-        Thread.currentThread().interrupt();
-        log.error("", e);
-        throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR,
-            "", e);
-      } catch (TimeoutException e) {
-        log.error("Could not connect to ZooKeeper", e);
-        throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR,
-            "", e);
-      } catch (IOException e) {
-        log.error("", e);
-        throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR,
-            "", e);
-      } catch (KeeperException e) {
-        log.error("", e);
-        throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR,
-            "", e);
-      }
-    }
-    
-  }
-
+  
   public Properties getContainerProperties() {
     return containerProperties;
   }
 
   // Helper class to initialize the CoreContainer
   public static class Initializer {
-    protected String containerConfigFilename = null;  // normally "solr.xml" becoming solr.properties in 5.0
-    protected String dataDir = null; // override datadir for single core mode
 
     // core container instantiation
     public CoreContainer initialize() throws FileNotFoundException {
       CoreContainer cores = null;
       String solrHome = SolrResourceLoader.locateSolrHome();
       // ContainerConfigFilename could  could be a properties file
-      File fconf = new File(solrHome, containerConfigFilename == null ? "solr.xml"
-          : containerConfigFilename);
+      File fconf = new File(solrHome, "solr.xml");
 
       log.info("looking for solr config file: " + fconf.getAbsolutePath());
       cores = new CoreContainer(solrHome);
 
-      if (! fconf.exists()) {
-        if (StringUtils.isBlank(containerConfigFilename) || containerConfigFilename.endsWith(".xml")) {
-          fconf = new File(solrHome, SolrProperties.SOLR_PROPERTIES_FILE);
-        }
-      }
-      // Either we have a config file or not. If it ends in .properties, assume new-style.
+      // first we find zkhost, then we check for solr.xml in zk
+      // 1. look for zkhost from sys prop 2. look for zkhost in {solr.home}/solr.properties
+      
+      // Either we have a config file or not.
       
       if (fconf.exists()) {
         cores.load(solrHome, fconf);
       } else {
-        log.info("no solr.xml or solr.properties file found - using default old-style solr.xml");
+        // Back compart support
+        log.info("no solr.xml found. using default old-style solr.xml");
         try {
-          cores.load(solrHome, new ByteArrayInputStream(ConfigSolrXmlBackCompat.DEF_SOLR_XML.getBytes("UTF-8")), true, null);
+          cores.load(solrHome, new ByteArrayInputStream(ConfigSolrXmlOld.DEF_SOLR_XML.getBytes("UTF-8")), null);
         } catch (Exception e) {
           throw new SolrException(ErrorCode.SERVER_ERROR,
               "CoreContainer.Initialize failed when trying to load default solr.xml file", e);
@@ -322,8 +198,6 @@ public class CoreContainer
         cores.configFile = fconf;
       }
       
-      containerConfigFilename = cores.getConfigFile().getName();
-      
       return cores;
     }
   }
@@ -342,7 +216,7 @@ public class CoreContainer
     this.configFile = configFile;
     InputStream in = new FileInputStream(configFile);
     try {
-      this.load(dir, in, configFile.getName().endsWith(".xml"),  configFile.getName());
+      this.load(dir, in,  configFile.getName());
     } finally {
       IOUtils.closeQuietly(in);
     }
@@ -356,7 +230,7 @@ public class CoreContainer
    */
 
   // Let's keep this ugly boolean out of public circulation.
-  protected void load(String dir, InputStream is, boolean isXmlFile, String fileName)  {
+  protected void load(String dir, InputStream is, String fileName)  {
     ThreadPoolExecutor coreLoadExecutor = null;
     if (null == dir) {
       // don't rely on SolrResourceLoader(), determine explicitly first
@@ -367,18 +241,22 @@ public class CoreContainer
     this.loader = new SolrResourceLoader(dir);
     solrHome = loader.getInstanceDir();
 
-    ConfigSolr cfg;
-    
-    // keep orig config for persist to consult
-    //TODO 5.0: Remove this confusing junk, the properties file is so fast to read that there's no good reason
-    //          to add this stuff. Furthermore, it would be good to persist comments when saving.....
     try {
-      if (isXmlFile) {
-        cfg = new ConfigSolrXmlBackCompat(loader, null, is, null, false);
-        this.cfg = new ConfigSolrXmlBackCompat(loader, (ConfigSolrXmlBackCompat)cfg);
+      Config config = new Config(loader, null, new InputSource(is), null, false);
+
+      // old style defines cores in solr.xml, new style disovers them by 
+      // directory structure
+      boolean oldStyle = (config.getNode("solr/cores", false) != null);
+      
+      if (oldStyle) {
+        // ConfigSolr handles keep orig values around for non solrcore level items,
+        // but this is still how original core lvl attributes are kept around
+        this.origCfg = new Config(loader, null, copyDoc(config.getDocument()));
+        
+        this.cfg = new ConfigSolrXmlOld(config, this);
       } else {
-        cfg = new SolrProperties(this, loader, is, fileName);
-        this.cfg = new SolrProperties(this, loader, (SolrProperties)cfg);
+        this.cfg = new ConfigSolrXml(config, this);
+
       }
     } catch (Exception e) {
       throw new SolrException(ErrorCode.SERVER_ERROR, "", e);
@@ -387,100 +265,68 @@ public class CoreContainer
     // now.
     cfg.substituteProperties();
 
-    shardHandlerFactory = cfg.initShardHandler();
+    // add the sharedLib to the shared resource loader before initializing cfg based plugins
+    libDir = cfg.get(ConfigSolr.CfgProp.SOLR_SHAREDLIB , null);
+    if (libDir != null) {
+      File f = FileUtils.resolvePath(new File(dir), libDir);
+      log.info("loading shared library: " + f.getAbsolutePath());
+      loader.addToClassLoader(libDir);
+      loader.reloadLuceneSPI();
+    }
 
-    coreMaps.allocateLazyCores(cfg, loader);
+    shardHandlerFactory = initShardHandler(cfg);
 
-    // Initialize Logging
-    if (cfg.getBool(ConfigSolr.ConfLevel.SOLR_LOGGING, "enabled", true)) {
-      String slf4jImpl = null;
-      String fname = cfg.get(ConfigSolr.ConfLevel.SOLR_LOGGING, "class", null);
-      try {
-        slf4jImpl = StaticLoggerBinder.getSingleton()
-            .getLoggerFactoryClassStr();
-        if (fname == null) {
-          if (slf4jImpl.indexOf("Log4j") > 0) {
-            log.warn("Log watching is not yet implemented for log4j");
-          } else if (slf4jImpl.indexOf("JDK") > 0) {
-            fname = "JUL";
-          }
-        }
-      } catch (Throwable ex) {
-        log.warn("Unable to read SLF4J version.  LogWatcher will be disabled: "
-            + ex);
-      }
-      
-      // Now load the framework
-      if (fname != null) {
-        if ("JUL".equalsIgnoreCase(fname)) {
-          logging = new JulWatcher(slf4jImpl);
-//      else if( "Log4j".equals(fname) ) {
-//        logging = new Log4jWatcher(slf4jImpl);
-//      }
-        } else {
-          try {
-            logging = loader.newInstance(fname, LogWatcher.class);
-          } catch (Throwable e) {
-            log.warn("Unable to load LogWatcher", e);
-          }
-        }
-        
-        if (logging != null) {
-          ListenerConfig v = new ListenerConfig();
-          v.size = cfg.getInt(ConfigSolr.ConfLevel.SOLR_LOGGING_WATCHER, "size", 50);
-          v.threshold = cfg.get(ConfigSolr.ConfLevel.SOLR_LOGGING_WATCHER, "threshold", null);
-          if (v.size > 0) {
-            log.info("Registering Log Listener");
-            logging.registerListener(v, this);
-          }
-        }
+    solrCores.allocateLazyCores(cfg, loader);
+
+    logging = JulWatcher.newRegisteredLogWatcher(cfg, loader);
+
+    if (cfg instanceof ConfigSolrXmlOld) { //TODO: Remove for 5.0
+      String dcoreName = cfg.get(ConfigSolr.CfgProp.SOLR_CORES_DEFAULT_CORE_NAME, null);
+      if (dcoreName != null && !dcoreName.isEmpty()) {
+        defaultCoreName = dcoreName;
       }
+      persistent = cfg.getBool(ConfigSolr.CfgProp.SOLR_PERSISTENT, false);
+      adminPath = cfg.get(ConfigSolr.CfgProp.SOLR_ADMINPATH, "/admin/cores");
+    } else {
+      adminPath = "/admin/cores";
     }
+    zkHost = cfg.get(ConfigSolr.CfgProp.SOLR_ZKHOST, null);
+    coreLoadThreads = cfg.getInt(ConfigSolr.CfgProp.SOLR_CORELOADTHREADS, CORE_LOAD_THREADS);
     
-    String dcoreName = cfg.get(ConfigSolr.ConfLevel.SOLR_CORES, "defaultCoreName", null);
-    if (dcoreName != null && !dcoreName.isEmpty()) {
-      defaultCoreName = dcoreName;
-    }
-    persistent = cfg.getBool(ConfigSolr.ConfLevel.SOLR, "persistent", false);
-    libDir = cfg.get(ConfigSolr.ConfLevel.SOLR, "sharedLib", null);
-    zkHost = cfg.get(ConfigSolr.ConfLevel.SOLR, "zkHost", null);
-    coreLoadThreads = cfg.getInt(ConfigSolr.ConfLevel.SOLR, "coreLoadThreads", CORE_LOAD_THREADS);
-    
-    adminPath = cfg.get(ConfigSolr.ConfLevel.SOLR_CORES, "adminPath", null);
-    shareSchema = cfg.getBool(ConfigSolr.ConfLevel.SOLR_CORES, "shareSchema", DEFAULT_SHARE_SCHEMA);
-    zkClientTimeout = cfg.getInt(ConfigSolr.ConfLevel.SOLR_CORES, "zkClientTimeout", DEFAULT_ZK_CLIENT_TIMEOUT);
-    
-    distribUpdateConnTimeout = cfg.getInt(ConfigSolr.ConfLevel.SOLR_CORES, "distribUpdateConnTimeout", 0);
-    distribUpdateSoTimeout = cfg.getInt(ConfigSolr.ConfLevel.SOLR_CORES, "distribUpdateSoTimeout", 0);
-    
-    hostPort = cfg.get(ConfigSolr.ConfLevel.SOLR_CORES, "hostPort", DEFAULT_HOST_PORT);
 
-    hostContext = cfg.get(ConfigSolr.ConfLevel.SOLR_CORES, "hostContext", DEFAULT_HOST_CONTEXT);
-    host = cfg.get(ConfigSolr.ConfLevel.SOLR_CORES, "host", null);
+    shareSchema = cfg.getBool(ConfigSolr.CfgProp.SOLR_SHARESCHEMA, DEFAULT_SHARE_SCHEMA);
+    zkClientTimeout = cfg.getInt(ConfigSolr.CfgProp.SOLR_ZKCLIENTTIMEOUT, DEFAULT_ZK_CLIENT_TIMEOUT);
     
-    leaderVoteWait = cfg.get(ConfigSolr.ConfLevel.SOLR_CORES, "leaderVoteWait", LEADER_VOTE_WAIT);
+    int distribUpdateConnTimeout = cfg.getInt(ConfigSolr.CfgProp.SOLR_DISTRIBUPDATECONNTIMEOUT, 0);
+    int distribUpdateSoTimeout = cfg.getInt(ConfigSolr.CfgProp.SOLR_DISTRIBUPDATESOTIMEOUT, 0);
+
+    // Note: initZooKeeper will apply hardcoded default if cloud mode
+    String hostPort = cfg.get(ConfigSolr.CfgProp.SOLR_HOSTPORT, null);
+    // Note: initZooKeeper will apply hardcoded default if cloud mode
+    String hostContext = cfg.get(ConfigSolr.CfgProp.SOLR_HOSTCONTEXT, null);
+
+    String host = cfg.get(ConfigSolr.CfgProp.SOLR_HOST, null);
     
+    String leaderVoteWait = cfg.get(ConfigSolr.CfgProp.SOLR_LEADERVOTEWAIT, LEADER_VOTE_WAIT);
+
+    adminHandler = cfg.get(ConfigSolr.CfgProp.SOLR_ADMINHANDLER, null);
+    managementPath = cfg.get(ConfigSolr.CfgProp.SOLR_MANAGEMENTPATH, null);
+
+    transientCacheSize = cfg.getInt(ConfigSolr.CfgProp.SOLR_TRANSIENTCACHESIZE, Integer.MAX_VALUE);
+
     if (shareSchema) {
       indexSchemaCache = new ConcurrentHashMap<String,IndexSchema>();
     }
-    adminHandler = cfg.get(ConfigSolr.ConfLevel.SOLR_CORES, "adminHandler", null);
-    managementPath = cfg.get(ConfigSolr.ConfLevel.SOLR_CORES, "managementPath", null);
-    
+
     zkClientTimeout = Integer.parseInt(System.getProperty("zkClientTimeout",
         Integer.toString(zkClientTimeout)));
-    initZooKeeper(zkHost, zkClientTimeout);
+    zkSys.initZooKeeper(this, solrHome, zkHost, zkClientTimeout, hostPort, hostContext, host, leaderVoteWait, distribUpdateConnTimeout, distribUpdateSoTimeout);
     
     if (isZooKeeperAware() && coreLoadThreads <= 1) {
       throw new SolrException(ErrorCode.SERVER_ERROR,
           "SolrCloud requires a value of at least 2 in solr.xml for coreLoadThreads");
     }
     
-    if (libDir != null) {
-      File f = FileUtils.resolvePath(new File(dir), libDir);
-      log.info("loading shared library: " + f.getAbsolutePath());
-      libLoader = SolrResourceLoader.createClassLoader(f, null);
-    }
-    
     if (adminPath != null) {
       if (adminHandler == null) {
         coreAdminHandler = new CoreAdminHandler(this);
@@ -490,7 +336,7 @@ public class CoreContainer
     }
     
     collectionsHandler = new CollectionsHandler(this);
-    containerProperties = cfg.getSolrProperties(cfg, DEFAULT_HOST_CONTEXT);
+    containerProperties = cfg.getSolrProperties("solr");
 
     // setup executor to load cores in parallel
     coreLoadExecutor = new ThreadPoolExecutor(coreLoadThreads, coreLoadThreads, 1,
@@ -527,7 +373,7 @@ public class CoreContainer
             p.setSchemaName(opt);
           }
           
-          if (zkController != null) {
+          if (zkSys.getZkController() != null) {
             opt = cfg.getProperty(oneCoreName, CoreDescriptor.CORE_SHARD, null);
             if (opt != null && opt.length() > 0) {
               p.getCloudDescriptor().setShardId(opt);
@@ -592,7 +438,7 @@ public class CoreContainer
           } else {
             // Store it away for later use. includes non-transient but not
             // loaded at startup cores.
-            coreMaps.putDynamicDescriptor(rawName, p);
+            solrCores.putDynamicDescriptor(rawName, p);
           }
         } catch (Throwable ex) {
           SolrException.log(log, null, ex);
@@ -610,7 +456,7 @@ public class CoreContainer
             SolrCore c = future.get();
             // track original names
             if (c != null) {
-              coreMaps.putCoreToOrigName(c, c.getName());
+              solrCores.putCoreToOrigName(c, c.getName());
             }
           } catch (ExecutionException e) {
             SolrException.log(SolrCore.log, "Error loading core", e);
@@ -623,7 +469,7 @@ public class CoreContainer
       }
 
       // Start the background thread
-      backgroundCloser = new CloserThread(this, coreMaps, cfg);
+      backgroundCloser = new CloserThread(this, solrCores, cfg);
       backgroundCloser.start();
 
     } finally {
@@ -632,11 +478,36 @@ public class CoreContainer
       }
     }
   }
+  
+  private ShardHandlerFactory initShardHandler(ConfigSolr configSolr) {
+    PluginInfo info = null;
+    Node shfn = configSolr.getConfig().getNode("solr/cores/shardHandlerFactory", false);
+
+    if (shfn != null) {
+      info = new PluginInfo(shfn, "shardHandlerFactory", false, true);
+    } else {
+      Map m = new HashMap();
+      m.put("class", HttpShardHandlerFactory.class.getName());
+      info = new PluginInfo("shardHandlerFactory", m, null, Collections.<PluginInfo>emptyList());
+    }
+
+    ShardHandlerFactory fac;
+    try {
+       fac = configSolr.getConfig().getResourceLoader().findClass(info.className, ShardHandlerFactory.class).newInstance();
+    } catch (Exception e) {
+      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
+                              "Error instantiating shardHandlerFactory class " + info.className);
+    }
+    if (fac instanceof PluginInfoInitialized) {
+      ((PluginInfoInitialized) fac).init(info);
+    }
+    return fac;
+  }
 
   // To make this available to TestHarness.
   protected void initShardHandler() {
     if (cfg != null) {
-      cfg.initShardHandler();
+      initShardHandler(cfg);
     } else {
       // Cough! Hack! But tests run this way.
       HttpShardHandlerFactory fac = new HttpShardHandlerFactory();
@@ -645,8 +516,6 @@ public class CoreContainer
   }
 
   private volatile boolean isShutDown = false;
-
-  private volatile ConfigSolr cfg;
   
   public boolean isShutDown() {
     return isShutDown;
@@ -661,7 +530,7 @@ public class CoreContainer
     
     if (isZooKeeperAware()) {
       try {
-        zkController.publishAndWaitForDownStates();
+        zkSys.getZkController().publishAndWaitForDownStates();
       } catch (KeeperException e) {
         log.error("", e);
       } catch (InterruptedException e) {
@@ -672,30 +541,33 @@ public class CoreContainer
     isShutDown = true;
 
     if (isZooKeeperAware()) {
-      coreMaps.publishCoresAsDown(zkController);
+      zkSys.publishCoresAsDown(solrCores.getCores());
       cancelCoreRecoveries();
     }
 
 
     try {
       // First wake up the closer thread, it'll terminate almost immediately since it checks isShutDown.
-      synchronized (coreMaps.getLocker()) {
-        coreMaps.getLocker().notifyAll(); // wake up anyone waiting
+      synchronized (solrCores.getModifyLock()) {
+        solrCores.getModifyLock().notifyAll(); // wake up anyone waiting
       }
       if (backgroundCloser != null) { // Doesn't seem right, but tests get in here without initializing the core.
         try {
           backgroundCloser.join();
         } catch (InterruptedException e) {
-          ; // Don't much care if this gets interrupted
+          Thread.currentThread().interrupt();
+          if (log.isDebugEnabled()) {
+            log.debug("backgroundCloser thread was interrupted before finishing");
+          }
         }
       }
       // Now clear all the cores that are being operated upon.
-      coreMaps.clearMaps(cfg);
+      solrCores.close();
 
       // It's still possible that one of the pending dynamic load operation is waiting, so wake it up if so.
       // Since all the pending operations queues have been drained, there should be nothing to do.
-      synchronized (coreMaps.getLocker()) {
-        coreMaps.getLocker().notifyAll(); // wake up the thread
+      synchronized (solrCores.getModifyLock()) {
+        solrCores.getModifyLock().notifyAll(); // wake up the thread
       }
 
     } finally {
@@ -704,24 +576,21 @@ public class CoreContainer
       }
       
       // we want to close zk stuff last
-      if (zkController != null) {
-        zkController.close();
-      }
-      if (zkServer != null) {
-        zkServer.stop();
-      }
+
+      zkSys.close();
+
     }
   }
 
   public void cancelCoreRecoveries() {
-    ArrayList<SolrCoreState> coreStates = new ArrayList<SolrCoreState>();
-    coreMaps.addCoresToList(coreStates);
+
+    List<SolrCore> cores = solrCores.getCores();
 
     // we must cancel without holding the cores sync
     // make sure we wait for any recoveries to stop
-    for (SolrCoreState coreState : coreStates) {
+    for (SolrCore core : cores) {
       try {
-        coreState.cancelRecovery();
+        core.getSolrCoreState().cancelRecovery();
       } catch (Throwable t) {
         SolrException.log(log, "Error canceling recovery for core", t);
       }
@@ -749,10 +618,10 @@ public class CoreContainer
       throw new RuntimeException( "Invalid core name: "+name );
     }
 
-    if (zkController != null) {
+    if (zkSys.getZkController() != null) {
       // this happens before we can receive requests
       try {
-        zkController.preRegister(core.getCoreDescriptor());
+        zkSys.getZkController().preRegister(core);
       } catch (KeeperException e) {
         log.error("", e);
         throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR,
@@ -772,9 +641,9 @@ public class CoreContainer
       throw new IllegalStateException("This CoreContainer has been shutdown");
     }
     if (isTransientCore) {
-      old = coreMaps.putTransientCore(cfg, name, core, loader);
+      old = solrCores.putTransientCore(cfg, name, core, loader);
     } else {
-      old = coreMaps.putCore(name, core);
+      old = solrCores.putCore(name, core);
     }
       /*
       * set both the name of the descriptor and the name of the
@@ -790,7 +659,7 @@ public class CoreContainer
 
     if( old == null || old == core) {
       log.info( "registering core: "+name );
-      registerInZk(core);
+      zkSys.registerInZk(core);
       return null;
     }
     else {
@@ -798,43 +667,11 @@ public class CoreContainer
       if (!returnPrevNotClosed) {
         old.close();
       }
-      registerInZk(core);
+      zkSys.registerInZk(core);
       return old;
     }
   }
 
-  private void registerInZk(SolrCore core) {
-    if (zkController != null) {
-      try {
-        zkController.register(core.getName(), core.getCoreDescriptor());
-      } catch (InterruptedException e) {
-        // Restore the interrupted status
-        Thread.currentThread().interrupt();
-        SolrException.log(log, "", e);
-        throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR, "",
-            e);
-      } catch (Exception e) {
-        // if register fails, this is really bad - close the zkController to
-        // minimize any damage we can cause
-        try {
-          zkController.publish(core.getCoreDescriptor(), ZkStateReader.DOWN);
-        } catch (KeeperException e1) {
-          log.error("", e);
-          throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR,
-              "", e);
-        } catch (InterruptedException e1) {
-          Thread.currentThread().interrupt();
-          log.error("", e);
-          throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR,
-              "", e);
-        }
-        SolrException.log(log, "", e);
-        throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR, "",
-            e);
-      }
-    }
-  }
-
   /**
    * Registers a SolrCore descriptor in the registry using the core's name.
    * If returnPrev==false, the old core, if different, is closed.
@@ -848,61 +685,25 @@ public class CoreContainer
     return registerCore(false, name, core, returnPrev);
   }
 
-
-  // Helper method to separate out creating a core from ZK as opposed to the "usual" way. See create()
-  private SolrCore createFromZk(String instanceDir, CoreDescriptor dcore)
-  {
-    try {
-      SolrResourceLoader solrLoader = null;
-      SolrConfig config = null;
-      String zkConfigName = null;
-      IndexSchema schema;
-      String collection = dcore.getCloudDescriptor().getCollectionName();
-      zkController.createCollectionZkNode(dcore.getCloudDescriptor());
-
-      zkConfigName = zkController.readConfigName(collection);
-      if (zkConfigName == null) {
-        log.error("Could not find config name for collection:" + collection);
-        throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR,
-            "Could not find config name for collection:" + collection);
-      }
-      solrLoader = new ZkSolrResourceLoader(instanceDir, zkConfigName, libLoader, SolrProperties.getCoreProperties(instanceDir, dcore), zkController);
-      config = getSolrConfigFromZk(zkConfigName, dcore.getConfigName(), solrLoader);
-      schema = getSchemaFromZk(zkConfigName, dcore.getSchemaName(), config);
-      return new SolrCore(dcore.getName(), null, config, schema, dcore);
-
-    } catch (KeeperException e) {
-      log.error("", e);
-      throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR,
-          "", e);
-    } catch (InterruptedException e) {
-      // Restore the interrupted status
-      Thread.currentThread().interrupt();
-      log.error("", e);
-      throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR,
-          "", e);
-    }
-  }
-
   // Helper method to separate out creating a core from local configuration files. See create()
   private SolrCore createFromLocal(String instanceDir, CoreDescriptor dcore) {
     SolrResourceLoader solrLoader = null;
 
     SolrConfig config = null;
-    solrLoader = new SolrResourceLoader(instanceDir, libLoader, SolrProperties.getCoreProperties(instanceDir, dcore));
+    solrLoader = new SolrResourceLoader(instanceDir, loader.getClassLoader(), ConfigSolrXml.getCoreProperties(instanceDir, dcore));
     try {
       config = new SolrConfig(solrLoader, dcore.getConfigName(), null);
     } catch (Exception e) {
-      log.error("Failed to load file {}/{}", instanceDir, dcore.getConfigName());
+      log.error("Failed to load file {}", new File(instanceDir, dcore.getConfigName()).getAbsolutePath());
       throw new SolrException(ErrorCode.SERVER_ERROR, "Could not load config for " + dcore.getConfigName(), e);
     }
 
     IndexSchema schema = null;
     if (indexSchemaCache != null) {
-      File schemaFile = new File(dcore.getSchemaName());
+      final String resourceNameToBeUsed = IndexSchemaFactory.getResourceNameToBeUsed(dcore.getSchemaName(), config);
+      File schemaFile = new File(resourceNameToBeUsed);
       if (!schemaFile.isAbsolute()) {
-        schemaFile = new File(solrLoader.getInstanceDir() + "conf"
-            + File.separator + dcore.getSchemaName());
+        schemaFile = new File(solrLoader.getConfigDir(), schemaFile.getPath());
       }
       if (schemaFile.exists()) {
         String key = schemaFile.getAbsolutePath()
@@ -912,7 +713,7 @@ public class CoreContainer
         schema = indexSchemaCache.get(key);
         if (schema == null) {
           log.info("creating new schema object for core: " + dcore.getProperty(CoreDescriptor.CORE_NAME));
-          schema = new IndexSchema(config, dcore.getSchemaName(), null);
+          schema = IndexSchemaFactory.buildIndexSchema(dcore.getSchemaName(), config);
           indexSchemaCache.put(key, schema);
         } else {
           log.info("re-using schema object for core: " + dcore.getProperty(CoreDescriptor.CORE_NAME));
@@ -921,7 +722,7 @@ public class CoreContainer
     }
 
     if (schema == null) {
-      schema = new IndexSchema(config, dcore.getSchemaName(), null);
+      schema = IndexSchemaFactory.buildIndexSchema(dcore.getSchemaName(), config);
     }
 
     SolrCore core = new SolrCore(dcore.getName(), null, config, schema, dcore);
@@ -941,6 +742,10 @@ public class CoreContainer
    */
   public SolrCore create(CoreDescriptor dcore) {
 
+    if (isShutDown) {
+      throw new SolrException(ErrorCode.SERVICE_UNAVAILABLE, "Solr has shutdown.");
+    }
+    
     final String name = dcore.getName();
 
     try {
@@ -951,12 +756,16 @@ public class CoreContainer
                dcore.getName(), instanceDir);
 
       // Initialize the solr config
-      if (zkController != null) {
-        return createFromZk(instanceDir, dcore);
+      SolrCore created = null;
+      if (zkSys.getZkController() != null) {
+        created = zkSys.createFromZk(instanceDir, dcore, loader);
       } else {
-        return createFromLocal(instanceDir, dcore);
+        created = createFromLocal(instanceDir, dcore);
       }
 
+      solrCores.addCreated(created); // For persisting newly-created cores.
+      return created;
+
       // :TODO: Java7...
       // http://docs.oracle.com/javase/7/docs/technotes/guides/language/catch-multiple.html
     } catch (Exception ex) {
@@ -968,21 +777,21 @@ public class CoreContainer
    * @return a Collection of registered SolrCores
    */
   public Collection<SolrCore> getCores() {
-    return coreMaps.getCores();
+    return solrCores.getCores();
   }
 
   /**
    * @return a Collection of the names that cores are mapped to
    */
   public Collection<String> getCoreNames() {
-    return coreMaps.getCoreNames();
+    return solrCores.getCoreNames();
   }
 
   /** This method is currently experimental.
    * @return a Collection of the names that a specific core is mapped to.
    */
   public Collection<String> getCoreNames(SolrCore core) {
-    return coreMaps.getCoreNames(core);
+    return solrCores.getCoreNames(core);
   }
 
   /**
@@ -990,7 +799,7 @@ public class CoreContainer
    * @return a list of al lthe available core names in either permanent or transient core lists.
    */
   public Collection<String> getAllCoreNames() {
-    return coreMaps.getAllCoreNames();
+    return solrCores.getAllCoreNames();
 
   }
 
@@ -1032,12 +841,12 @@ public class CoreContainer
     try {
       name = checkDefault(name);
 
-      SolrCore core = coreMaps.getCore(name);
+      SolrCore core = solrCores.getCore(name);
       if (core == null)
         throw new SolrException( SolrException.ErrorCode.BAD_REQUEST, "No such core: " + name );
 
       try {
-        coreMaps.waitAddPendingCoreOps(name);
+        solrCores.waitAddPendingCoreOps(name);
         CoreDescriptor cd = core.getCoreDescriptor();
 
         File instanceDir = new File(cd.getInstanceDir());
@@ -1045,21 +854,21 @@ public class CoreContainer
         log.info("Reloading SolrCore '{}' using instanceDir: {}",
                  cd.getName(), instanceDir.getAbsolutePath());
         SolrResourceLoader solrLoader;
-        if(zkController == null) {
-          solrLoader = new SolrResourceLoader(instanceDir.getAbsolutePath(), libLoader, SolrProperties.getCoreProperties(instanceDir.getAbsolutePath(), cd));
+        if(zkSys.getZkController() == null) {
+          solrLoader = new SolrResourceLoader(instanceDir.getAbsolutePath(), loader.getClassLoader(), ConfigSolrXml.getCoreProperties(instanceDir.getAbsolutePath(), cd));
         } else {
           try {
             String collection = cd.getCloudDescriptor().getCollectionName();
-            zkController.createCollectionZkNode(cd.getCloudDescriptor());
+            zkSys.getZkController().createCollectionZkNode(cd.getCloudDescriptor());
 
-            String zkConfigName = zkController.readConfigName(collection);
+            String zkConfigName = zkSys.getZkController().readConfigName(collection);
             if (zkConfigName == null) {
               log.error("Could not find config name for collection:" + collection);
               throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR,
                                            "Could not find config name for collection:" + collection);
             }
-            solrLoader = new ZkSolrResourceLoader(instanceDir.getAbsolutePath(), zkConfigName, libLoader,
-                SolrProperties.getCoreProperties(instanceDir.getAbsolutePath(), cd), zkController);
+            solrLoader = new ZkSolrResourceLoader(instanceDir.getAbsolutePath(), zkConfigName, loader.getClassLoader(),
+                ConfigSolrXml.getCoreProperties(instanceDir.getAbsolutePath(), cd), zkSys.getZkController());
           } catch (KeeperException e) {
             log.error("", e);
             throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR,
@@ -1074,10 +883,10 @@ public class CoreContainer
         }
         SolrCore newCore = core.reload(solrLoader, core);
         // keep core to orig name link
-        coreMaps.removeCoreToOrigName(newCore, core);
+        solrCores.removeCoreToOrigName(newCore, core);
         registerCore(false, name, newCore, false);
       } finally {
-        coreMaps.removeFromPendingOps(name);
+        solrCores.removeFromPendingOps(name);
       }
       // :TODO: Java7...
       // http://docs.oracle.com/javase/7/docs/technotes/guides/language/catch-multiple.html
@@ -1086,6 +895,7 @@ public class CoreContainer
     }
   }
 
+  //5.0 remove all checkDefaults?
   private String checkDefault(String name) {
     return (null == name || name.isEmpty()) ? defaultCoreName : name;
   } 
@@ -1099,7 +909,7 @@ public class CoreContainer
     }
     n0 = checkDefault(n0);
     n1 = checkDefault(n1);
-    coreMaps.swap(n0, n1);
+    solrCores.swap(n0, n1);
 
     log.info("swapped: "+n0 + " with " + n1);
   }
@@ -1108,7 +918,7 @@ public class CoreContainer
   public SolrCore remove( String name ) {
     name = checkDefault(name);    
 
-    return coreMaps.remove(name, true);
+    return solrCores.remove(name, true);
 
   }
 
@@ -1118,7 +928,7 @@ public class CoreContainer
       if (core != null) {
         registerCore(false, toName, core, false);
         name = checkDefault(name);
-        coreMaps.remove(name, false);
+        solrCores.remove(name, false);
       }
     } finally {
       if (core != null) {
@@ -1126,16 +936,20 @@ public class CoreContainer
       }
     }
   }
-  /** Gets a core by name and increase its refcount.
+  /** 
+   * Gets a core by name and increase its refcount.
+   *
    * @see SolrCore#close() 
    * @param name the core name
-   * @return the core if found
+   * @return the core if found, null if a SolrCore by this name does not exist
+   * @exception SolrException if a SolrCore with this name failed to be initialized
    */
   public SolrCore getCore(String name) {
 
     name = checkDefault(name);
+
     // Do this in two phases since we don't want to lock access to the cores over a load.
-    SolrCore core = coreMaps.getCoreFromAnyList(name);
+    SolrCore core = solrCores.getCoreFromAnyList(name);
 
     if (core != null) {
       core.open();
@@ -1143,13 +957,23 @@ public class CoreContainer
     }
 
     // OK, it's not presently in any list, is it in the list of dynamic cores but not loaded yet? If so, load it.
-    CoreDescriptor desc = coreMaps.getDynamicDescriptor(name);
+    CoreDescriptor desc = solrCores.getDynamicDescriptor(name);
     if (desc == null) { //Nope, no transient core with this name
+      
+      // if there was an error initalizing this core, throw a 500
+      // error with the details for clients attempting to access it.
+      Exception e = getCoreInitFailures().get(name);
+      if (null != e) {
+        throw new SolrException(ErrorCode.SERVER_ERROR, "SolrCore '" + name +
+                                "' is not available due to init failure: " +
+                                e.getMessage(), e);
+      }
+      // otherwise the user is simply asking for something that doesn't exist.
       return null;
     }
 
     // This will put an entry in pending core ops if the core isn't loaded
-    core = coreMaps.waitAddPendingCoreOps(name);
+    core = solrCores.waitAddPendingCoreOps(name);
 
     if (isShutDown) return null; // We're quitting, so stop. This needs to be after the wait above since we may come off
                                  // the wait as a consequence of shutting down.
@@ -1164,7 +988,7 @@ public class CoreContainer
     } catch(Exception ex){
       throw recordAndThrow(name, "Unable to create core: " + name, ex);
     } finally {
-      coreMaps.removeFromPendingOps(name);
+      solrCores.removeFromPendingOps(name);
     }
 
     return core;
@@ -1176,8 +1000,6 @@ public class CoreContainer
    * @return a CoreAdminHandler
    */
   protected CoreAdminHandler createMultiCoreHandler(final String adminHandlerClass) {
-    // :TODO: why create a new SolrResourceLoader? why not use this.loader ???
-    SolrResourceLoader loader = new SolrResourceLoader(solrHome, libLoader, null);
     return loader.newAdminHandlerInstance(CoreContainer.this, adminHandlerClass);
   }
 
@@ -1198,10 +1020,12 @@ public class CoreContainer
   
   // all of the following properties aren't synchronized
   // but this should be OK since they normally won't be changed rapidly
+  @Deprecated
   public boolean isPersistent() {
     return persistent;
   }
   
+  @Deprecated
   public void setPersistent(boolean persistent) {
     this.persistent = persistent;
   }
@@ -1209,11 +1033,6 @@ public class CoreContainer
   public String getAdminPath() {
     return adminPath;
   }
-  
-  public void setAdminPath(String adminPath) {
-      this.adminPath = adminPath;
-  }
-  
 
   public String getManagementPath() {
     return managementPath;
@@ -1246,12 +1065,13 @@ public class CoreContainer
    *
    */
   public boolean isLoaded(String name) {
-    return coreMaps.isLoaded(name);
+    return solrCores.isLoaded(name);
   }
 
   /** Persists the cores config file in cores.xml. */
+  @Deprecated
   public void persist() {
-    persistFile(null);
+    persistFile(configFile);
   }
 
   /**
@@ -1262,11 +1082,16 @@ public class CoreContainer
    * @return a coreDescriptor. May return null
    */
   public CoreDescriptor getUnloadedCoreDescriptor(String cname) {
-    return coreMaps.getUnloadedCoreDescriptor(cname);
+    return solrCores.getUnloadedCoreDescriptor(cname);
   }
 
   /** Persists the cores config file in a user provided file. */
+  @Deprecated
   public void persistFile(File file) {
+    assert file != null;
+    // only the old solrxml persists
+    if (cfg != null && !(cfg instanceof ConfigSolrXmlOld)) return;
+
     log.info("Persisting cores config to " + (file == null ? configFile : file));
 
     
@@ -1277,30 +1102,38 @@ public class CoreContainer
     
     // <solr attrib="value"> <cores attrib="value">
     Map<String,String> coresAttribs = new HashMap<String,String>();
-    addCoresAttrib(coresAttribs, "adminPath", this.adminPath, null);
-    addCoresAttrib(coresAttribs, "adminHandler", this.adminHandler, null);
-    addCoresAttrib(coresAttribs, "shareSchema",
+    addCoresAttrib(coresAttribs, ConfigSolr.CfgProp.SOLR_ADMINPATH, "adminPath", this.adminPath, null);
+    addCoresAttrib(coresAttribs, ConfigSolr.CfgProp.SOLR_ADMINHANDLER, "adminHandler", this.adminHandler, null);
+    addCoresAttrib(coresAttribs, ConfigSolr.CfgProp.SOLR_SHARESCHEMA,"shareSchema",
         Boolean.toString(this.shareSchema),
         Boolean.toString(DEFAULT_SHARE_SCHEMA));
-    addCoresAttrib(coresAttribs, "host", this.host, null);
+    addCoresAttrib(coresAttribs, ConfigSolr.CfgProp.SOLR_HOST, "host", zkSys.getHost(), null);
 
     if (! (null == defaultCoreName || defaultCoreName.equals("")) ) {
       coresAttribs.put("defaultCoreName", defaultCoreName);
     }
 
-    if (transientCacheSize != Integer.MAX_VALUE) {
-      coresAttribs.put("transientCacheSize", Integer.toString(transientCacheSize));
-    }
-    
-    addCoresAttrib(coresAttribs, "hostPort", this.hostPort, DEFAULT_HOST_PORT);
-    addCoresAttrib(coresAttribs, "zkClientTimeout",
+    addCoresAttrib(coresAttribs, ConfigSolr.CfgProp.SOLR_HOSTPORT, "hostPort",zkSys.getHostPort(), ZkContainer.DEFAULT_HOST_PORT);
+    addCoresAttrib(coresAttribs, ConfigSolr.CfgProp.SOLR_ZKCLIENTTIMEOUT, "zkClientTimeout",
         intToString(this.zkClientTimeout),
         Integer.toString(DEFAULT_ZK_CLIENT_TIMEOUT));
-    addCoresAttrib(coresAttribs, "hostContext", this.hostContext, DEFAULT_HOST_CONTEXT);
-    addCoresAttrib(coresAttribs, "leaderVoteWait", this.leaderVoteWait, LEADER_VOTE_WAIT);
-    addCoresAttrib(coresAttribs, "coreLoadThreads", Integer.toString(this.coreLoadThreads), Integer.toString(CORE_LOAD_THREADS));
+    addCoresAttrib(coresAttribs, ConfigSolr.CfgProp.SOLR_HOSTCONTEXT, "hostContext",
+        zkSys.getHostContext(), ZkContainer.DEFAULT_HOST_CONTEXT);
+    addCoresAttrib(coresAttribs, ConfigSolr.CfgProp.SOLR_LEADERVOTEWAIT, "leaderVoteWait",
+        zkSys.getLeaderVoteWait(), LEADER_VOTE_WAIT);
+    addCoresAttrib(coresAttribs, ConfigSolr.CfgProp.SOLR_CORELOADTHREADS, "coreLoadThreads",
+        Integer.toString(this.coreLoadThreads), Integer.toString(CORE_LOAD_THREADS));
+    if (transientCacheSize != Integer.MAX_VALUE) { // This test
+    // is a consequence of testing. I really hate it.
+      addCoresAttrib(coresAttribs, ConfigSolr.CfgProp.SOLR_TRANSIENTCACHESIZE, "transientCacheSize",
+          Integer.toString(this.transientCacheSize), Integer.toString(Integer.MAX_VALUE));
+    }
 
-    coreMaps.persistCores(cfg, containerProperties, rootSolrAttribs, coresAttribs, file, configFile, loader);
+    try {
+      solrCores.persistCores(origCfg, containerProperties, rootSolrAttribs, coresAttribs, file, configFile, loader);
+    } catch (XPathExpressionException e) {
+      throw new SolrException(ErrorCode.SERVER_ERROR, null, e);
+    }
 
   }
   private String intToString(Integer integer) {
@@ -1308,18 +1141,20 @@ public class CoreContainer
     return Integer.toString(integer);
   }
 
-  private void addCoresAttrib(Map<String,String> coresAttribs, String attribName, String attribValue, String defaultValue) {
+  private void addCoresAttrib(Map<String,String> coresAttribs, ConfigSolr.CfgProp prop,
+                              String attribName, String attribValue, String defaultValue) {
     if (cfg == null) {
       coresAttribs.put(attribName, attribValue);
       return;
     }
     
     if (attribValue != null) {
-      String rawValue = cfg.get(ConfigSolr.ConfLevel.SOLR_CORES, attribName, null);
-      if (rawValue == null && defaultValue != null && attribValue.equals(defaultValue)) return;
+      String origValue = cfg.getOrigProp(prop, null);
+      
+      if (origValue == null && defaultValue != null && attribValue.equals(defaultValue)) return;
 
-      if (attribValue.equals(PropertiesUtil.substituteProperty(rawValue, loader.getCoreProperties()))) {
-        coresAttribs.put(attribName, rawValue);
+      if (attribValue.equals(PropertiesUtil.substituteProperty(origValue, loader.getCoreProperties()))) {
+        coresAttribs.put(attribName, origValue);
       } else {
         coresAttribs.put(attribName, attribValue);
       }
@@ -1331,11 +1166,11 @@ public class CoreContainer
   }
   
   public boolean isZooKeeperAware() {
-    return zkController != null;
+    return zkSys.getZkController() != null;
   }
   
   public ZkController getZkController() {
-    return zkController;
+    return zkSys.getZkController();
   }
   
   public boolean isShareSchema() {
@@ -1347,11 +1182,6 @@ public class CoreContainer
     return shardHandlerFactory;
   }
   
-  private SolrConfig getSolrConfigFromZk(String zkConfigName, String solrConfigFileName,
-      SolrResourceLoader resourceLoader)
-  {
-    return cfg.getSolrConfigFromZk(zkController, zkConfigName, solrConfigFileName, resourceLoader);
-  }
   // Just to tidy up the code where it did this in-line.
   private SolrException recordAndThrow(String name, String msg, Exception ex) {
     synchronized (coreInitFailures) {
@@ -1361,550 +1191,30 @@ public class CoreContainer
     log.error(msg, ex);
     return new SolrException(ErrorCode.SERVER_ERROR, msg, ex);
   }
-
-  private IndexSchema getSchemaFromZk(String zkConfigName, String schemaName,
-      SolrConfig config)
-      throws KeeperException, InterruptedException {
-    return cfg.getSchemaFromZk(zkController, zkConfigName, schemaName, config);
-  }
-}
-
-
-// Introducing the two new maps (transientCores and dynamicDescriptors) introduced some locking complexities. Rather
-// than try to keep them all straight in the code, use this class you need to access any of:
-// cores
-// transientCores
-// dynamicDescriptors
-//
-
-
-class CoreMaps {
-
-  private static Object locker = new Object(); // for locking around manipulating any of the core maps.
-  private final Map<String, SolrCore> cores = new LinkedHashMap<String, SolrCore>(); // For "permanent" cores
-
-  //WARNING! The _only_ place you put anything into the list of transient cores is with the putTransientCore method!
-  private Map<String, SolrCore> transientCores = new LinkedHashMap<String, SolrCore>(); // For "lazily loaded" cores
-
-  private final Map<String, CoreDescriptor> dynamicDescriptors = new LinkedHashMap<String, CoreDescriptor>();
-
-  private int transientCacheSize = Integer.MAX_VALUE;
-
-  private Map<SolrCore, String> coreToOrigName = new ConcurrentHashMap<SolrCore, String>();
-
-  private final CoreContainer container;
-
-  // This map will hold objects that are being currently operated on. The core (value) may be null in the case of
-  // initial load. The rule is, never to any operation on a core that is currently being operated upon.
-  private static final Set<String> pendingCoreOps = new HashSet<String>();
-
-  // Due to the fact that closes happen potentially whenever anything is _added_ to the transient core list, we need
-  // to essentially queue them up to be handled via pendingCoreOps.
-  private static final List<SolrCore> pendingCloses = new ArrayList<SolrCore>();
-
-  CoreMaps(CoreContainer container) {
-    this.container = container;
-  }
-
-  // Trivial helper method for load, note it implements LRU on transient cores. Also note, if
-  // there is no setting for max size, nothing is done and all cores go in the regular "cores" list
-  protected void allocateLazyCores(final ConfigSolr cfg, final SolrResourceLoader loader) {
-    transientCacheSize = cfg.getInt(ConfigSolr.ConfLevel.SOLR_CORES, "transientCacheSize", Integer.MAX_VALUE);
-    if (transientCacheSize != Integer.MAX_VALUE) {
-      CoreContainer.log.info("Allocating transient cache for {} transient cores", transientCacheSize);
-      transientCores = new LinkedHashMap<String, SolrCore>(transientCacheSize, 0.75f, true) {
-        @Override
-        protected boolean removeEldestEntry(Map.Entry<String, SolrCore> eldest) {
-          if (size() > transientCacheSize) {
-            synchronized (locker) {
-              pendingCloses.add(eldest.getValue()); // Essentially just queue this core up for closing.
-              locker.notifyAll(); // Wakes up closer thread too
-            }
-            return true;
-          }
-          return false;
-        }
-      };
-    }
-  }
-
-  protected void putDynamicDescriptor(String rawName, CoreDescriptor p) {
-    synchronized (locker) {
-      dynamicDescriptors.put(rawName, p);
-    }
-  }
-
-  // We are shutting down. We don't want to risk deadlock, so do this manipulation the expensive way. Note, I've
-  // already deadlocked with closing/opening cores while keeping locks here....
-  protected void clearMaps(ConfigSolr cfg) {
-    List<String> coreNames;
-    List<String> transientNames;
-    List<SolrCore> pendingToClose;
-    synchronized (locker) {
-      coreNames = new ArrayList(cores.keySet());
-      transientNames = new ArrayList(transientCores.keySet());
-      pendingToClose = new ArrayList(pendingCloses);
-    }
-    for (String coreName : coreNames) {
-      SolrCore core = cores.get(coreName);
-      if (core != null) {
-        try {
-          addPersistOneCore(cfg, core, container.loader);
-
-          core.close();
-        } catch (Throwable t) {
-          SolrException.log(CoreContainer.log, "Error shutting down core", t);
-        }
-      }
-    }
-    cores.clear();
-
-    for (String coreName : transientNames) {
-      SolrCore core = transientCores.get(coreName);
-      if (core != null) {
-        try {
-          core.close();
-        } catch (Throwable t) {
-          SolrException.log(CoreContainer.log, "Error shutting down core", t);
-        }
-      }
-    }
-    transientCores.clear();
-
-    // We might have some cores that we were _thinking_ about shutting down, so take care of those too.
-    for (SolrCore core : pendingToClose) {
-      try {
-        core.close();
-      } catch (Throwable t) {
-        SolrException.log(CoreContainer.log, "Error shutting down core", t);
-      }
-    }
-
-  }
-
-  protected void addCoresToList(ArrayList<SolrCoreState> coreStates) {
-    List<SolrCore> addCores;
-    synchronized (locker) {
-      addCores = new ArrayList<SolrCore>(cores.values());
-    }
-    for (SolrCore core : addCores) {
-      coreStates.add(core.getUpdateHandler().getSolrCoreState());
-    }
-  }
-
-  //WARNING! This should be the _only_ place you put anything into the list of transient cores!
-  protected SolrCore putTransientCore(ConfigSolr cfg, String name, SolrCore core, SolrResourceLoader loader) {
-    SolrCore retCore;
-    CoreContainer.log.info("Opening transient core {}", name);
-    synchronized (locker) {
-      retCore = transientCores.put(name, core);
-  }
-    return retCore;
-  }
-
-  protected SolrCore putCore(String name, SolrCore core) {
-    synchronized (locker) {
-      return cores.put(name, core);
-    }
-  }
-
-  List<SolrCore> getCores() {
-    List<SolrCore> lst = new ArrayList<SolrCore>();
-
-    synchronized (locker) {
-      lst.addAll(cores.values());
-      return lst;
-    }
-  }
-
-  Set<String> getCoreNames() {
-    Set<String> set = new TreeSet<String>();
-
-    synchronized (locker) {
-      set.addAll(cores.keySet());
-      set.addAll(transientCores.keySet());
-    }
-    return set;
-  }
-
-  List<String> getCoreNames(SolrCore core) {
-    List<String> lst = new ArrayList<String>();
-
-    synchronized (locker) {
-      for (Map.Entry<String, SolrCore> entry : cores.entrySet()) {
-        if (core == entry.getValue()) {
-          lst.add(entry.getKey());
-        }
-      }
-      for (Map.Entry<String, SolrCore> entry : transientCores.entrySet()) {
-        if (core == entry.getValue()) {
-          lst.add(entry.getKey());
-        }
-      }
-    }
-    return lst;
-  }
-
-  /**
-   * Gets a list of all cores, loaded and unloaded (dynamic)
-   *
-   * @return all cores names, whether loaded or unloaded.
-   */
-  public Collection<String> getAllCoreNames() {
-    Set<String> set = new TreeSet<String>();
-    synchronized (locker) {
-      set.addAll(cores.keySet());
-      set.addAll(transientCores.keySet());
-      set.addAll(dynamicDescriptors.keySet());
-    }
-    return set;
-  }
-
-  SolrCore getCore(String name) {
-
-    synchronized (locker) {
-      return cores.get(name);
-    }
-  }
-
-  protected void swap(String n0, String n1) {
-
-    synchronized (locker) {
-      SolrCore c0 = cores.get(n0);
-      SolrCore c1 = cores.get(n1);
-      if (c0 == null)
-        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "No such core: " + n0);
-      if (c1 == null)
-        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "No such core: " + n1);
-      cores.put(n0, c1);
-      cores.put(n1, c0);
-
-      c0.setName(n1);
-      c0.getCoreDescriptor().putProperty(CoreDescriptor.CORE_NAME, n1);
-      c1.setName(n0);
-      c1.getCoreDescriptor().putProperty(CoreDescriptor.CORE_NAME, n0);
-    }
-
-  }
-
-  protected SolrCore remove(String name, boolean removeOrig) {
-
-    synchronized (locker) {
-      SolrCore core = cores.remove(name);
-      if (removeOrig && core != null) {
-        coreToOrigName.remove(core);
-      }
-
-      return core;
-    }
-  }
-
-  protected void putCoreToOrigName(SolrCore c, String name) {
-
-    synchronized (locker) {
-      coreToOrigName.put(c, name);
-    }
-
-  }
-
-  protected void removeCoreToOrigName(SolrCore newCore, SolrCore core) {
-
-    synchronized (locker) {
-      String origName = coreToOrigName.remove(core);
-      if (origName != null) {
-        coreToOrigName.put(newCore, origName);
-      }
-    }
-  }
-
-  protected SolrCore getCoreFromAnyList(String name) {
-    SolrCore core;
-
-    synchronized (locker) {
-      core = cores.get(name);
-      if (core != null) {
-        return core;
-      }
-
-      if (dynamicDescriptors.size() == 0) {
-        return null; // Nobody even tried to define any transient cores, so we're done.
-      }
-      // Now look for already loaded transient cores.
-      return transientCores.get(name);
-    }
-  }
-
-  protected CoreDescriptor getDynamicDescriptor(String name) {
-    synchronized (locker) {
-      return dynamicDescriptors.get(name);
-    }
-  }
-
-  protected boolean isLoaded(String name) {
-    synchronized (locker) {
-      if (cores.containsKey(name)) {
-        return true;
-      }
-      if (transientCores.containsKey(name)) {
-        return true;
-      }
-    }
-    return false;
-
-  }
-
-  protected CoreDescriptor getUnloadedCoreDescriptor(String cname) {
-    synchronized (locker) {
-      CoreDescriptor desc = dynamicDescriptors.get(cname);
-      if (desc == null) {
-        return null;
-      }
-      return new CoreDescriptor(desc);
-    }
-
-  }
-
-  protected String getCoreToOrigName(SolrCore solrCore) {
-    synchronized (locker) {
-      return coreToOrigName.get(solrCore);
-    }
-  }
-
-  protected void publishCoresAsDown(ZkController zkController) {
-    synchronized (locker) {
-      for (SolrCore core : cores.values()) {
-        try {
-          zkController.publish(core.getCoreDescriptor(), ZkStateReader.DOWN);
-        } catch (KeeperException e) {
-          CoreContainer.log.error("", e);
-        } catch (InterruptedException e) {
-          CoreContainer.log.error("", e);
-        }
-      }
-      for (SolrCore core : transientCores.values()) {
-        try {
-          zkController.publish(core.getCoreDescriptor(), ZkStateReader.DOWN);
-        } catch (KeeperException e) {
-          CoreContainer.log.error("", e);
-        } catch (InterruptedException e) {
-          CoreContainer.log.error("", e);
-        }
-      }
-    }
-  }
-
-  // Irrepressably ugly bit of the transition in SOLR-4196, but there as at least one test case that follows
-  // this path, presumably it's there for a reason.
-  // This is really perverse, but all we need the here is to call a couple of static methods that for back-compat
-  // purposes
-  public void persistCores(ConfigSolr cfg, Properties containerProperties, Map<String, String> rootSolrAttribs,
-                           Map<String, String> coresAttribs, File file, File configFile, SolrResourceLoader loader) {
-    // This is expensive in the maximal case, but I think necessary. It should keep a reference open to all of the
-    // current cores while they are saved. Remember that especially the transient core can come and go.
-    //
-    // Maybe the right thing to do is keep all the core descriptors NOT in the SolrCore, but keep all of the
-    // core descriptors in SolrProperties exclusively.
-    // TODO: 5.0 move coreDescriptors out of SolrCore and keep them only once in SolrProperties
-    //
-    synchronized (locker) {
-      if (cfg == null) {
-        ConfigSolrXmlBackCompat.initPersistStatic();
-        persistCores(cfg, cores, loader);
-        persistCores(cfg, transientCores, loader);
-        ConfigSolrXmlBackCompat.addPersistAllCoresStatic(containerProperties, rootSolrAttribs, coresAttribs,
-            (file == null ? configFile : file));
-      } else {
-        cfg.initPersist();
-        persistCores(cfg, cores, loader);
-        persistCores(cfg, transientCores, loader);
-        cfg.addPersistAllCores(containerProperties, rootSolrAttribs, coresAttribs, (file == null ? configFile : file));
-      }
-    }
-  }
-  // Wait here until any pending operations (load, unload or reload) are completed on this core.
-  protected SolrCore waitAddPendingCoreOps(String name) {
-
-    // Keep multiple threads from operating on a core at one time.
-    synchronized (locker) {
-      boolean pending;
-      do { // Are we currently doing anything to this core? Loading, unloading, reloading?
-        pending = pendingCoreOps.contains(name); // wait for the core to be done being operated upon
-        if (! pending) { // Linear list, but shouldn't be too long
-          for (SolrCore core : pendingCloses) {
-            if (core.getName().equals(name)) {
-              pending = true;
-              break;
-            }
-          }
-        }
-        if (container.isShutDown()) return null; // Just stop already.
-
-        if (pending) {
-          try {
-            locker.wait();
-          } catch (InterruptedException e) {
-            return null; // Seems best not to do anything at all if the thread is interrupted
-          }
-        }
-      } while (pending);
-      // We _really_ need to do this within the synchronized block!
-      if (! container.isShutDown()) {
-        if (! pendingCoreOps.add(name)) {
-          CoreContainer.log.warn("Replaced an entry in pendingCoreOps {}, we should not be doing this", name);
-        }
-        return getCoreFromAnyList(name); // we might have been _unloading_ the core, so return the core if it was loaded.
-      }
-    }
-    return null;
-  }
-
-  // We should always be removing the first thing in the list with our name! The idea here is to NOT do anything n
-  // any core while some other operation is working on that core.
-  protected void removeFromPendingOps(String name) {
-    synchronized (locker) {
-      if (! pendingCoreOps.remove(name)) {
-        CoreContainer.log.warn("Tried to remove core {} from pendingCoreOps and it wasn't there. ", name);
-      }
-      locker.notifyAll();
-    }
-  }
-
-
-  protected void persistCores(ConfigSolr cfg, Map<String, SolrCore> whichCores, SolrResourceLoader loader) {
-    for (SolrCore solrCore : whichCores.values()) {
-      addPersistOneCore(cfg, solrCore, loader);
-    }
-  }
-
-  private void addIfNotNull(Map<String, String> coreAttribs, String key, String value) {
-    if (value == null) return;
-    coreAttribs.put(key, value);
-  }
-
-  protected void addPersistOneCore(ConfigSolr cfg, SolrCore solrCore, SolrResourceLoader loader) {
-
-    CoreDescriptor dcore = solrCore.getCoreDescriptor();
-
-    String coreName = dcore.getProperty(CoreDescriptor.CORE_NAME);
-
-    String origCoreName = null;
-
-    Map<String, String> coreAttribs = new HashMap<String, String>();
-    Properties persistProps = new Properties();
-    CloudDescriptor cd = dcore.getCloudDescriptor();
-    String collection = null;
-    if (cd  != null) collection = cd.getCollectionName();
-    String instDir = dcore.getRawInstanceDir();
-
-    if (cfg == null) {
-      addIfNotNull(coreAttribs, CoreDescriptor.CORE_NAME, coreName);
-      addIfNotNull(coreAttribs, CoreDescriptor.CORE_CONFIG, dcore.getDefaultConfigName());
-      addIfNotNull(coreAttribs, CoreDescriptor.CORE_SCHEMA, dcore.getDefaultSchemaName());
-      addIfNotNull(coreAttribs, CoreDescriptor.CORE_DATADIR, dcore.getProperty(CoreDescriptor.CORE_DATADIR));
-      addIfNotNull(coreAttribs, CoreDescriptor.CORE_ULOGDIR, dcore.getProperty(CoreDescriptor.CORE_ULOGDIR));
-      addIfNotNull(coreAttribs, CoreDescriptor.CORE_TRANSIENT, dcore.getProperty(CoreDescriptor.CORE_TRANSIENT));
-      addIfNotNull(coreAttribs, CoreDescriptor.CORE_LOADONSTARTUP, dcore.getProperty(CoreDescriptor.CORE_LOADONSTARTUP));
-      // we don't try and preserve sys prop defs in these
-
-      addIfNotNull(coreAttribs, CoreDescriptor.CORE_PROPERTIES, dcore.getPropertiesName());
-      // Add in any non-standard bits of data
-      Set<String> std = new TreeSet<String>();
-
-      Properties allProps = dcore.getCoreProperties();
-
-      std.addAll(Arrays.asList(CoreDescriptor.standardPropNames));
-
-      for (String prop : allProps.stringPropertyNames()) {
-        if (! std.contains(prop)) {
-          persistProps.put(prop, dcore.getProperty(prop));
-        }
-      }
-      if (StringUtils.isNotBlank(collection) && !collection.equals(coreName)) {
-        coreAttribs.put(CoreDescriptor.CORE_COLLECTION, collection);
-      }
-
-    } else {
-
-      origCoreName = getCoreToOrigName(solrCore);
-
-      if (origCoreName == null) {
-        origCoreName = coreName;
-      }
-      String tmp = cfg.getCoreNameFromOrig(origCoreName, loader, coreName);
-      if (tmp != null) coreName = tmp;
-
-      coreAttribs = cfg.readCoreAttributes(origCoreName);
-      persistProps = cfg.readCoreProperties(origCoreName);
-      if (coreAttribs != null) {
-        coreAttribs.put(CoreDescriptor.CORE_NAME, coreName);
-        if (coreAttribs.containsKey(CoreDescriptor.CORE_COLLECTION)) collection = coreAttribs.get(CoreDescriptor.CORE_COLLECTION);
-        if (coreAttribs.containsKey(CoreDescriptor.CORE_INSTDIR)) instDir = coreAttribs.get(CoreDescriptor.CORE_INSTDIR);
-      }
-      addIfNotNull(coreAttribs, CoreDescriptor.CORE_INSTDIR, dcore.getRawInstanceDir());
-      coreAttribs.put(CoreDescriptor.CORE_COLLECTION, StringUtils.isNotBlank(collection) ? collection : dcore.getName());
-
-    }
-
-    // Default value here is same as old code.
-    addIfNotNull(coreAttribs, CoreDescriptor.CORE_INSTDIR, instDir);
-
-    // Emulating the old code, just overwrite shard and roles if present in the cloud descriptor
-    if (cd != null) {
-      addIfNotNull(coreAttribs, CoreDescriptor.CORE_SHARD, cd.getShardId());
-      addIfNotNull(coreAttribs, CoreDescriptor.CORE_ROLES, cd.getRoles());
-    }
-    coreAttribs.put(CoreDescriptor.CORE_LOADONSTARTUP, Boolean.toString(dcore.isLoadOnStartup()));
-    coreAttribs.put(CoreDescriptor.CORE_TRANSIENT, Boolean.toString(dcore.isTransient()));
-
-    // Now add back in any implicit properties that aren't in already. These are all "attribs" in this meaning
-    Properties implicit = dcore.initImplicitProperties();
-
-    if (! coreName.equals(container.getDefaultCoreName())) {
-      for (String prop : implicit.stringPropertyNames()) {
-        if (coreAttribs.get(prop) == null) {
-          coreAttribs.put(prop, implicit.getProperty(prop));
-        }
-      }
-    }
-    if (cfg != null) {
-      cfg.addPersistCore(coreName, persistProps, coreAttribs);
-    } else {
-      // Another awkward bit for back-compat for SOLR-4196
-      ConfigSolrXmlBackCompat.addPersistCore(persistProps, coreAttribs);
-    }
+  
+  String getCoreToOrigName(SolrCore core) {
+    return solrCores.getCoreToOrigName(core);
   }
-
-  protected Object getLocker() { return locker; }
-
-  // Be a little careful. We don't want to either open or close a core unless it's _not_ being opened or closed by
-  // another thread. So within this lock we'll walk along the list of pending closes until we find something NOT in
-  // the list of threads currently being loaded or reloaded. The "usual" case will probably return the very first
-  // one anyway..
-  protected SolrCore getCoreToClose() {
-    synchronized (locker) {
-      for (SolrCore core : pendingCloses) {
-        if (! pendingCoreOps.contains(core.getName())) {
-          pendingCoreOps.add(core.getName());
-          pendingCloses.remove(core);
-          return core;
-        }
-      }
-    }
-    return null;
+  
+  private Document copyDoc(Document document) throws TransformerException {
+    TransformerFactory tfactory = TransformerFactory.newInstance();
+    Transformer tx   = tfactory.newTransformer();
+    DOMSource source = new DOMSource(document);
+    DOMResult result = new DOMResult();
+    tx.transform(source,result);
+    return (Document)result.getNode();
   }
-
-
 }
 
 class CloserThread extends Thread {
   CoreContainer container;
-  CoreMaps coreMaps;
+  SolrCores solrCores;
   ConfigSolr cfg;
 
 
-  CloserThread(CoreContainer container, CoreMaps coreMaps, ConfigSolr cfg) {
+  CloserThread(CoreContainer container, SolrCores solrCores, ConfigSolr cfg) {
     this.container = container;
-    this.coreMaps = coreMaps;
+    this.solrCores = solrCores;
     this.cfg = cfg;
   }
 
@@ -1915,24 +1225,24 @@ class CloserThread extends Thread {
   @Override
   public void run() {
     while (! container.isShutDown()) {
-      synchronized (coreMaps.getLocker()) { // need this so we can wait and be awoken.
+      synchronized (solrCores.getModifyLock()) { // need this so we can wait and be awoken.
         try {
-          coreMaps.getLocker().wait();
+          solrCores.getModifyLock().wait();
         } catch (InterruptedException e) {
           // Well, if we've been told to stop, we will. Otherwise, continue on and check to see if there are
           // any cores to close.
         }
       }
-      for (SolrCore removeMe = coreMaps.getCoreToClose();
+      for (SolrCore removeMe = solrCores.getCoreToClose();
            removeMe != null && !container.isShutDown();
-           removeMe = coreMaps.getCoreToClose()) {
+           removeMe = solrCores.getCoreToClose()) {
         try {
-          coreMaps.addPersistOneCore(cfg, removeMe, container.loader);
           removeMe.close();
         } finally {
-          coreMaps.removeFromPendingOps(removeMe.getName());
+          solrCores.removeFromPendingOps(removeMe.getName());
         }
       }
     }
   }
+  
 }

Modified: lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/core/CoreDescriptor.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/core/CoreDescriptor.java?rev=1479862&r1=1479861&r2=1479862&view=diff
==============================================================================
--- lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/core/CoreDescriptor.java (original)
+++ lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/core/CoreDescriptor.java Tue May  7 11:20:55 2013
@@ -22,6 +22,7 @@ import java.io.File;
 
 import org.apache.commons.lang.StringUtils;
 import org.apache.solr.cloud.CloudDescriptor;
+import org.apache.solr.core.ConfigSolr.CfgProp;
 
 /**
  * A Solr core descriptor
@@ -33,7 +34,7 @@ public class CoreDescriptor {
   // Properties file name constants
   public static final String CORE_NAME = "name";
   public static final String CORE_CONFIG = "config";
-  public static final String CORE_INSTDIR = "instanceDir";
+  public static final String CORE_INSTDIR = "instanceDir"; // should probably be removed after 4x
   public static final String CORE_DATADIR = "dataDir";
   public static final String CORE_ULOGDIR = "ulogDir";
   public static final String CORE_SCHEMA = "schema";
@@ -77,6 +78,7 @@ public class CoreDescriptor {
     coreProperties.put(CORE_TRANSIENT, "false");
 
   }
+  
   public CoreDescriptor(CoreContainer container, String name, String instanceDir) {
     this(container);
     doInit(name, instanceDir);
@@ -134,7 +136,11 @@ public class CoreDescriptor {
   }
 
   public Properties initImplicitProperties() {
-    Properties implicitProperties = new Properties(coreContainer.getContainerProperties());
+
+    Properties implicitProperties = new Properties();
+    if (coreContainer != null && coreContainer.getContainerProperties() != null){
+      implicitProperties.putAll(coreContainer.getContainerProperties());
+    }
     implicitProperties.setProperty("solr.core.name", getName());
     implicitProperties.setProperty("solr.core.instanceDir", getInstanceDir());
     implicitProperties.setProperty("solr.core.dataDir", getDataDir());
@@ -197,12 +203,22 @@ public class CoreDescriptor {
    */
   public String getInstanceDir() {
     String instDir = coreProperties.getProperty(CORE_INSTDIR);
-    if (instDir == null) return null; // No worse than before.
+    if (instDir == null) return null;
 
     if (new File(instDir).isAbsolute()) {
       return SolrResourceLoader.normalizeDir(
           SolrResourceLoader.normalizeDir(instDir));
     }
+
+    if (coreContainer == null) return null;
+    if( coreContainer.cfg != null) {
+      String coreRootDir = coreContainer.cfg.get(
+          CfgProp.SOLR_COREROOTDIRECTORY, null);
+      if (coreRootDir != null) {
+        return SolrResourceLoader.normalizeDir(coreRootDir
+            + SolrResourceLoader.normalizeDir(instDir));
+      }
+    }
     return SolrResourceLoader.normalizeDir(coreContainer.getSolrHome() +
         SolrResourceLoader.normalizeDir(instDir));
   }
@@ -327,4 +343,22 @@ public class CoreDescriptor {
   public void putProperty(String prop, String val) {
     coreProperties.put(prop, val);
   }
+
+  // This is particularly useful for checking if any two cores have the same
+  // data dir.
+  public String getAbsoluteDataDir() {
+    String dataDir = getDataDir();
+    if (dataDir == null) return null; // No worse than before.
+
+    if (new File(dataDir).isAbsolute()) {
+      return SolrResourceLoader.normalizeDir(
+          SolrResourceLoader.normalizeDir(dataDir));
+    }
+
+    if (coreContainer == null) return null;
+
+    return SolrResourceLoader.normalizeDir(coreContainer.getSolrHome() +
+        SolrResourceLoader.normalizeDir(dataDir));
+
+  }
 }

Modified: lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/core/DirectoryFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/core/DirectoryFactory.java?rev=1479862&r1=1479861&r2=1479862&view=diff
==============================================================================
--- lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/core/DirectoryFactory.java (original)
+++ lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/core/DirectoryFactory.java Tue May  7 11:20:55 2013
@@ -140,9 +140,6 @@ public abstract class DirectoryFactory i
    * Returns the Directory for a given path, using the specified rawLockType.
    * Will return the same Directory instance for the same path.
    * 
-   * Note: sometimes you might pass null for the rawLockType when
-   * you know the Directory exists and the rawLockType is already
-   * in use.
    * 
    * @throws IOException If there is a low-level I/O error.
    */
@@ -150,18 +147,6 @@ public abstract class DirectoryFactory i
       throws IOException;
   
   /**
-   * Returns the Directory for a given path, using the specified rawLockType.
-   * Will return the same Directory instance for the same path unless forceNew,
-   * in which case a new Directory is returned. There is no need to call
-   * {@link #doneWithDirectory(Directory)} in this case - the old Directory
-   * will be closed when it's ref count hits 0.
-   * 
-   * @throws IOException If there is a low-level I/O error.
-   */
-  public abstract Directory get(String path,  DirContext dirContext, String rawLockType,
-      boolean forceNew) throws IOException;
-  
-  /**
    * Increment the number of references to the given Directory. You must call
    * release for every call to this method.
    * 

Modified: lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/core/EphemeralDirectoryFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/core/EphemeralDirectoryFactory.java?rev=1479862&r1=1479861&r2=1479862&view=diff
==============================================================================
--- lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/core/EphemeralDirectoryFactory.java (original)
+++ lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/core/EphemeralDirectoryFactory.java Tue May  7 11:20:55 2013
@@ -16,12 +16,9 @@ package org.apache.solr.core;
  * limitations under the License.
  */
 
-import java.io.File;
 import java.io.IOException;
 
-import org.apache.commons.io.FileUtils;
 import org.apache.lucene.store.Directory;
-import org.apache.solr.core.CachingDirectoryFactory.CacheValue;
 
 /**
  * Directory provider for implementations that do not persist over reboots.

Modified: lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/core/IndexDeletionPolicyWrapper.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/core/IndexDeletionPolicyWrapper.java?rev=1479862&r1=1479861&r2=1479862&view=diff
==============================================================================
--- lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/core/IndexDeletionPolicyWrapper.java (original)
+++ lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/core/IndexDeletionPolicyWrapper.java Tue May  7 11:20:55 2013
@@ -18,6 +18,7 @@ package org.apache.solr.core;
 
 import org.apache.lucene.index.IndexCommit;
 import org.apache.lucene.index.IndexDeletionPolicy;
+import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.store.Directory;
 import org.apache.solr.update.SolrIndexWriter;
 
@@ -32,11 +33,15 @@ import java.util.concurrent.atomic.Atomi
  * Provides features for looking up IndexCommit given a version. Allows reserving index
  * commit points for certain amounts of time to support features such as index replication
  * or snapshooting directly out of a live index directory.
- *
+ * <p/>
+ * <b>NOTE</b>: The {@link #clone()} method returns <tt>this</tt> in order to make
+ * this {@link IndexDeletionPolicy} instance trackable across {@link IndexWriter}
+ * instantiations. This is correct because each core has its own
+ * {@link IndexDeletionPolicy} and never has more than one open {@link IndexWriter}.
  *
  * @see org.apache.lucene.index.IndexDeletionPolicy
  */
-public class IndexDeletionPolicyWrapper implements IndexDeletionPolicy {
+public final class IndexDeletionPolicyWrapper extends IndexDeletionPolicy {
   private final IndexDeletionPolicy deletionPolicy;
   private volatile Map<Long, IndexCommit> solrVersionVsCommits = new ConcurrentHashMap<Long, IndexCommit>();
   private final Map<Long, Long> reserves = new ConcurrentHashMap<Long,Long>();
@@ -232,7 +237,9 @@ public class IndexDeletionPolicyWrapper 
         map.put(wrapper.delegate.getGeneration(), wrapper.delegate);
     }
     solrVersionVsCommits = map;
-    latestCommit = ((list.get(list.size() - 1)).delegate);
+    if (!list.isEmpty()) {
+      latestCommit = ((list.get(list.size() - 1)).delegate);
+    }
   }
 
   public static long getCommitTimestamp(IndexCommit commit) throws IOException {
@@ -244,5 +251,11 @@ public class IndexDeletionPolicyWrapper 
       return 0;
     }
   }
+
+  @Override
+  public IndexDeletionPolicy clone() {
+    // see class-level javadocs
+    return this;
+  }
 }
 

Modified: lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/core/IndexReaderFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/core/IndexReaderFactory.java?rev=1479862&r1=1479861&r2=1479862&view=diff
==============================================================================
--- lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/core/IndexReaderFactory.java (original)
+++ lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/core/IndexReaderFactory.java Tue May  7 11:20:55 2013
@@ -19,7 +19,6 @@ package org.apache.solr.core;
 import java.io.IOException;
 
 import org.apache.lucene.index.DirectoryReader;
-import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.store.Directory;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.util.plugin.NamedListInitializedPlugin;
@@ -61,7 +60,7 @@ public abstract class IndexReaderFactory
    * @param indexDir indexDir index location
    * @param core {@link SolrCore} instance where this reader will be used. NOTE:
    * this SolrCore instance may not be fully configured yet, but basic things like
-   * {@link SolrCore#getCoreDescriptor()}, {@link SolrCore#getSchema()} and
+   * {@link SolrCore#getCoreDescriptor()}, {@link SolrCore#getLatestSchema()} and
    * {@link SolrCore#getSolrConfig()} are valid.
    * @return An IndexReader instance
    * @throws IOException If there is a low-level I/O error.

Modified: lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/core/QuerySenderListener.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/core/QuerySenderListener.java?rev=1479862&r1=1479861&r2=1479862&view=diff
==============================================================================
--- lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/core/QuerySenderListener.java (original)
+++ lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/core/QuerySenderListener.java Tue May  7 11:20:55 2013
@@ -42,7 +42,7 @@ public class QuerySenderListener extends
   public void newSearcher(SolrIndexSearcher newSearcher, SolrIndexSearcher currentSearcher) {
     final SolrIndexSearcher searcher = newSearcher;
     log.info("QuerySenderListener sending requests to " + newSearcher);
-    List<NamedList> allLists = (List<NamedList>)args.get("queries");
+    List<NamedList> allLists = (List<NamedList>)getArgs().get("queries");
     if (allLists == null) return;
     for (NamedList nlst : allLists) {
       SolrQueryRequest req = null;
@@ -54,14 +54,14 @@ public class QuerySenderListener extends
         if (params.get("distrib") == null) {
           params.add("distrib", false);
         }
-        req = new LocalSolrQueryRequest(core,params) {
+        req = new LocalSolrQueryRequest(getCore(),params) {
           @Override public SolrIndexSearcher getSearcher() { return searcher; }
           @Override public void close() { }
         };
 
         SolrQueryResponse rsp = new SolrQueryResponse();
         SolrRequestInfo.setRequestInfo(new SolrRequestInfo(req, rsp));
-        core.execute(core.getRequestHandler(req.getParams().get(CommonParams.QT)), req, rsp);
+        getCore().execute(getCore().getRequestHandler(req.getParams().get(CommonParams.QT)), req, rsp);
 
         // Retrieve the Document instances (not just the ids) to warm
         // the OS disk cache, and any Solr document cache.  Only the top