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

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

Modified: lucene/dev/trunk/solr/core/src/test/org/apache/solr/core/TestSolrXmlPersistence.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/core/TestSolrXmlPersistence.java?rev=1502276&r1=1502275&r2=1502276&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/core/TestSolrXmlPersistence.java (original)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/core/TestSolrXmlPersistence.java Thu Jul 11 16:35:01 2013
@@ -18,6 +18,7 @@
 package org.apache.solr.core;
 
 import com.carrotsearch.randomizedtesting.rules.SystemPropertiesRestoreRule;
+import com.google.common.base.Charsets;
 import org.apache.commons.io.FileUtils;
 import org.apache.commons.lang.StringUtils;
 import org.apache.lucene.util.IOUtils;
@@ -25,6 +26,7 @@ import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.common.params.CoreAdminParams;
 import org.apache.solr.handler.admin.CoreAdminHandler;
 import org.apache.solr.response.SolrQueryResponse;
+import org.apache.solr.util.TestHarness;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.RuleChain;
@@ -38,8 +40,11 @@ import org.xml.sax.SAXException;
 import javax.xml.parsers.DocumentBuilder;
 import javax.xml.parsers.DocumentBuilderFactory;
 import javax.xml.parsers.ParserConfigurationException;
+import java.io.ByteArrayInputStream;
 import java.io.File;
+import java.io.FileInputStream;
 import java.io.IOException;
+import java.io.InputStream;
 import java.util.ArrayList;
 import java.util.List;
 
@@ -47,13 +52,6 @@ public class TestSolrXmlPersistence exte
 
   private File solrHomeDirectory = new File(TEMP_DIR, this.getClass().getName());
 
-  /*
-  @BeforeClass
-  public static void beforeClass() throws Exception {
-    initCore("solrconfig-minimal.xml", "schema-tiny.xml");
-  }
-  */
-
   @Rule
   public TestRule solrTestRules =
       RuleChain.outerRule(new SystemPropertiesRestoreRule());
@@ -89,18 +87,7 @@ public class TestSolrXmlPersistence exte
 
     CoreContainer cc = init(SOLR_XML_LOTS_SYSVARS, "SystemVars1", "SystemVars2");
     try {
-
-      // This seems odd, but it's just a little self check to see if the comparison strings are being created correctly
-      persistContainedInOrig(cc, new File(solrHomeDirectory, "solr_copy.xml"));
-
-      // Is everything in the persisted file identical to the original?
-      final File persistXml = new File(solrHomeDirectory, "sysvars.solr.xml");
-      // Side effect here is that the new file is persisted and available later.
-      persistContainedInOrig(cc, persistXml);
-
-      // Is everything in the original contained in the persisted one?
-      assertXmlFile(persistXml, getAllNodes(new File(solrHomeDirectory, "solr.xml")));
-
+      origMatchesPersist(cc, SOLR_XML_LOTS_SYSVARS);
     } finally {
       cc.shutdown();
       if (solrHomeDirectory.exists()) {
@@ -130,8 +117,7 @@ public class TestSolrXmlPersistence exte
               resp);
       assertNull("Exception on reload", resp.getException());
 
-      persistContainedInOrig(cc, new File(solrHomeDirectory, "reload1.solr.xml"));
-
+      origMatchesPersist(cc, SOLR_XML_LOTS_SYSVARS);
     } finally {
       cc.shutdown();
       if (solrHomeDirectory.exists()) {
@@ -149,6 +135,9 @@ public class TestSolrXmlPersistence exte
 
   private void doTestRename(String which) throws Exception {
     CoreContainer cc = init(SOLR_XML_LOTS_SYSVARS, "SystemVars1", "SystemVars2");
+    SolrXMLCoresLocator.NonPersistingLocator locator
+        = (SolrXMLCoresLocator.NonPersistingLocator) cc.getCoresLocator();
+
     try {
       final CoreAdminHandler admin = new CoreAdminHandler(cc);
       SolrQueryResponse resp = new SolrQueryResponse();
@@ -160,31 +149,29 @@ public class TestSolrXmlPersistence exte
               resp);
       assertNull("Exception on rename", resp.getException());
 
-      File persistXml = new File(solrHomeDirectory, "rename.solr.xml");
-      File origXml = new File(solrHomeDirectory, "solr.xml");
-
       // OK, Assure that if I change everything that has been renamed with the original value for the core, it matches
       // the old list
-      cc.persistFile(persistXml);
-      String[] persistList = getAllNodes(persistXml);
+      String[] persistList = getAllNodes();
       String[] expressions = new String[persistList.length];
 
       for (int idx = 0; idx < persistList.length; ++idx) {
         expressions[idx] = persistList[idx].replaceAll("RenamedCore", which);
       }
 
-      assertXmlFile(origXml, expressions);
+      //assertXmlFile(origXml, expressions);
+      TestHarness.validateXPath(SOLR_XML_LOTS_SYSVARS, expressions);
 
       // Now the other way, If I replace the original name in the original XML file with "RenamedCore", does it match
       // what was persisted?
-      persistList = getAllNodes(origXml);
+      persistList = getAllNodes(SOLR_XML_LOTS_SYSVARS);
       expressions = new String[persistList.length];
       for (int idx = 0; idx < persistList.length; ++idx) {
         // /solr/cores/core[@name='SystemVars1' and @collection='${collection:collection1}']
         expressions[idx] = persistList[idx].replace("@name='" + which + "'", "@name='RenamedCore'");
       }
 
-      assertXmlFile(persistXml, expressions);
+      TestHarness.validateXPath(locator.xml, expressions);
+
     } finally {
       cc.shutdown();
       if (solrHomeDirectory.exists()) {
@@ -212,11 +199,7 @@ public class TestSolrXmlPersistence exte
               resp);
       assertNull("Exception on swap", resp.getException());
 
-      File persistXml = new File(solrHomeDirectory, "rename.solr.xml");
-      File origXml = new File(solrHomeDirectory, "solr.xml");
-
-      cc.persistFile(persistXml);
-      String[] persistList = getAllNodes(persistXml);
+      String[] persistList = getAllNodes();
       String[] expressions = new String[persistList.length];
 
       // Now manually change the names back and it should match exactly to the original XML.
@@ -230,7 +213,8 @@ public class TestSolrXmlPersistence exte
         }
       }
 
-      assertXmlFile(origXml, expressions);
+      //assertXmlFile(origXml, expressions);
+      TestHarness.validateXPath(SOLR_XML_LOTS_SYSVARS, expressions);
 
     } finally {
       cc.shutdown();
@@ -244,8 +228,8 @@ public class TestSolrXmlPersistence exte
   public void testMinimalXml() throws Exception {
     CoreContainer cc = init(SOLR_XML_MINIMAL, "SystemVars1");
     try {
-      persistContainedInOrig(cc, new File(solrHomeDirectory, "minimal.solr.xml"));
-      origContainedInPersist(cc, new File(solrHomeDirectory, "minimal.solr.xml"));
+      cc.shutdown();
+      origMatchesPersist(cc, SOLR_XML_MINIMAL);
     } finally {
       cc.shutdown();
       if (solrHomeDirectory.exists()) {
@@ -254,7 +238,13 @@ public class TestSolrXmlPersistence exte
     }
   }
 
+  private void origMatchesPersist(CoreContainer cc, String originalSolrXML) throws Exception  {
+    String[] expressions = getAllNodes(originalSolrXML);
+    SolrXMLCoresLocator.NonPersistingLocator locator
+        = (SolrXMLCoresLocator.NonPersistingLocator) cc.getCoresLocator();
 
+    TestHarness.validateXPath(locator.xml, expressions);
+  }
 
   @Test
   public void testUnloadCreate() throws Exception {
@@ -275,7 +265,7 @@ public class TestSolrXmlPersistence exte
               resp);
       assertNull("Exception on unload", resp.getException());
 
-      persistContainedInOrig(cc, new File(solrHomeDirectory, "unloadcreate1.solr.xml"));
+      //origMatchesPersist(cc, new File(solrHomeDirectory, "unloadcreate1.solr.xml"));
 
       String instPath = new File(solrHomeDirectory, which).getAbsolutePath();
       admin.handleRequestBody
@@ -286,11 +276,7 @@ public class TestSolrXmlPersistence exte
               resp);
       assertNull("Exception on create", resp.getException());
 
-      File persistXml = new File(solrHomeDirectory, "rename.solr.xml");
-      File origXml = new File(solrHomeDirectory, "solr.xml");
-
-      cc.persistFile(persistXml);
-      String[] persistList = getAllNodes(persistXml);
+      String[] persistList = getAllNodes();
       String[] expressions = new String[persistList.length];
 
       // Now manually change the names back and it should match exactly to the original XML.
@@ -312,8 +298,8 @@ public class TestSolrXmlPersistence exte
         }
       }
 
