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

svn commit: r1470709 - in /lucene/dev/trunk/solr/core/src: java/org/apache/solr/core/ java/org/apache/solr/handler/admin/ test/org/apache/solr/core/

Author: markrmiller
Date: Mon Apr 22 20:39:24 2013
New Revision: 1470709

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

Added:
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/SolrCores.java
      - copied, changed from r1470674, lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/CoreMaps.java
Removed:
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/CoreMaps.java
Modified:
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/ConfigSolr.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/ConfigSolrXml.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/CoreContainer.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminHandler.java
    lucene/dev/trunk/solr/core/src/test/org/apache/solr/core/TestCoreDiscovery.java
    lucene/dev/trunk/solr/core/src/test/org/apache/solr/core/TestLazyCores.java

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/ConfigSolr.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/ConfigSolr.java?rev=1470709&r1=1470708&r2=1470709&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/ConfigSolr.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/ConfigSolr.java Mon Apr 22 20:39:24 2013
@@ -75,7 +75,6 @@ public abstract class ConfigSolr {
 
   protected Config config;
   protected Map<CfgProp, String> propMap = new HashMap<CfgProp, String>();
-  protected final Map<String, String> badConfigCores = new HashMap<String, String>();
 
   public ConfigSolr(Config config) {
     this.config = config;
@@ -84,12 +83,6 @@ public abstract class ConfigSolr {
   public Config getConfig() {
     return config;
   }
-  
-  // If the core is not to be loaded (say two cores defined with the same name or with the same data dir), return
-  // the reason. If it's OK to load the core, return null.
-  public String getBadConfigCoreMessage(String name) {
-    return badConfigCores.get(name);
-  }
 
   public int getInt(CfgProp prop, int def) {
     String val = propMap.get(prop);
@@ -151,38 +144,3 @@ public abstract class ConfigSolr {
 
 }
 
-// It's mightily convenient to have all of the original path names and property
-// values when persisting cores, so
-// this little convenience class is just for that.
-// Also, let's keep track of anything we added here, especially the instance dir
-// for persistence purposes. We don't
-// want, for instance, to persist instanceDir if it was not specified
-// originally.
-//
-// I suspect that for persistence purposes, we may want to expand this idea to
-// record, say, ${blah}
-class CoreDescriptorPlus {
-  private CoreDescriptor coreDescriptor;
-  private String filePath;
-  private Properties propsOrig; // TODO: 5.0. Remove this since it's only really
-                                // used for persisting.
-  
-  CoreDescriptorPlus(String filePath, CoreDescriptor descriptor,
-      Properties propsOrig) {
-    coreDescriptor = descriptor;
-    this.filePath = filePath;
-    this.propsOrig = propsOrig;
-  }
-  
-  CoreDescriptor getCoreDescriptor() {
-    return coreDescriptor;
-  }
-  
-  String getFilePath() {
-    return filePath;
-  }
-  
-  Properties getPropsOrig() {
-    return propsOrig;
-  }
-}

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/ConfigSolrXml.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/ConfigSolrXml.java?rev=1470709&r1=1470708&r2=1470709&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/ConfigSolrXml.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/ConfigSolrXml.java Mon Apr 22 20:39:24 2013
@@ -25,7 +25,6 @@ import java.io.InputStream;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
-import java.util.Locale;
 import java.util.Map;
 import java.util.Properties;
 
@@ -50,7 +49,7 @@ import org.xml.sax.SAXException;
 public class ConfigSolrXml extends ConfigSolr {
   protected static Logger log = LoggerFactory.getLogger(ConfigSolrXml.class);
 
-  private final Map<String, CoreDescriptorPlus> coreDescriptorPlusMap = new HashMap<String, CoreDescriptorPlus>();
+  private final Map<String, CoreDescriptor> coreDescriptorMap = new HashMap<String, CoreDescriptor>();
 
   public ConfigSolrXml(Config config, CoreContainer container)
       throws ParserConfigurationException, IOException, SAXException {
@@ -141,12 +140,12 @@ public class ConfigSolrXml extends Confi
   private void initCoreList(CoreContainer container) throws IOException {
     if (container != null) { // TODO: 5.0. Yet another bit of nonsense only
                              // because of the test harness.
-      synchronized (coreDescriptorPlusMap) {
-        String coreRoot = get(CfgProp.SOLR_COREROOTDIRECTORY,
-            container.getSolrHome());
-        walkFromHere(new File(coreRoot), container,
-            new HashMap<String,String>(), new HashMap<String,String>());
-      }
+      
+      String coreRoot = get(CfgProp.SOLR_COREROOTDIRECTORY,
+          container.getSolrHome());
+      walkFromHere(new File(coreRoot), container, new HashMap<String,String>(),
+          new HashMap<String,String>());
+      
     }
   }
 
@@ -206,34 +205,8 @@ public class ConfigSolrXml extends Confi
       props.setProperty(CoreDescriptor.CORE_NAME, childFile.getName());
     }
     CoreDescriptor desc = new CoreDescriptor(container, props);
-    CoreDescriptorPlus plus = new CoreDescriptorPlus(propFile.getAbsolutePath(), desc, propsOrig);
+    coreDescriptorMap.put(desc.getName(), desc);
 
-    // It's bad to have two cores with the same name or same data dir.
-    if (! seenCores.containsKey(desc.getName()) && ! seenDirs.containsKey(desc.getAbsoluteDataDir())) {
-      coreDescriptorPlusMap.put(desc.getName(), plus);
-      // Use the full path to the prop file so we can unambiguously report the place the error is.
-      seenCores.put(desc.getName(), propFile.getAbsolutePath());
-      seenDirs.put(desc.getAbsoluteDataDir(), propFile.getAbsolutePath());
-      return;
-    }
-
-    // record the appropriate error
-    if (seenCores.containsKey(desc.getName())) {
-      String msg = String.format(Locale.ROOT, "More than one core defined for core named '%s', paths are '%s' and '%s'  Removing both cores.",
-          desc.getName(), propFile.getAbsolutePath(), seenCores.get(desc.getName()));
-      log.error(msg);
-      // Load up as many errors as there are.
-      if (badConfigCores.containsKey(desc.getName())) msg += " " + badConfigCores.get(desc.getName());
-      badConfigCores.put(desc.getName(), msg);
-    }
-    // There's no reason both errors may not have occurred.
-    if (seenDirs.containsKey(desc.getAbsoluteDataDir())) {
-      String msg = String.format(Locale.ROOT, "More than one core points to data dir '%s'. They are in '%s' and '%s'. Removing all offending cores.",
-          desc.getAbsoluteDataDir(), propFile.getAbsolutePath(), seenDirs.get(desc.getAbsoluteDataDir()));
-      if (badConfigCores.containsKey(desc.getName())) msg += " " + badConfigCores.get(desc.getName());
-      log.warn(msg);
-    }
-    coreDescriptorPlusMap.remove(desc.getName());
   }
 
   public IndexSchema getSchemaFromZk(ZkController zkController, String zkConfigName, String schemaName,
@@ -251,11 +224,10 @@ public class ConfigSolrXml extends Confi
       SolrResourceLoader loader, String coreName) {
     
     // first look for an exact match
-    for (Map.Entry<String,CoreDescriptorPlus> ent : coreDescriptorPlusMap
+    for (Map.Entry<String,CoreDescriptor> ent : coreDescriptorMap
         .entrySet()) {
       
-      String name = ent.getValue().getCoreDescriptor()
-          .getProperty(CoreDescriptor.CORE_NAME, null);
+      String name = ent.getValue().getProperty(CoreDescriptor.CORE_NAME, null);
       if (origCoreName.equals(name)) {
         if (coreName.equals(origCoreName)) {
           return name;
@@ -264,10 +236,9 @@ public class ConfigSolrXml extends Confi
       }
     }
     
-    for (Map.Entry<String,CoreDescriptorPlus> ent : coreDescriptorPlusMap
+    for (Map.Entry<String,CoreDescriptor> ent : coreDescriptorMap
         .entrySet()) {
-      String name = ent.getValue().getCoreDescriptor()
-          .getProperty(CoreDescriptor.CORE_NAME, null);
+      String name = ent.getValue().getProperty(CoreDescriptor.CORE_NAME, null);
       // see if we match with substitution
       if (origCoreName.equals(PropertiesUtil.substituteProperty(name,
           loader.getCoreProperties()))) {
@@ -282,7 +253,7 @@ public class ConfigSolrXml extends Confi
 
   @Override
   public List<String> getAllCoreNames() {
-    List<String> ret = new ArrayList<String>(coreDescriptorPlusMap.keySet());
+    List<String> ret = new ArrayList<String>(coreDescriptorMap.keySet());
     
     return ret;
   }
@@ -290,20 +261,18 @@ public class ConfigSolrXml extends Confi
   @Override
   public String getProperty(String coreName, String property, String defaultVal) {
     
-    CoreDescriptorPlus plus = coreDescriptorPlusMap.get(coreName);
-    if (plus == null) return defaultVal;
-    CoreDescriptor desc = plus.getCoreDescriptor();
-    if (desc == null) return defaultVal;
-    return desc.getProperty(property, defaultVal);
-    
+    CoreDescriptor cd = coreDescriptorMap.get(coreName);
+    if (cd == null) return defaultVal;
+
+    return cd.getProperty(property, defaultVal);
   }
 
   @Override
   public Properties readCoreProperties(String coreName) {
     
-    CoreDescriptorPlus plus = coreDescriptorPlusMap.get(coreName);
+    CoreDescriptor plus = coreDescriptorMap.get(coreName);
     if (plus == null) return null;
-    return new Properties(plus.getCoreDescriptor().getCoreProperties());
+    return new Properties(plus.getCoreProperties());
 
   }
 

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/CoreContainer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/CoreContainer.java?rev=1470709&r1=1470708&r2=1470709&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/CoreContainer.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/CoreContainer.java Mon Apr 22 20:39:24 2013
@@ -108,14 +108,14 @@ public class CoreContainer
   protected static Logger log = LoggerFactory.getLogger(CoreContainer.class);
 
 
-  private final CoreMaps coreMaps = new CoreMaps(this);
+  private final SolrCores solrCores = new SolrCores(this);
 
   protected final Map<String,Exception> coreInitFailures =
     Collections.synchronizedMap(new LinkedHashMap<String,Exception>());
   
   protected boolean persistent = false;
   protected String adminPath = null;
-  protected String managementPath = null;
+  protected volatile String managementPath = null;
   protected String hostPort;
   protected String hostContext;
   protected String host;
@@ -335,16 +335,13 @@ public class CoreContainer
 
   // Helper class to initialize the CoreContainer
   public static class Initializer {
-    protected String containerConfigFilename = null;  // normally "solr.xml" becoming solr.properties in 5.0
-    protected String dataDir = null; // override datadir for single core mode
 
     // core container instantiation
     public CoreContainer initialize() throws FileNotFoundException {
       CoreContainer cores = null;
       String solrHome = SolrResourceLoader.locateSolrHome();
       // ContainerConfigFilename could  could be a properties file
-      File fconf = new File(solrHome, containerConfigFilename == null ? "solr.xml"
-          : containerConfigFilename);
+      File fconf = new File(solrHome, "solr.xml");
 
       log.info("looking for solr config file: " + fconf.getAbsolutePath());
       cores = new CoreContainer(solrHome);
@@ -365,8 +362,6 @@ public class CoreContainer
         cores.configFile = fconf;
       }
       
-      containerConfigFilename = cores.getConfigFile().getName();
-      
       return cores;
     }
   }
@@ -441,7 +436,7 @@ public class CoreContainer
 
     shardHandlerFactory = initShardHandler(cfg);
 
-    coreMaps.allocateLazyCores(cfg, loader);
+    solrCores.allocateLazyCores(cfg, loader);
 
     // Initialize Logging
     if (cfg.getBool(ConfigSolr.CfgProp.SOLR_LOGGING_ENABLED, true)) {
@@ -647,7 +642,7 @@ public class CoreContainer
           } else {
             // Store it away for later use. includes non-transient but not
             // loaded at startup cores.
-            coreMaps.putDynamicDescriptor(rawName, p);
+            solrCores.putDynamicDescriptor(rawName, p);
           }
         } catch (Throwable ex) {
           SolrException.log(log, null, ex);
@@ -665,7 +660,7 @@ public class CoreContainer
             SolrCore c = future.get();
             // track original names
             if (c != null) {
-              coreMaps.putCoreToOrigName(c, c.getName());
+              solrCores.putCoreToOrigName(c, c.getName());
             }
           } catch (ExecutionException e) {
             SolrException.log(SolrCore.log, "Error loading core", e);
@@ -678,7 +673,7 @@ public class CoreContainer
       }
 
       // Start the background thread
-      backgroundCloser = new CloserThread(this, coreMaps, cfg);
+      backgroundCloser = new CloserThread(this, solrCores, cfg);
       backgroundCloser.start();
 
     } finally {
@@ -750,15 +745,15 @@ public class CoreContainer
     isShutDown = true;
 
     if (isZooKeeperAware()) {
-      coreMaps.publishCoresAsDown(zkController);
+      publishCoresAsDown();
       cancelCoreRecoveries();
     }
 
 
     try {
       // First wake up the closer thread, it'll terminate almost immediately since it checks isShutDown.
-      synchronized (coreMaps.getLocker()) {
-        coreMaps.getLocker().notifyAll(); // wake up anyone waiting
+      synchronized (solrCores.getModifyLock()) {
+        solrCores.getModifyLock().notifyAll(); // wake up anyone waiting
       }
       if (backgroundCloser != null) { // Doesn't seem right, but tests get in here without initializing the core.
         try {
@@ -771,12 +766,12 @@ public class CoreContainer
         }
       }
       // Now clear all the cores that are being operated upon.
-      coreMaps.clearMaps(cfg);
+      solrCores.close();
 
       // It's still possible that one of the pending dynamic load operation is waiting, so wake it up if so.
       // Since all the pending operations queues have been drained, there should be nothing to do.
-      synchronized (coreMaps.getLocker()) {
-        coreMaps.getLocker().notifyAll(); // wake up the thread
+      synchronized (solrCores.getModifyLock()) {
+        solrCores.getModifyLock().notifyAll(); // wake up the thread
       }
 
     } finally {
@@ -796,7 +791,7 @@ public class CoreContainer
 
   public void cancelCoreRecoveries() {
     ArrayList<SolrCoreState> coreStates = new ArrayList<SolrCoreState>();
-    coreMaps.addCoresToList(coreStates);
+    solrCores.addCoresToList(coreStates);
 
     // we must cancel without holding the cores sync
     // make sure we wait for any recoveries to stop
@@ -853,9 +848,9 @@ public class CoreContainer
       throw new IllegalStateException("This CoreContainer has been shutdown");
     }
     if (isTransientCore) {
-      old = coreMaps.putTransientCore(cfg, name, core, loader);
+      old = solrCores.putTransientCore(cfg, name, core, loader);
     } else {
-      old = coreMaps.putCore(name, core);
+      old = solrCores.putCore(name, core);
     }
       /*
       * set both the name of the descriptor and the name of the
@@ -1044,7 +1039,7 @@ public class CoreContainer
         created = createFromLocal(instanceDir, dcore);
       }
 
-      coreMaps.addCreated(created); // For persisting newly-created cores.
+      solrCores.addCreated(created); // For persisting newly-created cores.
       return created;
 
       // :TODO: Java7...
@@ -1058,21 +1053,21 @@ public class CoreContainer
    * @return a Collection of registered SolrCores
    */
   public Collection<SolrCore> getCores() {
-    return coreMaps.getCores();
+    return solrCores.getCores();
   }
 
   /**
    * @return a Collection of the names that cores are mapped to
    */
   public Collection<String> getCoreNames() {
-    return coreMaps.getCoreNames();
+    return solrCores.getCoreNames();
   }
 
   /** This method is currently experimental.
    * @return a Collection of the names that a specific core is mapped to.
    */
   public Collection<String> getCoreNames(SolrCore core) {
-    return coreMaps.getCoreNames(core);
+    return solrCores.getCoreNames(core);
   }
 
   /**
@@ -1080,17 +1075,7 @@ public class CoreContainer
    * @return a list of al lthe available core names in either permanent or transient core lists.
    */
   public Collection<String> getAllCoreNames() {
-    return coreMaps.getAllCoreNames();
-
-  }
-
-  /**
-   * Checks that the data dir passed is is NOT shared by any other core
-   * @param targetPath - path to check
-   * @return - null if this path is unique, core name of the first other core that shares this path.
-   */
-  public String checkUniqueDataDir(String targetPath) {
-    return coreMaps.checkUniqueDataDir(targetPath);
+    return solrCores.getAllCoreNames();
 
   }
 
@@ -1132,19 +1117,12 @@ public class CoreContainer
     try {
       name = checkDefault(name);
 
-      if (cfg != null) { // Another test artifact.
-        String badMsg = cfg.getBadConfigCoreMessage(name);
-        if (badMsg != null) {
-          throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, badMsg);
-        }
-      }
-
-      SolrCore core = coreMaps.getCore(name);
+      SolrCore core = solrCores.getCore(name);
       if (core == null)
         throw new SolrException( SolrException.ErrorCode.BAD_REQUEST, "No such core: " + name );
 
       try {
-        coreMaps.waitAddPendingCoreOps(name);
+        solrCores.waitAddPendingCoreOps(name);
         CoreDescriptor cd = core.getCoreDescriptor();
 
         File instanceDir = new File(cd.getInstanceDir());
@@ -1181,10 +1159,10 @@ public class CoreContainer
         }
         SolrCore newCore = core.reload(solrLoader, core);
         // keep core to orig name link
-        coreMaps.removeCoreToOrigName(newCore, core);
+        solrCores.removeCoreToOrigName(newCore, core);
         registerCore(false, name, newCore, false);
       } finally {
-        coreMaps.removeFromPendingOps(name);
+        solrCores.removeFromPendingOps(name);
       }
       // :TODO: Java7...
       // http://docs.oracle.com/javase/7/docs/technotes/guides/language/catch-multiple.html
@@ -1207,7 +1185,7 @@ public class CoreContainer
     }
     n0 = checkDefault(n0);
     n1 = checkDefault(n1);
-    coreMaps.swap(n0, n1);
+    solrCores.swap(n0, n1);
 
     log.info("swapped: "+n0 + " with " + n1);
   }
@@ -1216,7 +1194,7 @@ public class CoreContainer
   public SolrCore remove( String name ) {
     name = checkDefault(name);    
 
-    return coreMaps.remove(name, true);
+    return solrCores.remove(name, true);
 
   }
 
@@ -1226,7 +1204,7 @@ public class CoreContainer
       if (core != null) {
         registerCore(false, toName, core, false);
         name = checkDefault(name);
-        coreMaps.remove(name, false);
+        solrCores.remove(name, false);
       }
     } finally {
       if (core != null) {
@@ -1246,15 +1224,8 @@ public class CoreContainer
 
     name = checkDefault(name);
 
-    if (cfg != null) { // Get this out of here sometime, this is test-code only stuff!
-      String badMsg = cfg.getBadConfigCoreMessage(name);
-      if (badMsg != null) {
-        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, badMsg);
-      }
-    }
-
     // Do this in two phases since we don't want to lock access to the cores over a load.
-    SolrCore core = coreMaps.getCoreFromAnyList(name);
+    SolrCore core = solrCores.getCoreFromAnyList(name);
 
     if (core != null) {
       core.open();
@@ -1262,7 +1233,7 @@ public class CoreContainer
     }
 
     // OK, it's not presently in any list, is it in the list of dynamic cores but not loaded yet? If so, load it.
-    CoreDescriptor desc = coreMaps.getDynamicDescriptor(name);
+    CoreDescriptor desc = solrCores.getDynamicDescriptor(name);
     if (desc == null) { //Nope, no transient core with this name
       
       // if there was an error initalizing this core, throw a 500
@@ -1278,7 +1249,7 @@ public class CoreContainer
     }
 
     // This will put an entry in pending core ops if the core isn't loaded
-    core = coreMaps.waitAddPendingCoreOps(name);
+    core = solrCores.waitAddPendingCoreOps(name);
 
     if (isShutDown) return null; // We're quitting, so stop. This needs to be after the wait above since we may come off
                                  // the wait as a consequence of shutting down.
@@ -1293,7 +1264,7 @@ public class CoreContainer
     } catch(Exception ex){
       throw recordAndThrow(name, "Unable to create core: " + name, ex);
     } finally {
-      coreMaps.removeFromPendingOps(name);
+      solrCores.removeFromPendingOps(name);
     }
 
     return core;
@@ -1330,6 +1301,7 @@ public class CoreContainer
     return persistent;
   }
   
+  @Deprecated
   public void setPersistent(boolean persistent) {
     this.persistent = persistent;
   }
@@ -1385,7 +1357,7 @@ public class CoreContainer
    *
    */
   public boolean isLoaded(String name) {
-    return coreMaps.isLoaded(name);
+    return solrCores.isLoaded(name);
   }
 
   /** Persists the cores config file in cores.xml. */
@@ -1401,7 +1373,7 @@ public class CoreContainer
    * @return a coreDescriptor. May return null
    */
   public CoreDescriptor getUnloadedCoreDescriptor(String cname) {
-    return coreMaps.getUnloadedCoreDescriptor(cname);
+    return solrCores.getUnloadedCoreDescriptor(cname);
   }
 
   /** Persists the cores config file in a user provided file. */
@@ -1449,7 +1421,7 @@ public class CoreContainer
     }
 
     try {
-      coreMaps.persistCores(origCfg, containerProperties, rootSolrAttribs, coresAttribs, file, configFile, loader);
+      solrCores.persistCores(origCfg, containerProperties, rootSolrAttribs, coresAttribs, file, configFile, loader);
     } catch (XPathExpressionException e) {
       throw new SolrException(ErrorCode.SERVER_ERROR, null, e);
     }
@@ -1513,11 +1485,7 @@ public class CoreContainer
   }
   
   String getCoreToOrigName(SolrCore core) {
-    return coreMaps.getCoreToOrigName(core);
-  }
-
-  public String getBadCoreMessage(String name) {
-    return cfg.getBadConfigCoreMessage(name);
+    return solrCores.getCoreToOrigName(core);
   }
   
   private Document copyDoc(Document document) throws TransformerException {
@@ -1529,19 +1497,30 @@ public class CoreContainer
     return (Document)result.getNode();
   }
 
+  protected void publishCoresAsDown() {
+    List<SolrCore> cores = solrCores.getCores();
+    
+    for (SolrCore core : cores) {
+      try {
+        zkController.publish(core.getCoreDescriptor(), ZkStateReader.DOWN);
+      } catch (KeeperException e) {
+        CoreContainer.log.error("", e);
+      } catch (InterruptedException e) {
+        CoreContainer.log.error("", e);
+      }
+    }
+  }
 }
 
-
-
 class CloserThread extends Thread {
   CoreContainer container;
-  CoreMaps coreMaps;
+  SolrCores solrCores;
   ConfigSolr cfg;
 
 
-  CloserThread(CoreContainer container, CoreMaps coreMaps, ConfigSolr cfg) {
+  CloserThread(CoreContainer container, SolrCores solrCores, ConfigSolr cfg) {
     this.container = container;
-    this.coreMaps = coreMaps;
+    this.solrCores = solrCores;
     this.cfg = cfg;
   }
 
@@ -1552,21 +1531,21 @@ class CloserThread extends Thread {
   @Override
   public void run() {
     while (! container.isShutDown()) {
-      synchronized (coreMaps.getLocker()) { // need this so we can wait and be awoken.
+      synchronized (solrCores.getModifyLock()) { // need this so we can wait and be awoken.
         try {
-          coreMaps.getLocker().wait();
+          solrCores.getModifyLock().wait();
         } catch (InterruptedException e) {
           // Well, if we've been told to stop, we will. Otherwise, continue on and check to see if there are
           // any cores to close.
         }
       }
-      for (SolrCore removeMe = coreMaps.getCoreToClose();
+      for (SolrCore removeMe = solrCores.getCoreToClose();
            removeMe != null && !container.isShutDown();
-           removeMe = coreMaps.getCoreToClose()) {
+           removeMe = solrCores.getCoreToClose()) {
         try {
           removeMe.close();
         } finally {
-          coreMaps.removeFromPendingOps(removeMe.getName());
+          solrCores.removeFromPendingOps(removeMe.getName());
         }
       }
     }

Copied: lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/SolrCores.java (from r1470674, lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/CoreMaps.java)
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/SolrCores.java?p2=lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/SolrCores.java&p1=lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/CoreMaps.java&r1=1470674&r2=1470709&rev=1470709&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/CoreMaps.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/SolrCores.java Mon Apr 22 20:39:24 2013
@@ -27,9 +27,9 @@ import org.apache.zookeeper.KeeperExcept
 import org.w3c.dom.Node;
 
 
-class CoreMaps {
+class SolrCores {
   private static SolrXMLSerializer SOLR_XML_SERIALIZER = new SolrXMLSerializer();
-  private static Object locker = new Object(); // for locking around manipulating any of the core maps.
+  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
 
   //WARNING! The _only_ place you put anything into the list of transient cores is with the putTransientCore method!
@@ -51,7 +51,7 @@ class CoreMaps {
   // to essentially queue them up to be handled via pendingCoreOps.
   private static final List<SolrCore> pendingCloses = new ArrayList<SolrCore>();
 
-  CoreMaps(CoreContainer container) {
+  SolrCores(CoreContainer container) {
     this.container = container;
   }
 
@@ -65,9 +65,9 @@ class CoreMaps {
         @Override
         protected boolean removeEldestEntry(Map.Entry<String, SolrCore> eldest) {
           if (size() > transientCacheSize) {
-            synchronized (locker) {
+            synchronized (modifyLock) {
               pendingCloses.add(eldest.getValue()); // Essentially just queue this core up for closing.
-              locker.notifyAll(); // Wakes up closer thread too
+              modifyLock.notifyAll(); // Wakes up closer thread too
             }
             return true;
           }
@@ -78,14 +78,14 @@ class CoreMaps {
   }
 
   protected void putDynamicDescriptor(String rawName, CoreDescriptor p) {
-    synchronized (locker) {
+    synchronized (modifyLock) {
       dynamicDescriptors.put(rawName, p);
     }
   }
 
   // We are shutting down. You can't hold the lock on the various lists of cores while they shut down, so we need to
   // make a temporary copy of the names and shut them down outside the lock.
-  protected void clearMaps(ConfigSolr cfg) {
+  protected void close() {
     List<String> coreNames;
     List<String> transientNames;
     List<SolrCore> pendingToClose;
@@ -95,7 +95,7 @@ class CoreMaps {
     // list to the pendingCloses list.
 
     while (true) {
-      synchronized (locker) {
+      synchronized (modifyLock) {
         coreNames = new ArrayList<String>(cores.keySet());
         transientNames = new ArrayList<String>(transientCores.keySet());
         pendingToClose = new ArrayList<SolrCore>(pendingCloses);
@@ -116,7 +116,7 @@ class CoreMaps {
           } catch (Throwable t) {
             SolrException.log(CoreContainer.log, "Error shutting down core", t);
           } finally {
-            synchronized (locker) {
+            synchronized (modifyLock) {
               cores.remove(coreName);
             }
           }
@@ -133,7 +133,7 @@ class CoreMaps {
           } catch (Throwable t) {
             SolrException.log(CoreContainer.log, "Error shutting down core", t);
           } finally {
-            synchronized (locker) {
+            synchronized (modifyLock) {
               transientCores.remove(coreName);
             }
           }
@@ -147,7 +147,7 @@ class CoreMaps {
         } catch (Throwable t) {
           SolrException.log(CoreContainer.log, "Error shutting down core", t);
         } finally {
-          synchronized (locker) {
+          synchronized (modifyLock) {
             pendingCloses.remove(core);
           }
         }
@@ -157,7 +157,7 @@ class CoreMaps {
 
   protected void addCoresToList(ArrayList<SolrCoreState> coreStates) {
     List<SolrCore> addCores;
-    synchronized (locker) {
+    synchronized (modifyLock) {
       addCores = new ArrayList<SolrCore>(cores.values());
     }
     for (SolrCore core : addCores) {
@@ -169,14 +169,14 @@ class CoreMaps {
   protected SolrCore putTransientCore(ConfigSolr cfg, String name, SolrCore core, SolrResourceLoader loader) {
     SolrCore retCore;
     CoreContainer.log.info("Opening transient core {}", name);
-    synchronized (locker) {
+    synchronized (modifyLock) {
       retCore = transientCores.put(name, core);
     }
     return retCore;
   }
 
   protected SolrCore putCore(String name, SolrCore core) {
-    synchronized (locker) {
+    synchronized (modifyLock) {
       return cores.put(name, core);
     }
   }
@@ -184,7 +184,7 @@ class CoreMaps {
   List<SolrCore> getCores() {
     List<SolrCore> lst = new ArrayList<SolrCore>();
 
-    synchronized (locker) {
+    synchronized (modifyLock) {
       lst.addAll(cores.values());
       return lst;
     }
@@ -193,7 +193,7 @@ class CoreMaps {
   Set<String> getCoreNames() {
     Set<String> set = new TreeSet<String>();
 
-    synchronized (locker) {
+    synchronized (modifyLock) {
       set.addAll(cores.keySet());
       set.addAll(transientCores.keySet());
     }
@@ -203,7 +203,7 @@ class CoreMaps {
   List<String> getCoreNames(SolrCore core) {
     List<String> lst = new ArrayList<String>();
 
-    synchronized (locker) {
+    synchronized (modifyLock) {
       for (Map.Entry<String, SolrCore> entry : cores.entrySet()) {
         if (core == entry.getValue()) {
           lst.add(entry.getKey());
@@ -225,7 +225,7 @@ class CoreMaps {
    */
   public Collection<String> getAllCoreNames() {
     Set<String> set = new TreeSet<String>();
-    synchronized (locker) {
+    synchronized (modifyLock) {
       set.addAll(cores.keySet());
       set.addAll(transientCores.keySet());
       set.addAll(dynamicDescriptors.keySet());
@@ -236,14 +236,14 @@ class CoreMaps {
 
   SolrCore getCore(String name) {
 
-    synchronized (locker) {
+    synchronized (modifyLock) {
       return cores.get(name);
     }
   }
 
   protected void swap(String n0, String n1) {
 
-    synchronized (locker) {
+    synchronized (modifyLock) {
       SolrCore c0 = cores.get(n0);
       SolrCore c1 = cores.get(n1);
       if (c0 == null)
@@ -263,7 +263,7 @@ class CoreMaps {
 
   protected SolrCore remove(String name, boolean removeOrig) {
 
-    synchronized (locker) {
+    synchronized (modifyLock) {
       SolrCore tmp = cores.remove(name);
       SolrCore ret = null;
       if (removeOrig && tmp != null) {
@@ -283,7 +283,7 @@ class CoreMaps {
 
   protected void putCoreToOrigName(SolrCore c, String name) {
 
-    synchronized (locker) {
+    synchronized (modifyLock) {
       coreToOrigName.put(c, name);
     }
 
@@ -291,7 +291,7 @@ class CoreMaps {
 
   protected void removeCoreToOrigName(SolrCore newCore, SolrCore core) {
 
-    synchronized (locker) {
+    synchronized (modifyLock) {
       String origName = coreToOrigName.remove(core);
       if (origName != null) {
         coreToOrigName.put(newCore, origName);
@@ -302,7 +302,7 @@ class CoreMaps {
   protected SolrCore getCoreFromAnyList(String name) {
     SolrCore core;
 
-    synchronized (locker) {
+    synchronized (modifyLock) {
       core = cores.get(name);
       if (core != null) {
         return core;
@@ -317,13 +317,13 @@ class CoreMaps {
   }
 
   protected CoreDescriptor getDynamicDescriptor(String name) {
-    synchronized (locker) {
+    synchronized (modifyLock) {
       return dynamicDescriptors.get(name);
     }
   }
 
   protected boolean isLoaded(String name) {
-    synchronized (locker) {
+    synchronized (modifyLock) {
       if (cores.containsKey(name)) {
         return true;
       }
@@ -336,7 +336,7 @@ class CoreMaps {
   }
 
   protected CoreDescriptor getUnloadedCoreDescriptor(String cname) {
-    synchronized (locker) {
+    synchronized (modifyLock) {
       CoreDescriptor desc = dynamicDescriptors.get(cname);
       if (desc == null) {
         return null;
@@ -347,33 +347,10 @@ class CoreMaps {
   }
 
   protected String getCoreToOrigName(SolrCore solrCore) {
-    synchronized (locker) {
+    synchronized (modifyLock) {
       return coreToOrigName.get(solrCore);
     }
   }
-
-  protected void publishCoresAsDown(ZkController zkController) {
-    synchronized (locker) {
-      for (SolrCore core : cores.values()) {
-        try {
-          zkController.publish(core.getCoreDescriptor(), ZkStateReader.DOWN);
-        } catch (KeeperException e) {
-          CoreContainer.log.error("", e);
-        } catch (InterruptedException e) {
-          CoreContainer.log.error("", e);
-        }
-      }
-      for (SolrCore core : transientCores.values()) {
-        try {
-          zkController.publish(core.getCoreDescriptor(), ZkStateReader.DOWN);
-        } catch (KeeperException e) {
-          CoreContainer.log.error("", e);
-        } catch (InterruptedException e) {
-          CoreContainer.log.error("", e);
-        }
-      }
-    }
-  }
   
   // Irrepressably ugly bit of the transition in SOLR-4196, but there as at least one test case that follows
   // this path, presumably it's there for a reason.
@@ -392,7 +369,7 @@ class CoreMaps {
     //
     
     List<SolrXMLSerializer.SolrCoreXMLDef> solrCoreXMLDefs = new ArrayList<SolrXMLSerializer.SolrCoreXMLDef>();
-    synchronized (locker) {
+    synchronized (modifyLock) {
       
       persistCores(cfg, cores, loader, solrCoreXMLDefs);
       persistCores(cfg, transientCores, loader, solrCoreXMLDefs);
@@ -401,14 +378,14 @@ class CoreMaps {
       for (Map.Entry<String,CoreDescriptor> ent : dynamicDescriptors.entrySet()) {
         if (!cores.containsKey(ent.getKey())
             && !transientCores.containsKey(ent.getKey())) {
-          addPersistOneCore(cfg, loader, ent.getValue(), null, solrCoreXMLDefs);
+          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())) {
-          addPersistOneCore(cfg, loader, ent.getValue().getCoreDescriptor(),
+          addCoreToPersistList(cfg, loader, ent.getValue().getCoreDescriptor(),
               null, solrCoreXMLDefs);
         }
       }
@@ -426,7 +403,7 @@ class CoreMaps {
   protected SolrCore waitAddPendingCoreOps(String name) {
 
     // Keep multiple threads from operating on a core at one time.
-    synchronized (locker) {
+    synchronized (modifyLock) {
       boolean pending;
       do { // Are we currently doing anything to this core? Loading, unloading, reloading?
         pending = pendingCoreOps.contains(name); // wait for the core to be done being operated upon
@@ -442,7 +419,7 @@ class CoreMaps {
 
         if (pending) {
           try {
-            locker.wait();
+            modifyLock.wait();
           } catch (InterruptedException e) {
             return null; // Seems best not to do anything at all if the thread is interrupted
           }
@@ -462,18 +439,18 @@ class CoreMaps {
   // We should always be removing the first thing in the list with our name! The idea here is to NOT do anything n
   // any core while some other operation is working on that core.
   protected void removeFromPendingOps(String name) {
-    synchronized (locker) {
+    synchronized (modifyLock) {
       if (! pendingCoreOps.remove(name)) {
         CoreContainer.log.warn("Tried to remove core {} from pendingCoreOps and it wasn't there. ", name);
       }
-      locker.notifyAll();
+      modifyLock.notifyAll();
     }
   }
 
 
   protected void persistCores(Config cfg, Map<String, SolrCore> whichCores, SolrResourceLoader loader, List<SolrCoreXMLDef> solrCoreXMLDefs) throws XPathExpressionException {
     for (SolrCore solrCore : whichCores.values()) {
-      addPersistOneCore(cfg, loader, solrCore.getCoreDescriptor(), getCoreToOrigName(solrCore), solrCoreXMLDefs);
+      addCoreToPersistList(cfg, loader, solrCore.getCoreDescriptor(), getCoreToOrigName(solrCore), solrCoreXMLDefs);
     }
   }
   
@@ -504,7 +481,7 @@ class CoreMaps {
 
   }
 
-  protected void addPersistOneCore(Config cfg, SolrResourceLoader loader,
+  protected void addCoreToPersistList(Config cfg, SolrResourceLoader loader,
       CoreDescriptor dcore, String origCoreName,
       List<SolrCoreXMLDef> solrCoreXMLDefs) throws XPathExpressionException {
     
@@ -567,14 +544,16 @@ class CoreMaps {
 
   }
 
-  protected Object getLocker() { return locker; }
+  protected Object getModifyLock() {
+    return modifyLock;
+  }
 
   // Be a little careful. We don't want to either open or close a core unless it's _not_ being opened or closed by
   // another thread. So within this lock we'll walk along the list of pending closes until we find something NOT in
   // the list of threads currently being loaded or reloaded. The "usual" case will probably return the very first
   // one anyway..
   protected SolrCore getCoreToClose() {
-    synchronized (locker) {
+    synchronized (modifyLock) {
       for (SolrCore core : pendingCloses) {
         if (! pendingCoreOps.contains(core.getName())) {
           pendingCoreOps.add(core.getName());
@@ -587,28 +566,8 @@ class CoreMaps {
   }
 
   protected void addCreated(SolrCore core) {
-    synchronized (locker) {
+    synchronized (modifyLock) {
       createdCores.put(core.getName(), core);
     }
   }
-
-  protected String checkUniqueDataDir(String targetPath) {
-    // Have to check
-    // loaded cores
-    // transient cores
-    // dynamic cores
-    synchronized (locker) {
-      for (SolrCore core : cores.values()) {
-        if (targetPath.equals(core.getDataDir())) return core.getName();
-      }
-      for (SolrCore core : transientCores.values()) {
-        if (targetPath.equals(core.getDataDir())) return core.getName();
-      }
-      for (CoreDescriptor desc : dynamicDescriptors.values()) {
-        if (targetPath.equals(desc.getDataDir())) return desc.getName();
-      }
-    }
-
-    return null;
-  }
 }

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=1470709&r1=1470708&r2=1470709&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 Mon Apr 22 20:39:24 2013
@@ -415,11 +415,8 @@ public class CoreAdminHandler extends Re
     }
     try {
       
-      //for now, do not allow creating new core with same name when in cloud mode
-      //XXX perhaps it should just be unregistered from cloud before reading it?,
-      //XXX perhaps we should also check that cores are of same type before adding new core to collection?
       if (coreContainer.getAllCoreNames().contains(name)) {
-        log.warn("Re-creating a core with existing name is not allowed");
+        log.warn("Creating a core with existing name is not allowed");
         throw new SolrException(ErrorCode.SERVER_ERROR,
             "Core with name '" + name + "' already exists.");
       }
@@ -509,14 +506,6 @@ public class CoreAdminHandler extends Re
       
       SolrCore core = coreContainer.create(dcore);
 
-      String sameDirCore = coreContainer.checkUniqueDataDir(core.getDataDir());
-      if (sameDirCore != null) {
-        if (core != null) core.close();
-        log.warn("Creating a core that points to the same data dir as core {} is not allowed", sameDirCore);
-        throw new SolrException(ErrorCode.SERVER_ERROR,
-            "Core with same data dir '" + sameDirCore + "' already exists.");
-      }
-
       coreContainer.register(name, core, false);
       rsp.add("core", core.getName());
       return coreContainer.isPersistent();

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=1470709&r1=1470708&r2=1470709&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 Mon Apr 22 20:39:24 2013
@@ -17,22 +17,18 @@ package org.apache.solr.core;
  * limitations under the License.
  */
 
+import java.io.File;
+import java.io.FileOutputStream;
+import java.util.Properties;
+
 import org.apache.commons.io.FileUtils;
 import org.apache.lucene.util.IOUtils;
 import org.apache.solr.SolrTestCaseJ4;
-import org.apache.solr.common.SolrException;
 import org.junit.After;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.FileOutputStream;
-import java.util.Properties;
-import java.util.Set;
-
 public class TestCoreDiscovery extends SolrTestCaseJ4 {
-  private static String NEW_LINE = System.getProperty("line.separator");
 
   @BeforeClass
   public static void beforeClass() throws Exception {
@@ -221,79 +217,6 @@ public class TestCoreDiscovery extends S
     }
   }
 
-  @Test
-  public void testCoresWithSameNameError() throws Exception {
-    setMeUp();
-    addCoreWithPropsDir("core1_1", makeCorePropFile("core1", false, true));
-    addCoreWithPropsDir("core1_2", makeCorePropFile("core1", false, true));
-    CoreContainer cc = null;
-    try {
-      cc = init();
-      String msg = cc.getBadCoreMessage("core1");
-      assertTrue("Should have found multiple cores with same name", msg.contains("More than one core defined for core named 'core1'"));
-      try {
-        cc.getCore("core1");
-      } catch (SolrException se) {
-        assertEquals("Should be returning proper error code of 500", 500, se.code());
-        msg = se.getMessage();
-        assertTrue("Should have found multiple cores with same name", msg.contains("More than one core defined for core named 'core1'"));
-      }
-    } finally {
-      if (cc != null) {
-        cc.shutdown();
-      }
-    }
-  }
-
-  @Test
-  public void testCoresWithSameNameErrorTransient() throws Exception {
-    setMeUp();
-    addCoreWithPropsDir("core1_1", makeCorePropFile("core1", true, false));
-    addCoreWithPropsDir("core1_2", makeCorePropFile("core1", true, false));
-    CoreContainer cc = null;
-    try {
-      cc = init();
-      String msg = cc.getBadCoreMessage("core1");
-      assertTrue("Should have found multiple cores with same name", msg.contains("More than one core defined for core named 'core1'"));
-      try {
-        cc.getCore("core1");
-      } catch (SolrException se) {
-        assertEquals("Should be returning proper error code of 500", 500, se.code());
-        msg = se.getMessage();
-        assertTrue("Should have found multiple cores with same name", msg.contains("More than one core defined for core named 'core1'"));
-      }
-    } finally {
-      if (cc != null) {
-        cc.shutdown();
-      }
-    }
-  }
-
-  @Test
-  public void testCoresWithSameNameErrorBoth() throws Exception {
-    setMeUp();
-    addCoreWithPropsDir("core1_1", makeCorePropFile("core1", true, false));
-    addCoreWithPropsDir("core1_2", makeCorePropFile("core1", false, false));
-    // Should just blow up here.
-    CoreContainer cc = null;
-    try {
-      cc = init();
-      String msg = cc.getBadCoreMessage("core1");
-      assertTrue("Should have found multiple cores with same name", msg.contains("More than one core defined for core named 'core1'"));
-      try {
-        cc.getCore("core1");
-      } catch (SolrException se) {
-        assertEquals("Should be returning proper error code of 500", 500, se.code());
-        msg = se.getMessage();
-        assertTrue("Should have found multiple cores with same name", msg.contains("More than one core defined for core named 'core1'"));
-      }
-    } finally {
-      if (cc != null) {
-        cc.shutdown();
-      }
-    }
-  }
-
   // For testing whether finding a solr.xml overrides looking at solr.properties
   private final static String SOLR_XML = "<solr> " +
       "<int name=\"transientCacheSize\">2</int> " +

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=1470709&r1=1470708&r2=1470709&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 Mon Apr 22 20:39:24 2013
@@ -315,29 +315,13 @@ public class TestLazyCores extends SolrT
       copyMinConf(new File(solrHomeDirectory, "t5"));
       copyMinConf(new File(solrHomeDirectory, "t6"));
 
-      tryCreateFail(admin, "t2", lc2.getDataDir(), "Core with same data dir", "collectionLazy2", "already exists");
-      tryCreateFail(admin, "t4", lc4.getDataDir(), "Core with same data dir", "collectionLazy4", "already exists");
-      tryCreateFail(admin, "t5", lc5.getDataDir(), "Core with same data dir", "collectionLazy5", "already exists");
-      tryCreateFail(admin, "t6", lc6.getDataDir(), "Core with same data dir", "collectionLazy6", "already exists");
-
-      // Insure a newly-created core fails too
-      CoreDescriptor d1 = new CoreDescriptor(cc, "core1", "./core1");
-      d1.setSchemaName("schema-tiny.xml");
-      d1.setConfigName("solrconfig-minimal.xml");
-      copyMinConf(new File(solrHomeDirectory, "core1"));
-      SolrCore core1 = cc.create(d1);
-      cc.register(core1, false);
-      copyMinConf(new File(solrHomeDirectory, "core77"));
-      tryCreateFail(admin, "core77", core1.getDataDir(), "Core with same data dir", "core1", "already exists");
 
       // Should also fail with the same name
       tryCreateFail(admin, "collectionLazy2", "t12", "Core with name", "collectionLazy2", "already exists");
       tryCreateFail(admin, "collectionLazy4", "t14", "Core with name", "collectionLazy4", "already exists");
       tryCreateFail(admin, "collectionLazy5", "t15", "Core with name", "collectionLazy5", "already exists");
       tryCreateFail(admin, "collectionLazy6", "t16", "Core with name", "collectionLazy6", "already exists");
-      tryCreateFail(admin, "core1", "t10", "Core with name", "core1", "already exists");
 
-      core1.close();
       lc2.close();
       lc4.close();
       lc5.close();