You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ro...@apache.org on 2013/07/11 18:38:54 UTC

svn commit: r1502280 [1/3] - in /lucene/dev/branches/branch_4x: ./ solr/ solr/contrib/ solr/contrib/dataimporthandler/src/test-files/ solr/core/ solr/core/src/java/org/apache/solr/cloud/ solr/core/src/java/org/apache/solr/core/ solr/core/src/java/org/a...

Author: romseygeek
Date: Thu Jul 11 16:38:53 2013
New Revision: 1502280

URL: http://svn.apache.org/r1502280
Log:
SOLR-4914: Factor out core discovery and persistence logic

Added:
    lucene/dev/branches/branch_4x/solr/contrib/dataimporthandler/src/test-files/log4j.properties
      - copied unchanged from r1502276, lucene/dev/trunk/solr/contrib/dataimporthandler/src/test-files/log4j.properties
    lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/core/CorePropertiesLocator.java
      - copied unchanged from r1502276, lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/CorePropertiesLocator.java
    lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/core/CoresLocator.java
      - copied unchanged from r1502276, lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/CoresLocator.java
    lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/core/SolrXMLCoresLocator.java
      - copied unchanged from r1502276, lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/SolrXMLCoresLocator.java
    lucene/dev/branches/branch_4x/solr/core/src/test/org/apache/solr/core/TestSolrXmlPersistor.java
      - copied unchanged from r1502276, lucene/dev/trunk/solr/core/src/test/org/apache/solr/core/TestSolrXmlPersistor.java
Removed:
    lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/core/SolrCoreDiscoverer.java
Modified:
    lucene/dev/branches/branch_4x/   (props changed)
    lucene/dev/branches/branch_4x/solr/   (props changed)
    lucene/dev/branches/branch_4x/solr/CHANGES.txt   (contents, props changed)
    lucene/dev/branches/branch_4x/solr/contrib/   (props changed)
    lucene/dev/branches/branch_4x/solr/core/   (props changed)
    lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/cloud/CloudDescriptor.java
    lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/cloud/ZkCLI.java
    lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/cloud/ZkController.java
    lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/core/ConfigSolr.java
    lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/core/ConfigSolrXml.java
    lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/core/ConfigSolrXmlOld.java
    lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/core/CoreContainer.java
    lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/core/CoreDescriptor.java
    lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/core/SolrCore.java
    lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/core/SolrCores.java
    lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/core/ZkContainer.java
    lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminHandler.java
    lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/util/PropertiesUtil.java
    lucene/dev/branches/branch_4x/solr/core/src/test/org/apache/solr/cloud/ClusterStateUpdateTest.java
    lucene/dev/branches/branch_4x/solr/core/src/test/org/apache/solr/core/TestCoreContainer.java
    lucene/dev/branches/branch_4x/solr/core/src/test/org/apache/solr/core/TestCoreDiscovery.java
    lucene/dev/branches/branch_4x/solr/core/src/test/org/apache/solr/core/TestLazyCores.java
    lucene/dev/branches/branch_4x/solr/core/src/test/org/apache/solr/core/TestSolrXml.java
    lucene/dev/branches/branch_4x/solr/core/src/test/org/apache/solr/core/TestSolrXmlPersistence.java
    lucene/dev/branches/branch_4x/solr/core/src/test/org/apache/solr/handler/admin/CoreAdminCreateDiscoverTest.java
    lucene/dev/branches/branch_4x/solr/core/src/test/org/apache/solr/handler/admin/CoreAdminHandlerTest.java
    lucene/dev/branches/branch_4x/solr/core/src/test/org/apache/solr/update/SolrIndexSplitterTest.java
    lucene/dev/branches/branch_4x/solr/solrj/   (props changed)
    lucene/dev/branches/branch_4x/solr/solrj/src/java/org/apache/solr/client/solrj/request/CoreAdminRequest.java
    lucene/dev/branches/branch_4x/solr/solrj/src/java/org/apache/solr/common/params/CoreAdminParams.java
    lucene/dev/branches/branch_4x/solr/solrj/src/test-files/solrj/solr/shared/solr.xml
    lucene/dev/branches/branch_4x/solr/solrj/src/test/org/apache/solr/client/solrj/embedded/AbstractEmbeddedSolrServerTestCase.java
    lucene/dev/branches/branch_4x/solr/solrj/src/test/org/apache/solr/client/solrj/embedded/TestSolrProperties.java
    lucene/dev/branches/branch_4x/solr/test-framework/   (props changed)
    lucene/dev/branches/branch_4x/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java
    lucene/dev/branches/branch_4x/solr/test-framework/src/java/org/apache/solr/util/TestHarness.java

Modified: lucene/dev/branches/branch_4x/solr/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/solr/CHANGES.txt?rev=1502280&r1=1502279&r2=1502280&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/solr/CHANGES.txt (original)
+++ lucene/dev/branches/branch_4x/solr/CHANGES.txt Thu Jul 11 16:38:53 2013
@@ -36,6 +36,11 @@ Upgrading from Solr 4.4.0
 Detailed Change List
 ----------------------
 
+Other Changes
+----------------------
+
+* SOLR-4914: Factor out core list persistence and discovery into a
+  new CoresLocator interface. (Alan Woodward)
 
 ==================  4.4.0 ==================
 

Modified: lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/cloud/CloudDescriptor.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/cloud/CloudDescriptor.java?rev=1502280&r1=1502279&r2=1502280&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/cloud/CloudDescriptor.java (original)
+++ lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/cloud/CloudDescriptor.java Thu Jul 11 16:38:53 2013
@@ -17,11 +17,16 @@ package org.apache.solr.cloud;
  * limitations under the License.
  */
 