-      assertXmlFile(origXml, expressions);
-
+      //assertXmlFile(origXml, expressions);
+      TestHarness.validateXPath(SOLR_XML_LOTS_SYSVARS, expressions);
 
     } finally {
       cc.shutdown();
@@ -323,134 +309,17 @@ public class TestSolrXmlPersistence exte
     }
   }
 
-  private void persistContainedInOrig(CoreContainer cc, File persistXml) throws IOException,
-      SAXException, ParserConfigurationException {
-    cc.persistFile(persistXml);
-    // Is everything that's in the original file persisted?
-    String[] expressions = getAllNodes(persistXml);
-    assertXmlFile(new File(solrHomeDirectory, "solr.xml"), expressions);
-  }
-
-  private void origContainedInPersist(CoreContainer cc, File persistXml) throws IOException,
-      SAXException, ParserConfigurationException {
-    cc.persistFile(persistXml);
-    // Is everything that's in the original file persisted?
-    String[] expressions = getAllNodes(new File(solrHomeDirectory, "solr.xml"));
-    assertXmlFile(persistXml, expressions);
-  }
-
-
-  @Test
-  public void testCreateAndManipulateCores() throws Exception {
-    CoreContainer cc = init(SOLR_XML_LOTS_SYSVARS, "SystemVars1", "SystemVars2", "new_one", "new_two");
-    try {
-      final CoreAdminHandler admin = new CoreAdminHandler(cc);
-      String instPathOne = new File(solrHomeDirectory, "new_one").getAbsolutePath();
-      SolrQueryResponse resp = new SolrQueryResponse();
-      admin.handleRequestBody
-          (req(CoreAdminParams.ACTION,
-              CoreAdminParams.CoreAdminAction.CREATE.toString(),
-              CoreAdminParams.INSTANCE_DIR, instPathOne,
-              CoreAdminParams.NAME, "new_one"),
-              resp);
-      assertNull("Exception on create", resp.getException());
-
-      admin.handleRequestBody
-          (req(CoreAdminParams.ACTION,
-              CoreAdminParams.CoreAdminAction.CREATE.toString(),
-              CoreAdminParams.NAME, "new_two"),
-              resp);
-      assertNull("Exception on create", resp.getException());
-
-      File persistXml1 = new File(solrHomeDirectory, "create_man_1.xml");
-      origContainedInPersist(cc, persistXml1);
-
-      // We know all the original data is in persist, now check for newly-created files.
-      String[] expressions = new  String[2];
-      String instHome = new File(solrHomeDirectory, "new_one").getAbsolutePath();
-      expressions[0] = "/solr/cores/core[@name='new_one' and @instanceDir='" + instHome + "']";
-      expressions[1] = "/solr/cores/core[@name='new_two' and @instanceDir='new_two" + File.separator + "']";
-
-      assertXmlFile(persistXml1, expressions);
-
-      // Next, swap a created core and check
-      resp = new SolrQueryResponse();
-      admin.handleRequestBody
-          (req(CoreAdminParams.ACTION,
-              CoreAdminParams.CoreAdminAction.SWAP.toString(),
-              CoreAdminParams.CORE, "new_one",
-              CoreAdminParams.OTHER, "SystemVars2"),
-              resp);
-      assertNull("Exception on swap", resp.getException());
-
-      File persistXml2 = new File(solrHomeDirectory, "create_man_2.xml");
-
-      cc.persistFile(persistXml2);
-      String[] persistList = getAllNodes(persistXml2);
-      expressions = new String[persistList.length];
-
-      // Now manually change the names back and it should match exactly to the original XML.
-      for (int idx = 0; idx < persistList.length; ++idx) {
-        String fromName = "@name='new_one'";
-        String toName = "@name='SystemVars2'";
-        if (persistList[idx].contains(fromName)) {
-          expressions[idx] = persistList[idx].replace(fromName, toName);
-        } else {
-          expressions[idx] = persistList[idx].replace(toName, fromName);
-        }
-      }
-
-      assertXmlFile(persistXml1, expressions);
-
-      // Then rename the other created core and check
-      admin.handleRequestBody
-          (req(CoreAdminParams.ACTION,
-              CoreAdminParams.CoreAdminAction.RENAME.toString(),
-              CoreAdminParams.CORE, "new_two",
-              CoreAdminParams.OTHER, "RenamedCore"),
-              resp);
-      assertNull("Exception on rename", resp.getException());
-
-      File persistXml3 = new File(solrHomeDirectory, "create_man_3.xml");
-
-      // OK, Assure that if I change everything that has been renamed with the original value for the core, it matches
-      // the old list
-      cc.persistFile(persistXml3);
-      persistList = getAllNodes(persistXml3);
-      expressions = new String[persistList.length];
-
-      for (int idx = 0; idx < persistList.length; ++idx) {
-        expressions[idx] = persistList[idx].replaceAll("RenamedCore", "new_two");
-      }
-      assertXmlFile(persistXml2, expressions);
-
-      // Now the other way, If I replace the original name in the original XML file with "RenamedCore", does it match
-      // what was persisted?
-      persistList = getAllNodes(persistXml2);
-      expressions = new String[persistList.length];
-      for (int idx = 0; idx < persistList.length; ++idx) {
-        // /solr/cores/core[@name='SystemVars1' and @collection='${collection:collection1}']
-        expressions[idx] = persistList[idx].replace("@name='new_two'", "@name='RenamedCore'");
-      }
-      assertXmlFile(persistXml3, expressions);
-
-    } finally {
-      cc.shutdown();
-      if (solrHomeDirectory.exists()) {
-        FileUtils.deleteDirectory(solrHomeDirectory);
-      }
-    }
-
-
-  }
   @Test
   public void testCreatePersistCore() throws Exception {
     // Template for creating a core.
     CoreContainer cc = init(SOLR_XML_LOTS_SYSVARS, "SystemVars1", "SystemVars2", "props1", "props2");
+    SolrXMLCoresLocator.NonPersistingLocator locator
+        = (SolrXMLCoresLocator.NonPersistingLocator) cc.getCoresLocator();
+
     try {
       final CoreAdminHandler admin = new CoreAdminHandler(cc);
       // create a new core (using CoreAdminHandler) w/ properties
-      String instPath1 = new File(solrHomeDirectory, "props1").getAbsolutePath();
+
       SolrQueryResponse resp = new SolrQueryResponse();
       admin.handleRequestBody
           (req(CoreAdminParams.ACTION,
@@ -480,14 +349,13 @@ public class TestSolrXmlPersistence exte
       assertNull("Exception on create", resp.getException());
 
       // Everything that was in the original XML file should be in the persisted one.
-      final File persistXml = new File(solrHomeDirectory, "persist_create_core.solr.xml");
-      cc.persistFile(persistXml);
-      assertXmlFile(persistXml, getAllNodes(new File(solrHomeDirectory, "solr.xml")));
+      TestHarness.validateXPath(locator.xml, getAllNodes(SOLR_XML_LOTS_SYSVARS));
 
       // And the params for the new core should be in the persisted file.
-      assertXmlFile
-          (persistXml
-              , "/solr/cores/core[@name='props1']/property[@name='prefix1' and @value='valuep1']"
+      TestHarness.validateXPath
+          (
+              locator.xml,
+              "/solr/cores/core[@name='props1']/property[@name='prefix1' and @value='valuep1']"
               , "/solr/cores/core[@name='props1']/property[@name='prefix2' and @value='valueP2']"
               , "/solr/cores/core[@name='props1' and @transient='true']"
               , "/solr/cores/core[@name='props1' and @loadOnStartup='true']"
@@ -511,18 +379,124 @@ public class TestSolrXmlPersistence exte
     }
   }
 
-  private String[] getAllNodes(File xmlFile) throws ParserConfigurationException, IOException, SAXException {
+  @Test
+  public void testPersist() throws Exception {
+
+    final CoreContainer cores = init(ConfigSolrXmlOld.DEF_SOLR_XML, "collection1");
+    SolrXMLCoresLocator.NonPersistingLocator locator
+        = (SolrXMLCoresLocator.NonPersistingLocator) cores.getCoresLocator();
+
+    String instDir = null;
+    {
+      SolrCore template = null;
+      try {
+        template = cores.getCore("collection1");
+        instDir = template.getCoreDescriptor().getRawInstanceDir();
+      } finally {
+        if (null != template) template.close();
+      }
+    }
+
+    final File instDirFile = new File(cores.getSolrHome(), instDir);
+    assertTrue("instDir doesn't exist: " + instDir, instDirFile.exists());
+
+    // sanity check the basic persistence of the default init
+    TestHarness.validateXPath(locator.xml,
+        "/solr[@persistent='true']",
+        "/solr/cores[@defaultCoreName='collection1' and not(@transientCacheSize)]",
+        "/solr/cores/core[@name='collection1' and @instanceDir='" + instDir +
+            "' and @transient='false' and @loadOnStartup='true' ]",
+        "1=count(/solr/cores/core)");
+
+    // create some new cores and sanity check the persistence
+
+    final File dataXfile = new File(solrHomeDirectory, "dataX");
+    final String dataX = dataXfile.getAbsolutePath();
+    assertTrue("dataXfile mkdirs failed: " + dataX, dataXfile.mkdirs());
+
+    final File instYfile = new File(solrHomeDirectory, "instY");
+    FileUtils.copyDirectory(instDirFile, instYfile);
+
+    // :HACK: dataDir leaves off trailing "/", but instanceDir uses it
+    final String instY = instYfile.getAbsolutePath() + "/";
+
+    final CoreDescriptor xd = buildCoreDescriptor(cores, "X", instDir)
+        .withDataDir(dataX).build();
+
+    final CoreDescriptor yd = new CoreDescriptor(cores, "Y", instY);
+
+    SolrCore x = null;
+    SolrCore y = null;
+    try {
+      x = cores.create(xd);
+      y = cores.create(yd);
+      cores.register(x, false);
+      cores.register(y, false);
+
+      assertEquals("cores not added?", 3, cores.getCoreNames().size());
+
+      TestHarness.validateXPath(locator.xml,
+          "/solr[@persistent='true']",
+          "/solr/cores[@defaultCoreName='collection1']",
+          "/solr/cores/core[@name='collection1' and @instanceDir='" + instDir
+              + "']", "/solr/cores/core[@name='X' and @instanceDir='" + instDir
+              + "' and @dataDir='" + dataX + "']",
+          "/solr/cores/core[@name='Y' and @instanceDir='" + instY + "']",
+          "3=count(/solr/cores/core)");
+
+      // Test for saving implicit properties, we should not do this.
+      TestHarness.validateXPath(locator.xml,
+          "/solr/cores/core[@name='X' and not(@solr.core.instanceDir) and not (@solr.core.configName)]");
+
+      // delete a core, check persistence again
+      assertNotNull("removing X returned null", cores.remove("X"));
+
+      TestHarness.validateXPath(locator.xml, "/solr[@persistent='true']",
+          "/solr/cores[@defaultCoreName='collection1']",
+          "/solr/cores/core[@name='collection1' and @instanceDir='" + instDir + "']",
+          "/solr/cores/core[@name='Y' and @instanceDir='" + instY + "']",
+          "2=count(/solr/cores/core)");
+
+    } finally {
+      // y is closed by the container, but
+      // x has been removed from the container
+      if (x != null) {
+        try {
+          x.close();
+        } catch (Exception e) {
+          log.error("", e);
+        }
+      }
+      cores.shutdown();
+    }
+  }
+
+
+  private String[] getAllNodes(InputStream is) throws ParserConfigurationException, IOException, SAXException {
     List<String> expressions = new ArrayList<String>(); // XPATH and value for all elements in the indicated XML
     DocumentBuilderFactory docBuilderFactory = DocumentBuilderFactory
         .newInstance();
     DocumentBuilder docBuilder = docBuilderFactory.newDocumentBuilder();
-    Document document = docBuilder.parse(xmlFile);
+    Document document = docBuilder.parse(is);
 
     Node root = document.getDocumentElement();
     gatherNodes(root, expressions, "");
     return expressions.toArray(new String[expressions.size()]);
   }
 
+  private String[] getAllNodes() throws ParserConfigurationException, IOException, SAXException {
+    return getAllNodes(new FileInputStream(new File(solrHomeDirectory, "solr.xml")));
+  }
+
+  private String[] getAllNodes(String xmlString) throws ParserConfigurationException, IOException, SAXException {
+    return getAllNodes(new ByteArrayInputStream(xmlString.getBytes(Charsets.UTF_8)));
+  }
+
+  /*
+  private void assertSolrXmlFile(String... xpathExpressions) throws IOException, SAXException {
+    assertXmlFile(new File(solrHomeDirectory, "solr.xml"), xpathExpressions);
+  }
+  */
 
   // Note this is pretty specialized for a solr.xml file because working with the DOM is such a pain.
 
@@ -595,7 +569,7 @@ public class TestSolrXmlPersistence exte
     }
   }
 
-  private static String SOLR_XML_LOTS_SYSVARS =
+  public static String SOLR_XML_LOTS_SYSVARS =
       "<solr persistent=\"${solr.xml.persist:false}\" coreLoadThreads=\"12\" sharedLib=\"${something:.}\" >\n" +
           "  <logging class=\"${logclass:log4j.class}\" enabled=\"{logenable:true}\">\n" +
           "     <watcher size=\"{watchSize:13}\" threshold=\"${logThresh:54}\" />\n" +

Added: lucene/dev/trunk/solr/core/src/test/org/apache/solr/core/TestSolrXmlPersistor.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/core/TestSolrXmlPersistor.java?rev=1502276&view=auto
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/core/TestSolrXmlPersistor.java (added)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/core/TestSolrXmlPersistor.java Thu Jul 11 16:35:01 2013
@@ -0,0 +1,122 @@
+package org.apache.solr.core;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import com.google.common.collect.ImmutableList;
+import org.junit.Test;
+
+import java.io.File;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+public class TestSolrXmlPersistor {
+
+  private static final List<CoreDescriptor> EMPTY_CD_LIST = ImmutableList.<CoreDescriptor>builder().build();
+
+  @Test
+  public void selfClosingCoresTagIsPersisted() {
+
+    final String solrxml = "<solr><cores adminHandler=\"/admin\"/></solr>";
+
+    SolrXMLCoresLocator persistor = new SolrXMLCoresLocator(new File("testfile.xml"), solrxml, null);
+    assertEquals(persistor.buildSolrXML(EMPTY_CD_LIST),
+        "<solr><cores adminHandler=\"/admin\"></cores></solr>");
+
+  }
+
+  @Test
+  public void emptyCoresTagIsPersisted() {
+    final String solrxml = "<solr><cores adminHandler=\"/admin\"></cores></solr>";
+
+    SolrXMLCoresLocator persistor = new SolrXMLCoresLocator(new File("testfile.xml"), solrxml, null);
+    assertEquals(persistor.buildSolrXML(EMPTY_CD_LIST), "<solr><cores adminHandler=\"/admin\"></cores></solr>");
+  }
+
+  @Test
+  public void emptySolrXmlIsPersisted() {
+    final String solrxml = "<solr></solr>";
+
+    SolrXMLCoresLocator persistor = new SolrXMLCoresLocator(new File("testfile.xml"), solrxml, null);
+    assertEquals(persistor.buildSolrXML(EMPTY_CD_LIST), "<solr><cores></cores></solr>");
+  }
+
+  @Test
+  public void simpleCoreDescriptorIsPersisted() {
+
+    final String solrxml = "<solr><cores></cores></solr>";
+
+    SolrResourceLoader loader = new SolrResourceLoader("solr/example/solr");
+    CoreContainer cc = new CoreContainer(loader);
+
+    final CoreDescriptor cd = new CoreDescriptor(cc, "testcore", "instance/dir/");
+    List<CoreDescriptor> cds = ImmutableList.of(cd);
+
+    SolrXMLCoresLocator persistor = new SolrXMLCoresLocator(new File("testfile.xml"), solrxml, null);
+    assertEquals(persistor.buildSolrXML(cds),
+          "<solr><cores>\n"
+        + "    <core name=\"testcore\" instanceDir=\"instance/dir/\"/>\n"
+        + "</cores></solr>");
+  }
+
+  @Test
+  public void shardHandlerInfoIsPersisted() {
+
+    final String solrxml =
+        "<solr>" +
+          "<cores adminHandler=\"whatever\">" +
+            "<core name=\"testcore\" instanceDir=\"instance/dir/\"/>" +
+            "<shardHandlerFactory name=\"shardHandlerFactory\" class=\"HttpShardHandlerFactory\">" +
+              "<int name=\"socketTimeout\">${socketTimeout:500}</int>" +
+              "<str name=\"arbitrary\">arbitraryValue</str>" +
+            "</shardHandlerFactory>" +
+          "</cores>" +
+        "</solr>";
+
+    SolrXMLCoresLocator locator = new SolrXMLCoresLocator(new File("testfile.xml"), solrxml, null);
+    assertTrue(locator.getTemplate().contains("{{CORES_PLACEHOLDER}}"));
+    assertTrue(locator.getTemplate().contains("<shardHandlerFactory "));
+    assertTrue(locator.getTemplate().contains("${socketTimeout:500}"));
+
+  }
+
+  @Test
+  public void simpleShardHandlerInfoIsPersisted() {
+
+    final String solrxml =
+        "<solr>" +
+          "<cores adminHandler=\"whatever\">" +
+            "<core name=\"testcore\" instanceDir=\"instance/dir/\"/>" +
+            "<shardHandlerFactory name=\"shardHandlerFactory\" class=\"HttpShardHandlerFactory\"/>" +
+          "</cores>" +
+        "</solr>";
+
+    SolrXMLCoresLocator locator = new SolrXMLCoresLocator(new File("testfile.xml"), solrxml, null);
+    assertTrue(locator.getTemplate().contains("{{CORES_PLACEHOLDER}}"));
+    assertTrue(locator.getTemplate().contains("<shardHandlerFactory "));
+  }
+
+  @Test
+  public void complexXmlIsParsed() {
+    SolrXMLCoresLocator locator = new SolrXMLCoresLocator(new File("testfile.xml"),
+                                        TestSolrXmlPersistence.SOLR_XML_LOTS_SYSVARS, null);
+    assertTrue(locator.getTemplate().contains("{{CORES_PLACEHOLDER}}"));
+  }
+
+}

Modified: lucene/dev/trunk/solr/core/src/test/org/apache/solr/handler/admin/CoreAdminCreateDiscoverTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/handler/admin/CoreAdminCreateDiscoverTest.java?rev=1502276&r1=1502275&r2=1502276&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/handler/admin/CoreAdminCreateDiscoverTest.java (original)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/handler/admin/CoreAdminCreateDiscoverTest.java Thu Jul 11 16:35:01 2013
@@ -19,9 +19,9 @@ package org.apache.solr.handler.admin;
 
 import org.apache.commons.io.FileUtils;
 import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.common.SolrException;
 import org.apache.solr.common.params.CoreAdminParams;
-import org.apache.solr.core.CoreDescriptor;
-import org.apache.solr.core.SolrCoreDiscoverer;
+import org.apache.solr.core.CorePropertiesLocator;
 import org.apache.solr.response.SolrQueryResponse;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
@@ -30,9 +30,6 @@ import org.junit.Test;
 import java.io.File;
 import java.io.FileInputStream;
 import java.io.IOException;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.HashSet;
 import java.util.Properties;
 
 public class CoreAdminCreateDiscoverTest extends SolrTestCaseJ4 {
@@ -43,6 +40,7 @@ public class CoreAdminCreateDiscoverTest
 
   private static String coreNormal = "normal";
   private static String coreSysProps = "sys_props";
+  private static String coreDuplicate = "duplicate";
 
   @BeforeClass
   public static void beforeClass() throws Exception {
@@ -111,7 +109,7 @@ public class CoreAdminCreateDiscoverTest
     // verify props are in persisted file
 
     Properties props = new Properties();
-    File propFile = new File(solrHomeDirectory, coreSysProps + "/" + SolrCoreDiscoverer.CORE_PROP_FILE);
+    File propFile = new File(solrHomeDirectory, coreSysProps + "/" + CorePropertiesLocator.PROPERTIES_FILENAME);
     FileInputStream is = new FileInputStream(propFile);
     try {
       props.load(is);
@@ -131,7 +129,8 @@ public class CoreAdminCreateDiscoverTest
     assertEquals("Unexpected value preserved in properties file " + propFile.getAbsolutePath(),
         props.getProperty(CoreAdminParams.DATA_DIR), "${DATA_TEST}");
 
-    checkOnlyKnown(propFile);
+    assertEquals(props.size(), 4);
+    //checkOnlyKnown(propFile);
 
     // Now assert that certain values are properly dereferenced in the process of creating the core, see
     // SOLR-4982. Really, we should be able to just verify that the index files exist.
@@ -151,6 +150,47 @@ public class CoreAdminCreateDiscoverTest
   }
 
   @Test
+  public void testCannotCreateTwoCoresWithSameInstanceDir() throws Exception {
+
+    setupCore(coreDuplicate, true);
+
+    File workDir = new File(solrHomeDirectory, coreDuplicate);
+    File data = new File(workDir, "data");
+
+    // Create one core
+    SolrQueryResponse resp = new SolrQueryResponse();
+    admin.handleRequestBody
+        (req(CoreAdminParams.ACTION,
+            CoreAdminParams.CoreAdminAction.CREATE.toString(),
+            CoreAdminParams.NAME, coreDuplicate,
+            CoreAdminParams.INSTANCE_DIR, workDir.getAbsolutePath(),
+            CoreAdminParams.CONFIG, "solrconfig_ren.xml",
+            CoreAdminParams.SCHEMA, "schema_ren.xml",
+            CoreAdminParams.DATA_DIR, data.getAbsolutePath()),
+            resp);
+    assertNull("Exception on create", resp.getException());
+
+    // Try to create another core with a different name, but the same instance dir
+    SolrQueryResponse resp2 = new SolrQueryResponse();
+    try {
+      admin.handleRequestBody
+          (req(CoreAdminParams.ACTION,
+              CoreAdminParams.CoreAdminAction.CREATE.toString(),
+              CoreAdminParams.NAME, "different_name_core",
+              CoreAdminParams.INSTANCE_DIR, workDir.getAbsolutePath(),
+              CoreAdminParams.CONFIG, "solrconfig_ren.xml",
+              CoreAdminParams.SCHEMA, "schema_ren.xml",
+              CoreAdminParams.DATA_DIR, data.getAbsolutePath()),
+              resp2);
+      fail("Creating two cores with a shared instance dir should throw an exception");
+    }
+    catch (SolrException e) {
+      assertTrue(e.getMessage().contains("already defined there"));
+    }
+
+  }
+
+  @Test
   public void testCreateSavesRegProps() throws Exception {
 
     setupCore(coreNormal, true);
@@ -174,7 +214,7 @@ public class CoreAdminCreateDiscoverTest
 
     // verify props are in persisted file
     Properties props = new Properties();
-    File propFile = new File(solrHomeDirectory, coreNormal + "/" + SolrCoreDiscoverer.CORE_PROP_FILE);
+    File propFile = new File(solrHomeDirectory, coreNormal + "/" + CorePropertiesLocator.PROPERTIES_FILENAME);
     FileInputStream is = new FileInputStream(propFile);
     try {
       props.load(is);
@@ -194,7 +234,9 @@ public class CoreAdminCreateDiscoverTest
     assertEquals("Unexpected value preserved in properties file " + propFile.getAbsolutePath(),
         props.getProperty(CoreAdminParams.DATA_DIR), data.getAbsolutePath());
 
-    checkOnlyKnown(propFile);
+    assertEquals(props.size(), 4);
+
+    //checkOnlyKnown(propFile);
     // For the other 3 vars, we couldn't get past creating the core if dereferencing didn't work correctly.
 
     // Should have segments in the directory pointed to by the ${DATA_TEST}.
@@ -205,23 +247,4 @@ public class CoreAdminCreateDiscoverTest
 
   }
 
-  // Insure that all the props we've preserved are ones that _should_ be in the properties file
-  private void checkOnlyKnown(File propFile) throws IOException {
-
-    Properties props = new Properties();
-    FileInputStream is = new FileInputStream(propFile);
-    try {
-      props.load(is);
-    } finally {
-      org.apache.commons.io.IOUtils.closeQuietly(is);
-    }
-
-    // Should never be preserving instanceDir in a core.properties file.
-    assertFalse("Should not be preserving instanceDir!", props.containsKey(CoreAdminParams.INSTANCE_DIR));
-
-    Collection<String> stds = new HashSet(Arrays.asList(CoreDescriptor.standardPropNames));
-    for (String key : props.stringPropertyNames()) {
-      assertTrue("Property '" + key + "' should NOT be preserved in the properties file", stds.contains(key));
-    }
-  }
 }

Modified: lucene/dev/trunk/solr/core/src/test/org/apache/solr/handler/admin/CoreAdminHandlerTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/handler/admin/CoreAdminHandlerTest.java?rev=1502276&r1=1502275&r2=1502276&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/handler/admin/CoreAdminHandlerTest.java (original)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/handler/admin/CoreAdminHandlerTest.java Thu Jul 11 16:35:01 2013
@@ -18,25 +18,25 @@
 package org.apache.solr.handler.admin;
 
 import com.carrotsearch.randomizedtesting.rules.SystemPropertiesRestoreRule;
-import org.apache.solr.core.CoreContainer;
-import org.apache.solr.core.SolrCore;
+import org.apache.commons.io.FileUtils;
+import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.params.CoreAdminParams;
 import org.apache.solr.common.util.NamedList;
+import org.apache.solr.core.CoreContainer;
+import org.apache.solr.core.CoreDescriptor;
+import org.apache.solr.core.SolrCore;
+import org.apache.solr.core.SolrXMLCoresLocator;
 import org.apache.solr.response.SolrQueryResponse;
-import org.apache.solr.SolrTestCaseJ4;
-
-import java.util.Map;
-import java.io.File;
-
-import org.apache.commons.io.FileUtils;
-
 import org.junit.BeforeClass;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.RuleChain;
 import org.junit.rules.TestRule;
 
+import java.io.File;
+import java.util.Map;
+
 public class CoreAdminHandlerTest extends SolrTestCaseJ4 {
   
   @BeforeClass
@@ -72,7 +72,8 @@ public class CoreAdminHandlerTest extend
         new File(subHome, "solrconfig.snippet.randomindexconfig.xml"));
 
     final CoreContainer cores = h.getCoreContainer();
-    cores.setPersistent(false); // we'll do this explicitly as needed
+    SolrXMLCoresLocator.NonPersistingLocator locator
+        = (SolrXMLCoresLocator.NonPersistingLocator) cores.getCoresLocator();
 
     final CoreAdminHandler admin = new CoreAdminHandler(cores);
 
@@ -96,14 +97,9 @@ public class CoreAdminHandlerTest extend
             resp);
     assertNull("Exception on create", resp.getException());
 
-    // verify props are in persisted file
-
-    final File xml = new File(workDir, "persist-solr.xml");
-    cores.persistFile(xml);
-
     // First assert that these values are persisted.
-    assertXmlFile
-        (xml
+    h.validateXPath
+        (locator.xml
             ,"/solr/cores/core[@name='" + getCoreName() + "' and @instanceDir='${INSTDIR_TEST}']"
             ,"/solr/cores/core[@name='" + getCoreName() + "' and @dataDir='${DATA_TEST}']"
             ,"/solr/cores/core[@name='" + getCoreName() + "' and @schema='${SCHEMA_TEST}']"
@@ -140,7 +136,6 @@ public class CoreAdminHandlerTest extend
     assertTrue("Failed to mkdirs workDir", workDir.mkdirs());
     
     final CoreContainer cores = h.getCoreContainer();
-    cores.setPersistent(false); // we'll do this explicitly as needed
 
     final CoreAdminHandler admin = new CoreAdminHandler(cores);
 
@@ -173,16 +168,10 @@ public class CoreAdminHandlerTest extend
        resp);
     assertNull("Exception on create", resp.getException());
 
-    // verify props are in persisted file
-
-    final File xml = new File(workDir, "persist-solr.xml");
-    cores.persistFile(xml);
-    
-    assertXmlFile
-      (xml
-       ,"/solr/cores/core[@name='props']/property[@name='hoss' and @value='man']"
-       ,"/solr/cores/core[@name='props']/property[@name='foo' and @value='baz']"
-       );
+    CoreDescriptor cd = cores.getCoreDescriptor("props");
+    assertNotNull("Core not added!", cd);
+    assertEquals(cd.getCoreProperty("hoss", null), "man");
+    assertEquals(cd.getCoreProperty("foo", null), "baz");
 
     // attempt to create a bogus core and confirm failure
     ignoreException("Could not load config");

Modified: lucene/dev/trunk/solr/core/src/test/org/apache/solr/update/SolrIndexSplitterTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/update/SolrIndexSplitterTest.java?rev=1502276&r1=1502275&r2=1502276&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/update/SolrIndexSplitterTest.java (original)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/update/SolrIndexSplitterTest.java Thu Jul 11 16:35:01 2013
@@ -133,21 +133,18 @@ public class SolrIndexSplitterTest exten
 
     SolrCore core1 = null, core2 = null;
     try {
-      CoreDescriptor dcore1 = new CoreDescriptor(h.getCoreContainer(), "split1", h.getCore().getCoreDescriptor().getInstanceDir());
-      dcore1.setDataDir(indexDir1.getAbsolutePath());
-      dcore1.setSchemaName("schema12.xml");
-      
+      String instanceDir = h.getCore().getCoreDescriptor().getInstanceDir();
+
+      CoreDescriptor dcore1 = buildCoreDescriptor(h.getCoreContainer(), "split1", instanceDir)
+          .withDataDir(indexDir1.getAbsolutePath()).withSchema("schema12.xml").build();
       if (h.getCoreContainer().getZkController() != null) {
         h.getCoreContainer().preRegisterInZk(dcore1);
       }
-      
       core1 = h.getCoreContainer().create(dcore1);
       h.getCoreContainer().register(core1, false);
 
-      CoreDescriptor dcore2 = new CoreDescriptor(h.getCoreContainer(), "split2", h.getCore().getCoreDescriptor().getInstanceDir());
-      dcore2.setDataDir(indexDir2.getAbsolutePath());
-      dcore2.setSchemaName("schema12.xml");
-      
+      CoreDescriptor dcore2 = buildCoreDescriptor(h.getCoreContainer(), "split2", instanceDir)
+          .withDataDir(indexDir2.getAbsolutePath()).withSchema("schema12.xml").build();
       if (h.getCoreContainer().getZkController() != null) {
         h.getCoreContainer().preRegisterInZk(dcore2);
       }

Modified: lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/client/solrj/request/CoreAdminRequest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/client/solrj/request/CoreAdminRequest.java?rev=1502276&r1=1502275&r2=1502276&view=diff
==============================================================================
--- lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/client/solrj/request/CoreAdminRequest.java (original)
+++ lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/client/solrj/request/CoreAdminRequest.java Thu Jul 11 16:35:01 2013
@@ -17,22 +17,22 @@
 
 package org.apache.solr.client.solrj.request;
 
-import java.io.IOException;
-import java.util.Collection;
-import java.util.List;
-import java.util.Arrays;
-
 import org.apache.solr.client.solrj.SolrRequest;
 import org.apache.solr.client.solrj.SolrServer;
 import org.apache.solr.client.solrj.SolrServerException;
 import org.apache.solr.client.solrj.response.CoreAdminResponse;
 import org.apache.solr.common.cloud.ZkStateReader;
-import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.params.CoreAdminParams;
-import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.params.CoreAdminParams.CoreAdminAction;
+import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.ContentStream;
 
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+
 /**
  * This class is experimental and subject to change.
  *
@@ -534,6 +534,7 @@ public class CoreAdminRequest extends So
     return req.process( server );
   }
 
+  @Deprecated
   public static CoreAdminResponse persist(String fileName, SolrServer server) throws SolrServerException, IOException 
   {
     CoreAdminRequest.Persist req = new CoreAdminRequest.Persist();

Modified: lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/common/params/CoreAdminParams.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/common/params/CoreAdminParams.java?rev=1502276&r1=1502275&r2=1502276&view=diff
==============================================================================
--- lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/common/params/CoreAdminParams.java (original)
+++ lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/common/params/CoreAdminParams.java Thu Jul 11 16:35:01 2013
@@ -23,7 +23,7 @@ import java.util.Locale;
 /**
  * @since solr 1.3
  */
-public interface CoreAdminParams 
+public abstract class CoreAdminParams
 {
   /** What Core are we talking about **/
   public final static String CORE = "core";

Modified: lucene/dev/trunk/solr/solrj/src/test-files/solrj/solr/shared/solr.xml
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/solrj/src/test-files/solrj/solr/shared/solr.xml?rev=1502276&r1=1502275&r2=1502276&view=diff
==============================================================================
--- lucene/dev/trunk/solr/solrj/src/test-files/solrj/solr/shared/solr.xml (original)
+++ lucene/dev/trunk/solr/solrj/src/test-files/solrj/solr/shared/solr.xml Thu Jul 11 16:35:01 2013
@@ -22,7 +22,7 @@
   persistent: Save changes made via the API to this file
   sharedLib: path to a lib directory that will be shared across all cores
 -->
-<solr persistent="true">
+<solr persistent="false">
   <property name="version" value="1.3"/>
   <property name="lang" value="english, french"/>
 

Modified: lucene/dev/trunk/solr/solrj/src/test/org/apache/solr/client/solrj/embedded/AbstractEmbeddedSolrServerTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/solrj/src/test/org/apache/solr/client/solrj/embedded/AbstractEmbeddedSolrServerTestCase.java?rev=1502276&r1=1502275&r2=1502276&view=diff
==============================================================================
--- lucene/dev/trunk/solr/solrj/src/test/org/apache/solr/client/solrj/embedded/AbstractEmbeddedSolrServerTestCase.java (original)
+++ lucene/dev/trunk/solr/solrj/src/test/org/apache/solr/client/solrj/embedded/AbstractEmbeddedSolrServerTestCase.java Thu Jul 11 16:35:01 2013
@@ -63,7 +63,7 @@ public abstract class AbstractEmbeddedSo
     System.setProperty("tempDir", tempDir.getAbsolutePath());
     System.setProperty("tests.shardhandler.randomSeed", Long.toString(random().nextLong()));
     cores = CoreContainer.createAndLoad(SOLR_HOME.getAbsolutePath(), getSolrXml());
-    cores.setPersistent(false);
+    //cores.setPersistent(false);
   }
   
   protected abstract File getSolrXml() throws Exception;

Modified: lucene/dev/trunk/solr/solrj/src/test/org/apache/solr/client/solrj/embedded/TestSolrProperties.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/solrj/src/test/org/apache/solr/client/solrj/embedded/TestSolrProperties.java?rev=1502276&r1=1502275&r2=1502276&view=diff
==============================================================================
--- lucene/dev/trunk/solr/solrj/src/test/org/apache/solr/client/solrj/embedded/TestSolrProperties.java (original)
+++ lucene/dev/trunk/solr/solrj/src/test/org/apache/solr/client/solrj/embedded/TestSolrProperties.java Thu Jul 11 16:35:01 2013
@@ -18,8 +18,6 @@
 package org.apache.solr.client.solrj.embedded;
 
 import com.carrotsearch.randomizedtesting.rules.SystemPropertiesRestoreRule;
-import com.google.common.io.ByteStreams;
-import org.apache.commons.io.IOUtils;
 import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.client.solrj.SolrQuery;
 import org.apache.solr.client.solrj.SolrServer;
@@ -29,30 +27,16 @@ import org.apache.solr.client.solrj.requ
 import org.apache.solr.client.solrj.request.UpdateRequest;
 import org.apache.solr.client.solrj.response.CoreAdminResponse;
 import org.apache.solr.common.SolrInputDocument;
-import org.apache.solr.core.CoreContainer;
-import org.apache.solr.util.FileUtils;
-import org.junit.After;
-import org.junit.Before;
+import org.apache.solr.core.SolrXMLCoresLocator;
+import org.apache.solr.util.TestHarness;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.RuleChain;
 import org.junit.rules.TestRule;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import org.w3c.dom.Document;
-import org.w3c.dom.Node;
 
-import javax.xml.parsers.DocumentBuilder;
-import javax.xml.parsers.DocumentBuilderFactory;
-import javax.xml.xpath.XPath;
-import javax.xml.xpath.XPathConstants;
-import javax.xml.xpath.XPathExpressionException;
-import javax.xml.xpath.XPathFactory;
 import java.io.File;
-import java.io.FileInputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.InputStreamReader;
 
 /**
  *
@@ -62,56 +46,14 @@ public class TestSolrProperties extends 
   protected static Logger log = LoggerFactory.getLogger(TestSolrProperties.class);
 
   private static final String SOLR_XML = "solr.xml";
-  private static final String SOLR_PERSIST_XML = "solr-persist.xml";
 
   @Rule
   public TestRule solrTestRules = 
     RuleChain.outerRule(new SystemPropertiesRestoreRule());
 
-  private static final XPathFactory xpathFactory = XPathFactory.newInstance();
-
-  @Override
-  @Before
-  public void setUp() throws Exception {
-    super.setUp();
-  }
-  
-  @Override
-  @After
-  public void tearDown() throws Exception {
-   super.tearDown(); 
-  }
-
-  private static void dumpFile(File fileToDump) throws IOException {
-    System.out.println("Dumping " + fileToDump.getAbsolutePath());
-    InputStream is = new FileInputStream(fileToDump);
-    try {
-      ByteStreams.copy(is, System.out);
-    }
-    finally {
-      IOUtils.closeQuietly(is);
-    }
-  }
-  
   @Override
   protected File getSolrXml() throws Exception {
-    //This test writes on the directory where the solr.xml is located. Better to copy the solr.xml to
-    //the temporary directory where we store the index
-    File origSolrXml = new File(SOLR_HOME, SOLR_XML);
-    File solrXml = new File(tempDir, SOLR_XML);
-    FileUtils.copyFile(origSolrXml, solrXml);
-    return solrXml;
-  }
-
-  @Override
-  protected void deleteAdditionalFiles() {
-    super.deleteAdditionalFiles();
-
-    //Cleans the solr.xml persisted while testing and the solr.xml copied to the temporary directory
-    File persistedFile = new File(tempDir, SOLR_PERSIST_XML);
-    assertTrue("Failed to delete "+persistedFile, persistedFile.delete());
-    File solrXml = new File(tempDir, SOLR_XML);
-    assertTrue("Failed to delete "+ solrXml, solrXml.delete());
+    return new File(SOLR_HOME, SOLR_XML);
   }
 
   protected SolrServer getSolrAdmin() {
@@ -125,8 +67,8 @@ public class TestSolrProperties extends 
   @Test
   public void testProperties() throws Exception {
 
-    String persistedSolrXml = new File(tempDir, SOLR_PERSIST_XML).getAbsolutePath();
-    log.info("persistedSolrXml: {}", persistedSolrXml);
+    SolrXMLCoresLocator.NonPersistingLocator locator
+        = (SolrXMLCoresLocator.NonPersistingLocator) cores.getCoresLocator();
 
     UpdateRequest up = new UpdateRequest();
     up.setAction(ACTION.COMMIT, true, true);
@@ -197,51 +139,22 @@ public class TestSolrProperties extends 
     long after = mcr.getStartTime(name).getTime();
     assertTrue("should have more recent time: " + after + "," + before, after > before);
 
-    mcr = CoreAdminRequest.persist(persistedSolrXml, coreadmin);
-
-    DocumentBuilder builder = DocumentBuilderFactory.newInstance().newDocumentBuilder();
-    FileInputStream fis = new FileInputStream(new File(persistedSolrXml));
-    try {
-      Document document = builder.parse(fis);
-      fis.close();
-      fis = new FileInputStream(new File(persistedSolrXml));
-      String solrPersistXml = IOUtils.toString(new InputStreamReader(fis, "UTF-8"));
-      //System.out.println("xml:" + solrPersistXml);
-      assertTrue("\"/solr/cores[@defaultCoreName='core0']\" doesn't match in:\n" + solrPersistXml,
-                 exists("/solr/cores[@defaultCoreName='core0']", document));
-      assertTrue("\"/solr/cores[@host='127.0.0.1']\" doesn't match in:\n" + solrPersistXml,
-                 exists("/solr/cores[@host='127.0.0.1']", document));
-      assertTrue("\"/solr/cores[@hostPort='${hostPort:8983}']\" doesn't match in:\n" + solrPersistXml,
-                 exists("/solr/cores[@hostPort='${hostPort:8983}']", document));
-      assertTrue("\"/solr/cores[@zkClientTimeout='8000']\" doesn't match in:\n" + solrPersistXml,
-                 exists("/solr/cores[@zkClientTimeout='8000']", document));
-      assertTrue("\"/solr/cores[@hostContext='${hostContext:solr}']\" doesn't match in:\n" + solrPersistXml,
-                 exists("/solr/cores[@hostContext='${hostContext:solr}']", document));
-      assertTrue("\"/solr/cores[@genericCoreNodeNames='${genericCoreNodeNames:true}']\" doesn't match in:\n" + solrPersistXml,
-          exists("/solr/cores[@genericCoreNodeNames='${genericCoreNodeNames:true}']", document));
-    } finally {
-      fis.close();
-    }
+    TestHarness.validateXPath(locator.xml,
+        "/solr/cores[@defaultCoreName='core0']",
+        "/solr/cores[@host='127.0.0.1']",
+        "/solr/cores[@hostPort='${hostPort:8983}']",
+        "/solr/cores[@zkClientTimeout='8000']",
+        "/solr/cores[@hostContext='${hostContext:solr}']",
+        "/solr/cores[@genericCoreNodeNames='${genericCoreNodeNames:true}']"
+        );
     
     CoreAdminRequest.renameCore(name, "renamed_core", coreadmin);
-    
-    mcr = CoreAdminRequest.persist(persistedSolrXml, getRenamedSolrAdmin());
-    
-//    fis = new FileInputStream(new File(tempDir, SOLR_PERSIST_XML));
-//    String solrPersistXml = IOUtils.toString(fis);
-//    System.out.println("xml:" + solrPersistXml);
-//    fis.close();
-    
-    fis = new FileInputStream(new File(persistedSolrXml));
-    try {
-      Document document = builder.parse(fis);
-      assertTrue(exists("/solr/cores/core[@name='renamed_core']", document));
-      assertTrue(exists("/solr/cores/core[@instanceDir='${theInstanceDir:./}']", document));
-      assertTrue(exists("/solr/cores/core[@collection='${collection:acollection}']", document));
-      
-    } finally {
-      fis.close();
-    }
+
+    TestHarness.validateXPath(locator.xml,
+        "/solr/cores/core[@name='renamed_core']",
+        "/solr/cores/core[@instanceDir='${theInstanceDir:./}']",
+        "/solr/cores/core[@collection='${collection:acollection}']"
+        );
     
     coreadmin = getRenamedSolrAdmin();
     File dataDir = new File(tempDir,"data3");
@@ -251,49 +164,8 @@ public class TestSolrProperties extends 
         coreadmin, null, null, dataDir.getAbsolutePath(),
         tlogDir.getAbsolutePath());
 
-//    fis = new FileInputStream(new File(solrXml.getParent(), SOLR_PERSIST_XML));
-//    solrPersistXml = IOUtils.toString(fis);
-//    System.out.println("xml:" + solrPersistXml);
-//    fis.close();
-    
-    mcr = CoreAdminRequest.persist(persistedSolrXml, getRenamedSolrAdmin());
-    
-//    fis = new FileInputStream(new File(solrXml.getParent(), SOLR_PERSIST_XML));
-//    solrPersistXml = IOUtils.toString(fis);
-//    System.out.println("xml:" + solrPersistXml);
-//    fis.close();
-    
-    fis = new FileInputStream(new File(persistedSolrXml));
-    try {
-      Document document = builder.parse(fis);
-      assertTrue(exists("/solr/cores/core[@name='collection1' and @instanceDir='.']", document));
-    } finally {
-      fis.close();
-    }
-    
-    // test reload and parse
-    cores.shutdown();
-    
-//   fis = new FileInputStream(new File(getSolrXml().getParent(),
-//   SOLR_PERSIST_XML));
-//   String solrPersistXml = IOUtils.toString(fis);
-//   System.out.println("xml:" + solrPersistXml);
-//   fis.close();
-    
-    cores = CoreContainer.createAndLoad(SOLR_HOME.getAbsolutePath(), new File(persistedSolrXml));
- 
-    //mcr = CoreAdminRequest.persist(SOLR_PERSIST_XML, getRenamedSolrAdmin());
-    
-//     fis = new FileInputStream(new File(solrXml.getParent(),
-//     SOLR_PERSIST_XML));
-//     solrPersistXml = IOUtils.toString(fis);
-//     System.out.println("xml:" + solrPersistXml);
-//     fis.close();
-  }
-  
-  public static boolean exists(String xpathStr, Node node)
-      throws XPathExpressionException {
-    XPath xpath = xpathFactory.newXPath();
-    return (Boolean) xpath.evaluate(xpathStr, node, XPathConstants.BOOLEAN);
+    TestHarness.validateXPath(locator.xml, "/solr/cores/core[@name='collection1' and @instanceDir='.']");
+
   }
+
 }

Modified: lucene/dev/trunk/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java?rev=1502276&r1=1502275&r2=1502276&view=diff
==============================================================================
--- lucene/dev/trunk/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java (original)
+++ lucene/dev/trunk/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java Thu Jul 11 16:35:01 2013
@@ -35,6 +35,7 @@ import org.apache.solr.common.params.Sol
 import org.apache.solr.common.util.XML;
 import org.apache.solr.core.ConfigSolr;
 import org.apache.solr.core.CoreContainer;
+import org.apache.solr.core.CoreDescriptor;
 import org.apache.solr.core.SolrConfig;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.core.SolrResourceLoader;
@@ -73,6 +74,7 @@ import java.util.HashSet;
 import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Properties;
 import java.util.logging.ConsoleHandler;
 import java.util.logging.Handler;
 import java.util.logging.Level;
@@ -1576,4 +1578,51 @@ public abstract class SolrTestCaseJ4 ext
     FileUtils.copyFile(new File(top, "solrconfig.snippet.randomindexconfig.xml"), new File(subHome, "solrconfig.snippet.randomindexconfig.xml"));
   }
 
+  public static CoreDescriptorBuilder buildCoreDescriptor(CoreContainer container, String name, String instancedir) {
+    return new CoreDescriptorBuilder(container, name, instancedir);
+  }
+
+  public static class CoreDescriptorBuilder {
+
+    final String name;
+    final String instanceDir;
+    final CoreContainer container;
+    final Properties properties = new Properties();
+
+    public CoreDescriptorBuilder(CoreContainer container, String name, String instancedir) {
+      this.name = name;
+      this.instanceDir = instancedir;
+      this.container = container;
+    }
+
+    public CoreDescriptorBuilder withSchema(String schema) {
+      properties.setProperty(CoreDescriptor.CORE_SCHEMA, schema);
+      return this;
+    }
+
+    public CoreDescriptorBuilder withConfig(String config) {
+      properties.setProperty(CoreDescriptor.CORE_CONFIG, config);
+      return this;
+    }
+
+    public CoreDescriptorBuilder withDataDir(String datadir) {
+      properties.setProperty(CoreDescriptor.CORE_DATADIR, datadir);
+      return this;
+    }
+
+    public CoreDescriptor build() {
+      return new CoreDescriptor(container, name, instanceDir, properties);
+    }
+
+    public CoreDescriptorBuilder isTransient(boolean isTransient) {
+      properties.setProperty(CoreDescriptor.CORE_TRANSIENT, Boolean.toString(isTransient));
+      return this;
+    }
+
+    public CoreDescriptorBuilder loadOnStartup(boolean loadOnStartup) {
+      properties.setProperty(CoreDescriptor.CORE_LOADONSTARTUP, Boolean.toString(loadOnStartup));
+      return this;
+    }
+  }
+
 }

Modified: lucene/dev/trunk/solr/test-framework/src/java/org/apache/solr/util/TestHarness.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/test-framework/src/java/org/apache/solr/util/TestHarness.java?rev=1502276&r1=1502275&r2=1502276&view=diff
==============================================================================
--- lucene/dev/trunk/solr/test-framework/src/java/org/apache/solr/util/TestHarness.java (original)
+++ lucene/dev/trunk/solr/test-framework/src/java/org/apache/solr/util/TestHarness.java Thu Jul 11 16:35:01 2013
@@ -17,7 +17,6 @@
 
 package org.apache.solr.util;
 
-import com.google.common.base.Charsets;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.params.CommonParams;
 import org.apache.solr.common.util.NamedList;
@@ -38,14 +37,12 @@ import org.apache.solr.schema.IndexSchem
 import org.apache.solr.schema.IndexSchemaFactory;
 import org.apache.solr.servlet.DirectSolrConnection;
 
-import java.io.ByteArrayInputStream;
 import java.io.File;
 import java.io.IOException;
 import java.io.StringWriter;
 import java.util.HashMap;
 import java.util.Map;
 
-
 /**
  * This class provides a simple harness that may be useful when
  * writing testcases.
@@ -156,7 +153,7 @@ public class TestHarness extends BaseTes
    */
   public TestHarness(String solrHome, String solrXml) {
     this(new SolrResourceLoader(solrHome),
-          ConfigSolr.fromInputStream(null, new ByteArrayInputStream(solrXml.getBytes(Charsets.UTF_8))));
+          ConfigSolr.fromString(solrXml));
   }
 
   /**
@@ -188,7 +185,7 @@ public class TestHarness extends BaseTes
         + "\" transient=\"false\" loadOnStartup=\"true\""
         + " shard=\"${shard:shard1}\" collection=\"${collection:collection1}\" instanceDir=\"" + coreName + "/\" />\n"
         + "  </cores>\n" + "</solr>";
-    return ConfigSolr.fromString(new SolrResourceLoader(dataDir), solrxml);
+    return ConfigSolr.fromString(solrxml);
   }
   
   public CoreContainer getCoreContainer() {
@@ -423,4 +420,6 @@ public class TestHarness extends BaseTes
       return new LocalSolrQueryRequest(TestHarness.this.getCore(), new NamedList(entries));
     }
   }
+
+
 }