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:35:02 UTC

svn commit: r1502276 [2/3] - in /lucene/dev/trunk/solr: ./ contrib/dataimporthandler/src/test-files/ core/src/java/org/apache/solr/cloud/ core/src/java/org/apache/solr/core/ core/src/java/org/apache/solr/handler/admin/ core/src/java/org/apache/solr/uti...

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/SolrCores.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/SolrCores.java?rev=1502276&r1=1502275&r2=1502276&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/SolrCores.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/SolrCores.java Thu Jul 11 16:35:01 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;
+  }
 }

Added: lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/SolrXMLCoresLocator.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/SolrXMLCoresLocator.java?rev=1502276&view=auto
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/SolrXMLCoresLocator.java (added)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/SolrXMLCoresLocator.java Thu Jul 11 16:35:01 2013
@@ -0,0 +1,221 @@
+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 com.google.common.base.Charsets;
+import com.google.common.collect.ImmutableList;
+import org.apache.commons.lang.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.OutputStreamWriter;
+import java.io.Writer;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+/**
+ * Writes any changes in core definitions to this instance's solr.xml
+ */
+public class SolrXMLCoresLocator implements CoresLocator {
+
+  private static final Logger logger = LoggerFactory.getLogger(SolrXMLCoresLocator.class);
+
+  private final File file;
+  private final String solrXmlTemplate;
+  private final ConfigSolrXmlOld cfg;
+
+  /** Core name to use if a core definition has no name */
+  public static final String DEFAULT_CORE_NAME = "collection1";
+
+  /**
+   * Create a new SolrXMLCoresLocator
+   * @param file          a File object representing the file to write out to
+   * @param originalXML   the original content of the solr.xml file
+   * @param cfg           the CoreContainer's config object
+   */
+  public SolrXMLCoresLocator(File file, String originalXML, ConfigSolrXmlOld cfg) {
+    this.solrXmlTemplate = buildTemplate(originalXML);
+    this.file = file;
+    this.cfg = cfg;
+  }
+
+  private static Pattern POPULATED_CORES_TAG
+      = Pattern.compile("^(.*<cores[^>]*>)(.*)(</cores>.*)$", Pattern.DOTALL);
+  private static Pattern EMPTY_CORES_TAG
+      = Pattern.compile("^(.*<cores[^>]*)/>(.*)$", Pattern.DOTALL);
+
+  private static Pattern SHARD_HANDLER_TAG
+      = Pattern.compile("(<shardHandlerFactory[^>]*>.*</shardHandlerFactory>)|(<shardHandlerFactory[^>]*/>)",
+                          Pattern.DOTALL);
+
+  private static String CORES_PLACEHOLDER = "{{CORES_PLACEHOLDER}}";
+
+  // Package-private for testing
+  // We replace the existing <cores></cores> contents with a template pattern
+  // that we can later replace with the up-to-date core definitions.  We also
+  // need to extract the <shardHandlerFactory> section, as, annoyingly, it's
+  // kept inside <cores/>.
+  static String buildTemplate(String originalXML) {
+
+    String shardHandlerConfig = "";
+    Matcher shfMatcher = SHARD_HANDLER_TAG.matcher(originalXML);
+    if (shfMatcher.find()) {
+      shardHandlerConfig = shfMatcher.group(0);
+    }
+
+    Matcher popMatcher = POPULATED_CORES_TAG.matcher(originalXML);
+    if (popMatcher.matches()) {
+      return new StringBuilder(popMatcher.group(1))
+          .append(CORES_PLACEHOLDER).append(shardHandlerConfig).append(popMatcher.group(3)).toString();
+    }
+
+    // Self-closing <cores/> tag gets expanded to <cores></cores>
+    Matcher emptyMatcher = EMPTY_CORES_TAG.matcher(originalXML);
+    if (emptyMatcher.matches())
+      return new StringBuilder(emptyMatcher.group(1))
+          .append(">").append(CORES_PLACEHOLDER).append("</cores>")
+          .append(emptyMatcher.group(2)).toString();
+
+    // If there's no <cores> tag at all, add one at the end of the file
+    return originalXML.replace("</solr>", "<cores>" + CORES_PLACEHOLDER + "</cores></solr>");
+  }
+
+  // protected access for testing
+  protected String buildSolrXML(List<CoreDescriptor> cds) {
+    StringBuilder builder = new StringBuilder();
+    for (CoreDescriptor cd : cds) {
+      builder.append(buildCoreTag(cd));
+    }
+    return solrXmlTemplate.replace(CORES_PLACEHOLDER, builder.toString());
+  }
+
+  public static final String NEWLINE = System.getProperty("line.separator");
+  public static final String INDENT = "    ";
+
+  /**
+   * Serialize a coredescriptor as a String containing an XML &lt;core> tag.
+   * @param cd the CoreDescriptor
+   * @return an XML representation of the CoreDescriptor
+   */
+  protected static String buildCoreTag(CoreDescriptor cd) {
+
+    StringBuilder builder = new StringBuilder(NEWLINE).append(INDENT).append("<core");
+    for (Map.Entry<Object, Object> entry : cd.getPersistableStandardProperties().entrySet()) {
+      builder.append(" ").append(entry.getKey()).append("=\"").append(entry.getValue()).append("\"");
+    }
+
+    Properties userProperties = cd.getPersistableUserProperties();
+    if (userProperties.isEmpty()) {
+      return builder.append("/>").append(NEWLINE).toString();
+    }
+
+    builder.append(">").append(NEWLINE);
+    for (Map.Entry<Object, Object> entry : userProperties.entrySet()) {
+      builder.append(INDENT).append(INDENT)
+          .append("<property name=\"").append(entry.getKey()).append("\" value=\"")
+          .append(entry.getValue()).append("\"/>").append(NEWLINE);
+    }
+
+    return builder.append("</core>").append(NEWLINE).toString();
+
+  }
+
+  @Override
+  public final void persist(CoreContainer cc, CoreDescriptor... coreDescriptors) {
+    doPersist(buildSolrXML(cc.getCoreDescriptors()));
+  }
+
+  protected void doPersist(String xml) {
+    try {
+      Writer writer = new OutputStreamWriter(new FileOutputStream(file), Charsets.UTF_8);
+      writer.write(xml);
+      writer.close();
+      logger.info("Persisted core descriptions to {}", file.getAbsolutePath());
+    }
+    catch (IOException e) {
+      logger.error("Couldn't persist core descriptions to {} : {}", file.getAbsolutePath(), e);
+    }
+  }
+
+  @Override
+  public void create(CoreContainer cc, CoreDescriptor... coreDescriptors) {
+    this.persist(cc, coreDescriptors);
+  }
+
+  @Override
+  public void delete(CoreContainer cc, CoreDescriptor... coreDescriptors) {
+    this.persist(cc, coreDescriptors);
+  }
+
+  @Override
+  public void rename(CoreContainer cc, CoreDescriptor oldCD, CoreDescriptor newCD) {
+    this.persist(cc, oldCD, newCD);
+  }
+
+  @Override
+  public List<CoreDescriptor> discover(CoreContainer cc) {
+
+    ImmutableList.Builder<CoreDescriptor> listBuilder = ImmutableList.builder();
+
+    for (String coreName : cfg.getAllCoreNames()) {
+
+      String name = cfg.getProperty(coreName, CoreDescriptor.CORE_NAME, DEFAULT_CORE_NAME);
+      String instanceDir = cfg.getProperty(coreName, CoreDescriptor.CORE_INSTDIR, "");
+
+      Properties coreProperties = new Properties();
+      for (String propName : CoreDescriptor.standardPropNames) {
+        String propValue = cfg.getProperty(coreName, propName, "");
+        if (StringUtils.isNotEmpty(propValue))
+          coreProperties.setProperty(propName, propValue);
+      }
+      coreProperties.putAll(cfg.getCoreProperties(coreName));
+
+      listBuilder.add(new CoreDescriptor(cc, name, instanceDir, coreProperties));
+    }
+
+    return listBuilder.build();
+  }
+
+  // for testing
+  String getTemplate() {
+    return solrXmlTemplate;
+  }
+
+  public static class NonPersistingLocator extends SolrXMLCoresLocator {
+
+    public NonPersistingLocator(File file, String originalXML, ConfigSolrXmlOld cfg) {
+      super(file, originalXML, cfg);
+      this.xml = originalXML;
+    }
+
+    @Override
+    public void doPersist(String xml) {
+      this.xml = xml;
+    }
+
+    public String xml;
+
+  }
+
+}

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/ZkContainer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/ZkContainer.java?rev=1502276&r1=1502275&r2=1502276&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/ZkContainer.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/ZkContainer.java Thu Jul 11 16:35:01 2013
@@ -203,7 +203,7 @@ public class ZkContainer {
 
         
         if(boostrapConf) {
-          ZkController.bootstrapConf(zkController.getZkClient(), cc.cfg, solrHome);
+          ZkController.bootstrapConf(zkController.getZkClient(), cc, solrHome);
         }
         
       } catch (InterruptedException e) {
@@ -249,8 +249,7 @@ public class ZkContainer {
             "Could not find config name for collection:" + collection);
       }
       solrLoader = new ZkSolrResourceLoader(instanceDir, zkConfigName,
-          loader.getClassLoader(), ConfigSolrXml.getCoreProperties(instanceDir,
-              dcore), zkController);
+          loader.getClassLoader(), dcore.getCoreProperties(), zkController);
       config = getSolrConfigFromZk(zkConfigName, dcore.getConfigName(),
           solrLoader);
       schema = IndexSchemaFactory.buildIndexSchema(dcore.getSchemaName(),

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminHandler.java?rev=1502276&r1=1502275&r2=1502276&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminHandler.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminHandler.java Thu Jul 11 16:35:01 2013
@@ -17,6 +17,7 @@
 
 package org.apache.solr.handler.admin;
 
+import com.google.common.collect.ImmutableMap;
 import org.apache.commons.io.FileUtils;
 import org.apache.commons.lang.StringUtils;
 import org.apache.lucene.index.DirectoryReader;
@@ -57,7 +58,6 @@ import org.apache.solr.update.UpdateLog;
 import org.apache.solr.update.processor.UpdateRequestProcessor;
 import org.apache.solr.update.processor.UpdateRequestProcessorChain;
 import org.apache.solr.util.NumberUtils;
-import org.apache.solr.util.PropertiesUtil;
 import org.apache.solr.util.RefCounted;
 import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;
@@ -127,7 +127,7 @@ public class CoreAdminHandler extends Re
       throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
               "Core container instance missing");
     }
