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 2015/11/30 15:27:42 UTC

svn commit: r1717260 [1/2] - in /lucene/dev/branches/branch_5x: ./ solr/ solr/contrib/ solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/ solr/core/ solr/core/src/java/org/apache/solr/cloud/ solr/core/src/java/org/apache/solr/core/ solr/core/src/...

Author: romseygeek
Date: Mon Nov 30 14:27:42 2015
New Revision: 1717260

URL: http://svn.apache.org/viewvc?rev=1717260&view=rev
Log:
SOLR-8336: CoreDescriptor takes a Path for its instance dir, rather than a String

Modified:
    lucene/dev/branches/branch_5x/   (props changed)
    lucene/dev/branches/branch_5x/solr/   (props changed)
    lucene/dev/branches/branch_5x/solr/CHANGES.txt   (contents, props changed)
    lucene/dev/branches/branch_5x/solr/contrib/   (props changed)
    lucene/dev/branches/branch_5x/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/SolrRecordWriter.java
    lucene/dev/branches/branch_5x/solr/core/   (props changed)
    lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/cloud/CloudConfigSetService.java
    lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/cloud/CloudDescriptor.java
    lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/cloud/CloudUtil.java
    lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/cloud/ZkController.java
    lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/ConfigSetService.java
    lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/CoreContainer.java
    lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/CoreDescriptor.java
    lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/CorePropertiesLocator.java
    lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/DirectoryFactory.java
    lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/SolrCore.java
    lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/handler/SnapShooter.java
    lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminHandler.java
    lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/TestCrossCoreJoin.java
    lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/cloud/ClusterStateUpdateTest.java
    lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/cloud/CollectionsAPISolrJTests.java
    lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/cloud/LeaderElectionIntegrationTest.java
    lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/core/TestConfigSets.java
    lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/core/TestCoreContainer.java
    lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/core/TestCoreDiscovery.java
    lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/core/TestLazyCores.java
    lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/handler/admin/CoreAdminHandlerTest.java
    lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/handler/admin/CoreAdminRequestStatusTest.java
    lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/handler/component/SuggestComponentTest.java
    lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/search/TestIndexSearcher.java
    lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/update/SolrIndexSplitterTest.java
    lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/util/MockCoreContainer.java
    lucene/dev/branches/branch_5x/solr/test-framework/   (props changed)
    lucene/dev/branches/branch_5x/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java
    lucene/dev/branches/branch_5x/solr/test-framework/src/java/org/apache/solr/util/TestHarness.java

Modified: lucene/dev/branches/branch_5x/solr/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/CHANGES.txt?rev=1717260&r1=1717259&r2=1717260&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/CHANGES.txt (original)
+++ lucene/dev/branches/branch_5x/solr/CHANGES.txt Mon Nov 30 14:27:42 2015
@@ -17,7 +17,8 @@ Other Changes
 * LUCENE-6900: Added test for score ordered grouping, and refactored TopGroupsResultTransformer.
   (David Smiley)
 
-
+* SOLR-8336: CoreDescriptor now takes a Path for its instance directory, rather
+  than a String (Alan Woodward)
 
 ==================  5.4.0 ==================
 

Modified: lucene/dev/branches/branch_5x/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/SolrRecordWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/SolrRecordWriter.java?rev=1717260&r1=1717259&r2=1717260&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/SolrRecordWriter.java (original)
+++ lucene/dev/branches/branch_5x/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/SolrRecordWriter.java Mon Nov 30 14:27:42 2015
@@ -25,10 +25,10 @@ import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Locale;
-import java.util.Properties;
 import java.util.Set;
 import java.util.concurrent.TimeUnit;
 
+import com.google.common.collect.ImmutableMap;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.filecache.DistributedCache;
 import org.apache.hadoop.fs.FileSystem;
@@ -158,13 +158,8 @@ class SolrRecordWriter<K, V> extends Rec
     
     CoreContainer container = new CoreContainer(loader);
     container.load();
