You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by er...@apache.org on 2013/07/06 16:45:48 UTC

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

Author: erick
Date: Sat Jul  6 14:45:47 2013
New Revision: 1500284

URL: http://svn.apache.org/r1500284
Log:
Fix for SOLR-4982, creating cores with sysprops does not dereference them properly

Added:
    lucene/dev/trunk/solr/core/src/test/org/apache/solr/handler/admin/CoreAdminCreateDiscoverTest.java   (with props)
Modified:
    lucene/dev/trunk/solr/CHANGES.txt
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/CoreDescriptor.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/SolrCore.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminHandler.java
    lucene/dev/trunk/solr/core/src/test/org/apache/solr/handler/admin/CoreAdminHandlerTest.java
    lucene/dev/trunk/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java

Modified: lucene/dev/trunk/solr/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/CHANGES.txt?rev=1500284&r1=1500283&r2=1500284&view=diff
==============================================================================
--- lucene/dev/trunk/solr/CHANGES.txt (original)
+++ lucene/dev/trunk/solr/CHANGES.txt Sat Jul  6 14:45:47 2013
@@ -253,6 +253,12 @@ Bug Fixes
 
 * SOLR-5000: ManagedIndexSchema doesn't persist uniqueKey tag after calling addFields
   method. (Jun Ohtani, Steve Rowe)