-    boolean doPersist = false;
+    //boolean doPersist = false;
 
     // Pick the action
     SolrParams params = req.getParams();
@@ -136,54 +136,54 @@ public class CoreAdminHandler extends Re
     if (a != null) {
       action = CoreAdminAction.get(a);
       if (action == null) {
-        doPersist = this.handleCustomAction(req, rsp);
+        this.handleCustomAction(req, rsp);
       }
     }
     if (action != null) {
       switch (action) {
         case CREATE: {
-          doPersist = this.handleCreateAction(req, rsp);
+          this.handleCreateAction(req, rsp);
           break;
         }
 
         case RENAME: {
-          doPersist = this.handleRenameAction(req, rsp);
+          this.handleRenameAction(req, rsp);
           break;
         }
 
         case UNLOAD: {
-          doPersist = this.handleUnloadAction(req, rsp);
+          this.handleUnloadAction(req, rsp);
           break;
         }
 
         case STATUS: {
-          doPersist = this.handleStatusAction(req, rsp);
+          this.handleStatusAction(req, rsp);
           break;
 
         }
 
         case PERSIST: {
-          doPersist = this.handlePersistAction(req, rsp);
+          this.handlePersistAction(req, rsp);
           break;
         }
 
         case RELOAD: {
-          doPersist = this.handleReloadAction(req, rsp);
+          this.handleReloadAction(req, rsp);
           break;
         }
 
         case SWAP: {
-          doPersist = this.handleSwapAction(req, rsp);
+          this.handleSwapAction(req, rsp);
           break;
         }
 
         case MERGEINDEXES: {
-          doPersist = this.handleMergeAction(req, rsp);
+          this.handleMergeAction(req, rsp);
           break;
         }
 
         case SPLIT: {
-          doPersist = this.handleSplitAction(req, rsp);
+          this.handleSplitAction(req, rsp);
           break;
         }
 
@@ -209,28 +209,21 @@ public class CoreAdminHandler extends Re
         }
         
         default: {
-          doPersist = this.handleCustomAction(req, rsp);
+          this.handleCustomAction(req, rsp);
           break;
         }
         case LOAD:
           break;
       }
     }
-    // Should we persist the changes?
-    if (doPersist) {
-      cores.persist();
-      rsp.add("saved", cores.getConfigFile().getAbsolutePath());
-    }
     rsp.setHttpCaching(false);
   }
 
   
   /**
    * Handle the core admin SPLIT action.
-   * @return true if a modification has resulted that requires persistence 
-   *         of the CoreContainer configuration.
    */
-  protected boolean handleSplitAction(SolrQueryRequest adminReq, SolrQueryResponse rsp) throws IOException {
+  protected void handleSplitAction(SolrQueryRequest adminReq, SolrQueryResponse rsp) throws IOException {
     SolrParams params = adminReq.getParams();
     List<DocRouter.Range> ranges = null;
 
@@ -299,11 +292,10 @@ public class CoreAdminHandler extends Re
       }
     }
 
-    return false;
   }
 
 
-  protected boolean handleMergeAction(SolrQueryRequest req, SolrQueryResponse rsp) throws IOException {
+  protected void handleMergeAction(SolrQueryRequest req, SolrQueryResponse rsp) throws IOException {
     SolrParams params = req.getParams();
     String cname = params.required().get(CoreAdminParams.CORE);
     SolrCore core = coreContainer.getCore(cname);
@@ -386,7 +378,6 @@ public class CoreAdminHandler extends Re
         core.close();
       }
     }