-    
-    Properties props = new Properties();
-    props.setProperty(CoreDescriptor.CORE_DATADIR, dataDirStr);
-    
-    CoreDescriptor descr = new CoreDescriptor(container, "core1", solrHomeDir.toString(), props);
-    
-    SolrCore core = container.create(descr);
+
+    SolrCore core = container.create("core1", ImmutableMap.of(CoreDescriptor.CORE_DATADIR, dataDirStr));
     
     if (!(core.getDirectoryFactory() instanceof HdfsDirectoryFactory)) {
       throw new UnsupportedOperationException(

Modified: lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/cloud/CloudConfigSetService.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/cloud/CloudConfigSetService.java?rev=1717260&r1=1717259&r2=1717260&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/cloud/CloudConfigSetService.java (original)
+++ lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/cloud/CloudConfigSetService.java Mon Nov 30 14:27:42 2015
@@ -17,8 +17,6 @@
 
 package org.apache.solr.cloud;
 
-import java.nio.file.Paths;
-
 import org.apache.solr.core.ConfigSetService;
 import org.apache.solr.core.CoreDescriptor;
 import org.apache.solr.core.SolrResourceLoader;
@@ -37,7 +35,7 @@ public class CloudConfigSetService exten
     // TODO: Shouldn't the collection node be created by the Collections API?
     zkController.createCollectionZkNode(cd.getCloudDescriptor());
     String configName = zkController.getZkStateReader().readConfigName(cd.getCollectionName());
-    return new ZkSolrResourceLoader(Paths.get(cd.getInstanceDir()), configName, parentLoader.getClassLoader(),
+    return new ZkSolrResourceLoader(cd.getInstanceDir(), configName, parentLoader.getClassLoader(),
         cd.getSubstitutableProperties(), zkController);
   }
 

Modified: lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/cloud/CloudDescriptor.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/cloud/CloudDescriptor.java?rev=1717260&r1=1717259&r2=1717260&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/cloud/CloudDescriptor.java (original)
+++ lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/cloud/CloudDescriptor.java Mon Nov 30 14:27:42 2015
@@ -17,31 +17,24 @@ package org.apache.solr.cloud;
  * limitations under the License.
  */
 
+import java.util.HashMap;
+import java.util.Map;
 import java.util.Properties;
 
+import com.google.common.base.Strings;
 import org.apache.solr.common.cloud.Replica;
-import org.apache.solr.common.cloud.Slice;
-import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.core.CoreDescriptor;
 import org.apache.solr.util.PropertiesUtil;
 
-import com.google.common.base.Strings;
-
 public class CloudDescriptor {
 
   private final CoreDescriptor cd;
   private String shardId;
   private String collectionName;
-  private SolrParams params;
   private String roles = null;
   private Integer numShards;
   private String nodeName = null;
-
-  /* shardRange and shardState are used once-only during sub shard creation for shard splits
-   * Use the values from {@link Slice} instead */
-  volatile String shardRange = null;
-  volatile Slice.State shardState = Slice.State.ACTIVE;
-  volatile String shardParent = null;
+  private Map<String, String> collectionParams = new HashMap<>();
 
   private volatile boolean isLeader = false;
   
@@ -64,6 +57,12 @@ public class CloudDescriptor {
     if (Strings.isNullOrEmpty(nodeName))
       this.nodeName = null;
     this.numShards = PropertiesUtil.toInteger(props.getProperty(CloudDescriptor.NUM_SHARDS), null);
+
+    for (String propName : props.stringPropertyNames()) {
+      if (propName.startsWith(ZkController.COLLECTION_PARAM_PREFIX)) {
+        collectionParams.put(propName.substring(ZkController.COLLECTION_PARAM_PREFIX.length()), props.getProperty(propName));
+      }
+    }
   }
   
   public Replica.State getLastPublished() {
@@ -115,12 +114,8 @@ public class CloudDescriptor {
   }
   
   /** Optional parameters that can change how a core is created. */
-  public SolrParams getParams() {
-    return params;
-  }
-
-  public void setParams(SolrParams params) {
-    this.params = params;
+  public Map<String, String> getParams() {
+    return collectionParams;
   }
 
   // setting only matters on core creation

Modified: lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/cloud/CloudUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/cloud/CloudUtil.java?rev=1717260&r1=1717259&r2=1717260&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/cloud/CloudUtil.java (original)
+++ lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/cloud/CloudUtil.java Mon Nov 30 14:27:42 2015
@@ -72,12 +72,11 @@ public class CloudUtil {
               cc.unload(desc.getName());
             }
             
-            File instanceDir = new File(desc.getInstanceDir());
             try {
-              FileUtils.deleteDirectory(instanceDir);
+              FileUtils.deleteDirectory(desc.getInstanceDir().toFile());
             } catch (IOException e) {
               SolrException.log(log, "Failed to delete instance dir for core:"
-                  + desc.getName() + " dir:" + instanceDir.getAbsolutePath());
+                  + desc.getName() + " dir:" + desc.getInstanceDir());
             }
             log.error("", new SolrException(ErrorCode.SERVER_ERROR,
                 "Will not load SolrCore " + desc.getName()

Modified: lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/cloud/ZkController.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/cloud/ZkController.java?rev=1717260&r1=1717259&r2=1717260&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/cloud/ZkController.java (original)
+++ lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/cloud/ZkController.java Mon Nov 30 14:27:42 2015
@@ -25,19 +25,7 @@ import java.net.URLEncoder;
 import java.net.UnknownHostException;
 import java.nio.charset.StandardCharsets;
 import java.nio.file.Path;
-import java.nio.file.Paths;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Enumeration;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Locale;
-import java.util.Map;
-import java.util.Properties;
-import java.util.Set;
+import java.util.*;
 import java.util.concurrent.Callable;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Future;
@@ -52,25 +40,7 @@ import org.apache.solr.cloud.overseer.Ov
 import org.apache.solr.cloud.overseer.SliceMutator;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;
-import org.apache.solr.common.cloud.BeforeReconnect;
-import org.apache.solr.common.cloud.ClusterState;
-import org.apache.solr.common.cloud.ClusterStateUtil;
-import org.apache.solr.common.cloud.DefaultConnectionStrategy;
-import org.apache.solr.common.cloud.DefaultZkACLProvider;
-import org.apache.solr.common.cloud.DefaultZkCredentialsProvider;
-import org.apache.solr.common.cloud.DocCollection;
-import org.apache.solr.common.cloud.OnReconnect;
-import org.apache.solr.common.cloud.Replica;
-import org.apache.solr.common.cloud.Slice;
-import org.apache.solr.common.cloud.SolrZkClient;
-import org.apache.solr.common.cloud.ZkACLProvider;
-import org.apache.solr.common.cloud.ZkCmdExecutor;
-import org.apache.solr.common.cloud.ZkConfigManager;
-import org.apache.solr.common.cloud.ZkCoreNodeProps;
-import org.apache.solr.common.cloud.ZkCredentialsProvider;
-import org.apache.solr.common.cloud.ZkNodeProps;
-import org.apache.solr.common.cloud.ZkStateReader;
-import org.apache.solr.common.cloud.ZooKeeperException;
+import org.apache.solr.common.cloud.*;
 import org.apache.solr.common.params.CollectionParams;
 import org.apache.solr.common.params.CommonParams;
 import org.apache.solr.common.params.SolrParams;
@@ -102,11 +72,11 @@ import org.slf4j.MDC;
 import static org.apache.solr.common.cloud.ZkStateReader.BASE_URL_PROP;
 import static org.apache.solr.common.cloud.ZkStateReader.COLLECTION_PROP;
 import static org.apache.solr.common.cloud.ZkStateReader.CORE_NAME_PROP;
+import static org.apache.solr.common.cloud.ZkStateReader.CORE_NODE_NAME_PROP;
 import static org.apache.solr.common.cloud.ZkStateReader.ELECTION_NODE_PROP;
 import static org.apache.solr.common.cloud.ZkStateReader.NODE_NAME_PROP;
 import static org.apache.solr.common.cloud.ZkStateReader.REJOIN_AT_HEAD_PROP;
 import static org.apache.solr.common.cloud.ZkStateReader.SHARD_ID_PROP;
-import static org.apache.solr.common.cloud.ZkStateReader.CORE_NODE_NAME_PROP;
 
 /**
  * Handle ZooKeeper interactions.
@@ -1337,7 +1307,6 @@ public final class ZkController {
     try {
       if (!zkClient.exists(collectionPath, true)) {
         log.info("Creating collection in ZooKeeper:" + collection);
-        SolrParams params = cd.getParams();
 
         try {
           Map<String, Object> collectionProps = new HashMap<>();
@@ -1346,15 +1315,8 @@ public final class ZkController {
           String defaultConfigName = System.getProperty(COLLECTION_PARAM_PREFIX + CONFIGNAME_PROP, collection);
 
           // params passed in - currently only done via core admin (create core commmand).
-          if (params != null) {
-            Iterator<String> iter = params.getParameterNamesIterator();
-            while (iter.hasNext()) {
-              String paramName = iter.next();
-              if (paramName.startsWith(COLLECTION_PARAM_PREFIX)) {
-                collectionProps.put(paramName.substring(COLLECTION_PARAM_PREFIX.length()), params.get(paramName));
-              }
-            }
-
+          if (cd.getParams().size() > 0) {
+            collectionProps.putAll(cd.getParams());
             // if the config name wasn't passed in, use the default
             if (!collectionProps.containsKey(CONFIGNAME_PROP)) {
               // TODO: getting the configName from the collectionPath should fail since we already know it doesn't exist?
@@ -1795,8 +1757,7 @@ public final class ZkController {
       String confName = cd.getCollectionName();
       if (StringUtils.isEmpty(confName))
         confName = coreName;
-      String instanceDir = cd.getInstanceDir();
-      Path udir = Paths.get(instanceDir).resolve("conf");
+      Path udir = cd.getInstanceDir().resolve("conf");
       log.info("Uploading directory " + udir + " with name " + confName + " for SolrCore " + coreName);
       configManager.uploadConfigDir(udir, confName);
     }
@@ -2531,4 +2492,22 @@ public final class ZkController {
       super(code, msg);
     }
   }
+
+  public boolean checkIfCoreNodeNameAlreadyExists(CoreDescriptor dcore) {
+    DocCollection collection = zkStateReader.getClusterState().getCollectionOrNull(dcore.getCollectionName());
+    if (collection != null) {
+      Collection<Slice> slices = collection.getSlices();
+
+      for (Slice slice : slices) {
+        Collection<Replica> replicas = slice.getReplicas();
+        for (Replica replica : replicas) {
+          if (replica.getName().equals(
+              dcore.getCloudDescriptor().getCoreNodeName())) {
+            return true;
+          }
+        }
+      }
+    }
+    return false;
+  }
 }

Modified: lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/ConfigSetService.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/ConfigSetService.java?rev=1717260&r1=1717259&r2=1717260&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/ConfigSetService.java (original)
+++ lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/ConfigSetService.java Mon Nov 30 14:27:42 2015
@@ -167,7 +167,7 @@ public abstract class ConfigSetService {
     protected Path locateInstanceDir(CoreDescriptor cd) {
       String configSet = cd.getConfigSet();
       if (configSet == null)
-        return Paths.get(cd.getInstanceDir());
+        return cd.getInstanceDir();
       Path configSetDirectory = configSetBase.resolve(configSet);
       if (!Files.isDirectory(configSetDirectory))
         throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,

Modified: lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/CoreContainer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/CoreContainer.java?rev=1717260&r1=1717259&r2=1717260&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/CoreContainer.java (original)
+++ lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/CoreContainer.java Mon Nov 30 14:27:42 2015
@@ -36,6 +36,7 @@ import com.google.common.collect.Immutab
 import com.google.common.collect.Maps;
 import org.apache.solr.client.solrj.impl.HttpClientConfigurer;
 import org.apache.solr.client.solrj.impl.HttpClientUtil;
+import org.apache.solr.cloud.Overseer;
 import org.apache.solr.cloud.ZkController;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;
@@ -518,7 +519,7 @@ public class CoreContainer {
   }
 
   private static void checkForDuplicateCoreNames(List<CoreDescriptor> cds) {
-    Map<String, String> addedCores = Maps.newHashMap();
+    Map<String, Path> addedCores = Maps.newHashMap();
     for (CoreDescriptor cd : cds) {
       final String name = cd.getName();
       if (addedCores.containsKey(name))
@@ -707,12 +708,82 @@ public class CoreContainer {
   }
 
   /**
-   * Creates a new core based on a CoreDescriptor, publishing the core state to the cluster
-   * @param cd the CoreDescriptor
+   * Creates a new core, publishing the core state to the cluster
+   * @param coreName the core name
+   * @param parameters the core parameters
    * @return the newly created core
    */
-  public SolrCore create(CoreDescriptor cd) {
-    return create(cd, true);
+  public SolrCore create(String coreName, Map<String, String> parameters) {
+    return create(coreName, cfg.getCoreRootDirectory().resolve(coreName), parameters);
+  }
+
+  /**
+   * Creates a new core in a specified instance directory, publishing the core state to the cluster
+   * @param coreName the core name
+   * @param instancePath the instance directory
+   * @param parameters the core parameters
+   * @return the newly created core
+   */
+  public SolrCore create(String coreName, Path instancePath, Map<String, String> parameters) {
+
+    CoreDescriptor cd = new CoreDescriptor(this, coreName, instancePath, parameters);
+
+    // TODO: There's a race here, isn't there?
+    if (getAllCoreNames().contains(coreName)) {
+      log.warn("Creating a core with existing name is not allowed");
+      // TODO: Shouldn't this be a BAD_REQUEST?
+      throw new SolrException(ErrorCode.SERVER_ERROR, "Core with name '" + coreName + "' already exists.");
+    }
+
+    boolean preExisitingZkEntry = false;
+    try {
+      if (getZkController() != null) {
+        if (!Overseer.isLegacy(getZkController().getZkStateReader().getClusterProps())) {
+          if (cd.getCloudDescriptor().getCoreNodeName() == null) {
+            throw new SolrException(ErrorCode.SERVER_ERROR, "non legacy mode coreNodeName missing " + parameters.toString());
+
+          }
+        }
+        preExisitingZkEntry = getZkController().checkIfCoreNodeNameAlreadyExists(cd);
+      }
+
+      SolrCore core = create(cd, true);
+
+      // only write out the descriptor if the core is successfully created
+      coresLocator.create(this, cd);
+
+      return core;
+    }
+    catch (Exception ex) {
+      if (isZooKeeperAware() && !preExisitingZkEntry) {
+        try {
+          getZkController().unregister(coreName, cd);
+        } catch (InterruptedException e) {
+          Thread.currentThread().interrupt();
+          SolrException.log(log, null, e);
+        } catch (KeeperException e) {
+          SolrException.log(log, null, e);
+        }
+      }
+
+      Throwable tc = ex;
+      Throwable c = null;
+      do {
+        tc = tc.getCause();
+        if (tc != null) {
+          c = tc;
+        }
+      } while (tc != null);
+
+      String rootMsg = "";
+      if (c != null) {
+        rootMsg = " Caused by: " + c.getMessage();
+      }
+
+      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
+          "Error CREATEing SolrCore '" + coreName + "': " + ex.getMessage() + rootMsg, ex);
+    }
+
   }
 
   /**
@@ -723,7 +794,7 @@ public class CoreContainer {
    *
    * @return the newly created core
    */
-  public SolrCore create(CoreDescriptor dcore, boolean publishState) {
+  private SolrCore create(CoreDescriptor dcore, boolean publishState) {
 
     if (isShutDown) {
       throw new SolrException(ErrorCode.SERVICE_UNAVAILABLE, "Solr has been shutdown.");
@@ -960,8 +1031,8 @@ public class CoreContainer {
     return null;
   }
 
-  public String getCoreRootDirectory() {
-    return cfg.getCoreRootDirectory().toString();
+  public Path getCoreRootDirectory() {
+    return cfg.getCoreRootDirectory();
   }
 
   /**
@@ -1007,7 +1078,7 @@ public class CoreContainer {
         if (zkSys.getZkController() != null) {
           zkSys.getZkController().throwErrorIfReplicaReplaced(desc);
         }
-        core = create(desc); // This should throw an error if it fails.
+        core = create(desc, true); // This should throw an error if it fails.
       }
       core.open();
     }

Modified: lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/CoreDescriptor.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/CoreDescriptor.java?rev=1717260&r1=1717259&r2=1717260&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/CoreDescriptor.java (original)
+++ lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/CoreDescriptor.java Mon Nov 30 14:27:42 2015
@@ -17,25 +17,25 @@
 
 package org.apache.solr.core;
 
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableMap;
-import org.apache.commons.lang.StringUtils;
-import org.apache.solr.cloud.CloudDescriptor;
-import org.apache.solr.common.SolrException;
-import org.apache.solr.common.params.SolrParams;
-import org.apache.solr.common.util.IOUtils;
-import org.apache.solr.util.PropertiesUtil;
-
 import java.io.File;
-import java.io.FileInputStream;
 import java.io.IOException;
+import java.io.InputStream;
 import java.io.InputStreamReader;
 import java.nio.charset.StandardCharsets;
-import java.nio.file.Paths;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.Collections;
+import java.util.HashMap;
 import java.util.Locale;
+import java.util.Map;
 import java.util.Properties;
 
-import static com.google.common.base.Preconditions.checkNotNull;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import org.apache.commons.lang.StringUtils;
+import org.apache.solr.cloud.CloudDescriptor;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.util.PropertiesUtil;
 
 /**
  * A Solr core descriptor
@@ -47,8 +47,6 @@ public class CoreDescriptor {
   // Properties file name constants
   public static final String CORE_NAME = "name";
   public static final String CORE_CONFIG = "config";
-  public static final String CORE_INSTDIR = "instanceDir";
-  public static final String CORE_ABS_INSTDIR = "absoluteInstDir";
   public static final String CORE_DATADIR = "dataDir";
   public static final String CORE_ULOGDIR = "ulogDir";
   public static final String CORE_SCHEMA = "schema";
@@ -91,13 +89,12 @@ public class CoreDescriptor {
       .build();
 
   private static ImmutableList<String> requiredProperties = ImmutableList.of(
-      CORE_NAME, CORE_INSTDIR, CORE_ABS_INSTDIR
+      CORE_NAME
   );
 
   public static ImmutableList<String> standardPropNames = ImmutableList.of(
       CORE_NAME,
       CORE_CONFIG,
-      CORE_INSTDIR,
       CORE_DATADIR,
       CORE_ULOGDIR,
       CORE_SCHEMA,
@@ -118,6 +115,8 @@ public class CoreDescriptor {
 
   private final CloudDescriptor cloudDesc;
 
+  private final Path instanceDir;
+
   /** The original standard core properties, before substitution */
   protected final Properties originalCoreProperties = new Properties();
 
@@ -130,61 +129,72 @@ public class CoreDescriptor {
   /** The properties for this core, substitutable by resource loaders */
   protected final Properties substitutableProperties = new Properties();
 
-  /**
-   * Create a new CoreDescriptor.
-   * @param container       the CoreDescriptor's container
-   * @param name            the CoreDescriptor's name
-   * @param instanceDir     a String containing the instanceDir
-   * @param coreProps       a Properties object of the properties for this core
-   */
-  public CoreDescriptor(CoreContainer container, String name, String instanceDir,
-                        Properties coreProps) {
-    this(container, name, instanceDir, coreProps, null);
-  }
-
-  public CoreDescriptor(CoreContainer container, String name, String instanceDir, String... properties) {
-    this(container, name, instanceDir, toProperties(properties));
+  public CoreDescriptor(CoreContainer container, String name, Path instanceDir, String... properties) {
+    this(container, name, instanceDir, toMap(properties));
   }
 
-  private static Properties toProperties(String... properties) {
-    Properties props = new Properties();
+  private static Map<String, String> toMap(String... properties) {
+    Map<String, String> props = new HashMap<>();
     assert properties.length % 2 == 0;
     for (int i = 0; i < properties.length; i += 2) {
-      props.setProperty(properties[i], properties[i+1]);
+      props.put(properties[i], properties[i+1]);
     }
     return props;
   }
-  
+
+  /**
+   * Create a new CoreDescriptor with a given name and instancedir
+   * @param container     the CoreDescriptor's container
+   * @param name          the CoreDescriptor's name
+   * @param instanceDir   the CoreDescriptor's instancedir
+   */
+  public CoreDescriptor(CoreContainer container, String name, Path instanceDir) {
+    this(container, name, instanceDir, Collections.<String,String>emptyMap());
+  }
+
+  /**
+   * Create a new CoreDescriptor using the properties of an existing one
+   * @param coreName the new CoreDescriptor's name
+   * @param other    the CoreDescriptor to copy
+   */
+  public CoreDescriptor(String coreName, CoreDescriptor other) {
+    this.coreContainer = other.coreContainer;
+    this.cloudDesc = other.cloudDesc;
+    this.instanceDir = other.instanceDir;
+    this.originalExtraProperties.putAll(other.originalExtraProperties);
+    this.originalCoreProperties.putAll(other.originalCoreProperties);
+    this.coreProperties.putAll(other.coreProperties);
+    this.substitutableProperties.putAll(other.substitutableProperties);
+    this.coreProperties.setProperty(CORE_NAME, coreName);
+    this.originalCoreProperties.setProperty(CORE_NAME, coreName);
+    this.substitutableProperties.setProperty(SOLR_CORE_PROP_PREFIX + CORE_NAME, coreName);
+  }
+
   /**
    * Create a new CoreDescriptor.
    * @param container       the CoreDescriptor's container
    * @param name            the CoreDescriptor's name
-   * @param instanceDir     a String containing the instanceDir
-   * @param coreProps       a Properties object of the properties for this core
-   * @param params          additional params
+   * @param instanceDir     a Path resolving to the instanceDir
+   * @param coreProps       a Map of the properties for this core
    */
-  public CoreDescriptor(CoreContainer container, String name, String instanceDir,
-                        Properties coreProps, SolrParams params) {
+  public CoreDescriptor(CoreContainer container, String name, Path instanceDir,
+                        Map<String, String> coreProps) {
 
     this.coreContainer = container;
+    this.instanceDir = instanceDir;
 
     originalCoreProperties.setProperty(CORE_NAME, name);
-    originalCoreProperties.setProperty(CORE_INSTDIR, instanceDir);
 
     Properties containerProperties = container.getContainerProperties();
     name = PropertiesUtil.substituteProperty(checkPropertyIsNotEmpty(name, CORE_NAME),
                                              containerProperties);
-    instanceDir = PropertiesUtil.substituteProperty(checkPropertyIsNotEmpty(instanceDir, CORE_INSTDIR),
-                                                    containerProperties);
 
     coreProperties.putAll(defaultProperties);
     coreProperties.put(CORE_NAME, name);
-    coreProperties.put(CORE_INSTDIR, instanceDir);
-    coreProperties.put(CORE_ABS_INSTDIR, convertToAbsolute(instanceDir, container.getCoreRootDirectory()));
 
-    for (String propname : coreProps.stringPropertyNames()) {
+    for (String propname : coreProps.keySet()) {
 
-      String propvalue = coreProps.getProperty(propname);
+      String propvalue = coreProps.get(propname);
 
       if (isUserDefinedProperty(propname))
         originalExtraProperties.put(propname, propvalue);
@@ -202,9 +212,6 @@ public class CoreDescriptor {
     // TODO maybe make this a CloudCoreDescriptor subclass?
     if (container.isZooKeeperAware()) {
       cloudDesc = new CloudDescriptor(name, coreProperties, this);
-      if (params != null) {
-        cloudDesc.setParams(params);
-      }
     }
     else {
       cloudDesc = null;
@@ -226,20 +233,16 @@ public class CoreDescriptor {
    */
   protected void loadExtraProperties() {
     String filename = coreProperties.getProperty(CORE_PROPERTIES, DEFAULT_EXTERNAL_PROPERTIES_FILE);
-    File propertiesFile = resolvePaths(filename);
-    if (propertiesFile.exists()) {
-      FileInputStream in = null;
-      try {
-        in = new FileInputStream(propertiesFile);
+    Path propertiesFile = instanceDir.resolve(filename);
+    if (Files.exists(propertiesFile)) {
+      try (InputStream is = Files.newInputStream(propertiesFile)) {
         Properties externalProps = new Properties();
-        externalProps.load(new InputStreamReader(in, StandardCharsets.UTF_8));
+        externalProps.load(new InputStreamReader(is, StandardCharsets.UTF_8));
         coreProperties.putAll(externalProps);
       } catch (IOException e) {
         String message = String.format(Locale.ROOT, "Could not load properties from %s: %s:",
-            propertiesFile.getAbsoluteFile(), e.toString());
+            propertiesFile.toString(), e.toString());
         throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, message);
-      } finally {
-        IOUtils.closeQuietly(in);
       }
     }
   }
@@ -258,13 +261,6 @@ public class CoreDescriptor {
     }
   }
 
-  protected File resolvePaths(String filepath) {
-    File file = new File(filepath);
-    if (file.isAbsolute())
-      return file;
-    return new File(getInstanceDir(), filepath);
-  }
-
   /**
    * Is this property a Solr-standard property, or is it an extra property
    * defined per-core by the user?
@@ -283,33 +279,6 @@ public class CoreDescriptor {
     return value;
   }
 
-  /**
-   * Create a new CoreDescriptor with a given name and instancedir
-   * @param container     the CoreDescriptor's container
-   * @param name          the CoreDescriptor's name
-   * @param instanceDir   the CoreDescriptor's instancedir
-   */
-  public CoreDescriptor(CoreContainer container, String name, String instanceDir) {
-    this(container, name, instanceDir, new Properties());
-  }
-
-  /**
-   * Create a new CoreDescriptor using the properties of an existing one
-   * @param coreName the new CoreDescriptor's name
-   * @param other    the CoreDescriptor to copy
-   */
-  public CoreDescriptor(String coreName, CoreDescriptor other) {
-    this.coreContainer = other.coreContainer;
-    this.cloudDesc = other.cloudDesc;
-    this.originalExtraProperties.putAll(other.originalExtraProperties);
-    this.originalCoreProperties.putAll(other.originalCoreProperties);
-    this.coreProperties.putAll(other.coreProperties);
-    this.substitutableProperties.putAll(other.substitutableProperties);
-    this.coreProperties.setProperty(CORE_NAME, coreName);
-    this.originalCoreProperties.setProperty(CORE_NAME, coreName);
-    this.substitutableProperties.setProperty(SOLR_CORE_PROP_PREFIX + CORE_NAME, coreName);
-  }
-
   public String getPropertiesName() {
     return coreProperties.getProperty(CORE_PROPERTIES);
   }
@@ -322,22 +291,11 @@ public class CoreDescriptor {
     return defaultProperties.get(CORE_DATADIR).equals(coreProperties.getProperty(CORE_DATADIR));
   }
 
-  /**@return the core instance directory. */
-  public String getRawInstanceDir() {
-    return coreProperties.getProperty(CORE_INSTDIR);
-  }
-
-  private static String convertToAbsolute(String instDir, String solrHome) {
-    checkNotNull(instDir);
-    return SolrResourceLoader.normalizeDir(Paths.get(solrHome).resolve(instDir).toString());
-  }
-
   /**
-   *
-   * @return the core instance directory, prepended with solr_home if not an absolute path.
+   * @return the core instance directory
    */
-  public String getInstanceDir() {
-    return coreProperties.getProperty(CORE_ABS_INSTDIR);
+  public Path getInstanceDir() {
+    return instanceDir;
   }
 
   /**@return the core configuration resource name. */
@@ -401,12 +359,7 @@ public class CoreDescriptor {
 
   @Override
   public String toString() {
-    return new StringBuilder("CoreDescriptor[name=")
-        .append(this.getName())
-        .append(";instanceDir=")
-        .append(this.getInstanceDir())
-        .append("]")
-        .toString();
+    return "CoreDescriptor[name=" + this.getName() + ";instanceDir=" + this.getInstanceDir() + "]";
   }
 
   public String getConfigSet() {

Modified: lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/CorePropertiesLocator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/CorePropertiesLocator.java?rev=1717260&r1=1717259&r2=1717260&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/CorePropertiesLocator.java (original)
+++ lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/CorePropertiesLocator.java Mon Nov 30 14:27:42 2015
@@ -28,7 +28,9 @@ import java.nio.file.Files;
 import java.nio.file.Path;
 import java.nio.file.SimpleFileVisitor;
 import java.nio.file.attribute.BasicFileAttributes;
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
 import java.util.Properties;
 
 import com.google.common.collect.Lists;
@@ -158,7 +160,11 @@ public class CorePropertiesLocator imple
     try (InputStream fis = Files.newInputStream(propertiesFile)) {
       coreProperties.load(new InputStreamReader(fis, StandardCharsets.UTF_8));
       String name = createName(coreProperties, instanceDir);
-      return new CoreDescriptor(cc, name, instanceDir.toString(), coreProperties);
+      Map<String, String> propMap = new HashMap<>();
+      for (String key : coreProperties.stringPropertyNames()) {
+        propMap.put(key, coreProperties.getProperty(key));
+      }
+      return new CoreDescriptor(cc, name, instanceDir, propMap);
     }
     catch (IOException e) {
       logger.error("Couldn't load core descriptor from {}:{}", propertiesFile, e.toString());
@@ -175,9 +181,6 @@ public class CorePropertiesLocator imple
     Properties p = new Properties();
     p.putAll(cd.getPersistableStandardProperties());
     p.putAll(cd.getPersistableUserProperties());
-    // We don't persist the instance directory, as that's defined by the location
-    // of the properties file.
-    p.remove(CoreDescriptor.CORE_INSTDIR);
     return p;
   }
 

Modified: lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/DirectoryFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/DirectoryFactory.java?rev=1717260&r1=1717259&r2=1717260&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/DirectoryFactory.java (original)
+++ lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/DirectoryFactory.java Mon Nov 30 14:27:42 2015
@@ -265,8 +265,7 @@ public abstract class DirectoryFactory i
 
   public String getDataHome(CoreDescriptor cd) throws IOException {
     // by default, we go off the instance directory
-    String instanceDir = new File(cd.getInstanceDir()).getAbsolutePath();
-    return normalize(SolrResourceLoader.normalizeDir(instanceDir) + cd.getDataDir());
+    return cd.getInstanceDir().resolve(cd.getDataDir()).toAbsolutePath().toString();
   }
 
   /**

Modified: lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/SolrCore.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/SolrCore.java?rev=1717260&r1=1717259&r2=1717260&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/SolrCore.java (original)
+++ lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/core/SolrCore.java Mon Nov 30 14:27:42 2015
@@ -1006,10 +1006,7 @@ public final class SolrCore implements S
   private String initUpdateLogDir(CoreDescriptor coreDescriptor) {
     String updateLogDir = coreDescriptor.getUlogDir();
     if (updateLogDir == null) {
-      updateLogDir = dataDir;
-      if (new File(updateLogDir).isAbsolute() == false) {
-        updateLogDir = SolrResourceLoader.normalizeDir(coreDescriptor.getInstanceDir()) + updateLogDir;
-      }
+      updateLogDir = coreDescriptor.getInstanceDir().resolve(dataDir).normalize().toAbsolutePath().toString();
     }
     return updateLogDir;
   }
@@ -2492,12 +2489,11 @@ public final class SolrCore implements S
         public void postClose(SolrCore core) {
           CoreDescriptor cd = core.getCoreDescriptor();
           if (cd != null) {
-            File instanceDir = new File(cd.getInstanceDir());
             try {
-              FileUtils.deleteDirectory(instanceDir);
+              FileUtils.deleteDirectory(cd.getInstanceDir().toFile());
             } catch (IOException e) {
               SolrException.log(log, "Failed to delete instance dir for core:"
-                  + core.getName() + " dir:" + instanceDir.getAbsolutePath());
+                  + core.getName() + " dir:" + cd.getInstanceDir());
             }
           }
         }
@@ -2516,12 +2512,11 @@ public final class SolrCore implements S
       }
     }
     if (deleteInstanceDir) {
-      File instanceDir = new File(cd.getInstanceDir());
       try {
-        FileUtils.deleteDirectory(instanceDir);
+        FileUtils.deleteDirectory(cd.getInstanceDir().toFile());
       } catch (IOException e) {
         SolrException.log(log, "Failed to delete instance dir for unloaded core:" + cd.getName()
-            + " dir:" + instanceDir.getAbsolutePath());
+            + " dir:" + cd.getInstanceDir());
       }
     }
   }

Modified: lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/handler/SnapShooter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/handler/SnapShooter.java?rev=1717260&r1=1717259&r2=1717260&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/handler/SnapShooter.java (original)
+++ lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/handler/SnapShooter.java Mon Nov 30 14:27:42 2015
@@ -18,7 +18,6 @@ package org.apache.solr.handler;
 
 import java.io.File;
 import java.io.IOException;
-import java.nio.file.Paths;
 import java.text.SimpleDateFormat;
 import java.util.ArrayList;
 import java.util.Collection;
@@ -26,8 +25,6 @@ import java.util.Collections;
 import java.util.Date;
 import java.util.List;
 import java.util.Locale;
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
 
 import org.apache.lucene.index.IndexCommit;
 import org.apache.lucene.store.Directory;
@@ -63,8 +60,7 @@ public class SnapShooter {
       snapDir = core.getDataDir();
     }
     else  {
-      File base = new File(core.getCoreDescriptor().getInstanceDir());
-      snapDir = org.apache.solr.util.FileUtils.resolvePath(base, location).getAbsolutePath();
+      snapDir = core.getCoreDescriptor().getInstanceDir().resolve(location).normalize().toString();
     }
     this.snapshotName = snapshotName;
 

Modified: lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminHandler.java?rev=1717260&r1=1717259&r2=1717260&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminHandler.java (original)
+++ lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminHandler.java Mon Nov 30 14:27:42 2015
@@ -19,7 +19,16 @@ package org.apache.solr.handler.admin;
 
 import java.io.File;
 import java.io.IOException;
-import java.util.*;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Future;
 
@@ -31,7 +40,6 @@ import org.apache.lucene.search.MatchAll
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.IOUtils;
 import org.apache.solr.cloud.CloudDescriptor;
-import org.apache.solr.cloud.Overseer;
 import org.apache.solr.cloud.SyncStrategy;
 import org.apache.solr.cloud.ZkController;
 import org.apache.solr.common.SolrException;
@@ -71,6 +79,7 @@ import org.apache.solr.update.processor.
 import org.apache.solr.update.processor.UpdateRequestProcessorChain;
 import org.apache.solr.util.DefaultSolrThreadFactory;
 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;
@@ -547,37 +556,33 @@ public class CoreAdminHandler extends Re
 
   public static ImmutableMap<String, String> cloudParamToProp;
 
-  protected static CoreDescriptor buildCoreDescriptor(SolrParams params, CoreContainer container) {
+  protected static Map<String, String> buildCoreParams(SolrParams params) {
 
-    String name = checkNotEmpty(params.get(CoreAdminParams.NAME),
-        "Missing parameter [" + CoreAdminParams.NAME + "]");
+    Map<String, String> coreParams = new HashMap<>();
 
-    Properties coreProps = new Properties();
+    // standard core create parameters
     for (String param : paramToProp.keySet()) {
       String value = params.get(param, null);
       if (StringUtils.isNotEmpty(value)) {
-        coreProps.setProperty(paramToProp.get(param), value);
+        coreParams.put(paramToProp.get(param), value);
       }
     }
+
+    // extra properties
     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);
-    }
-
-    String instancedir = params.get(CoreAdminParams.INSTANCE_DIR);
-    if (StringUtils.isEmpty(instancedir) && coreProps.getProperty(CoreAdminParams.INSTANCE_DIR) != null) {
-      instancedir = coreProps.getProperty(CoreAdminParams.INSTANCE_DIR);
-    } else if (StringUtils.isEmpty(instancedir)){
-      instancedir = name; // will be resolved later against solr.home
-      //instancedir = container.getSolrHome() + "/" + name;
+      if (param.startsWith(CoreAdminParams.PROPERTY_PREFIX)) {
+        String propName = param.substring(CoreAdminParams.PROPERTY_PREFIX.length());
+        String propValue = params.get(param);
+        coreParams.put(propName, propValue);
+      }
+      if (param.startsWith(ZkController.COLLECTION_PARAM_PREFIX)) {
+        coreParams.put(param, params.get(param));
+      }
     }
 
-    return new CoreDescriptor(container, name, instancedir, coreProps, params);
+    return coreParams;
   }
 
   private static String checkNotEmpty(String value, String message) {
@@ -595,88 +600,23 @@ public class CoreAdminHandler extends Re
 
     SolrParams params = req.getParams();
     log.info("core create command {}", params);
-    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.");
-    }
+    String coreName = params.required().get(CoreAdminParams.NAME);
+    Map<String, String> coreParams = buildCoreParams(params);
 
-    // TODO this should be moved into CoreContainer, really...
-    boolean preExisitingZkEntry = false;
-    try {
-      if (coreContainer.getZkController() != null) {
-        if (!Overseer.isLegacy(coreContainer.getZkController().getZkStateReader().getClusterProps())) {
-          if (dcore.getCloudDescriptor().getCoreNodeName() == null) {
-            throw new SolrException(ErrorCode.SERVER_ERROR,
-                "non legacy mode coreNodeName missing " + params);
-            
-          }
-        }
-        
-        preExisitingZkEntry = checkIfCoreNodeNameAlreadyExists(dcore);
+    Path instancePath = coreContainer.getCoreRootDirectory().resolve(coreName);
 
-      }
-      
-      SolrCore core = coreContainer.create(dcore);
-      
-      // only write out the descriptor if the core is successfully created
-      coreContainer.getCoresLocator().create(coreContainer, dcore);
-
-      rsp.add("core", core.getName());
-    }
-    catch (Exception ex) {
-      if (coreContainer.isZooKeeperAware() && dcore != null && !preExisitingZkEntry) {
-        try {
-          coreContainer.getZkController().unregister(dcore.getName(), dcore);
-        } catch (InterruptedException e) {
-          Thread.currentThread().interrupt();
-          SolrException.log(log, null, e);
-        } catch (KeeperException e) {
-          SolrException.log(log, null, e);
-        }
-      }
-      
-      Throwable tc = ex;
-      Throwable c = null;
-      do {
-        tc = tc.getCause();
-        if (tc != null) {
-          c = tc;
-        }
-      } while (tc != null);
-      
-      String rootMsg = "";
-      if (c != null) {
-        rootMsg = " Caused by: " + c.getMessage();
-      }
-      
-      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
-          "Error CREATEing SolrCore '" + dcore.getName() + "': " +
-          ex.getMessage() + rootMsg, ex);
+    // TODO: Should we nuke setting odd instance paths?  They break core discovery, generally
+    String instanceDir = req.getParams().get(CoreAdminParams.INSTANCE_DIR);
+    if (instanceDir == null)
+      instanceDir = req.getParams().get("property.instanceDir");
+    if (instanceDir != null) {
+      instanceDir = PropertiesUtil.substituteProperty(instanceDir, coreContainer.getContainerProperties());
+      instancePath = coreContainer.getCoreRootDirectory().resolve(instanceDir).normalize();
     }
-  }
 
+    coreContainer.create(coreName, instancePath, coreParams);
 
-  private boolean checkIfCoreNodeNameAlreadyExists(CoreDescriptor dcore) {
-    ZkStateReader zkStateReader = coreContainer.getZkController()
-        .getZkStateReader();
-    DocCollection collection = zkStateReader.getClusterState().getCollectionOrNull(dcore.getCollectionName());
-    if (collection != null) {
-      Collection<Slice> slices = collection.getSlices();
-      
-      for (Slice slice : slices) {
-        Collection<Replica> replicas = slice.getReplicas();
-        for (Replica replica : replicas) {
-          if (replica.getName().equals(
-              dcore.getCloudDescriptor().getCoreNodeName())) {
-            return true;
-          }
-        }
-      }
-    }
-    return false;
+    rsp.add("core", coreName);
   }
 
   /**

Modified: lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/TestCrossCoreJoin.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/TestCrossCoreJoin.java?rev=1717260&r1=1717259&r2=1717260&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/TestCrossCoreJoin.java (original)
+++ lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/TestCrossCoreJoin.java Mon Nov 30 14:27:42 2015
@@ -20,9 +20,9 @@ package org.apache.solr;
 import java.io.StringWriter;
 import java.util.Collections;
 
+import com.google.common.collect.ImmutableMap;
 import org.apache.solr.common.SolrException.ErrorCode;
 import org.apache.solr.core.CoreContainer;
-import org.apache.solr.core.CoreDescriptor;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.request.LocalSolrQueryRequest;
 import org.apache.solr.request.SolrQueryRequest;
@@ -49,15 +49,8 @@ public class TestCrossCoreJoin extends S
     // FileUtils.copyDirectory(getFile("solrj/solr"), testHome);
     initCore("solrconfig.xml", "schema12.xml", TEST_HOME(), "collection1");
     final CoreContainer coreContainer = h.getCoreContainer();
-    final CoreDescriptor toCoreDescriptor = coreContainer.getCoreDescriptor("collection1");
-    final CoreDescriptor fromCoreDescriptor = new CoreDescriptor("fromCore", toCoreDescriptor) {
-      @Override
-      public String getSchemaName() {
-        return "schema.xml";
-      }
-    };
 
-    fromCore = coreContainer.create(fromCoreDescriptor);
+    fromCore = coreContainer.create("fromCore", ImmutableMap.of("configSet", "minimal"));
 
     assertU(add(doc("id", "1", "name", "john", "title", "Director", "dept_s", "Engineering")));
     assertU(add(doc("id", "2", "name", "mark", "title", "VP", "dept_s", "Marketing")));
@@ -108,15 +101,17 @@ public class TestCrossCoreJoin extends S
   @Test
   public void testCoresAreDifferent() throws Exception {
     assertQEx("schema12.xml" + " has no \"cat\" field", req("cat:*"), ErrorCode.BAD_REQUEST);
-    final LocalSolrQueryRequest req = new LocalSolrQueryRequest(fromCore, "cat:*", "lucene", 0, 100, Collections.emptyMap());
+    final LocalSolrQueryRequest req = new LocalSolrQueryRequest(fromCore, "cat:*", "/select", 0, 100, Collections.emptyMap());
     final String resp = query(fromCore, req);
     assertTrue(resp, resp.contains("numFound=\"1\""));
-    assertTrue(resp, resp.contains("<int name=\"id\">10</int>"));
+    assertTrue(resp, resp.contains("<str name=\"id\">10</str>"));
 
   }
 
   public String query(SolrCore core, SolrQueryRequest req) throws Exception {
     String handler = "standard";
+    if (req.getParams().get("qt") != null)
+      handler = req.getParams().get("qt");
     SolrQueryResponse rsp = new SolrQueryResponse();
     SolrRequestInfo.setRequestInfo(new SolrRequestInfo(req, rsp));
     core.execute(core.getRequestHandler(handler), req, rsp);

Modified: lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/cloud/ClusterStateUpdateTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/cloud/ClusterStateUpdateTest.java?rev=1717260&r1=1717259&r2=1717260&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/cloud/ClusterStateUpdateTest.java (original)
+++ lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/cloud/ClusterStateUpdateTest.java Mon Nov 30 14:27:42 2015
@@ -17,6 +17,13 @@ package org.apache.solr.cloud;
  * limitations under the License.
  */
 
+import java.io.File;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+
+import com.google.common.collect.ImmutableMap;
 import org.apache.lucene.util.LuceneTestCase.Slow;
 import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.common.cloud.ClusterState;
@@ -27,8 +34,6 @@ import org.apache.solr.common.cloud.ZkNo
 import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.util.Utils;
 import org.apache.solr.core.CoreContainer;
-import org.apache.solr.core.CoreDescriptor;
-import org.apache.solr.core.SolrCore;
 import org.apache.zookeeper.CreateMode;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
@@ -36,12 +41,6 @@ import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.io.File;
-import java.io.IOException;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Set;
-
 @Slow
 public class ClusterStateUpdateTest extends SolrTestCaseJ4  {
   protected static Logger log = LoggerFactory
@@ -157,10 +156,7 @@ public class ClusterStateUpdateTest exte
         CreateMode.PERSISTENT, true);
     zkClient.close();
 
-    CoreDescriptor dcore = buildCoreDescriptor(container1, "testcore", "testcore")
-                              .withDataDir(dataDir4.getAbsolutePath()).build();
-
-    SolrCore core = container1.create(dcore);
+    container1.create("testcore", ImmutableMap.of("dataDir", dataDir4.getAbsolutePath()));
     
     ZkController zkController2 = container2.getZkController();
 

Modified: lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/cloud/CollectionsAPISolrJTests.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/cloud/CollectionsAPISolrJTests.java?rev=1717260&r1=1717259&r2=1717260&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/cloud/CollectionsAPISolrJTests.java (original)
+++ lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/cloud/CollectionsAPISolrJTests.java Mon Nov 30 14:27:42 2015
@@ -17,6 +17,17 @@ package org.apache.solr.cloud;
  * limitations under the License.
  */
 
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Properties;
+import java.util.concurrent.TimeUnit;
+
 import org.apache.commons.codec.binary.StringUtils;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.TestUtil;
@@ -37,17 +48,6 @@ import org.apache.solr.util.TimeOut;
 import org.apache.zookeeper.KeeperException;
 import org.junit.Test;
 
-import java.io.File;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
-import java.util.Objects;
-import java.util.Properties;
-import java.util.concurrent.TimeUnit;
-
 import static org.apache.solr.cloud.ReplicaPropertiesBase.verifyUniqueAcrossCollection;
 
 @LuceneTestCase.Slow
@@ -232,12 +232,10 @@ public class CollectionsAPISolrJTests ex
     String collectionName = "solrj_test_core_props";
     
     File tmpDir = createTempDir("testPropertyParamsForCreate").toFile();
-    File instanceDir = new File(tmpDir, "instanceDir-" + TestUtil.randomSimpleString(random(), 1, 5));
     File dataDir = new File(tmpDir, "dataDir-" + TestUtil.randomSimpleString(random(), 1, 5));
     File ulogDir = new File(tmpDir, "ulogDir-" + TestUtil.randomSimpleString(random(), 1, 5));
 
     Properties properties = new Properties();
-    properties.put(CoreAdminParams.INSTANCE_DIR, instanceDir.getAbsolutePath());
     properties.put(CoreAdminParams.DATA_DIR, dataDir.getAbsolutePath());
     properties.put(CoreAdminParams.ULOG_DIR, ulogDir.getAbsolutePath());
 
@@ -262,9 +260,6 @@ public class CollectionsAPISolrJTests ex
       CoreAdminResponse status = CoreAdminRequest.getStatus(replica1.getStr("core"), client);
       NamedList<Object> coreStatus = status.getCoreStatus(replica1.getStr("core"));
       String dataDirStr = (String) coreStatus.get("dataDir");
-      String instanceDirStr = (String) coreStatus.get("instanceDir");
-      assertEquals("Instance dir does not match param passed in property.instanceDir syntax",
-          new File(instanceDirStr).getAbsolutePath(), instanceDir.getAbsolutePath());
       assertEquals("Data dir does not match param given in property.dataDir syntax",
           new File(dataDirStr).getAbsolutePath(), dataDir.getAbsolutePath());
     }

Modified: lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/cloud/LeaderElectionIntegrationTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/cloud/LeaderElectionIntegrationTest.java?rev=1717260&r1=1717259&r2=1717260&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/cloud/LeaderElectionIntegrationTest.java (original)
+++ lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/cloud/LeaderElectionIntegrationTest.java Mon Nov 30 14:27:42 2015
@@ -17,13 +17,26 @@ package org.apache.solr.cloud;
  * limitations under the License.
  */
 
+import javax.xml.parsers.ParserConfigurationException;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import com.google.common.collect.ImmutableMap;
 import org.apache.lucene.util.LuceneTestCase.Slow;
 import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.common.cloud.SolrZkClient;
 import org.apache.solr.common.cloud.ZkNodeProps;
 import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.core.CoreContainer;
-import org.apache.solr.core.CoreDescriptor;
+import org.apache.solr.core.SolrResourceLoader;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -31,17 +44,6 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.xml.sax.SAXException;
 
-import javax.xml.parsers.ParserConfigurationException;
-import java.io.File;
-import java.io.IOException;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
-
 @Slow
 public class LeaderElectionIntegrationTest extends SolrTestCaseJ4 {
   protected static Logger log = LoggerFactory
@@ -133,11 +135,11 @@ public class LeaderElectionIntegrationTe
      
   private void setupContainer(int port, String shard) throws IOException,
       ParserConfigurationException, SAXException {
-    File data = createTempDir().toFile();
+    Path data = createTempDir();
     
     System.setProperty("hostPort", Integer.toString(port));
     System.setProperty("shard", shard);
-    System.setProperty("solr.data.dir", data.getAbsolutePath());
+    System.setProperty("solr.data.dir", data.toString());
     System.setProperty("solr.solr.home", TEST_HOME());
     Set<Integer> ports = shardPorts.get(shard);
     if (ports == null) {
@@ -145,9 +147,12 @@ public class LeaderElectionIntegrationTe
       shardPorts.put(shard, ports);
     }
     ports.add(port);
-    CoreContainer container = new CoreContainer();
+
+    SolrResourceLoader loader = new SolrResourceLoader(createTempDir());
+    Files.copy(TEST_PATH().resolve("solr.xml"), loader.getInstancePath().resolve("solr.xml"));
+    CoreContainer container = new CoreContainer(loader);
     container.load();
-    container.create(new CoreDescriptor(container, "collection1", "collection1", "collection", "collection1"));
+    container.create("collection1_" + shard, ImmutableMap.of("collection", "collection1"));
     containerMap.put(port, container);
     System.clearProperty("solr.solr.home");
     System.clearProperty("hostPort");

Modified: lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/core/TestConfigSets.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/core/TestConfigSets.java?rev=1717260&r1=1717259&r2=1717260&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/core/TestConfigSets.java (original)
+++ lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/core/TestConfigSets.java Mon Nov 30 14:27:42 2015
@@ -23,6 +23,7 @@ import java.nio.file.Path;
 import java.nio.file.Paths;
 
 import com.carrotsearch.randomizedtesting.rules.SystemPropertiesRestoreRule;
+import com.google.common.collect.ImmutableMap;
 import org.apache.commons.io.FileUtils;
 import org.apache.solr.SolrTestCaseJ4;
 import org.junit.Rule;
@@ -76,8 +77,7 @@ public class TestConfigSets extends Solr
       container = setupContainer(TEST_PATH().resolve("configsets").toString());
       Path testDirectory = container.getResourceLoader().getInstancePath();
 
-      SolrCore core1 = container.create(new CoreDescriptor(container, "core1", testDirectory.resolve("core1").toString(),
-                                                "configSet", "configset-2"));
+      SolrCore core1 = container.create("core1", ImmutableMap.of("configSet", "configset-2"));
       assertThat(core1.getCoreDescriptor().getName(), is("core1"));
       assertThat(Paths.get(core1.getDataDir()).toString(), is(testDirectory.resolve("core1").resolve("data").toString()));
     }
@@ -94,7 +94,7 @@ public class TestConfigSets extends Solr
       container = setupContainer(getFile("solr/configsets").getAbsolutePath());
       Path testDirectory = container.getResourceLoader().getInstancePath();
 
-      container.create(new CoreDescriptor(container, "core1", testDirectory.resolve("core1").toString(), "configSet", "nonexistent"));
+      container.create("core1", ImmutableMap.of("configSet", "nonexistent"));
       fail("Expected core creation to fail");
     }
     catch (Exception e) {
@@ -122,12 +122,12 @@ public class TestConfigSets extends Solr
     CoreContainer container = new CoreContainer(SolrXmlConfig.fromString(loader, solrxml));
     container.load();
 
-    // We initially don't have a /get handler defined
-    SolrCore core = container.create(new CoreDescriptor(container, "core1", testDirectory + "/core", "configSet", "configset-2"));
+    // We initially don't have a /dump handler defined
+    SolrCore core = container.create("core1", ImmutableMap.of("configSet", "configset-2"));
     assertThat("No /dump handler should be defined in the initial configuration",
         core.getRequestHandler("/dump"), is(nullValue()));
 
-    // Now copy in a config with a /get handler and reload
+    // Now copy in a config with a /dump handler and reload
     FileUtils.copyFile(getFile("solr/collection1/conf/solrconfig-withgethandler.xml"),
         new File(new File(configSetsDir, "configset-2/conf"), "solrconfig.xml"));
     container.reload("core1");

Modified: lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/core/TestCoreContainer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/core/TestCoreContainer.java?rev=1717260&r1=1717259&r2=1717260&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/core/TestCoreContainer.java (original)
+++ lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/core/TestCoreContainer.java Mon Nov 30 14:27:42 2015
@@ -29,6 +29,8 @@ import java.util.jar.JarEntry;
 import java.util.jar.JarOutputStream;
 import java.util.regex.Pattern;
 
+import com.google.common.base.Throwables;
+import com.google.common.collect.ImmutableMap;
 import org.apache.commons.io.FileUtils;
 import org.apache.lucene.util.IOUtils;
 import org.apache.solr.SolrTestCaseJ4;
@@ -88,11 +90,8 @@ public class TestCoreContainer extends S
     CoreContainer cores = init(CONFIGSETS_SOLR_XML);
 
     try {
-      CoreDescriptor descriptor1 = new CoreDescriptor(cores, "core1", "./collection1", "configSet", "minimal");
-      SolrCore core1 = cores.create(descriptor1);
-      
-      CoreDescriptor descriptor2 = new CoreDescriptor(cores, "core2", "./collection1", "configSet", "minimal");
-      SolrCore core2 = cores.create(descriptor2);
+      SolrCore core1 = cores.create("core1", ImmutableMap.of("configSet", "minimal"));
+      SolrCore core2 = cores.create("core2", ImmutableMap.of("configSet", "minimal"));
       
       assertSame(core1.getLatestSchema(), core2.getLatestSchema());
 
@@ -105,9 +104,8 @@ public class TestCoreContainer extends S
   @Test
   public void testReloadSequential() throws Exception {
     final CoreContainer cc = init(CONFIGSETS_SOLR_XML);
-    CoreDescriptor descriptor1 = new CoreDescriptor(cc, "core1", "./collection1", "configSet", "minimal");
-    cc.create(descriptor1);
     try {
+      cc.create("core1", ImmutableMap.of("configSet", "minimal"));
       cc.reload("core1");
       cc.reload("core1");
       cc.reload("core1");
@@ -121,8 +119,7 @@ public class TestCoreContainer extends S
   @Test
   public void testReloadThreaded() throws Exception {
     final CoreContainer cc = init(CONFIGSETS_SOLR_XML);
-    CoreDescriptor descriptor1 = new CoreDescriptor(cc, "core1", "./collection1", "configSet", "minimal");
-    cc.create(descriptor1);
+    cc.create("core1", ImmutableMap.of("configSet", "minimal"));
 
     class TestThread extends Thread {
       @Override
@@ -159,8 +156,7 @@ public class TestCoreContainer extends S
       assertEquals("There should not be cores", 0, cores.getCores().size());
       
       //add a new core
-      CoreDescriptor coreDescriptor = new CoreDescriptor(cores, "core1", "collection1", CoreDescriptor.CORE_CONFIGSET, "minimal");
-      SolrCore newCore = cores.create(coreDescriptor);
+      cores.create("core1", ImmutableMap.of("configSet", "minimal"));
 
       //assert one registered core
 
@@ -218,7 +214,8 @@ public class TestCoreContainer extends S
     System.setProperty("configsets", getFile("solr/configsets").getAbsolutePath());
 
     final CoreContainer cc = new CoreContainer(SolrXmlConfig.fromString(resourceLoader, CONFIGSETS_SOLR_XML), new Properties(), cl);
-    CoreDescriptor badcore = new CoreDescriptor(cc, "badcore", "badcore", "configSet", "nosuchconfigset");
+    Path corePath = resourceLoader.getInstancePath().resolve("badcore");
+    CoreDescriptor badcore = new CoreDescriptor(cc, "badcore", corePath, "configSet", "nosuchconfigset");
     cl.add(badcore);
 
     try {
@@ -229,7 +226,7 @@ public class TestCoreContainer extends S
       assertThat(cc.getCoreInitFailures().size(), is(0));
 
       // can we create the core now with a good config?
-      SolrCore core = cc.create(new CoreDescriptor(cc, "badcore", "badcore", "configSet", "minimal"));
+      SolrCore core = cc.create("badcore", ImmutableMap.of("configSet", "minimal"));
       assertThat(core, not(nullValue()));
 
     }
@@ -246,8 +243,7 @@ public class TestCoreContainer extends S
       ClassLoader contextLoader = Thread.currentThread().getContextClassLoader();
       assertSame(contextLoader, sharedLoader.getParent());
 
-      CoreDescriptor descriptor1 = new CoreDescriptor(cc, "core1", "./collection1", "configSet", "minimal");
-      SolrCore core1 = cc.create(descriptor1);
+      SolrCore core1 = cc.create("core1", ImmutableMap.of("configSet", "minimal"));
       ClassLoader coreLoader = core1.getResourceLoader().getClassLoader();
       assertSame(sharedLoader, coreLoader.getParent());
 
@@ -413,15 +409,15 @@ public class TestCoreContainer extends S
     assertEquals("wrong number of core failures", 0, failures.size());
 
     // -----
-    // try to add a collection with a path that doesn't exist
-    final CoreDescriptor bogus = new CoreDescriptor(cc, "bogus", "bogus_path");
+    // try to add a collection with a configset that doesn't exist
     try {
       ignoreException(Pattern.quote("bogus_path"));
-      cc.create(bogus);
+      cc.create("bogus", ImmutableMap.of("configSet", "bogus_path"));
       fail("bogus inst dir failed to trigger exception from create");
     } catch (SolrException e) {
-      assertTrue("init exception doesn't mention bogus dir: " + e.getCause().getCause().getMessage(),
-          0 < e.getCause().getCause().getMessage().indexOf("bogus_path"));
+      Throwable cause = Throwables.getRootCause(e);
+      assertTrue("init exception doesn't mention bogus dir: " + cause.getMessage(),
+          0 < cause.getMessage().indexOf("bogus_path"));
 
     }
 
@@ -436,8 +432,8 @@ public class TestCoreContainer extends S
     assertEquals("wrong number of core failures", 1, failures.size());
     fail = failures.get("bogus").exception;
     assertNotNull("null failure for test core", fail);
-    assertTrue("init failure doesn't mention problem: " + fail.getCause().getMessage(),
-        0 < fail.getCause().getMessage().indexOf("bogus_path"));
+    assertTrue("init failure doesn't mention problem: " + fail.getMessage(),
+        0 < fail.getMessage().indexOf("bogus_path"));
 
     // check that we get null accessing a non-existent core
     assertNull(cc.getCore("does_not_exist"));
@@ -447,8 +443,7 @@ public class TestCoreContainer extends S
       fail("Failed to get Exception on accessing core with init failure");
     } catch (SolrException ex) {
       assertEquals(500, ex.code());
-      // double wrapped
-      String cause = ex.getCause().getCause().getMessage();
+      String cause = Throwables.getRootCause(ex).getMessage();
       assertTrue("getCore() ex cause doesn't mention init fail: " + cause,
           0 < cause.indexOf("bogus_path"));
 
@@ -474,8 +469,8 @@ public class TestCoreContainer extends S
     System.setProperty("configsets", getFile("solr/configsets").getAbsolutePath());
 
     final CoreContainer cc = new CoreContainer(SolrXmlConfig.fromString(resourceLoader, CONFIGSETS_SOLR_XML), new Properties(), cl);
-    cl.add(new CoreDescriptor(cc, "col_ok", "col_ok", "configSet", "minimal"));
-    cl.add(new CoreDescriptor(cc, "col_bad", "col_bad", "configSet", "bad-mergepolicy"));
+    cl.add(new CoreDescriptor(cc, "col_ok", resourceLoader.getInstancePath().resolve("col_ok"), "configSet", "minimal"));
+    cl.add(new CoreDescriptor(cc, "col_bad", resourceLoader.getInstancePath().resolve("col_bad"), "configSet", "bad-mergepolicy"));
     cc.load();
 
     // check that we have the cores we expect
@@ -513,8 +508,7 @@ public class TestCoreContainer extends S
         FileUtils.getFile(cc.getSolrHome(), "col_bad", "conf", "solrconfig.xml"));
     FileUtils.copyFile(getFile("solr/collection1/conf/schema-minimal.xml"),
         FileUtils.getFile(cc.getSolrHome(), "col_bad", "conf", "schema.xml"));
-    final CoreDescriptor fixed = new CoreDescriptor(cc, "col_bad", "col_bad");
-    cc.create(fixed);
+    cc.create("col_bad", ImmutableMap.<String, String>of());
 
     // check that we have the cores we expect
     cores = cc.getCoreNames();
@@ -531,10 +525,9 @@ public class TestCoreContainer extends S
 
     // -----
     // try to add a collection with a path that doesn't exist
-    final CoreDescriptor bogus = new CoreDescriptor(cc, "bogus", "bogus_path");
     try {
       ignoreException(Pattern.quote("bogus_path"));
-      cc.create(bogus);
+      cc.create("bogus", ImmutableMap.of("configSet", "bogus_path"));
       fail("bogus inst dir failed to trigger exception from create");
     } catch (SolrException e) {
       assertTrue("init exception doesn't mention bogus dir: " + e.getCause().getCause().getMessage(),
@@ -555,8 +548,8 @@ public class TestCoreContainer extends S
     assertEquals("wrong number of core failures", 1, failures.size());
     fail = failures.get("bogus").exception;
     assertNotNull("null failure for test core", fail);
-    assertTrue("init failure doesn't mention problem: " + fail.getCause().getMessage(),
-        0 < fail.getCause().getMessage().indexOf("bogus_path"));
+    assertTrue("init failure doesn't mention problem: " + fail.getMessage(),
+        0 < fail.getMessage().indexOf("bogus_path"));
 
     // check that we get null accessing a non-existent core
     assertNull(cc.getCore("does_not_exist"));
@@ -567,7 +560,7 @@ public class TestCoreContainer extends S
     } catch (SolrException ex) {
       assertEquals(500, ex.code());
       // double wrapped
-      String cause = ex.getCause().getCause().getMessage();
+      String cause = ex.getCause().getMessage();
       assertTrue("getCore() ex cause doesn't mention init fail: " + cause,
           0 < cause.indexOf("bogus_path"));
     }

Modified: lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/core/TestCoreDiscovery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/core/TestCoreDiscovery.java?rev=1717260&r1=1717259&r2=1717260&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/core/TestCoreDiscovery.java (original)
+++ lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/core/TestCoreDiscovery.java Mon Nov 30 14:27:42 2015
@@ -25,6 +25,7 @@ import java.nio.charset.StandardCharsets
 import java.nio.file.Paths;
 import java.util.Properties;
 
+import com.google.common.collect.ImmutableMap;
 import org.apache.commons.io.FileUtils;
 import org.apache.lucene.util.IOUtils;
 import org.apache.solr.SolrTestCaseJ4;
@@ -68,7 +69,6 @@ public class TestCoreDiscovery extends S
     props.put(CoreDescriptor.CORE_TRANSIENT, Boolean.toString(isLazy));
     props.put(CoreDescriptor.CORE_LOADONSTARTUP, Boolean.toString(loadOnStartup));
     props.put(CoreDescriptor.CORE_DATADIR, "${core.dataDir:stuffandnonsense}");
-    props.put(CoreDescriptor.CORE_INSTDIR, "totallybogus"); // For testing that this property is ignored if present.
 
     for (String extra : extraProps) {
       String[] parts = extra.split("=");
@@ -155,9 +155,6 @@ public class TestCoreDiscovery extends S
         // This is too long and ugly to put in. Besides, it varies.
         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.getDataDir());
         assertEquals("solrconfig-minimal.xml", desc.getConfigName());
         assertEquals("schema-tiny.xml", desc.getSchemaName());
@@ -302,8 +299,8 @@ public class TestCoreDiscovery extends S
 
       assertNull(cc.getCore("core0"));
 
-      SolrCore core3 = cc.create(new CoreDescriptor(cc, "core3", "core3", "configSet", "minimal"));
-      assertThat(core3.getCoreDescriptor().getInstanceDir(), containsString("relative"));
+      SolrCore core3 = cc.create("core3", ImmutableMap.of("configSet", "minimal"));
+      assertThat(core3.getCoreDescriptor().getInstanceDir().toAbsolutePath().toString(), containsString("relative"));
 
     } finally {
       cc.shutdown();