+  
+* SOLR-4982: Creating a core while referencing system properties looks like it loses files
+  Actually, instanceDir, config, dataDir and schema are not dereferenced properly
+  when creating cores that reference sys vars (e.g. &dataDir=${dir}). In the dataDir
+  case in particular this leads to the index being put in a directory literally named
+  ${dir} but on restart the sysvar will be properly dereferenced.
 
 Optimizations
 ----------------------

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/CoreDescriptor.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/CoreDescriptor.java?rev=1500284&r1=1500283&r2=1500284&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/CoreDescriptor.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/CoreDescriptor.java Sat Jul  6 14:45:47 2013
@@ -46,7 +46,7 @@ public class CoreDescriptor {
   public static final String CORE_TRANSIENT = "transient";
   public static final String CORE_NODE_NAME = "coreNodeName";
 
-  static final String[] standardPropNames = {
+  public static final String[] standardPropNames = {
       CORE_NAME,
       CORE_CONFIG,
       CORE_INSTDIR,
@@ -65,7 +65,7 @@ public class CoreDescriptor {
   // them individually.
   private Properties coreProperties = new Properties();
 
-  //TODO: 5.0 remove this, this is solely a hack for persistence.
+  //TODO: 5.0 remove this, this is solely a hack for persistence. And perhaps creating cores in discovery mode?
   private Properties createdProperties = new Properties();
 
   private boolean loadedImplicit = false;

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/SolrCore.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/SolrCore.java?rev=1500284&r1=1500283&r2=1500284&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/SolrCore.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/SolrCore.java Sat Jul  6 14:45:47 2013
@@ -26,6 +26,7 @@ import java.io.Writer;
 import java.lang.reflect.Constructor;
 import java.net.URL;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.Date;
@@ -873,6 +874,18 @@ public final class SolrCore implements S
       propFile.getParentFile().mkdirs();
       Properties props = new Properties();
       props.put("name", cd.getName());
+
+      // This must be being created since there's no file here already. So write out all of the params we were
+      // created with. This _may_ overwrite the name above, but that's OK.
+      Collection<String> stds = new HashSet(Arrays.asList(CoreDescriptor.standardPropNames));
+      for (String prop : cd.getCreatedProperties().stringPropertyNames()) {
+        // Only preserve things that are legal, and let's just keep instDir right out of the persisted file even
+        // though it's part of the create properties on the URL.
+        if (! CoreDescriptor.CORE_INSTDIR.equals(prop) && stds.contains(prop)) {
+          props.put(prop, cd.getCreatedProperties().getProperty(prop));
+        }
+      }
+
       if (cc.isZooKeeperAware()) {
         String collection = cd.getCloudDescriptor().getCollectionName();
         if (collection != null) {

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminHandler.java?rev=1500284&r1=1500283&r2=1500284&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminHandler.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminHandler.java Sat Jul  6 14:45:47 2013
@@ -57,6 +57,7 @@ import org.apache.solr.update.UpdateLog;
 import org.apache.solr.update.processor.UpdateRequestProcessor;
 import org.apache.solr.update.processor.UpdateRequestProcessorChain;
 import org.apache.solr.util.NumberUtils;
+import org.apache.solr.util.PropertiesUtil;
 import org.apache.solr.util.RefCounted;
 import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;
@@ -414,6 +415,7 @@ public class CoreAdminHandler extends Re
       throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
                               "Core name is mandatory to CREATE a SolrCore");
     }
+
     CoreDescriptor dcore = null;
     try {
       
@@ -427,22 +429,30 @@ public class CoreAdminHandler extends Re
       if (instanceDir == null) {
         // instanceDir = coreContainer.getSolrHome() + "/" + name;
         instanceDir = name; // bare name is already relative to solr home
+      } else {
+        instanceDir = PropertiesUtil.substituteProperty(instanceDir, null);
       }
 
       dcore = new CoreDescriptor(coreContainer, name, instanceDir);
 
       //  fillup optional parameters
       String opts = params.get(CoreAdminParams.CONFIG);
-      if (opts != null)
+      if (opts != null) {
+        opts = PropertiesUtil.substituteProperty(opts, null);
         dcore.setConfigName(opts);
+      }
 
       opts = params.get(CoreAdminParams.SCHEMA);
-      if (opts != null)
+      if (opts != null) {
+        opts = PropertiesUtil.substituteProperty(opts, null);
         dcore.setSchemaName(opts);
+      }
 
       opts = params.get(CoreAdminParams.DATA_DIR);
-      if (opts != null)
+      if (opts != null) {
+        opts = PropertiesUtil.substituteProperty(opts, null);
         dcore.setDataDir(opts);
+      }
 
       opts = params.get(CoreAdminParams.ULOG_DIR);
       if (opts != null)

Added: 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=1500284&view=auto
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/handler/admin/CoreAdminCreateDiscoverTest.java (added)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/handler/admin/CoreAdminCreateDiscoverTest.java Sat Jul  6 14:45:47 2013
@@ -0,0 +1,228 @@
+/*
+ * 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.
+ */
+
+package org.apache.solr.handler.admin;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.common.params.CoreAdminParams;
+import org.apache.solr.core.CoreDescriptor;
+import org.apache.solr.core.SolrCoreDiscoverer;
+import org.apache.solr.response.SolrQueryResponse;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+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 {
+
+  private static File solrHomeDirectory = null;
+
+  private static CoreAdminHandler admin = null;
+//  private static CoreContainer cc = null;
+
+  private static String coreNormal = "normal";
+  private static String coreSysProps = "sys_props";
+
+  @BeforeClass
+  public static void before() throws Exception {
+    useFactory(null); // I require FS-based indexes for this test.
+
+    solrHomeDirectory = new File(TEMP_DIR, "solrHome/" + CoreAdminCreateDiscoverTest.getClassName());
+    if (solrHomeDirectory.exists()) {
+      FileUtils.deleteDirectory(solrHomeDirectory);
+    }
+    assertTrue("Failed to mkdirs workDir", solrHomeDirectory.mkdirs());
+
+    setupNoCoreTest(solrHomeDirectory, null);
+
+    admin = new CoreAdminHandler(h.getCoreContainer());
+  }
+
+  @AfterClass
+  public static void after() throws Exception {
+    h.close();
+    if (solrHomeDirectory.exists()) {
+      FileUtils.deleteDirectory(solrHomeDirectory);
+    }
+  }
+
+  private static void setupCore(String coreName, boolean blivet) throws IOException {
+    File instDir = new File(solrHomeDirectory, coreName);
+    File subHome = new File(instDir, "conf");
+    assertTrue("Failed to make subdirectory ", subHome.mkdirs());
+
+    // Be sure we pick up sysvars when we create this
+    String srcDir = SolrTestCaseJ4.TEST_HOME() + "/collection1/conf";
+    FileUtils.copyFile(new File(srcDir, "schema-tiny.xml"), new File(subHome, "schema_ren.xml"));
+    FileUtils.copyFile(new File(srcDir, "solrconfig-minimal.xml"), new File(subHome, "solrconfig_ren.xml"));
+
+    FileUtils.copyFile(new File(srcDir, "solrconfig.snippet.randomindexconfig.xml"),
+        new File(subHome, "solrconfig.snippet.randomindexconfig.xml"));
+  }
+
+  @Test
+  public void testCreateSavesSysProps() throws Exception {
+
+    setupCore(coreSysProps, true);
+
+    // create a new core (using CoreAdminHandler) w/ properties
+    // Just to be sure its NOT written to the core.properties file
+    File workDir = new File(solrHomeDirectory, coreSysProps);
+    System.setProperty("INSTDIR_TEST", workDir.getAbsolutePath());
+    System.setProperty("CONFIG_TEST", "solrconfig_ren.xml");
+    System.setProperty("SCHEMA_TEST", "schema_ren.xml");
+
+    File dataDir = new File(workDir.getAbsolutePath(), "data_diff");
+    System.setProperty("DATA_TEST", "data_diff");
+
+    SolrQueryResponse resp = new SolrQueryResponse();
+    admin.handleRequestBody
+        (req(CoreAdminParams.ACTION,
+            CoreAdminParams.CoreAdminAction.CREATE.toString(),
+            CoreAdminParams.NAME, coreSysProps,
+            CoreAdminParams.INSTANCE_DIR, "${INSTDIR_TEST}",
+            CoreAdminParams.CONFIG, "${CONFIG_TEST}",
+            CoreAdminParams.SCHEMA, "${SCHEMA_TEST}",
+            CoreAdminParams.DATA_DIR, "${DATA_TEST}"),
+            resp);
+    assertNull("Exception on create", resp.getException());
+
+    // verify props are in persisted file
+
+    Properties props = new Properties();
+    File propFile = new File(solrHomeDirectory, coreSysProps + "/" + SolrCoreDiscoverer.CORE_PROP_FILE);
+    FileInputStream is = new FileInputStream(propFile);
+    try {
+      props.load(is);
+    } finally {
+      org.apache.commons.io.IOUtils.closeQuietly(is);
+    }
+
+    assertEquals("Unexpected value preserved in properties file " + propFile.getAbsolutePath(),
+        props.getProperty(CoreAdminParams.NAME), coreSysProps);
+
+    assertEquals("Unexpected value preserved in properties file " + propFile.getAbsolutePath(),
+        props.getProperty(CoreAdminParams.CONFIG), "${CONFIG_TEST}");
+
+    assertEquals("Unexpected value preserved in properties file " + propFile.getAbsolutePath(),
+        props.getProperty(CoreAdminParams.SCHEMA), "${SCHEMA_TEST}");
+
+    assertEquals("Unexpected value preserved in properties file " + propFile.getAbsolutePath(),
+        props.getProperty(CoreAdminParams.DATA_DIR), "${DATA_TEST}");
+
+    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.
+
+    // Should NOT be a datadir named ${DATA_TEST} (literal).
+    File badDir = new File(workDir, "${DATA_TEST}");
+    assertFalse("Should have substituted the sys var, found file " + badDir.getAbsolutePath(), badDir.exists());
+
+    // 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}.
+    File test = new File(dataDir, "index");
+    assertTrue("Should have found index dir at " + test.getAbsolutePath(), test.exists());
+    File gen = new File(test, "segments.gen");
+    assertTrue("Should be segments.gen in the dir at " + gen.getAbsolutePath(), gen.exists());
+
+  }
+
+  @Test
+  public void testCreateSavesRegProps() throws Exception {
+
+    setupCore(coreNormal, true);
+
+    // create a new core (using CoreAdminHandler) w/ properties
+    // Just to be sure its NOT written to the core.properties file
+    File workDir = new File(solrHomeDirectory, coreNormal);
+    File data = new File(workDir, "data");
+
+    SolrQueryResponse resp = new SolrQueryResponse();
+    admin.handleRequestBody
+        (req(CoreAdminParams.ACTION,
+            CoreAdminParams.CoreAdminAction.CREATE.toString(),
+            CoreAdminParams.NAME, coreNormal,
+            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());
+
+    // verify props are in persisted file
+    Properties props = new Properties();
+    File propFile = new File(solrHomeDirectory, coreNormal + "/" + SolrCoreDiscoverer.CORE_PROP_FILE);
+    FileInputStream is = new FileInputStream(propFile);
+    try {
+      props.load(is);
+    } finally {
+      org.apache.commons.io.IOUtils.closeQuietly(is);
+    }
+
+    assertEquals("Unexpected value preserved in properties file " + propFile.getAbsolutePath(),
+        props.getProperty(CoreAdminParams.NAME), coreNormal);
+
+    assertEquals("Unexpected value preserved in properties file " + propFile.getAbsolutePath(),
+        props.getProperty(CoreAdminParams.CONFIG), "solrconfig_ren.xml");
+
+    assertEquals("Unexpected value preserved in properties file " + propFile.getAbsolutePath(),
+        props.getProperty(CoreAdminParams.SCHEMA), "schema_ren.xml");
+
+    assertEquals("Unexpected value preserved in properties file " + propFile.getAbsolutePath(),
+        props.getProperty(CoreAdminParams.DATA_DIR), data.getAbsolutePath());
+
+    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}.
+    File test = new File(data, "index");
+    assertTrue("Should have found index dir at " + test.getAbsolutePath(), test.exists());
+    File gen = new File(test, "segments.gen");
+    assertTrue("Should be segments.gen in the dir at " + gen.getAbsolutePath(), gen.exists());
+
+  }
+
+  // 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=1500284&r1=1500283&r2=1500284&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 Sat Jul  6 14:45:47 2013
@@ -17,9 +17,9 @@
 
 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.solr.handler.admin.CoreAdminHandler;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.params.CoreAdminParams;
 import org.apache.solr.common.util.NamedList;
@@ -28,15 +28,14 @@ import org.apache.solr.SolrTestCaseJ4;
 
 import java.util.Map;
 import java.io.File;
-import java.io.IOException;
-
-import javax.xml.xpath.XPathExpressionException;
 
 import org.apache.commons.io.FileUtils;
 
 import org.junit.BeforeClass;
+import org.junit.Rule;
 import org.junit.Test;
-import org.xml.sax.SAXException;
+import org.junit.rules.RuleChain;
+import org.junit.rules.TestRule;
 
 public class CoreAdminHandlerTest extends SolrTestCaseJ4 {
   
@@ -44,7 +43,93 @@ public class CoreAdminHandlerTest extend
   public static void beforeClass() throws Exception {
     initCore("solrconfig.xml", "schema.xml");
   }
-  
+
+  @Rule
+  public TestRule solrTestRules = RuleChain.outerRule(new SystemPropertiesRestoreRule());
+
+  public String getCoreName() { return this.getClass().getName() + "_sys_vars"; }
+
+  @Test
+  public void testCreateWithSysVars() throws Exception {
+    useFactory(null); // I require FS-based indexes for this test.
+
+    final File workDir = new File(TEMP_DIR, getCoreName());
+
+    if (workDir.exists()) {
+      FileUtils.deleteDirectory(workDir);
+    }
+    assertTrue("Failed to mkdirs workDir", workDir.mkdirs());
+    String coreName = "with_sys_vars";
+    File instDir = new File(workDir, coreName);
+    File subHome = new File(instDir, "conf");
+    assertTrue("Failed to make subdirectory ", subHome.mkdirs());
+
+    // Be sure we pick up sysvars when we create this
+    String srcDir = SolrTestCaseJ4.TEST_HOME() + "/collection1/conf";
+    FileUtils.copyFile(new File(srcDir, "schema-tiny.xml"), new File(subHome, "schema_ren.xml"));
+    FileUtils.copyFile(new File(srcDir, "solrconfig-minimal.xml"), new File(subHome, "solrconfig_ren.xml"));
+    FileUtils.copyFile(new File(srcDir, "solrconfig.snippet.randomindexconfig.xml"),
+        new File(subHome, "solrconfig.snippet.randomindexconfig.xml"));
+
+    final CoreContainer cores = h.getCoreContainer();
+    cores.setPersistent(false); // we'll do this explicitly as needed
+
+    final CoreAdminHandler admin = new CoreAdminHandler(cores);
+
+    // create a new core (using CoreAdminHandler) w/ properties
+    System.setProperty("INSTDIR_TEST", instDir.getAbsolutePath());
+    System.setProperty("CONFIG_TEST", "solrconfig_ren.xml");
+    System.setProperty("SCHEMA_TEST", "schema_ren.xml");
+
+    File dataDir = new File(workDir.getAbsolutePath(), "data_diff");
+    System.setProperty("DATA_TEST", dataDir.getAbsolutePath());
+
+    SolrQueryResponse resp = new SolrQueryResponse();
+    admin.handleRequestBody
+        (req(CoreAdminParams.ACTION,
+            CoreAdminParams.CoreAdminAction.CREATE.toString(),
+            CoreAdminParams.NAME, getCoreName(),
+            CoreAdminParams.INSTANCE_DIR, "${INSTDIR_TEST}",
+            CoreAdminParams.CONFIG, "${CONFIG_TEST}",
+            CoreAdminParams.SCHEMA, "${SCHEMA_TEST}",
+            CoreAdminParams.DATA_DIR, "${DATA_TEST}"),
+            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
+            ,"/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}']"
+            ,"/solr/cores/core[@name='" + getCoreName() + "' and @config='${CONFIG_TEST}']"
+        );
+
+    // Now assert that certain values are properly dereferenced in the process of creating the core, see
+    // SOLR-4982.
+
+    // Should NOT be a datadir named ${DATA_TEST} (literal). This is the bug after all
+    File badDir = new File(instDir, "${DATA_TEST}");
+    assertFalse("Should have substituted the sys var, found file " + badDir.getAbsolutePath(), badDir.exists());
+
+    // For the other 3 vars, we couldn't get past creating the core fi dereferencing didn't work correctly.
+
+    // Should have segments in the directory pointed to by the ${DATA_TEST}.
+    File test = new File(dataDir, "index");
+    assertTrue("Should have found index dir at " + test.getAbsolutePath(), test.exists());
+    test = new File(test,"segments.gen");
+    assertTrue("Should have found segments.gen at " + test.getAbsolutePath(), test.exists());
+
+    // Cleanup
+    FileUtils.deleteDirectory(workDir);
+
+  }
+
   @Test
   public void testCoreAdminHandler() throws Exception {
     final File workDir = new File(TEMP_DIR, this.getClass().getName());
@@ -141,6 +226,8 @@ public class CoreAdminHandlerTest extend
                
     // :TODO: because of SOLR-3665 we can't ask for status from all cores
 
-  }
+    // cleanup
+    FileUtils.deleteDirectory(workDir);
 
+  }
 }

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=1500284&r1=1500283&r2=1500284&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 Sat Jul  6 14:45:47 2013
@@ -21,6 +21,7 @@ import com.carrotsearch.randomizedtestin
 import com.carrotsearch.randomizedtesting.annotations.ThreadLeakFilters;
 import com.carrotsearch.randomizedtesting.rules.SystemPropertiesRestoreRule;
 import org.apache.commons.io.FileUtils;
+import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.QuickPatchThreadsFilter;
 import org.apache.solr.client.solrj.util.ClientUtils;
@@ -32,9 +33,11 @@ import org.apache.solr.common.params.Com
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.XML;
+import org.apache.solr.core.ConfigSolr;
 import org.apache.solr.core.CoreContainer;
 import org.apache.solr.core.SolrConfig;
 import org.apache.solr.core.SolrCore;
+import org.apache.solr.core.SolrResourceLoader;
 import org.apache.solr.handler.JsonUpdateRequestHandler;
 import org.apache.solr.request.LocalSolrQueryRequest;
 import org.apache.solr.request.SolrQueryRequest;
@@ -155,6 +158,25 @@ public abstract class SolrTestCaseJ4 ext
     }
   }
 
+  /**
+   * Call this from @BeforeClass to set up the test harness and update handler with no cores.
+   *
+   * @param solrHome The solr home directory.
+   * @param xmlStr - the text of an XML file to use. If null, use the what's the absolute minimal file.
+   * @throws Exception Lost of file-type things can go wrong.
+   */
+  public static void setupNoCoreTest(File solrHome, String xmlStr) throws Exception {
+
+    File tmpFile = new File(solrHome, ConfigSolr.SOLR_XML_FILE);
+    if (xmlStr == null) {
+      xmlStr = "<solr></solr>";
+    }
+    FileUtils.write(tmpFile, xmlStr, IOUtils.CHARSET_UTF_8.toString());
+
+    SolrResourceLoader loader = new SolrResourceLoader(solrHome.getAbsolutePath());
+    h = new TestHarness(loader, ConfigSolr.fromFile(loader, new File(solrHome, "solr.xml")));
+    lrf = h.getRequestFactory("standard", 0, 20, CommonParams.VERSION, "2.2");
+  }
 
   @Override
   public void setUp() throws Exception {



Re: svn commit: r1500284 - in /lucene/dev/trunk/solr: ./ core/src/java/org/apache/solr/core/ core/src/java/org/apache/solr/handler/admin/ core/src/test/org/apache/solr/handler/admin/ test-framework/src/java/org/apache/solr/

Posted by Erick Erickson <er...@gmail.com>.
Thanks for the confirmation! The code I modeled after created a new
CoreAdminHandler in non-static methods so the variable went out of
scope and was cleaned up automatically.

Since I put the admin handler creation in a BeforeClass where it was
static and tripped this check, just setting my static var to null was
the fix.... So it's kind of a test artifact.

One interesting bit is that "ant -Dtestcase=.... test" didn't trip
this, it's only when I run the full test suite that I see it...

Sorry for the noise
Erick

On Sun, Jul 7, 2013 at 4:28 AM, Uwe Schindler <uw...@thetaphi.de> wrote:
> Looks good - and works. No failures caused by this test anymore!
>
> Uwe
>
> -----
> Uwe Schindler
> H.-H.-Meier-Allee 63, D-28213 Bremen
> http://www.thetaphi.de
> eMail: uwe@thetaphi.de
>
>
>> -----Original Message-----
>> From: Erick Erickson [mailto:erickerickson@gmail.com]
>> Sent: Sunday, July 07, 2013 4:41 AM
>> To: dev@lucene.apache.org
>> Subject: Re: svn commit: r1500284 - in /lucene/dev/trunk/solr: ./
>> core/src/java/org/apache/solr/core/
>> core/src/java/org/apache/solr/handler/admin/
>> core/src/test/org/apache/solr/handler/admin/ test-
>> framework/src/java/org/apache/solr/
>>
>> OK, I was able to reproduce this on my machine and checked in what appears
>> to be a fix. But then I thought I'd handled this case before, so we'll see what
>> happens overnight.
>>
>> On Sat, Jul 6, 2013 at 7:10 PM, Uwe Schindler <uw...@thetaphi.de> wrote:
>> > This seems to cause a memory leak, see recently failing Jenkins tests!
>> >
>> > -----
>> > Uwe Schindler
>> > H.-H.-Meier-Allee 63, D-28213 Bremen
>> > http://www.thetaphi.de
>> > eMail: uwe@thetaphi.de
>> >
>> >
>> >> -----Original Message-----
>> >> From: erick@apache.org [mailto:erick@apache.org]
>> >> Sent: Saturday, July 06, 2013 4:46 PM
>> >> To: commits@lucene.apache.org
>> >> Subject: svn commit: r1500284 - in /lucene/dev/trunk/solr: ./
>> >> core/src/java/org/apache/solr/core/
>> >> core/src/java/org/apache/solr/handler/admin/
>> >> core/src/test/org/apache/solr/handler/admin/ test-
>> >> framework/src/java/org/apache/solr/
>> >>
>> >> Author: erick
>> >> Date: Sat Jul  6 14:45:47 2013
>> >> New Revision: 1500284
>> >>
>> >> URL: http://svn.apache.org/r1500284
>> >> Log:
>> >> Fix for SOLR-4982, creating cores with sysprops does not dereference
>> >> them properly
>> >>
>> >> Added:
>> >>
>> >>
>> lucene/dev/trunk/solr/core/src/test/org/apache/solr/handler/admin/CoreA
>> >> dminCreateDiscoverTest.java   (with props)
>> >> Modified:
>> >>     lucene/dev/trunk/solr/CHANGES.txt
>> >>
>> >> lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/CoreDescript
>> >> or.j
>> >> ava
>> >>
>> >> lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/SolrCore.jav
>> >> a
>> >>
>> >>
>> lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/admin/Cor
>> >> eA
>> >> dminHandler.java
>> >>
>> >> lucene/dev/trunk/solr/core/src/test/org/apache/solr/handler/admin/Cor
>> >> eA
>> >> dminHandlerTest.java
>> >>     lucene/dev/trunk/solr/test-
>> >> framework/src/java/org/apache/solr/SolrTestCaseJ4.java
>> >>
>> >> Modified: lucene/dev/trunk/solr/CHANGES.txt
>> >> URL:
>> >>
>> http://svn.apache.org/viewvc/lucene/dev/trunk/solr/CHANGES.txt?rev=15
>> >> 0
>> >> 0284&r1=1500283&r2=1500284&view=diff
>> >>
>> ==========================================================
>> >> ====================
>> >> --- lucene/dev/trunk/solr/CHANGES.txt (original)
>> >> +++ lucene/dev/trunk/solr/CHANGES.txt Sat Jul  6 14:45:47 2013
>> >> @@ -253,6 +253,12 @@ Bug Fixes
>> >>
>> >>  * SOLR-5000: ManagedIndexSchema doesn't persist uniqueKey tag after
>> >> calling addFields
>> >>    method. (Jun Ohtani, Steve Rowe)
>> >> +
>> >> +* SOLR-4982: Creating a core while referencing system properties
>> >> +looks like it loses files
>> >> +  Actually, instanceDir, config, dataDir and schema are not
>> >> +dereferenced properly
>> >> +  when creating cores that reference sys vars (e.g. &dataDir=${dir}).
>> >> +In the dataDir
>> >> +  case in particular this leads to the index being put in a
>> >> +directory literally named
>> >> +  ${dir} but on restart the sysvar will be properly dereferenced.
>> >>
>> >>  Optimizations
>> >>  ----------------------
>> >>
>> >> Modified:
>> >> lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/CoreDescript
>> >> or.j
>> >> ava
>> >> URL:
>> >> http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/
>> >> apa
>> >>
>> che/solr/core/CoreDescriptor.java?rev=1500284&r1=1500283&r2=1500284&v
>> >> iew=diff
>> >>
>> ==========================================================
>> >> ====================
>> >> ---
>> >> lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/CoreDescript
>> >> or.j
>> >> ava (original)
>> >> +++
>> lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/CoreDesc
>> >> +++ rip tor.java Sat Jul  6 14:45:47 2013
>> >> @@ -46,7 +46,7 @@ public class CoreDescriptor {
>> >>    public static final String CORE_TRANSIENT = "transient";
>> >>    public static final String CORE_NODE_NAME = "coreNodeName";
>> >>
>> >> -  static final String[] standardPropNames = {
>> >> +  public static final String[] standardPropNames = {
>> >>        CORE_NAME,
>> >>        CORE_CONFIG,
>> >>        CORE_INSTDIR,
>> >> @@ -65,7 +65,7 @@ public class CoreDescriptor {
>> >>    // them individually.
>> >>    private Properties coreProperties = new Properties();
>> >>
>> >> -  //TODO: 5.0 remove this, this is solely a hack for persistence.
>> >> +  //TODO: 5.0 remove this, this is solely a hack for persistence.
>> >> + And perhaps
>> >> creating cores in discovery mode?
>> >>    private Properties createdProperties = new Properties();
>> >>
>> >>    private boolean loadedImplicit = false;
>> >>
>> >> Modified:
>> >> lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/SolrCore.jav
>> >> a
>> >> URL:
>> >> http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/
>> >> apa
>> >>
>> che/solr/core/SolrCore.java?rev=1500284&r1=1500283&r2=1500284&view=di
>> >> ff
>> >>
>> ==========================================================
>> >> ====================
>> >> ---
>> >> lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/SolrCore.jav
>> >> a
>> >> (original)
>> >> +++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/SolrCore
>> >> +++ .ja
>> >> +++ va Sat Jul  6 14:45:47 2013
>> >> @@ -26,6 +26,7 @@ import java.io.Writer;  import
>> >> java.lang.reflect.Constructor;  import java.net.URL;  import
>> >> java.util.ArrayList;
>> >> +import java.util.Arrays;
>> >>  import java.util.Collection;
>> >>  import java.util.Collections;
>> >>  import java.util.Date;
>> >> @@ -873,6 +874,18 @@ public final class SolrCore implements S
>> >>        propFile.getParentFile().mkdirs();
>> >>        Properties props = new Properties();
>> >>        props.put("name", cd.getName());
>> >> +
>> >> +      // This must be being created since there's no file here
>> >> + already. So write
>> >> out all of the params we were
>> >> +      // created with. This _may_ overwrite the name above, but that's OK.
>> >> +      Collection<String> stds = new
>> >> HashSet(Arrays.asList(CoreDescriptor.standardPropNames));
>> >> +      for (String prop : cd.getCreatedProperties().stringPropertyNames()) {
>> >> +        // Only preserve things that are legal, and let's just keep
>> >> + instDir right out
>> >> of the persisted file even
>> >> +        // though it's part of the create properties on the URL.
>> >> +        if (! CoreDescriptor.CORE_INSTDIR.equals(prop) &&
>> >> stds.contains(prop)) {
>> >> +          props.put(prop, cd.getCreatedProperties().getProperty(prop));
>> >> +        }
>> >> +      }
>> >> +
>> >>        if (cc.isZooKeeperAware()) {
>> >>          String collection = cd.getCloudDescriptor().getCollectionName();
>> >>          if (collection != null) {
>> >>
>> >> Modified:
>> >>
>> lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/admin/Cor
>> >> eA
>> >> dminHandler.java
>> >> URL:
>> >> http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/
>> >> apa
>> >>
>> che/solr/handler/admin/CoreAdminHandler.java?rev=1500284&r1=1500283
>> >> &r2=1500284&view=diff
>> >>
>> ==========================================================
>> >> ====================
>> >> ---
>> >>
>> lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/admin/Cor
>> >> eA
>> >> dminHandler.java (original)
>> >> +++
>> >> lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/admin/Co
>> >> +++ reAdminHandler.java Sat Jul  6 14:45:47 2013
>> >> @@ -57,6 +57,7 @@ import org.apache.solr.update.UpdateLog;  import
>> >> org.apache.solr.update.processor.UpdateRequestProcessor;
>> >>  import
>> org.apache.solr.update.processor.UpdateRequestProcessorChain;
>> >>  import org.apache.solr.util.NumberUtils;
>> >> +import org.apache.solr.util.PropertiesUtil;
>> >>  import org.apache.solr.util.RefCounted;  import
>> >> org.apache.zookeeper.KeeperException;
>> >>  import org.slf4j.Logger;
>> >> @@ -414,6 +415,7 @@ public class CoreAdminHandler extends Re
>> >>        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
>> >>                                "Core name is mandatory to CREATE a SolrCore");
>> >>      }
>> >> +
>> >>      CoreDescriptor dcore = null;
>> >>      try {
>> >>
>> >> @@ -427,22 +429,30 @@ public class CoreAdminHandler extends Re
>> >>        if (instanceDir == null) {
>> >>          // instanceDir = coreContainer.getSolrHome() + "/" + name;
>> >>          instanceDir = name; // bare name is already relative to solr
>> >> home
>> >> +      } else {
>> >> +        instanceDir = PropertiesUtil.substituteProperty(instanceDir,
>> >> + null);
>> >>        }
>> >>
>> >>        dcore = new CoreDescriptor(coreContainer, name, instanceDir);
>> >>
>> >>        //  fillup optional parameters
>> >>        String opts = params.get(CoreAdminParams.CONFIG);
>> >> -      if (opts != null)
>> >> +      if (opts != null) {
>> >> +        opts = PropertiesUtil.substituteProperty(opts, null);
>> >>          dcore.setConfigName(opts);
>> >> +      }
>> >>
>> >>        opts = params.get(CoreAdminParams.SCHEMA);
>> >> -      if (opts != null)
>> >> +      if (opts != null) {
>> >> +        opts = PropertiesUtil.substituteProperty(opts, null);
>> >>          dcore.setSchemaName(opts);
>> >> +      }
>> >>
>> >>        opts = params.get(CoreAdminParams.DATA_DIR);
>> >> -      if (opts != null)
>> >> +      if (opts != null) {
>> >> +        opts = PropertiesUtil.substituteProperty(opts, null);
>> >>          dcore.setDataDir(opts);
>> >> +      }
>> >>
>> >>        opts = params.get(CoreAdminParams.ULOG_DIR);
>> >>        if (opts != null)
>> >>
>> >> Added:
>> >> lucene/dev/trunk/solr/core/src/test/org/apache/solr/handler/admin/Cor
>> >> eA
>> >> dminCreateDiscoverTest.java
>> >> URL:
>> >> http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/
>> >> apa
>> >>
>> che/solr/handler/admin/CoreAdminCreateDiscoverTest.java?rev=1500284&
>> >> view=auto
>> >>
>> ==========================================================
>> >> ====================
>> >> ---
>> >> lucene/dev/trunk/solr/core/src/test/org/apache/solr/handler/admin/Cor
>> >> eA
>> >> dminCreateDiscoverTest.java (added)
>> >> +++
>> >> lucene/dev/trunk/solr/core/src/test/org/apache/solr/handler/admin/Co
>> >> +++ reAdminCreateDiscoverTest.java Sat Jul  6 14:45:47 2013
>> >> @@ -0,0 +1,228 @@
>> >> +/*
>> >> + * 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.
>> >> + */
>> >> +
>> >> +package org.apache.solr.handler.admin;
>> >> +
>> >> +import org.apache.commons.io.FileUtils; import
>> >> +org.apache.solr.SolrTestCaseJ4; import
>> >> +org.apache.solr.common.params.CoreAdminParams;
>> >> +import org.apache.solr.core.CoreDescriptor;
>> >> +import org.apache.solr.core.SolrCoreDiscoverer;
>> >> +import org.apache.solr.response.SolrQueryResponse;
>> >> +import org.junit.AfterClass;
>> >> +import org.junit.BeforeClass;
>> >> +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 {
>> >> +
>> >> +  private static File solrHomeDirectory = null;
>> >> +
>> >> +  private static CoreAdminHandler admin = null; //  private static
>> >> +CoreContainer cc = null;
>> >> +
>> >> +  private static String coreNormal = "normal";  private static
>> >> + String coreSysProps = "sys_props";
>> >> +
>> >> +  @BeforeClass
>> >> +  public static void before() throws Exception {
>> >> +    useFactory(null); // I require FS-based indexes for this test.
>> >> +
>> >> +    solrHomeDirectory = new File(TEMP_DIR, "solrHome/" +
>> >> CoreAdminCreateDiscoverTest.getClassName());
>> >> +    if (solrHomeDirectory.exists()) {
>> >> +      FileUtils.deleteDirectory(solrHomeDirectory);
>> >> +    }
>> >> +    assertTrue("Failed to mkdirs workDir",
>> >> + solrHomeDirectory.mkdirs());
>> >> +
>> >> +    setupNoCoreTest(solrHomeDirectory, null);
>> >> +
>> >> +    admin = new CoreAdminHandler(h.getCoreContainer());
>> >> +  }
>> >> +
>> >> +  @AfterClass
>> >> +  public static void after() throws Exception {
>> >> +    h.close();
>> >> +    if (solrHomeDirectory.exists()) {
>> >> +      FileUtils.deleteDirectory(solrHomeDirectory);
>> >> +    }
>> >> +  }
>> >> +
>> >> +  private static void setupCore(String coreName, boolean blivet)
>> >> + throws
>> >> IOException {
>> >> +    File instDir = new File(solrHomeDirectory, coreName);
>> >> +    File subHome = new File(instDir, "conf");
>> >> +    assertTrue("Failed to make subdirectory ", subHome.mkdirs());
>> >> +
>> >> +    // Be sure we pick up sysvars when we create this
>> >> +    String srcDir = SolrTestCaseJ4.TEST_HOME() + "/collection1/conf";
>> >> +    FileUtils.copyFile(new File(srcDir, "schema-tiny.xml"), new
>> >> + File(subHome,
>> >> "schema_ren.xml"));
>> >> +    FileUtils.copyFile(new File(srcDir, "solrconfig-minimal.xml"),
>> >> + new File(subHome, "solrconfig_ren.xml"));
>> >> +
>> >> +    FileUtils.copyFile(new File(srcDir,
>> >> "solrconfig.snippet.randomindexconfig.xml"),
>> >> +        new File(subHome,
>> >> + "solrconfig.snippet.randomindexconfig.xml"));
>> >> +  }
>> >> +
>> >> +  @Test
>> >> +  public void testCreateSavesSysProps() throws Exception {
>> >> +
>> >> +    setupCore(coreSysProps, true);
>> >> +
>> >> +    // create a new core (using CoreAdminHandler) w/ properties
>> >> +    // Just to be sure its NOT written to the core.properties file
>> >> +    File workDir = new File(solrHomeDirectory, coreSysProps);
>> >> +    System.setProperty("INSTDIR_TEST", workDir.getAbsolutePath());
>> >> +    System.setProperty("CONFIG_TEST", "solrconfig_ren.xml");
>> >> +    System.setProperty("SCHEMA_TEST", "schema_ren.xml");
>> >> +
>> >> +    File dataDir = new File(workDir.getAbsolutePath(), "data_diff");
>> >> +    System.setProperty("DATA_TEST", "data_diff");
>> >> +
>> >> +    SolrQueryResponse resp = new SolrQueryResponse();
>> >> +    admin.handleRequestBody
>> >> +        (req(CoreAdminParams.ACTION,
>> >> +            CoreAdminParams.CoreAdminAction.CREATE.toString(),
>> >> +            CoreAdminParams.NAME, coreSysProps,
>> >> +            CoreAdminParams.INSTANCE_DIR, "${INSTDIR_TEST}",
>> >> +            CoreAdminParams.CONFIG, "${CONFIG_TEST}",
>> >> +            CoreAdminParams.SCHEMA, "${SCHEMA_TEST}",
>> >> +            CoreAdminParams.DATA_DIR, "${DATA_TEST}"),
>> >> +            resp);
>> >> +    assertNull("Exception on create", resp.getException());
>> >> +
>> >> +    // verify props are in persisted file
>> >> +
>> >> +    Properties props = new Properties();
>> >> +    File propFile = new File(solrHomeDirectory, coreSysProps + "/" +
>> >> SolrCoreDiscoverer.CORE_PROP_FILE);
>> >> +    FileInputStream is = new FileInputStream(propFile);
>> >> +    try {
>> >> +      props.load(is);
>> >> +    } finally {
>> >> +      org.apache.commons.io.IOUtils.closeQuietly(is);
>> >> +    }
>> >> +
>> >> +    assertEquals("Unexpected value preserved in properties file " +
>> >> propFile.getAbsolutePath(),
>> >> +        props.getProperty(CoreAdminParams.NAME), coreSysProps);
>> >> +
>> >> +    assertEquals("Unexpected value preserved in properties file " +
>> >> propFile.getAbsolutePath(),
>> >> +        props.getProperty(CoreAdminParams.CONFIG),
>> >> + "${CONFIG_TEST}");
>> >> +
>> >> +    assertEquals("Unexpected value preserved in properties file " +
>> >> propFile.getAbsolutePath(),
>> >> +        props.getProperty(CoreAdminParams.SCHEMA),
>> >> + "${SCHEMA_TEST}");
>> >> +
>> >> +    assertEquals("Unexpected value preserved in properties file " +
>> >> propFile.getAbsolutePath(),
>> >> +        props.getProperty(CoreAdminParams.DATA_DIR),
>> >> + "${DATA_TEST}");
>> >> +
>> >> +    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.
>> >> +
>> >> +    // Should NOT be a datadir named ${DATA_TEST} (literal).
>> >> +    File badDir = new File(workDir, "${DATA_TEST}");
>> >> +    assertFalse("Should have substituted the sys var, found file " +
>> >> + badDir.getAbsolutePath(), badDir.exists());
>> >> +
>> >> +    // 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}.
>> >> +    File test = new File(dataDir, "index");
>> >> +    assertTrue("Should have found index dir at " +
>> >> + test.getAbsolutePath(),
>> >> test.exists());
>> >> +    File gen = new File(test, "segments.gen");
>> >> +    assertTrue("Should be segments.gen in the dir at " +
>> >> + gen.getAbsolutePath(), gen.exists());
>> >> +
>> >> +  }
>> >> +
>> >> +  @Test
>> >> +  public void testCreateSavesRegProps() throws Exception {
>> >> +
>> >> +    setupCore(coreNormal, true);
>> >> +
>> >> +    // create a new core (using CoreAdminHandler) w/ properties
>> >> +    // Just to be sure its NOT written to the core.properties file
>> >> +    File workDir = new File(solrHomeDirectory, coreNormal);
>> >> +    File data = new File(workDir, "data");
>> >> +
>> >> +    SolrQueryResponse resp = new SolrQueryResponse();
>> >> +    admin.handleRequestBody
>> >> +        (req(CoreAdminParams.ACTION,
>> >> +            CoreAdminParams.CoreAdminAction.CREATE.toString(),
>> >> +            CoreAdminParams.NAME, coreNormal,
>> >> +            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());
>> >> +
>> >> +    // verify props are in persisted file
>> >> +    Properties props = new Properties();
>> >> +    File propFile = new File(solrHomeDirectory, coreNormal + "/" +
>> >> SolrCoreDiscoverer.CORE_PROP_FILE);
>> >> +    FileInputStream is = new FileInputStream(propFile);
>> >> +    try {
>> >> +      props.load(is);
>> >> +    } finally {
>> >> +      org.apache.commons.io.IOUtils.closeQuietly(is);
>> >> +    }
>> >> +
>> >> +    assertEquals("Unexpected value preserved in properties file " +
>> >> propFile.getAbsolutePath(),
>> >> +        props.getProperty(CoreAdminParams.NAME), coreNormal);
>> >> +
>> >> +    assertEquals("Unexpected value preserved in properties file " +
>> >> propFile.getAbsolutePath(),
>> >> +        props.getProperty(CoreAdminParams.CONFIG),
>> >> + "solrconfig_ren.xml");
>> >> +
>> >> +    assertEquals("Unexpected value preserved in properties file " +
>> >> propFile.getAbsolutePath(),
>> >> +        props.getProperty(CoreAdminParams.SCHEMA),
>> >> + "schema_ren.xml");
>> >> +
>> >> +    assertEquals("Unexpected value preserved in properties file " +
>> >> propFile.getAbsolutePath(),
>> >> +        props.getProperty(CoreAdminParams.DATA_DIR),
>> >> + data.getAbsolutePath());
>> >> +
>> >> +    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}.
>> >> +    File test = new File(data, "index");
>> >> +    assertTrue("Should have found index dir at " +
>> >> + test.getAbsolutePath(),
>> >> test.exists());
>> >> +    File gen = new File(test, "segments.gen");
>> >> +    assertTrue("Should be segments.gen in the dir at " +
>> >> + gen.getAbsolutePath(), gen.exists());
>> >> +
>> >> +  }
>> >> +
>> >> +  // 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/Cor
>> >> eA
>> >> dminHandlerTest.java
>> >> URL:
>> >> http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/
>> >> apa
>> >>
>> che/solr/handler/admin/CoreAdminHandlerTest.java?rev=1500284&r1=1500
>> >> 283&r2=1500284&view=diff
>> >>
>> ==========================================================
>> >> ====================
>> >> ---
>> >> lucene/dev/trunk/solr/core/src/test/org/apache/solr/handler/admin/Cor
>> >> eA
>> >> dminHandlerTest.java (original)
>> >> +++
>> >> lucene/dev/trunk/solr/core/src/test/org/apache/solr/handler/admin/Co
>> >> +++ reAdminHandlerTest.java Sat Jul  6 14:45:47 2013
>> >> @@ -17,9 +17,9 @@
>> >>
>> >>  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.solr.handler.admin.CoreAdminHandler;
>> >>  import org.apache.solr.common.SolrException;
>> >>  import org.apache.solr.common.params.CoreAdminParams;
>> >>  import org.apache.solr.common.util.NamedList;
>> >> @@ -28,15 +28,14 @@ import org.apache.solr.SolrTestCaseJ4;
>> >>
>> >>  import java.util.Map;
>> >>  import java.io.File;
>> >> -import java.io.IOException;
>> >> -
>> >> -import javax.xml.xpath.XPathExpressionException;
>> >>
>> >>  import org.apache.commons.io.FileUtils;
>> >>
>> >>  import org.junit.BeforeClass;
>> >> +import org.junit.Rule;
>> >>  import org.junit.Test;
>> >> -import org.xml.sax.SAXException;
>> >> +import org.junit.rules.RuleChain;
>> >> +import org.junit.rules.TestRule;
>> >>
>> >>  public class CoreAdminHandlerTest extends SolrTestCaseJ4 {
>> >>
>> >> @@ -44,7 +43,93 @@ public class CoreAdminHandlerTest extend
>> >>    public static void beforeClass() throws Exception {
>> >>      initCore("solrconfig.xml", "schema.xml");
>> >>    }
>> >> -
>> >> +
>> >> +  @Rule
>> >> +  public TestRule solrTestRules = RuleChain.outerRule(new
>> >> + SystemPropertiesRestoreRule());
>> >> +
>> >> +  public String getCoreName() { return this.getClass().getName() +
>> >> + "_sys_vars"; }
>> >> +
>> >> +  @Test
>> >> +  public void testCreateWithSysVars() throws Exception {
>> >> +    useFactory(null); // I require FS-based indexes for this test.
>> >> +
>> >> +    final File workDir = new File(TEMP_DIR, getCoreName());
>> >> +
>> >> +    if (workDir.exists()) {
>> >> +      FileUtils.deleteDirectory(workDir);
>> >> +    }
>> >> +    assertTrue("Failed to mkdirs workDir", workDir.mkdirs());
>> >> +    String coreName = "with_sys_vars";
>> >> +    File instDir = new File(workDir, coreName);
>> >> +    File subHome = new File(instDir, "conf");
>> >> +    assertTrue("Failed to make subdirectory ", subHome.mkdirs());
>> >> +
>> >> +    // Be sure we pick up sysvars when we create this
>> >> +    String srcDir = SolrTestCaseJ4.TEST_HOME() + "/collection1/conf";
>> >> +    FileUtils.copyFile(new File(srcDir, "schema-tiny.xml"), new
>> >> + File(subHome,
>> >> "schema_ren.xml"));
>> >> +    FileUtils.copyFile(new File(srcDir, "solrconfig-minimal.xml"),
>> >> + new
>> >> File(subHome, "solrconfig_ren.xml"));
>> >> +    FileUtils.copyFile(new File(srcDir,
>> >> "solrconfig.snippet.randomindexconfig.xml"),
>> >> +        new File(subHome,
>> >> + "solrconfig.snippet.randomindexconfig.xml"));
>> >> +
>> >> +    final CoreContainer cores = h.getCoreContainer();
>> >> +    cores.setPersistent(false); // we'll do this explicitly as
>> >> + needed
>> >> +
>> >> +    final CoreAdminHandler admin = new CoreAdminHandler(cores);
>> >> +
>> >> +    // create a new core (using CoreAdminHandler) w/ properties
>> >> +    System.setProperty("INSTDIR_TEST", instDir.getAbsolutePath());
>> >> +    System.setProperty("CONFIG_TEST", "solrconfig_ren.xml");
>> >> +    System.setProperty("SCHEMA_TEST", "schema_ren.xml");
>> >> +
>> >> +    File dataDir = new File(workDir.getAbsolutePath(), "data_diff");
>> >> +    System.setProperty("DATA_TEST", dataDir.getAbsolutePath());
>> >> +
>> >> +    SolrQueryResponse resp = new SolrQueryResponse();
>> >> +    admin.handleRequestBody
>> >> +        (req(CoreAdminParams.ACTION,
>> >> +            CoreAdminParams.CoreAdminAction.CREATE.toString(),
>> >> +            CoreAdminParams.NAME, getCoreName(),
>> >> +            CoreAdminParams.INSTANCE_DIR, "${INSTDIR_TEST}",
>> >> +            CoreAdminParams.CONFIG, "${CONFIG_TEST}",
>> >> +            CoreAdminParams.SCHEMA, "${SCHEMA_TEST}",
>> >> +            CoreAdminParams.DATA_DIR, "${DATA_TEST}"),
>> >> +            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
>> >> +            ,"/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}']"
>> >> +            ,"/solr/cores/core[@name='" + getCoreName() + "' and
>> >> @config='${CONFIG_TEST}']"
>> >> +        );
>> >> +
>> >> +    // Now assert that certain values are properly dereferenced in
>> >> + the
>> >> process of creating the core, see
>> >> +    // SOLR-4982.
>> >> +
>> >> +    // Should NOT be a datadir named ${DATA_TEST} (literal). This is
>> >> + the bug
>> >> after all
>> >> +    File badDir = new File(instDir, "${DATA_TEST}");
>> >> +    assertFalse("Should have substituted the sys var, found file " +
>> >> + badDir.getAbsolutePath(), badDir.exists());
>> >> +
>> >> +    // For the other 3 vars, we couldn't get past creating the core
>> >> + fi
>> >> dereferencing didn't work correctly.
>> >> +
>> >> +    // Should have segments in the directory pointed to by the
>> >> ${DATA_TEST}.
>> >> +    File test = new File(dataDir, "index");
>> >> +    assertTrue("Should have found index dir at " +
>> >> + test.getAbsolutePath(),
>> >> test.exists());
>> >> +    test = new File(test,"segments.gen");
>> >> +    assertTrue("Should have found segments.gen at " +
>> >> + test.getAbsolutePath(), test.exists());
>> >> +
>> >> +    // Cleanup
>> >> +    FileUtils.deleteDirectory(workDir);
>> >> +
>> >> +  }
>> >> +
>> >>    @Test
>> >>    public void testCoreAdminHandler() throws Exception {
>> >>      final File workDir = new File(TEMP_DIR,
>> >> this.getClass().getName()); @@ -
>> >> 141,6 +226,8 @@ public class CoreAdminHandlerTest extend
>> >>
>> >>      // :TODO: because of SOLR-3665 we can't ask for status from all
>> >> cores
>> >>
>> >> -  }
>> >> +    // cleanup
>> >> +    FileUtils.deleteDirectory(workDir);
>> >>
>> >> +  }
>> >>  }
>> >>
>> >> 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=1500284&r1
>> >> =
>> >> 1500283&r2=1500284&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/Sol
>> >> +++ rTe stCaseJ4.java Sat Jul  6 14:45:47 2013
>> >> @@ -21,6 +21,7 @@ import com.carrotsearch.randomizedtestin  import
>> >> com.carrotsearch.randomizedtesting.annotations.ThreadLeakFilters;
>> >>  import
>> >> com.carrotsearch.randomizedtesting.rules.SystemPropertiesRestoreRule;
>> >>  import org.apache.commons.io.FileUtils;
>> >> +import org.apache.lucene.util.IOUtils;
>> >>  import org.apache.lucene.util.LuceneTestCase;
>> >>  import org.apache.lucene.util.QuickPatchThreadsFilter;
>> >>  import org.apache.solr.client.solrj.util.ClientUtils;
>> >> @@ -32,9 +33,11 @@ import org.apache.solr.common.params.Com
>> import
>> >> org.apache.solr.common.params.ModifiableSolrParams;
>> >>  import org.apache.solr.common.params.SolrParams;
>> >>  import org.apache.solr.common.util.XML;
>> >> +import org.apache.solr.core.ConfigSolr;
>> >>  import org.apache.solr.core.CoreContainer;
>> >>  import org.apache.solr.core.SolrConfig;  import
>> >> org.apache.solr.core.SolrCore;
>> >> +import org.apache.solr.core.SolrResourceLoader;
>> >>  import org.apache.solr.handler.JsonUpdateRequestHandler;
>> >>  import org.apache.solr.request.LocalSolrQueryRequest;
>> >>  import org.apache.solr.request.SolrQueryRequest;
>> >> @@ -155,6 +158,25 @@ public abstract class SolrTestCaseJ4 ext
>> >>      }
>> >>    }
>> >>
>> >> +  /**
>> >> +   * Call this from @BeforeClass to set up the test harness and
>> >> + update
>> >> handler with no cores.
>> >> +   *
>> >> +   * @param solrHome The solr home directory.
>> >> +   * @param xmlStr - the text of an XML file to use. If null, use
>> >> + the what's the
>> >> absolute minimal file.
>> >> +   * @throws Exception Lost of file-type things can go wrong.
>> >> +   */
>> >> +  public static void setupNoCoreTest(File solrHome, String xmlStr)
>> >> + throws Exception {
>> >> +
>> >> +    File tmpFile = new File(solrHome, ConfigSolr.SOLR_XML_FILE);
>> >> +    if (xmlStr == null) {
>> >> +      xmlStr = "<solr></solr>";
>> >> +    }
>> >> +    FileUtils.write(tmpFile, xmlStr,
>> >> + IOUtils.CHARSET_UTF_8.toString());
>> >> +
>> >> +    SolrResourceLoader loader = new
>> >> SolrResourceLoader(solrHome.getAbsolutePath());
>> >> +    h = new TestHarness(loader, ConfigSolr.fromFile(loader, new
>> >> File(solrHome, "solr.xml")));
>> >> +    lrf = h.getRequestFactory("standard", 0, 20,
>> >> + CommonParams.VERSION, "2.2");  }
>> >>
>> >>    @Override
>> >>    public void setUp() throws Exception {
>> >
>> >
>> >
>> > ---------------------------------------------------------------------
>> > To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org For
>> > additional commands, e-mail: dev-help@lucene.apache.org
>> >
>>
>> ---------------------------------------------------------------------
>> To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org For additional
>> commands, e-mail: dev-help@lucene.apache.org
>
>
> ---------------------------------------------------------------------
> To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
> For additional commands, e-mail: dev-help@lucene.apache.org
>

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org


RE: svn commit: r1500284 - in /lucene/dev/trunk/solr: ./ core/src/java/org/apache/solr/core/ core/src/java/org/apache/solr/handler/admin/ core/src/test/org/apache/solr/handler/admin/ test-framework/src/java/org/apache/solr/

Posted by Uwe Schindler <uw...@thetaphi.de>.
Looks good - and works. No failures caused by this test anymore!

Uwe

-----
Uwe Schindler
H.-H.-Meier-Allee 63, D-28213 Bremen
http://www.thetaphi.de
eMail: uwe@thetaphi.de


> -----Original Message-----
> From: Erick Erickson [mailto:erickerickson@gmail.com]
> Sent: Sunday, July 07, 2013 4:41 AM
> To: dev@lucene.apache.org
> Subject: Re: svn commit: r1500284 - in /lucene/dev/trunk/solr: ./
> core/src/java/org/apache/solr/core/
> core/src/java/org/apache/solr/handler/admin/
> core/src/test/org/apache/solr/handler/admin/ test-
> framework/src/java/org/apache/solr/
> 
> OK, I was able to reproduce this on my machine and checked in what appears
> to be a fix. But then I thought I'd handled this case before, so we'll see what
> happens overnight.
> 
> On Sat, Jul 6, 2013 at 7:10 PM, Uwe Schindler <uw...@thetaphi.de> wrote:
> > This seems to cause a memory leak, see recently failing Jenkins tests!
> >
> > -----
> > Uwe Schindler
> > H.-H.-Meier-Allee 63, D-28213 Bremen
> > http://www.thetaphi.de
> > eMail: uwe@thetaphi.de
> >
> >
> >> -----Original Message-----
> >> From: erick@apache.org [mailto:erick@apache.org]
> >> Sent: Saturday, July 06, 2013 4:46 PM
> >> To: commits@lucene.apache.org
> >> Subject: svn commit: r1500284 - in /lucene/dev/trunk/solr: ./
> >> core/src/java/org/apache/solr/core/
> >> core/src/java/org/apache/solr/handler/admin/
> >> core/src/test/org/apache/solr/handler/admin/ test-
> >> framework/src/java/org/apache/solr/
> >>
> >> Author: erick
> >> Date: Sat Jul  6 14:45:47 2013
> >> New Revision: 1500284
> >>
> >> URL: http://svn.apache.org/r1500284
> >> Log:
> >> Fix for SOLR-4982, creating cores with sysprops does not dereference
> >> them properly
> >>
> >> Added:
> >>
> >>
> lucene/dev/trunk/solr/core/src/test/org/apache/solr/handler/admin/CoreA
> >> dminCreateDiscoverTest.java   (with props)
> >> Modified:
> >>     lucene/dev/trunk/solr/CHANGES.txt
> >>
> >> lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/CoreDescript
> >> or.j
> >> ava
> >>
> >> lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/SolrCore.jav
> >> a
> >>
> >>
> lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/admin/Cor
> >> eA
> >> dminHandler.java
> >>
> >> lucene/dev/trunk/solr/core/src/test/org/apache/solr/handler/admin/Cor
> >> eA
> >> dminHandlerTest.java
> >>     lucene/dev/trunk/solr/test-
> >> framework/src/java/org/apache/solr/SolrTestCaseJ4.java
> >>
> >> Modified: lucene/dev/trunk/solr/CHANGES.txt
> >> URL:
> >>
> http://svn.apache.org/viewvc/lucene/dev/trunk/solr/CHANGES.txt?rev=15
> >> 0
> >> 0284&r1=1500283&r2=1500284&view=diff
> >>
> ==========================================================
> >> ====================
> >> --- lucene/dev/trunk/solr/CHANGES.txt (original)
> >> +++ lucene/dev/trunk/solr/CHANGES.txt Sat Jul  6 14:45:47 2013
> >> @@ -253,6 +253,12 @@ Bug Fixes
> >>
> >>  * SOLR-5000: ManagedIndexSchema doesn't persist uniqueKey tag after
> >> calling addFields
> >>    method. (Jun Ohtani, Steve Rowe)
> >> +
> >> +* SOLR-4982: Creating a core while referencing system properties
> >> +looks like it loses files
> >> +  Actually, instanceDir, config, dataDir and schema are not
> >> +dereferenced properly
> >> +  when creating cores that reference sys vars (e.g. &dataDir=${dir}).
> >> +In the dataDir
> >> +  case in particular this leads to the index being put in a
> >> +directory literally named
> >> +  ${dir} but on restart the sysvar will be properly dereferenced.
> >>
> >>  Optimizations
> >>  ----------------------
> >>
> >> Modified:
> >> lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/CoreDescript
> >> or.j
> >> ava
> >> URL:
> >> http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/
> >> apa
> >>
> che/solr/core/CoreDescriptor.java?rev=1500284&r1=1500283&r2=1500284&v
> >> iew=diff
> >>
> ==========================================================
> >> ====================
> >> ---
> >> lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/CoreDescript
> >> or.j
> >> ava (original)
> >> +++
> lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/CoreDesc
> >> +++ rip tor.java Sat Jul  6 14:45:47 2013
> >> @@ -46,7 +46,7 @@ public class CoreDescriptor {
> >>    public static final String CORE_TRANSIENT = "transient";
> >>    public static final String CORE_NODE_NAME = "coreNodeName";
> >>
> >> -  static final String[] standardPropNames = {
> >> +  public static final String[] standardPropNames = {
> >>        CORE_NAME,
> >>        CORE_CONFIG,
> >>        CORE_INSTDIR,
> >> @@ -65,7 +65,7 @@ public class CoreDescriptor {
> >>    // them individually.
> >>    private Properties coreProperties = new Properties();
> >>
> >> -  //TODO: 5.0 remove this, this is solely a hack for persistence.
> >> +  //TODO: 5.0 remove this, this is solely a hack for persistence.
> >> + And perhaps
> >> creating cores in discovery mode?
> >>    private Properties createdProperties = new Properties();
> >>
> >>    private boolean loadedImplicit = false;
> >>
> >> Modified:
> >> lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/SolrCore.jav
> >> a
> >> URL:
> >> http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/
> >> apa
> >>
> che/solr/core/SolrCore.java?rev=1500284&r1=1500283&r2=1500284&view=di
> >> ff
> >>
> ==========================================================
> >> ====================
> >> ---
> >> lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/SolrCore.jav
> >> a
> >> (original)
> >> +++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/SolrCore
> >> +++ .ja
> >> +++ va Sat Jul  6 14:45:47 2013
> >> @@ -26,6 +26,7 @@ import java.io.Writer;  import
> >> java.lang.reflect.Constructor;  import java.net.URL;  import
> >> java.util.ArrayList;
> >> +import java.util.Arrays;
> >>  import java.util.Collection;
> >>  import java.util.Collections;
> >>  import java.util.Date;
> >> @@ -873,6 +874,18 @@ public final class SolrCore implements S
> >>        propFile.getParentFile().mkdirs();
> >>        Properties props = new Properties();
> >>        props.put("name", cd.getName());
> >> +
> >> +      // This must be being created since there's no file here
> >> + already. So write
> >> out all of the params we were
> >> +      // created with. This _may_ overwrite the name above, but that's OK.
> >> +      Collection<String> stds = new
> >> HashSet(Arrays.asList(CoreDescriptor.standardPropNames));
> >> +      for (String prop : cd.getCreatedProperties().stringPropertyNames()) {
> >> +        // Only preserve things that are legal, and let's just keep
> >> + instDir right out
> >> of the persisted file even
> >> +        // though it's part of the create properties on the URL.
> >> +        if (! CoreDescriptor.CORE_INSTDIR.equals(prop) &&
> >> stds.contains(prop)) {
> >> +          props.put(prop, cd.getCreatedProperties().getProperty(prop));
> >> +        }
> >> +      }
> >> +
> >>        if (cc.isZooKeeperAware()) {
> >>          String collection = cd.getCloudDescriptor().getCollectionName();
> >>          if (collection != null) {
> >>
> >> Modified:
> >>
> lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/admin/Cor
> >> eA
> >> dminHandler.java
> >> URL:
> >> http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/
> >> apa
> >>
> che/solr/handler/admin/CoreAdminHandler.java?rev=1500284&r1=1500283
> >> &r2=1500284&view=diff
> >>
> ==========================================================
> >> ====================
> >> ---
> >>
> lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/admin/Cor
> >> eA
> >> dminHandler.java (original)
> >> +++
> >> lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/admin/Co
> >> +++ reAdminHandler.java Sat Jul  6 14:45:47 2013
> >> @@ -57,6 +57,7 @@ import org.apache.solr.update.UpdateLog;  import
> >> org.apache.solr.update.processor.UpdateRequestProcessor;
> >>  import
> org.apache.solr.update.processor.UpdateRequestProcessorChain;
> >>  import org.apache.solr.util.NumberUtils;
> >> +import org.apache.solr.util.PropertiesUtil;
> >>  import org.apache.solr.util.RefCounted;  import
> >> org.apache.zookeeper.KeeperException;
> >>  import org.slf4j.Logger;
> >> @@ -414,6 +415,7 @@ public class CoreAdminHandler extends Re
> >>        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
> >>                                "Core name is mandatory to CREATE a SolrCore");
> >>      }
> >> +
> >>      CoreDescriptor dcore = null;
> >>      try {
> >>
> >> @@ -427,22 +429,30 @@ public class CoreAdminHandler extends Re
> >>        if (instanceDir == null) {
> >>          // instanceDir = coreContainer.getSolrHome() + "/" + name;
> >>          instanceDir = name; // bare name is already relative to solr
> >> home
> >> +      } else {
> >> +        instanceDir = PropertiesUtil.substituteProperty(instanceDir,
> >> + null);
> >>        }
> >>
> >>        dcore = new CoreDescriptor(coreContainer, name, instanceDir);
> >>
> >>        //  fillup optional parameters
> >>        String opts = params.get(CoreAdminParams.CONFIG);
> >> -      if (opts != null)
> >> +      if (opts != null) {
> >> +        opts = PropertiesUtil.substituteProperty(opts, null);
> >>          dcore.setConfigName(opts);
> >> +      }
> >>
> >>        opts = params.get(CoreAdminParams.SCHEMA);
> >> -      if (opts != null)
> >> +      if (opts != null) {
> >> +        opts = PropertiesUtil.substituteProperty(opts, null);
> >>          dcore.setSchemaName(opts);
> >> +      }
> >>
> >>        opts = params.get(CoreAdminParams.DATA_DIR);
> >> -      if (opts != null)
> >> +      if (opts != null) {
> >> +        opts = PropertiesUtil.substituteProperty(opts, null);
> >>          dcore.setDataDir(opts);
> >> +      }
> >>
> >>        opts = params.get(CoreAdminParams.ULOG_DIR);
> >>        if (opts != null)
> >>
> >> Added:
> >> lucene/dev/trunk/solr/core/src/test/org/apache/solr/handler/admin/Cor
> >> eA
> >> dminCreateDiscoverTest.java
> >> URL:
> >> http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/
> >> apa
> >>
> che/solr/handler/admin/CoreAdminCreateDiscoverTest.java?rev=1500284&
> >> view=auto
> >>
> ==========================================================
> >> ====================
> >> ---
> >> lucene/dev/trunk/solr/core/src/test/org/apache/solr/handler/admin/Cor
> >> eA
> >> dminCreateDiscoverTest.java (added)
> >> +++
> >> lucene/dev/trunk/solr/core/src/test/org/apache/solr/handler/admin/Co
> >> +++ reAdminCreateDiscoverTest.java Sat Jul  6 14:45:47 2013
> >> @@ -0,0 +1,228 @@
> >> +/*
> >> + * 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.
> >> + */
> >> +
> >> +package org.apache.solr.handler.admin;
> >> +
> >> +import org.apache.commons.io.FileUtils; import
> >> +org.apache.solr.SolrTestCaseJ4; import
> >> +org.apache.solr.common.params.CoreAdminParams;
> >> +import org.apache.solr.core.CoreDescriptor;
> >> +import org.apache.solr.core.SolrCoreDiscoverer;
> >> +import org.apache.solr.response.SolrQueryResponse;
> >> +import org.junit.AfterClass;
> >> +import org.junit.BeforeClass;
> >> +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 {
> >> +
> >> +  private static File solrHomeDirectory = null;
> >> +
> >> +  private static CoreAdminHandler admin = null; //  private static
> >> +CoreContainer cc = null;
> >> +
> >> +  private static String coreNormal = "normal";  private static
> >> + String coreSysProps = "sys_props";
> >> +
> >> +  @BeforeClass
> >> +  public static void before() throws Exception {
> >> +    useFactory(null); // I require FS-based indexes for this test.
> >> +
> >> +    solrHomeDirectory = new File(TEMP_DIR, "solrHome/" +
> >> CoreAdminCreateDiscoverTest.getClassName());
> >> +    if (solrHomeDirectory.exists()) {
> >> +      FileUtils.deleteDirectory(solrHomeDirectory);
> >> +    }
> >> +    assertTrue("Failed to mkdirs workDir",
> >> + solrHomeDirectory.mkdirs());
> >> +
> >> +    setupNoCoreTest(solrHomeDirectory, null);
> >> +
> >> +    admin = new CoreAdminHandler(h.getCoreContainer());
> >> +  }
> >> +
> >> +  @AfterClass
> >> +  public static void after() throws Exception {
> >> +    h.close();
> >> +    if (solrHomeDirectory.exists()) {
> >> +      FileUtils.deleteDirectory(solrHomeDirectory);
> >> +    }
> >> +  }
> >> +
> >> +  private static void setupCore(String coreName, boolean blivet)
> >> + throws
> >> IOException {
> >> +    File instDir = new File(solrHomeDirectory, coreName);
> >> +    File subHome = new File(instDir, "conf");
> >> +    assertTrue("Failed to make subdirectory ", subHome.mkdirs());
> >> +
> >> +    // Be sure we pick up sysvars when we create this
> >> +    String srcDir = SolrTestCaseJ4.TEST_HOME() + "/collection1/conf";
> >> +    FileUtils.copyFile(new File(srcDir, "schema-tiny.xml"), new
> >> + File(subHome,
> >> "schema_ren.xml"));
> >> +    FileUtils.copyFile(new File(srcDir, "solrconfig-minimal.xml"),
> >> + new File(subHome, "solrconfig_ren.xml"));
> >> +
> >> +    FileUtils.copyFile(new File(srcDir,
> >> "solrconfig.snippet.randomindexconfig.xml"),
> >> +        new File(subHome,
> >> + "solrconfig.snippet.randomindexconfig.xml"));
> >> +  }
> >> +
> >> +  @Test
> >> +  public void testCreateSavesSysProps() throws Exception {
> >> +
> >> +    setupCore(coreSysProps, true);
> >> +
> >> +    // create a new core (using CoreAdminHandler) w/ properties
> >> +    // Just to be sure its NOT written to the core.properties file
> >> +    File workDir = new File(solrHomeDirectory, coreSysProps);
> >> +    System.setProperty("INSTDIR_TEST", workDir.getAbsolutePath());
> >> +    System.setProperty("CONFIG_TEST", "solrconfig_ren.xml");
> >> +    System.setProperty("SCHEMA_TEST", "schema_ren.xml");
> >> +
> >> +    File dataDir = new File(workDir.getAbsolutePath(), "data_diff");
> >> +    System.setProperty("DATA_TEST", "data_diff");
> >> +
> >> +    SolrQueryResponse resp = new SolrQueryResponse();
> >> +    admin.handleRequestBody
> >> +        (req(CoreAdminParams.ACTION,
> >> +            CoreAdminParams.CoreAdminAction.CREATE.toString(),
> >> +            CoreAdminParams.NAME, coreSysProps,
> >> +            CoreAdminParams.INSTANCE_DIR, "${INSTDIR_TEST}",
> >> +            CoreAdminParams.CONFIG, "${CONFIG_TEST}",
> >> +            CoreAdminParams.SCHEMA, "${SCHEMA_TEST}",
> >> +            CoreAdminParams.DATA_DIR, "${DATA_TEST}"),
> >> +            resp);
> >> +    assertNull("Exception on create", resp.getException());
> >> +
> >> +    // verify props are in persisted file
> >> +
> >> +    Properties props = new Properties();
> >> +    File propFile = new File(solrHomeDirectory, coreSysProps + "/" +
> >> SolrCoreDiscoverer.CORE_PROP_FILE);
> >> +    FileInputStream is = new FileInputStream(propFile);
> >> +    try {
> >> +      props.load(is);
> >> +    } finally {
> >> +      org.apache.commons.io.IOUtils.closeQuietly(is);
> >> +    }
> >> +
> >> +    assertEquals("Unexpected value preserved in properties file " +
> >> propFile.getAbsolutePath(),
> >> +        props.getProperty(CoreAdminParams.NAME), coreSysProps);
> >> +
> >> +    assertEquals("Unexpected value preserved in properties file " +
> >> propFile.getAbsolutePath(),
> >> +        props.getProperty(CoreAdminParams.CONFIG),
> >> + "${CONFIG_TEST}");
> >> +
> >> +    assertEquals("Unexpected value preserved in properties file " +
> >> propFile.getAbsolutePath(),
> >> +        props.getProperty(CoreAdminParams.SCHEMA),
> >> + "${SCHEMA_TEST}");
> >> +
> >> +    assertEquals("Unexpected value preserved in properties file " +
> >> propFile.getAbsolutePath(),
> >> +        props.getProperty(CoreAdminParams.DATA_DIR),
> >> + "${DATA_TEST}");
> >> +
> >> +    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.
> >> +
> >> +    // Should NOT be a datadir named ${DATA_TEST} (literal).
> >> +    File badDir = new File(workDir, "${DATA_TEST}");
> >> +    assertFalse("Should have substituted the sys var, found file " +
> >> + badDir.getAbsolutePath(), badDir.exists());
> >> +
> >> +    // 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}.
> >> +    File test = new File(dataDir, "index");
> >> +    assertTrue("Should have found index dir at " +
> >> + test.getAbsolutePath(),
> >> test.exists());
> >> +    File gen = new File(test, "segments.gen");
> >> +    assertTrue("Should be segments.gen in the dir at " +
> >> + gen.getAbsolutePath(), gen.exists());
> >> +
> >> +  }
> >> +
> >> +  @Test
> >> +  public void testCreateSavesRegProps() throws Exception {
> >> +
> >> +    setupCore(coreNormal, true);
> >> +
> >> +    // create a new core (using CoreAdminHandler) w/ properties
> >> +    // Just to be sure its NOT written to the core.properties file
> >> +    File workDir = new File(solrHomeDirectory, coreNormal);
> >> +    File data = new File(workDir, "data");
> >> +
> >> +    SolrQueryResponse resp = new SolrQueryResponse();
> >> +    admin.handleRequestBody
> >> +        (req(CoreAdminParams.ACTION,
> >> +            CoreAdminParams.CoreAdminAction.CREATE.toString(),
> >> +            CoreAdminParams.NAME, coreNormal,
> >> +            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());
> >> +
> >> +    // verify props are in persisted file
> >> +    Properties props = new Properties();
> >> +    File propFile = new File(solrHomeDirectory, coreNormal + "/" +
> >> SolrCoreDiscoverer.CORE_PROP_FILE);
> >> +    FileInputStream is = new FileInputStream(propFile);
> >> +    try {
> >> +      props.load(is);
> >> +    } finally {
> >> +      org.apache.commons.io.IOUtils.closeQuietly(is);
> >> +    }
> >> +
> >> +    assertEquals("Unexpected value preserved in properties file " +
> >> propFile.getAbsolutePath(),
> >> +        props.getProperty(CoreAdminParams.NAME), coreNormal);
> >> +
> >> +    assertEquals("Unexpected value preserved in properties file " +
> >> propFile.getAbsolutePath(),
> >> +        props.getProperty(CoreAdminParams.CONFIG),
> >> + "solrconfig_ren.xml");
> >> +
> >> +    assertEquals("Unexpected value preserved in properties file " +
> >> propFile.getAbsolutePath(),
> >> +        props.getProperty(CoreAdminParams.SCHEMA),
> >> + "schema_ren.xml");
> >> +
> >> +    assertEquals("Unexpected value preserved in properties file " +
> >> propFile.getAbsolutePath(),
> >> +        props.getProperty(CoreAdminParams.DATA_DIR),
> >> + data.getAbsolutePath());
> >> +
> >> +    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}.
> >> +    File test = new File(data, "index");
> >> +    assertTrue("Should have found index dir at " +
> >> + test.getAbsolutePath(),
> >> test.exists());
> >> +    File gen = new File(test, "segments.gen");
> >> +    assertTrue("Should be segments.gen in the dir at " +
> >> + gen.getAbsolutePath(), gen.exists());
> >> +
> >> +  }
> >> +
> >> +  // 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/Cor
> >> eA
> >> dminHandlerTest.java
> >> URL:
> >> http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/
> >> apa
> >>
> che/solr/handler/admin/CoreAdminHandlerTest.java?rev=1500284&r1=1500
> >> 283&r2=1500284&view=diff
> >>
> ==========================================================
> >> ====================
> >> ---
> >> lucene/dev/trunk/solr/core/src/test/org/apache/solr/handler/admin/Cor
> >> eA
> >> dminHandlerTest.java (original)
> >> +++
> >> lucene/dev/trunk/solr/core/src/test/org/apache/solr/handler/admin/Co
> >> +++ reAdminHandlerTest.java Sat Jul  6 14:45:47 2013
> >> @@ -17,9 +17,9 @@
> >>
> >>  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.solr.handler.admin.CoreAdminHandler;
> >>  import org.apache.solr.common.SolrException;
> >>  import org.apache.solr.common.params.CoreAdminParams;
> >>  import org.apache.solr.common.util.NamedList;
> >> @@ -28,15 +28,14 @@ import org.apache.solr.SolrTestCaseJ4;
> >>
> >>  import java.util.Map;
> >>  import java.io.File;
> >> -import java.io.IOException;
> >> -
> >> -import javax.xml.xpath.XPathExpressionException;
> >>
> >>  import org.apache.commons.io.FileUtils;
> >>
> >>  import org.junit.BeforeClass;
> >> +import org.junit.Rule;
> >>  import org.junit.Test;
> >> -import org.xml.sax.SAXException;
> >> +import org.junit.rules.RuleChain;
> >> +import org.junit.rules.TestRule;
> >>
> >>  public class CoreAdminHandlerTest extends SolrTestCaseJ4 {
> >>
> >> @@ -44,7 +43,93 @@ public class CoreAdminHandlerTest extend
> >>    public static void beforeClass() throws Exception {
> >>      initCore("solrconfig.xml", "schema.xml");
> >>    }
> >> -
> >> +
> >> +  @Rule
> >> +  public TestRule solrTestRules = RuleChain.outerRule(new
> >> + SystemPropertiesRestoreRule());
> >> +
> >> +  public String getCoreName() { return this.getClass().getName() +
> >> + "_sys_vars"; }
> >> +
> >> +  @Test
> >> +  public void testCreateWithSysVars() throws Exception {
> >> +    useFactory(null); // I require FS-based indexes for this test.
> >> +
> >> +    final File workDir = new File(TEMP_DIR, getCoreName());
> >> +
> >> +    if (workDir.exists()) {
> >> +      FileUtils.deleteDirectory(workDir);
> >> +    }
> >> +    assertTrue("Failed to mkdirs workDir", workDir.mkdirs());
> >> +    String coreName = "with_sys_vars";
> >> +    File instDir = new File(workDir, coreName);
> >> +    File subHome = new File(instDir, "conf");
> >> +    assertTrue("Failed to make subdirectory ", subHome.mkdirs());
> >> +
> >> +    // Be sure we pick up sysvars when we create this
> >> +    String srcDir = SolrTestCaseJ4.TEST_HOME() + "/collection1/conf";
> >> +    FileUtils.copyFile(new File(srcDir, "schema-tiny.xml"), new
> >> + File(subHome,
> >> "schema_ren.xml"));
> >> +    FileUtils.copyFile(new File(srcDir, "solrconfig-minimal.xml"),
> >> + new
> >> File(subHome, "solrconfig_ren.xml"));
> >> +    FileUtils.copyFile(new File(srcDir,
> >> "solrconfig.snippet.randomindexconfig.xml"),
> >> +        new File(subHome,
> >> + "solrconfig.snippet.randomindexconfig.xml"));
> >> +
> >> +    final CoreContainer cores = h.getCoreContainer();
> >> +    cores.setPersistent(false); // we'll do this explicitly as
> >> + needed
> >> +
> >> +    final CoreAdminHandler admin = new CoreAdminHandler(cores);
> >> +
> >> +    // create a new core (using CoreAdminHandler) w/ properties
> >> +    System.setProperty("INSTDIR_TEST", instDir.getAbsolutePath());
> >> +    System.setProperty("CONFIG_TEST", "solrconfig_ren.xml");
> >> +    System.setProperty("SCHEMA_TEST", "schema_ren.xml");
> >> +
> >> +    File dataDir = new File(workDir.getAbsolutePath(), "data_diff");
> >> +    System.setProperty("DATA_TEST", dataDir.getAbsolutePath());
> >> +
> >> +    SolrQueryResponse resp = new SolrQueryResponse();
> >> +    admin.handleRequestBody
> >> +        (req(CoreAdminParams.ACTION,
> >> +            CoreAdminParams.CoreAdminAction.CREATE.toString(),
> >> +            CoreAdminParams.NAME, getCoreName(),
> >> +            CoreAdminParams.INSTANCE_DIR, "${INSTDIR_TEST}",
> >> +            CoreAdminParams.CONFIG, "${CONFIG_TEST}",
> >> +            CoreAdminParams.SCHEMA, "${SCHEMA_TEST}",
> >> +            CoreAdminParams.DATA_DIR, "${DATA_TEST}"),
> >> +            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
> >> +            ,"/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}']"
> >> +            ,"/solr/cores/core[@name='" + getCoreName() + "' and
> >> @config='${CONFIG_TEST}']"
> >> +        );
> >> +
> >> +    // Now assert that certain values are properly dereferenced in
> >> + the
> >> process of creating the core, see
> >> +    // SOLR-4982.
> >> +
> >> +    // Should NOT be a datadir named ${DATA_TEST} (literal). This is
> >> + the bug
> >> after all
> >> +    File badDir = new File(instDir, "${DATA_TEST}");
> >> +    assertFalse("Should have substituted the sys var, found file " +
> >> + badDir.getAbsolutePath(), badDir.exists());
> >> +
> >> +    // For the other 3 vars, we couldn't get past creating the core
> >> + fi
> >> dereferencing didn't work correctly.
> >> +
> >> +    // Should have segments in the directory pointed to by the
> >> ${DATA_TEST}.
> >> +    File test = new File(dataDir, "index");
> >> +    assertTrue("Should have found index dir at " +
> >> + test.getAbsolutePath(),
> >> test.exists());
> >> +    test = new File(test,"segments.gen");
> >> +    assertTrue("Should have found segments.gen at " +
> >> + test.getAbsolutePath(), test.exists());
> >> +
> >> +    // Cleanup
> >> +    FileUtils.deleteDirectory(workDir);
> >> +
> >> +  }
> >> +
> >>    @Test
> >>    public void testCoreAdminHandler() throws Exception {
> >>      final File workDir = new File(TEMP_DIR,
> >> this.getClass().getName()); @@ -
> >> 141,6 +226,8 @@ public class CoreAdminHandlerTest extend
> >>
> >>      // :TODO: because of SOLR-3665 we can't ask for status from all
> >> cores
> >>
> >> -  }
> >> +    // cleanup
> >> +    FileUtils.deleteDirectory(workDir);
> >>
> >> +  }
> >>  }
> >>
> >> 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=1500284&r1
> >> =
> >> 1500283&r2=1500284&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/Sol
> >> +++ rTe stCaseJ4.java Sat Jul  6 14:45:47 2013
> >> @@ -21,6 +21,7 @@ import com.carrotsearch.randomizedtestin  import
> >> com.carrotsearch.randomizedtesting.annotations.ThreadLeakFilters;
> >>  import
> >> com.carrotsearch.randomizedtesting.rules.SystemPropertiesRestoreRule;
> >>  import org.apache.commons.io.FileUtils;
> >> +import org.apache.lucene.util.IOUtils;
> >>  import org.apache.lucene.util.LuceneTestCase;
> >>  import org.apache.lucene.util.QuickPatchThreadsFilter;
> >>  import org.apache.solr.client.solrj.util.ClientUtils;
> >> @@ -32,9 +33,11 @@ import org.apache.solr.common.params.Com
> import
> >> org.apache.solr.common.params.ModifiableSolrParams;
> >>  import org.apache.solr.common.params.SolrParams;
> >>  import org.apache.solr.common.util.XML;
> >> +import org.apache.solr.core.ConfigSolr;
> >>  import org.apache.solr.core.CoreContainer;
> >>  import org.apache.solr.core.SolrConfig;  import
> >> org.apache.solr.core.SolrCore;
> >> +import org.apache.solr.core.SolrResourceLoader;
> >>  import org.apache.solr.handler.JsonUpdateRequestHandler;
> >>  import org.apache.solr.request.LocalSolrQueryRequest;
> >>  import org.apache.solr.request.SolrQueryRequest;
> >> @@ -155,6 +158,25 @@ public abstract class SolrTestCaseJ4 ext
> >>      }
> >>    }
> >>
> >> +  /**
> >> +   * Call this from @BeforeClass to set up the test harness and
> >> + update
> >> handler with no cores.
> >> +   *
> >> +   * @param solrHome The solr home directory.
> >> +   * @param xmlStr - the text of an XML file to use. If null, use
> >> + the what's the
> >> absolute minimal file.
> >> +   * @throws Exception Lost of file-type things can go wrong.
> >> +   */
> >> +  public static void setupNoCoreTest(File solrHome, String xmlStr)
> >> + throws Exception {
> >> +
> >> +    File tmpFile = new File(solrHome, ConfigSolr.SOLR_XML_FILE);
> >> +    if (xmlStr == null) {
> >> +      xmlStr = "<solr></solr>";
> >> +    }
> >> +    FileUtils.write(tmpFile, xmlStr,
> >> + IOUtils.CHARSET_UTF_8.toString());
> >> +
> >> +    SolrResourceLoader loader = new
> >> SolrResourceLoader(solrHome.getAbsolutePath());
> >> +    h = new TestHarness(loader, ConfigSolr.fromFile(loader, new
> >> File(solrHome, "solr.xml")));
> >> +    lrf = h.getRequestFactory("standard", 0, 20,
> >> + CommonParams.VERSION, "2.2");  }
> >>
> >>    @Override
> >>    public void setUp() throws Exception {
> >
> >
> >
> > ---------------------------------------------------------------------
> > To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org For
> > additional commands, e-mail: dev-help@lucene.apache.org
> >
> 
> ---------------------------------------------------------------------
> To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org For additional
> commands, e-mail: dev-help@lucene.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org


Re: svn commit: r1500284 - in /lucene/dev/trunk/solr: ./ core/src/java/org/apache/solr/core/ core/src/java/org/apache/solr/handler/admin/ core/src/test/org/apache/solr/handler/admin/ test-framework/src/java/org/apache/solr/

Posted by Erick Erickson <er...@gmail.com>.
OK, I was able to reproduce this on my machine and checked in what
appears to be a fix. But then I thought I'd handled this case before,
so we'll see what happens overnight.

On Sat, Jul 6, 2013 at 7:10 PM, Uwe Schindler <uw...@thetaphi.de> wrote:
> This seems to cause a memory leak, see recently failing Jenkins tests!
>
> -----
> Uwe Schindler
> H.-H.-Meier-Allee 63, D-28213 Bremen
> http://www.thetaphi.de
> eMail: uwe@thetaphi.de
>
>
>> -----Original Message-----
>> From: erick@apache.org [mailto:erick@apache.org]
>> Sent: Saturday, July 06, 2013 4:46 PM
>> To: commits@lucene.apache.org
>> Subject: svn commit: r1500284 - in /lucene/dev/trunk/solr: ./
>> core/src/java/org/apache/solr/core/
>> core/src/java/org/apache/solr/handler/admin/
>> core/src/test/org/apache/solr/handler/admin/ test-
>> framework/src/java/org/apache/solr/
>>
>> Author: erick
>> Date: Sat Jul  6 14:45:47 2013
>> New Revision: 1500284
>>
>> URL: http://svn.apache.org/r1500284
>> Log:
>> Fix for SOLR-4982, creating cores with sysprops does not dereference them
>> properly
>>
>> Added:
>>
>> lucene/dev/trunk/solr/core/src/test/org/apache/solr/handler/admin/CoreA
>> dminCreateDiscoverTest.java   (with props)
>> Modified:
>>     lucene/dev/trunk/solr/CHANGES.txt
>>
>> lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/CoreDescriptor.j
>> ava
>>     lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/SolrCore.java
>>
>> lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/admin/CoreA
>> dminHandler.java
>>
>> lucene/dev/trunk/solr/core/src/test/org/apache/solr/handler/admin/CoreA
>> dminHandlerTest.java
>>     lucene/dev/trunk/solr/test-
>> framework/src/java/org/apache/solr/SolrTestCaseJ4.java
>>
>> Modified: lucene/dev/trunk/solr/CHANGES.txt
>> URL:
>> http://svn.apache.org/viewvc/lucene/dev/trunk/solr/CHANGES.txt?rev=150
>> 0284&r1=1500283&r2=1500284&view=diff
>> ==========================================================
>> ====================
>> --- lucene/dev/trunk/solr/CHANGES.txt (original)
>> +++ lucene/dev/trunk/solr/CHANGES.txt Sat Jul  6 14:45:47 2013
>> @@ -253,6 +253,12 @@ Bug Fixes
>>
>>  * SOLR-5000: ManagedIndexSchema doesn't persist uniqueKey tag after
>> calling addFields
>>    method. (Jun Ohtani, Steve Rowe)
>> +
>> +* SOLR-4982: Creating a core while referencing system properties looks
>> +like it loses files
>> +  Actually, instanceDir, config, dataDir and schema are not
>> +dereferenced properly
>> +  when creating cores that reference sys vars (e.g. &dataDir=${dir}).
>> +In the dataDir
>> +  case in particular this leads to the index being put in a directory
>> +literally named
>> +  ${dir} but on restart the sysvar will be properly dereferenced.
>>
>>  Optimizations
>>  ----------------------
>>
>> Modified:
>> lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/CoreDescriptor.j
>> ava
>> URL:
>> http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apa
>> che/solr/core/CoreDescriptor.java?rev=1500284&r1=1500283&r2=1500284&v
>> iew=diff
>> ==========================================================
>> ====================
>> ---
>> lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/CoreDescriptor.j
>> ava (original)
>> +++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/CoreDescrip
>> +++ tor.java Sat Jul  6 14:45:47 2013
>> @@ -46,7 +46,7 @@ public class CoreDescriptor {
>>    public static final String CORE_TRANSIENT = "transient";
>>    public static final String CORE_NODE_NAME = "coreNodeName";
>>
>> -  static final String[] standardPropNames = {
>> +  public static final String[] standardPropNames = {
>>        CORE_NAME,
>>        CORE_CONFIG,
>>        CORE_INSTDIR,
>> @@ -65,7 +65,7 @@ public class CoreDescriptor {
>>    // them individually.
>>    private Properties coreProperties = new Properties();
>>
>> -  //TODO: 5.0 remove this, this is solely a hack for persistence.
>> +  //TODO: 5.0 remove this, this is solely a hack for persistence. And perhaps
>> creating cores in discovery mode?
>>    private Properties createdProperties = new Properties();
>>
>>    private boolean loadedImplicit = false;
>>
>> Modified:
>> lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/SolrCore.java
>> URL:
>> http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apa
>> che/solr/core/SolrCore.java?rev=1500284&r1=1500283&r2=1500284&view=di
>> ff
>> ==========================================================
>> ====================
>> --- lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/SolrCore.java
>> (original)
>> +++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/SolrCore.ja
>> +++ va Sat Jul  6 14:45:47 2013
>> @@ -26,6 +26,7 @@ import java.io.Writer;  import
>> java.lang.reflect.Constructor;  import java.net.URL;  import java.util.ArrayList;
>> +import java.util.Arrays;
>>  import java.util.Collection;
>>  import java.util.Collections;
>>  import java.util.Date;
>> @@ -873,6 +874,18 @@ public final class SolrCore implements S
>>        propFile.getParentFile().mkdirs();
>>        Properties props = new Properties();
>>        props.put("name", cd.getName());
>> +
>> +      // This must be being created since there's no file here already. So write
>> out all of the params we were
>> +      // created with. This _may_ overwrite the name above, but that's OK.
>> +      Collection<String> stds = new
>> HashSet(Arrays.asList(CoreDescriptor.standardPropNames));
>> +      for (String prop : cd.getCreatedProperties().stringPropertyNames()) {
>> +        // Only preserve things that are legal, and let's just keep instDir right out
>> of the persisted file even
>> +        // though it's part of the create properties on the URL.
>> +        if (! CoreDescriptor.CORE_INSTDIR.equals(prop) &&
>> stds.contains(prop)) {
>> +          props.put(prop, cd.getCreatedProperties().getProperty(prop));
>> +        }
>> +      }
>> +
>>        if (cc.isZooKeeperAware()) {
>>          String collection = cd.getCloudDescriptor().getCollectionName();
>>          if (collection != null) {
>>
>> Modified:
>> lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/admin/CoreA
>> dminHandler.java
>> URL:
>> http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apa
>> che/solr/handler/admin/CoreAdminHandler.java?rev=1500284&r1=1500283
>> &r2=1500284&view=diff
>> ==========================================================
>> ====================
>> ---
>> lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/admin/CoreA
>> dminHandler.java (original)
>> +++
>> lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/admin/Co
>> +++ reAdminHandler.java Sat Jul  6 14:45:47 2013
>> @@ -57,6 +57,7 @@ import org.apache.solr.update.UpdateLog;  import
>> org.apache.solr.update.processor.UpdateRequestProcessor;
>>  import org.apache.solr.update.processor.UpdateRequestProcessorChain;
>>  import org.apache.solr.util.NumberUtils;
>> +import org.apache.solr.util.PropertiesUtil;
>>  import org.apache.solr.util.RefCounted;  import
>> org.apache.zookeeper.KeeperException;
>>  import org.slf4j.Logger;
>> @@ -414,6 +415,7 @@ public class CoreAdminHandler extends Re
>>        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
>>                                "Core name is mandatory to CREATE a SolrCore");
>>      }
>> +
>>      CoreDescriptor dcore = null;
>>      try {
>>
>> @@ -427,22 +429,30 @@ public class CoreAdminHandler extends Re
>>        if (instanceDir == null) {
>>          // instanceDir = coreContainer.getSolrHome() + "/" + name;
>>          instanceDir = name; // bare name is already relative to solr home
>> +      } else {
>> +        instanceDir = PropertiesUtil.substituteProperty(instanceDir,
>> + null);
>>        }
>>
>>        dcore = new CoreDescriptor(coreContainer, name, instanceDir);
>>
>>        //  fillup optional parameters
>>        String opts = params.get(CoreAdminParams.CONFIG);
>> -      if (opts != null)
>> +      if (opts != null) {
>> +        opts = PropertiesUtil.substituteProperty(opts, null);
>>          dcore.setConfigName(opts);
>> +      }
>>
>>        opts = params.get(CoreAdminParams.SCHEMA);
>> -      if (opts != null)
>> +      if (opts != null) {
>> +        opts = PropertiesUtil.substituteProperty(opts, null);
>>          dcore.setSchemaName(opts);
>> +      }
>>
>>        opts = params.get(CoreAdminParams.DATA_DIR);
>> -      if (opts != null)
>> +      if (opts != null) {
>> +        opts = PropertiesUtil.substituteProperty(opts, null);
>>          dcore.setDataDir(opts);
>> +      }
>>
>>        opts = params.get(CoreAdminParams.ULOG_DIR);
>>        if (opts != null)
>>
>> Added:
>> lucene/dev/trunk/solr/core/src/test/org/apache/solr/handler/admin/CoreA
>> dminCreateDiscoverTest.java
>> URL:
>> http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apa
>> che/solr/handler/admin/CoreAdminCreateDiscoverTest.java?rev=1500284&
>> view=auto
>> ==========================================================
>> ====================
>> ---
>> lucene/dev/trunk/solr/core/src/test/org/apache/solr/handler/admin/CoreA
>> dminCreateDiscoverTest.java (added)
>> +++
>> lucene/dev/trunk/solr/core/src/test/org/apache/solr/handler/admin/Co
>> +++ reAdminCreateDiscoverTest.java Sat Jul  6 14:45:47 2013
>> @@ -0,0 +1,228 @@
>> +/*
>> + * 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.
>> + */
>> +
>> +package org.apache.solr.handler.admin;
>> +
>> +import org.apache.commons.io.FileUtils; import
>> +org.apache.solr.SolrTestCaseJ4; import
>> +org.apache.solr.common.params.CoreAdminParams;
>> +import org.apache.solr.core.CoreDescriptor;
>> +import org.apache.solr.core.SolrCoreDiscoverer;
>> +import org.apache.solr.response.SolrQueryResponse;
>> +import org.junit.AfterClass;
>> +import org.junit.BeforeClass;
>> +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 {
>> +
>> +  private static File solrHomeDirectory = null;
>> +
>> +  private static CoreAdminHandler admin = null; //  private static
>> +CoreContainer cc = null;
>> +
>> +  private static String coreNormal = "normal";  private static String
>> + coreSysProps = "sys_props";
>> +
>> +  @BeforeClass
>> +  public static void before() throws Exception {
>> +    useFactory(null); // I require FS-based indexes for this test.
>> +
>> +    solrHomeDirectory = new File(TEMP_DIR, "solrHome/" +
>> CoreAdminCreateDiscoverTest.getClassName());
>> +    if (solrHomeDirectory.exists()) {
>> +      FileUtils.deleteDirectory(solrHomeDirectory);
>> +    }
>> +    assertTrue("Failed to mkdirs workDir", solrHomeDirectory.mkdirs());
>> +
>> +    setupNoCoreTest(solrHomeDirectory, null);
>> +
>> +    admin = new CoreAdminHandler(h.getCoreContainer());
>> +  }
>> +
>> +  @AfterClass
>> +  public static void after() throws Exception {
>> +    h.close();
>> +    if (solrHomeDirectory.exists()) {
>> +      FileUtils.deleteDirectory(solrHomeDirectory);
>> +    }
>> +  }
>> +
>> +  private static void setupCore(String coreName, boolean blivet) throws
>> IOException {
>> +    File instDir = new File(solrHomeDirectory, coreName);
>> +    File subHome = new File(instDir, "conf");
>> +    assertTrue("Failed to make subdirectory ", subHome.mkdirs());
>> +
>> +    // Be sure we pick up sysvars when we create this
>> +    String srcDir = SolrTestCaseJ4.TEST_HOME() + "/collection1/conf";
>> +    FileUtils.copyFile(new File(srcDir, "schema-tiny.xml"), new File(subHome,
>> "schema_ren.xml"));
>> +    FileUtils.copyFile(new File(srcDir, "solrconfig-minimal.xml"), new
>> + File(subHome, "solrconfig_ren.xml"));
>> +
>> +    FileUtils.copyFile(new File(srcDir,
>> "solrconfig.snippet.randomindexconfig.xml"),
>> +        new File(subHome, "solrconfig.snippet.randomindexconfig.xml"));
>> +  }
>> +
>> +  @Test
>> +  public void testCreateSavesSysProps() throws Exception {
>> +
>> +    setupCore(coreSysProps, true);
>> +
>> +    // create a new core (using CoreAdminHandler) w/ properties
>> +    // Just to be sure its NOT written to the core.properties file
>> +    File workDir = new File(solrHomeDirectory, coreSysProps);
>> +    System.setProperty("INSTDIR_TEST", workDir.getAbsolutePath());
>> +    System.setProperty("CONFIG_TEST", "solrconfig_ren.xml");
>> +    System.setProperty("SCHEMA_TEST", "schema_ren.xml");
>> +
>> +    File dataDir = new File(workDir.getAbsolutePath(), "data_diff");
>> +    System.setProperty("DATA_TEST", "data_diff");
>> +
>> +    SolrQueryResponse resp = new SolrQueryResponse();
>> +    admin.handleRequestBody
>> +        (req(CoreAdminParams.ACTION,
>> +            CoreAdminParams.CoreAdminAction.CREATE.toString(),
>> +            CoreAdminParams.NAME, coreSysProps,
>> +            CoreAdminParams.INSTANCE_DIR, "${INSTDIR_TEST}",
>> +            CoreAdminParams.CONFIG, "${CONFIG_TEST}",
>> +            CoreAdminParams.SCHEMA, "${SCHEMA_TEST}",
>> +            CoreAdminParams.DATA_DIR, "${DATA_TEST}"),
>> +            resp);
>> +    assertNull("Exception on create", resp.getException());
>> +
>> +    // verify props are in persisted file
>> +
>> +    Properties props = new Properties();
>> +    File propFile = new File(solrHomeDirectory, coreSysProps + "/" +
>> SolrCoreDiscoverer.CORE_PROP_FILE);
>> +    FileInputStream is = new FileInputStream(propFile);
>> +    try {
>> +      props.load(is);
>> +    } finally {
>> +      org.apache.commons.io.IOUtils.closeQuietly(is);
>> +    }
>> +
>> +    assertEquals("Unexpected value preserved in properties file " +
>> propFile.getAbsolutePath(),
>> +        props.getProperty(CoreAdminParams.NAME), coreSysProps);
>> +
>> +    assertEquals("Unexpected value preserved in properties file " +
>> propFile.getAbsolutePath(),
>> +        props.getProperty(CoreAdminParams.CONFIG), "${CONFIG_TEST}");
>> +
>> +    assertEquals("Unexpected value preserved in properties file " +
>> propFile.getAbsolutePath(),
>> +        props.getProperty(CoreAdminParams.SCHEMA), "${SCHEMA_TEST}");
>> +
>> +    assertEquals("Unexpected value preserved in properties file " +
>> propFile.getAbsolutePath(),
>> +        props.getProperty(CoreAdminParams.DATA_DIR), "${DATA_TEST}");
>> +
>> +    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.
>> +
>> +    // Should NOT be a datadir named ${DATA_TEST} (literal).
>> +    File badDir = new File(workDir, "${DATA_TEST}");
>> +    assertFalse("Should have substituted the sys var, found file " +
>> + badDir.getAbsolutePath(), badDir.exists());
>> +
>> +    // 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}.
>> +    File test = new File(dataDir, "index");
>> +    assertTrue("Should have found index dir at " + test.getAbsolutePath(),
>> test.exists());
>> +    File gen = new File(test, "segments.gen");
>> +    assertTrue("Should be segments.gen in the dir at " +
>> + gen.getAbsolutePath(), gen.exists());
>> +
>> +  }
>> +
>> +  @Test
>> +  public void testCreateSavesRegProps() throws Exception {
>> +
>> +    setupCore(coreNormal, true);
>> +
>> +    // create a new core (using CoreAdminHandler) w/ properties
>> +    // Just to be sure its NOT written to the core.properties file
>> +    File workDir = new File(solrHomeDirectory, coreNormal);
>> +    File data = new File(workDir, "data");
>> +
>> +    SolrQueryResponse resp = new SolrQueryResponse();
>> +    admin.handleRequestBody
>> +        (req(CoreAdminParams.ACTION,
>> +            CoreAdminParams.CoreAdminAction.CREATE.toString(),
>> +            CoreAdminParams.NAME, coreNormal,
>> +            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());
>> +
>> +    // verify props are in persisted file
>> +    Properties props = new Properties();
>> +    File propFile = new File(solrHomeDirectory, coreNormal + "/" +
>> SolrCoreDiscoverer.CORE_PROP_FILE);
>> +    FileInputStream is = new FileInputStream(propFile);
>> +    try {
>> +      props.load(is);
>> +    } finally {
>> +      org.apache.commons.io.IOUtils.closeQuietly(is);
>> +    }
>> +
>> +    assertEquals("Unexpected value preserved in properties file " +
>> propFile.getAbsolutePath(),
>> +        props.getProperty(CoreAdminParams.NAME), coreNormal);
>> +
>> +    assertEquals("Unexpected value preserved in properties file " +
>> propFile.getAbsolutePath(),
>> +        props.getProperty(CoreAdminParams.CONFIG),
>> + "solrconfig_ren.xml");
>> +
>> +    assertEquals("Unexpected value preserved in properties file " +
>> propFile.getAbsolutePath(),
>> +        props.getProperty(CoreAdminParams.SCHEMA), "schema_ren.xml");
>> +
>> +    assertEquals("Unexpected value preserved in properties file " +
>> propFile.getAbsolutePath(),
>> +        props.getProperty(CoreAdminParams.DATA_DIR),
>> + data.getAbsolutePath());
>> +
>> +    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}.
>> +    File test = new File(data, "index");
>> +    assertTrue("Should have found index dir at " + test.getAbsolutePath(),
>> test.exists());
>> +    File gen = new File(test, "segments.gen");
>> +    assertTrue("Should be segments.gen in the dir at " +
>> + gen.getAbsolutePath(), gen.exists());
>> +
>> +  }
>> +
>> +  // 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/CoreA
>> dminHandlerTest.java
>> URL:
>> http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apa
>> che/solr/handler/admin/CoreAdminHandlerTest.java?rev=1500284&r1=1500
>> 283&r2=1500284&view=diff
>> ==========================================================
>> ====================
>> ---
>> lucene/dev/trunk/solr/core/src/test/org/apache/solr/handler/admin/CoreA
>> dminHandlerTest.java (original)
>> +++
>> lucene/dev/trunk/solr/core/src/test/org/apache/solr/handler/admin/Co
>> +++ reAdminHandlerTest.java Sat Jul  6 14:45:47 2013
>> @@ -17,9 +17,9 @@
>>
>>  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.solr.handler.admin.CoreAdminHandler;
>>  import org.apache.solr.common.SolrException;
>>  import org.apache.solr.common.params.CoreAdminParams;
>>  import org.apache.solr.common.util.NamedList;
>> @@ -28,15 +28,14 @@ import org.apache.solr.SolrTestCaseJ4;
>>
>>  import java.util.Map;
>>  import java.io.File;
>> -import java.io.IOException;
>> -
>> -import javax.xml.xpath.XPathExpressionException;
>>
>>  import org.apache.commons.io.FileUtils;
>>
>>  import org.junit.BeforeClass;
>> +import org.junit.Rule;
>>  import org.junit.Test;
>> -import org.xml.sax.SAXException;
>> +import org.junit.rules.RuleChain;
>> +import org.junit.rules.TestRule;
>>
>>  public class CoreAdminHandlerTest extends SolrTestCaseJ4 {
>>
>> @@ -44,7 +43,93 @@ public class CoreAdminHandlerTest extend
>>    public static void beforeClass() throws Exception {
>>      initCore("solrconfig.xml", "schema.xml");
>>    }
>> -
>> +
>> +  @Rule
>> +  public TestRule solrTestRules = RuleChain.outerRule(new
>> + SystemPropertiesRestoreRule());
>> +
>> +  public String getCoreName() { return this.getClass().getName() +
>> + "_sys_vars"; }
>> +
>> +  @Test
>> +  public void testCreateWithSysVars() throws Exception {
>> +    useFactory(null); // I require FS-based indexes for this test.
>> +
>> +    final File workDir = new File(TEMP_DIR, getCoreName());
>> +
>> +    if (workDir.exists()) {
>> +      FileUtils.deleteDirectory(workDir);
>> +    }
>> +    assertTrue("Failed to mkdirs workDir", workDir.mkdirs());
>> +    String coreName = "with_sys_vars";
>> +    File instDir = new File(workDir, coreName);
>> +    File subHome = new File(instDir, "conf");
>> +    assertTrue("Failed to make subdirectory ", subHome.mkdirs());
>> +
>> +    // Be sure we pick up sysvars when we create this
>> +    String srcDir = SolrTestCaseJ4.TEST_HOME() + "/collection1/conf";
>> +    FileUtils.copyFile(new File(srcDir, "schema-tiny.xml"), new File(subHome,
>> "schema_ren.xml"));
>> +    FileUtils.copyFile(new File(srcDir, "solrconfig-minimal.xml"), new
>> File(subHome, "solrconfig_ren.xml"));
>> +    FileUtils.copyFile(new File(srcDir,
>> "solrconfig.snippet.randomindexconfig.xml"),
>> +        new File(subHome, "solrconfig.snippet.randomindexconfig.xml"));
>> +
>> +    final CoreContainer cores = h.getCoreContainer();
>> +    cores.setPersistent(false); // we'll do this explicitly as needed
>> +
>> +    final CoreAdminHandler admin = new CoreAdminHandler(cores);
>> +
>> +    // create a new core (using CoreAdminHandler) w/ properties
>> +    System.setProperty("INSTDIR_TEST", instDir.getAbsolutePath());
>> +    System.setProperty("CONFIG_TEST", "solrconfig_ren.xml");
>> +    System.setProperty("SCHEMA_TEST", "schema_ren.xml");
>> +
>> +    File dataDir = new File(workDir.getAbsolutePath(), "data_diff");
>> +    System.setProperty("DATA_TEST", dataDir.getAbsolutePath());
>> +
>> +    SolrQueryResponse resp = new SolrQueryResponse();
>> +    admin.handleRequestBody
>> +        (req(CoreAdminParams.ACTION,
>> +            CoreAdminParams.CoreAdminAction.CREATE.toString(),
>> +            CoreAdminParams.NAME, getCoreName(),
>> +            CoreAdminParams.INSTANCE_DIR, "${INSTDIR_TEST}",
>> +            CoreAdminParams.CONFIG, "${CONFIG_TEST}",
>> +            CoreAdminParams.SCHEMA, "${SCHEMA_TEST}",
>> +            CoreAdminParams.DATA_DIR, "${DATA_TEST}"),
>> +            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
>> +            ,"/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}']"
>> +            ,"/solr/cores/core[@name='" + getCoreName() + "' and
>> @config='${CONFIG_TEST}']"
>> +        );
>> +
>> +    // Now assert that certain values are properly dereferenced in the
>> process of creating the core, see
>> +    // SOLR-4982.
>> +
>> +    // Should NOT be a datadir named ${DATA_TEST} (literal). This is the bug
>> after all
>> +    File badDir = new File(instDir, "${DATA_TEST}");
>> +    assertFalse("Should have substituted the sys var, found file " +
>> + badDir.getAbsolutePath(), badDir.exists());
>> +
>> +    // For the other 3 vars, we couldn't get past creating the core fi
>> dereferencing didn't work correctly.
>> +
>> +    // Should have segments in the directory pointed to by the
>> ${DATA_TEST}.
>> +    File test = new File(dataDir, "index");
>> +    assertTrue("Should have found index dir at " + test.getAbsolutePath(),
>> test.exists());
>> +    test = new File(test,"segments.gen");
>> +    assertTrue("Should have found segments.gen at " +
>> + test.getAbsolutePath(), test.exists());
>> +
>> +    // Cleanup
>> +    FileUtils.deleteDirectory(workDir);
>> +
>> +  }
>> +
>>    @Test
>>    public void testCoreAdminHandler() throws Exception {
>>      final File workDir = new File(TEMP_DIR, this.getClass().getName()); @@ -
>> 141,6 +226,8 @@ public class CoreAdminHandlerTest extend
>>
>>      // :TODO: because of SOLR-3665 we can't ask for status from all cores
>>
>> -  }
>> +    // cleanup
>> +    FileUtils.deleteDirectory(workDir);
>>
>> +  }
>>  }
>>
>> 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=1500284&r1=
>> 1500283&r2=1500284&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/SolrTe
>> +++ stCaseJ4.java Sat Jul  6 14:45:47 2013
>> @@ -21,6 +21,7 @@ import com.carrotsearch.randomizedtestin  import
>> com.carrotsearch.randomizedtesting.annotations.ThreadLeakFilters;
>>  import
>> com.carrotsearch.randomizedtesting.rules.SystemPropertiesRestoreRule;
>>  import org.apache.commons.io.FileUtils;
>> +import org.apache.lucene.util.IOUtils;
>>  import org.apache.lucene.util.LuceneTestCase;
>>  import org.apache.lucene.util.QuickPatchThreadsFilter;
>>  import org.apache.solr.client.solrj.util.ClientUtils;
>> @@ -32,9 +33,11 @@ import org.apache.solr.common.params.Com  import
>> org.apache.solr.common.params.ModifiableSolrParams;
>>  import org.apache.solr.common.params.SolrParams;
>>  import org.apache.solr.common.util.XML;
>> +import org.apache.solr.core.ConfigSolr;
>>  import org.apache.solr.core.CoreContainer;
>>  import org.apache.solr.core.SolrConfig;  import
>> org.apache.solr.core.SolrCore;
>> +import org.apache.solr.core.SolrResourceLoader;
>>  import org.apache.solr.handler.JsonUpdateRequestHandler;
>>  import org.apache.solr.request.LocalSolrQueryRequest;
>>  import org.apache.solr.request.SolrQueryRequest;
>> @@ -155,6 +158,25 @@ public abstract class SolrTestCaseJ4 ext
>>      }
>>    }
>>
>> +  /**
>> +   * Call this from @BeforeClass to set up the test harness and update
>> handler with no cores.
>> +   *
>> +   * @param solrHome The solr home directory.
>> +   * @param xmlStr - the text of an XML file to use. If null, use the what's the
>> absolute minimal file.
>> +   * @throws Exception Lost of file-type things can go wrong.
>> +   */
>> +  public static void setupNoCoreTest(File solrHome, String xmlStr)
>> + throws Exception {
>> +
>> +    File tmpFile = new File(solrHome, ConfigSolr.SOLR_XML_FILE);
>> +    if (xmlStr == null) {
>> +      xmlStr = "<solr></solr>";
>> +    }
>> +    FileUtils.write(tmpFile, xmlStr, IOUtils.CHARSET_UTF_8.toString());
>> +
>> +    SolrResourceLoader loader = new
>> SolrResourceLoader(solrHome.getAbsolutePath());
>> +    h = new TestHarness(loader, ConfigSolr.fromFile(loader, new
>> File(solrHome, "solr.xml")));
>> +    lrf = h.getRequestFactory("standard", 0, 20, CommonParams.VERSION,
>> + "2.2");  }
>>
>>    @Override
>>    public void setUp() throws Exception {
>
>
>
> ---------------------------------------------------------------------
> To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
> For additional commands, e-mail: dev-help@lucene.apache.org
>

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org


Re: svn commit: r1500284 - in /lucene/dev/trunk/solr: ./ core/src/java/org/apache/solr/core/ core/src/java/org/apache/solr/handler/admin/ core/src/test/org/apache/solr/handler/admin/ test-framework/src/java/org/apache/solr/

Posted by Erick Erickson <er...@gmail.com>.
Yeah, just saw this and starting to look. I might have a clue, we'll see.

Erick

On Sat, Jul 6, 2013 at 7:10 PM, Uwe Schindler <uw...@thetaphi.de> wrote:
> This seems to cause a memory leak, see recently failing Jenkins tests!
>
> -----
> Uwe Schindler
> H.-H.-Meier-Allee 63, D-28213 Bremen
> http://www.thetaphi.de
> eMail: uwe@thetaphi.de
>
>
>> -----Original Message-----
>> From: erick@apache.org [mailto:erick@apache.org]
>> Sent: Saturday, July 06, 2013 4:46 PM
>> To: commits@lucene.apache.org
>> Subject: svn commit: r1500284 - in /lucene/dev/trunk/solr: ./
>> core/src/java/org/apache/solr/core/
>> core/src/java/org/apache/solr/handler/admin/
>> core/src/test/org/apache/solr/handler/admin/ test-
>> framework/src/java/org/apache/solr/
>>
>> Author: erick
>> Date: Sat Jul  6 14:45:47 2013
>> New Revision: 1500284
>>
>> URL: http://svn.apache.org/r1500284
>> Log:
>> Fix for SOLR-4982, creating cores with sysprops does not dereference them
>> properly
>>
>> Added:
>>
>> lucene/dev/trunk/solr/core/src/test/org/apache/solr/handler/admin/CoreA
>> dminCreateDiscoverTest.java   (with props)
>> Modified:
>>     lucene/dev/trunk/solr/CHANGES.txt
>>
>> lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/CoreDescriptor.j
>> ava
>>     lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/SolrCore.java
>>
>> lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/admin/CoreA
>> dminHandler.java
>>
>> lucene/dev/trunk/solr/core/src/test/org/apache/solr/handler/admin/CoreA
>> dminHandlerTest.java
>>     lucene/dev/trunk/solr/test-
>> framework/src/java/org/apache/solr/SolrTestCaseJ4.java
>>
>> Modified: lucene/dev/trunk/solr/CHANGES.txt
>> URL:
>> http://svn.apache.org/viewvc/lucene/dev/trunk/solr/CHANGES.txt?rev=150
>> 0284&r1=1500283&r2=1500284&view=diff
>> ==========================================================
>> ====================
>> --- lucene/dev/trunk/solr/CHANGES.txt (original)
>> +++ lucene/dev/trunk/solr/CHANGES.txt Sat Jul  6 14:45:47 2013
>> @@ -253,6 +253,12 @@ Bug Fixes
>>
>>  * SOLR-5000: ManagedIndexSchema doesn't persist uniqueKey tag after
>> calling addFields
>>    method. (Jun Ohtani, Steve Rowe)
>> +
>> +* SOLR-4982: Creating a core while referencing system properties looks
>> +like it loses files
>> +  Actually, instanceDir, config, dataDir and schema are not
>> +dereferenced properly
>> +  when creating cores that reference sys vars (e.g. &dataDir=${dir}).
>> +In the dataDir
>> +  case in particular this leads to the index being put in a directory
>> +literally named
>> +  ${dir} but on restart the sysvar will be properly dereferenced.
>>
>>  Optimizations
>>  ----------------------
>>
>> Modified:
>> lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/CoreDescriptor.j
>> ava
>> URL:
>> http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apa
>> che/solr/core/CoreDescriptor.java?rev=1500284&r1=1500283&r2=1500284&v
>> iew=diff
>> ==========================================================
>> ====================
>> ---
>> lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/CoreDescriptor.j
>> ava (original)
>> +++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/CoreDescrip
>> +++ tor.java Sat Jul  6 14:45:47 2013
>> @@ -46,7 +46,7 @@ public class CoreDescriptor {
>>    public static final String CORE_TRANSIENT = "transient";
>>    public static final String CORE_NODE_NAME = "coreNodeName";
>>
>> -  static final String[] standardPropNames = {
>> +  public static final String[] standardPropNames = {
>>        CORE_NAME,
>>        CORE_CONFIG,
>>        CORE_INSTDIR,
>> @@ -65,7 +65,7 @@ public class CoreDescriptor {
>>    // them individually.
>>    private Properties coreProperties = new Properties();
>>
>> -  //TODO: 5.0 remove this, this is solely a hack for persistence.
>> +  //TODO: 5.0 remove this, this is solely a hack for persistence. And perhaps
>> creating cores in discovery mode?
>>    private Properties createdProperties = new Properties();
>>
>>    private boolean loadedImplicit = false;
>>
>> Modified:
>> lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/SolrCore.java
>> URL:
>> http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apa
>> che/solr/core/SolrCore.java?rev=1500284&r1=1500283&r2=1500284&view=di
>> ff
>> ==========================================================
>> ====================
>> --- lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/SolrCore.java
>> (original)
>> +++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/SolrCore.ja
>> +++ va Sat Jul  6 14:45:47 2013
>> @@ -26,6 +26,7 @@ import java.io.Writer;  import
>> java.lang.reflect.Constructor;  import java.net.URL;  import java.util.ArrayList;
>> +import java.util.Arrays;
>>  import java.util.Collection;
>>  import java.util.Collections;
>>  import java.util.Date;
>> @@ -873,6 +874,18 @@ public final class SolrCore implements S
>>        propFile.getParentFile().mkdirs();
>>        Properties props = new Properties();
>>        props.put("name", cd.getName());
>> +
>> +      // This must be being created since there's no file here already. So write
>> out all of the params we were
>> +      // created with. This _may_ overwrite the name above, but that's OK.
>> +      Collection<String> stds = new
>> HashSet(Arrays.asList(CoreDescriptor.standardPropNames));
>> +      for (String prop : cd.getCreatedProperties().stringPropertyNames()) {
>> +        // Only preserve things that are legal, and let's just keep instDir right out
>> of the persisted file even
>> +        // though it's part of the create properties on the URL.
>> +        if (! CoreDescriptor.CORE_INSTDIR.equals(prop) &&
>> stds.contains(prop)) {
>> +          props.put(prop, cd.getCreatedProperties().getProperty(prop));
>> +        }
>> +      }
>> +
>>        if (cc.isZooKeeperAware()) {
>>          String collection = cd.getCloudDescriptor().getCollectionName();
>>          if (collection != null) {
>>
>> Modified:
>> lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/admin/CoreA
>> dminHandler.java
>> URL:
>> http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apa
>> che/solr/handler/admin/CoreAdminHandler.java?rev=1500284&r1=1500283
>> &r2=1500284&view=diff
>> ==========================================================
>> ====================
>> ---
>> lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/admin/CoreA
>> dminHandler.java (original)
>> +++
>> lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/admin/Co
>> +++ reAdminHandler.java Sat Jul  6 14:45:47 2013
>> @@ -57,6 +57,7 @@ import org.apache.solr.update.UpdateLog;  import
>> org.apache.solr.update.processor.UpdateRequestProcessor;
>>  import org.apache.solr.update.processor.UpdateRequestProcessorChain;
>>  import org.apache.solr.util.NumberUtils;
>> +import org.apache.solr.util.PropertiesUtil;
>>  import org.apache.solr.util.RefCounted;  import
>> org.apache.zookeeper.KeeperException;
>>  import org.slf4j.Logger;
>> @@ -414,6 +415,7 @@ public class CoreAdminHandler extends Re
>>        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
>>                                "Core name is mandatory to CREATE a SolrCore");
>>      }
>> +
>>      CoreDescriptor dcore = null;
>>      try {
>>
>> @@ -427,22 +429,30 @@ public class CoreAdminHandler extends Re
>>        if (instanceDir == null) {
>>          // instanceDir = coreContainer.getSolrHome() + "/" + name;
>>          instanceDir = name; // bare name is already relative to solr home
>> +      } else {
>> +        instanceDir = PropertiesUtil.substituteProperty(instanceDir,
>> + null);
>>        }
>>
>>        dcore = new CoreDescriptor(coreContainer, name, instanceDir);
>>
>>        //  fillup optional parameters
>>        String opts = params.get(CoreAdminParams.CONFIG);
>> -      if (opts != null)
>> +      if (opts != null) {
>> +        opts = PropertiesUtil.substituteProperty(opts, null);
>>          dcore.setConfigName(opts);
>> +      }
>>
>>        opts = params.get(CoreAdminParams.SCHEMA);
>> -      if (opts != null)
>> +      if (opts != null) {
>> +        opts = PropertiesUtil.substituteProperty(opts, null);
>>          dcore.setSchemaName(opts);
>> +      }
>>
>>        opts = params.get(CoreAdminParams.DATA_DIR);
>> -      if (opts != null)
>> +      if (opts != null) {
>> +        opts = PropertiesUtil.substituteProperty(opts, null);
>>          dcore.setDataDir(opts);
>> +      }
>>
>>        opts = params.get(CoreAdminParams.ULOG_DIR);
>>        if (opts != null)
>>
>> Added:
>> lucene/dev/trunk/solr/core/src/test/org/apache/solr/handler/admin/CoreA
>> dminCreateDiscoverTest.java
>> URL:
>> http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apa
>> che/solr/handler/admin/CoreAdminCreateDiscoverTest.java?rev=1500284&
>> view=auto
>> ==========================================================
>> ====================
>> ---
>> lucene/dev/trunk/solr/core/src/test/org/apache/solr/handler/admin/CoreA
>> dminCreateDiscoverTest.java (added)
>> +++
>> lucene/dev/trunk/solr/core/src/test/org/apache/solr/handler/admin/Co
>> +++ reAdminCreateDiscoverTest.java Sat Jul  6 14:45:47 2013
>> @@ -0,0 +1,228 @@
>> +/*
>> + * 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.
>> + */
>> +
>> +package org.apache.solr.handler.admin;
>> +
>> +import org.apache.commons.io.FileUtils; import
>> +org.apache.solr.SolrTestCaseJ4; import
>> +org.apache.solr.common.params.CoreAdminParams;
>> +import org.apache.solr.core.CoreDescriptor;
>> +import org.apache.solr.core.SolrCoreDiscoverer;
>> +import org.apache.solr.response.SolrQueryResponse;
>> +import org.junit.AfterClass;
>> +import org.junit.BeforeClass;
>> +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 {
>> +
>> +  private static File solrHomeDirectory = null;
>> +
>> +  private static CoreAdminHandler admin = null; //  private static
>> +CoreContainer cc = null;
>> +
>> +  private static String coreNormal = "normal";  private static String
>> + coreSysProps = "sys_props";
>> +
>> +  @BeforeClass
>> +  public static void before() throws Exception {
>> +    useFactory(null); // I require FS-based indexes for this test.
>> +
>> +    solrHomeDirectory = new File(TEMP_DIR, "solrHome/" +
>> CoreAdminCreateDiscoverTest.getClassName());
>> +    if (solrHomeDirectory.exists()) {
>> +      FileUtils.deleteDirectory(solrHomeDirectory);
>> +    }
>> +    assertTrue("Failed to mkdirs workDir", solrHomeDirectory.mkdirs());
>> +
>> +    setupNoCoreTest(solrHomeDirectory, null);
>> +
>> +    admin = new CoreAdminHandler(h.getCoreContainer());
>> +  }
>> +
>> +  @AfterClass
>> +  public static void after() throws Exception {
>> +    h.close();
>> +    if (solrHomeDirectory.exists()) {
>> +      FileUtils.deleteDirectory(solrHomeDirectory);
>> +    }
>> +  }
>> +
>> +  private static void setupCore(String coreName, boolean blivet) throws
>> IOException {
>> +    File instDir = new File(solrHomeDirectory, coreName);
>> +    File subHome = new File(instDir, "conf");
>> +    assertTrue("Failed to make subdirectory ", subHome.mkdirs());
>> +
>> +    // Be sure we pick up sysvars when we create this
>> +    String srcDir = SolrTestCaseJ4.TEST_HOME() + "/collection1/conf";
>> +    FileUtils.copyFile(new File(srcDir, "schema-tiny.xml"), new File(subHome,
>> "schema_ren.xml"));
>> +    FileUtils.copyFile(new File(srcDir, "solrconfig-minimal.xml"), new
>> + File(subHome, "solrconfig_ren.xml"));
>> +
>> +    FileUtils.copyFile(new File(srcDir,
>> "solrconfig.snippet.randomindexconfig.xml"),
>> +        new File(subHome, "solrconfig.snippet.randomindexconfig.xml"));
>> +  }
>> +
>> +  @Test
>> +  public void testCreateSavesSysProps() throws Exception {
>> +
>> +    setupCore(coreSysProps, true);
>> +
>> +    // create a new core (using CoreAdminHandler) w/ properties
>> +    // Just to be sure its NOT written to the core.properties file
>> +    File workDir = new File(solrHomeDirectory, coreSysProps);
>> +    System.setProperty("INSTDIR_TEST", workDir.getAbsolutePath());
>> +    System.setProperty("CONFIG_TEST", "solrconfig_ren.xml");
>> +    System.setProperty("SCHEMA_TEST", "schema_ren.xml");
>> +
>> +    File dataDir = new File(workDir.getAbsolutePath(), "data_diff");
>> +    System.setProperty("DATA_TEST", "data_diff");
>> +
>> +    SolrQueryResponse resp = new SolrQueryResponse();
>> +    admin.handleRequestBody
>> +        (req(CoreAdminParams.ACTION,
>> +            CoreAdminParams.CoreAdminAction.CREATE.toString(),
>> +            CoreAdminParams.NAME, coreSysProps,
>> +            CoreAdminParams.INSTANCE_DIR, "${INSTDIR_TEST}",
>> +            CoreAdminParams.CONFIG, "${CONFIG_TEST}",
>> +            CoreAdminParams.SCHEMA, "${SCHEMA_TEST}",
>> +            CoreAdminParams.DATA_DIR, "${DATA_TEST}"),
>> +            resp);
>> +    assertNull("Exception on create", resp.getException());
>> +
>> +    // verify props are in persisted file
>> +
>> +    Properties props = new Properties();
>> +    File propFile = new File(solrHomeDirectory, coreSysProps + "/" +
>> SolrCoreDiscoverer.CORE_PROP_FILE);
>> +    FileInputStream is = new FileInputStream(propFile);
>> +    try {
>> +      props.load(is);
>> +    } finally {
>> +      org.apache.commons.io.IOUtils.closeQuietly(is);
>> +    }
>> +
>> +    assertEquals("Unexpected value preserved in properties file " +
>> propFile.getAbsolutePath(),
>> +        props.getProperty(CoreAdminParams.NAME), coreSysProps);
>> +
>> +    assertEquals("Unexpected value preserved in properties file " +
>> propFile.getAbsolutePath(),
>> +        props.getProperty(CoreAdminParams.CONFIG), "${CONFIG_TEST}");
>> +
>> +    assertEquals("Unexpected value preserved in properties file " +
>> propFile.getAbsolutePath(),
>> +        props.getProperty(CoreAdminParams.SCHEMA), "${SCHEMA_TEST}");
>> +
>> +    assertEquals("Unexpected value preserved in properties file " +
>> propFile.getAbsolutePath(),
>> +        props.getProperty(CoreAdminParams.DATA_DIR), "${DATA_TEST}");
>> +
>> +    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.
>> +
>> +    // Should NOT be a datadir named ${DATA_TEST} (literal).
>> +    File badDir = new File(workDir, "${DATA_TEST}");
>> +    assertFalse("Should have substituted the sys var, found file " +
>> + badDir.getAbsolutePath(), badDir.exists());
>> +
>> +    // 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}.
>> +    File test = new File(dataDir, "index");
>> +    assertTrue("Should have found index dir at " + test.getAbsolutePath(),
>> test.exists());
>> +    File gen = new File(test, "segments.gen");
>> +    assertTrue("Should be segments.gen in the dir at " +
>> + gen.getAbsolutePath(), gen.exists());
>> +
>> +  }
>> +
>> +  @Test
>> +  public void testCreateSavesRegProps() throws Exception {
>> +
>> +    setupCore(coreNormal, true);
>> +
>> +    // create a new core (using CoreAdminHandler) w/ properties
>> +    // Just to be sure its NOT written to the core.properties file
>> +    File workDir = new File(solrHomeDirectory, coreNormal);
>> +    File data = new File(workDir, "data");
>> +
>> +    SolrQueryResponse resp = new SolrQueryResponse();
>> +    admin.handleRequestBody
>> +        (req(CoreAdminParams.ACTION,
>> +            CoreAdminParams.CoreAdminAction.CREATE.toString(),
>> +            CoreAdminParams.NAME, coreNormal,
>> +            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());
>> +
>> +    // verify props are in persisted file
>> +    Properties props = new Properties();
>> +    File propFile = new File(solrHomeDirectory, coreNormal + "/" +
>> SolrCoreDiscoverer.CORE_PROP_FILE);
>> +    FileInputStream is = new FileInputStream(propFile);
>> +    try {
>> +      props.load(is);
>> +    } finally {
>> +      org.apache.commons.io.IOUtils.closeQuietly(is);
>> +    }
>> +
>> +    assertEquals("Unexpected value preserved in properties file " +
>> propFile.getAbsolutePath(),
>> +        props.getProperty(CoreAdminParams.NAME), coreNormal);
>> +
>> +    assertEquals("Unexpected value preserved in properties file " +
>> propFile.getAbsolutePath(),
>> +        props.getProperty(CoreAdminParams.CONFIG),
>> + "solrconfig_ren.xml");
>> +
>> +    assertEquals("Unexpected value preserved in properties file " +
>> propFile.getAbsolutePath(),
>> +        props.getProperty(CoreAdminParams.SCHEMA), "schema_ren.xml");
>> +
>> +    assertEquals("Unexpected value preserved in properties file " +
>> propFile.getAbsolutePath(),
>> +        props.getProperty(CoreAdminParams.DATA_DIR),
>> + data.getAbsolutePath());
>> +
>> +    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}.
>> +    File test = new File(data, "index");
>> +    assertTrue("Should have found index dir at " + test.getAbsolutePath(),
>> test.exists());
>> +    File gen = new File(test, "segments.gen");
>> +    assertTrue("Should be segments.gen in the dir at " +
>> + gen.getAbsolutePath(), gen.exists());
>> +
>> +  }
>> +
>> +  // 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/CoreA
>> dminHandlerTest.java
>> URL:
>> http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apa
>> che/solr/handler/admin/CoreAdminHandlerTest.java?rev=1500284&r1=1500
>> 283&r2=1500284&view=diff
>> ==========================================================
>> ====================
>> ---
>> lucene/dev/trunk/solr/core/src/test/org/apache/solr/handler/admin/CoreA
>> dminHandlerTest.java (original)
>> +++
>> lucene/dev/trunk/solr/core/src/test/org/apache/solr/handler/admin/Co
>> +++ reAdminHandlerTest.java Sat Jul  6 14:45:47 2013
>> @@ -17,9 +17,9 @@
>>
>>  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.solr.handler.admin.CoreAdminHandler;
>>  import org.apache.solr.common.SolrException;
>>  import org.apache.solr.common.params.CoreAdminParams;
>>  import org.apache.solr.common.util.NamedList;
>> @@ -28,15 +28,14 @@ import org.apache.solr.SolrTestCaseJ4;
>>
>>  import java.util.Map;
>>  import java.io.File;
>> -import java.io.IOException;
>> -
>> -import javax.xml.xpath.XPathExpressionException;
>>
>>  import org.apache.commons.io.FileUtils;
>>
>>  import org.junit.BeforeClass;
>> +import org.junit.Rule;
>>  import org.junit.Test;
>> -import org.xml.sax.SAXException;
>> +import org.junit.rules.RuleChain;
>> +import org.junit.rules.TestRule;
>>
>>  public class CoreAdminHandlerTest extends SolrTestCaseJ4 {
>>
>> @@ -44,7 +43,93 @@ public class CoreAdminHandlerTest extend
>>    public static void beforeClass() throws Exception {
>>      initCore("solrconfig.xml", "schema.xml");
>>    }
>> -
>> +
>> +  @Rule
>> +  public TestRule solrTestRules = RuleChain.outerRule(new
>> + SystemPropertiesRestoreRule());
>> +
>> +  public String getCoreName() { return this.getClass().getName() +
>> + "_sys_vars"; }
>> +
>> +  @Test
>> +  public void testCreateWithSysVars() throws Exception {
>> +    useFactory(null); // I require FS-based indexes for this test.
>> +
>> +    final File workDir = new File(TEMP_DIR, getCoreName());
>> +
>> +    if (workDir.exists()) {
>> +      FileUtils.deleteDirectory(workDir);
>> +    }
>> +    assertTrue("Failed to mkdirs workDir", workDir.mkdirs());
>> +    String coreName = "with_sys_vars";
>> +    File instDir = new File(workDir, coreName);
>> +    File subHome = new File(instDir, "conf");
>> +    assertTrue("Failed to make subdirectory ", subHome.mkdirs());
>> +
>> +    // Be sure we pick up sysvars when we create this
>> +    String srcDir = SolrTestCaseJ4.TEST_HOME() + "/collection1/conf";
>> +    FileUtils.copyFile(new File(srcDir, "schema-tiny.xml"), new File(subHome,
>> "schema_ren.xml"));
>> +    FileUtils.copyFile(new File(srcDir, "solrconfig-minimal.xml"), new
>> File(subHome, "solrconfig_ren.xml"));
>> +    FileUtils.copyFile(new File(srcDir,
>> "solrconfig.snippet.randomindexconfig.xml"),
>> +        new File(subHome, "solrconfig.snippet.randomindexconfig.xml"));
>> +
>> +    final CoreContainer cores = h.getCoreContainer();
>> +    cores.setPersistent(false); // we'll do this explicitly as needed
>> +
>> +    final CoreAdminHandler admin = new CoreAdminHandler(cores);
>> +
>> +    // create a new core (using CoreAdminHandler) w/ properties
>> +    System.setProperty("INSTDIR_TEST", instDir.getAbsolutePath());
>> +    System.setProperty("CONFIG_TEST", "solrconfig_ren.xml");
>> +    System.setProperty("SCHEMA_TEST", "schema_ren.xml");
>> +
>> +    File dataDir = new File(workDir.getAbsolutePath(), "data_diff");
>> +    System.setProperty("DATA_TEST", dataDir.getAbsolutePath());
>> +
>> +    SolrQueryResponse resp = new SolrQueryResponse();
>> +    admin.handleRequestBody
>> +        (req(CoreAdminParams.ACTION,
>> +            CoreAdminParams.CoreAdminAction.CREATE.toString(),
>> +            CoreAdminParams.NAME, getCoreName(),
>> +            CoreAdminParams.INSTANCE_DIR, "${INSTDIR_TEST}",
>> +            CoreAdminParams.CONFIG, "${CONFIG_TEST}",
>> +            CoreAdminParams.SCHEMA, "${SCHEMA_TEST}",
>> +            CoreAdminParams.DATA_DIR, "${DATA_TEST}"),
>> +            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
>> +            ,"/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}']"
>> +            ,"/solr/cores/core[@name='" + getCoreName() + "' and
>> @config='${CONFIG_TEST}']"
>> +        );
>> +
>> +    // Now assert that certain values are properly dereferenced in the
>> process of creating the core, see
>> +    // SOLR-4982.
>> +
>> +    // Should NOT be a datadir named ${DATA_TEST} (literal). This is the bug
>> after all
>> +    File badDir = new File(instDir, "${DATA_TEST}");
>> +    assertFalse("Should have substituted the sys var, found file " +
>> + badDir.getAbsolutePath(), badDir.exists());
>> +
>> +    // For the other 3 vars, we couldn't get past creating the core fi
>> dereferencing didn't work correctly.
>> +
>> +    // Should have segments in the directory pointed to by the
>> ${DATA_TEST}.
>> +    File test = new File(dataDir, "index");
>> +    assertTrue("Should have found index dir at " + test.getAbsolutePath(),
>> test.exists());
>> +    test = new File(test,"segments.gen");
>> +    assertTrue("Should have found segments.gen at " +
>> + test.getAbsolutePath(), test.exists());
>> +
>> +    // Cleanup
>> +    FileUtils.deleteDirectory(workDir);
>> +
>> +  }
>> +
>>    @Test
>>    public void testCoreAdminHandler() throws Exception {
>>      final File workDir = new File(TEMP_DIR, this.getClass().getName()); @@ -
>> 141,6 +226,8 @@ public class CoreAdminHandlerTest extend
>>
>>      // :TODO: because of SOLR-3665 we can't ask for status from all cores
>>
>> -  }
>> +    // cleanup
>> +    FileUtils.deleteDirectory(workDir);
>>
>> +  }
>>  }
>>
>> 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=1500284&r1=
>> 1500283&r2=1500284&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/SolrTe
>> +++ stCaseJ4.java Sat Jul  6 14:45:47 2013
>> @@ -21,6 +21,7 @@ import com.carrotsearch.randomizedtestin  import
>> com.carrotsearch.randomizedtesting.annotations.ThreadLeakFilters;
>>  import
>> com.carrotsearch.randomizedtesting.rules.SystemPropertiesRestoreRule;
>>  import org.apache.commons.io.FileUtils;
>> +import org.apache.lucene.util.IOUtils;
>>  import org.apache.lucene.util.LuceneTestCase;
>>  import org.apache.lucene.util.QuickPatchThreadsFilter;
>>  import org.apache.solr.client.solrj.util.ClientUtils;
>> @@ -32,9 +33,11 @@ import org.apache.solr.common.params.Com  import
>> org.apache.solr.common.params.ModifiableSolrParams;
>>  import org.apache.solr.common.params.SolrParams;
>>  import org.apache.solr.common.util.XML;
>> +import org.apache.solr.core.ConfigSolr;
>>  import org.apache.solr.core.CoreContainer;
>>  import org.apache.solr.core.SolrConfig;  import
>> org.apache.solr.core.SolrCore;
>> +import org.apache.solr.core.SolrResourceLoader;
>>  import org.apache.solr.handler.JsonUpdateRequestHandler;
>>  import org.apache.solr.request.LocalSolrQueryRequest;
>>  import org.apache.solr.request.SolrQueryRequest;
>> @@ -155,6 +158,25 @@ public abstract class SolrTestCaseJ4 ext
>>      }
>>    }
>>
>> +  /**
>> +   * Call this from @BeforeClass to set up the test harness and update
>> handler with no cores.
>> +   *
>> +   * @param solrHome The solr home directory.
>> +   * @param xmlStr - the text of an XML file to use. If null, use the what's the
>> absolute minimal file.
>> +   * @throws Exception Lost of file-type things can go wrong.
>> +   */
>> +  public static void setupNoCoreTest(File solrHome, String xmlStr)
>> + throws Exception {
>> +
>> +    File tmpFile = new File(solrHome, ConfigSolr.SOLR_XML_FILE);
>> +    if (xmlStr == null) {
>> +      xmlStr = "<solr></solr>";
>> +    }
>> +    FileUtils.write(tmpFile, xmlStr, IOUtils.CHARSET_UTF_8.toString());
>> +
>> +    SolrResourceLoader loader = new
>> SolrResourceLoader(solrHome.getAbsolutePath());
>> +    h = new TestHarness(loader, ConfigSolr.fromFile(loader, new
>> File(solrHome, "solr.xml")));
>> +    lrf = h.getRequestFactory("standard", 0, 20, CommonParams.VERSION,
>> + "2.2");  }
>>
>>    @Override
>>    public void setUp() throws Exception {
>
>
>
> ---------------------------------------------------------------------
> To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
> For additional commands, e-mail: dev-help@lucene.apache.org
>

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org


RE: svn commit: r1500284 - in /lucene/dev/trunk/solr: ./ core/src/java/org/apache/solr/core/ core/src/java/org/apache/solr/handler/admin/ core/src/test/org/apache/solr/handler/admin/ test-framework/src/java/org/apache/solr/

Posted by Uwe Schindler <uw...@thetaphi.de>.
This seems to cause a memory leak, see recently failing Jenkins tests!

-----
Uwe Schindler
H.-H.-Meier-Allee 63, D-28213 Bremen
http://www.thetaphi.de
eMail: uwe@thetaphi.de


> -----Original Message-----
> From: erick@apache.org [mailto:erick@apache.org]
> Sent: Saturday, July 06, 2013 4:46 PM
> To: commits@lucene.apache.org
> Subject: svn commit: r1500284 - in /lucene/dev/trunk/solr: ./
> core/src/java/org/apache/solr/core/
> core/src/java/org/apache/solr/handler/admin/
> core/src/test/org/apache/solr/handler/admin/ test-
> framework/src/java/org/apache/solr/
> 
> Author: erick
> Date: Sat Jul  6 14:45:47 2013
> New Revision: 1500284
> 
> URL: http://svn.apache.org/r1500284
> Log:
> Fix for SOLR-4982, creating cores with sysprops does not dereference them
> properly
> 
> Added:
> 
> lucene/dev/trunk/solr/core/src/test/org/apache/solr/handler/admin/CoreA
> dminCreateDiscoverTest.java   (with props)
> Modified:
>     lucene/dev/trunk/solr/CHANGES.txt
> 
> lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/CoreDescriptor.j
> ava
>     lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/SolrCore.java
> 
> lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/admin/CoreA
> dminHandler.java
> 
> lucene/dev/trunk/solr/core/src/test/org/apache/solr/handler/admin/CoreA
> dminHandlerTest.java
>     lucene/dev/trunk/solr/test-
> framework/src/java/org/apache/solr/SolrTestCaseJ4.java
> 
> Modified: lucene/dev/trunk/solr/CHANGES.txt
> URL:
> http://svn.apache.org/viewvc/lucene/dev/trunk/solr/CHANGES.txt?rev=150
> 0284&r1=1500283&r2=1500284&view=diff
> ==========================================================
> ====================
> --- lucene/dev/trunk/solr/CHANGES.txt (original)
> +++ lucene/dev/trunk/solr/CHANGES.txt Sat Jul  6 14:45:47 2013
> @@ -253,6 +253,12 @@ Bug Fixes
> 
>  * SOLR-5000: ManagedIndexSchema doesn't persist uniqueKey tag after
> calling addFields
>    method. (Jun Ohtani, Steve Rowe)
> +
> +* SOLR-4982: Creating a core while referencing system properties looks
> +like it loses files
> +  Actually, instanceDir, config, dataDir and schema are not
> +dereferenced properly
> +  when creating cores that reference sys vars (e.g. &dataDir=${dir}).
> +In the dataDir
> +  case in particular this leads to the index being put in a directory
> +literally named
> +  ${dir} but on restart the sysvar will be properly dereferenced.
> 
>  Optimizations
>  ----------------------
> 
> Modified:
> lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/CoreDescriptor.j
> ava
> URL:
> http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apa
> che/solr/core/CoreDescriptor.java?rev=1500284&r1=1500283&r2=1500284&v
> iew=diff
> ==========================================================
> ====================
> ---
> lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/CoreDescriptor.j
> ava (original)
> +++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/CoreDescrip
> +++ tor.java Sat Jul  6 14:45:47 2013
> @@ -46,7 +46,7 @@ public class CoreDescriptor {
>    public static final String CORE_TRANSIENT = "transient";
>    public static final String CORE_NODE_NAME = "coreNodeName";
> 
> -  static final String[] standardPropNames = {
> +  public static final String[] standardPropNames = {
>        CORE_NAME,
>        CORE_CONFIG,
>        CORE_INSTDIR,
> @@ -65,7 +65,7 @@ public class CoreDescriptor {
>    // them individually.
>    private Properties coreProperties = new Properties();
> 
> -  //TODO: 5.0 remove this, this is solely a hack for persistence.
> +  //TODO: 5.0 remove this, this is solely a hack for persistence. And perhaps
> creating cores in discovery mode?
>    private Properties createdProperties = new Properties();
> 
>    private boolean loadedImplicit = false;
> 
> Modified:
> lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/SolrCore.java
> URL:
> http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apa
> che/solr/core/SolrCore.java?rev=1500284&r1=1500283&r2=1500284&view=di
> ff
> ==========================================================
> ====================
> --- lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/SolrCore.java
> (original)
> +++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/SolrCore.ja
> +++ va Sat Jul  6 14:45:47 2013
> @@ -26,6 +26,7 @@ import java.io.Writer;  import
> java.lang.reflect.Constructor;  import java.net.URL;  import java.util.ArrayList;
> +import java.util.Arrays;
>  import java.util.Collection;
>  import java.util.Collections;
>  import java.util.Date;
> @@ -873,6 +874,18 @@ public final class SolrCore implements S
>        propFile.getParentFile().mkdirs();
>        Properties props = new Properties();
>        props.put("name", cd.getName());
> +
> +      // This must be being created since there's no file here already. So write
> out all of the params we were
> +      // created with. This _may_ overwrite the name above, but that's OK.
> +      Collection<String> stds = new
> HashSet(Arrays.asList(CoreDescriptor.standardPropNames));
> +      for (String prop : cd.getCreatedProperties().stringPropertyNames()) {
> +        // Only preserve things that are legal, and let's just keep instDir right out
> of the persisted file even
> +        // though it's part of the create properties on the URL.
> +        if (! CoreDescriptor.CORE_INSTDIR.equals(prop) &&
> stds.contains(prop)) {
> +          props.put(prop, cd.getCreatedProperties().getProperty(prop));
> +        }
> +      }
> +
>        if (cc.isZooKeeperAware()) {
>          String collection = cd.getCloudDescriptor().getCollectionName();
>          if (collection != null) {
> 
> Modified:
> lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/admin/CoreA
> dminHandler.java
> URL:
> http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apa
> che/solr/handler/admin/CoreAdminHandler.java?rev=1500284&r1=1500283
> &r2=1500284&view=diff
> ==========================================================
> ====================
> ---
> lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/admin/CoreA
> dminHandler.java (original)
> +++
> lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/admin/Co
> +++ reAdminHandler.java Sat Jul  6 14:45:47 2013
> @@ -57,6 +57,7 @@ import org.apache.solr.update.UpdateLog;  import
> org.apache.solr.update.processor.UpdateRequestProcessor;
>  import org.apache.solr.update.processor.UpdateRequestProcessorChain;
>  import org.apache.solr.util.NumberUtils;
> +import org.apache.solr.util.PropertiesUtil;
>  import org.apache.solr.util.RefCounted;  import
> org.apache.zookeeper.KeeperException;
>  import org.slf4j.Logger;
> @@ -414,6 +415,7 @@ public class CoreAdminHandler extends Re
>        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
>                                "Core name is mandatory to CREATE a SolrCore");
>      }
> +
>      CoreDescriptor dcore = null;
>      try {
> 
> @@ -427,22 +429,30 @@ public class CoreAdminHandler extends Re
>        if (instanceDir == null) {
>          // instanceDir = coreContainer.getSolrHome() + "/" + name;
>          instanceDir = name; // bare name is already relative to solr home
> +      } else {
> +        instanceDir = PropertiesUtil.substituteProperty(instanceDir,
> + null);
>        }
> 
>        dcore = new CoreDescriptor(coreContainer, name, instanceDir);
> 
>        //  fillup optional parameters
>        String opts = params.get(CoreAdminParams.CONFIG);
> -      if (opts != null)
> +      if (opts != null) {
> +        opts = PropertiesUtil.substituteProperty(opts, null);
>          dcore.setConfigName(opts);
> +      }
> 
>        opts = params.get(CoreAdminParams.SCHEMA);
> -      if (opts != null)
> +      if (opts != null) {
> +        opts = PropertiesUtil.substituteProperty(opts, null);
>          dcore.setSchemaName(opts);
> +      }
> 
>        opts = params.get(CoreAdminParams.DATA_DIR);
> -      if (opts != null)
> +      if (opts != null) {
> +        opts = PropertiesUtil.substituteProperty(opts, null);
>          dcore.setDataDir(opts);
> +      }
> 
>        opts = params.get(CoreAdminParams.ULOG_DIR);
>        if (opts != null)
> 
> Added:
> lucene/dev/trunk/solr/core/src/test/org/apache/solr/handler/admin/CoreA
> dminCreateDiscoverTest.java
> URL:
> http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apa
> che/solr/handler/admin/CoreAdminCreateDiscoverTest.java?rev=1500284&
> view=auto
> ==========================================================
> ====================
> ---
> lucene/dev/trunk/solr/core/src/test/org/apache/solr/handler/admin/CoreA
> dminCreateDiscoverTest.java (added)
> +++
> lucene/dev/trunk/solr/core/src/test/org/apache/solr/handler/admin/Co
> +++ reAdminCreateDiscoverTest.java Sat Jul  6 14:45:47 2013
> @@ -0,0 +1,228 @@
> +/*
> + * 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.
> + */
> +
> +package org.apache.solr.handler.admin;
> +
> +import org.apache.commons.io.FileUtils; import
> +org.apache.solr.SolrTestCaseJ4; import
> +org.apache.solr.common.params.CoreAdminParams;
> +import org.apache.solr.core.CoreDescriptor;
> +import org.apache.solr.core.SolrCoreDiscoverer;
> +import org.apache.solr.response.SolrQueryResponse;
> +import org.junit.AfterClass;
> +import org.junit.BeforeClass;
> +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 {
> +
> +  private static File solrHomeDirectory = null;
> +
> +  private static CoreAdminHandler admin = null; //  private static
> +CoreContainer cc = null;
> +
> +  private static String coreNormal = "normal";  private static String
> + coreSysProps = "sys_props";
> +
> +  @BeforeClass
> +  public static void before() throws Exception {
> +    useFactory(null); // I require FS-based indexes for this test.
> +
> +    solrHomeDirectory = new File(TEMP_DIR, "solrHome/" +
> CoreAdminCreateDiscoverTest.getClassName());
> +    if (solrHomeDirectory.exists()) {
> +      FileUtils.deleteDirectory(solrHomeDirectory);
> +    }
> +    assertTrue("Failed to mkdirs workDir", solrHomeDirectory.mkdirs());
> +
> +    setupNoCoreTest(solrHomeDirectory, null);
> +
> +    admin = new CoreAdminHandler(h.getCoreContainer());
> +  }
> +
> +  @AfterClass
> +  public static void after() throws Exception {
> +    h.close();
> +    if (solrHomeDirectory.exists()) {
> +      FileUtils.deleteDirectory(solrHomeDirectory);
> +    }
> +  }
> +
> +  private static void setupCore(String coreName, boolean blivet) throws
> IOException {
> +    File instDir = new File(solrHomeDirectory, coreName);
> +    File subHome = new File(instDir, "conf");
> +    assertTrue("Failed to make subdirectory ", subHome.mkdirs());
> +
> +    // Be sure we pick up sysvars when we create this
> +    String srcDir = SolrTestCaseJ4.TEST_HOME() + "/collection1/conf";
> +    FileUtils.copyFile(new File(srcDir, "schema-tiny.xml"), new File(subHome,
> "schema_ren.xml"));
> +    FileUtils.copyFile(new File(srcDir, "solrconfig-minimal.xml"), new
> + File(subHome, "solrconfig_ren.xml"));
> +
> +    FileUtils.copyFile(new File(srcDir,
> "solrconfig.snippet.randomindexconfig.xml"),
> +        new File(subHome, "solrconfig.snippet.randomindexconfig.xml"));
> +  }
> +
> +  @Test
> +  public void testCreateSavesSysProps() throws Exception {
> +
> +    setupCore(coreSysProps, true);
> +
> +    // create a new core (using CoreAdminHandler) w/ properties
> +    // Just to be sure its NOT written to the core.properties file
> +    File workDir = new File(solrHomeDirectory, coreSysProps);
> +    System.setProperty("INSTDIR_TEST", workDir.getAbsolutePath());
> +    System.setProperty("CONFIG_TEST", "solrconfig_ren.xml");
> +    System.setProperty("SCHEMA_TEST", "schema_ren.xml");
> +
> +    File dataDir = new File(workDir.getAbsolutePath(), "data_diff");
> +    System.setProperty("DATA_TEST", "data_diff");
> +
> +    SolrQueryResponse resp = new SolrQueryResponse();
> +    admin.handleRequestBody
> +        (req(CoreAdminParams.ACTION,
> +            CoreAdminParams.CoreAdminAction.CREATE.toString(),
> +            CoreAdminParams.NAME, coreSysProps,
> +            CoreAdminParams.INSTANCE_DIR, "${INSTDIR_TEST}",
> +            CoreAdminParams.CONFIG, "${CONFIG_TEST}",
> +            CoreAdminParams.SCHEMA, "${SCHEMA_TEST}",
> +            CoreAdminParams.DATA_DIR, "${DATA_TEST}"),
> +            resp);
> +    assertNull("Exception on create", resp.getException());
> +
> +    // verify props are in persisted file
> +
> +    Properties props = new Properties();
> +    File propFile = new File(solrHomeDirectory, coreSysProps + "/" +
> SolrCoreDiscoverer.CORE_PROP_FILE);
> +    FileInputStream is = new FileInputStream(propFile);
> +    try {
> +      props.load(is);
> +    } finally {
> +      org.apache.commons.io.IOUtils.closeQuietly(is);
> +    }
> +
> +    assertEquals("Unexpected value preserved in properties file " +
> propFile.getAbsolutePath(),
> +        props.getProperty(CoreAdminParams.NAME), coreSysProps);
> +
> +    assertEquals("Unexpected value preserved in properties file " +
> propFile.getAbsolutePath(),
> +        props.getProperty(CoreAdminParams.CONFIG), "${CONFIG_TEST}");
> +
> +    assertEquals("Unexpected value preserved in properties file " +
> propFile.getAbsolutePath(),
> +        props.getProperty(CoreAdminParams.SCHEMA), "${SCHEMA_TEST}");
> +
> +    assertEquals("Unexpected value preserved in properties file " +
> propFile.getAbsolutePath(),
> +        props.getProperty(CoreAdminParams.DATA_DIR), "${DATA_TEST}");
> +
> +    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.
> +
> +    // Should NOT be a datadir named ${DATA_TEST} (literal).
> +    File badDir = new File(workDir, "${DATA_TEST}");
> +    assertFalse("Should have substituted the sys var, found file " +
> + badDir.getAbsolutePath(), badDir.exists());
> +
> +    // 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}.
> +    File test = new File(dataDir, "index");
> +    assertTrue("Should have found index dir at " + test.getAbsolutePath(),
> test.exists());
> +    File gen = new File(test, "segments.gen");
> +    assertTrue("Should be segments.gen in the dir at " +
> + gen.getAbsolutePath(), gen.exists());
> +
> +  }
> +
> +  @Test
> +  public void testCreateSavesRegProps() throws Exception {
> +
> +    setupCore(coreNormal, true);
> +
> +    // create a new core (using CoreAdminHandler) w/ properties
> +    // Just to be sure its NOT written to the core.properties file
> +    File workDir = new File(solrHomeDirectory, coreNormal);
> +    File data = new File(workDir, "data");
> +
> +    SolrQueryResponse resp = new SolrQueryResponse();
> +    admin.handleRequestBody
> +        (req(CoreAdminParams.ACTION,
> +            CoreAdminParams.CoreAdminAction.CREATE.toString(),
> +            CoreAdminParams.NAME, coreNormal,
> +            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());
> +
> +    // verify props are in persisted file
> +    Properties props = new Properties();
> +    File propFile = new File(solrHomeDirectory, coreNormal + "/" +
> SolrCoreDiscoverer.CORE_PROP_FILE);
> +    FileInputStream is = new FileInputStream(propFile);
> +    try {
> +      props.load(is);
> +    } finally {
> +      org.apache.commons.io.IOUtils.closeQuietly(is);
> +    }
> +
> +    assertEquals("Unexpected value preserved in properties file " +
> propFile.getAbsolutePath(),
> +        props.getProperty(CoreAdminParams.NAME), coreNormal);
> +
> +    assertEquals("Unexpected value preserved in properties file " +
> propFile.getAbsolutePath(),
> +        props.getProperty(CoreAdminParams.CONFIG),
> + "solrconfig_ren.xml");
> +
> +    assertEquals("Unexpected value preserved in properties file " +
> propFile.getAbsolutePath(),
> +        props.getProperty(CoreAdminParams.SCHEMA), "schema_ren.xml");
> +
> +    assertEquals("Unexpected value preserved in properties file " +
> propFile.getAbsolutePath(),
> +        props.getProperty(CoreAdminParams.DATA_DIR),
> + data.getAbsolutePath());
> +
> +    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}.
> +    File test = new File(data, "index");
> +    assertTrue("Should have found index dir at " + test.getAbsolutePath(),
> test.exists());
> +    File gen = new File(test, "segments.gen");
> +    assertTrue("Should be segments.gen in the dir at " +
> + gen.getAbsolutePath(), gen.exists());
> +
> +  }
> +
> +  // 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/CoreA
> dminHandlerTest.java
> URL:
> http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apa
> che/solr/handler/admin/CoreAdminHandlerTest.java?rev=1500284&r1=1500
> 283&r2=1500284&view=diff
> ==========================================================
> ====================
> ---
> lucene/dev/trunk/solr/core/src/test/org/apache/solr/handler/admin/CoreA
> dminHandlerTest.java (original)
> +++
> lucene/dev/trunk/solr/core/src/test/org/apache/solr/handler/admin/Co
> +++ reAdminHandlerTest.java Sat Jul  6 14:45:47 2013
> @@ -17,9 +17,9 @@
> 
>  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.solr.handler.admin.CoreAdminHandler;
>  import org.apache.solr.common.SolrException;
>  import org.apache.solr.common.params.CoreAdminParams;
>  import org.apache.solr.common.util.NamedList;
> @@ -28,15 +28,14 @@ import org.apache.solr.SolrTestCaseJ4;
> 
>  import java.util.Map;
>  import java.io.File;
> -import java.io.IOException;
> -
> -import javax.xml.xpath.XPathExpressionException;
> 
>  import org.apache.commons.io.FileUtils;
> 
>  import org.junit.BeforeClass;
> +import org.junit.Rule;
>  import org.junit.Test;
> -import org.xml.sax.SAXException;
> +import org.junit.rules.RuleChain;
> +import org.junit.rules.TestRule;
> 
>  public class CoreAdminHandlerTest extends SolrTestCaseJ4 {
> 
> @@ -44,7 +43,93 @@ public class CoreAdminHandlerTest extend
>    public static void beforeClass() throws Exception {
>      initCore("solrconfig.xml", "schema.xml");
>    }
> -
> +
> +  @Rule
> +  public TestRule solrTestRules = RuleChain.outerRule(new
> + SystemPropertiesRestoreRule());
> +
> +  public String getCoreName() { return this.getClass().getName() +
> + "_sys_vars"; }
> +
> +  @Test
> +  public void testCreateWithSysVars() throws Exception {
> +    useFactory(null); // I require FS-based indexes for this test.
> +
> +    final File workDir = new File(TEMP_DIR, getCoreName());
> +
> +    if (workDir.exists()) {
> +      FileUtils.deleteDirectory(workDir);
> +    }
> +    assertTrue("Failed to mkdirs workDir", workDir.mkdirs());
> +    String coreName = "with_sys_vars";
> +    File instDir = new File(workDir, coreName);
> +    File subHome = new File(instDir, "conf");
> +    assertTrue("Failed to make subdirectory ", subHome.mkdirs());
> +
> +    // Be sure we pick up sysvars when we create this
> +    String srcDir = SolrTestCaseJ4.TEST_HOME() + "/collection1/conf";
> +    FileUtils.copyFile(new File(srcDir, "schema-tiny.xml"), new File(subHome,
> "schema_ren.xml"));
> +    FileUtils.copyFile(new File(srcDir, "solrconfig-minimal.xml"), new
> File(subHome, "solrconfig_ren.xml"));
> +    FileUtils.copyFile(new File(srcDir,
> "solrconfig.snippet.randomindexconfig.xml"),
> +        new File(subHome, "solrconfig.snippet.randomindexconfig.xml"));
> +
> +    final CoreContainer cores = h.getCoreContainer();
> +    cores.setPersistent(false); // we'll do this explicitly as needed
> +
> +    final CoreAdminHandler admin = new CoreAdminHandler(cores);
> +
> +    // create a new core (using CoreAdminHandler) w/ properties
> +    System.setProperty("INSTDIR_TEST", instDir.getAbsolutePath());
> +    System.setProperty("CONFIG_TEST", "solrconfig_ren.xml");
> +    System.setProperty("SCHEMA_TEST", "schema_ren.xml");
> +
> +    File dataDir = new File(workDir.getAbsolutePath(), "data_diff");
> +    System.setProperty("DATA_TEST", dataDir.getAbsolutePath());
> +
> +    SolrQueryResponse resp = new SolrQueryResponse();
> +    admin.handleRequestBody
> +        (req(CoreAdminParams.ACTION,
> +            CoreAdminParams.CoreAdminAction.CREATE.toString(),
> +            CoreAdminParams.NAME, getCoreName(),
> +            CoreAdminParams.INSTANCE_DIR, "${INSTDIR_TEST}",
> +            CoreAdminParams.CONFIG, "${CONFIG_TEST}",
> +            CoreAdminParams.SCHEMA, "${SCHEMA_TEST}",
> +            CoreAdminParams.DATA_DIR, "${DATA_TEST}"),
> +            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
> +            ,"/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}']"
> +            ,"/solr/cores/core[@name='" + getCoreName() + "' and
> @config='${CONFIG_TEST}']"
> +        );
> +
> +    // Now assert that certain values are properly dereferenced in the
> process of creating the core, see
> +    // SOLR-4982.
> +
> +    // Should NOT be a datadir named ${DATA_TEST} (literal). This is the bug
> after all
> +    File badDir = new File(instDir, "${DATA_TEST}");
> +    assertFalse("Should have substituted the sys var, found file " +
> + badDir.getAbsolutePath(), badDir.exists());
> +
> +    // For the other 3 vars, we couldn't get past creating the core fi
> dereferencing didn't work correctly.
> +
> +    // Should have segments in the directory pointed to by the
> ${DATA_TEST}.
> +    File test = new File(dataDir, "index");
> +    assertTrue("Should have found index dir at " + test.getAbsolutePath(),
> test.exists());
> +    test = new File(test,"segments.gen");
> +    assertTrue("Should have found segments.gen at " +
> + test.getAbsolutePath(), test.exists());
> +
> +    // Cleanup
> +    FileUtils.deleteDirectory(workDir);
> +
> +  }
> +
>    @Test
>    public void testCoreAdminHandler() throws Exception {
>      final File workDir = new File(TEMP_DIR, this.getClass().getName()); @@ -
> 141,6 +226,8 @@ public class CoreAdminHandlerTest extend
> 
>      // :TODO: because of SOLR-3665 we can't ask for status from all cores
> 
> -  }
> +    // cleanup
> +    FileUtils.deleteDirectory(workDir);
> 
> +  }
>  }
> 
> 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=1500284&r1=
> 1500283&r2=1500284&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/SolrTe
> +++ stCaseJ4.java Sat Jul  6 14:45:47 2013
> @@ -21,6 +21,7 @@ import com.carrotsearch.randomizedtestin  import
> com.carrotsearch.randomizedtesting.annotations.ThreadLeakFilters;
>  import
> com.carrotsearch.randomizedtesting.rules.SystemPropertiesRestoreRule;
>  import org.apache.commons.io.FileUtils;
> +import org.apache.lucene.util.IOUtils;
>  import org.apache.lucene.util.LuceneTestCase;
>  import org.apache.lucene.util.QuickPatchThreadsFilter;
>  import org.apache.solr.client.solrj.util.ClientUtils;
> @@ -32,9 +33,11 @@ import org.apache.solr.common.params.Com  import
> org.apache.solr.common.params.ModifiableSolrParams;
>  import org.apache.solr.common.params.SolrParams;
>  import org.apache.solr.common.util.XML;
> +import org.apache.solr.core.ConfigSolr;
>  import org.apache.solr.core.CoreContainer;
>  import org.apache.solr.core.SolrConfig;  import
> org.apache.solr.core.SolrCore;
> +import org.apache.solr.core.SolrResourceLoader;
>  import org.apache.solr.handler.JsonUpdateRequestHandler;
>  import org.apache.solr.request.LocalSolrQueryRequest;
>  import org.apache.solr.request.SolrQueryRequest;
> @@ -155,6 +158,25 @@ public abstract class SolrTestCaseJ4 ext
>      }
>    }
> 
> +  /**
> +   * Call this from @BeforeClass to set up the test harness and update
> handler with no cores.
> +   *
> +   * @param solrHome The solr home directory.
> +   * @param xmlStr - the text of an XML file to use. If null, use the what's the
> absolute minimal file.
> +   * @throws Exception Lost of file-type things can go wrong.
> +   */
> +  public static void setupNoCoreTest(File solrHome, String xmlStr)
> + throws Exception {
> +
> +    File tmpFile = new File(solrHome, ConfigSolr.SOLR_XML_FILE);
> +    if (xmlStr == null) {
> +      xmlStr = "<solr></solr>";
> +    }
> +    FileUtils.write(tmpFile, xmlStr, IOUtils.CHARSET_UTF_8.toString());
> +
> +    SolrResourceLoader loader = new
> SolrResourceLoader(solrHome.getAbsolutePath());
> +    h = new TestHarness(loader, ConfigSolr.fromFile(loader, new
> File(solrHome, "solr.xml")));
> +    lrf = h.getRequestFactory("standard", 0, 20, CommonParams.VERSION,
> + "2.2");  }
> 
>    @Override
>    public void setUp() throws Exception {



---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org