-    return coreContainer.isPersistent();
   }
 
   /**
@@ -395,140 +386,93 @@ public class CoreAdminHandler extends Re
    * This method could be overridden by derived classes to handle custom actions. <br> By default - this method throws a
    * solr exception. Derived classes are free to write their derivation if necessary.
    */
-  protected boolean handleCustomAction(SolrQueryRequest req, SolrQueryResponse rsp) {
+  protected void handleCustomAction(SolrQueryRequest req, SolrQueryResponse rsp) {
     throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Unsupported operation: " +
             req.getParams().get(CoreAdminParams.ACTION));
   }
 
+  public static ImmutableMap<String, String> paramToProp = ImmutableMap.<String, String>builder()
+      .put(CoreAdminParams.CONFIG, CoreDescriptor.CORE_CONFIG)
+      .put(CoreAdminParams.SCHEMA, CoreDescriptor.CORE_SCHEMA)
+      .put(CoreAdminParams.DATA_DIR, CoreDescriptor.CORE_DATADIR)
+      .put(CoreAdminParams.ULOG_DIR, CoreDescriptor.CORE_ULOGDIR)
+      .put(CoreAdminParams.LOAD_ON_STARTUP, CoreDescriptor.CORE_LOADONSTARTUP)
+      .put(CoreAdminParams.TRANSIENT, CoreDescriptor.CORE_TRANSIENT)
+      .put(CoreAdminParams.SHARD, CoreDescriptor.CORE_SHARD)
+      .put(CoreAdminParams.COLLECTION, CoreDescriptor.CORE_COLLECTION)
+      .put(CoreAdminParams.ROLES, CoreDescriptor.CORE_ROLES)
+      .put(CoreAdminParams.CORE_NODE_NAME, CoreDescriptor.CORE_NODE_NAME)
+      .put(CoreAdminParams.SHARD_STATE, CloudDescriptor.SHARD_STATE)
+      .put(CoreAdminParams.SHARD_RANGE, CloudDescriptor.SHARD_RANGE)
+      .put(ZkStateReader.NUM_SHARDS_PROP, CloudDescriptor.NUM_SHARDS)
+      .build();
+
+  public static ImmutableMap<String, String> cloudParamToProp;
+
+  protected static CoreDescriptor buildCoreDescriptor(SolrParams params, CoreContainer container) {
+
+    String name = checkNotEmpty(params.get(CoreAdminParams.NAME),
+        "Missing parameter [" + CoreAdminParams.NAME + "]");
+    String instancedir = params.get(CoreAdminParams.INSTANCE_DIR);
+    if (StringUtils.isEmpty(instancedir))
+      instancedir = container.getSolrHome() + File.separator + name;
+
+    Properties coreProps = new Properties();
+    for (String param : paramToProp.keySet()) {
+      String value = params.get(param, null);
+      if (StringUtils.isNotEmpty(value)) {
+        coreProps.setProperty(paramToProp.get(param), value);
+      }
+    }
+    Iterator<String> paramsIt = params.getParameterNamesIterator();
+    while (paramsIt.hasNext()) {
+      String param = paramsIt.next();
+      if (!param.startsWith(CoreAdminParams.PROPERTY_PREFIX))
+        continue;
+      String propName = param.substring(CoreAdminParams.PROPERTY_PREFIX.length());
+      String propValue = params.get(param);
+      coreProps.setProperty(propName, propValue);
+    }
+
+    return new CoreDescriptor(container, name, instancedir, coreProps);
+  }
+
+  private static String checkNotEmpty(String value, String message) {
+    if (StringUtils.isEmpty(value))
+      throw new SolrException(ErrorCode.BAD_REQUEST, message);
+    return value;
+  }
+
   /**
    * Handle 'CREATE' action.
    *
-   * @return true if a modification has resulted that requires persistance 
-   *         of the CoreContainer configuration.
-   *
    * @throws SolrException in case of a configuration error.
    */
-  protected boolean handleCreateAction(SolrQueryRequest req, SolrQueryResponse rsp) throws SolrException {
+  protected void handleCreateAction(SolrQueryRequest req, SolrQueryResponse rsp) throws SolrException {
+
     SolrParams params = req.getParams();
-    String name = params.get(CoreAdminParams.NAME);
-    if (null == name || "".equals(name)) {
-      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
-                              "Core name is mandatory to CREATE a SolrCore");
+    CoreDescriptor dcore = buildCoreDescriptor(params, coreContainer);
+
+    if (coreContainer.getAllCoreNames().contains(dcore.getName())) {
+      log.warn("Creating a core with existing name is not allowed");
+      throw new SolrException(ErrorCode.SERVER_ERROR,
+          "Core with name '" + dcore.getName() + "' already exists.");
     }
 
-    CoreDescriptor dcore = null;
+    // TODO this should be moved into CoreContainer, really...
     try {
-      
-      if (coreContainer.getAllCoreNames().contains(name)) {
-        log.warn("Creating a core with existing name is not allowed");
-        throw new SolrException(ErrorCode.SERVER_ERROR,
-            "Core with name '" + name + "' already exists.");
-      }
-
-      String instanceDir = params.get(CoreAdminParams.INSTANCE_DIR);
-      if (instanceDir == null) {
-        // instanceDir = coreContainer.getSolrHome() + "/" + name;
-        instanceDir = name; // bare name is already relative to solr home
-      } else {
-        instanceDir = PropertiesUtil.substituteProperty(instanceDir, null);
-      }
-
-      dcore = new CoreDescriptor(coreContainer, name, instanceDir);
-
-      //  fillup optional parameters
-      String opts = params.get(CoreAdminParams.CONFIG);
-      if (opts != null) {
-        opts = PropertiesUtil.substituteProperty(opts, null);
-        dcore.setConfigName(opts);
-      }
-
-      opts = params.get(CoreAdminParams.SCHEMA);
-      if (opts != null) {
-        opts = PropertiesUtil.substituteProperty(opts, null);
-        dcore.setSchemaName(opts);
-      }
-
-      opts = params.get(CoreAdminParams.DATA_DIR);
-      if (opts != null) {
-        opts = PropertiesUtil.substituteProperty(opts, null);
-        dcore.setDataDir(opts);
-      }
-
-      opts = params.get(CoreAdminParams.ULOG_DIR);
-      if (opts != null)
-        dcore.setUlogDir(opts);
-
-      opts = params.get(CoreAdminParams.LOAD_ON_STARTUP);
-      if (opts != null){
-        Boolean value = Boolean.valueOf(opts);
-        dcore.setLoadOnStartup(value);
-      }
-      
-      opts = params.get(CoreAdminParams.TRANSIENT);
-      if (opts != null){
-        Boolean value = Boolean.valueOf(opts);
-        dcore.setTransient(value);
-      }
-      
-      CloudDescriptor cd = dcore.getCloudDescriptor();
-      if (cd != null) {
-        cd.setParams(req.getParams());
-
-        opts = params.get(CoreAdminParams.COLLECTION);
-        if (opts != null)
-          cd.setCollectionName(opts);
-        
-        opts = params.get(CoreAdminParams.SHARD);
-        if (opts != null)
-          cd.setShardId(opts);
-        
-        opts = params.get(CoreAdminParams.SHARD_RANGE);
-        if (opts != null)
-          cd.setShardRange(opts);
-
-        opts = params.get(CoreAdminParams.SHARD_STATE);
-        if (opts != null)
-          cd.setShardState(opts);
-        
-        opts = params.get(CoreAdminParams.ROLES);
-        if (opts != null)
-          cd.setRoles(opts);
-        
-        opts = params.get(CoreAdminParams.CORE_NODE_NAME);
-        if (opts != null)
-          cd.setCoreNodeName(opts);
-                        
-        Integer numShards = params.getInt(ZkStateReader.NUM_SHARDS_PROP);
-        if (numShards != null)
-          cd.setNumShards(numShards);
-      }
-      
-      // Process all property.name=value parameters and set them as name=value core properties
-      Properties coreProperties = new Properties();
-      Iterator<String> parameterNamesIterator = params.getParameterNamesIterator();
-      while (parameterNamesIterator.hasNext()) {
-          String parameterName = parameterNamesIterator.next();
-          String parameterValue = params.get(parameterName);
-          dcore.addCreatedProperty(parameterName, parameterValue); // Need this junk for silly persistence
-
-          if(parameterName.startsWith(CoreAdminParams.PROPERTY_PREFIX)) {
-            String propertyName = parameterName.substring(CoreAdminParams.PROPERTY_PREFIX.length()); // skip prefix
-            coreProperties.put(propertyName, parameterValue);
-          }
-      }
-      dcore.setCoreProperties(coreProperties);
       if (coreContainer.getZkController() != null) {
         coreContainer.preRegisterInZk(dcore);
       }
+      coreContainer.getCoresLocator().create(coreContainer, dcore);
       SolrCore core = coreContainer.create(dcore);
-
-      coreContainer.register(name, core, false);
+      coreContainer.register(dcore.getName(), core, false);
       rsp.add("core", core.getName());
-      return coreContainer.isPersistent();
-    } catch (Exception ex) {
+    }
+    catch (Exception ex) {
       if (coreContainer.isZooKeeperAware() && dcore != null) {
         try {
-          coreContainer.getZkController().unregister(name, dcore);
+          coreContainer.getZkController().unregister(dcore.getName(), dcore);
         } catch (InterruptedException e) {
           Thread.currentThread().interrupt();
           SolrException.log(log, null, e);
@@ -552,46 +496,37 @@ public class CoreAdminHandler extends Re
       }
       
       throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
-                              "Error CREATEing SolrCore '" + name + "': " +
-                              ex.getMessage() + rootMsg, ex);
+                              "Error CREATEing SolrCore '" + dcore.getName() + "': " +
+                              ex.getMessage(), ex);
     }
   }
 
   /**
    * Handle "RENAME" Action
-   *
-   * @return true if a modification has resulted that requires persistence 
-   *         of the CoreContainer configuration.
    */
