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/18 14:14:04 UTC

[lucene-solr] branch master updated: SOLR-14561 CoreAdminAPI's parameters instanceDir and dataDir are now validated (#1572)

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

janhoy 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 936b9d7  SOLR-14561 CoreAdminAPI's parameters instanceDir and dataDir are now validated (#1572)
936b9d7 is described below

commit 936b9d770e769c9018a9f408d576f52e7c4e8be2
Author: Jan Høydahl <ja...@users.noreply.github.com>
AuthorDate: Thu Jun 18 16:13:52 2020 +0200

    SOLR-14561 CoreAdminAPI's parameters instanceDir and dataDir are now validated (#1572)
---
 solr/CHANGES.txt                                   |  4 +
 solr/bin/solr.in.cmd                               |  4 +
 solr/bin/solr.in.sh                                |  5 ++
 .../java/org/apache/solr/core/CoreContainer.java   | 43 ++++++++++-
 .../src/java/org/apache/solr/core/NodeConfig.java  | 20 ++++-
 .../src/java/org/apache/solr/core/SolrPaths.java   | 35 +++++++++
 .../java/org/apache/solr/core/SolrXmlConfig.java   | 16 ++++
 solr/core/src/test-files/solr/solr-50-all.xml      |  1 +
 .../core/src/test-files/solr/solr-solrreporter.xml |  2 +
 solr/core/src/test-files/solr/solr.xml             |  1 +
 .../apache/solr/cloud/CollectionsAPISolrJTest.java |  4 +-
 .../apache/solr/cloud/UnloadDistributedZkTest.java |  2 +
 .../CollectionsAPIDistributedZkTest.java           |  2 +
 .../org/apache/solr/core/TestCoreContainer.java    | 85 ++++++++++++++++++++++
 .../src/test/org/apache/solr/core/TestSolrXml.java | 10 +++
 .../solr/handler/admin/CoreAdminHandlerTest.java   |  2 +
 .../handler/admin/CoreAdminRequestStatusTest.java  |  1 +
 .../reporters/solr/SolrShardReporterTest.java      | 12 +++
 .../apache/solr/update/SolrIndexSplitterTest.java  |  3 +
 solr/server/solr/solr.xml                          |  1 +
 solr/solr-ref-guide/src/coreadmin-api.adoc         |  4 +-
 solr/solr-ref-guide/src/format-of-solr-xml.adoc    |  4 +
 .../solr/client/solrj/request/TestCoreAdmin.java   |  4 +-
 23 files changed, 257 insertions(+), 8 deletions(-)

diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 0bc9f7f..23ceea9 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -147,6 +147,10 @@ Improvements
   Also fixes SolrIndexSearcher.warm which should have re-instated previous SRI.
   (Nazerke Seidan, David Smiley)
 
+* SOLR-14561: CoreAdminAPI's parameters instanceDir and dataDir are now validated, and must be relative to either
+  SOLR_HOME, SOLR_DATA_HOME or coreRootDir. Added new solr.xml config 'allowPaths', controlled by system property
+  'solr.allowPaths' that allows you to add other allowed paths when needed.
+
 Optimizations
 ---------------------
 * SOLR-8306: Do not collect expand documents when expand.rows=0 (Marshall Sanders, Amelia Henderson)
diff --git a/solr/bin/solr.in.cmd b/solr/bin/solr.in.cmd
index 6aa392c..c8a6c8c 100755
--- a/solr/bin/solr.in.cmd
+++ b/solr/bin/solr.in.cmd
@@ -209,3 +209,7 @@ REM change the value to true. The option is configured as a system property as d
 REM scripts.
 REM set SOLR_ADMIN_UI_DISABLED=false
 
+REM Solr is by default allowed to read and write data from/to SOLR_HOME and a few other well defined locations
+REM Sometimes it may be necessary to place a core or a backup on a different location or a different disk
+REM This parameter lets you specify file system path(s) to explicitly allow. The special value of '*' will allow any path
+REM SOLR_OPTS="%SOLR_OPTS% -Dsolr.allowPaths=D:\,E:\other\path"
diff --git a/solr/bin/solr.in.sh b/solr/bin/solr.in.sh
index 6774d12..3ecd7b9 100644
--- a/solr/bin/solr.in.sh
+++ b/solr/bin/solr.in.sh
@@ -239,3 +239,8 @@
 # change the value to true. The option is configured as a system property as defined in SOLR_START_OPTS in the start
 # scripts.
 # SOLR_ADMIN_UI_DISABLED=false
+
+# Solr is by default allowed to read and write data from/to SOLR_HOME and a few other well defined locations
+# Sometimes it may be necessary to place a core or a backup on a different location or a different disk
+# This parameter lets you specify file system path(s) to explicitly allow. The special value of '*' will allow any path
+#SOLR_OPTS="$SOLR_OPTS -Dsolr.allowPaths=/mnt/bigdisk,/other/path"
diff --git a/solr/core/src/java/org/apache/solr/core/CoreContainer.java b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
index dd98ac6..71ef939 100644
--- a/solr/core/src/java/org/apache/solr/core/CoreContainer.java
+++ b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
@@ -42,6 +42,7 @@ import java.util.concurrent.ExecutionException;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Future;
 