-import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.cloud.Slice;
+import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.params.SolrParams;
+import org.apache.solr.core.CoreDescriptor;
+import org.apache.solr.util.PropertiesUtil;
+
+import java.util.Properties;
 
 public class CloudDescriptor {
+
   private String shardId;
   private String collectionName;
   private SolrParams params;
@@ -36,6 +41,21 @@ public class CloudDescriptor {
 
   volatile boolean isLeader = false;
   volatile String lastPublished = ZkStateReader.ACTIVE;
+
+  public static final String SHARD_STATE = "shardState";
+  public static final String NUM_SHARDS = "numShards";
+  public static final String SHARD_RANGE = "shardRange";
+
+  public CloudDescriptor(String coreName, Properties props) {
+    this.shardId = props.getProperty(CoreDescriptor.CORE_SHARD, null);
+    // If no collection name is specified, we default to the core name
+    this.collectionName = props.getProperty(CoreDescriptor.CORE_COLLECTION, coreName);
+    this.roles = props.getProperty(CoreDescriptor.CORE_ROLES, null);
+    this.nodeName = props.getProperty(CoreDescriptor.CORE_NODE_NAME);
+    this.shardState = props.getProperty(CloudDescriptor.SHARD_STATE, Slice.ACTIVE);
+    this.numShards = PropertiesUtil.toInteger(props.getProperty(CloudDescriptor.NUM_SHARDS), null);
+    this.shardRange = props.getProperty(CloudDescriptor.SHARD_RANGE, null);
+  }
   
   public String getLastPublished() {
     return lastPublished;

Modified: lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/cloud/ZkCLI.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/cloud/ZkCLI.java?rev=1502280&r1=1502279&r2=1502280&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/cloud/ZkCLI.java (original)
+++ lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/cloud/ZkCLI.java Thu Jul 11 16:38:53 2013
@@ -10,13 +10,11 @@ import org.apache.commons.cli.ParseExcep
 import org.apache.commons.cli.PosixParser;
 import org.apache.solr.common.cloud.OnReconnect;
 import org.apache.solr.common.cloud.SolrZkClient;
-import org.apache.solr.core.ConfigSolr;
-import org.apache.solr.core.SolrResourceLoader;
+import org.apache.solr.core.CoreContainer;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.ZooDefs;
 import org.apache.zookeeper.data.ACL;
-import org.xml.sax.InputSource;
 import org.xml.sax.SAXException;
 
 import javax.xml.parsers.ParserConfigurationException;
@@ -173,17 +171,18 @@ public class ZkCLI {
                 + " is required for " + BOOTSTRAP);
             System.exit(1);
           }
-          SolrResourceLoader loader = new SolrResourceLoader(solrHome);
-          solrHome = loader.getInstanceDir();
 
-          ConfigSolr cfg = ConfigSolr.fromSolrHome(loader, solrHome);
+          CoreContainer cc = new CoreContainer(solrHome);
 
           if(!ZkController.checkChrootPath(zkServerAddress, true)) {
             System.out.println("A chroot was specified in zkHost but the znode doesn't exist. ");
             System.exit(1);
           }
 
-          ZkController.bootstrapConf(zkClient, cfg, solrHome);
+          ZkController.bootstrapConf(zkClient, cc, solrHome);
+
+          // No need to shutdown the CoreContainer, as it wasn't started
+          // up in the first place...
           
         } else if (line.getOptionValue(CMD).equals(UPCONFIG)) {
           if (!line.hasOption(CONFDIR) || !line.hasOption(CONFNAME)) {

Modified: lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/cloud/ZkController.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/cloud/ZkController.java?rev=1502280&r1=1502279&r2=1502280&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/cloud/ZkController.java (original)
+++ lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/cloud/ZkController.java Thu Jul 11 16:38:53 2013
@@ -18,6 +18,7 @@ package org.apache.solr.cloud;
  */
 
 import org.apache.commons.io.FileUtils;
+import org.apache.commons.lang.StringUtils;
 import org.apache.solr.client.solrj.impl.HttpSolrServer;
 import org.apache.solr.client.solrj.request.CoreAdminRequest.WaitForState;
 import org.apache.solr.common.SolrException;
@@ -36,14 +37,12 @@ import org.apache.solr.common.cloud.ZkNo
 import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.cloud.ZooKeeperException;
 import org.apache.solr.common.params.SolrParams;
-import org.apache.solr.core.ConfigSolr;
 import org.apache.solr.core.CoreContainer;
 import org.apache.solr.core.CoreDescriptor;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.handler.component.ShardHandler;
 import org.apache.solr.update.UpdateLog;
 import org.apache.solr.update.UpdateShardHandler;
-import org.apache.solr.util.PropertiesUtil;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.KeeperException.NoNodeException;
@@ -1493,27 +1492,22 @@ public final class ZkController {
   /**
    * If in SolrCloud mode, upload config sets for each SolrCore in solr.xml.
    */
-  public static void bootstrapConf(SolrZkClient zkClient, ConfigSolr cfg, String solrHome) throws IOException,
+  public static void bootstrapConf(SolrZkClient zkClient, CoreContainer cc, String solrHome) throws IOException,
       KeeperException, InterruptedException {
 
-    List<String> allCoreNames = cfg.getAllCoreNames();
+    //List<String> allCoreNames = cfg.getAllCoreNames();
+    List<CoreDescriptor> cds = cc.getCoresLocator().discover(cc);
     
-    log.info("bootstraping config for " + allCoreNames.size() + " cores into ZooKeeper using solr.xml from " + solrHome);
+    log.info("bootstrapping config for " + cds.size() + " cores into ZooKeeper using solr.xml from " + solrHome);
 
-    for (String coreName : allCoreNames) {
-      String rawName = PropertiesUtil.substituteProperty(cfg.getProperty(coreName, "name", null), new Properties());
-      String instanceDir = cfg.getProperty(coreName, "instanceDir", null);
-      File idir = new File(instanceDir);
-      System.out.println("idir:" + idir);
-      if (!idir.isAbsolute()) {
-        idir = new File(solrHome, instanceDir);
-      }
-      String confName = PropertiesUtil.substituteProperty(cfg.getProperty(coreName, "collection", null), new Properties());
-      if (confName == null) {
-        confName = rawName;
-      }
-      File udir = new File(idir, "conf");
-      log.info("Uploading directory " + udir + " with name " + confName + " for SolrCore " + rawName);
+    for (CoreDescriptor cd : cds) {
+      String coreName = cd.getName();
+      String confName = cd.getCollectionName();
+      if (StringUtils.isEmpty(confName))
+        confName = coreName;
+      String instanceDir = cd.getInstanceDir();
+      File udir = new File(instanceDir, "conf");
+      log.info("Uploading directory " + udir + " with name " + confName + " for SolrCore " + coreName);
       ZkController.uploadConfigDir(zkClient, udir, confName);
     }
   }

Modified: lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/core/ConfigSolr.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/core/ConfigSolr.java?rev=1502280&r1=1502279&r2=1502280&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/core/ConfigSolr.java (original)
+++ lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/core/ConfigSolr.java Thu Jul 11 16:38:53 2013
@@ -18,6 +18,7 @@ package org.apache.solr.core;
  */
 
 import com.google.common.base.Charsets;
+import com.google.common.io.ByteStreams;
 import org.apache.commons.io.IOUtils;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.util.DOMUtil;
@@ -32,11 +33,11 @@ import javax.xml.xpath.XPath;
 import javax.xml.xpath.XPathConstants;
 import javax.xml.xpath.XPathExpressionException;
 import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
 import java.io.File;
 import java.io.FileInputStream;
 import java.io.InputStream;
 import java.util.HashMap;
-import java.util.List;
 import java.util.Map;
 import java.util.Properties;
 
@@ -59,7 +60,10 @@ public abstract class ConfigSolr {
       else {
         inputStream = new FileInputStream(configFile);
       }
-      return fromInputStream(loader, inputStream);
+      ByteArrayOutputStream baos = new ByteArrayOutputStream();
+      ByteStreams.copy(inputStream, baos);
+      String originalXml = IOUtils.toString(new ByteArrayInputStream(baos.toByteArray()), "UTF-8");
+      return fromInputStream(loader, new ByteArrayInputStream(baos.toByteArray()), configFile, originalXml);
     }
     catch (Exception e) {
       throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
@@ -70,15 +74,14 @@ public abstract class ConfigSolr {
     }
   }
 
-  public static ConfigSolr fromString(SolrResourceLoader loader, String xml) {
-    return fromInputStream(loader, new ByteArrayInputStream(xml.getBytes(Charsets.UTF_8)));
+  public static ConfigSolr fromString(String xml) {
+    return fromInputStream(null, new ByteArrayInputStream(xml.getBytes(Charsets.UTF_8)), null, xml);
   }
 
-  public static ConfigSolr fromInputStream(SolrResourceLoader loader, InputStream is) {
+  public static ConfigSolr fromInputStream(SolrResourceLoader loader, InputStream is, File file, String originalXml) {
     try {
       Config config = new Config(loader, null, new InputSource(is), null, false);
-      //config.substituteProperties();
-      return fromConfig(config);
+      return fromConfig(config, file, originalXml);
     }
     catch (Exception e) {
       throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
@@ -89,11 +92,13 @@ public abstract class ConfigSolr {
     return fromFile(loader, new File(solrHome, SOLR_XML_FILE));
   }
 
-  public static ConfigSolr fromConfig(Config config) {
+  public static ConfigSolr fromConfig(Config config, File file, String originalXml) {
     boolean oldStyle = (config.getNode("solr/cores", false) != null);
-    return oldStyle ? new ConfigSolrXmlOld(config)
-                    : new ConfigSolrXml(config, null);
+    return oldStyle ? new ConfigSolrXmlOld(config, file, originalXml)
+                    : new ConfigSolrXml(config);
   }
+  
+  public abstract CoresLocator getCoresLocator();
 
 
   public PluginInfo getShardHandlerFactoryPluginInfo() {
@@ -171,12 +176,6 @@ public abstract class ConfigSolr {
     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(
@@ -200,15 +199,5 @@ public abstract class ConfigSolr {
     return properties;
   }
 
-  public abstract void substituteProperties();
-
-  public abstract List<String> getAllCoreNames();
-
-  public abstract String getProperty(String coreName, String property, String defaultVal);
-
-  public abstract Properties readCoreProperties(String coreName);
-
-  public abstract Map<String, String> readCoreAttributes(String coreName);
-
 }
 

Modified: lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/core/ConfigSolrXml.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/core/ConfigSolrXml.java?rev=1502280&r1=1502279&r2=1502280&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/core/ConfigSolrXml.java (original)
+++ lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/core/ConfigSolrXml.java Thu Jul 11 16:38:53 2013
@@ -17,40 +17,32 @@ package org.apache.solr.core;
  * limitations under the License.
  */
 
-import org.apache.commons.io.IOUtils;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.util.PropertiesUtil;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-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.List;
-import java.util.Map;
-import java.util.Properties;
 
 
 /**
  *
  */
 public class ConfigSolrXml extends ConfigSolr {
+
   protected static Logger log = LoggerFactory.getLogger(ConfigSolrXml.class);
 
-  private SolrCoreDiscoverer solrCoreDiscoverer = new SolrCoreDiscoverer();
-  private final Map<String, CoreDescriptor> coreDescriptorMap;
+  private final CoresLocator coresLocator;
 
-  public ConfigSolrXml(Config config, CoreContainer container) {
+  public ConfigSolrXml(Config config) {
     super(config);
     try {
       checkForIllegalConfig();
       fillPropMap();
       config.substituteProperties();
-      String coreRoot = get(CfgProp.SOLR_COREROOTDIRECTORY, (container == null ? config.getResourceLoader().getInstanceDir() : container.getSolrHome()));
-      coreDescriptorMap = solrCoreDiscoverer.discover(container, new File(coreRoot));
+      log.info("Config-defined core root directory: {}", get(CfgProp.SOLR_COREROOTDIRECTORY, ""));
+      String coreRoot = get(CfgProp.SOLR_COREROOTDIRECTORY, config.getResourceLoader().getInstanceDir());
+      coresLocator = new CorePropertiesLocator(coreRoot);
     }
     catch (IOException e) {
       throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
@@ -133,68 +125,14 @@ public class ConfigSolrXml extends Confi
     propMap.put(CfgProp.SOLR_LOGGING_WATCHER_THRESHOLD, doSub("solr/logging/watcher/int[@name='threshold']"));
   }
 
-
-
-  @Override
-  public Map<String,String> readCoreAttributes(String coreName) {
-    Map<String,String> attrs = new HashMap<String,String>();
-    
-    return attrs; // this is a no-op.... intentionally
-  }
-
-  @Override
-  public List<String> getAllCoreNames() {
-    List<String> ret = new ArrayList<String>(coreDescriptorMap.keySet());
-    
-    return ret;
-  }
-  
-  @Override
-  public String getProperty(String coreName, String property, String defaultVal) {
-    CoreDescriptor cd = coreDescriptorMap.get(coreName);
-    if (cd == null) return defaultVal;
-
-    return cd.getProperty(property, defaultVal);
-  }
-
-  @Override
-  public Properties readCoreProperties(String coreName) {
-    CoreDescriptor cd = coreDescriptorMap.get(coreName);
-    if (cd == null) return null;
-    return new Properties(cd.getCoreProperties());
-  }
-
-  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;
-  }
-
   @Override
   protected String getShardHandlerFactoryConfigPath() {
     return "solr/shardHandlerFactory";
   }
 
   @Override
-  public void substituteProperties() {
-    config.substituteProperties();
+  public CoresLocator getCoresLocator() {
+    return coresLocator;
   }
 
 }

Modified: lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/core/ConfigSolrXmlOld.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/core/ConfigSolrXmlOld.java?rev=1502280&r1=1502279&r2=1502280&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/core/ConfigSolrXmlOld.java (original)
+++ lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/core/ConfigSolrXmlOld.java Thu Jul 11 16:38:53 2013
@@ -22,7 +22,6 @@ 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;
 
@@ -44,27 +43,37 @@ import java.util.Set;
  *
  */
 public class ConfigSolrXmlOld extends ConfigSolr {
+
   protected static Logger log = LoggerFactory.getLogger(ConfigSolrXmlOld.class);
 
   private NodeList coreNodes = null;
+  
+  private final CoresLocator persistor;
 
   @Override
   protected String getShardHandlerFactoryConfigPath() {
     return "solr/cores/shardHandlerFactory";
   }
 
-  public ConfigSolrXmlOld(Config config) {
+  public ConfigSolrXmlOld(Config config, File configFile, String originalXML) {
     super(config);
     try {
       checkForIllegalConfig();
       fillPropMap();
       config.substituteProperties();
       initCoreList();
+      this.persistor = isPersistent() ? new SolrXMLCoresLocator(configFile, originalXML, this)
+                                      : new SolrXMLCoresLocator.NonPersistingLocator(configFile, originalXML, this);
     }
     catch (IOException e) {
       throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
     }
   }
+
+  @Override
+  public CoresLocator getCoresLocator() {
+    return this.persistor;
+  }
   
   private void checkForIllegalConfig() throws IOException {
     // Do sanity checks - we don't want to find new style
@@ -100,6 +109,10 @@ public class ConfigSolrXmlOld extends Co
           " solr.xml may be a mix of old and new style formats.");
     }
   }
+
+  public boolean isPersistent() {
+    return config.getBool("solr/@persistent", false);
+  }
   
   private void fillPropMap() {
     
@@ -208,34 +221,6 @@ public class ConfigSolrXmlOld extends Co
     
   }
 
-  @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 = PropertiesUtil.substituteProperty(attribute.getNodeValue(), null);
-            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;
-  }
-
-  @Override
   public List<String> getAllCoreNames() {
     List<String> ret = new ArrayList<String>();
     
@@ -249,7 +234,6 @@ public class ConfigSolrXmlOld extends Co
     return ret;
   }
 
-  @Override
   public String getProperty(String coreName, String property, String defaultVal) {
     
     synchronized (coreNodes) {
@@ -257,7 +241,9 @@ public class ConfigSolrXmlOld extends Co
         Node node = coreNodes.item(idx);
         if (coreName.equals(DOMUtil.getAttr(node, CoreDescriptor.CORE_NAME,
             null))) {
-          String propVal = DOMUtil.getAttr(node, property, defaultVal);
+          String propVal = DOMUtil.getAttr(node, property);
+          if (propVal == null)
+            propVal = defaultVal;
           return PropertiesUtil.substituteProperty(propVal, null);
         }
       }
@@ -266,24 +252,20 @@ public class ConfigSolrXmlOld extends Co
     
   }
 
-  @Override
-  public Properties readCoreProperties(String coreName) {
-    
+  public Properties getCoreProperties(String coreName) {
     synchronized (coreNodes) {
-      for (int idx = 0; idx < coreNodes.getLength(); ++idx) {
+      for (int idx = 0; idx < coreNodes.getLength(); idx++) {
         Node node = coreNodes.item(idx);
-        if (coreName.equals(DOMUtil.getAttr(node, CoreDescriptor.CORE_NAME,
-            null))) {
+        if (coreName.equals(DOMUtil.getAttr(node, CoreDescriptor.CORE_NAME, null))) {
           try {
             return readProperties(node);
           } catch (XPathExpressionException e) {
-            return null;
+            SolrException.log(log, e);
           }
         }
       }
     }
-    
-    return null;
+    return new Properties();
   }
 
   public static final String DEF_SOLR_XML = "<?xml version=\"1.0\" encoding=\"UTF-8\" ?>\n"
@@ -298,9 +280,4 @@ public class ConfigSolrXmlOld extends Co
         + "\" shard=\"${shard:}\" collection=\"${collection:collection1}\" instanceDir=\"collection1\" />\n"
         + "  </cores>\n" + "</solr>";
 
-  @Override
-  public void substituteProperties() {
-    config.substituteProperties();
-  }
-
 }

Modified: lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/core/CoreContainer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/core/CoreContainer.java?rev=1502280&r1=1502279&r2=1502280&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/core/CoreContainer.java (original)
+++ lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/core/CoreContainer.java Thu Jul 11 16:38:53 2013
@@ -17,6 +17,7 @@
 
 package org.apache.solr.core;
 
+import com.google.common.collect.Maps;
 import org.apache.solr.cloud.ZkController;
 import org.apache.solr.cloud.ZkSolrResourceLoader;
 import org.apache.solr.common.SolrException;
@@ -33,18 +34,15 @@ import org.apache.solr.schema.IndexSchem
 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.zookeeper.KeeperException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import javax.xml.xpath.XPathExpressionException;
 import java.io.File;
 import java.text.SimpleDateFormat;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.Date;
-import java.util.HashMap;
 import java.util.HashSet;
 import java.util.LinkedHashMap;
 import java.util.List;
@@ -116,8 +114,11 @@ public class CoreContainer
   protected final ConfigSolr cfg;
   protected final SolrResourceLoader loader;
   protected final String solrHome;
+
   private InfoHandler infoHandler;
-  
+
+  protected final CoresLocator coresLocator;
+
   {
     log.info("New CoreContainer " + System.identityHashCode(this));
   }
@@ -152,8 +153,9 @@ public class CoreContainer
   }
 
   /**
-   * Create a new CoreContainer using the given SolrResourceLoader and
-   * configuration.  The container's cores are not loaded.
+   * Create a new CoreContainer using the given SolrResourceLoader,
+   * configuration and CoresLocator.  The container's cores are
+   * not loaded.
    * @param loader the SolrResourceLoader
    * @param config a ConfigSolr representation of this container's configuration
    * @see #load()
@@ -162,6 +164,14 @@ public class CoreContainer
     this.loader = checkNotNull(loader);
     this.solrHome = loader.getInstanceDir();
     this.cfg = checkNotNull(config);
+    this.coresLocator = config.getCoresLocator();
+  }
+
+  public CoreContainer(SolrResourceLoader loader, ConfigSolr config, CoresLocator locator) {
+    this.loader = checkNotNull(loader);
+    this.solrHome = loader.getInstanceDir();
+    this.cfg = checkNotNull(config);
+    this.coresLocator = locator;
   }
 
   /**
@@ -280,81 +290,20 @@ public class CoreContainer
           coreLoadExecutor);
       Set<Future<SolrCore>> pending = new HashSet<Future<SolrCore>>();
 
-      List<String> allCores = cfg.getAllCoreNames();
+      List<CoreDescriptor> cds = coresLocator.discover(this);
+      checkForDuplicateCoreNames(cds);
 
-      for (String oneCoreName : allCores) {
+      for (final CoreDescriptor cd : cds) {
 
+        final String name = cd.getName();
         try {
-          String rawName = cfg.getProperty(oneCoreName, CoreDescriptor.CORE_NAME, null);
-
-          if (null == rawName) {
-            throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
-                "Each core in solr.xml must have a 'name'");
-          }
-          final String name = rawName;
-          final CoreDescriptor p = new CoreDescriptor(this, name,
-              cfg.getProperty(oneCoreName, CoreDescriptor.CORE_INSTDIR, null));
-          
-          // deal with optional settings
-          String opt = cfg.getProperty(oneCoreName, CoreDescriptor.CORE_CONFIG, null);
-          
-          if (opt != null) {
-            p.setConfigName(opt);
-          }
-          opt = cfg.getProperty(oneCoreName, CoreDescriptor.CORE_SCHEMA, null);
-          if (opt != null) {
-            p.setSchemaName(opt);
-          }
-          
-          if (zkSys.getZkController() != null) {
-            opt = cfg.getProperty(oneCoreName, CoreDescriptor.CORE_SHARD, null);
-            if (opt != null && opt.length() > 0) {
-              p.getCloudDescriptor().setShardId(opt);
-            }
-            opt = cfg.getProperty(oneCoreName, CoreDescriptor.CORE_COLLECTION, null);
-            if (opt != null) {
-              p.getCloudDescriptor().setCollectionName(opt);
-            }
-            opt = cfg.getProperty(oneCoreName, CoreDescriptor.CORE_ROLES, null);
-            if (opt != null) {
-              p.getCloudDescriptor().setRoles(opt);
-            }
-
-            opt = cfg.getProperty(oneCoreName, CoreDescriptor.CORE_NODE_NAME, null);
-            if (opt != null && opt.length() > 0) {
-              p.getCloudDescriptor().setCoreNodeName(opt);
-            }
-          }
-          opt = cfg.getProperty(oneCoreName, CoreDescriptor.CORE_PROPERTIES, null);
-          if (opt != null) {
-            p.setPropertiesName(opt);
-          }
-          opt = cfg.getProperty(oneCoreName, CoreDescriptor.CORE_DATADIR, null);
-          if (opt != null) {
-            p.setDataDir(opt);
-          }
-          
-          p.setCoreProperties(cfg.readCoreProperties(oneCoreName));
-          
-          opt = cfg.getProperty(oneCoreName, CoreDescriptor.CORE_LOADONSTARTUP, null);
-          if (opt != null) {
-            p.setLoadOnStartup(("true".equalsIgnoreCase(opt) || "on"
-                .equalsIgnoreCase(opt)) ? true : false);
-          }
-          
-          opt = cfg.getProperty(oneCoreName, CoreDescriptor.CORE_TRANSIENT, null);
-          if (opt != null) {
-            p.setTransient(("true".equalsIgnoreCase(opt) || "on"
-                .equalsIgnoreCase(opt)) ? true : false);
-          }
 
-          if (p.isTransient() || ! p.isLoadOnStartup()) {
+          if (cd.isTransient() || ! cd.isLoadOnStartup()) {
             // Store it away for later use. includes non-transient but not
             // loaded at startup cores.
-            solrCores.putDynamicDescriptor(rawName, p);
+            solrCores.putDynamicDescriptor(name, cd);
           }
-
-          if (p.isLoadOnStartup()) { // The normal case
+          if (cd.isLoadOnStartup()) { // The normal case
 
             Callable<SolrCore> task = new Callable<SolrCore>() {
               @Override
@@ -362,14 +311,14 @@ public class CoreContainer
                 SolrCore c = null;
                 try {
                   if (zkSys.getZkController() != null) {
-                    preRegisterInZk(p);
+                    preRegisterInZk(cd);
                   }
-                  c = create(p);
-                  registerCore(p.isTransient(), name, c, false);
+                  c = create(cd);
+                  registerCore(cd.isTransient(), name, c, false);
                 } catch (Throwable t) {
                   if (isZooKeeperAware()) {
                     try {
-                      zkSys.zkController.unregister(name, p);
+                      zkSys.zkController.unregister(name, cd);
                     } catch (InterruptedException e) {
                       Thread.currentThread().interrupt();
                       SolrException.log(log, null, e);
@@ -427,6 +376,18 @@ public class CoreContainer
     }
   }
 
+  private static void checkForDuplicateCoreNames(List<CoreDescriptor> cds) {
+    Map<String, String> addedCores = Maps.newHashMap();
+    for (CoreDescriptor cd : cds) {
+      final String name = cd.getName();
+      if (addedCores.containsKey(name))
+        throw new SolrException(ErrorCode.SERVER_ERROR,
+            String.format(Locale.ROOT, "Found multiple cores with the name [%s], with instancedirs [%s] and [%s]",
+                name, addedCores.get(name), cd.getInstanceDir()));
+      addedCores.put(name, cd.getInstanceDir());
+    }
+  }
+
   private volatile boolean isShutDown = false;
   
   public boolean isShutDown() {
@@ -521,6 +482,10 @@ public class CoreContainer
     }
   }
 
+  public CoresLocator getCoresLocator() {
+    return coresLocator;
+  }
+
   protected SolrCore registerCore(boolean isTransientCore, String name, SolrCore core, boolean returnPrevNotClosed) {
     if( core == null ) {
       throw new RuntimeException( "Can not register a null core." );
@@ -548,7 +513,6 @@ public class CoreContainer
       */
 
     core.setName(name);
-    core.getCoreDescriptor().putProperty(CoreDescriptor.CORE_NAME, name);
 
     synchronized (coreInitFailures) {
       coreInitFailures.remove(name);
@@ -587,7 +551,7 @@ public class CoreContainer
     SolrResourceLoader solrLoader = null;
 
     SolrConfig config = null;
-    solrLoader = new SolrResourceLoader(instanceDir, loader.getClassLoader(), ConfigSolrXml.getCoreProperties(instanceDir, dcore));
+    solrLoader = new SolrResourceLoader(instanceDir, loader.getClassLoader(), dcore.getCoreProperties());
     try {
       config = new SolrConfig(solrLoader, dcore.getConfigName(), null);
     } catch (Exception e) {
@@ -609,11 +573,11 @@ public class CoreContainer
             schemaFile.lastModified()));
         schema = indexSchemaCache.get(key);
         if (schema == null) {
-          log.info("creating new schema object for core: " + dcore.getProperty(CoreDescriptor.CORE_NAME));
+          log.info("creating new schema object for core: " + dcore.getName());
           schema = IndexSchemaFactory.buildIndexSchema(dcore.getSchemaName(), config);
           indexSchemaCache.put(key, schema);
         } else {
-          log.info("re-using schema object for core: " + dcore.getProperty(CoreDescriptor.CORE_NAME));
+          log.info("re-using schema object for core: " + dcore.getName());
         }
       }
     }
@@ -752,7 +716,8 @@ public class CoreContainer
                  cd.getName(), instanceDir.getAbsolutePath());
         SolrResourceLoader solrLoader;
         if(zkSys.getZkController() == null) {
-          solrLoader = new SolrResourceLoader(instanceDir.getAbsolutePath(), loader.getClassLoader(), ConfigSolrXml.getCoreProperties(instanceDir.getAbsolutePath(), cd));
+          solrLoader = new SolrResourceLoader(instanceDir.getAbsolutePath(), loader.getClassLoader(),
+                                                cd.getCoreProperties());
         } else {
           try {
             String collection = cd.getCloudDescriptor().getCollectionName();
@@ -765,7 +730,7 @@ public class CoreContainer
                                            "Could not find config name for collection:" + collection);
             }
             solrLoader = new ZkSolrResourceLoader(instanceDir.getAbsolutePath(), zkConfigName, loader.getClassLoader(),
-                ConfigSolrXml.getCoreProperties(instanceDir.getAbsolutePath(), cd), zkSys.getZkController());
+                cd.getCoreProperties(), zkSys.getZkController());
           } catch (KeeperException e) {
             log.error("", e);
             throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR,
@@ -808,15 +773,18 @@ public class CoreContainer
     n1 = checkDefault(n1);
     solrCores.swap(n0, n1);
 
+    coresLocator.persist(this, solrCores.getCoreDescriptor(n0), solrCores.getCoreDescriptor(n1));
+
     log.info("swapped: "+n0 + " with " + n1);
   }
   
   /** Removes and returns registered core w/o decrementing it's reference count */
   public SolrCore remove( String name ) {
-    name = checkDefault(name);    
-
-    return solrCores.remove(name, true);
-
+    name = checkDefault(name);
+    CoreDescriptor cd = solrCores.getCoreDescriptor(name);
+    SolrCore removed = solrCores.remove(name, true);
+    coresLocator.delete(this, cd);
+    return removed;
   }
 
   public void rename(String name, String toName) {
@@ -825,7 +793,8 @@ public class CoreContainer
       if (core != null) {
         registerCore(false, toName, core, false);
         name = checkDefault(name);
-        solrCores.remove(name, false);
+        SolrCore old = solrCores.remove(name, false);
+        coresLocator.rename(this, old.getCoreDescriptor(), core.getCoreDescriptor());
       }
     } finally {
       if (core != null) {
@@ -833,6 +802,24 @@ public class CoreContainer
       }
     }
   }
+
+  /**
+   * Get the CoreDescriptors for all cores managed by this container
+   * @return a List of CoreDescriptors
+   */
+  public List<CoreDescriptor> getCoreDescriptors() {
+    return solrCores.getCoreDescriptors();
+  }
+
+  public CoreDescriptor getCoreDescriptor(String coreName) {
+    // TODO make this less hideous!
+    for (CoreDescriptor cd : getCoreDescriptors()) {
+      if (cd.getName().equals(coreName))
+        return cd;
+    }
+    return null;
+  }
+
   /** 
    * Gets a core by name and increase its refcount.
    *
@@ -971,10 +958,6 @@ public class CoreContainer
   public void setLogging(LogWatcher v) {
     logging = v;
   }
-  
-  public File getConfigFile() {
-    return new File(solrHome, ConfigSolr.SOLR_XML_FILE);
-  }
 
   /**
    * Determines whether the core is already loaded or not but does NOT load the core
@@ -984,12 +967,6 @@ public class CoreContainer
     return solrCores.isLoaded(name);
   }
 
-  /** Persists the cores config file in cores.xml. */
-  @Deprecated
-  public void persist() {
-    persistFile(getConfigFile());
-  }
-
   /**
    * Gets a solr core descriptor for a core that is not loaded. Note that if the caller calls this on a
    * loaded core, the unloaded descriptor will be returned.
@@ -1001,127 +978,6 @@ public class CoreContainer
     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 ? getConfigFile() : file));
-
-    // <solr attrib="value">
-    Map<String,String> rootSolrAttribs = new HashMap<String,String>();
-
-    addAttrib(rootSolrAttribs, ConfigSolr.CfgProp.SOLR_SHAREDLIB, "sharedLib", this.libDir);
-    addAttrib(rootSolrAttribs, ConfigSolr.CfgProp.SOLR_PERSISTENT, "persistent",
-        Boolean.toString(isPersistent()), "false");
-    addAttrib(rootSolrAttribs, ConfigSolr.CfgProp.SOLR_CORELOADTHREADS, "coreLoadThreads",
-        Integer.toString(this.coreLoadThreads), Integer.toString(CORE_LOAD_THREADS));
-    addAttrib(rootSolrAttribs, ConfigSolr.CfgProp.SOLR_ZKHOST, "zkHost", this.zkHost);
-
-    // <solr attrib="value"> <cores attrib="value">
-    Map<String,String> coresAttribs = new HashMap<String,String>();
-    addAttrib(coresAttribs, ConfigSolr.CfgProp.SOLR_ADMINPATH, "adminPath", this.adminPath, this.getAdminPath());
-    addAttrib(coresAttribs, ConfigSolr.CfgProp.SOLR_ADMINHANDLER, "adminHandler", this.adminHandler);
-    addAttrib(coresAttribs, ConfigSolr.CfgProp.SOLR_SHARESCHEMA, "shareSchema",
-        Boolean.toString(this.shareSchema),
-        Boolean.toString(DEFAULT_SHARE_SCHEMA));
-    addAttrib(coresAttribs, ConfigSolr.CfgProp.SOLR_HOST, "host", zkSys.getHost());
-
-    if (! (null == defaultCoreName || defaultCoreName.equals("")) ) {
-      coresAttribs.put("defaultCoreName", defaultCoreName);
-    }
-
-    addAttrib(coresAttribs, ConfigSolr.CfgProp.SOLR_HOSTPORT, "hostPort", zkSys.getHostPort());
-
-    addAttrib(coresAttribs, ConfigSolr.CfgProp.SOLR_ZKCLIENTTIMEOUT, "zkClientTimeout",
-        intToString(this.zkClientTimeout),
-        Integer.toString(DEFAULT_ZK_CLIENT_TIMEOUT));
-    addAttrib(coresAttribs, ConfigSolr.CfgProp.SOLR_HOSTCONTEXT, "hostContext",
-        zkSys.getHostContext());
-    addAttrib(coresAttribs, ConfigSolr.CfgProp.SOLR_LEADERVOTEWAIT, "leaderVoteWait",
-        zkSys.getLeaderVoteWait(), LEADER_VOTE_WAIT);
-    addAttrib(coresAttribs, ConfigSolr.CfgProp.SOLR_GENERICCORENODENAMES, "genericCoreNodeNames",
-        Boolean.toString(zkSys.getGenericCoreNodeNames()), "false");
-    if (transientCacheSize != Integer.MAX_VALUE) { // This test
-    // is a consequence of testing. I really hate it.
-      addAttrib(coresAttribs, ConfigSolr.CfgProp.SOLR_TRANSIENTCACHESIZE, "transientCacheSize",
-          Integer.toString(this.transientCacheSize), Integer.toString(Integer.MAX_VALUE));
-    }
-    addAttrib(coresAttribs, ConfigSolr.CfgProp.SOLR_DISTRIBUPDATECONNTIMEOUT, "distribUpdateConnTimeout",
-        Integer.toString(this.distribUpdateConnTimeout), Integer.toString(this.distribUpdateConnTimeout));
-    addAttrib(coresAttribs, ConfigSolr.CfgProp.SOLR_DISTRIBUPDATESOTIMEOUT, "distribUpdateSoTimeout",
-        Integer.toString(this.distribUpdateSoTimeout), Integer.toString(this.distribUpdateSoTimeout));
-    addAttrib(coresAttribs, ConfigSolr.CfgProp.SOLR_MANAGEMENTPATH, "managementPath",
-        this.managementPath);
-
-    // don't forget the logging stuff
-    Map<String, String> loggingAttribs = new HashMap<String, String>();
-    addAttrib(loggingAttribs, ConfigSolr.CfgProp.SOLR_LOGGING_CLASS, "class",
-        cfg.get(ConfigSolr.CfgProp.SOLR_LOGGING_CLASS, null));
-    addAttrib(loggingAttribs, ConfigSolr.CfgProp.SOLR_LOGGING_ENABLED, "enabled",
-        cfg.get(ConfigSolr.CfgProp.SOLR_LOGGING_ENABLED, null));
-
-    Map<String, String> watcherAttribs = new HashMap<String, String>();
-    addAttrib(watcherAttribs, ConfigSolr.CfgProp.SOLR_LOGGING_WATCHER_SIZE, "size",
-        cfg.get(ConfigSolr.CfgProp.SOLR_LOGGING_WATCHER_SIZE, null));
-    addAttrib(watcherAttribs, ConfigSolr.CfgProp.SOLR_LOGGING_WATCHER_THRESHOLD, "threshold",
-        cfg.get(ConfigSolr.CfgProp.SOLR_LOGGING_WATCHER_THRESHOLD, null));
-
-
-    /*
-    Map<String, String> shardHandlerAttrib = new HashMap<String, String>();
-    addAttrib(shardHandlerAttrib, ConfigSolr.CfgProp.SOLR_SHARDHANDLERFACTORY_CLASS, "class",
-        cfg.get(ConfigSolr.CfgProp.SOLR_SHARDHANDLERFACTORY_CLASS, null));
-    addAttrib(shardHandlerAttrib, ConfigSolr.CfgProp.SOLR_SHARDHANDLERFACTORY_NAME, "name",
-        cfg.get(ConfigSolr.CfgProp.SOLR_SHARDHANDLERFACTORY_NAME, null));
-
-    Map<String, String> shardHandlerProps = new HashMap<String, String>();
-    addAttrib(shardHandlerProps, ConfigSolr.CfgProp.SOLR_SHARDHANDLERFACTORY_CONNTIMEOUT, "connTimeout",
-        cfg.get(ConfigSolr.CfgProp.SOLR_SHARDHANDLERFACTORY_CONNTIMEOUT, null));
-    addAttrib(shardHandlerProps, ConfigSolr.CfgProp.SOLR_SHARDHANDLERFACTORY_SOCKETTIMEOUT, "socketTimeout",
-        cfg.get(ConfigSolr.CfgProp.SOLR_SHARDHANDLERFACTORY_SOCKETTIMEOUT, null));
-    */
-
-    try {
-      solrCores.persistCores(cfg.config.getOriginalConfig(), containerProperties, rootSolrAttribs,coresAttribs,
-          loggingAttribs, watcherAttribs, cfg.getUnsubsititutedShardHandlerFactoryPluginNode(), file, loader);
-    } catch (XPathExpressionException e) {
-      throw new SolrException(ErrorCode.SERVER_ERROR, null, e);
-    }
-
-  }
-  private String intToString(Integer integer) {
-    if (integer == null) return null;
-    return Integer.toString(integer);
-  }
-
-  private void addAttrib(Map<String, String> attribs, ConfigSolr.CfgProp prop,
-                         String attribName, String attribValue) {
-    addAttrib(attribs, prop, attribName, attribValue, null);
-  }
-
-    private void addAttrib(Map<String, String> attribs, ConfigSolr.CfgProp prop,
-                         String attribName, String attribValue, String defaultValue) {
-    if (cfg == null) {
-      attribs.put(attribName, attribValue);
-      return;
-    }
-
-    if (attribValue != null) {
-      String origValue = cfg.getOrigProp(prop, null);
-
-      if (origValue == null && defaultValue != null && attribValue.equals(defaultValue)) return;
-
-      if (attribValue.equals(PropertiesUtil.substituteProperty(origValue, loader.getCoreProperties()))) {
-        attribs.put(attribName, origValue);
-      } else {
-        attribs.put(attribName, attribValue);
-      }
-    }
-  }
-
   public void preRegisterInZk(final CoreDescriptor p) {
     zkSys.getZkController().preRegister(p);
   }

Modified: lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/core/CoreDescriptor.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/core/CoreDescriptor.java?rev=1502280&r1=1502279&r2=1502280&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/core/CoreDescriptor.java (original)
+++ lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/core/CoreDescriptor.java Thu Jul 11 16:38:53 2013
@@ -17,12 +17,20 @@
 
 package org.apache.solr.core;
 
-import java.util.Properties;
-import java.io.File;
-
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
 import org.apache.commons.lang.StringUtils;
 import org.apache.solr.cloud.CloudDescriptor;
-import org.apache.solr.core.ConfigSolr.CfgProp;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.util.PropertiesUtil;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.util.Locale;
+import java.util.Properties;
+
+import static com.google.common.base.Preconditions.checkNotNull;
 
 /**
  * A Solr core descriptor
@@ -34,7 +42,8 @@ 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"; // should probably be removed after 4x
+  public static final String CORE_INSTDIR = "instanceDir";
+  public static final String CORE_ABS_INSTDIR = "absoluteInstDir";
   public static final String CORE_DATADIR = "dataDir";
   public static final String CORE_ULOGDIR = "ulogDir";
   public static final String CORE_SCHEMA = "schema";
@@ -46,153 +55,207 @@ public class CoreDescriptor {
   public static final String CORE_TRANSIENT = "transient";
   public static final String CORE_NODE_NAME = "coreNodeName";
 
-  public static final String[] standardPropNames = {
+  public static final String DEFAULT_EXTERNAL_PROPERTIES_FILE = "conf" + File.separator + "solrcore.properties";
+
+  /**
+   * Get the standard properties in persistable form
+   * @return the standard core properties in persistable form
+   */
+  public Properties getPersistableStandardProperties() {
+    return originalCoreProperties;
+  }
+
+  /**
+   * Get user-defined core properties in persistable form
+   * @return user-defined core properties in persistable form
+   */
+  public Properties getPersistableUserProperties() {
+    return originalExtraProperties;
+  }
+
+  private static ImmutableMap<String, String> defaultProperties = ImmutableMap.of(
+      CORE_CONFIG, "solrconfig.xml",
+      CORE_SCHEMA, "schema.xml",
+      CORE_DATADIR, "data" + File.separator,
+      CORE_TRANSIENT, "false",
+      CORE_LOADONSTARTUP, "true"
+  );
+
+  private static ImmutableList<String> requiredProperties = ImmutableList.of(
+      CORE_NAME, CORE_INSTDIR, CORE_ABS_INSTDIR
+  );
+
+  public static ImmutableList<String> standardPropNames = ImmutableList.of(
       CORE_NAME,
       CORE_CONFIG,
       CORE_INSTDIR,
       CORE_DATADIR,
       CORE_ULOGDIR,
       CORE_SCHEMA,
+      CORE_PROPERTIES,
+      CORE_LOADONSTARTUP,
+      CORE_TRANSIENT,
+      // cloud props
       CORE_SHARD,
       CORE_COLLECTION,
       CORE_ROLES,
-      CORE_PROPERTIES,
-      CORE_LOADONSTARTUP,
-      CORE_TRANSIENT
-  };
-
-  // As part of moving away from solr.xml (see SOLR-4196), it's _much_ easier to keep these as properties than set
-  // them individually.
-  private Properties coreProperties = new Properties();
-
-  //TODO: 5.0 remove this, this is solely a hack for persistence. And perhaps creating cores in discovery mode?
-  private Properties createdProperties = new Properties();
-
-  private boolean loadedImplicit = false;
+      CORE_NODE_NAME,
+      CloudDescriptor.NUM_SHARDS,
+      CloudDescriptor.SHARD_STATE
+  );
 
   private final CoreContainer coreContainer;
 
-  private CloudDescriptor cloudDesc;
+  private final CloudDescriptor cloudDesc;
 
-  private CoreDescriptor(CoreContainer cont) {
-    // Just a place to put initialization since it's a pain to add to the descriptor in every c'tor.
-    this.coreContainer = cont;
-    coreProperties.put(CORE_LOADONSTARTUP, "true");
-    coreProperties.put(CORE_TRANSIENT, "false");
-
-  }
-  
-  public CoreDescriptor(CoreContainer container, String name, String instanceDir) {
-    this(container);
-    doInit(name, instanceDir);
-  }
+  /** The original standard core properties, before substitution */
+  protected final Properties originalCoreProperties = new Properties();
 
+  /** The original extra core properties, before substitution */
+  protected final Properties originalExtraProperties = new Properties();
 
-  public CoreDescriptor(CoreDescriptor descr) {
-    this(descr.coreContainer);
-    coreProperties.put(CORE_INSTDIR, descr.getInstanceDir());
-    coreProperties.put(CORE_CONFIG, descr.getConfigName());
-    coreProperties.put(CORE_SCHEMA, descr.getSchemaName());
-    coreProperties.put(CORE_NAME, descr.getName());
-    coreProperties.put(CORE_DATADIR, descr.getDataDir());
-  }
+  /** The properties for this core, as available through getProperty() */
+  protected final Properties coreProperties = new Properties();
 
   /**
-   * CoreDescriptor - create a core descriptor given default properties from a core.properties file. This will be
-   * used in the "solr.xml-less (See SOLR-4196) world where there are no &lt;core&gt; &lt;/core&gt; tags at all, thus  much
-   * of the initialization that used to be done when reading solr.xml needs to be done here instead, particularly
-   * setting any defaults (e.g. schema.xml, directories, whatever).
-   *
-   * @param container - the CoreContainer that holds all the information about our cores, loaded, lazy etc.
-   * @param propsIn - A properties structure "core.properties" found while walking the file tree to discover cores.
-   *                  Any properties set in this param will overwrite the any defaults.
+   * Create a new CoreDescriptor.
+   * @param container       the CoreDescriptor's container
+   * @param name            the CoreDescriptor's name
+   * @param instanceDir     a String containing the instanceDir
+   * @param coreProps       a Properties object of the properties for this core
    */
-  public CoreDescriptor(CoreContainer container, Properties propsIn) {
-    this(container);
+  public CoreDescriptor(CoreContainer container, String name, String instanceDir,
+                        Properties coreProps) {
 
-    // Set some default, normalize a directory or two
-    doInit(propsIn.getProperty(CORE_NAME), propsIn.getProperty(CORE_INSTDIR));
+    this.coreContainer = container;
 
-    coreProperties.putAll(propsIn);
-  }
+    originalCoreProperties.setProperty(CORE_NAME, name);
+    originalCoreProperties.setProperty(CORE_INSTDIR, instanceDir);
 
-  private void doInit(String name, String instanceDir) {
-    if (name == null) {
-      throw new RuntimeException("Core needs a name");
-    }
+    Properties containerProperties = container.getContainerProperties();
+    name = PropertiesUtil.substituteProperty(checkPropertyIsNotEmpty(name, CORE_NAME),
+                                             containerProperties);
+    instanceDir = PropertiesUtil.substituteProperty(checkPropertyIsNotEmpty(instanceDir, CORE_INSTDIR),
+                                                    containerProperties);
 
+    coreProperties.putAll(defaultProperties);
     coreProperties.put(CORE_NAME, name);
+    coreProperties.put(CORE_INSTDIR, instanceDir);
+    coreProperties.put(CORE_ABS_INSTDIR, convertToAbsolute(instanceDir, container.getSolrHome()));
 
-    if(coreContainer != null && coreContainer.getZkController() != null) {
-      this.cloudDesc = new CloudDescriptor();
-      // cloud collection defaults to core name
-      cloudDesc.setCollectionName(name);
+    for (String propname : coreProps.stringPropertyNames()) {
+
+      String propvalue = coreProps.getProperty(propname);
+
+      if (isUserDefinedProperty(propname))
+        originalExtraProperties.put(propname, propvalue);
+      else
+        originalCoreProperties.put(propname, propvalue);
+
+      if (!requiredProperties.contains(propname))   // Required props are already dealt with
+        coreProperties.setProperty(propname,
+            PropertiesUtil.substituteProperty(propvalue, containerProperties));
     }
 
-    if (instanceDir == null) {
-      throw new NullPointerException("Missing required \'instanceDir\'");
+    loadExtraProperties();
+
+    // TODO maybe make this a CloudCoreDescriptor subclass?
+    if (container.isZooKeeperAware()) {
+      cloudDesc = new CloudDescriptor(name, coreProperties);
+    }
+    else {
+      cloudDesc = null;
     }
-    instanceDir = SolrResourceLoader.normalizeDir(instanceDir);
-    coreProperties.put(CORE_INSTDIR, instanceDir);
-    coreProperties.put(CORE_CONFIG, getDefaultConfigName());
-    coreProperties.put(CORE_SCHEMA, getDefaultSchemaName());
   }
 
-  public Properties initImplicitProperties() {
-
-    Properties implicitProperties = new Properties();
-    if (coreContainer != null && coreContainer.getContainerProperties() != null){
-      implicitProperties.putAll(coreContainer.getContainerProperties());
+  /**
+   * Load properties specified in an external properties file.
+   *
+   * The file to load can be specified in a {@code properties} property on
+   * the original Properties object used to create this CoreDescriptor.  If
+   * this has not been set, then we look for {@code conf/solrcore.properties}
+   * underneath the instance dir.
+   *
+   * File paths are taken as read from the core's instance directory
+   * if they are not absolute.
+   */
+  protected void loadExtraProperties() {
+    String filename = coreProperties.getProperty(CORE_PROPERTIES, DEFAULT_EXTERNAL_PROPERTIES_FILE);
+    File propertiesFile = resolvePaths(filename);
+    if (propertiesFile.exists()) {
+      try {
+        Properties externalProps = new Properties();
+        externalProps.load(new FileInputStream(propertiesFile));
+        coreProperties.putAll(externalProps);
+      }
+      catch (IOException e) {
+        String message = String.format(Locale.ROOT, "Could not load properties from %s: %s:",
+            propertiesFile.getAbsoluteFile(), e.toString());
+        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, message);
+      }
     }
-    implicitProperties.setProperty("solr.core.name", getName());
-    implicitProperties.setProperty("solr.core.instanceDir", getInstanceDir());
-    implicitProperties.setProperty("solr.core.dataDir", getDataDir());
-    implicitProperties.setProperty("solr.core.configName", getConfigName());
-    implicitProperties.setProperty("solr.core.schemaName", getSchemaName());
-    return implicitProperties;
   }
 
-  /**@return the default config name. */
-  public String getDefaultConfigName() {
-    return "solrconfig.xml";
+  protected File resolvePaths(String filepath) {
+    File file = new File(filepath);
+    if (file.isAbsolute())
+      return file;
+    return new File(getInstanceDir(), filepath);
   }
 
-  /**@return the default schema name. */
-  public String getDefaultSchemaName() {
-    return "schema.xml";
+  /**
+   * Is this property a Solr-standard property, or is it an extra property
+   * defined per-core by the user?
+   * @param propName the Property name
+   * @return @{code true} if this property is user-defined
+   */
+  protected static boolean isUserDefinedProperty(String propName) {
+    return !standardPropNames.contains(propName);
   }
 
-  /**@return the default data directory. */
-  public String getDefaultDataDir() {
-    return "data" + File.separator;
+  public static String checkPropertyIsNotEmpty(String value, String propName) {
+    if (StringUtils.isEmpty(value)) {
+      String message = String.format(Locale.ROOT, "Cannot create core with empty %s value", propName);
+      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, message);
+    }
+    return value;
   }
 
-  public String getPropertiesName() {
-    return coreProperties.getProperty(CORE_PROPERTIES);
+  /**
+   * Create a new CoreDescriptor with a given name and instancedir
+   * @param container     the CoreDescriptor's container
+   * @param name          the CoreDescriptor's name
+   * @param instanceDir   the CoreDescriptor's instancedir
+   */
+  public CoreDescriptor(CoreContainer container, String name, String instanceDir) {
+    this(container, name, instanceDir, new Properties());
   }
 
-  public void setPropertiesName(String propertiesName) {
-    coreProperties.put(CORE_PROPERTIES, propertiesName);
+  /**
+   * Create a new CoreDescriptor using the properties of an existing one
+   * @param coreName the new CoreDescriptor's name
+   * @param other    the CoreDescriptor to copy
+   */
+  public CoreDescriptor(String coreName, CoreDescriptor other) {
+    this.coreContainer = other.coreContainer;
+    this.originalExtraProperties.putAll(other.originalExtraProperties);
+    this.originalCoreProperties.putAll(other.originalCoreProperties);
+    this.coreProperties.putAll(other.coreProperties);
+    this.coreProperties.setProperty(CORE_NAME, coreName);
+    this.originalCoreProperties.setProperty(CORE_NAME, coreName);
+    this.cloudDesc = other.cloudDesc;
   }
 
-  public String getDataDir() {
-    String dataDir = coreProperties.getProperty(CORE_DATADIR);
-    if (dataDir == null) dataDir = getDefaultDataDir();
-    return dataDir;
+  public String getPropertiesName() {
+    return coreProperties.getProperty(CORE_PROPERTIES);
   }
 
-  public void setDataDir(String s) {
-    // normalize zero length to null.
-    if (StringUtils.isBlank(s)) {
-      coreProperties.remove(s);
-    } else {
-      coreProperties.put(CORE_DATADIR, s);
-    }
+  public String getDataDir() {
+    return coreProperties.getProperty(CORE_DATADIR);
   }
   
   public boolean usingDefaultDataDir() {
-    // DO NOT use the getDataDir method here since it'll assign something regardless.
-    return coreProperties.getProperty(CORE_DATADIR) == null;
+    return defaultProperties.get(CORE_DATADIR).equals(coreProperties.getProperty(CORE_DATADIR));
   }
 
   /**@return the core instance directory. */
@@ -200,37 +263,20 @@ public class CoreDescriptor {
     return coreProperties.getProperty(CORE_INSTDIR);
   }
 
+  private static String convertToAbsolute(String instDir, String solrHome) {
+    checkNotNull(instDir);
+    File f = new File(instDir);
+    if (f.isAbsolute())
+      return SolrResourceLoader.normalizeDir(instDir);
+    return SolrResourceLoader.normalizeDir(solrHome + SolrResourceLoader.normalizeDir(instDir));
+  }
+
   /**
    *
    * @return the core instance directory, prepended with solr_home if not an absolute path.
    */
   public String getInstanceDir() {
-    String instDir = coreProperties.getProperty(CORE_INSTDIR);
-    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));
-  }
-
-  /**Sets the core configuration resource name. */
-  public void setConfigName(String name) {
-    if (name == null || name.length() == 0)
-      throw new IllegalArgumentException("name can not be null or empty");
-    coreProperties.put(CORE_CONFIG, name);
+    return coreProperties.getProperty(CORE_ABS_INSTDIR);
   }
 
   /**@return the core configuration resource name. */
@@ -238,13 +284,6 @@ public class CoreDescriptor {
     return coreProperties.getProperty(CORE_CONFIG);
   }
 
-  /**Sets the core schema resource name. */
-  public void setSchemaName(String name) {
-    if (name == null || name.length() == 0)
-      throw new IllegalArgumentException("name can not be null or empty");
-    coreProperties.put(CORE_SCHEMA, name);
-  }
-
   /**@return the core schema resource name. */
   public String getSchemaName() {
     return coreProperties.getProperty(CORE_SCHEMA);
@@ -255,103 +294,57 @@ public class CoreDescriptor {
     return coreProperties.getProperty(CORE_NAME);
   }
 
-  public CoreContainer getCoreContainer() {
-    return coreContainer;
-  }
-
-  Properties getCoreProperties() {
-    return coreProperties;
-  }
-
-  /**
-   * Set this core's properties. Please note that some implicit values will be added to the
-   * Properties instance passed into this method. This means that the Properties instance
-   * sent to this method will have different (less) key/value pairs than the Properties
-   * instance returned by #getCoreProperties method.
-   *
-   * Under any circumstance, the properties passed in will override any already present.Merge
-   */
-  public void setCoreProperties(Properties coreProperties) {
-    if (! loadedImplicit) {
-      loadedImplicit = true;
-      Properties p = initImplicitProperties();
-      this.coreProperties.putAll(p);
-      // The caller presumably wants whatever properties passed in to override the current core props, so just add them.
-      if (coreProperties != null) {
-        this.coreProperties.putAll(coreProperties);
-      }
-    }
+  public String getCollectionName() {
+    return cloudDesc == null ? null : cloudDesc.getCollectionName();
   }
 
-  public void addCreatedProperty(String key, String value) {
-    createdProperties.put(key, value);
-  }
-
-  public final Properties getCreatedProperties() {
-    return createdProperties;
+  public CoreContainer getCoreContainer() {
+    return coreContainer;
   }
 
   public CloudDescriptor getCloudDescriptor() {
     return cloudDesc;
   }
-  
-  public void setCloudDescriptor(CloudDescriptor cloudDesc) {
-    this.cloudDesc = cloudDesc;
-  }
+
   public boolean isLoadOnStartup() {
     String tmp = coreProperties.getProperty(CORE_LOADONSTARTUP, "false");
     return Boolean.parseBoolean(tmp);
   }
 
-  public void setLoadOnStartup(boolean loadOnStartup) {
-    coreProperties.put(CORE_LOADONSTARTUP, Boolean.toString(loadOnStartup));
-  }
-
   public boolean isTransient() {
     String tmp = coreProperties.getProperty(CORE_TRANSIENT, "false");
-    return (Boolean.parseBoolean(tmp));
-  }
-
-  public void setTransient(boolean isTransient) {
-    coreProperties.put(CORE_TRANSIENT, Boolean.toString(isTransient));
+    return PropertiesUtil.toBoolean(tmp);
   }
 
   public String getUlogDir() {
     return coreProperties.getProperty(CORE_ULOGDIR);
   }
 
-  public void setUlogDir(String ulogDir) {
-    coreProperties.put(CORE_ULOGDIR, ulogDir);
-  }
-
   /**
-   * Reads a property defined in the core.properties file that's replacing solr.xml (if present).
+   * Returns a specific property defined on this CoreDescriptor
    * @param prop    - value to read from the properties structure.
    * @param defVal  - return if no property found.
    * @return associated string. May be null.
    */
-  public String getProperty(String prop, String defVal) {
+  public String getCoreProperty(String prop, String defVal) {
     return coreProperties.getProperty(prop, defVal);
   }
 
   /**
-   * gReads a property defined in the core.properties file that's replacing solr.xml (if present).
-   * @param prop  value to read from the properties structure.
-   * @return associated string. May be null.
+   * Returns all properties defined on this CoreDescriptor
+   * @return all properties defined on this CoreDescriptor
    */
-  public String getProperty(String prop) {
-    return coreProperties.getProperty(prop);
+  public Properties getCoreProperties() {
+    return coreProperties;
   }
-  /**
-   * This will eventually replace _all_ of the setters. Puts a value in the "new" (obsoleting solr.xml JIRAs) properties
-   * structures.
-   *
-   * Will replace any currently-existing property with the key "prop".
-   *
-   * @param prop - property name
-   * @param val  - property value
-   */
-  public void putProperty(String prop, String val) {
-    coreProperties.put(prop, val);
+
+  @Override
+  public String toString() {
+    return new StringBuilder("CoreDescriptor[name=")
+        .append(this.getName())
+        .append(";instanceDir=")
+        .append(this.getInstanceDir())
+        .append("]")
+        .toString();
   }
 }

Modified: lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/core/SolrCore.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/core/SolrCore.java?rev=1502280&r1=1502279&r2=1502280&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/core/SolrCore.java (original)
+++ lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/core/SolrCore.java Thu Jul 11 16:38:53 2013
@@ -17,42 +17,6 @@
 
 package org.apache.solr.core;
 
-import java.io.File;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.io.Writer;
-import java.lang.reflect.Constructor;
-import java.net.URL;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Date;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.IdentityHashMap;
-import java.util.LinkedHashMap;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Locale;
-import java.util.Map;
-import java.util.Properties;
-import java.util.Set;
-import java.util.StringTokenizer;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.Future;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicLong;
-import java.util.concurrent.locks.ReentrantLock;
-
-import javax.xml.parsers.ParserConfigurationException;
-
 import org.apache.commons.io.IOUtils;
 import org.apache.lucene.codecs.Codec;
 import org.apache.lucene.index.DirectoryReader;
@@ -65,7 +29,6 @@ import org.apache.lucene.store.IndexInpu
 import org.apache.lucene.store.LockObtainFailedException;
 import org.apache.solr.cloud.CloudDescriptor;
 import org.apache.solr.common.SolrException;
-import org.apache.solr.common.SolrException.ErrorCode;
 import org.apache.solr.common.cloud.Slice;
 import org.apache.solr.common.params.CommonParams;
 import org.apache.solr.common.params.CommonParams.EchoParamStyle;
@@ -129,6 +92,38 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.xml.sax.SAXException;
 
+import javax.xml.parsers.ParserConfigurationException;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.Writer;
+import java.lang.reflect.Constructor;
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.IdentityHashMap;
+import java.util.LinkedHashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+import java.util.StringTokenizer;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.locks.ReentrantLock;
+
 
 /**
  *
@@ -146,7 +141,7 @@ public final class SolrCore implements S
 
   private String name;
   private String logid; // used to show what name is set
-  private final CoreDescriptor coreDescriptor;
+  private CoreDescriptor coreDescriptor;
 
   private boolean isReloaded = false;
 
@@ -306,6 +301,7 @@ public final class SolrCore implements S
   public void setName(String v) {
     this.name = v;
     this.logid = (v==null)?"":("["+v+"] ");
+    this.coreDescriptor = new CoreDescriptor(v, this.coreDescriptor);
   }
 
   public String getLogId()
@@ -848,12 +844,6 @@ public final class SolrCore implements S
     resourceLoader.inform(infoRegistry);
     
     CoreContainer cc = cd.getCoreContainer();
-    
-    if (cc != null) {
-      if (cc.cfg != null && cc.cfg instanceof ConfigSolrXml) {
-        writePropFile(cd, cc);
-      }
-    }
 
     if (cc != null && cc.isZooKeeperAware() && Slice.CONSTRUCTION.equals(cd.getCloudDescriptor().getShardState())) {
       // set update log to buffer before publishing the core
@@ -867,56 +857,6 @@ public final class SolrCore implements S
 //    numOpens.incrementAndGet();
 //    openHandles.put(this, new RuntimeException("unclosed core - name:" + getName() + " refs: " + refCount.get()));
   }
-
-  private void writePropFile(CoreDescriptor cd, CoreContainer cc) {
-    File propFile = new File(cd.getInstanceDir(), "core.properties");
-    if (!propFile.exists()) {
-      propFile.getParentFile().mkdirs();
-      Properties props = new Properties();
-      props.put("name", cd.getName());
-
-      // This must be being created since there's no file here already. So write out all of the params we were
-      // created with. This _may_ overwrite the name above, but that's OK.
-      Collection<String> stds = new HashSet(Arrays.asList(CoreDescriptor.standardPropNames));
-      for (String prop : cd.getCreatedProperties().stringPropertyNames()) {
-        // Only preserve things that are legal, and let's just keep instDir right out of the persisted file even
-        // though it's part of the create properties on the URL.
-        if (! CoreDescriptor.CORE_INSTDIR.equals(prop) && stds.contains(prop)) {
-          props.put(prop, cd.getCreatedProperties().getProperty(prop));
-        }
-      }
-
-      if (cc.isZooKeeperAware()) {
-        String collection = cd.getCloudDescriptor().getCollectionName();
-        if (collection != null) {
-          props.put("collection", collection);
-        }
-        String coreNodeName = cd.getCloudDescriptor().getCoreNodeName();
-        if (coreNodeName != null) {
-          props.put("coreNodeName", coreNodeName);
-        }
-        String roles = cd.getCloudDescriptor().getRoles();
-        if (roles != null) {
-          props.put("roles", roles);
-        }
-        String shardId = cd.getCloudDescriptor().getShardId();
-        if (shardId != null) {
-          props.put("shard", shardId);
-        }
-      }
-      OutputStream out = null;
-      try {
-        out = new FileOutputStream(propFile);
-        props.store(out, "");
-      } catch (IOException e) {
-        throw new SolrException(ErrorCode.SERVER_ERROR, null, e);
-      } finally {
-        if (out != null) {
-          IOUtils.closeQuietly(out);
-        }
-      }
-    }
-  }
     
   private Codec initCodec(SolrConfig solrConfig, final IndexSchema schema) {
     final PluginInfo info = solrConfig.getPluginInfo(CodecFactory.class.getName());

Modified: lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/core/SolrCores.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/core/SolrCores.java?rev=1502280&r1=1502279&r2=1502280&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/core/SolrCores.java (original)
+++ lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/core/SolrCores.java Thu Jul 11 16:38:53 2013
@@ -17,32 +17,24 @@ package org.apache.solr.core;
  * limitations under the License.
  */
 
-import org.apache.commons.lang.StringUtils;
-import org.apache.solr.cloud.CloudDescriptor;
+import com.google.common.collect.Lists;
 import org.apache.solr.common.SolrException;
-import org.apache.solr.common.params.CoreAdminParams;
-import org.apache.solr.core.SolrXMLSerializer.SolrCoreXMLDef;
-import org.apache.solr.util.DOMUtil;
-import org.w3c.dom.Node;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
-import javax.xml.xpath.XPathExpressionException;
-import java.io.File;
 import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.Collection;
-import java.util.HashMap;
 import java.util.HashSet;
 import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
-import java.util.Properties;
 import java.util.Set;
 import java.util.TreeSet;
 import java.util.concurrent.ConcurrentHashMap;
 
 
 class SolrCores {
-  private static SolrXMLSerializer SOLR_XML_SERIALIZER = new SolrXMLSerializer();
+
   private static Object modifyLock = new Object(); // for locking around manipulating any of the core maps.
   private final Map<String, SolrCore> cores = new LinkedHashMap<String, SolrCore>(); // For "permanent" cores
 
@@ -57,6 +49,8 @@ class SolrCores {
 
   private final CoreContainer container;
 
+  private static final Logger logger = LoggerFactory.getLogger(SolrCores.class);
+
   // 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>();
@@ -80,7 +74,9 @@ class SolrCores {
         protected boolean removeEldestEntry(Map.Entry<String, SolrCore> eldest) {
           if (size() > transientCacheSize) {
             synchronized (modifyLock) {
-              pendingCloses.add(eldest.getValue()); // Essentially just queue this core up for closing.
+              SolrCore coreToClose = eldest.getValue();
+              logger.info("Closing transient core [{}]", coreToClose.getName());
+              pendingCloses.add(coreToClose); // Essentially just queue this core up for closing.
               modifyLock.notifyAll(); // Wakes up closer thread too
             }
             return true;
@@ -227,9 +223,7 @@ class SolrCores {
       cores.put(n1, c0);
 
       c0.setName(n1);
-      c0.getCoreDescriptor().putProperty(CoreDescriptor.CORE_NAME, n1);
       c1.setName(n0);
-      c1.getCoreDescriptor().putProperty(CoreDescriptor.CORE_NAME, n0);
     }
 
   }
@@ -314,7 +308,7 @@ class SolrCores {
       if (desc == null) {
         return null;
       }
-      return new CoreDescriptor(desc);
+      return new CoreDescriptor(cname, desc);
     }
 
   }
@@ -325,47 +319,6 @@ class SolrCores {
     }
   }
 
-  public void persistCores(Config cfg, Properties containerProperties,
-      Map<String,String> rootSolrAttribs, Map<String,String> coresAttribs,
-      Map<String, String> loggingAttribs, Map<String,String> watcherAttribs,
-      Node shardHandlerNode,
-      File file, SolrResourceLoader loader) throws XPathExpressionException {
-
-
-    List<SolrXMLSerializer.SolrCoreXMLDef> solrCoreXMLDefs = new ArrayList<SolrXMLSerializer.SolrCoreXMLDef>();
-    synchronized (modifyLock) {
-
-      persistCores(cfg, cores, loader, solrCoreXMLDefs);
-      persistCores(cfg, transientCores, loader, solrCoreXMLDefs);
-      // add back all the cores that aren't loaded, either in cores or transient
-      // cores
-      for (Map.Entry<String,CoreDescriptor> ent : dynamicDescriptors.entrySet()) {
-        if (!cores.containsKey(ent.getKey())
-            && !transientCores.containsKey(ent.getKey())) {
-          addCoreToPersistList(cfg, loader, ent.getValue(), null, solrCoreXMLDefs);
-        }
-      }
-      for (Map.Entry<String,SolrCore> ent : createdCores.entrySet()) {
-        if (!cores.containsKey(ent.getKey())
-            && !transientCores.containsKey(ent.getKey())
-            && !dynamicDescriptors.containsKey(ent.getKey())) {
-          addCoreToPersistList(cfg, loader, ent.getValue().getCoreDescriptor(),
-              null, solrCoreXMLDefs);
-        }
-      }
-
-      SolrXMLSerializer.SolrXMLDef solrXMLDef = new SolrXMLSerializer.SolrXMLDef();
-      solrXMLDef.coresDefs = solrCoreXMLDefs;
-      solrXMLDef.containerProperties = containerProperties;
-      solrXMLDef.solrAttribs = rootSolrAttribs;
-      solrXMLDef.coresAttribs = coresAttribs;
-      solrXMLDef.loggingAttribs = loggingAttribs;
-      solrXMLDef.watcherAttribs = watcherAttribs;
-      solrXMLDef.shardHandlerNode = shardHandlerNode;
-      SOLR_XML_SERIALIZER.persistFile(file, solrXMLDef);
-    }
-
-  }
   // Wait here until any pending operations (load, unload or reload) are completed on this core.
   protected SolrCore waitAddPendingCoreOps(String name) {
 
@@ -414,170 +367,6 @@ class SolrCores {
     }
   }
 
-
-  protected void persistCores(Config cfg, Map<String, SolrCore> whichCores, SolrResourceLoader loader, List<SolrCoreXMLDef> solrCoreXMLDefs) throws XPathExpressionException {
-    for (SolrCore solrCore : whichCores.values()) {
-      addCoreToPersistList(cfg, loader, solrCore.getCoreDescriptor(), getCoreToOrigName(solrCore), solrCoreXMLDefs);
-    }
-  }
-
-  private void addCoreProperty(Map<String,String> propMap, SolrResourceLoader loader, Node node, String name,
-                               String value) {
-    addCoreProperty(propMap, loader, node, name, value, null);
-  }
-
-  private void addCoreProperty(Map<String,String> propMap, SolrResourceLoader loader, Node node, String name,
-      String value, String defaultValue) {
-
-    if (node == null) {
-      propMap.put(name, value);
-      return;
-    }
-
-    if (node != null) {
-      String rawAttribValue = DOMUtil.getAttr(node, name, null);
-
-      if (rawAttribValue == null) {
-        return; // It was never in the original definition.
-      }
-
-      if (value == null) {
-        propMap.put(name, rawAttribValue);
-        return;
-      }
-
-      // There are some _really stupid_ additions/subtractions of the slash that we should look out for. I'm (EOE)
-      // ashamed of this but it fixes some things and we're throwing persistence away anyway (although
-      // maybe not for core.properties files).
-      String defComp = regularizeAttr(defaultValue);
-
-      if (defComp != null && regularizeAttr(value).equals(defComp)) {
-        return;
-      }
-      String rawComp = regularizeAttr(rawAttribValue);
-      if (rawComp != null && regularizeAttr(value).equals(
-          regularizeAttr(DOMUtil.substituteProperty(rawAttribValue, loader.getCoreProperties())))) {
-        propMap.put(name, rawAttribValue);
-      } else {
-        propMap.put(name, value);
-      }
-    }
-  }
-
-  protected String regularizeAttr(String path) {
-    if (path == null)
-      return null;
-    path = path.replace('/', File.separatorChar);
-    path = path.replace('\\', File.separatorChar);
-    if (path.endsWith(File.separator)) {
-      path = path.substring(0, path.length() - 1);
-    }
-    return path;
-  }
-  protected void addCoreToPersistList(Config cfg, SolrResourceLoader loader,
-      CoreDescriptor dcore, String origCoreName,
-      List<SolrCoreXMLDef> solrCoreXMLDefs) throws XPathExpressionException {
-
-    Map<String,String> coreAttribs = new HashMap<String,String>();
-    Properties newProps = new Properties();
-
-    // This is simple, just take anything sent in and saved away in at core creation and write it out.
-    if (dcore.getCreatedProperties().size() > 0) {
-      final List<String> stdNames = new ArrayList<String>(Arrays.asList(CoreDescriptor.standardPropNames));
-      coreAttribs.put(CoreDescriptor.CORE_NAME, dcore.getName()); // NOTE: may have been swapped or renamed!
-      for (String key : dcore.getCreatedProperties().stringPropertyNames()) {
-        if (! stdNames.contains(key) && ! key.startsWith(CoreAdminParams.PROPERTY_PREFIX)) continue;
-        if (key.indexOf(CoreAdminParams.PROPERTY_PREFIX) == 0) {
-          newProps.put(key.substring(CoreAdminParams.PROPERTY_PREFIX.length()), dcore.getCreatedProperties().getProperty(key));
-        } else if (! CoreDescriptor.CORE_NAME.equals(key)) {
-          coreAttribs.put(key, dcore.getCreatedProperties().getProperty(key));
-        }
-      }
-      // Insure instdir is persisted if it's the default since it's checked at startup even if not specified on the
-      // create command.
-      if (! dcore.getCreatedProperties().containsKey(CoreDescriptor.CORE_INSTDIR)) {
-        coreAttribs.put(CoreDescriptor.CORE_INSTDIR, dcore.getRawInstanceDir());
-      }
-    } else {
-
-      String coreName = dcore.getProperty(CoreDescriptor.CORE_NAME);
-
-      CloudDescriptor cd = dcore.getCloudDescriptor();
-      String collection = null;
-      if (cd != null) collection = cd.getCollectionName();
-
-      if (origCoreName == null) {
-        origCoreName = coreName;
-      }
-
-      Node node = null;
-      if (cfg != null) {
-        node = cfg.getNode("/solr/cores/core[@name='" + origCoreName + "']",
-            false);
-      }
-
-      coreAttribs.put(CoreDescriptor.CORE_NAME, coreName);
-      //coreAttribs.put(CoreDescriptor.CORE_INSTDIR, dcore.getRawInstanceDir());
-      addCoreProperty(coreAttribs, loader, node, CoreDescriptor.CORE_INSTDIR, dcore.getRawInstanceDir(), null);
-
-      addCoreProperty(coreAttribs, loader, node, CoreDescriptor.CORE_COLLECTION,
-          StringUtils.isNotBlank(collection) ? collection : dcore.getName());
-
-      addCoreProperty(coreAttribs, loader, node, CoreDescriptor.CORE_DATADIR,
-          dcore.getDataDir());
-      addCoreProperty(coreAttribs, loader, node, CoreDescriptor.CORE_ULOGDIR,
-          dcore.getUlogDir());
-      addCoreProperty(coreAttribs, loader, node, CoreDescriptor.CORE_TRANSIENT,
-          Boolean.toString(dcore.isTransient()));
-      addCoreProperty(coreAttribs, loader, node, CoreDescriptor.CORE_LOADONSTARTUP,
-          Boolean.toString(dcore.isLoadOnStartup()));
-      addCoreProperty(coreAttribs, loader, node, CoreDescriptor.CORE_CONFIG,
-          dcore.getConfigName());
-      addCoreProperty(coreAttribs, loader, node, CoreDescriptor.CORE_SCHEMA,
-          dcore.getSchemaName());
-
-      addCoreProperty(coreAttribs, loader, node, CoreDescriptor.CORE_COLLECTION,
-          collection, dcore.getName());
-
-      String shard = null;
-      String roles = null;
-      String node_name = null;
-      if (cd != null) {
-        shard = cd.getShardId();
-        roles = cd.getRoles();
-        node_name = cd.getCoreNodeName();
-      }
-      addCoreProperty(coreAttribs, loader, node, CoreDescriptor.CORE_SHARD,
-          shard);
-
-      addCoreProperty(coreAttribs, loader, node, CoreDescriptor.CORE_ROLES,
-          roles);
-
-      addCoreProperty(coreAttribs, loader, node, CoreDescriptor.CORE_NODE_NAME,
-          node_name);
-
-
-      for (Object key : dcore.getCoreProperties().keySet()) {
-
-        if (cfg != null) {
-          Node propNode = cfg.getNode("/solr/cores/core[@name='" + origCoreName + "']/property[@name='" + key + "']",
-              false);
-
-          if (propNode != null) { // This means it was in the original DOM element, so just copy it.
-            newProps.put(DOMUtil.getAttr(propNode, "name", null), DOMUtil.getAttr(propNode, "value", null));
-          }
-        }
-      }
-    }
-
-
-
-    SolrXMLSerializer.SolrCoreXMLDef solrCoreXMLDef = new SolrXMLSerializer.SolrCoreXMLDef();
-    solrCoreXMLDef.coreAttribs = coreAttribs;
-    solrCoreXMLDef.coreProperties = newProps;
-    solrCoreXMLDefs.add(solrCoreXMLDef);
-  }
-
   protected Object getModifyLock() {
     return modifyLock;
   }
@@ -604,4 +393,37 @@ class SolrCores {
       createdCores.put(core.getName(), core);
     }
   }
+
+  /**
+   * Return the CoreDescriptor corresponding to a given core name.
+   * @param coreName the name of the core
+   * @return the CoreDescriptor
+   */
+  public CoreDescriptor getCoreDescriptor(String coreName) {
+    synchronized (modifyLock) {
+      if (cores.containsKey(coreName))
+        return cores.get(coreName).getCoreDescriptor();
+      if (dynamicDescriptors.containsKey(coreName))
+        return dynamicDescriptors.get(coreName);
+      return null;
+    }
+  }
+
+  /**
+   * Get the CoreDescriptors for every SolrCore managed here
+   * @return a List of CoreDescriptors
+   */
+  public List<CoreDescriptor> getCoreDescriptors() {
+    List<CoreDescriptor> cds = Lists.newArrayList();
+    synchronized (modifyLock) {
+      for (String coreName : getAllCoreNames()) {
+        // TODO: This null check is a bit suspicious - it seems that
+        // getAllCoreNames might return deleted cores as well?
+        CoreDescriptor cd = getCoreDescriptor(coreName);
+        if (cd != null)
+          cds.add(cd);
+      }
+    }
+    return cds;
+  }
 }