-  protected boolean handleRenameAction(SolrQueryRequest req, SolrQueryResponse rsp) throws SolrException {
+  protected void handleRenameAction(SolrQueryRequest req, SolrQueryResponse rsp) throws SolrException {
     SolrParams params = req.getParams();
 
     String name = params.get(CoreAdminParams.OTHER);
     String cname = params.get(CoreAdminParams.CORE);
-    boolean doPersist = false;
 
-    if (cname.equals(name)) return doPersist;
-    
-    doPersist = coreContainer.isPersistent();
+    if (cname.equals(name)) return;
+
     coreContainer.rename(cname, name);
-    
-    return doPersist;
+
   }
 
   /**
    * Handle "ALIAS" action
-   *
-   * @return true if a modification has resulted that requires persistance 
-   *         of the CoreContainer configuration.
    */
   @Deprecated
-  protected boolean handleAliasAction(SolrQueryRequest req, SolrQueryResponse rsp) {
+  protected void handleAliasAction(SolrQueryRequest req, SolrQueryResponse rsp) {
     SolrParams params = req.getParams();
 
     String name = params.get(CoreAdminParams.OTHER);
     String cname = params.get(CoreAdminParams.CORE);
     boolean doPersist = false;
-    if (cname.equals(name)) return doPersist;
+    if (cname.equals(name)) return;
 
     SolrCore core = coreContainer.getCore(cname);
     if (core != null) {
@@ -599,17 +534,14 @@ public class CoreAdminHandler extends Re
       coreContainer.register(name, core, false);
       // no core.close() since each entry in the cores map should increase the ref
     }
-    return doPersist;
+    return;
   }
 
 
   /**
    * Handle "UNLOAD" Action
-   *
-   * @return true if a modification has resulted that requires persistance 
-   *         of the CoreContainer configuration.
    */
-  protected boolean handleUnloadAction(SolrQueryRequest req,
+  protected void handleUnloadAction(SolrQueryRequest req,
       SolrQueryResponse rsp) throws SolrException {
     SolrParams params = req.getParams();
     String cname = params.get(CoreAdminParams.CORE);
@@ -687,17 +619,13 @@ public class CoreAdminHandler extends Re
         core.close();
       }
     }
-    return coreContainer.isPersistent();
     
   }
 
   /**
    * Handle "STATUS" action
-   *
-   * @return true if a modification has resulted that requires persistance 
-   *         of the CoreContainer configuration.
    */
-  protected boolean handleStatusAction(SolrQueryRequest req, SolrQueryResponse rsp)
+  protected void handleStatusAction(SolrQueryRequest req, SolrQueryResponse rsp)
           throws SolrException {
     SolrParams params = req.getParams();
 
@@ -722,8 +650,6 @@ public class CoreAdminHandler extends Re
         status.add(cname, getCoreStatus(coreContainer, cname, isIndexInfoNeeded));
       }
       rsp.add("status", status);
-      doPersist = false; // no state change
-      return doPersist;
     } catch (Exception ex) {
       throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
               "Error handling 'status' action ", ex);
@@ -732,42 +658,20 @@ public class CoreAdminHandler extends Re
 
   /**
    * Handler "PERSIST" action
-   *
-   * @return true if a modification has resulted that requires persistence 
-   *         of the CoreContainer configuration.
    */
-  protected boolean handlePersistAction(SolrQueryRequest req, SolrQueryResponse rsp)
+  protected void handlePersistAction(SolrQueryRequest req, SolrQueryResponse rsp)
           throws SolrException {
-    SolrParams params = req.getParams();
-    boolean doPersist = false;
-    String fileName = params.get(CoreAdminParams.FILE);
-    if (fileName != null) {
-      File file = new File(fileName);
-      if (!file.isAbsolute())
-        file = new File(coreContainer.getConfigFile().getParentFile(), fileName);
-      coreContainer.persistFile(file);
-      rsp.add("saved", file.getAbsolutePath());
-      doPersist = false;
-    } else if (!coreContainer.isPersistent()) {
-      throw new SolrException(SolrException.ErrorCode.FORBIDDEN, "Persistence is not enabled");
-    } else
-      doPersist = true;
-
-    return doPersist;
+    rsp.add("message", "The PERSIST action has been deprecated");
   }
 
   /**
    * Handler "RELOAD" action
-   *
-   * @return true if a modification has resulted that requires persistence 
-   *         of the CoreContainer configuration.
    */