+import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Maps;
 import org.apache.commons.lang3.StringUtils;
@@ -238,6 +239,7 @@ public class CoreContainer {
   private PackageStoreAPI packageStoreAPI;
   private PackageLoader packageLoader;
 
+  private Set<Path> allowPaths;
 
   // Bits for the state variable.
   public final static long LOAD_COMPLETE = 0x1L;
@@ -338,6 +340,19 @@ public class CoreContainer {
         ExecutorUtil.newMDCAwareCachedThreadPool(
             cfg.getReplayUpdatesThreads(),
             new SolrNamedThreadFactory("replayUpdatesExecutor")));
+
+    this.allowPaths = new java.util.HashSet<>();
+    this.allowPaths.add(cfg.getSolrHome());
+    this.allowPaths.add(cfg.getCoreRootDirectory());
+    if (cfg.getSolrDataHome() != null) {
+      this.allowPaths.add(cfg.getSolrDataHome());
+    }
+    if (!cfg.getAllowPaths().isEmpty()) {
+      this.allowPaths.addAll(cfg.getAllowPaths());
+      if (log.isInfoEnabled()) {
+        log.info("Allowing use of paths: {}", cfg.getAllowPaths());
+      }
+    }
   }
 
   @SuppressWarnings({"unchecked"})
@@ -1200,6 +1215,10 @@ public class CoreContainer {
       throw new SolrException(ErrorCode.SERVER_ERROR, "Core with name '" + coreName + "' already exists.");
     }
 
