You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ho...@apache.org on 2019/08/01 16:15:29 UTC

[lucene-solr] branch master updated: SOLR-13664: Fixed SolrTestCaseJ4.deleteCore() to properly reset the dataDir used by initCore()

This is an automated email from the ASF dual-hosted git repository.

hossman pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/lucene-solr.git


The following commit(s) were added to refs/heads/master by this push:
     new ab470a6  SOLR-13664: Fixed SolrTestCaseJ4.deleteCore() to properly reset the dataDir used by initCore()
ab470a6 is described below

commit ab470a6564b1184c2d77892131f56a9912f7d8c6
Author: Chris Hostetter <ho...@apache.org>
AuthorDate: Thu Aug 1 09:15:18 2019 -0700

    SOLR-13664: Fixed SolrTestCaseJ4.deleteCore() to properly reset the dataDir used by initCore()
---
 solr/CHANGES.txt                                   | 12 ++++
 .../TestSolrEntityProcessorEndToEnd.java           |  7 ++-
 .../test/org/apache/solr/TestSolrTestCaseJ4.java   | 36 ++++++++++++
 .../solr/core/SolrCoreCheckLockOnStartupTest.java  |  6 +-
 .../test/org/apache/solr/core/TestConfigSets.java  |  7 +--
 .../src/test/org/apache/solr/core/TestNRTOpen.java | 14 +----
 .../solr/handler/PingRequestHandlerTest.java       |  4 +-
 .../solr/handler/admin/CoreAdminHandlerTest.java   | 12 +---
 .../component/QueryElevationComponentTest.java     |  4 +-
 .../solr/metrics/SolrMetricsIntegrationTest.java   |  6 +-
 .../reporters/SolrGraphiteReporterTest.java        |  6 +-
 .../metrics/reporters/SolrSlf4jReporterTest.java   |  6 +-
 .../solr/schema/TestUseDocValuesAsStored.java      |  2 +
 .../test/org/apache/solr/search/TestRecovery.java  |  8 ---
 .../org/apache/solr/servlet/CacheHeaderTest.java   |  2 +-
 .../src/java/org/apache/solr/SolrTestCaseJ4.java   | 65 +++++++++++++++++++---
 16 files changed, 138 insertions(+), 59 deletions(-)

diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index f333c84..03279b7 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -78,6 +78,16 @@ Velocity 2.0 and Velocity Tools 3.0
 Apache ZooKeeper 3.5.5
 Jetty 9.4.19.v20190610
 
+Upgrade Notes
+----------------------
+
+* Users who have written test cases that extend SolrTestCaseJ4 may see NullPointerExceptions if
+  their tests directly reference both SolrTestCaseJ4.initCoreDataDir and SolrTestCaseJ4.deleteCore().
+  This change in behavior is due to a bug fix in deleteCore() to ensure the dataDir is properly reset
+  in tests that call initCore()/deleteCore() multiple times in a given test (class).  initCoreDataDir
+  is now deprecated, and users are encouraged to use SolrTestCaseJ4.initAndGetDataDir() in it's place.
+  See SOLR-13664 for more details.
+
 New Features
 ----------------------
 
@@ -113,6 +123,8 @@ Bug Fixes
 * SOLR-13660: Fixed AbstractFullDistribZkTestBase.waitForActiveReplicaCount() to ensure
   replicas are active. (hossman)
 
+* SOLR-13664: Fixed SolrTestCaseJ4.deleteCore() to properly reset the dataDir used by initCore()
+  (hossman)
 
 Other Changes
 ----------------------