-  protected boolean handleReloadAction(SolrQueryRequest req, SolrQueryResponse rsp) {
+  protected void handleReloadAction(SolrQueryRequest req, SolrQueryResponse rsp) {
     SolrParams params = req.getParams();
     String cname = params.get(CoreAdminParams.CORE);
     try {
       coreContainer.reload(cname);
-      return false; // no change on reload
     } catch (Exception ex) {
       throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Error handling 'reload' action", ex);
     }
@@ -775,19 +679,14 @@ public class CoreAdminHandler extends Re
 
   /**
    * Handle "SWAP" action
-   *
-   * @return true if a modification has resulted that requires persistence 
-   *         of the CoreContainer configuration.
    */
-  protected boolean handleSwapAction(SolrQueryRequest req, SolrQueryResponse rsp) {
+  protected void handleSwapAction(SolrQueryRequest req, SolrQueryResponse rsp) {
     final SolrParams params = req.getParams();
     final SolrParams required = params.required();
 
     final String cname = params.get(CoreAdminParams.CORE);
-    boolean doPersist = params.getBool(CoreAdminParams.PERSISTENT, coreContainer.isPersistent());
     String other = required.get(CoreAdminParams.OTHER);
     coreContainer.swap(cname, other);
-    return doPersist;
 
   }
   

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/util/PropertiesUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/util/PropertiesUtil.java?rev=1502276&r1=1502275&r2=1502276&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/util/PropertiesUtil.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/util/PropertiesUtil.java Thu Jul 11 16:35:01 2013
@@ -129,4 +129,24 @@ public class PropertiesUtil {
     }
   }
 
+  /**
+   * Parse the given String value as an integer.  If the string cannot
+   * be parsed, returns the default
+   * @param value    the value to parse
+   * @param defValue the default to return if the value cannot be parsed
+   * @return an integer version of the passed in value
+   */
+  public static Integer toInteger(String value, Integer defValue) {
+    try {
+      return Integer.parseInt(value);
+    }
+    catch (NumberFormatException e) {
+      return defValue;
+    }
+  }
+
+  public static boolean toBoolean(String value) {
+    return "true".equalsIgnoreCase(value) || "on".equalsIgnoreCase(value);
+  }
+
 }

Modified: lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/ClusterStateUpdateTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/ClusterStateUpdateTest.java?rev=1502276&r1=1502275&r2=1502276&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/ClusterStateUpdateTest.java (original)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/ClusterStateUpdateTest.java Thu Jul 11 16:35:01 2013
@@ -150,14 +150,8 @@ public class ClusterStateUpdateTest exte
         CreateMode.PERSISTENT, true);
     zkClient.close();
 
-    CoreDescriptor dcore = new CoreDescriptor(container1, "testcore",
-        "testcore");
-    
-    dcore.setDataDir(dataDir4.getAbsolutePath());
-    
-    CloudDescriptor cloudDesc = new CloudDescriptor();
-    cloudDesc.setCollectionName("testcore");
-    dcore.setCloudDescriptor(cloudDesc);
+    CoreDescriptor dcore = buildCoreDescriptor(container1, "testcore", "testcore")
+                              .withDataDir(dataDir4.getAbsolutePath()).build();
 
     if (container1.getZkController() != null) {
       container1.preRegisterInZk(dcore);

Modified: lucene/dev/trunk/solr/core/src/test/org/apache/solr/core/TestCoreContainer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/core/TestCoreContainer.java?rev=1502276&r1=1502275&r2=1502276&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/core/TestCoreContainer.java (original)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/core/TestCoreContainer.java Thu Jul 11 16:35:01 2013
@@ -143,118 +143,7 @@ public class TestCoreContainer extends S
 
   }
 