+    // Validate paths are relative to known locations to avoid path traversal
+    assertPathAllowed(cd.getInstanceDir());
+    assertPathAllowed(Paths.get(cd.getDataDir()));
+
     boolean preExisitingZkEntry = false;
     try {
       if (getZkController() != null) {
@@ -1260,6 +1279,29 @@ public class CoreContainer {
   }
 
   /**
+   * Checks that the given path is relative to SOLR_HOME, SOLR_DATA_HOME, coreRootDirectory or one of the paths
+   * specified in solr.xml's allowPaths element. Delegates to {@link SolrPaths#assertPathAllowed(Path, Set)}
+   * @param pathToAssert path to check
+   * @throws SolrException if path is outside allowed paths
+   */
+  public void assertPathAllowed(Path pathToAssert) throws SolrException {
+    SolrPaths.assertPathAllowed(pathToAssert, allowPaths);
+  }
+
+  /**
+   * <p>Return the file system paths that should be allowed for various API requests.
+   * This list is compiled at startup from SOLR_HOME, SOLR_DATA_HOME and the
+   * <code>allowPaths</code> configuration of solr.xml.
+   * These paths are used by the {@link #assertPathAllowed(Path)} method call.</p>
+   * <p><b>NOTE:</b></p> This method is currently only in use in tests in order to
+   * modify the mutable Set directly. Please treat this as a private method.
+   */
+  @VisibleForTesting
+  public Set<Path> getAllowPaths() {
+    return allowPaths;
+  }
+
+  /**
    * Creates a new core based on a CoreDescriptor.
    *
    * @param dcore        a core descriptor
@@ -1502,7 +1544,6 @@ public class CoreContainer {
     return ImmutableMap.copyOf(coreInitFailures);
   }
 
-
   // ---------------- Core name related methods ---------------
 
   private CoreDescriptor reloadCoreDescriptor(CoreDescriptor oldDesc) {
diff --git a/solr/core/src/java/org/apache/solr/core/NodeConfig.java b/solr/core/src/java/org/apache/solr/core/NodeConfig.java
index 353d83e..ef8fddb 100644
--- a/solr/core/src/java/org/apache/solr/core/NodeConfig.java
+++ b/solr/core/src/java/org/apache/solr/core/NodeConfig.java
@@ -18,6 +18,7 @@ package org.apache.solr.core;
 
 import java.nio.file.Path;
 import java.util.Arrays;
+import java.util.Collections;
 import java.util.HashSet;
 import java.util.Properties;
 import java.util.Set;
@@ -41,6 +42,8 @@ public class NodeConfig {
   
   private final Path configSetBaseDirectory;
 
+  private final Set<Path> allowPaths;
+
   private final String sharedLibDirectory;
 
   private final PluginInfo shardHandlerFactoryConfig;
@@ -95,7 +98,7 @@ public class NodeConfig {
                      Path solrHome, SolrResourceLoader loader,
                      Properties solrProperties, PluginInfo[] backupRepositoryPlugins,
                      MetricsConfig metricsConfig, PluginInfo transientCacheConfig, PluginInfo tracerConfig,
-                     boolean fromZookeeper) {
+                     boolean fromZookeeper, Set<Path> allowPaths) {
     // all Path params here are absolute and normalized.
     this.nodeName = nodeName;
     this.coreRootDirectory = coreRootDirectory;
@@ -125,6 +128,7 @@ public class NodeConfig {
     this.transientCacheConfig = transientCacheConfig;
     this.tracerConfig = tracerConfig;
     this.fromZookeeper = fromZookeeper;
+    this.allowPaths = allowPaths;
 
     if (this.cloudConfig != null && this.getCoreLoadThreadCount(false) < 2) {
       throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
@@ -263,6 +267,12 @@ public class NodeConfig {
     return fromZookeeper;
   }
 
+  /**
+   * Extra file paths that will be allowed for core creation, in addition to
+   * SOLR_HOME, SOLR_DATA_HOME and coreRootDir
+   */
+  public Set<Path> getAllowPaths() { return allowPaths; }
+
   public static class NodeConfigBuilder {
     // all Path fields here are absolute and normalized.
     private SolrResourceLoader loader;
@@ -293,6 +303,7 @@ public class NodeConfig {
     private PluginInfo transientCacheConfig;
     private PluginInfo tracerConfig;
     private boolean fromZookeeper = false;
+    private Set<Path> allowPaths = Collections.emptySet();
 
     private final Path solrHome;
     private final String nodeName;
@@ -457,6 +468,11 @@ public class NodeConfig {
       return this;
     }
 
+    public NodeConfigBuilder setAllowPaths(Set<Path> paths) {
+      this.allowPaths = paths;
+      return this;
+    }
+
     public NodeConfig build() {
       // if some things weren't set then set them now.  Simple primitives are set on the field declaration
       if (loader == null) {
@@ -467,7 +483,7 @@ public class NodeConfig {
                             updateShardHandlerConfig, coreAdminHandlerClass, collectionsAdminHandlerClass, healthCheckHandlerClass, infoHandlerClass, configSetsHandlerClass,
                             logWatcherConfig, cloudConfig, coreLoadThreads, replayUpdatesThreads, transientCacheSize, useSchemaCache, managementPath,
                             solrHome, loader, solrProperties,
-                            backupRepositoryPlugins, metricsConfig, transientCacheConfig, tracerConfig, fromZookeeper);
+                            backupRepositoryPlugins, metricsConfig, transientCacheConfig, tracerConfig, fromZookeeper, allowPaths);
     }
 
     public NodeConfigBuilder setSolrResourceLoader(SolrResourceLoader resourceLoader) {
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 344a67a..c72bff7 100644
--- a/solr/core/src/java/org/apache/solr/core/SolrPaths.java
+++ b/solr/core/src/java/org/apache/solr/core/SolrPaths.java
@@ -28,6 +28,8 @@ import java.nio.file.Paths;
 import java.util.Set;
 import java.util.concurrent.ConcurrentSkipListSet;
 
+import org.apache.commons.exec.OS;
+import org.apache.solr.common.SolrException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -128,4 +130,37 @@ public final class SolrPaths {
       log.info(msg);
     }
   }
+
+  /**
+   * Checks that the given path is relative to one of the allowPaths supplied. Typically this will be
+   * called from {@link CoreContainer#assertPathAllowed(Path)} and allowPaths pre-filled with the node's
+   * SOLR_HOME, SOLR_DATA_HOME and coreRootDirectory folders, as well as any paths specified in
+   * solr.xml's allowPaths element. The following paths will always fail validation:
+   * <ul>
+   *   <li>Relative paths starting with <code>..</code></li>
+   *   <li>Windows UNC paths (such as <code>\\host\share\path</code>)</li>
+   *   <li>Paths which are not relative to any of allowPaths</li>
+   * </ul>
+   * @param pathToAssert path to check
+   * @param allowPaths list of paths that should be allowed prefixes for pathToAssert
+   * @throws SolrException if path is outside allowed paths
+   */
+  public static void assertPathAllowed(Path pathToAssert, Set<Path> allowPaths) throws SolrException {
+    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.");
+    }
+    // Conversion Path -> String -> Path is to be able to compare against org.apache.lucene.mockfile.FilterPath instances
+    final Path path = Path.of(pathToAssert.toString()).normalize();
+    if (path.startsWith("..")) {
+      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
+          "Path " + pathToAssert + " disallowed due to path traversal..");
+    }
+    if (!path.isAbsolute()) return; // All relative paths are accepted
+    if (allowPaths.contains(Paths.get("_ALL_"))) return; // Catch-all path "*"/"_ALL_" will allow all other paths
+    if (allowPaths.stream().noneMatch(p -> path.startsWith(Path.of(p.toString())))) {
+      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
+          "Path " + path + " must be relative to SOLR_HOME, SOLR_DATA_HOME coreRootDirectory. Set system property 'solr.allowPaths' to add other allowed paths.");
+    }
+  }
 }
diff --git a/solr/core/src/java/org/apache/solr/core/SolrXmlConfig.java b/solr/core/src/java/org/apache/solr/core/SolrXmlConfig.java
index a379ae6..1c96d2b 100644
--- a/solr/core/src/java/org/apache/solr/core/SolrXmlConfig.java
+++ b/solr/core/src/java/org/apache/solr/core/SolrXmlConfig.java
@@ -26,13 +26,17 @@ import java.lang.invoke.MethodHandles;
 import java.nio.charset.StandardCharsets;
 import java.nio.file.Files;
 import java.nio.file.Path;
+import java.nio.file.Paths;
 import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Properties;
 import java.util.Set;
