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

svn commit: r1470674 [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/util/ core/src/test/org/apache/solr/core/ test-framework/src/java/org/apache/solr/util/

Author: markrmiller
Date: Mon Apr 22 19:35:10 2013
New Revision: 1470674

URL: http://svn.apache.org/r1470674
Log:
SOLR-4749: Clean up and refactor CoreContainer code around solr.xml and SolrCore management.

Added:
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/ConfigSolrXmlOld.java   (with props)
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/CoreMaps.java   (with props)
Modified:
    lucene/dev/trunk/solr/CHANGES.txt
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/ZkCLI.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/ConfigSolr.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/ConfigSolrXml.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/CoreContainer.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/util/DOMUtil.java
    lucene/dev/trunk/solr/core/src/test/org/apache/solr/core/TestSolrXml.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=1470674&r1=1470673&r2=1470674&view=diff
==============================================================================
--- lucene/dev/trunk/solr/CHANGES.txt (original)
+++ lucene/dev/trunk/solr/CHANGES.txt Mon Apr 22 19:35:10 2013
@@ -78,6 +78,9 @@ Other Changes
 
 * SOLR-4738: Update Jetty to 8.1.10.v20130312 (Mark Miller, Robert Muir)
 
+* SOLR-4749: Clean up and refactor CoreContainer code around solr.xml and SolrCore
+  management. (Mark Miller)
+
 ==================  4.3.0 ==================
 
 Versions of Major Components

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/ZkCLI.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/ZkCLI.java?rev=1470674&r1=1470673&r2=1470674&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/ZkCLI.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/ZkCLI.java Mon Apr 22 19:35:10 2013
@@ -20,10 +20,13 @@ import org.apache.commons.cli.PosixParse
 import org.apache.commons.io.IOUtils;
 import org.apache.solr.common.cloud.OnReconnect;
 import org.apache.solr.common.cloud.SolrZkClient;
+import org.apache.solr.core.Config;
 import org.apache.solr.core.ConfigSolr;
 import org.apache.solr.core.ConfigSolrXml;
+import org.apache.solr.core.ConfigSolrXmlOld;
 import org.apache.solr.core.SolrResourceLoader;
 import org.apache.zookeeper.KeeperException;
+import org.xml.sax.InputSource;
 import org.xml.sax.SAXException;
 
 /*
@@ -181,7 +184,15 @@ public class ZkCLI {
           ConfigSolr cfg;
 
           try {
-            cfg = new ConfigSolrXml(loader, null, is, null, false, null);
+            Config config = new Config(loader, null, new InputSource(is), null, false);
+            
+            boolean oldStyle = (config.getNode("solr/cores", false) != null);
+            // cfg = new ConfigSolrXml(loader, null, is, null, false, this);
+             if (oldStyle) {
+               cfg = new ConfigSolrXmlOld(config, null);
+             } else {
+               cfg = new ConfigSolrXml(config, null);
+             }
           } finally {
             IOUtils.closeQuietly(is);
           }

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=1470674&r1=1470673&r2=1470674&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 Mon Apr 22 19:35:10 2013
@@ -37,11 +37,16 @@ import javax.xml.namespace.QName;
 import javax.xml.parsers.DocumentBuilder;
 import javax.xml.parsers.DocumentBuilderFactory;
 import javax.xml.parsers.ParserConfigurationException;
+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.XPath;
 import javax.xml.xpath.XPathConstants;
 import javax.xml.xpath.XPathExpressionException;
 import javax.xml.xpath.XPathFactory;
+
 import java.io.IOException;
 import java.util.Arrays;
 import java.util.HashSet;
@@ -75,18 +80,6 @@ public class Config {
     this( loader, name, null, null );
   }
 
-  /**
-   * For the transition from using solr.xml to solr.properties, see SOLR-4196. Remove
-   * for 5.0, thus it's already deprecated
-   * @param loader - Solr resource loader
-   * @param cfg    - SolrConfig, for backwards compatability with solr.xml layer.
-   * @throws TransformerException if the XML file is mal-formed
-   */
-  @Deprecated
-  public Config(SolrResourceLoader loader, Config cfg) throws TransformerException {
-    this(loader, null, ConfigSolrXml.copyDoc(cfg.getDocument()));
-  }
-
   public Config(SolrResourceLoader loader, String name, InputSource is, String prefix) throws ParserConfigurationException, IOException, SAXException 
   {
     this(loader, name, is, prefix, true);
@@ -164,6 +157,16 @@ public class Config {
     this.loader = loader;
   }
 
+  
+  private static Document copyDoc(Document doc) throws TransformerException {
+    TransformerFactory tfactory = TransformerFactory.newInstance();
+    Transformer tx = tfactory.newTransformer();
+    DOMSource source = new DOMSource(doc);
+    DOMResult result = new DOMResult();
+    tx.transform(source, result);
+    return (Document) result.getNode();
+  }
+  
   /**
    * @since solr 1.3
    */

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/ConfigSolr.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/ConfigSolr.java?rev=1470674&r1=1470673&r2=1470674&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/ConfigSolr.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/ConfigSolr.java Mon Apr 22 19:35:10 2013
@@ -17,26 +17,30 @@ package org.apache.solr.core;
  * limitations under the License.
  */
 
-import org.apache.solr.cloud.ZkController;
-import org.apache.solr.handler.component.ShardHandlerFactory;
-
-import java.io.File;
+import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Properties;
 
-/**
- * ConfigSolr is a new interface  to aid us in obsoleting solr.xml and replacing it with solr.properties. The problem here
- * is that the Config class is used for _all_ the xml file, e.g. solrconfig.xml and we can't mess with _that_ as part
- * of this issue. Primarily used in CoreContainer at present.
- * <p/>
- * This is already deprecated, it's only intended to exist for while transitioning to properties-based replacement for
- * solr.xml
- *
- * @since solr 4.3
- */
-public interface ConfigSolr {
-
+import javax.xml.xpath.XPath;
+import javax.xml.xpath.XPathConstants;
+import javax.xml.xpath.XPathExpressionException;
+
+import org.apache.solr.common.SolrException;
+import org.apache.solr.util.DOMUtil;
+import org.apache.solr.util.PropertiesUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.w3c.dom.Node;
+import org.w3c.dom.NodeList;
+
+
+public abstract class ConfigSolr {
+  protected static Logger log = LoggerFactory.getLogger(ConfigSolr.class);
+  
+  public final static String CORE_PROP_FILE = "core.properties";
+  public final static String SOLR_XML_FILE = "solr.xml";
+  
   // Ugly for now, but we'll at least be able to centralize all of the differences between 4x and 5x.
   public static enum CfgProp {
     SOLR_ADMINHANDLER,
@@ -67,48 +71,118 @@ public interface ConfigSolr {
     SOLR_PERSISTENT,
     SOLR_CORES_DEFAULT_CORE_NAME,
     SOLR_ADMINPATH
-  };
-
-  public final static String CORE_PROP_FILE = "core.properties";
-  public final static String SOLR_XML_FILE = "solr.xml";
-
-  public int getInt(CfgProp prop, int def);
-
-  public boolean getBool(CfgProp prop,boolean defValue);
-
-  public String get(CfgProp prop, String def);
-
-  public String getOrigProp(CfgProp prop, String def);
+  }
 
-  public void substituteProperties();
-
-  public ShardHandlerFactory initShardHandler();
-
-  public Properties getSolrProperties(String context);
-
-  public SolrConfig getSolrConfigFromZk(ZkController zkController, String zkConfigName, String solrConfigFileName,
-                                        SolrResourceLoader resourceLoader);
-
-  public void initPersist();
-
-  public void addPersistCore(String coreName, Properties attribs, Map<String, String> props);
-
-  public void addPersistAllCores(Properties containerProperties, Map<String, String> rootSolrAttribs, Map<String, String> coresAttribs,
-                                 File file);
-
-  public String getCoreNameFromOrig(String origCoreName, SolrResourceLoader loader, String coreName);
+  protected Config config;
+  protected Map<CfgProp, String> propMap = new HashMap<CfgProp, String>();
+  protected final Map<String, String> badConfigCores = new HashMap<String, String>();
+
+  public ConfigSolr(Config config) {
+    this.config = config;
+  }
+  
+  public Config getConfig() {
+    return config;
+  }
+  
+  // If the core is not to be loaded (say two cores defined with the same name or with the same data dir), return
+  // the reason. If it's OK to load the core, return null.
+  public String getBadConfigCoreMessage(String name) {
+    return badConfigCores.get(name);
+  }
+
+  public int getInt(CfgProp prop, int def) {
+    String val = propMap.get(prop);
+    if (val != null) val = PropertiesUtil.substituteProperty(val, null);
+    return (val == null) ? def : Integer.parseInt(val);
+  }
+
+  public boolean getBool(CfgProp prop, boolean defValue) {
+    String val = propMap.get(prop);
+    if (val != null) val = PropertiesUtil.substituteProperty(val, null);
+    return (val == null) ? defValue : Boolean.parseBoolean(val);
+  }
+
+  public String get(CfgProp prop, String def) {
+    String val = propMap.get(prop);
+    if (val != null) val = PropertiesUtil.substituteProperty(val, null);
+    return (val == null) ? def : val;
+  }
+
+  // For saving the original property, ${} syntax and all.
+  public String getOrigProp(CfgProp prop, String def) {
+    String val = propMap.get(prop);
+    return (val == null) ? def : val;
+  }
+
+  public Properties getSolrProperties(String path) {
+    try {
+      return readProperties(((NodeList) config.evaluate(
+          path, XPathConstants.NODESET)).item(0));
+    } catch (Throwable e) {
+      SolrException.log(log, null, e);
+    }
+    return null;
+
+  }
+  
+  protected Properties readProperties(Node node) throws XPathExpressionException {
+    XPath xpath = config.getXPath();
+    NodeList props = (NodeList) xpath.evaluate("property", node, XPathConstants.NODESET);
+    Properties properties = new Properties();
+    for (int i = 0; i < props.getLength(); i++) {
+      Node prop = props.item(i);
+      properties.setProperty(DOMUtil.getAttr(prop, "name"), DOMUtil.getAttr(prop, "value"));
+    }
+    return properties;
+  }
+
+  public abstract void substituteProperties();
+
+  public abstract String getCoreNameFromOrig(String origCoreName, SolrResourceLoader loader, String coreName);
 
-  public List<String> getAllCoreNames();
+  public abstract List<String> getAllCoreNames();
 
-  public String getProperty(String coreName, String property, String defaultVal);
+  public abstract String getProperty(String coreName, String property, String defaultVal);
 
-  public Properties readCoreProperties(String coreName);
+  public abstract Properties readCoreProperties(String coreName);
 
-  public Map<String, String> readCoreAttributes(String coreName);
+  public abstract Map<String, String> readCoreAttributes(String coreName);
 
-  // If the core is not to be loaded (say two cores defined with the same name or with the same data dir), return
-  // the reason. If it's OK to load the core, return null.
-  public String getBadConfigCoreMessage(String name);
+}
 
-  public boolean is50OrLater();
+// It's mightily convenient to have all of the original path names and property
+// values when persisting cores, so
+// this little convenience class is just for that.
+// Also, let's keep track of anything we added here, especially the instance dir
+// for persistence purposes. We don't
+// want, for instance, to persist instanceDir if it was not specified
+// originally.
+//
+// I suspect that for persistence purposes, we may want to expand this idea to
+// record, say, ${blah}
+class CoreDescriptorPlus {
+  private CoreDescriptor coreDescriptor;
+  private String filePath;
+  private Properties propsOrig; // TODO: 5.0. Remove this since it's only really
+                                // used for persisting.
+  
+  CoreDescriptorPlus(String filePath, CoreDescriptor descriptor,
+      Properties propsOrig) {
+    coreDescriptor = descriptor;
+    this.filePath = filePath;
+    this.propsOrig = propsOrig;
+  }
+  
+  CoreDescriptor getCoreDescriptor() {
+    return coreDescriptor;
+  }
+  
+  String getFilePath() {
+    return filePath;
+  }
+  
+  Properties getPropsOrig() {
+    return propsOrig;
+  }
 }

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/ConfigSolrXml.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/ConfigSolrXml.java?rev=1470674&r1=1470673&r2=1470674&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/ConfigSolrXml.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/ConfigSolrXml.java Mon Apr 22 19:35:10 2013
@@ -17,153 +17,86 @@ package org.apache.solr.core;
  * limitations under the License.
  */
 
-import org.apache.commons.io.IOUtils;
-import org.apache.solr.cloud.ZkController;
-import org.apache.solr.common.SolrException;
-import org.apache.solr.handler.component.HttpShardHandlerFactory;
-import org.apache.solr.handler.component.ShardHandlerFactory;
-import org.apache.solr.util.DOMUtil;
-import org.apache.solr.util.PropertiesUtil;
-import org.apache.solr.util.SystemIdResolver;
-import org.apache.solr.util.plugin.PluginInfoInitialized;
-import org.w3c.dom.Document;
-import org.w3c.dom.NamedNodeMap;
-import org.w3c.dom.Node;
-import org.w3c.dom.NodeList;
-import org.xml.sax.InputSource;
-import org.xml.sax.SAXException;
-
-import javax.xml.parsers.ParserConfigurationException;
-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.XPath;
-import javax.xml.xpath.XPathConstants;
-import javax.xml.xpath.XPathExpressionException;
 import java.io.ByteArrayInputStream;
 import java.io.File;
 import java.io.FileInputStream;
 import java.io.IOException;
 import java.io.InputStream;
 import java.util.ArrayList;
-import java.util.Collections;
 import java.util.HashMap;
-import java.util.HashSet;
 import java.util.List;
 import java.util.Locale;
 import java.util.Map;
 import java.util.Properties;
-import java.util.Set;
 
-/**
- * ConfigSolrXml
- * <p/>
- * This class is entirely to localize the dealing with specific issues when transitioning from old-style solr.xml
- * to new-style xml, enumeration/discovery of defined cores. See SOLR-4196 for background.
- * <p/>
- * @since solr 4.3
- *
- * It's a bit twisted, but we decided to NOT do the solr.properties switch. But since there's already an interface
- * it makes sense to leave it in so we can use other methods of providing the Solr information that is contained
- * in solr.xml. Perhaps something form SolrCloud in the future?
- *
- */
+import javax.xml.parsers.ParserConfigurationException;
 
+import org.apache.commons.io.IOUtils;
+import org.apache.solr.cloud.ZkController;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.schema.IndexSchema;
+import org.apache.solr.util.PropertiesUtil;
+import org.apache.solr.util.SystemIdResolver;
+import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.xml.sax.InputSource;
+import org.xml.sax.SAXException;
 
-public class ConfigSolrXml extends Config implements ConfigSolr {
 
-  private boolean is50OrLater = false;
+/**
+ *
+ */
+public class ConfigSolrXml extends ConfigSolr {
+  protected static Logger log = LoggerFactory.getLogger(ConfigSolrXml.class);
 
   private final Map<String, CoreDescriptorPlus> coreDescriptorPlusMap = new HashMap<String, CoreDescriptorPlus>();
-  private NodeList coreNodes = null;
-  private final Map<String, String> badConfigCores = new HashMap<String, String>();
-    // List of cores that we should _never_ load. Ones with dup names or duplicate datadirs or...
-
 
-  private Map<CfgProp, String> propMap = new HashMap<CfgProp, String>();
-
-  public ConfigSolrXml(SolrResourceLoader loader, String name, InputStream is, String prefix,
-                       boolean subProps, CoreContainer container)
+  public ConfigSolrXml(Config config, CoreContainer container)
       throws ParserConfigurationException, IOException, SAXException {
-
-    super(loader, name, new InputSource(is), prefix, subProps);
-    init(container);
-  }
-
-
-  public ConfigSolrXml(SolrResourceLoader loader, Config cfg, CoreContainer container)
-      throws TransformerException, IOException {
-
-    super(loader, null, copyDoc(cfg.getDocument())); // Mimics a call from CoreContainer.
+    super(config);
     init(container);
   }
   
   private void init(CoreContainer container) throws IOException {
-    is50OrLater = getNode("solr/cores", false) == null;
-
-    // Do sanity checks, old and new style. Pretty exhaustive for now, but want to hammer this.
-    // TODO: 5.0 maybe remove this checking, it's mostly for correctness as we make this transition.
-
-    if (is50OrLater()) {
-      failIfFound("solr/@coreLoadThreads");
-      failIfFound("solr/@persist");
-      failIfFound("solr/@sharedLib");
-      failIfFound("solr/@zkHost");
-
-      failIfFound("solr/logging/@class");
-      failIfFound("solr/logging/@enabled");
-      failIfFound("solr/logging/watcher/@size");
-      failIfFound("solr/logging/watcher/@threshold");
-
-      failIfFound("solr/cores/@adminHandler");
-      failIfFound("solr/cores/@distribUpdateConnTimeout");
-      failIfFound("solr/cores/@distribUpdateSoTimeout");
-      failIfFound("solr/cores/@host");
-      failIfFound("solr/cores/@hostContext");
-      failIfFound("solr/cores/@hostPort");
-      failIfFound("solr/cores/@leaderVoteWait");
-      failIfFound("solr/cores/@managementPath");
-      failIfFound("solr/cores/@shareSchema");
-      failIfFound("solr/cores/@transientCacheSize");
-      failIfFound("solr/cores/@zkClientTimeout");
-
-      // These have no counterpart in 5.0, asking for any of these in Solr 5.0 will result in an error being
-      // thrown.
-      failIfFound("solr/cores/@defaultCoreName");
-      failIfFound("solr/@persistent");
-      failIfFound("solr/cores/@adminPath");
-    } else {
-      failIfFound("solr/str[@name='adminHandler']");
-      failIfFound("solr/int[@name='coreLoadThreads']");
-      failIfFound("solr/str[@name='coreRootDirectory']");
-      failIfFound("solr/solrcloud/int[@name='distribUpdateConnTimeout']");
-      failIfFound("solr/solrcloud/int[@name='distribUpdateSoTimeout']");
-      failIfFound("solr/solrcloud/str[@name='host']");
-      failIfFound("solr/solrcloud/str[@name='hostContext']");
-      failIfFound("solr/solrcloud/int[@name='hostPort']");
-      failIfFound("solr/solrcloud/int[@name='leaderVoteWait']");
-      failIfFound("solr/str[@name='managementPath']");
-      failIfFound("solr/str[@name='sharedLib']");
-      failIfFound("solr/str[@name='shareSchema']");
-      failIfFound("solr/int[@name='transientCacheSize']");
-      failIfFound("solr/solrcloud/int[@name='zkClientTimeout']");
-      failIfFound("solr/solrcloud/int[@name='zkHost']");
-
-      failIfFound("solr/logging/str[@name='class']");
-      failIfFound("solr/logging/str[@name='enabled']");
-
-      failIfFound("solr/logging/watcher/int[@name='size']");
-      failIfFound("solr/logging/watcher/int[@name='threshold']");
-
-    }
+    
+    // Do sanity checks - we don't want to find old style config
+    failIfFound("solr/@coreLoadThreads");
+    failIfFound("solr/@persist");
+    failIfFound("solr/@sharedLib");
+    failIfFound("solr/@zkHost");
+    
+    failIfFound("solr/logging/@class");
+    failIfFound("solr/logging/@enabled");
+    failIfFound("solr/logging/watcher/@size");
+    failIfFound("solr/logging/watcher/@threshold");
+    
+    failIfFound("solr/cores/@adminHandler");
+    failIfFound("solr/cores/@distribUpdateConnTimeout");
+    failIfFound("solr/cores/@distribUpdateSoTimeout");
+    failIfFound("solr/cores/@host");
+    failIfFound("solr/cores/@hostContext");
+    failIfFound("solr/cores/@hostPort");
+    failIfFound("solr/cores/@leaderVoteWait");
+    failIfFound("solr/cores/@managementPath");
+    failIfFound("solr/cores/@shareSchema");
+    failIfFound("solr/cores/@transientCacheSize");
+    failIfFound("solr/cores/@zkClientTimeout");
+    
+    // These have no counterpart in 5.0, asking for any of these in Solr 5.0
+    // will result in an error being
+    // thrown.
+    failIfFound("solr/cores/@defaultCoreName");
+    failIfFound("solr/@persistent");
+    failIfFound("solr/cores/@adminPath");
+    
     fillPropMap();
     initCoreList(container);
   }
+  
   private void failIfFound(String xPath) {
 
-    if (getVal(xPath, false) != null) {
+    if (config.getVal(xPath, false) != null) {
       throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Should not have found " + xPath +
           " solr.xml may be a mix of old and new style formats.");
     }
@@ -171,7 +104,7 @@ public class ConfigSolrXml extends Confi
 
   // We can do this in 5.0 when we read the solr.xml since we don't need to keep the original around for persistence.
   private String doSub(String path) {
-    String val = getVal(path, false);
+    String val = config.getVal(path, false);
     if (val != null) {
       val = PropertiesUtil.substituteProperty(val, null);
     }
@@ -179,228 +112,49 @@ public class ConfigSolrXml extends Confi
   }
   
   private void fillPropMap() {
-    if (is50OrLater) { // Can do the prop subs early here since we don't need to preserve them for persistence.
-      propMap.put(CfgProp.SOLR_ADMINHANDLER, doSub("solr/str[@name='adminHandler']"));
-      propMap.put(CfgProp.SOLR_CORELOADTHREADS, doSub("solr/int[@name='coreLoadThreads']"));
-      propMap.put(CfgProp.SOLR_COREROOTDIRECTORY, doSub("solr/str[@name='coreRootDirectory']"));
-      propMap.put(CfgProp.SOLR_DISTRIBUPDATECONNTIMEOUT, doSub("solr/solrcloud/int[@name='distribUpdateConnTimeout']"));
-      propMap.put(CfgProp.SOLR_DISTRIBUPDATESOTIMEOUT, doSub("solr/solrcloud/int[@name='distribUpdateSoTimeout']"));
-      propMap.put(CfgProp.SOLR_HOST, doSub("solr/solrcloud/str[@name='host']"));
-      propMap.put(CfgProp.SOLR_HOSTCONTEXT, doSub("solr/solrcloud/str[@name='hostContext']"));
-      propMap.put(CfgProp.SOLR_HOSTPORT, doSub("solr/solrcloud/int[@name='hostPort']"));
-      propMap.put(CfgProp.SOLR_LEADERVOTEWAIT, doSub("solr/solrcloud/int[@name='leaderVoteWait']"));
-      propMap.put(CfgProp.SOLR_MANAGEMENTPATH, doSub("solr/str[@name='managementPath']"));
-      propMap.put(CfgProp.SOLR_SHAREDLIB, doSub("solr/str[@name='sharedLib']"));
-      propMap.put(CfgProp.SOLR_SHARESCHEMA, doSub("solr/str[@name='shareSchema']"));
-      propMap.put(CfgProp.SOLR_TRANSIENTCACHESIZE, doSub("solr/int[@name='transientCacheSize']"));
-      propMap.put(CfgProp.SOLR_ZKCLIENTTIMEOUT, doSub("solr/solrcloud/int[@name='zkClientTimeout']"));
-      propMap.put(CfgProp.SOLR_ZKHOST, doSub("solr/solrcloud/str[@name='zkHost']"));
-
-      propMap.put(CfgProp.SOLR_LOGGING_CLASS, doSub("solr/logging/str[@name='class']"));
-      propMap.put(CfgProp.SOLR_LOGGING_ENABLED, doSub("solr/logging/str[@name='enabled']"));
-
-      propMap.put(CfgProp.SOLR_LOGGING_WATCHER_SIZE, doSub("solr/logging/watcher/int[@name='size']"));
-      propMap.put(CfgProp.SOLR_LOGGING_WATCHER_THRESHOLD, doSub("solr/logging/watcher/int[@name='threshold']"));
-      propMap.put(CfgProp.SOLR_SHARDHANDLERFACTORY_CLASS, doSub("solr/shardHandlerFactory/@class"));
-      propMap.put(CfgProp.SOLR_SHARDHANDLERFACTORY_NAME, doSub("solr/shardHandlerFactory/@name"));
-      propMap.put(CfgProp.SOLR_SHARDHANDLERFACTORY_CONNTIMEOUT, doSub("solr/shardHandlerFactory/int[@name='connTimeout']"));
-      propMap.put(CfgProp.SOLR_SHARDHANDLERFACTORY_SOCKETTIMEOUT, doSub("solr/shardHandlerFactory/int[@name='socketTimeout']"));
-    } else {
-      propMap.put(CfgProp.SOLR_CORELOADTHREADS, getVal("solr/@coreLoadThreads", false));
-      propMap.put(CfgProp.SOLR_SHAREDLIB, getVal("solr/@sharedLib", false));
-      propMap.put(CfgProp.SOLR_ZKHOST, getVal("solr/@zkHost", false));
-
-      propMap.put(CfgProp.SOLR_LOGGING_CLASS, getVal("solr/logging/@class", false));
-      propMap.put(CfgProp.SOLR_LOGGING_ENABLED, getVal("solr/logging/@enabled", false));
-      propMap.put(CfgProp.SOLR_LOGGING_WATCHER_SIZE, getVal("solr/logging/watcher/@size", false));
-      propMap.put(CfgProp.SOLR_LOGGING_WATCHER_THRESHOLD, getVal("solr/logging/watcher/@threshold", false));
-
-      propMap.put(CfgProp.SOLR_ADMINHANDLER, getVal("solr/cores/@adminHandler", false));
-      propMap.put(CfgProp.SOLR_DISTRIBUPDATECONNTIMEOUT, getVal("solr/cores/@distribUpdateConnTimeout", false));
-      propMap.put(CfgProp.SOLR_DISTRIBUPDATESOTIMEOUT, getVal("solr/cores/@distribUpdateSoTimeout", false));
-      propMap.put(CfgProp.SOLR_HOST, getVal("solr/cores/@host", false));
-      propMap.put(CfgProp.SOLR_HOSTCONTEXT, getVal("solr/cores/@hostContext", false));
-      propMap.put(CfgProp.SOLR_HOSTPORT, getVal("solr/cores/@hostPort", false));
-      propMap.put(CfgProp.SOLR_LEADERVOTEWAIT, getVal("solr/cores/@leaderVoteWait", false));
-      propMap.put(CfgProp.SOLR_MANAGEMENTPATH, getVal("solr/cores/@managementPath", false));
-      propMap.put(CfgProp.SOLR_SHARESCHEMA, getVal("solr/cores/@shareSchema", false));
-      propMap.put(CfgProp.SOLR_TRANSIENTCACHESIZE, getVal("solr/cores/@transientCacheSize", false));
-      propMap.put(CfgProp.SOLR_ZKCLIENTTIMEOUT, getVal("solr/cores/@zkClientTimeout", false));
-      propMap.put(CfgProp.SOLR_SHARDHANDLERFACTORY_CLASS, getVal("solr/shardHandlerFactory/@class", false));
-      propMap.put(CfgProp.SOLR_SHARDHANDLERFACTORY_NAME, getVal("solr/shardHandlerFactory/@name", false));
-      propMap.put(CfgProp.SOLR_SHARDHANDLERFACTORY_CONNTIMEOUT, getVal(  "solr/shardHandlerFactory/int[@connTimeout]", false));
-      propMap.put(CfgProp.SOLR_SHARDHANDLERFACTORY_SOCKETTIMEOUT, getVal("solr/shardHandlerFactory/int[@socketTimeout]", false));
-
-      // These have no counterpart in 5.0, asking, for any of these in Solr 5.0 will result in an error being
-      // thrown.
-      propMap.put(CfgProp.SOLR_CORES_DEFAULT_CORE_NAME, getVal("solr/cores/@defaultCoreName", false));
-      propMap.put(CfgProp.SOLR_PERSISTENT, getVal("solr/@persistent", false));
-      propMap.put(CfgProp.SOLR_ADMINPATH, getVal("solr/cores/@adminPath", false));
-    }
+    propMap.put(CfgProp.SOLR_ADMINHANDLER, doSub("solr/str[@name='adminHandler']"));
+    propMap.put(CfgProp.SOLR_CORELOADTHREADS, doSub("solr/int[@name='coreLoadThreads']"));
+    propMap.put(CfgProp.SOLR_COREROOTDIRECTORY, doSub("solr/str[@name='coreRootDirectory']"));
+    propMap.put(CfgProp.SOLR_DISTRIBUPDATECONNTIMEOUT, doSub("solr/solrcloud/int[@name='distribUpdateConnTimeout']"));
+    propMap.put(CfgProp.SOLR_DISTRIBUPDATESOTIMEOUT, doSub("solr/solrcloud/int[@name='distribUpdateSoTimeout']"));
+    propMap.put(CfgProp.SOLR_HOST, doSub("solr/solrcloud/str[@name='host']"));
+    propMap.put(CfgProp.SOLR_HOSTCONTEXT, doSub("solr/solrcloud/str[@name='hostContext']"));
+    propMap.put(CfgProp.SOLR_HOSTPORT, doSub("solr/solrcloud/int[@name='hostPort']"));
+    propMap.put(CfgProp.SOLR_LEADERVOTEWAIT, doSub("solr/solrcloud/int[@name='leaderVoteWait']"));
+    propMap.put(CfgProp.SOLR_MANAGEMENTPATH, doSub("solr/str[@name='managementPath']"));
+    propMap.put(CfgProp.SOLR_SHAREDLIB, doSub("solr/str[@name='sharedLib']"));
+    propMap.put(CfgProp.SOLR_SHARESCHEMA, doSub("solr/str[@name='shareSchema']"));
+    propMap.put(CfgProp.SOLR_TRANSIENTCACHESIZE, doSub("solr/int[@name='transientCacheSize']"));
+    propMap.put(CfgProp.SOLR_ZKCLIENTTIMEOUT, doSub("solr/solrcloud/int[@name='zkClientTimeout']"));
+    propMap.put(CfgProp.SOLR_ZKHOST, doSub("solr/solrcloud/str[@name='zkHost']"));
+
+    propMap.put(CfgProp.SOLR_LOGGING_CLASS, doSub("solr/logging/str[@name='class']"));
+    propMap.put(CfgProp.SOLR_LOGGING_ENABLED, doSub("solr/logging/str[@name='enabled']"));
+    propMap.put(CfgProp.SOLR_LOGGING_WATCHER_SIZE, doSub("solr/logging/watcher/int[@name='size']"));
+    propMap.put(CfgProp.SOLR_LOGGING_WATCHER_THRESHOLD, doSub("solr/logging/watcher/int[@name='threshold']"));
+    propMap.put(CfgProp.SOLR_SHARDHANDLERFACTORY_CLASS, doSub("solr/shardHandlerFactory/@class"));
+    propMap.put(CfgProp.SOLR_SHARDHANDLERFACTORY_NAME, doSub("solr/shardHandlerFactory/@name"));
+    propMap.put(CfgProp.SOLR_SHARDHANDLERFACTORY_CONNTIMEOUT, doSub("solr/shardHandlerFactory/int[@name='connTimeout']"));
+    propMap.put(CfgProp.SOLR_SHARDHANDLERFACTORY_SOCKETTIMEOUT, doSub("solr/shardHandlerFactory/int[@name='socketTimeout']"));
   }
 
   private void initCoreList(CoreContainer container) throws IOException {
-    if (is50OrLater) {
-      if (container != null) { //TODO: 5.0. Yet another bit of nonsense only because of the test harness.
-        synchronized (coreDescriptorPlusMap) {
-          String coreRoot = get(CfgProp.SOLR_COREROOTDIRECTORY, container.getSolrHome());
-          walkFromHere(new File(coreRoot), container, new HashMap<String, String>(), new HashMap<String, String>());
-        }
-      }
-    } else {
-      coreNodes = (NodeList) evaluate("solr/cores/core",
-          XPathConstants.NODESET);
-      // Check a couple of error conditions
-      Set<String> names = new HashSet<String>(); // for duplicate names
-      Map<String, String> dirs = new HashMap<String, String>(); // for duplicate data dirs.
-
-      for (int idx = 0; idx < coreNodes.getLength(); ++idx) {
-        Node node = coreNodes.item(idx);
-        String name = DOMUtil.getAttr(node,  CoreDescriptor.CORE_NAME, null);
-        String dataDir = DOMUtil.getAttr(node,  CoreDescriptor.CORE_DATADIR, null);
-        if (name != null) {
-          if (! names.contains(name)) {
-            names.add(name);
-          } else {
-            String msg = String.format(Locale.ROOT, "More than one core defined for core named %s", name);
-            log.error(msg);
-          }
-        }
-
-        if (dataDir != null) {
-          if (! dirs.containsKey(dataDir)) {
-            dirs.put(dataDir, name);
-          } else {
-            String msg = String.format(Locale.ROOT, "More than one core points to data dir %s. They are in %s and %s",
-                dataDir, dirs.get(dataDir), name);
-            log.warn(msg);
-          }
-        }
+    if (container != null) { // TODO: 5.0. Yet another bit of nonsense only
+                             // because of the test harness.
+      synchronized (coreDescriptorPlusMap) {
+        String coreRoot = get(CfgProp.SOLR_COREROOTDIRECTORY,
+            container.getSolrHome());
+        walkFromHere(new File(coreRoot), container,
+            new HashMap<String,String>(), new HashMap<String,String>());
       }
     }
   }
 
   @Override
-  public String getBadConfigCoreMessage(String name) {
-    return badConfigCores.get(name);
-  }
-  
-  public static Document copyDoc(Document doc) throws TransformerException {
-    TransformerFactory tfactory = TransformerFactory.newInstance();
-    Transformer tx = tfactory.newTransformer();
-    DOMSource source = new DOMSource(doc);
-    DOMResult result = new DOMResult();
-    tx.transform(source, result);
-    return (Document) result.getNode();
-  }
-
-  //TODO: For 5.0, you shouldn't have to do the sbustituteProperty, this is anothe bit
-  // of awkward back-compat due to persistence.
-  @Override
-  public int getInt(CfgProp prop, int def) {
-    String val = propMap.get(prop);
-    if (val != null) val = PropertiesUtil.substituteProperty(val, null);
-    return (val == null) ? def : Integer.parseInt(val);
-  }
-
-  @Override
-  public boolean getBool(CfgProp prop, boolean defValue) {
-    String val = propMap.get(prop);
-    if (val != null) val = PropertiesUtil.substituteProperty(val, null);
-    return (val == null) ? defValue : Boolean.parseBoolean(val);
-  }
-
-  @Override
-  public String get(CfgProp prop, String def) {
-    String val = propMap.get(prop);
-    if (val != null) val = PropertiesUtil.substituteProperty(val, null);
-    return (val == null) ? def : val;
-  }
-
-  // For saving the original property, ${} syntax and all.
-  @Override
-  public String getOrigProp(CfgProp prop, String def) {
-    String val = propMap.get(prop);
-    return (val == null) ? def : val;
-  }
-
-
-  public ShardHandlerFactory initShardHandler() {
-    PluginInfo info = null;
-    Node shfn = 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 = 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;
-  }
-
-  @Override
-  public Properties getSolrProperties(String path) {
-    try {
-      return readProperties(((NodeList) evaluate(
-          path, XPathConstants.NODESET)).item(0));
-    } catch (Throwable e) {
-      SolrException.log(log, null, e);
-    }
-    return null;
-
-  }
-
-  Properties readProperties(Node node) throws XPathExpressionException {
-    XPath xpath = getXPath();
-    NodeList props = (NodeList) xpath.evaluate("property", node, XPathConstants.NODESET);
-    Properties properties = new Properties();
-    for (int i = 0; i < props.getLength(); i++) {
-      Node prop = props.item(i);
-      properties.setProperty(DOMUtil.getAttr(prop, "name"), DOMUtil.getAttr(prop, "value"));
-    }
-    return properties;
-  }
-
-  @Override
-  public Map<String, String> readCoreAttributes(String coreName) {
-    Map<String, String> attrs = new HashMap<String, String>();
-
-    if (is50OrLater) {
-      return attrs; // this is a no-op.... intentionally
-    }
-    synchronized (coreNodes) {
-      for (int idx = 0; idx < coreNodes.getLength(); ++idx) {
-        Node node = coreNodes.item(idx);
-        if (coreName.equals(DOMUtil.getAttr(node, CoreDescriptor.CORE_NAME, null))) {
-          NamedNodeMap attributes = node.getAttributes();
-          for (int i = 0; i < attributes.getLength(); i++) {
-            Node attribute = attributes.item(i);
-            String val = attribute.getNodeValue();
-            if (CoreDescriptor.CORE_DATADIR.equals(attribute.getNodeName()) ||
-                CoreDescriptor.CORE_INSTDIR.equals(attribute.getNodeName())) {
-              if (val.indexOf('$') == -1) {
-                val = (val != null && !val.endsWith("/")) ? val + '/' : val;
-              }
-            }
-            attrs.put(attribute.getNodeName(), val);
-          }
-          return attrs;
-        }
-      }
-    }
-    return attrs;
+  public Map<String,String> readCoreAttributes(String coreName) {
+    Map<String,String> attrs = new HashMap<String,String>();
+    
+    return attrs; // this is a no-op.... intentionally
   }
 
   // Basic recursive tree walking, looking for "core.properties" files. Once one is found, we'll stop going any
@@ -482,117 +236,45 @@ public class ConfigSolrXml extends Confi
     coreDescriptorPlusMap.remove(desc.getName());
   }
 
-  @Override
-  public SolrConfig getSolrConfigFromZk(ZkController zkController, String zkConfigName, String solrConfigFileName,
-                                        SolrResourceLoader resourceLoader) {
-    SolrConfig cfg = null;
-    try {
-      byte[] config = zkController.getConfigFileData(zkConfigName, solrConfigFileName);
-      InputSource is = new InputSource(new ByteArrayInputStream(config));
-      is.setSystemId(SystemIdResolver.createSystemIdFromResourceName(solrConfigFileName));
-      cfg = solrConfigFileName == null ? new SolrConfig(
-          resourceLoader, SolrConfig.DEFAULT_CONF_FILE, is) : new SolrConfig(
-          resourceLoader, solrConfigFileName, is);
-    } catch (Exception e) {
-      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
-          "getSolrConfigFromZK failed for " + zkConfigName + " " + solrConfigFileName, e);
-    }
-    return cfg;
-  }
-
-  static List<SolrXMLSerializer.SolrCoreXMLDef> solrCoreXMLDefs = new ArrayList<SolrXMLSerializer.SolrCoreXMLDef>();
-  // Do this when re-using a ConfigSolrXml.
-
-  // These two methods are part of SOLR-4196 and are awkward, should go away with 5.0
-  @Override
-  public void initPersist() {
-    initPersistStatic();
-  }
-
-  public static void initPersistStatic() {
-    solrCoreXMLDefs = new ArrayList<SolrXMLSerializer.SolrCoreXMLDef>();
-    solrXMLSerializer = new SolrXMLSerializer();
-  }
-
-  @Override
-  public void addPersistCore(String coreName, Properties attribs, Map<String, String> props) {
-    addPersistCore(attribs, props);
-  }
-
-  static void addPersistCore(Properties props, Map<String, String> attribs) {
-    SolrXMLSerializer.SolrCoreXMLDef solrCoreXMLDef = new SolrXMLSerializer.SolrCoreXMLDef();
-    solrCoreXMLDef.coreAttribs = attribs;
-    solrCoreXMLDef.coreProperties = props;
-    solrCoreXMLDefs.add(solrCoreXMLDef);
+  public IndexSchema getSchemaFromZk(ZkController zkController, String zkConfigName, String schemaName,
+                                     SolrConfig config)
+      throws KeeperException, InterruptedException {
+    byte[] configBytes = zkController.getConfigFileData(zkConfigName, schemaName);
+    InputSource is = new InputSource(new ByteArrayInputStream(configBytes));
+    is.setSystemId(SystemIdResolver.createSystemIdFromResourceName(schemaName));
+    IndexSchema schema = new IndexSchema(config, schemaName, is);
+    return schema;
   }
 
-  private static SolrXMLSerializer solrXMLSerializer = new SolrXMLSerializer();
-
   @Override
-  public void addPersistAllCores(Properties containerProperties, Map<String, String> rootSolrAttribs, Map<String, String> coresAttribs,
-                                 File file) {
-    addPersistAllCoresStatic(containerProperties, rootSolrAttribs, coresAttribs, file);
-  }
-
-  // Fortunately, we don't iterate over these too often, so the waste is probably tolerable.
-
-  @Override
-  public String getCoreNameFromOrig(String origCoreName, SolrResourceLoader loader, String coreName) {
-
-    if (is50OrLater) {
-      // first look for an exact match
-      for (Map.Entry<String, CoreDescriptorPlus> ent : coreDescriptorPlusMap.entrySet()) {
-
-        String name = ent.getValue().getCoreDescriptor().getProperty(CoreDescriptor.CORE_NAME, null);
-        if (origCoreName.equals(name)) {
-          if (coreName.equals(origCoreName)) {
-            return name;
-          }
-          return coreName;
-        }
-      }
-
-      for (Map.Entry<String, CoreDescriptorPlus> ent : coreDescriptorPlusMap.entrySet()) {
-        String name = ent.getValue().getCoreDescriptor().getProperty(CoreDescriptor.CORE_NAME, null);
-        // see if we match with substitution
-        if (origCoreName.equals(PropertiesUtil.substituteProperty(name, loader.getCoreProperties()))) {
-          if (coreName.equals(origCoreName)) {
-            return name;
-          }
-          return coreName;
-        }
-      }
-    } else {
-      // look for an existing node
-      synchronized (coreNodes) {
-        // first look for an exact match
-        Node coreNode = null;
-        for (int i = 0; i < coreNodes.getLength(); i++) {
-          Node node = coreNodes.item(i);
-
-          String name = DOMUtil.getAttr(node, CoreDescriptor.CORE_NAME, null);
-          if (origCoreName.equals(name)) {
-            if (coreName.equals(origCoreName)) {
-              return name;
-            }
-            return coreName;
-          }
-        }
-
-        if (coreNode == null) {
-          // see if we match with substitution
-          for (int i = 0; i < coreNodes.getLength(); i++) {
-            Node node = coreNodes.item(i);
-            String name = DOMUtil.getAttr(node, CoreDescriptor.CORE_NAME, null);
-            if (origCoreName.equals(PropertiesUtil.substituteProperty(name,
-                loader.getCoreProperties()))) {
-              if (coreName.equals(origCoreName)) {
-                return name;
-              }
-              return coreName;
-            }
-          }
+  public String getCoreNameFromOrig(String origCoreName,
+      SolrResourceLoader loader, String coreName) {
+    
+    // first look for an exact match
+    for (Map.Entry<String,CoreDescriptorPlus> ent : coreDescriptorPlusMap
+        .entrySet()) {
+      
+      String name = ent.getValue().getCoreDescriptor()
+          .getProperty(CoreDescriptor.CORE_NAME, null);
+      if (origCoreName.equals(name)) {
+        if (coreName.equals(origCoreName)) {
+          return name;
+        }
+        return coreName;
+      }
+    }
+    
+    for (Map.Entry<String,CoreDescriptorPlus> ent : coreDescriptorPlusMap
+        .entrySet()) {
+      String name = ent.getValue().getCoreDescriptor()
+          .getProperty(CoreDescriptor.CORE_NAME, null);
+      // see if we match with substitution
+      if (origCoreName.equals(PropertiesUtil.substituteProperty(name,
+          loader.getCoreProperties()))) {
+        if (coreName.equals(origCoreName)) {
+          return name;
         }
+        return coreName;
       }
     }
     return null;
@@ -600,66 +282,30 @@ public class ConfigSolrXml extends Confi
 
   @Override
   public List<String> getAllCoreNames() {
-    List<String> ret = new ArrayList<String>();
-    if (is50OrLater) {
-      ret = new ArrayList<String>(coreDescriptorPlusMap.keySet());
-    } else {
-      synchronized (coreNodes) {
-        for (int idx = 0; idx < coreNodes.getLength(); ++idx) {
-          Node node = coreNodes.item(idx);
-          ret.add(DOMUtil.getAttr(node, CoreDescriptor.CORE_NAME, null));
-        }
-      }
-    }
+    List<String> ret = new ArrayList<String>(coreDescriptorPlusMap.keySet());
+    
     return ret;
   }
-
+  
   @Override
   public String getProperty(String coreName, String property, String defaultVal) {
-    if (is50OrLater) {
-      CoreDescriptorPlus plus = coreDescriptorPlusMap.get(coreName);
-      if (plus == null) return defaultVal;
-      CoreDescriptor desc = plus.getCoreDescriptor();
-      if (desc == null) return defaultVal;
-      return desc.getProperty(property, defaultVal);
-    } else {
-      synchronized (coreNodes) {
-        for (int idx = 0; idx < coreNodes.getLength(); ++idx) {
-          Node node = coreNodes.item(idx);
-          if (coreName.equals(DOMUtil.getAttr(node, CoreDescriptor.CORE_NAME, null))) {
-            return DOMUtil.getAttr(node, property, defaultVal);
-          }
-        }
-      }
-      return defaultVal;
-    }
+    
+    CoreDescriptorPlus plus = coreDescriptorPlusMap.get(coreName);
+    if (plus == null) return defaultVal;
+    CoreDescriptor desc = plus.getCoreDescriptor();
+    if (desc == null) return defaultVal;
+    return desc.getProperty(property, defaultVal);
+    
   }
 
   @Override
   public Properties readCoreProperties(String coreName) {
-    if (is50OrLater) {
-      CoreDescriptorPlus plus = coreDescriptorPlusMap.get(coreName);
-      if (plus == null) return null;
-      return new Properties(plus.getCoreDescriptor().getCoreProperties());
-    } else {
-      synchronized (coreNodes) {
-        for (int idx = 0; idx < coreNodes.getLength(); ++idx) {
-          Node node = coreNodes.item(idx);
-          if (coreName.equals(DOMUtil.getAttr(node, CoreDescriptor.CORE_NAME, null))) {
-            try {
-              return readProperties(node);
-            } catch (XPathExpressionException e) {
-              return null;
-            }
-          }
-        }
-      }
-    }
-    return null;
-  }
+    
+    CoreDescriptorPlus plus = coreDescriptorPlusMap.get(coreName);
+    if (plus == null) return null;
+    return new Properties(plus.getCoreDescriptor().getCoreProperties());
 
-  @Override
-  public boolean is50OrLater() { return is50OrLater; }
+  }
 
   static Properties getCoreProperties(String instanceDir, CoreDescriptor dcore) {
     String file = dcore.getPropertiesName();
@@ -684,59 +330,10 @@ public class ConfigSolrXml extends Confi
     return p;
   }
 
-
-  static void addPersistAllCoresStatic(Properties containerProperties, Map<String, String> rootSolrAttribs, Map<String, String> coresAttribs,
-                                       File file) {
-    SolrXMLSerializer.SolrXMLDef solrXMLDef = new SolrXMLSerializer.SolrXMLDef();
-    solrXMLDef.coresDefs = solrCoreXMLDefs;
-    solrXMLDef.containerProperties = containerProperties;
-    solrXMLDef.solrAttribs = rootSolrAttribs;
-    solrXMLDef.coresAttribs = coresAttribs;
-    solrXMLSerializer.persistFile(file, solrXMLDef);
-
-  }
-
-  static final String DEF_SOLR_XML = "<?xml version=\"1.0\" encoding=\"UTF-8\" ?>\n"
-      + "<solr persistent=\"false\">\n"
-      + "  <cores adminPath=\"/admin/cores\" defaultCoreName=\""
-      + CoreContainer.DEFAULT_DEFAULT_CORE_NAME
-      + "\""
-      + " host=\"${host:}\" hostPort=\"${hostPort:}\" hostContext=\"${hostContext:}\" zkClientTimeout=\"${zkClientTimeout:15000}\""
-      + ">\n"
-      + "    <core name=\""
-      + CoreContainer.DEFAULT_DEFAULT_CORE_NAME
-      + "\" shard=\"${shard:}\" collection=\"${collection:}\" instanceDir=\"collection1\" />\n"
-      + "  </cores>\n" + "</solr>";
-
-}
-
-// It's mightily convenient to have all of the original path names and property values when persisting cores, so
-// this little convenience class is just for that.
-// Also, let's keep track of anything we added here, especially the instance dir for persistence purposes. We don't
-// want, for instance, to persist instanceDir if it was not specified originally.
-//
-// I suspect that for persistence purposes, we may want to expand this idea to record, say, ${blah}
-class CoreDescriptorPlus {
-  private CoreDescriptor coreDescriptor;
-  private String filePath;
-  private Properties propsOrig; // TODO: 5.0. Remove this since it's only really used for persisting.
-
-  CoreDescriptorPlus(String filePath, CoreDescriptor descriptor, Properties propsOrig) {
-    coreDescriptor = descriptor;
-    this.filePath = filePath;
-    this.propsOrig = propsOrig;
-  }
-
-  CoreDescriptor getCoreDescriptor() {
-    return coreDescriptor;
-  }
-
-  String getFilePath() {
-    return filePath;
+  @Override
+  public void substituteProperties() {
+    config.substituteProperties();
   }
 
-  Properties getPropsOrig() {
-    return propsOrig;
-  }
 }
 

Added: lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/ConfigSolrXmlOld.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/ConfigSolrXmlOld.java?rev=1470674&view=auto
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/ConfigSolrXmlOld.java (added)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/ConfigSolrXmlOld.java Mon Apr 22 19:35:10 2013
@@ -0,0 +1,358 @@
+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.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+
+import javax.xml.parsers.ParserConfigurationException;
+import javax.xml.xpath.XPathConstants;
+import javax.xml.xpath.XPathExpressionException;
+
+import org.apache.commons.io.IOUtils;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.util.DOMUtil;
+import org.apache.solr.util.PropertiesUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.w3c.dom.NamedNodeMap;
+import org.w3c.dom.Node;
+import org.w3c.dom.NodeList;
+import org.xml.sax.SAXException;
+
+
+/**
+ *
+ */
+public class ConfigSolrXmlOld extends ConfigSolr {
+  protected static Logger log = LoggerFactory.getLogger(ConfigSolrXmlOld.class);
+
+  private NodeList coreNodes = null;
+
+  public ConfigSolrXmlOld(Config config, CoreContainer container)
+      throws ParserConfigurationException, IOException, SAXException {
+
+    super(config);
+    init(container);
+  }
+  
+  private void init(CoreContainer container) throws IOException {
+    
+    // Do sanity checks - we don't want to find new style
+    // config
+    failIfFound("solr/str[@name='adminHandler']");
+    failIfFound("solr/int[@name='coreLoadThreads']");
+    failIfFound("solr/str[@name='coreRootDirectory']");
+    failIfFound("solr/solrcloud/int[@name='distribUpdateConnTimeout']");
+    failIfFound("solr/solrcloud/int[@name='distribUpdateSoTimeout']");
+    failIfFound("solr/solrcloud/str[@name='host']");
+    failIfFound("solr/solrcloud/str[@name='hostContext']");
+    failIfFound("solr/solrcloud/int[@name='hostPort']");
+    failIfFound("solr/solrcloud/int[@name='leaderVoteWait']");
+    failIfFound("solr/str[@name='managementPath']");
+    failIfFound("solr/str[@name='sharedLib']");
+    failIfFound("solr/str[@name='shareSchema']");
+    failIfFound("solr/int[@name='transientCacheSize']");
+    failIfFound("solr/solrcloud/int[@name='zkClientTimeout']");
+    failIfFound("solr/solrcloud/int[@name='zkHost']");
+    
+    failIfFound("solr/logging/str[@name='class']");
+    failIfFound("solr/logging/str[@name='enabled']");
+    
+    failIfFound("solr/logging/watcher/int[@name='size']");
+    failIfFound("solr/logging/watcher/int[@name='threshold']");
+    
+    fillPropMap();
+    initCoreList(container);
+  }
+  
+  private void failIfFound(String xPath) {
+
+    if (config.getVal(xPath, false) != null) {
+      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Should not have found " + xPath +
+          " solr.xml may be a mix of old and new style formats.");
+    }
+  }
+  
+  private void fillPropMap() {
+    
+    propMap.put(CfgProp.SOLR_CORELOADTHREADS,
+        config.getVal("solr/@coreLoadThreads", false));
+    propMap
+        .put(CfgProp.SOLR_SHAREDLIB, config.getVal("solr/@sharedLib", false));
+    propMap.put(CfgProp.SOLR_ZKHOST, config.getVal("solr/@zkHost", false));
+    
+    propMap.put(CfgProp.SOLR_LOGGING_CLASS,
+        config.getVal("solr/logging/@class", false));
+    propMap.put(CfgProp.SOLR_LOGGING_ENABLED,
+        config.getVal("solr/logging/@enabled", false));
+    propMap.put(CfgProp.SOLR_LOGGING_WATCHER_SIZE,
+        config.getVal("solr/logging/watcher/@size", false));
+    propMap.put(CfgProp.SOLR_LOGGING_WATCHER_THRESHOLD,
+        config.getVal("solr/logging/watcher/@threshold", false));
+    
+    propMap.put(CfgProp.SOLR_ADMINHANDLER,
+        config.getVal("solr/cores/@adminHandler", false));
+    propMap.put(CfgProp.SOLR_DISTRIBUPDATECONNTIMEOUT,
+        config.getVal("solr/cores/@distribUpdateConnTimeout", false));
+    propMap.put(CfgProp.SOLR_DISTRIBUPDATESOTIMEOUT,
+        config.getVal("solr/cores/@distribUpdateSoTimeout", false));
+    propMap.put(CfgProp.SOLR_HOST, config.getVal("solr/cores/@host", false));
+    propMap.put(CfgProp.SOLR_HOSTCONTEXT,
+        config.getVal("solr/cores/@hostContext", false));
+    propMap.put(CfgProp.SOLR_HOSTPORT,
+        config.getVal("solr/cores/@hostPort", false));
+    propMap.put(CfgProp.SOLR_LEADERVOTEWAIT,
+        config.getVal("solr/cores/@leaderVoteWait", false));
+    propMap.put(CfgProp.SOLR_MANAGEMENTPATH,
+        config.getVal("solr/cores/@managementPath", false));
+    propMap.put(CfgProp.SOLR_SHARESCHEMA,
+        config.getVal("solr/cores/@shareSchema", false));
+    propMap.put(CfgProp.SOLR_TRANSIENTCACHESIZE,
+        config.getVal("solr/cores/@transientCacheSize", false));
+    propMap.put(CfgProp.SOLR_ZKCLIENTTIMEOUT,
+        config.getVal("solr/cores/@zkClientTimeout", false));
+    propMap.put(CfgProp.SOLR_SHARDHANDLERFACTORY_CLASS,
+        config.getVal("solr/shardHandlerFactory/@class", false));
+    propMap.put(CfgProp.SOLR_SHARDHANDLERFACTORY_NAME,
+        config.getVal("solr/shardHandlerFactory/@name", false));
+    propMap.put(CfgProp.SOLR_SHARDHANDLERFACTORY_CONNTIMEOUT,
+        config.getVal("solr/shardHandlerFactory/int[@connTimeout]", false));
+    propMap.put(CfgProp.SOLR_SHARDHANDLERFACTORY_SOCKETTIMEOUT,
+        config.getVal("solr/shardHandlerFactory/int[@socketTimeout]", false));
+    
+    // These have no counterpart in 5.0, asking, for any of these in Solr 5.0
+    // will result in an error being
+    // thrown.
+    propMap.put(CfgProp.SOLR_CORES_DEFAULT_CORE_NAME,
+        config.getVal("solr/cores/@defaultCoreName", false));
+    propMap.put(CfgProp.SOLR_PERSISTENT,
+        config.getVal("solr/@persistent", false));
+    propMap.put(CfgProp.SOLR_ADMINPATH,
+        config.getVal("solr/cores/@adminPath", false));
+    
+  }
+
+  private void initCoreList(CoreContainer container) throws IOException {
+    
+    coreNodes = (NodeList) config.evaluate("solr/cores/core",
+        XPathConstants.NODESET);
+    // Check a couple of error conditions
+    Set<String> names = new HashSet<String>(); // for duplicate names
+    Map<String,String> dirs = new HashMap<String,String>(); // for duplicate
+                                                            // data dirs.
+    
+    for (int idx = 0; idx < coreNodes.getLength(); ++idx) {
+      Node node = coreNodes.item(idx);
+      String name = DOMUtil.getAttr(node, CoreDescriptor.CORE_NAME, null);
+      String dataDir = DOMUtil.getAttr(node, CoreDescriptor.CORE_DATADIR, null);
+      if (name != null) {
+        if (!names.contains(name)) {
+          names.add(name);
+        } else {
+          String msg = String.format(Locale.ROOT,
+              "More than one core defined for core named %s", name);
+          log.error(msg);
+        }
+      }
+      
+      if (dataDir != null) {
+        if (!dirs.containsKey(dataDir)) {
+          dirs.put(dataDir, name);
+        } else {
+          String msg = String
+              .format(
+                  Locale.ROOT,
+                  "More than one core points to data dir %s. They are in %s and %s",
+                  dataDir, dirs.get(dataDir), name);
+          log.warn(msg);
+        }
+      }
+    }
+    
+  }
+
+  @Override
+  public Map<String, String> readCoreAttributes(String coreName) {
+    Map<String, String> attrs = new HashMap<String, String>();
+
+    synchronized (coreNodes) {
+      for (int idx = 0; idx < coreNodes.getLength(); ++idx) {
+        Node node = coreNodes.item(idx);
+        if (coreName.equals(DOMUtil.getAttr(node, CoreDescriptor.CORE_NAME, null))) {
+          NamedNodeMap attributes = node.getAttributes();
+          for (int i = 0; i < attributes.getLength(); i++) {
+            Node attribute = attributes.item(i);
+            String val = attribute.getNodeValue();
+            if (CoreDescriptor.CORE_DATADIR.equals(attribute.getNodeName()) ||
+                CoreDescriptor.CORE_INSTDIR.equals(attribute.getNodeName())) {
+              if (val.indexOf('$') == -1) {
+                val = (val != null && !val.endsWith("/")) ? val + '/' : val;
+              }
+            }
+            attrs.put(attribute.getNodeName(), val);
+          }
+          return attrs;
+        }
+      }
+    }
+    return attrs;
+  }
+
+  // Fortunately, we don't iterate over these too often, so the waste is probably tolerable.
+
+  @Override
+  public String getCoreNameFromOrig(String origCoreName,
+      SolrResourceLoader loader, String coreName) {
+    
+    // look for an existing node
+    synchronized (coreNodes) {
+      // first look for an exact match
+      Node coreNode = null;
+      for (int i = 0; i < coreNodes.getLength(); i++) {
+        Node node = coreNodes.item(i);
+        
+        String name = DOMUtil.getAttr(node, CoreDescriptor.CORE_NAME, null);
+        if (origCoreName.equals(name)) {
+          if (coreName.equals(origCoreName)) {
+            return name;
+          }
+          return coreName;
+        }
+      }
+      
+      if (coreNode == null) {
+        // see if we match with substitution
+        for (int i = 0; i < coreNodes.getLength(); i++) {
+          Node node = coreNodes.item(i);
+          String name = DOMUtil.getAttr(node, CoreDescriptor.CORE_NAME, null);
+          if (origCoreName.equals(PropertiesUtil.substituteProperty(name,
+              loader.getCoreProperties()))) {
+            if (coreName.equals(origCoreName)) {
+              return name;
+            }
+            return coreName;
+          }
+        }
+      }
+    }
+    
+    return null;
+  }
+
+  @Override
+  public List<String> getAllCoreNames() {
+    List<String> ret = new ArrayList<String>();
+    
+    synchronized (coreNodes) {
+      for (int idx = 0; idx < coreNodes.getLength(); ++idx) {
+        Node node = coreNodes.item(idx);
+        ret.add(DOMUtil.getAttr(node, CoreDescriptor.CORE_NAME, null));
+      }
+    }
+    
+    return ret;
+  }
+
+  @Override
+  public String getProperty(String coreName, String property, String defaultVal) {
+    
+    synchronized (coreNodes) {
+      for (int idx = 0; idx < coreNodes.getLength(); ++idx) {
+        Node node = coreNodes.item(idx);
+        if (coreName.equals(DOMUtil.getAttr(node, CoreDescriptor.CORE_NAME,
+            null))) {
+          return DOMUtil.getAttr(node, property, defaultVal);
+        }
+      }
+    }
+    return defaultVal;
+    
+  }
+
+  @Override
+  public Properties readCoreProperties(String coreName) {
+    
+    synchronized (coreNodes) {
+      for (int idx = 0; idx < coreNodes.getLength(); ++idx) {
+        Node node = coreNodes.item(idx);
+        if (coreName.equals(DOMUtil.getAttr(node, CoreDescriptor.CORE_NAME,
+            null))) {
+          try {
+            return readProperties(node);
+          } catch (XPathExpressionException e) {
+            return null;
+          }
+        }
+      }
+    }
+    
+    return null;
+  }
+
+  static Properties getCoreProperties(String instanceDir, CoreDescriptor dcore) {
+    String file = dcore.getPropertiesName();
+    if (file == null) file = "conf" + File.separator + "solrcore.properties";
+    File corePropsFile = new File(file);
+    if (!corePropsFile.isAbsolute()) {
+      corePropsFile = new File(instanceDir, file);
+    }
+    Properties p = dcore.getCoreProperties();
+    if (corePropsFile.exists() && corePropsFile.isFile()) {
+      p = new Properties(dcore.getCoreProperties());
+      InputStream is = null;
+      try {
+        is = new FileInputStream(corePropsFile);
+        p.load(is);
+      } catch (IOException e) {
+        log.warn("Error loading properties ", e);
+      } finally {
+        IOUtils.closeQuietly(is);
+      }
+    }
+    return p;
+  }
+
+  public static final String DEF_SOLR_XML = "<?xml version=\"1.0\" encoding=\"UTF-8\" ?>\n"
+      + "<solr persistent=\"false\">\n"
+      + "  <cores adminPath=\"/admin/cores\" defaultCoreName=\""
+      + CoreContainer.DEFAULT_DEFAULT_CORE_NAME
+      + "\""
+      + " host=\"${host:}\" hostPort=\"${hostPort:}\" hostContext=\"${hostContext:}\" zkClientTimeout=\"${zkClientTimeout:15000}\""
+      + ">\n"
+      + "    <core name=\""
+      + CoreContainer.DEFAULT_DEFAULT_CORE_NAME
+      + "\" shard=\"${shard:}\" collection=\"${collection:}\" instanceDir=\"collection1\" />\n"
+      + "  </cores>\n" + "</solr>";
+
+  @Override
+  public void substituteProperties() {
+    config.substituteProperties();
+  }
+
+}