-  @Test
-  public void testPersist() throws Exception {
-    final File workDir = new File(TEMP_DIR, this.getClass().getName()
-        + "_persist");
-    if (workDir.exists()) {
-      FileUtils.deleteDirectory(workDir);
-    }
-    assertTrue("Failed to mkdirs workDir", workDir.mkdirs());
-    
-    final CoreContainer cores = h.getCoreContainer();
-
-    cores.setPersistent(true); // is this needed since we make explicit calls?
-
-    String instDir = null;
-    {
-      SolrCore template = null;
-      try {
-        template = cores.getCore("collection1");
-        instDir = template.getCoreDescriptor().getRawInstanceDir();
-      } finally {
-        if (null != template) template.close();
-      }
-    }
-    
-    final File instDirFile = new File(cores.getSolrHome(), instDir);
-    assertTrue("instDir doesn't exist: " + instDir, instDirFile.exists());
-    
-    // sanity check the basic persistence of the default init
-    
-    final File oneXml = new File(workDir, "1.solr.xml");
-    cores.persistFile(oneXml);
-
-    assertXmlFile(oneXml, "/solr[@persistent='true']",
-        "/solr/cores[@defaultCoreName='collection1' and not(@transientCacheSize)]",
-        "/solr/cores/core[@name='collection1' and @instanceDir='" + instDir +
-        "' and @transient='false' and @loadOnStartup='true' ]", "1=count(/solr/cores/core)");
-
-    // create some new cores and sanity check the persistence
-    
-    final File dataXfile = new File(workDir, "dataX");
-    final String dataX = dataXfile.getAbsolutePath();
-    assertTrue("dataXfile mkdirs failed: " + dataX, dataXfile.mkdirs());
-    
-    final File instYfile = new File(workDir, "instY");
-    FileUtils.copyDirectory(instDirFile, instYfile);
-    
-    // :HACK: dataDir leaves off trailing "/", but instanceDir uses it
-    final String instY = instYfile.getAbsolutePath() + "/";
-    
-    final CoreDescriptor xd = new CoreDescriptor(cores, "X", instDir);
-    xd.setDataDir(dataX);
-    
-    final CoreDescriptor yd = new CoreDescriptor(cores, "Y", instY);
-    
-    SolrCore x = null;
-    SolrCore y = null;
-    try {
-      x = cores.create(xd);
-      y = cores.create(yd);
-      cores.register(x, false);
-      cores.register(y, false);
-      
-      assertEquals("cores not added?", 3, cores.getCoreNames().size());
-      
-      final File twoXml = new File(workDir, "2.solr.xml");
-
-      cores.persistFile(twoXml);
 
-      assertXmlFile(twoXml, "/solr[@persistent='true']",
-          "/solr/cores[@defaultCoreName='collection1']",
-          "/solr/cores/core[@name='collection1' and @instanceDir='" + instDir
-              + "']", "/solr/cores/core[@name='X' and @instanceDir='" + instDir
-              + "' and @dataDir='" + dataX + "']",
-          "/solr/cores/core[@name='Y' and @instanceDir='" + instY + "']",
-          "3=count(/solr/cores/core)");
-
-      // Test for saving implicit properties, we should not do this.
-      assertXmlFile(twoXml, "/solr/cores/core[@name='X' and not(@solr.core.instanceDir) and not (@solr.core.configName)]");
-
-      // delete a core, check persistence again
-      assertNotNull("removing X returned null", cores.remove("X"));
-      
-      final File threeXml = new File(workDir, "3.solr.xml");
-      cores.persistFile(threeXml);
-      
-      assertXmlFile(threeXml, "/solr[@persistent='true']",
-          "/solr/cores[@defaultCoreName='collection1']",
-          "/solr/cores/core[@name='collection1' and @instanceDir='" + instDir + "']",
-          "/solr/cores/core[@name='Y' and @instanceDir='" + instY + "']",
-          "2=count(/solr/cores/core)");
-      
-      // sanity check that persisting w/o changes has no changes
-      
-      final File fourXml = new File(workDir, "4.solr.xml");
-      cores.persistFile(fourXml);
-      
-      assertTrue("3 and 4 should be identical files",
-          FileUtils.contentEquals(threeXml, fourXml));
-      
-    } finally {
-      // y is closed by the container, but
-      // x has been removed from the container
-      if (x != null) {
-        try {
-          x.close();
-        } catch (Exception e) {
-          log.error("", e);
-        }
-      }
-    }
-  }
-  
 
   @Test
   public void testNoCores() throws IOException, ParserConfigurationException, SAXException {

Modified: lucene/dev/trunk/solr/core/src/test/org/apache/solr/core/TestCoreDiscovery.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/core/TestCoreDiscovery.java?rev=1502276&r1=1502275&r2=1502276&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/core/TestCoreDiscovery.java (original)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/core/TestCoreDiscovery.java Thu Jul 11 16:35:01 2013
@@ -91,7 +91,7 @@ public class TestCoreDiscovery extends S
 
   private void addCoreWithProps(String name, Properties stockProps) throws Exception {
 
-    File propFile = new File(new File(solrHomeDirectory, name), SolrCoreDiscoverer.CORE_PROP_FILE);
+    File propFile = new File(new File(solrHomeDirectory, name), CorePropertiesLocator.PROPERTIES_FILENAME);
     File parent = propFile.getParentFile();
     assertTrue("Failed to mkdirs for " + parent.getAbsolutePath(), parent.mkdirs());
     addCoreWithProps(stockProps, propFile);
@@ -144,17 +144,17 @@ public class TestCoreDiscovery extends S
 
       // Let's assert we did the right thing for implicit properties too.
       CoreDescriptor desc = core1.getCoreDescriptor();
-      assertEquals("core1", desc.getProperty("solr.core.name"));
-
-      // Prove we're ignoring this even though it's set in the properties file
-      assertFalse("InstanceDir should be ignored", desc.getProperty("solr.core.instanceDir").contains("totallybogus"));
+      assertEquals("core1", desc.getName());
 
       // This is too long and ugly to put in. Besides, it varies.
-      assertNotNull(desc.getProperty("solr.core.instanceDir"));
+      assertNotNull(desc.getInstanceDir());
+
+      // Prove we're ignoring this even though it's set in the properties file
+      assertFalse("InstanceDir should be ignored", desc.getInstanceDir().contains("totallybogus"));
 
-      assertEquals("core1", desc.getProperty("solr.core.dataDir"));
-      assertEquals("solrconfig-minimal.xml", desc.getProperty("solr.core.configName"));
-      assertEquals("schema-tiny.xml", desc.getProperty("solr.core.schemaName"));
+      assertEquals("core1", desc.getDataDir());
+      assertEquals("solrconfig-minimal.xml", desc.getConfigName());
+      assertEquals("schema-tiny.xml", desc.getSchemaName());
 
       SolrCore core2 = cc.getCore("core2");
       SolrCore lazy1 = cc.getCore("lazy1");
@@ -180,10 +180,9 @@ public class TestCoreDiscovery extends S
       cc = init();
       fail("Should have thrown exception in testDuplicateNames");
     } catch (SolrException se) {
-      Throwable cause = se.getCause();
-      String message = cause.getMessage();
-      assertTrue("Should have seen an exception because two cores had the same name",
-          message.indexOf("Core core1 defined more than once") != -1);
+      String message = se.getMessage();
+      assertTrue("Wrong exception thrown on duplicate core names",
+          message.indexOf("Found multiple cores with the name [core1]") != -1);
       assertTrue(File.separator + "core1 should have been mentioned in the message: " + message,
           message.indexOf(File.separator + "core1") != -1);
       assertTrue(File.separator + "core2 should have been mentioned in the message:" + message,
@@ -203,9 +202,9 @@ public class TestCoreDiscovery extends S
     alt.mkdirs();
     setMeUp(alt.getAbsolutePath());
     addCoreWithProps(makeCorePropFile("core1", false, true, "dataDir=core1"),
-        new File(alt, "core1" + File.separator + SolrCoreDiscoverer.CORE_PROP_FILE));
+        new File(alt, "core1" + File.separator + CorePropertiesLocator.PROPERTIES_FILENAME));
     addCoreWithProps(makeCorePropFile("core2", false, false, "dataDir=core2"),
-        new File(alt, "core2" + File.separator + SolrCoreDiscoverer.CORE_PROP_FILE));
+        new File(alt, "core2" + File.separator + CorePropertiesLocator.PROPERTIES_FILENAME));
     CoreContainer cc = init();
     try {
       SolrCore core1 = cc.getCore("core1");
@@ -226,9 +225,9 @@ public class TestCoreDiscovery extends S
     noCoreDir.mkdirs();
     setMeUp(noCoreDir.getAbsolutePath());
     addCoreWithProps(makeCorePropFile("core1", false, true),
-        new File(noCoreDir, "core1" + File.separator + SolrCoreDiscoverer.CORE_PROP_FILE));
+        new File(noCoreDir, "core1" + File.separator + CorePropertiesLocator.PROPERTIES_FILENAME));
     addCoreWithProps(makeCorePropFile("core2", false, false),
-        new File(noCoreDir, "core2" + File.separator + SolrCoreDiscoverer.CORE_PROP_FILE));
+        new File(noCoreDir, "core2" + File.separator + CorePropertiesLocator.PROPERTIES_FILENAME));
     CoreContainer cc = init();
     try {
       SolrCore core1 = cc.getCore("core1");

Modified: lucene/dev/trunk/solr/core/src/test/org/apache/solr/core/TestLazyCores.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/core/TestLazyCores.java?rev=1502276&r1=1502275&r2=1502276&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/core/TestLazyCores.java (original)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/core/TestLazyCores.java Thu Jul 11 16:35:01 2013
@@ -32,6 +32,7 @@ import org.apache.solr.update.AddUpdateC
 import org.apache.solr.update.CommitUpdateCommand;
 import org.apache.solr.update.UpdateHandler;
 import org.apache.solr.util.RefCounted;
+import org.apache.solr.util.TestHarness;
 import org.junit.After;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -52,7 +53,6 @@ public class TestLazyCores extends SolrT
 
   private final File solrHomeDirectory = new File(TEMP_DIR, "org.apache.solr.core.TestLazyCores_testlazy");
 
-
   private CoreContainer init() throws Exception {
 
     if (solrHomeDirectory.exists()) {
@@ -63,13 +63,17 @@ public class TestLazyCores extends SolrT
       copyMinConf(new File(solrHomeDirectory, "collection" + idx));
     }
 
+    SolrResourceLoader loader = new SolrResourceLoader(solrHomeDirectory.getAbsolutePath());
+
     File solrXml = new File(solrHomeDirectory, "solr.xml");
     FileUtils.write(solrXml, LOTS_SOLR_XML, IOUtils.CHARSET_UTF_8.toString());
-    final CoreContainer cores = new CoreContainer(solrHomeDirectory.getAbsolutePath());
-    cores.load();
-    //  h.getCoreContainer().load(solrHomeDirectory.getAbsolutePath(), new File(solrHomeDirectory, "solr.xml"));
+    ConfigSolrXmlOld config = (ConfigSolrXmlOld) ConfigSolr.fromFile(loader, solrXml);
+
+    CoresLocator locator = new SolrXMLCoresLocator.NonPersistingLocator(solrXml, LOTS_SOLR_XML, config);
 
-    cores.setPersistent(false);
+
+    final CoreContainer cores = new CoreContainer(loader, config, locator);
+    cores.load();
     return cores;
   }
 
@@ -91,15 +95,15 @@ public class TestLazyCores extends SolrT
 
       SolrCore core1 = cc.getCore("collection1");
       assertFalse("core1 should not be transient", core1.getCoreDescriptor().isTransient());
-      assertTrue("core1 should  be loadable", core1.getCoreDescriptor().isLoadOnStartup());
+      assertTrue("core1 should be loadable", core1.getCoreDescriptor().isLoadOnStartup());
       assertNotNull(core1.getSolrConfig());
 
       SolrCore core2 = cc.getCore("collectionLazy2");
-      assertTrue("core2 should not be transient", core2.getCoreDescriptor().isTransient());
+      assertTrue("core2 should be transient", core2.getCoreDescriptor().isTransient());
       assertTrue("core2 should be loadable", core2.getCoreDescriptor().isLoadOnStartup());
 
       SolrCore core3 = cc.getCore("collectionLazy3");
-      assertTrue("core3 should not be transient", core3.getCoreDescriptor().isTransient());
+      assertTrue("core3 should be transient", core3.getCoreDescriptor().isTransient());
       assertFalse("core3 should not be loadable", core3.getCoreDescriptor().isLoadOnStartup());
 
       SolrCore core4 = cc.getCore("collectionLazy4");
@@ -108,7 +112,7 @@ public class TestLazyCores extends SolrT
 
       SolrCore core5 = cc.getCore("collectionLazy5");
       assertFalse("core5 should not be transient", core5.getCoreDescriptor().isTransient());
-      assertTrue("core5 should  be loadable", core5.getCoreDescriptor().isLoadOnStartup());
+      assertTrue("core5 should be loadable", core5.getCoreDescriptor().isLoadOnStartup());
 
       core1.close();
       core2.close();
@@ -291,11 +295,11 @@ public class TestLazyCores extends SolrT
       admin.handleRequestBody(request, resp);
       fail("Should have thrown an error");
     } catch (SolrException se) {
-      SolrException cause = (SolrException)se.getCause();
-      assertEquals("Exception code should be 500", 500, cause.code());
+      //SolrException cause = (SolrException)se.getCause();
+      assertEquals("Exception code should be 500", 500, se.code());
       for (String err : errs) {
        assertTrue("Should have seen an exception containing the an error",
-            cause.getMessage().contains(err));
+            se.getMessage().contains(err));
       }
     }
   }
