You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ja...@apache.org on 2020/06/29 19:41:18 UTC

[lucene-solr] branch branch_8x updated: SOLR-14561 Followup - validate params for more core operations (#1629)

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

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


The following commit(s) were added to refs/heads/branch_8x by this push:
     new b52bf26  SOLR-14561 Followup - validate params for more core operations  (#1629)
b52bf26 is described below

commit b52bf267c7aa528b282d09c637344650f9034956
Author: Jan Høydahl <ja...@apache.org>
AuthorDate: Mon Jun 29 13:18:24 2020 +0200

    SOLR-14561 Followup - validate params for more core operations  (#1629)
    
    Add template to solr.in scripts
    Also testes Windows paths
    Added RefGuide documentation to some params
    
    (cherry picked from commit 49a3f0a11d41f7124b893a08dc9e67594c32e2ee and adapted to Java8)
---
 .../src/java/org/apache/solr/core/SolrPaths.java   |  1 +
 .../apache/solr/handler/ReplicationHandler.java    | 23 +++++++++++++-------
 .../java/org/apache/solr/handler/SnapShooter.java  |  3 +++
 .../apache/solr/handler/admin/MergeIndexesOp.java  |  8 +++----
 .../apache/solr/cloud/BasicDistributedZk2Test.java |  5 +++--
 .../AbstractCloudBackupRestoreTestCase.java        |  7 ++++++
 .../core/snapshots/TestSolrCloudSnapshots.java     |  2 ++
 .../solr/core/snapshots/TestSolrCoreSnapshots.java |  2 ++
 .../org/apache/solr/handler/TestCoreBackup.java    | 22 ++++++++++---------
 .../solr/handler/TestReplicationHandler.java       |  4 +++-
 .../org/apache/solr/handler/TestRestoreCore.java   | 11 ++++++++++
 .../apache/solr/handler/V2ApiIntegrationTest.java  |  3 ++-
 .../admin/CoreMergeIndexesAdminHandlerTest.java    |  8 +++----
 solr/solr-ref-guide/src/index-replication.adoc     | 25 +++++++++++++++++++++-
 .../client/solrj/MergeIndexesExampleTestBase.java  |  3 ++-
 .../apache/solr/cloud/MiniSolrCloudCluster.java    |  1 +
 16 files changed, 96 insertions(+), 32 deletions(-)

diff --git a/solr/core/src/java/org/apache/solr/core/SolrPaths.java b/solr/core/src/java/org/apache/solr/core/SolrPaths.java
index db1b500..0d5d358 100644
--- a/solr/core/src/java/org/apache/solr/core/SolrPaths.java
+++ b/solr/core/src/java/org/apache/solr/core/SolrPaths.java
@@ -119,6 +119,7 @@ public final class SolrPaths {
    * @throws SolrException if path is outside allowed paths
    */
   public static void assertPathAllowed(Path pathToAssert, Set<Path> allowPaths) throws SolrException {
+    if (pathToAssert == null) return;
     if (OS.isFamilyWindows() && pathToAssert.toString().startsWith("\\\\")) {
       throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
           "Path " + pathToAssert + " disallowed. UNC paths not supported. Please use drive letter instead.");
diff --git a/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java b/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java
index 484e662..d6b779e 100644
--- a/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java
@@ -326,10 +326,13 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
     }
   }
 
+  @SuppressWarnings("deprecation")
   private void deleteSnapshot(ModifiableSolrParams params, SolrQueryResponse rsp) {
-    String name = params.required().get(NAME);
+    params.required().get(NAME);
 
-    SnapShooter snapShooter = new SnapShooter(core, params.get(CoreAdminParams.BACKUP_LOCATION), params.get(NAME));
+    String location = params.get(CoreAdminParams.BACKUP_LOCATION);
+    core.getCoreContainer().assertPathAllowed(location == null ? null : Paths.get(location));
+    SnapShooter snapShooter = new SnapShooter(core, location, params.get(NAME));
     snapShooter.validateDeleteSnapshot();
     snapShooter.deleteSnapAsync(this);
     rsp.add(STATUS, OK_STATUS);
@@ -450,7 +453,6 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
     }
     String name = params.get(NAME);
     String location = params.get(CoreAdminParams.BACKUP_LOCATION);
-
     String repoName = params.get(CoreAdminParams.BACKUP_REPOSITORY);
     CoreContainer cc = core.getCoreContainer();
     BackupRepository repo = null;
@@ -462,11 +464,13 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
       }
     } else {
       repo = new LocalFileSystemRepository();
+      //If location is not provided then assume that the restore index is present inside the data directory.
+      if (location == null) {
+        location = core.getDataDir();
+      }
     }
-
-    //If location is not provided then assume that the restore index is present inside the data directory.
-    if (location == null) {
-      location = core.getDataDir();
+    if ("file".equals(repo.createURI("x").getScheme())) {
+      core.getCoreContainer().assertPathAllowed(Paths.get(location));
     }
 
     URI locationUri = repo.createURI(location);
@@ -575,8 +579,11 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
           location = core.getCoreDescriptor().getInstanceDir().resolve(location).normalize().toString();
         }
       }