diff --git a/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestSolrEntityProcessorEndToEnd.java b/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestSolrEntityProcessorEndToEnd.java
index badaf1b..0529ec3 100644
--- a/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestSolrEntityProcessorEndToEnd.java
+++ b/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestSolrEntityProcessorEndToEnd.java
@@ -321,6 +321,7 @@ public class TestSolrEntityProcessorEndToEnd extends AbstractDataImportHandlerTe
   private static class SolrInstance {
     File homeDir;
     File confDir;
+    File dataDir;
     
     public String getHomeDir() {
       return homeDir.toString();
@@ -331,7 +332,7 @@ public class TestSolrEntityProcessorEndToEnd extends AbstractDataImportHandlerTe
     }
     
     public String getDataDir() {
-      return initCoreDataDir.toString();
+      return dataDir.toString();
     }
     
     public String getSolrConfigFile() {
@@ -344,11 +345,11 @@ public class TestSolrEntityProcessorEndToEnd extends AbstractDataImportHandlerTe
 
     public void setUp() throws Exception {
       homeDir = createTempDir().toFile();
-      initCoreDataDir = new File(homeDir + "/collection1", "data");
+      dataDir = new File(homeDir + "/collection1", "data");
       confDir = new File(homeDir + "/collection1", "conf");
       
       homeDir.mkdirs();
-      initCoreDataDir.mkdirs();
+      dataDir.mkdirs();
       confDir.mkdirs();
 
       FileUtils.copyFile(getFile(getSolrXmlFile()), new File(homeDir, "solr.xml"));
diff --git a/solr/core/src/test/org/apache/solr/TestSolrTestCaseJ4.java b/solr/core/src/test/org/apache/solr/TestSolrTestCaseJ4.java
new file mode 100644
index 0000000..deb8edc
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/TestSolrTestCaseJ4.java
@@ -0,0 +1,36 @@
+/*
+ * 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;
+
+/** Test the test */
+public class TestSolrTestCaseJ4 extends SolrTestCaseJ4 {
+
+  public void testDeleteCore() throws Exception {
+    initCore("solrconfig.xml","schema.xml");
+    assertU(adoc("id", "1"));
+    assertU(commit());
+    assertQ(req("q", "*:*"), "//*[@numFound='1']");
+
+    deleteCore();
+    initCore("solrconfig.xml","schema.xml");
+    assertQ(req("q", "*:*"), "//*[@numFound='0']");
+  }
+
+
+
+}
diff --git a/solr/core/src/test/org/apache/solr/core/SolrCoreCheckLockOnStartupTest.java b/solr/core/src/test/org/apache/solr/core/SolrCoreCheckLockOnStartupTest.java
index 6867d74..e1e43de 100644
--- a/solr/core/src/test/org/apache/solr/core/SolrCoreCheckLockOnStartupTest.java
+++ b/solr/core/src/test/org/apache/solr/core/SolrCoreCheckLockOnStartupTest.java
@@ -46,13 +46,13 @@ public class SolrCoreCheckLockOnStartupTest extends SolrTestCaseJ4 {
     // test tests native and simple in the same jvm in the same exact directory:
     // the file will remain after the native test (it cannot safely be deleted without the risk of deleting another guys lock)
     // it's ok, these aren't "compatible" anyway: really this test should not re-use the same directory at all.
-    Files.deleteIfExists(new File(new File(initCoreDataDir, "index"), IndexWriter.WRITE_LOCK_NAME).toPath());
+    Files.deleteIfExists(new File(new File(initAndGetDataDir(), "index"), IndexWriter.WRITE_LOCK_NAME).toPath());
   }
 
   @Test
   public void testSimpleLockErrorOnStartup() throws Exception {
 
-    Directory directory = newFSDirectory(new File(initCoreDataDir, "index").toPath(), SimpleFSLockFactory.INSTANCE);
+    Directory directory = newFSDirectory(new File(initAndGetDataDir(), "index").toPath(), SimpleFSLockFactory.INSTANCE);
     //creates a new IndexWriter without releasing the lock yet
     IndexWriter indexWriter = new IndexWriter(directory, new IndexWriterConfig(null));
 
@@ -76,7 +76,7 @@ public class SolrCoreCheckLockOnStartupTest extends SolrTestCaseJ4 {
   @Test
   public void testNativeLockErrorOnStartup() throws Exception {
 
-    File indexDir = new File(initCoreDataDir, "index");
+    File indexDir = new File(initAndGetDataDir(), "index");
     log.info("Acquiring lock on {}", indexDir.getAbsolutePath());
     Directory directory = newFSDirectory(indexDir.toPath(), NativeFSLockFactory.INSTANCE);
     //creates a new IndexWriter without releasing the lock yet
diff --git a/solr/core/src/test/org/apache/solr/core/TestConfigSets.java b/solr/core/src/test/org/apache/solr/core/TestConfigSets.java
index 6c110c3..84aa65d 100644
--- a/solr/core/src/test/org/apache/solr/core/TestConfigSets.java
+++ b/solr/core/src/test/org/apache/solr/core/TestConfigSets.java
@@ -105,16 +105,15 @@ public class TestConfigSets extends SolrTestCaseJ4 {
 
   @Test
   public void testConfigSetOnCoreReload() throws IOException {
-    File testDirectory = new File(initCoreDataDir, "core-reload");
-    testDirectory.mkdirs();
-    File configSetsDir = new File(testDirectory, "configsets");
+    Path testDirectory = createTempDir("core-reload");
+    File configSetsDir = new File(testDirectory.toFile(), "configsets");
 
     FileUtils.copyDirectory(getFile("solr/configsets"), configSetsDir);
 
     String csd = configSetsDir.getAbsolutePath();
     System.setProperty("configsets", csd);
 
-    SolrResourceLoader loader = new SolrResourceLoader(testDirectory.toPath());
+    SolrResourceLoader loader = new SolrResourceLoader(testDirectory);
     CoreContainer container = new CoreContainer(SolrXmlConfig.fromString(loader, solrxml));
     container.load();
 
diff --git a/solr/core/src/test/org/apache/solr/core/TestNRTOpen.java b/solr/core/src/test/org/apache/solr/core/TestNRTOpen.java
index 4c4fed6..2b28110 100644
--- a/solr/core/src/test/org/apache/solr/core/TestNRTOpen.java
+++ b/solr/core/src/test/org/apache/solr/core/TestNRTOpen.java
@@ -16,7 +16,6 @@
  */
 package org.apache.solr.core;
 
-import java.io.File;
 import java.io.IOException;
 import java.util.Collections;
 import java.util.IdentityHashMap;
@@ -33,10 +32,6 @@ public class TestNRTOpen extends SolrTestCaseJ4 {
 
   @BeforeClass
   public static void beforeClass() throws Exception {
-    // use a filesystem, because we need to create an index, then "start up solr"
-    System.setProperty("solr.directoryFactory", "solr.StandardDirectoryFactory");
-    // and dont delete it initially
-    System.setProperty("solr.test.leavedatadir", "true");
     // set these so that merges won't break the test
     System.setProperty("solr.tests.maxBufferedDocs", "100000");
     systemSetPropertySolrTestsMergePolicyFactory(LogDocMergePolicyFactory.class.getName());
@@ -44,19 +39,14 @@ public class TestNRTOpen extends SolrTestCaseJ4 {
     // add a doc
     assertU(adoc("foo", "bar"));
     assertU(commit());
-    File myDir = initCoreDataDir;
-    deleteCore();
-    // boot up again over the same index
-    initCoreDataDir = myDir;
-    initCore("solrconfig-basic.xml", "schema-minimal.xml");
-    // startup
+    // reload the core again over the same index
+    h.reload();
     assertNRT(1);
   }
 
   @AfterClass
   public static void afterClass() throws Exception {
     // ensure we clean up after ourselves, this will fire before superclass...
-    System.clearProperty("solr.test.leavedatadir");
     System.clearProperty("solr.directoryFactory");
     System.clearProperty("solr.tests.maxBufferedDocs");
     systemClearPropertySolrTestsMergePolicyFactory();
diff --git a/solr/core/src/test/org/apache/solr/handler/PingRequestHandlerTest.java b/solr/core/src/test/org/apache/solr/handler/PingRequestHandlerTest.java
index b9ab2c1..166b0b9 100644
--- a/solr/core/src/test/org/apache/solr/handler/PingRequestHandlerTest.java
+++ b/solr/core/src/test/org/apache/solr/handler/PingRequestHandlerTest.java
@@ -51,14 +51,12 @@ public class PingRequestHandlerTest extends SolrTestCaseJ4 {
 
   @Before
   public void before() throws IOException {
-    File tmpDir = initCoreDataDir;
     // by default, use relative file in dataDir
-    healthcheckFile = new File(tmpDir, fileName);
+    healthcheckFile = new File(initAndGetDataDir(), fileName);
     String fileNameParam = fileName;
 
     // sometimes randomly use an absolute File path instead 
     if (random().nextBoolean()) {
-      healthcheckFile = new File(tmpDir, fileName);
       fileNameParam = healthcheckFile.getAbsolutePath();
     } 
       
diff --git a/solr/core/src/test/org/apache/solr/handler/admin/CoreAdminHandlerTest.java b/solr/core/src/test/org/apache/solr/handler/admin/CoreAdminHandlerTest.java
index 161675b..2c68320 100644
--- a/solr/core/src/test/org/apache/solr/handler/admin/CoreAdminHandlerTest.java
+++ b/solr/core/src/test/org/apache/solr/handler/admin/CoreAdminHandlerTest.java
@@ -253,9 +253,7 @@ public class CoreAdminHandlerTest extends SolrTestCaseJ4 {
 
   @Test
   public void testDeleteInstanceDir() throws Exception  {
-    File solrHomeDirectory = new File(initCoreDataDir, getClass().getName() + "-corex-"
-        + System.nanoTime());
-    solrHomeDirectory.mkdirs();
+    File solrHomeDirectory = createTempDir("solr-home").toFile();
     copySolrHomeToTemp(solrHomeDirectory, "corex");
     File corex = new File(solrHomeDirectory, "corex");
     FileUtils.write(new File(corex, "core.properties"), "", StandardCharsets.UTF_8);
@@ -317,9 +315,7 @@ public class CoreAdminHandlerTest extends SolrTestCaseJ4 {
 
   @Test
   public void testUnloadForever() throws Exception  {
-    File solrHomeDirectory = new File(initCoreDataDir, getClass().getName() + "-corex-"
-        + System.nanoTime());
-    solrHomeDirectory.mkdirs();
+    File solrHomeDirectory = createTempDir("solr-home").toFile();
     copySolrHomeToTemp(solrHomeDirectory, "corex");
     File corex = new File(solrHomeDirectory, "corex");
     FileUtils.write(new File(corex, "core.properties"), "", StandardCharsets.UTF_8);
@@ -362,9 +358,7 @@ public class CoreAdminHandlerTest extends SolrTestCaseJ4 {
   @Test
   public void testDeleteInstanceDirAfterCreateFailure() throws Exception  {
     assumeFalse("Ignore test on windows because it does not delete data directory immediately after unload", Constants.WINDOWS);
-    File solrHomeDirectory = new File(initCoreDataDir, getClass().getName() + "-corex-"
-        + System.nanoTime());
-    solrHomeDirectory.mkdirs();
+    File solrHomeDirectory = createTempDir("solr-home").toFile();
     copySolrHomeToTemp(solrHomeDirectory, "corex");
     File corex = new File(solrHomeDirectory, "corex");
     FileUtils.write(new File(corex, "core.properties"), "", StandardCharsets.UTF_8);
diff --git a/solr/core/src/test/org/apache/solr/handler/component/QueryElevationComponentTest.java b/solr/core/src/test/org/apache/solr/handler/component/QueryElevationComponentTest.java
index 0db6b0f..7c7495d 100644
--- a/solr/core/src/test/org/apache/solr/handler/component/QueryElevationComponentTest.java
+++ b/solr/core/src/test/org/apache/solr/handler/component/QueryElevationComponentTest.java
@@ -88,7 +88,7 @@ public class QueryElevationComponentTest extends SolrTestCaseJ4 {
     //write out elevate-data.xml to the Data dir first by copying it from conf, which we know exists, this way we can test both conf and data configurations
     File parent = new File(TEST_HOME() + "/collection1", "conf");
     File elevateFile = new File(parent, "elevate.xml");
-    File elevateDataFile = new File(initCoreDataDir, "elevate-data.xml");
+    File elevateDataFile = new File(initAndGetDataDir(), "elevate-data.xml");
     FileUtils.copyFile(elevateFile, elevateDataFile);
 
 
@@ -794,4 +794,4 @@ public class QueryElevationComponentTest extends SolrTestCaseJ4 {
     }
   }
 
-}
\ No newline at end of file
+}
diff --git a/solr/core/src/test/org/apache/solr/metrics/SolrMetricsIntegrationTest.java b/solr/core/src/test/org/apache/solr/metrics/SolrMetricsIntegrationTest.java
index 7f37031..d71588c 100644
--- a/solr/core/src/test/org/apache/solr/metrics/SolrMetricsIntegrationTest.java
+++ b/solr/core/src/test/org/apache/solr/metrics/SolrMetricsIntegrationTest.java
@@ -74,8 +74,10 @@ public class SolrMetricsIntegrationTest extends SolrTestCaseJ4 {
     System.setProperty("solr.test.sys.prop2", "proptwo");
     String solrXml = FileUtils.readFileToString(Paths.get(home.toString(), "solr-metricreporter.xml").toFile(), "UTF-8");
     NodeConfig cfg = SolrXmlConfig.fromString(new SolrResourceLoader(home), solrXml);
-    cc = createCoreContainer(cfg,
-        new TestHarness.TestCoresLocator(DEFAULT_TEST_CORENAME, initCoreDataDir.getAbsolutePath(), "solrconfig.xml", "schema.xml"));
+    cc = createCoreContainer(cfg, new TestHarness.TestCoresLocator
+                             (DEFAULT_TEST_CORENAME, initAndGetDataDir().getAbsolutePath(),
+                              "solrconfig.xml", "schema.xml"));
+                             
     h.coreName = DEFAULT_TEST_CORENAME;
     jmxReporter = JmxUtil.findFirstMBeanServer() != null ? 1 : 0;
     metricManager = cc.getMetricManager();
diff --git a/solr/core/src/test/org/apache/solr/metrics/reporters/SolrGraphiteReporterTest.java b/solr/core/src/test/org/apache/solr/metrics/reporters/SolrGraphiteReporterTest.java
index 651fe79..df18706 100644
--- a/solr/core/src/test/org/apache/solr/metrics/reporters/SolrGraphiteReporterTest.java
+++ b/solr/core/src/test/org/apache/solr/metrics/reporters/SolrGraphiteReporterTest.java
@@ -61,8 +61,10 @@ public class SolrGraphiteReporterTest extends SolrTestCaseJ4 {
       System.setProperty("mock-graphite-port", String.valueOf(mock.port));
       String solrXml = FileUtils.readFileToString(Paths.get(home.toString(), "solr-graphitereporter.xml").toFile(), "UTF-8");
       NodeConfig cfg = SolrXmlConfig.fromString(new SolrResourceLoader(home), solrXml);
-      CoreContainer cc = createCoreContainer(cfg,
-          new TestHarness.TestCoresLocator(DEFAULT_TEST_CORENAME, initCoreDataDir.getAbsolutePath(), "solrconfig.xml", "schema.xml"));
+      CoreContainer cc = createCoreContainer(cfg, new TestHarness.TestCoresLocator
+                                             (DEFAULT_TEST_CORENAME, initAndGetDataDir().getAbsolutePath(),
+                                              "solrconfig.xml", "schema.xml"));
+                                             
       h.coreName = DEFAULT_TEST_CORENAME;
       SolrMetricManager metricManager = cc.getMetricManager();
       Map<String, SolrMetricReporter> reporters = metricManager.getReporters("solr.node");
diff --git a/solr/core/src/test/org/apache/solr/metrics/reporters/SolrSlf4jReporterTest.java b/solr/core/src/test/org/apache/solr/metrics/reporters/SolrSlf4jReporterTest.java
index 7646864..5fe5e08 100644
--- a/solr/core/src/test/org/apache/solr/metrics/reporters/SolrSlf4jReporterTest.java
+++ b/solr/core/src/test/org/apache/solr/metrics/reporters/SolrSlf4jReporterTest.java
@@ -54,8 +54,10 @@ public class SolrSlf4jReporterTest extends SolrTestCaseJ4 {
 
     String solrXml = FileUtils.readFileToString(Paths.get(home.toString(), "solr-slf4jreporter.xml").toFile(), "UTF-8");
     NodeConfig cfg = SolrXmlConfig.fromString(new SolrResourceLoader(home), solrXml);
-    CoreContainer cc = createCoreContainer(cfg,
-        new TestHarness.TestCoresLocator(DEFAULT_TEST_CORENAME, initCoreDataDir.getAbsolutePath(), "solrconfig.xml", "schema.xml"));
+    CoreContainer cc = createCoreContainer(cfg, new TestHarness.TestCoresLocator
+                                           (DEFAULT_TEST_CORENAME, initAndGetDataDir().getAbsolutePath(),
+                                            "solrconfig.xml", "schema.xml"));
+                                           
     h.coreName = DEFAULT_TEST_CORENAME;
     SolrMetricManager metricManager = cc.getMetricManager();
     Map<String, SolrMetricReporter> reporters = metricManager.getReporters("solr.node");
diff --git a/solr/core/src/test/org/apache/solr/schema/TestUseDocValuesAsStored.java b/solr/core/src/test/org/apache/solr/schema/TestUseDocValuesAsStored.java
index 899212a..3694fb2 100644
--- a/solr/core/src/test/org/apache/solr/schema/TestUseDocValuesAsStored.java
+++ b/solr/core/src/test/org/apache/solr/schema/TestUseDocValuesAsStored.java
@@ -104,6 +104,8 @@ public class TestUseDocValuesAsStored extends AbstractBadConfigTestBase {
     System.setProperty("enable.update.log", "false");
     System.setProperty("managed.schema.mutable", "true");
     initCore("solrconfig-managed-schema.xml", "schema-non-stored-docvalues.xml", tmpSolrHome.getPath());
+
+    assertQ("sanity check", req("q", "*:*"), "//*[@numFound='0']");
   }
 
   @After
diff --git a/solr/core/src/test/org/apache/solr/search/TestRecovery.java b/solr/core/src/test/org/apache/solr/search/TestRecovery.java
index aa7a300..147cfe2 100644
--- a/solr/core/src/test/org/apache/solr/search/TestRecovery.java
+++ b/solr/core/src/test/org/apache/solr/search/TestRecovery.java
@@ -18,7 +18,6 @@ package org.apache.solr.search;
 
 
 import java.io.File;
-import java.io.IOException;
 import java.io.RandomAccessFile;
 import java.lang.invoke.MethodHandles;
 import java.nio.charset.StandardCharsets;
@@ -40,7 +39,6 @@ import com.codahale.metrics.Gauge;
 import com.codahale.metrics.Meter;
 import com.codahale.metrics.Metric;
 import com.codahale.metrics.MetricRegistry;
-import org.apache.commons.io.FileUtils;
 import org.apache.lucene.util.TestUtil;
 import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.common.util.TimeSource;
@@ -97,12 +95,6 @@ public class TestRecovery extends SolrTestCaseJ4 {
     }
     
     deleteCore();
-    
-    try {
-      FileUtils.deleteDirectory(initCoreDataDir);
-    } catch (IOException e) {
-      log.error("Exception deleting core directory.", e);
-    }
   }
 
   private Map<String, Metric> getMetrics() {
diff --git a/solr/core/src/test/org/apache/solr/servlet/CacheHeaderTest.java b/solr/core/src/test/org/apache/solr/servlet/CacheHeaderTest.java
index b71cbc7..a986422 100644
--- a/solr/core/src/test/org/apache/solr/servlet/CacheHeaderTest.java
+++ b/solr/core/src/test/org/apache/solr/servlet/CacheHeaderTest.java
@@ -257,7 +257,7 @@ public class CacheHeaderTest extends CacheHeaderTestBase {
 
   protected File makeFile(String contents, String charset) {
     try {
-      File f = new File(initCoreDataDir, "cachetest_csv");
+      File f = createTempFile("cachetest","csv").toFile();
       Writer out = new OutputStreamWriter(new FileOutputStream(f), charset);
       out.write(contents);
       out.close();
diff --git a/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java b/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java
index de166e3..21a91ee 100644
--- a/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java
+++ b/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java
@@ -59,6 +59,7 @@ import java.util.Map.Entry;
 import java.util.Optional;
 import java.util.Properties;
 import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.CopyOnWriteArraySet;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.SynchronousQueue;
@@ -283,8 +284,10 @@ public abstract class SolrTestCaseJ4 extends SolrTestCase {
         new SolrjNamedThreadFactory("testExecutor"),
         true);
 
-    initCoreDataDir = createTempDir("init-core-data").toFile();
-    System.err.println("Creating dataDir: " + initCoreDataDir.getAbsolutePath());
+    // not strictly needed by this class at this point in the control lifecycle, but for
+    // backcompat create it now in case any third party tests expect initCoreDataDir to be
+    // non-null after calling setupTestCases()
+    initAndGetDataDir();
 
     System.setProperty("solr.zkclienttimeout", "90000"); 
     
@@ -588,6 +591,44 @@ public abstract class SolrTestCaseJ4 extends SolrTestCase {
     super.tearDown();
   }
 
+  /**
+   * Subclasses may call this method to access the "dataDir" that will be used by 
+   * {@link #initCore} (either prior to or after the core is created).
+   * <p>
+   * If the dataDir has not yet been initialized when this method is called, this method will do so.
+   * Calling {@link #deleteCore} will "reset" the value, such that subsequent calls will 
+   * re-initialize a new value.  All directories returned by any calls to this method will 
+   * automatically be cleaned up per {@link #createTempDir}
+   * </p>
+   * <p>
+   * NOTE: calling this method is not requried, it will be implicitly called as needed when
+   * initializing cores.  Callers that don't care about using {@link #initCore} and just want
+   * a temporary directory to put data in sould instead be using {@link #createTempDir} directly.
+   * </p>
+   *
+   * @see #initCoreDataDir
+   */
+  protected static File initAndGetDataDir() {
+    File dataDir = initCoreDataDir;
+    if (null == dataDir) {
+      final int id = dataDirCount.incrementAndGet();
+      dataDir = initCoreDataDir = createTempDir("data-dir-"+ id).toFile();
+      assertNotNull(dataDir);
+      log.info("Created dataDir: {}", dataDir.getAbsolutePath());
+    }
+    return dataDir;
+  }
+  /** 
+   * Counter for ensuring we don't ask {@link #createTempDir} to try and 
+   * re-create the same dir prefix over and over.
+   * <p>
+   * (createTempDir has it's own counter for uniqueness, but it tries all numbers in a loop 
+   * until it finds one available.  No reason to force that O(N^2) behavior when we know we've 
+   * already created N previous directories with the same prefix.)
+   * </p>
+   */
+  private static final AtomicInteger dataDirCount = new AtomicInteger(0);
+  
   /** Call initCore in @BeforeClass to instantiate a solr core in your test class.
    * deleteCore will be called for you via SolrTestCaseJ4 @AfterClass */
   public static void initCore(String config, String schema) throws Exception {
@@ -696,8 +737,17 @@ public abstract class SolrTestCaseJ4 extends SolrTestCase {
   }
 
   /**
-   * The directory used to story the index managed by the TestHarness
+   * The directory used as the <code>dataDir</code> for the TestHarness unless 
+   * {@link #hdfsDataDir} is non null.  
+   * <p>
+   * Will be set to null by {@link #deleteCore} and re-initialized as needed by {@link #createCore}.  
+   * In the event of a test failure, the contents will be left on disk.
+   * </p>
+   * @see #createTempDir(String)
+   * @see #initAndGetDataDir()
+   * @deprecated use initAndGetDataDir instead of directly accessing this variable
    */
+  @Deprecated
   protected static volatile File initCoreDataDir;
   
   // hack due to File dataDir
@@ -740,7 +790,7 @@ public abstract class SolrTestCaseJ4 extends SolrTestCase {
   public static void createCore() {
     assertNotNull(testSolrHome);
     solrConfig = TestHarness.createConfig(testSolrHome, coreName, getSolrConfigFile());
-    h = new TestHarness( coreName, hdfsDataDir == null ? initCoreDataDir.getAbsolutePath() : hdfsDataDir,
+    h = new TestHarness( coreName, hdfsDataDir == null ? initAndGetDataDir().getAbsolutePath() : hdfsDataDir,
             solrConfig,
             getSchemaFile());
     lrf = h.getRequestFactory
@@ -771,7 +821,7 @@ public abstract class SolrTestCaseJ4 extends SolrTestCase {
 
   public static CoreContainer createDefaultCoreContainer(Path solrHome) {
     testSolrHome = requireNonNull(solrHome);
-    h = new TestHarness("collection1", initCoreDataDir.getAbsolutePath(), "solrconfig.xml", "schema.xml");
+    h = new TestHarness("collection1", initAndGetDataDir().getAbsolutePath(), "solrconfig.xml", "schema.xml");
     lrf = h.getRequestFactory("", 0, 20, CommonParams.VERSION, "2.2");
     return h.getCoreContainer();
   }
@@ -810,9 +860,7 @@ public abstract class SolrTestCaseJ4 extends SolrTestCase {
   }
 
   /**
-   * Shuts down the test harness, and makes the best attempt possible
-   * to delete dataDir, unless the system property "solr.test.leavedatadir"
-   * is set.
+   * Shuts down the test harness and nulls out the values setup by {@link #initCore}
    */
   public static void deleteCore() {
     if (h != null) {
@@ -845,6 +893,7 @@ public abstract class SolrTestCaseJ4 extends SolrTestCase {
     h = null;
     lrf = null;
     configString = schemaString = null;
+    initCoreDataDir = null;
   }
 
   /**