@@ -343,39 +347,25 @@ public class TestLazyCores extends SolrT
       copyMinConf(new File(solrHomeDirectory, "core3"));
       copyMinConf(new File(solrHomeDirectory, "core4"));
 
-      cc.setPersistent(true);
-      CoreDescriptor d1 = new CoreDescriptor(cc, "core1", "./core1");
-      d1.setTransient(true);
-      d1.setLoadOnStartup(true);
-      d1.setSchemaName("schema.xml");
-      d1.setConfigName("solrconfig.xml");
-      SolrCore core1 = cc.create(d1);
-
-      CoreDescriptor d2 = new CoreDescriptor(cc, "core2", "./core2");
-      d2.setTransient(true);
-      d2.setLoadOnStartup(false);
-      d2.setSchemaName("schema.xml");
-      d2.setConfigName("solrconfig.xml");
-      SolrCore core2 = cc.create(d2);
-
-      CoreDescriptor d3 = new CoreDescriptor(cc, "core3", "./core3");
-      d3.setTransient(false);
-      d3.setLoadOnStartup(true);
-      d3.setSchemaName("schema.xml");
-      d3.setConfigName("solrconfig.xml");
-      SolrCore core3 = cc.create(d3);
-
-      CoreDescriptor d4 = new CoreDescriptor(cc, "core4", "./core4");
-      d4.setTransient(false);
-      d4.setLoadOnStartup(false);
-      d4.setSchemaName("schema.xml");
-      d4.setConfigName("solrconfig.xml");
-      SolrCore core4 = cc.create(d4);
+      final CoreDescriptor cd1 = buildCoreDescriptor(cc, "core1", "./core1")
+          .isTransient(true).loadOnStartup(true).build();
+      final CoreDescriptor cd2 = buildCoreDescriptor(cc, "core2", "./core2")
+          .isTransient(true).loadOnStartup(false).build();
+      final CoreDescriptor cd3 = buildCoreDescriptor(cc, "core3", "./core3")
+          .isTransient(false).loadOnStartup(true).build();
+      final CoreDescriptor cd4 = buildCoreDescriptor(cc, "core4", "./core4")
+          .isTransient(false).loadOnStartup(false).build();
+
+
+      SolrCore core1 = cc.create(cd1);
+      SolrCore core2 = cc.create(cd2);
+      SolrCore core3 = cc.create(cd3);
+      SolrCore core4 = cc.create(cd4);
 
-      final File oneXml = new File(solrHomeDirectory, "lazy1.solr.xml");
-      cc.persistFile(oneXml);
+      SolrXMLCoresLocator.NonPersistingLocator locator =
+          (SolrXMLCoresLocator.NonPersistingLocator) cc.getCoresLocator();
 
-      assertXmlFile(oneXml,
+      TestHarness.validateXPath(locator.xml,
           "/solr/cores/core[@name='collection1']",
           "/solr/cores/core[@name='collectionLazy2']",
           "/solr/cores/core[@name='collectionLazy3']",
@@ -388,8 +378,8 @@ public class TestLazyCores extends SolrT
           "/solr/cores/core[@name='core1']",
           "/solr/cores/core[@name='core2']",
           "/solr/cores/core[@name='core3']",
-          "/solr/cores/core[@name='core4']");
-      assertXmlFile(oneXml, "13=count(/solr/cores/core)");
+          "/solr/cores/core[@name='core4']",
+          "13=count(/solr/cores/core)");
 
       removeOne(cc, "collectionLazy2");
       removeOne(cc, "collectionLazy3");