+import java.util.stream.Collectors;
 
 import com.google.common.base.Strings;
 import org.apache.commons.io.IOUtils;
@@ -277,6 +281,9 @@ public class SolrXmlConfig {
         case "sharedLib":
           builder.setSharedLibDirectory(value);
           break;
+        case "allowPaths":
+          builder.setAllowPaths(stringToPaths(value));
+          break;
         case "configSetBaseDir":
           builder.setConfigSetBaseDirectory(value);
           break;
@@ -300,6 +307,15 @@ public class SolrXmlConfig {
     return builder.build();
   }
 
+  private static Set<Path> stringToPaths(String commaSeparatedString) {
+    if (Strings.isNullOrEmpty(commaSeparatedString)) {
+      return Collections.emptySet();
+    }
+    // Parse list of paths. The special value '*' is mapped to _ALL_ to mean all paths
+    return Arrays.stream(commaSeparatedString.split(",\\s?"))
+        .map(p -> Paths.get("*".equals(p) ? "_ALL_" : p)).collect(Collectors.toSet());
+  }
+
   private static UpdateShardHandlerConfig loadUpdateConfig(NamedList<Object> nl, boolean alwaysDefine) {
 
     if (nl == null && !alwaysDefine)
diff --git a/solr/core/src/test-files/solr/solr-50-all.xml b/solr/core/src/test-files/solr/solr-50-all.xml
index f69d904..736349f 100644
--- a/solr/core/src/test-files/solr/solr-50-all.xml
+++ b/solr/core/src/test-files/solr/solr-50-all.xml
@@ -24,6 +24,7 @@
   <str name="configSetsHandler">testConfigSetsHandler</str>
   <str name="managementPath">testManagementPath</str>
   <str name="sharedLib">testSharedLib</str>
+  <str name="allowPaths">${solr.allowPaths:}</str>
   <str name="shareSchema">${shareSchema:true}</str>
   <int name="transientCacheSize">66</int>
   <int name="replayUpdatesThreads">100</int>
diff --git a/solr/core/src/test-files/solr/solr-solrreporter.xml b/solr/core/src/test-files/solr/solr-solrreporter.xml
index f324962..2a74165 100644
--- a/solr/core/src/test-files/solr/solr-solrreporter.xml
+++ b/solr/core/src/test-files/solr/solr-solrreporter.xml
@@ -17,6 +17,8 @@
 -->
 
 <solr>
+  <str name="allowPaths">${solr.allowPaths:}</str>
+
   <shardHandlerFactory name="shardHandlerFactory" class="HttpShardHandlerFactory">
     <str name="urlScheme">${urlScheme:}</str>
     <int name="socketTimeout">${socketTimeout:90000}</int>
diff --git a/solr/core/src/test-files/solr/solr.xml b/solr/core/src/test-files/solr/solr.xml
index 5e30ffd..fb2791d 100644
--- a/solr/core/src/test-files/solr/solr.xml
+++ b/solr/core/src/test-files/solr/solr.xml
@@ -24,6 +24,7 @@
   <str name="shareSchema">${shareSchema:false}</str>
   <str name="configSetBaseDir">${configSetBaseDir:configsets}</str>
   <str name="coreRootDirectory">${coreRootDirectory:.}</str>
+  <str name="allowPaths">${solr.allowPaths:}</str>
 
   <shardHandlerFactory name="shardHandlerFactory" class="HttpShardHandlerFactory">
     <str name="urlScheme">${urlScheme:}</str>
diff --git a/solr/core/src/test/org/apache/solr/cloud/CollectionsAPISolrJTest.java b/solr/core/src/test/org/apache/solr/cloud/CollectionsAPISolrJTest.java
index a4d62fe..33a0a6d 100644
--- a/solr/core/src/test/org/apache/solr/cloud/CollectionsAPISolrJTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/CollectionsAPISolrJTest.java
@@ -38,6 +38,7 @@ import java.util.Optional;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicReference;
 
+import com.google.common.collect.ImmutableList;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.TestUtil;
 import org.apache.solr.client.solrj.SolrClient;
@@ -75,8 +76,6 @@ import org.apache.zookeeper.KeeperException;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
-
-import com.google.common.collect.ImmutableList;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -475,6 +474,7 @@ public class CollectionsAPISolrJTest extends SolrCloudTestCase {
     Path tmpDir = createTempDir("testPropertyParamsForCreate");
     Path dataDir = tmpDir.resolve("dataDir-" + TestUtil.randomSimpleString(random(), 1, 5));
     Path ulogDir = tmpDir.resolve("ulogDir-" + TestUtil.randomSimpleString(random(), 1, 5));
+    cluster.getJettySolrRunners().forEach(j -> j.getCoreContainer().getAllowPaths().add(tmpDir));
 
     CollectionAdminResponse response = CollectionAdminRequest.createCollection(collectionName, "conf", 1, 1)
         .withProperty(CoreAdminParams.DATA_DIR, dataDir.toString())
diff --git a/solr/core/src/test/org/apache/solr/cloud/UnloadDistributedZkTest.java b/solr/core/src/test/org/apache/solr/cloud/UnloadDistributedZkTest.java
index 3111517..2d9da76 100644
--- a/solr/core/src/test/org/apache/solr/cloud/UnloadDistributedZkTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/UnloadDistributedZkTest.java
@@ -40,6 +40,7 @@ import org.apache.solr.util.TimeOut;
 import org.junit.Test;
 
 import java.io.IOException;
+import java.nio.file.Path;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.Random;
@@ -64,6 +65,7 @@ public class UnloadDistributedZkTest extends BasicDistributedZkTest {
 
   @Test
   public void test() throws Exception {
+    jettys.forEach(j -> j.getCoreContainer().getAllowPaths().add(Path.of("_ALL_"))); // Allow non-standard core instance path
     testCoreUnloadAndLeaders(); // long
     testUnloadLotsOfCores(); // long
 
diff --git a/solr/core/src/test/org/apache/solr/cloud/api/collections/CollectionsAPIDistributedZkTest.java b/solr/core/src/test/org/apache/solr/cloud/api/collections/CollectionsAPIDistributedZkTest.java
index af3cd55..1d2c5b1 100644
--- a/solr/core/src/test/org/apache/solr/cloud/api/collections/CollectionsAPIDistributedZkTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/api/collections/CollectionsAPIDistributedZkTest.java
@@ -100,6 +100,7 @@ public class CollectionsAPIDistributedZkTest extends SolrCloudTestCase {
     System.setProperty("createCollectionWaitTimeTillActive", "5");
     TestInjection.randomDelayInCoreCreation = "true:5";
     System.setProperty("validateAfterInactivity", "200");
+    System.setProperty("solr.allowPaths", "*");
 
     configureCluster(4)
         .addConfig("conf", configset(getConfigSet()))
@@ -114,6 +115,7 @@ public class CollectionsAPIDistributedZkTest extends SolrCloudTestCase {
       shutdownCluster();
     } finally {
       System.clearProperty("createCollectionWaitTimeTillActive");
+      System.clearProperty("solr.allowPaths");
       super.tearDown();
     }
   }
diff --git a/solr/core/src/test/org/apache/solr/core/TestCoreContainer.java b/solr/core/src/test/org/apache/solr/core/TestCoreContainer.java
index 6d1f088..2ebeb79 100644
--- a/solr/core/src/test/org/apache/solr/core/TestCoreContainer.java
+++ b/solr/core/src/test/org/apache/solr/core/TestCoreContainer.java
@@ -19,16 +19,19 @@ package org.apache.solr.core;
 import java.io.File;
 import java.io.FileOutputStream;
 import java.nio.file.Path;
+import java.nio.file.Paths;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 import java.util.jar.JarEntry;
 import java.util.jar.JarOutputStream;
 import java.util.regex.Pattern;
 
 import com.google.common.base.Throwables;
 import com.google.common.collect.ImmutableMap;
+import org.apache.commons.exec.OS;
 import org.apache.commons.io.FileUtils;
 import org.apache.lucene.util.IOUtils;
 import org.apache.solr.SolrTestCaseJ4;
@@ -38,6 +41,7 @@ import org.apache.solr.handler.admin.ConfigSetsHandler;
 import org.apache.solr.handler.admin.CoreAdminHandler;
 import org.apache.solr.handler.admin.InfoHandler;
 import org.junit.AfterClass;
+import org.junit.Assume;
 import org.junit.BeforeClass;
 import org.junit.Test;
 import org.xml.sax.SAXParseException;
@@ -310,6 +314,11 @@ public class TestCoreContainer extends SolrTestCaseJ4 {
       "<str name=\"shareSchema\">${shareSchema:false}</str>\n" +
       "</solr>";
 
+  private static final String ALLOW_PATHS_SOLR_XML ="<?xml version=\"1.0\" encoding=\"UTF-8\" ?>\n" +
+      "<solr>\n" +
+      "<str name=\"allowPaths\">${solr.allowPaths:}</str>\n" +
+      "</solr>";
+
   private static final String CUSTOM_HANDLERS_SOLR_XML = "<?xml version=\"1.0\" encoding=\"UTF-8\" ?>\n" +
       "<solr>" +
       " <str name=\"collectionsHandler\">" + CustomCollectionsHandler.class.getName() + "</str>" +
@@ -343,6 +352,82 @@ public class TestCoreContainer extends SolrTestCaseJ4 {
   }
 
   @Test
+  public void assertAllowPathFromSolrXml() throws Exception {
+    Assume.assumeFalse(OS.isFamilyWindows());
+    System.setProperty("solr.allowPaths", "/var/solr");
+    CoreContainer cc = init(ALLOW_PATHS_SOLR_XML);
+    cc.assertPathAllowed(Paths.get("/var/solr/foo"));
+    try {
+      cc.assertPathAllowed(Paths.get("/tmp"));
+      fail("Path /tmp should not be allowed");
+    } catch(SolrException e) {
+      /* Ignore */
+    } finally {
+      cc.shutdown();
+      System.clearProperty("solr.allowPaths");
+    }
+  }
+
+  @Test
+  public void assertAllowPathFromSolrXmlWin() throws Exception {
+    Assume.assumeTrue(OS.isFamilyWindows());
+    System.setProperty("solr.allowPaths", "C:\\solr");
+    CoreContainer cc = init(ALLOW_PATHS_SOLR_XML);
+    cc.assertPathAllowed(Paths.get("C:\\solr\\foo"));
+    try {
+      cc.assertPathAllowed(Paths.get("C:\\tmp"));
+      fail("Path C:\\tmp should not be allowed");
+    } catch(SolrException e) {
+      /* Ignore */
+    } finally {
+      cc.shutdown();
+      System.clearProperty("solr.allowPaths");
+    }
+  }
+
+  @Test
+  public void assertAllowPath() {
+    Assume.assumeFalse(OS.isFamilyWindows());
+    assertPathAllowed("/var/solr/foo");
+    assertPathAllowed("/var/log/../solr/foo");
+    assertPathAllowed("relative");
+
+    assertPathBlocked("../../false");
+    assertPathBlocked("./../../false");
+    assertPathBlocked("/var/solr/../../etc");
+  }
+
+  @Test
+  public void assertAllowPathWindows() {
+    Assume.assumeTrue(OS.isFamilyWindows());
+    assertPathAllowed("C:\\var\\solr\\foo");
+    assertPathAllowed("C:\\var\\log\\..\\solr\\foo");
+    assertPathAllowed("relative");
+
+    assertPathBlocked("..\\..\\false");
+    assertPathBlocked(".\\../\\..\\false");
+    assertPathBlocked("C:\\var\\solr\\..\\..\\etc");
+
+    // UNC paths are always blocked
+    assertPathBlocked("\\\\unc-server\\share\\path");
+  }
+
+  private static Set<Path> ALLOWED_PATHS = Set.of(Path.of("/var/solr"));
+  private static Set<Path> ALLOWED_PATHS_WIN = Set.of(Path.of("C:\\var\\solr"));
+
+  private void assertPathBlocked(String path) {
+    try {
+
+      SolrPaths.assertPathAllowed(Path.of(path), OS.isFamilyWindows() ? ALLOWED_PATHS_WIN : ALLOWED_PATHS);
+      fail("Path " + path + " sould have been blocked.");
+    } catch (SolrException e) { /* Expected */ }
+  }
+
+  private void assertPathAllowed(String path) {
+    SolrPaths.assertPathAllowed(Path.of(path), OS.isFamilyWindows() ? ALLOWED_PATHS_WIN : ALLOWED_PATHS);
+  }
+
+  @Test
   public void testCustomHandlers() throws Exception {
 
     CoreContainer cc = init(CUSTOM_HANDLERS_SOLR_XML);
diff --git a/solr/core/src/test/org/apache/solr/core/TestSolrXml.java b/solr/core/src/test/org/apache/solr/core/TestSolrXml.java
index 2fcf971..f45ae27 100644
--- a/solr/core/src/test/org/apache/solr/core/TestSolrXml.java
+++ b/solr/core/src/test/org/apache/solr/core/TestSolrXml.java
@@ -22,8 +22,11 @@ import java.nio.file.Files;
 import java.nio.file.Path;
 import java.util.Locale;
 import java.util.Properties;
+import java.util.Set;
+import java.util.stream.Collectors;
 
 import com.carrotsearch.randomizedtesting.rules.SystemPropertiesRestoreRule;
+import org.apache.commons.exec.OS;
 import org.apache.lucene.util.TestUtil;
 import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.common.SolrException;
@@ -56,6 +59,7 @@ public class TestSolrXml extends SolrTestCaseJ4 {
     Path testSrcRoot = TEST_PATH();
     Files.copy(testSrcRoot.resolve("solr-50-all.xml"), solrHome.resolve("solr.xml"));
 
+    System.setProperty("solr.allowPaths", OS.isFamilyWindows() ? "C:\\tmp,C:\\home\\john" : "/tmp,/home/john");
     NodeConfig cfg = SolrXmlConfig.fromSolrHome(solrHome, new Properties());
     CloudConfig ccfg = cfg.getCloudConfig();
     UpdateShardHandlerConfig ucfg = cfg.getUpdateShardHandlerConfig();
@@ -98,6 +102,12 @@ public class TestSolrXml extends SolrTestCaseJ4 {
     assertEquals("a.b.C", backupRepoConfigs[0].className);
     assertEquals("true", backupRepoConfigs[0].attributes.get("default"));
     assertEquals(0, backupRepoConfigs[0].initArgs.size());
+    assertTrue("allowPaths", cfg.getAllowPaths().containsAll(OS.isFamilyWindows() ?
+            Set.of("C:\\tmp", "C:\\home\\john").stream().map(s -> Path.of(s)).collect(Collectors.toSet()) :
+            Set.of("/tmp", "/home/john").stream().map(s -> Path.of(s)).collect(Collectors.toSet())
+        )
+    );
+    System.clearProperty("solr.allowPaths");
   }
 
   // Test  a few property substitutions that happen to be in solr-50-all.xml.
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 7eae212..eab817b 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
@@ -82,6 +82,7 @@ public class CoreAdminHandlerTest extends SolrTestCaseJ4 {
         new File(subHome, "solrconfig.snippet.randomindexconfig.xml"));
 
     final CoreContainer cores = h.getCoreContainer();
+    cores.getAllowPaths().add(workDir.toPath());
 
     final CoreAdminHandler admin = new CoreAdminHandler(cores);
 
@@ -125,6 +126,7 @@ public class CoreAdminHandlerTest extends SolrTestCaseJ4 {
     final File workDir = createTempDir().toFile();
     
     final CoreContainer cores = h.getCoreContainer();
+    cores.getAllowPaths().add(workDir.toPath());
 
     final CoreAdminHandler admin = new CoreAdminHandler(cores);
 
diff --git a/solr/core/src/test/org/apache/solr/handler/admin/CoreAdminRequestStatusTest.java b/solr/core/src/test/org/apache/solr/handler/admin/CoreAdminRequestStatusTest.java
index 4152d37..d957733 100644
--- a/solr/core/src/test/org/apache/solr/handler/admin/CoreAdminRequestStatusTest.java
+++ b/solr/core/src/test/org/apache/solr/handler/admin/CoreAdminRequestStatusTest.java
@@ -42,6 +42,7 @@ public class CoreAdminRequestStatusTest extends SolrTestCaseJ4{
     final File workDir = createTempDir().toFile();
 
     final CoreContainer cores = h.getCoreContainer();
+    cores.getAllowPaths().add(workDir.toPath()); // Allow core to be created in workDir
 
     final CoreAdminHandler admin = new CoreAdminHandler(cores);
 
diff --git a/solr/core/src/test/org/apache/solr/metrics/reporters/solr/SolrShardReporterTest.java b/solr/core/src/test/org/apache/solr/metrics/reporters/solr/SolrShardReporterTest.java
index c8a6af4..3d8669a 100644
--- a/solr/core/src/test/org/apache/solr/metrics/reporters/solr/SolrShardReporterTest.java
+++ b/solr/core/src/test/org/apache/solr/metrics/reporters/solr/SolrShardReporterTest.java
@@ -32,6 +32,8 @@ import org.apache.solr.core.CoreDescriptor;
 import org.apache.solr.metrics.AggregateMetric;
 import org.apache.solr.metrics.SolrCoreMetricManager;
 import org.apache.solr.metrics.SolrMetricManager;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
 import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -46,6 +48,16 @@ public class SolrShardReporterTest extends AbstractFullDistribZkTestBase {
     schemaString = "schema15.xml";      // we need a string id
   }
 
+  @BeforeClass
+  public static void shardReporterBeforeClass() {
+    System.setProperty("solr.allowPaths", "*");
+  }
+
+  @AfterClass
+  public static void shardReporterAfterClass() {
+    System.clearProperty("solr.allowPaths");
+  }
+
   @Override
   public String getSolrXml() {
     return "solr-solrreporter.xml";
diff --git a/solr/core/src/test/org/apache/solr/update/SolrIndexSplitterTest.java b/solr/core/src/test/org/apache/solr/update/SolrIndexSplitterTest.java
index 0bcc851..fb64985 100644
--- a/solr/core/src/test/org/apache/solr/update/SolrIndexSplitterTest.java
+++ b/solr/core/src/test/org/apache/solr/update/SolrIndexSplitterTest.java
@@ -21,6 +21,7 @@ import java.io.UnsupportedEncodingException;
 import java.lang.invoke.MethodHandles;
 import java.nio.charset.StandardCharsets;
 import java.util.List;
+import java.util.Set;
 
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Lists;
@@ -55,6 +56,7 @@ public class SolrIndexSplitterTest extends SolrTestCaseJ4 {
    // System.setProperty("enable.update.log", "false"); // schema12 doesn't support _version_
     System.setProperty("solr.directoryFactory", "solr.NRTCachingDirectoryFactory");
     System.setProperty("solr.tests.lockType", DirectoryFactory.LOCK_TYPE_SIMPLE);
+
     initCore("solrconfig.xml", "schema15.xml");
   }
 
@@ -67,6 +69,7 @@ public class SolrIndexSplitterTest extends SolrTestCaseJ4 {
     indexDir1 = createTempDir("_testSplit1").toFile();
     indexDir2 = createTempDir("_testSplit2").toFile();
     indexDir3 = createTempDir("_testSplit3").toFile();
+    h.getCoreContainer().getAllowPaths().addAll(Set.of(indexDir1.toPath(), indexDir2.toPath(), indexDir3.toPath()));
   }
 
   @Test
diff --git a/solr/server/solr/solr.xml b/solr/server/solr/solr.xml
index 7fce0e8..1bb9b28 100644
--- a/solr/server/solr/solr.xml
+++ b/solr/server/solr/solr.xml
@@ -30,6 +30,7 @@
 
   <int name="maxBooleanClauses">${solr.max.booleanClauses:1024}</int>
   <str name="sharedLib">${solr.sharedLib:}</str>
+  <str name="allowPaths">${solr.allowPaths:}</str>
 
   <solrcloud>
 
diff --git a/solr/solr-ref-guide/src/coreadmin-api.adoc b/solr/solr-ref-guide/src/coreadmin-api.adoc
index 22b7d1c..90f0ab8 100644
--- a/solr/solr-ref-guide/src/coreadmin-api.adoc
+++ b/solr/solr-ref-guide/src/coreadmin-api.adoc
@@ -77,7 +77,7 @@ The `core.properties` file is built as part of the CREATE command. If you create
 The name of the new core. Same as `name` on the `<core>` element. This parameter is required.
 
 `instanceDir`::
-The directory where files for this core should be stored. Same as `instanceDir` on the `<core>` element. The default is the value specified for the `name` parameter if not supplied.
+The directory where files for this core should be stored. Same as `instanceDir` on the `<core>` element. The default is the value specified for the `name` parameter if not supplied. This directory must be inside `SOLR_HOME`, `SOLR_DATA_HOME` or one of the paths specified by system property `solr.allowPaths`.
 
 `config`::
 Name of the config file (i.e., `solrconfig.xml`) relative to `instanceDir`.
@@ -86,7 +86,7 @@ Name of the config file (i.e., `solrconfig.xml`) relative to `instanceDir`.
 Name of the schema file to use for the core. Please note that if you are using a "managed schema" (the default behavior) then any value for this property which does not match the effective `managedSchemaResourceName` will be read once, backed up, and converted for managed schema use. See <<schema-factory-definition-in-solrconfig.adoc#schema-factory-definition-in-solrconfig,Schema Factory Definition in SolrConfig>> for details.
 
 `dataDir`::
-Name of the data directory relative to `instanceDir`.
+Name of the data directory relative to `instanceDir`. If absolute value is used, it must be inside `SOLR_HOME`, `SOLR_DATA_HOME` or one of the paths specified by system property `solr.allowPaths`.
 
 `configSet`::
 Name of the configset to use for this core. For more information, see the section <<config-sets.adoc#config-sets,Configsets>>.
diff --git a/solr/solr-ref-guide/src/format-of-solr-xml.adoc b/solr/solr-ref-guide/src/format-of-solr-xml.adoc
index 7d47a72..4b47b77 100644
--- a/solr/solr-ref-guide/src/format-of-solr-xml.adoc
+++ b/solr/solr-ref-guide/src/format-of-solr-xml.adoc
@@ -30,6 +30,7 @@ You can find `solr.xml` in your `$SOLR_HOME` directory (usually `server/solr` or
 
   <int name="maxBooleanClauses">${solr.max.booleanClauses:1024}</int>
   <str name="sharedLib">${solr.sharedLib:}</str>
+  <str name="allowPaths">${solr.allowPaths:}</str>
 
   <solrcloud>
     <str name="host">${host:}</str>
@@ -91,6 +92,9 @@ Currently non-operational.
 `sharedLib`::
 Specifies the path to a common library directory that will be shared across all cores. Any JAR files in this directory will be added to the search path for Solr plugins. If the specified path is not absolute, it will be relative to `$SOLR_HOME`. Custom handlers may be placed in this directory. Note that specifying `sharedLib` will not remove `$SOLR_HOME/lib` from Solr's class path.
 
+`allowPaths`::
+Solr will normally only access folders relative to `$SOLR_HOME`, `$SOLR_DATA_HOME` or `coreRootDir`. If you need to e.g. create a core outside of these paths, you can explicitly allow the path with `allowPaths`. It is a comma separated string of file system paths to allow. The special value of `*` will allow any path on the system.
+
 `shareSchema`::
 This attribute, when set to `true`, ensures that the multiple cores pointing to the same Schema resource file will be referring to the same IndexSchema Object. Sharing the IndexSchema Object makes loading the core faster. If you use this feature, make sure that no core-specific property is used in your Schema file.
 
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/request/TestCoreAdmin.java b/solr/solrj/src/test/org/apache/solr/client/solrj/request/TestCoreAdmin.java
index 44247a7..9038f0b 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/request/TestCoreAdmin.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/request/TestCoreAdmin.java
@@ -90,8 +90,8 @@ public class TestCoreAdmin extends AbstractEmbeddedSolrServerTestCase {
 
     SolrClient client = getSolrAdmin();
     File testDir = createTempDir(LuceneTestCase.getTestClass().getSimpleName()).toFile();
-
     File newCoreInstanceDir = new File(testDir, "newcore");
+    cores.getAllowPaths().add(testDir.toPath()); // Allow the test dir
 
     CoreAdminRequest.Create req = new CoreAdminRequest.Create();
     req.setCoreName("corewithconfigset");
@@ -115,6 +115,8 @@ public class TestCoreAdmin extends AbstractEmbeddedSolrServerTestCase {
       File dataDir = createTempDir("data").toFile();
 
       File newCoreInstanceDir = createTempDir("instance").toFile();
+      cores.getAllowPaths().add(dataDir.toPath()); // Allow the test dir
+      cores.getAllowPaths().add(newCoreInstanceDir.toPath()); // Allow the test dir
 
       File instanceDir = new File(cores.getSolrHome());
       FileUtils.copyDirectory(instanceDir, new File(newCoreInstanceDir,