+      if ("file".equals(repo.createURI("x").getScheme())) {
+        core.getCoreContainer().assertPathAllowed(Paths.get(location));
+      }
 
-      // small race here before the commit point is saved
+        // small race here before the commit point is saved
       URI locationUri = repo.createURI(location);
       String commitName = params.get(CoreAdminParams.COMMIT_NAME);
       SnapShooter snapShooter = new SnapShooter(repo, core, locationUri, params.get(NAME), commitName);
diff --git a/solr/core/src/java/org/apache/solr/handler/SnapShooter.java b/solr/core/src/java/org/apache/solr/handler/SnapShooter.java
index c238d55..10c0c8a 100644
--- a/solr/core/src/java/org/apache/solr/handler/SnapShooter.java
+++ b/solr/core/src/java/org/apache/solr/handler/SnapShooter.java
@@ -86,6 +86,9 @@ public class SnapShooter {
     this.backupRepo = Objects.requireNonNull(backupRepo);
     this.baseSnapDirPath = location;
     this.snapshotName = snapshotName;
+    if ("file".equals(location.getScheme())) {
+      solrCore.getCoreContainer().assertPathAllowed(Paths.get(location));
+    }
     if (snapshotName != null) {
       directoryName = "snapshot." + snapshotName;
     } else {
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/MergeIndexesOp.java b/solr/core/src/java/org/apache/solr/handler/admin/MergeIndexesOp.java
index 90690ff..d9f5de3 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/MergeIndexesOp.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/MergeIndexesOp.java
@@ -18,10 +18,8 @@
 package org.apache.solr.handler.admin;
 
 import java.lang.invoke.MethodHandles;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
+import java.nio.file.Paths;
+import java.util.*;
 
 import com.google.common.collect.Lists;
 import org.apache.lucene.index.DirectoryReader;
@@ -79,6 +77,8 @@ class MergeIndexesOp implements CoreAdminHandler.CoreAdminOp {
           sourceCores.add(srcCore);
         }
       } else {
+        // Validate each 'indexDir' input as valid
+        Arrays.stream(dirNames).forEach(indexDir -> core.getCoreContainer().assertPathAllowed(Paths.get(indexDir)));
         DirectoryFactory dirFactory = core.getDirectoryFactory();
         for (int i = 0; i < dirNames.length; i++) {
           boolean markAsDone = false;
diff --git a/solr/core/src/test/org/apache/solr/cloud/BasicDistributedZk2Test.java b/solr/core/src/test/org/apache/solr/cloud/BasicDistributedZk2Test.java
index 73487d7..dc2e9a7 100644
--- a/solr/core/src/test/org/apache/solr/cloud/BasicDistributedZk2Test.java
+++ b/solr/core/src/test/org/apache/solr/cloud/BasicDistributedZk2Test.java
@@ -403,8 +403,9 @@ public class BasicDistributedZk2Test extends AbstractFullDistribZkTestBase {
       params.set("qt", ReplicationHandler.PATH);
       params.set("command", "backup");
       params.set("name", backupName);
-      Path location = createTempDir();
-      location = FilterPath.unwrap(location).toRealPath();
+      final Path location = FilterPath.unwrap(createTempDir()).toRealPath();
+      // Allow non-standard location outside SOLR_HOME
+      jettys.forEach(j -> j.getCoreContainer().getAllowPaths().add(location));
       params.set("location", location.toString());
 
       QueryRequest request = new QueryRequest(params);
diff --git a/solr/core/src/test/org/apache/solr/cloud/api/collections/AbstractCloudBackupRestoreTestCase.java b/solr/core/src/test/org/apache/solr/cloud/api/collections/AbstractCloudBackupRestoreTestCase.java
index e4bb328..c59b906 100644
--- a/solr/core/src/test/org/apache/solr/cloud/api/collections/AbstractCloudBackupRestoreTestCase.java
+++ b/solr/core/src/test/org/apache/solr/cloud/api/collections/AbstractCloudBackupRestoreTestCase.java
@@ -47,6 +47,7 @@ import org.apache.solr.common.cloud.ImplicitDocRouter;
 import org.apache.solr.common.cloud.Slice;
 import org.apache.solr.common.params.CollectionAdminParams;
 import org.apache.solr.common.params.CoreAdminParams;
+import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
 import org.slf4j.Logger;
@@ -72,6 +73,12 @@ public abstract class AbstractCloudBackupRestoreTestCase extends SolrCloudTestCa
   @BeforeClass
   public static void createCluster() throws Exception {
     docsSeed = random().nextLong();
+    System.setProperty("solr.allowPaths", "*");
+  }
+
+  @AfterClass
+  public static void afterClass() throws Exception {
+    System.clearProperty("solr.allowPaths");
   }
 
   /**
diff --git a/solr/core/src/test/org/apache/solr/core/snapshots/TestSolrCloudSnapshots.java b/solr/core/src/test/org/apache/solr/core/snapshots/TestSolrCloudSnapshots.java
index 328f069..4a3678b 100644
--- a/solr/core/src/test/org/apache/solr/core/snapshots/TestSolrCloudSnapshots.java
+++ b/solr/core/src/test/org/apache/solr/core/snapshots/TestSolrCloudSnapshots.java
@@ -65,6 +65,7 @@ public class TestSolrCloudSnapshots extends SolrCloudTestCase {
   @BeforeClass
   public static void setupClass() throws Exception {
     useFactory("solr.StandardDirectoryFactory");
+    System.setProperty("solr.allowPaths", "*");
     configureCluster(NUM_NODES)// nodes
         .addConfig("conf1", TEST_PATH().resolve("configsets").resolve("cloud-minimal").resolve("conf"))
         .configure();
@@ -76,6 +77,7 @@ public class TestSolrCloudSnapshots extends SolrCloudTestCase {
   public static void teardownClass() throws Exception {
     System.clearProperty("test.build.data");
     System.clearProperty("test.cache.data");
+    System.clearProperty("solr.allowPaths");
   }
 
   @Test
diff --git a/solr/core/src/test/org/apache/solr/core/snapshots/TestSolrCoreSnapshots.java b/solr/core/src/test/org/apache/solr/core/snapshots/TestSolrCoreSnapshots.java
index 92b3c45..751d61e 100644
--- a/solr/core/src/test/org/apache/solr/core/snapshots/TestSolrCoreSnapshots.java
+++ b/solr/core/src/test/org/apache/solr/core/snapshots/TestSolrCoreSnapshots.java
@@ -65,6 +65,7 @@ public class TestSolrCoreSnapshots extends SolrCloudTestCase {
 
   @BeforeClass
   public static void setupClass() throws Exception {
+    System.setProperty("solr.allowPaths", "*");
     useFactory("solr.StandardDirectoryFactory");
     configureCluster(1)// nodes
         .addConfig("conf1", TEST_PATH().resolve("configsets").resolve("cloud-minimal").resolve("conf"))
@@ -76,6 +77,7 @@ public class TestSolrCoreSnapshots extends SolrCloudTestCase {
   public static void teardownClass() throws Exception {
     System.clearProperty("test.build.data");
     System.clearProperty("test.cache.data");
+    System.clearProperty("solr.allowPaths");
   }
 
   @Test
diff --git a/solr/core/src/test/org/apache/solr/handler/TestCoreBackup.java b/solr/core/src/test/org/apache/solr/handler/TestCoreBackup.java
index ee87253..858b807 100644
--- a/solr/core/src/test/org/apache/solr/handler/TestCoreBackup.java
+++ b/solr/core/src/test/org/apache/solr/handler/TestCoreBackup.java
@@ -16,16 +16,11 @@
  */
 package org.apache.solr.handler;
 
-import java.io.File;
-import java.io.IOException;
-import java.util.Arrays;
-
-import org.apache.lucene.index.IndexCommit;
 import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.IndexCommit;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.FSDirectory;
 import org.apache.lucene.util.TestUtil;
-
 import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.common.params.CoreAdminParams;
 import org.apache.solr.core.CoreContainer;
@@ -34,8 +29,12 @@ import org.apache.solr.response.SolrQueryResponse;
 import org.junit.After;
 import org.junit.Before;
 
-public class TestCoreBackup extends SolrTestCaseJ4 {
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.Arrays;
 
+public class TestCoreBackup extends SolrTestCaseJ4 {
   @Before // unique core per test
   public void coreInit() throws Exception {
     initCore("solrconfig.xml", "schema.xml");
@@ -63,6 +62,7 @@ public class TestCoreBackup extends SolrTestCaseJ4 {
     String snapshotName = TestUtil.randomSimpleString(random(), 1, 5);
 
     final CoreContainer cores = h.getCoreContainer();
+    cores.getAllowPaths().add(Paths.get(location));
     try (final CoreAdminHandler admin = new CoreAdminHandler(cores)) {
       SolrQueryResponse resp = new SolrQueryResponse();
       admin.handleRequestBody
@@ -96,7 +96,8 @@ public class TestCoreBackup extends SolrTestCaseJ4 {
     final CoreAdminHandler admin = new CoreAdminHandler(cores);
 
     final File backupDir = createTempDir().toFile();
-    
+    cores.getAllowPaths().add(backupDir.toPath());
+
     { // first a backup before we've ever done *anything*...
       SolrQueryResponse resp = new SolrQueryResponse();
       admin.handleRequestBody
@@ -197,8 +198,9 @@ public class TestCoreBackup extends SolrTestCaseJ4 {
     final CoreAdminHandler admin = new CoreAdminHandler(cores);
     
     final File backupDir = createTempDir().toFile();
-    
-    
+    cores.getAllowPaths().add(backupDir.toPath());
+
+
     { // take an initial 'backup1a' containing our 1 document
       final SolrQueryResponse resp = new SolrQueryResponse();
       admin.handleRequestBody
diff --git a/solr/core/src/test/org/apache/solr/handler/TestReplicationHandler.java b/solr/core/src/test/org/apache/solr/handler/TestReplicationHandler.java
index e5c5c71..2c91e3a 100644
--- a/solr/core/src/test/org/apache/solr/handler/TestReplicationHandler.java
+++ b/solr/core/src/test/org/apache/solr/handler/TestReplicationHandler.java
@@ -1618,7 +1618,9 @@ public class TestReplicationHandler extends SolrTestCaseJ4 {
   public void testEmptyBackups() throws Exception {
     final File backupDir = createTempDir().toFile();
     final BackupStatusChecker backupStatus = new BackupStatusChecker(masterClient);
-    
+
+    masterJetty.getCoreContainer().getAllowPaths().add(backupDir.toPath());
+
     { // initial request w/o any committed docs
       final String backupName = "empty_backup1";
       final GenericSolrRequest req = new GenericSolrRequest
diff --git a/solr/core/src/test/org/apache/solr/handler/TestRestoreCore.java b/solr/core/src/test/org/apache/solr/handler/TestRestoreCore.java
index 8b04acc..77e43cb 100644
--- a/solr/core/src/test/org/apache/solr/handler/TestRestoreCore.java
+++ b/solr/core/src/test/org/apache/solr/handler/TestRestoreCore.java
@@ -125,6 +125,7 @@ public class TestRestoreCore extends SolrJettyTestBase {
     //Use the default backup location or an externally provided location.
     if (random().nextBoolean()) {
       location = createTempDir().toFile().getAbsolutePath();
+      masterJetty.getCoreContainer().getAllowPaths().add(Paths.get(location)); // Allow core to be created outside SOLR_HOME
       params += "&location=" + URLEncoder.encode(location, "UTF-8");
     }
 
@@ -181,11 +182,21 @@ public class TestRestoreCore extends SolrJettyTestBase {
 
   }
 
+  public void testBackupFailsMissingAllowPaths() throws Exception {
+    final String params = "&location=" + URLEncoder.encode(createTempDir().toFile().getAbsolutePath(), "UTF-8");
+    Throwable t = expectThrows(IOException.class, () -> {
+      TestReplicationHandlerBackup.runBackupCommand(masterJetty, ReplicationHandler.CMD_BACKUP, params);
+    });
+    // The backup command will fail since the tmp dir is outside allowPaths
+    assertTrue(t.getMessage().contains("Server returned HTTP response code: 400"));
+  }
+
   @Test
   public void testFailedRestore() throws Exception {
     int nDocs = BackupRestoreUtils.indexDocs(masterClient, "collection1", docsSeed);
 
     String location = createTempDir().toFile().getAbsolutePath();
+    masterJetty.getCoreContainer().getAllowPaths().add(Paths.get(location));
     String snapshotName = TestUtil.randomSimpleString(random(), 1, 5);
     String params = "&name=" + snapshotName + "&location=" + URLEncoder.encode(location, "UTF-8");
     String baseUrl = masterJetty.getBaseUrl().toString();
diff --git a/solr/core/src/test/org/apache/solr/handler/V2ApiIntegrationTest.java b/solr/core/src/test/org/apache/solr/handler/V2ApiIntegrationTest.java
index 4620c94..5db5883 100644
--- a/solr/core/src/test/org/apache/solr/handler/V2ApiIntegrationTest.java
+++ b/solr/core/src/test/org/apache/solr/handler/V2ApiIntegrationTest.java
@@ -17,8 +17,8 @@
 
 package org.apache.solr.handler;
 
-
 import java.io.IOException;
+import java.nio.file.Paths;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -176,6 +176,7 @@ public class V2ApiIntegrationTest extends SolrCloudTestCase {
     backupParams.put("name", "backup_test");
     backupParams.put("collection", COLL_NAME);
     backupParams.put("location", tempDir);
+    cluster.getJettySolrRunners().forEach(j -> j.getCoreContainer().getAllowPaths().add(Paths.get(tempDir)));
     client.request(new V2Request.Builder("/c")
         .withMethod(SolrRequest.METHOD.POST)
         .withPayload(Utils.toJSONString(backupPayload))
diff --git a/solr/core/src/test/org/apache/solr/handler/admin/CoreMergeIndexesAdminHandlerTest.java b/solr/core/src/test/org/apache/solr/handler/admin/CoreMergeIndexesAdminHandlerTest.java
index 6fdccc7..9ceaa8a 100644
--- a/solr/core/src/test/org/apache/solr/handler/admin/CoreMergeIndexesAdminHandlerTest.java
+++ b/solr/core/src/test/org/apache/solr/handler/admin/CoreMergeIndexesAdminHandlerTest.java
@@ -16,9 +16,7 @@
  */
 package org.apache.solr.handler.admin;
 
-import java.io.File;
-import java.io.IOException;
-
+import com.carrotsearch.randomizedtesting.rules.SystemPropertiesRestoreRule;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.LockFactory;
 import org.apache.solr.SolrTestCaseJ4;
@@ -35,7 +33,8 @@ import org.junit.Test;
 import org.junit.rules.RuleChain;
 import org.junit.rules.TestRule;
 
-import com.carrotsearch.randomizedtesting.rules.SystemPropertiesRestoreRule;
+import java.io.File;
+import java.io.IOException;
 
 public class CoreMergeIndexesAdminHandlerTest extends SolrTestCaseJ4 {
   
@@ -74,6 +73,7 @@ public class CoreMergeIndexesAdminHandlerTest extends SolrTestCaseJ4 {
     final File workDir = createTempDir().toFile();
 
     final CoreContainer cores = h.getCoreContainer();
+    cores.getAllowPaths().add(workDir.toPath());
 
     try (final CoreAdminHandler admin = new CoreAdminHandler(cores);
          SolrCore core = cores.getCore("collection1")) {
diff --git a/solr/solr-ref-guide/src/index-replication.adoc b/solr/solr-ref-guide/src/index-replication.adoc
index 592e57f..ce7d66e 100644
--- a/solr/solr-ref-guide/src/index-replication.adoc
+++ b/solr/solr-ref-guide/src/index-replication.adoc
@@ -328,7 +328,30 @@ This command is useful for making periodic backups. There are several supported
 +
 * `numberToKeep:`: This can be used with the backup command unless the `maxNumberOfBackups` initialization parameter has been specified on the handler – in which case `maxNumberOfBackups` is always used and attempts to use the `numberToKeep` request parameter will cause an error.
 * `name`: (optional) Backup name. The snapshot will be created in a directory called `snapshot.<name>` within the data directory of the core. By default the name is generated using date in `yyyyMMddHHmmssSSS` format. If `location` parameter is passed, that would be used instead of the data directory
-* `location`: Backup location.
+* `repository`: The name of the backup repository to use. When not specified, it defaults to local file system.
+* `location`: Backup location. Value depends on the repository in use. For file system repository, location defaults to core's dataDir, and if specified, it needs to be within `SOLR_HOME`, `SOLR_DATA_HOME` or the paths specified by solr.xml `allowPaths`.
+
+`restore`::
+Restore a backup from a backup repository.
++
+[source,bash]
+http://_master_host:port_/solr/_core_name_/replication?command=restore
++
+This command is used to restore a backup. There are several supported request parameters:
++
+* `name`: (optional) Backup name. The name of the backed up index snapshot to be restored. If the name is not provided, it looks for backups with snapshot.<timestamp> format in the location directory. It picks the latest timestamp backup in that case.
+* `repository`: The name of the backup repository where the backup resides. When not specified, it defaults to local file system.
+* `location`: Backup location. Value depends on the repository in use. For file system repository, location defaults to core's dataDir, and if specified, it needs to be within `SOLR_HOME`, `SOLR_DATA_HOME` or the paths specified by solr.xml `allowPaths`.
+
+`restorestatus`::
+Check the status of a running restore operation.
++
+[source,bash]
+http://_master_host:port_/solr/_core_name_/replication?command=restorestatus
++
+This command is used to check the status of a restore operation. This command takes no parameters.
++
+The status value can be "In Progress" , "success" or "failed". If it failed then an "exception" will also be sent in the response.
 
 `deletebackup`::
 Delete any backup created using the `backup` command.
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/MergeIndexesExampleTestBase.java b/solr/solrj/src/test/org/apache/solr/client/solrj/MergeIndexesExampleTestBase.java
index e542203..23a70ce 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/MergeIndexesExampleTestBase.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/MergeIndexesExampleTestBase.java
@@ -77,7 +77,8 @@ public abstract class MergeIndexesExampleTestBase extends SolrTestCaseJ4 {
     if (log.isInfoEnabled()) {
       log.info("CORES={} : {}", cores, cores.getLoadedCoreNames());
     }
-
+    cores.getAllowPaths().add(dataDir1.toPath());
+    cores.getAllowPaths().add(dataDir2.toPath());
   }
 
   @Override
diff --git a/solr/test-framework/src/java/org/apache/solr/cloud/MiniSolrCloudCluster.java b/solr/test-framework/src/java/org/apache/solr/cloud/MiniSolrCloudCluster.java
index 0507c6f..d552e6d 100644
--- a/solr/test-framework/src/java/org/apache/solr/cloud/MiniSolrCloudCluster.java
+++ b/solr/test-framework/src/java/org/apache/solr/cloud/MiniSolrCloudCluster.java
@@ -91,6 +91,7 @@ public class MiniSolrCloudCluster {
   public static final String DEFAULT_CLOUD_SOLR_XML = "<solr>\n" +
       "\n" +
       "  <str name=\"shareSchema\">${shareSchema:false}</str>\n" +
+      "  <str name=\"allowPaths\">${solr.allowPaths:}</str>\n" +
       "  <str name=\"configSetBaseDir\">${configSetBaseDir:configsets}</str>\n" +
       "  <str name=\"coreRootDirectory\">${coreRootDirectory:.}</str>\n" +
       "  <str name=\"collectionsHandler\">${collectionsHandler:solr.CollectionsHandler}</str>\n" +