@@ -403,11 +393,8 @@ public class TestLazyCores extends SolrT
       removeOne(cc, "core4");
 
       // now test that unloading a core means the core is not persisted
+      TestHarness.validateXPath(locator.xml, "3=count(/solr/cores/core)");
 
-      final File twoXml = new File(solrHomeDirectory, "lazy2.solr.xml");
-      cc.persistFile(twoXml);
-
-      assertXmlFile(twoXml, "3=count(/solr/cores/core)");
     } finally {
       cc.shutdown();
     }

Modified: lucene/dev/trunk/solr/core/src/test/org/apache/solr/core/TestSolrXml.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/core/TestSolrXml.java?rev=1502276&r1=1502275&r2=1502276&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/core/TestSolrXml.java (original)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/core/TestSolrXml.java Thu Jul 11 16:35:01 2013
@@ -17,37 +17,34 @@ package org.apache.solr.core;
  * limitations under the License.
  */
 
+import com.carrotsearch.randomizedtesting.rules.SystemPropertiesRestoreRule;
 import org.apache.commons.io.FileUtils;
 import org.apache.solr.SolrTestCaseJ4;
+import org.junit.Rule;
 import org.junit.Test;
-import org.xml.sax.InputSource;
-import org.xml.sax.SAXException;
-
-import javax.xml.parsers.ParserConfigurationException;
+import org.junit.rules.RuleChain;
+import org.junit.rules.TestRule;
 
 import java.io.File;
-import java.io.FileInputStream;
 import java.io.IOException;
-import java.io.InputStream;
 
 public class TestSolrXml extends SolrTestCaseJ4 {
+
+  @Rule
+  public TestRule solrTestRules = RuleChain.outerRule(new SystemPropertiesRestoreRule());
+
   private final File solrHome = new File(TEMP_DIR, TestSolrXml.getClassName() + File.separator + "solrHome");
 
   @Test
-  public void testAllInfoPresent() throws IOException, ParserConfigurationException, SAXException {
-    CoreContainer cc = null;
+  public void testAllInfoPresent() throws IOException {
+
     File testSrcRoot = new File(SolrTestCaseJ4.TEST_HOME());
     FileUtils.copyFile(new File(testSrcRoot, "solr-50-all.xml"), new File(solrHome, "solr.xml"));
+
+    SolrResourceLoader loader = null;
     try {
-      InputStream is = new FileInputStream(new File(solrHome, "solr.xml"));
-      Config config = new Config(new SolrResourceLoader("solr/collection1"), null, new InputSource(is), null, false);
-      boolean oldStyle = (config.getNode("solr/cores", false) != null);
-      ConfigSolr cfg;
-      if (oldStyle) {
-        cfg = new ConfigSolrXmlOld(config);
-      } else {
-        cfg = new ConfigSolrXml(config, cc);
-      }
+      loader = new SolrResourceLoader(solrHome.getAbsolutePath());
+      ConfigSolr cfg = ConfigSolr.fromSolrHome(loader, solrHome.getAbsolutePath());
 
       assertEquals("Did not find expected value", cfg.get(ConfigSolr.CfgProp.SOLR_ADMINHANDLER, null), "testAdminHandler");
       assertEquals("Did not find expected value", cfg.getInt(ConfigSolr.CfgProp.SOLR_CORELOADTHREADS, 0), 11);
@@ -71,59 +68,35 @@ public class TestSolrXml extends SolrTes
       assertNull("Did not find expected value", cfg.get(ConfigSolr.CfgProp.SOLR_PERSISTENT, null));
       assertNull("Did not find expected value", cfg.get(ConfigSolr.CfgProp.SOLR_CORES_DEFAULT_CORE_NAME, null));
       assertNull("Did not find expected value", cfg.get(ConfigSolr.CfgProp.SOLR_ADMINPATH, null));
-    } finally {
-      if (cc != null) cc.shutdown();
     }
+    finally {
+      loader.close();
+    }
+
   }
 
   // Test  a few property substitutions that happen to be in solr-50-all.xml.
-  public void testPropretySub() throws IOException, ParserConfigurationException, SAXException {
+  public void testPropertySub() throws IOException {
 
-    String coreRoot = System.getProperty("coreRootDirectory");
-    String hostPort = System.getProperty("hostPort");
-    String shareSchema = System.getProperty("shareSchema");
-    String socketTimeout = System.getProperty("socketTimeout");
-    String connTimeout = System.getProperty("connTimeout");
     System.setProperty("coreRootDirectory", "myCoreRoot");
     System.setProperty("hostPort", "8888");
     System.setProperty("shareSchema", "newShareSchema");
     System.setProperty("socketTimeout", "220");
     System.setProperty("connTimeout", "200");
 
-    CoreContainer cc = null;
     File testSrcRoot = new File(SolrTestCaseJ4.TEST_HOME());
     FileUtils.copyFile(new File(testSrcRoot, "solr-50-all.xml"), new File(solrHome, "solr.xml"));
+
+    SolrResourceLoader loader = null;
     try {
-      InputStream is = new FileInputStream(new File(solrHome, "solr.xml"));
-      Config config = new Config(new SolrResourceLoader("solr/collection1"), null, new InputSource(is), null, false);
-      boolean oldStyle = (config.getNode("solr/cores", false) != null);
-      ConfigSolr cfg;
-      if (oldStyle) {
-        cfg = new ConfigSolrXmlOld(config);
-      } else {
-        cfg = new ConfigSolrXml(config, cc);
-      }
+      loader = new SolrResourceLoader(solrHome.getAbsolutePath());
+      ConfigSolr cfg = ConfigSolr.fromSolrHome(loader, solrHome.getAbsolutePath());
       assertEquals("Did not find expected value", cfg.get(ConfigSolr.CfgProp.SOLR_COREROOTDIRECTORY, null), "myCoreRoot");
       assertEquals("Did not find expected value", cfg.getInt(ConfigSolr.CfgProp.SOLR_HOSTPORT, 0), 8888);
       assertEquals("Did not find expected value", cfg.get(ConfigSolr.CfgProp.SOLR_SHARESCHEMA, null), "newShareSchema");
-
-    } finally {
-      if (cc != null) cc.shutdown();
-      if (coreRoot != null) System.setProperty("coreRootDirectory", coreRoot);
-      else System.clearProperty("coreRootDirectory");
-
-      if (hostPort != null) System.setProperty("hostPort", hostPort);
-      else System.clearProperty("hostPort");
-
-      if (shareSchema != null) System.setProperty("shareSchema", shareSchema);
-      else System.clearProperty("shareSchema");
-
-      if (socketTimeout != null) System.setProperty("socketTimeout", socketTimeout);
-      else System.clearProperty("socketTimeout");
-
-      if (connTimeout != null) System.setProperty("connTimeout", connTimeout);
-      else System.clearProperty("connTimeout");
-
+    }
+    finally {
+      loader.close();
     }
   }
 }