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

[lucene-solr] branch jira/solr-14071 created (now 8748f42)

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

ishan pushed a change to branch jira/solr-14071
in repository https://gitbox.apache.org/repos/asf/lucene-solr.git.


      at 8748f42  SOLR-14071: Untrusted configsets shouldn't be allowed to use <lib> directive

This branch includes the following new commits:

     new 8748f42  SOLR-14071: Untrusted configsets shouldn't be allowed to use <lib> directive

The 1 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.



[lucene-solr] 01/01: SOLR-14071: Untrusted configsets shouldn't be allowed to use directive

Posted by is...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 8748f42a3cde13596dbd018613cf9161d64fb825
Author: Ishan Chattopadhyaya <is...@apache.org>
AuthorDate: Thu Dec 12 13:33:34 2019 +0530

    SOLR-14071: Untrusted configsets shouldn't be allowed to use <lib> directive
---
 .../org/apache/solr/core/ConfigSetService.java     |  7 +--
 .../src/java/org/apache/solr/core/SolrConfig.java  | 59 ++++++----------------
 .../upload/with-lib-directive/managed-schema       | 25 +++++++++
 .../upload/with-lib-directive/solrconfig.xml       | 53 +++++++++++++++++++
 .../org/apache/solr/cloud/TestConfigSetsAPI.java   | 43 ++++++++++++++++
 .../src/test/org/apache/solr/core/TestConfig.java  | 12 ++---
 .../test/org/apache/solr/schema/DateFieldTest.java |  3 +-
 .../apache/solr/schema/PrimitiveFieldTypeTest.java |  3 +-
 .../solr/schema/TestManagedSchemaThreadSafety.java |  2 +-
 .../apache/solr/update/SolrIndexConfigTest.java    | 12 ++---
 .../src/java/org/apache/solr/util/TestHarness.java |  2 +-
 11 files changed, 157 insertions(+), 64 deletions(-)

diff --git a/solr/core/src/java/org/apache/solr/core/ConfigSetService.java b/solr/core/src/java/org/apache/solr/core/ConfigSetService.java
index 7ce1a52..bc0d4a9 100644
--- a/solr/core/src/java/org/apache/solr/core/ConfigSetService.java
+++ b/solr/core/src/java/org/apache/solr/core/ConfigSetService.java
@@ -87,7 +87,7 @@ public abstract class ConfigSetService {
               && !flags.getBooleanArg("trusted")
               ) ? false: true;
 
-      SolrConfig solrConfig = createSolrConfig(dcore, coreLoader);
+      SolrConfig solrConfig = createSolrConfig(dcore, coreLoader, trusted);
       IndexSchema schema = createIndexSchema(dcore, solrConfig);
       return new ConfigSet(configName(dcore), solrConfig, schema, properties, trusted);
     } catch (Exception e) {
@@ -102,10 +102,11 @@ public abstract class ConfigSetService {
    * Create a SolrConfig object for a core
    * @param cd the core's CoreDescriptor
    * @param loader the core's resource loader
+   * @param isTrusted is the configset trusted?
    * @return a SolrConfig object
    */
-  protected SolrConfig createSolrConfig(CoreDescriptor cd, SolrResourceLoader loader) {
-    return SolrConfig.readFromResourceLoader(loader, cd.getConfigName());
+  protected SolrConfig createSolrConfig(CoreDescriptor cd, SolrResourceLoader loader, boolean isTrusted) {
+    return SolrConfig.readFromResourceLoader(loader, cd.getConfigName(), isTrusted);
   }
 
   /**
diff --git a/solr/core/src/java/org/apache/solr/core/SolrConfig.java b/solr/core/src/java/org/apache/solr/core/SolrConfig.java
index dd5489a..3acb9c6 100644
--- a/solr/core/src/java/org/apache/solr/core/SolrConfig.java
+++ b/solr/core/src/java/org/apache/solr/core/SolrConfig.java
@@ -133,53 +133,20 @@ public class SolrConfig extends XmlConfigFile implements MapSerializable {
   private final SolrRequestParsers solrRequestParsers;
 
   /**
-   * Creates a default instance from the solrconfig.xml.
-   */
-  public SolrConfig()
-      throws ParserConfigurationException, IOException, SAXException {
-    this((SolrResourceLoader) null, DEFAULT_CONF_FILE, null);
-  }
-
-  /**
-   * Creates a configuration instance from a configuration name.
-   * A default resource loader will be created (@see SolrResourceLoader)
-   *
-   * @param name the configuration name used by the loader
-   */
-  public SolrConfig(String name)
-      throws ParserConfigurationException, IOException, SAXException {
-    this((SolrResourceLoader) null, name, null);
-  }
-
-  /**
-   * Creates a configuration instance from a configuration name and stream.
-   * A default resource loader will be created (@see SolrResourceLoader).
-   * If the stream is null, the resource loader will open the configuration stream.
-   * If the stream is not null, no attempt to load the resource will occur (the name is not used).
-   *
-   * @param name the configuration name
-   * @param is   the configuration stream
-   */
-  public SolrConfig(String name, InputSource is)
-      throws ParserConfigurationException, IOException, SAXException {
-    this((SolrResourceLoader) null, name, is);
-  }
-
-  /**
    * Creates a configuration instance from an instance directory, configuration name and stream.
    *
    * @param instanceDir the directory used to create the resource loader
    * @param name        the configuration name used by the loader if the stream is null
    * @param is          the configuration stream
    */
-  public SolrConfig(Path instanceDir, String name, InputSource is)
+  public SolrConfig(Path instanceDir, String name, InputSource is, boolean isConfigsetTrusted)
       throws ParserConfigurationException, IOException, SAXException {
-    this(new SolrResourceLoader(instanceDir), name, is);
+    this(new SolrResourceLoader(instanceDir), name, is, isConfigsetTrusted);
   }
 
-  public static SolrConfig readFromResourceLoader(SolrResourceLoader loader, String name) {
+  public static SolrConfig readFromResourceLoader(SolrResourceLoader loader, String name, boolean isConfigsetTrusted) {
     try {
-      return new SolrConfig(loader, name, null);
+      return new SolrConfig(loader, name, null, isConfigsetTrusted);
     } catch (Exception e) {
       String resource;
       if (loader instanceof ZkSolrResourceLoader) {
@@ -196,16 +163,17 @@ public class SolrConfig extends XmlConfigFile implements MapSerializable {
    * If the stream is null, the resource loader will open the configuration stream.
    * If the stream is not null, no attempt to load the resource will occur (the name is not used).
    *
-   * @param loader the resource loader
-   * @param name   the configuration name
-   * @param is     the configuration stream
+   * @param                     loader the resource loader
+   * @param name                the configuration name
+   * @param is                  the configuration stream
+   * @param isConfigsetTrusted  false if configset was uploaded using unsecured configset upload API, true otherwise
    */
-  public SolrConfig(SolrResourceLoader loader, String name, InputSource is)
+  private SolrConfig(SolrResourceLoader loader, String name, InputSource is, boolean isConfigsetTrusted)
       throws ParserConfigurationException, IOException, SAXException {
     super(loader, name, is, "/config/");
     getOverlay();//just in case it is not initialized
     getRequestParams();
-    initLibs();
+    initLibs(isConfigsetTrusted);
     luceneMatchVersion = SolrConfig.parseLuceneVersionString(getVal(IndexSchema.LUCENE_MATCH_VERSION_PARAM, true));
     log.info("Using Lucene MatchVersion: {}", luceneMatchVersion);
 
@@ -786,9 +754,14 @@ public class SolrConfig extends XmlConfigFile implements MapSerializable {
         "Multiple plugins configured for type: " + type);
   }
 
-  private void initLibs() {
+  private void initLibs(boolean isConfigsetTrusted) {
     NodeList nodes = (NodeList) evaluate("lib", XPathConstants.NODESET);
     if (nodes == null || nodes.getLength() == 0) return;
+    if (!isConfigsetTrusted) {
+      throw new SolrException(ErrorCode.UNAUTHORIZED, "The configset for this collection was uploaded without any authentication in place,"
+          + " and use of <lib> is not available for collections with untrusted configsets. To use this component, re-upload the configset"
+          + " after enabling authentication and authorization.");
+    }
 
     log.debug("Adding specified lib dirs to ClassLoader");
     SolrResourceLoader loader = getResourceLoader();
diff --git a/solr/core/src/test-files/solr/configsets/upload/with-lib-directive/managed-schema b/solr/core/src/test-files/solr/configsets/upload/with-lib-directive/managed-schema
new file mode 100644
index 0000000..9e2f947
--- /dev/null
+++ b/solr/core/src/test-files/solr/configsets/upload/with-lib-directive/managed-schema
@@ -0,0 +1,25 @@
+<?xml version="1.0" encoding="UTF-8" ?>
+<!--
+ 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.
+-->
+<schema name="minimal" version="1.1">
+ <types>
+  <fieldType name="string" class="solr.StrField"/>
+ </types>
+ <fields>
+   <dynamicField name="*" type="string" indexed="true" stored="true" />
+ </fields>
+</schema>
diff --git a/solr/core/src/test-files/solr/configsets/upload/with-lib-directive/solrconfig.xml b/solr/core/src/test-files/solr/configsets/upload/with-lib-directive/solrconfig.xml
new file mode 100644
index 0000000..bb875e3
--- /dev/null
+++ b/solr/core/src/test-files/solr/configsets/upload/with-lib-directive/solrconfig.xml
@@ -0,0 +1,53 @@
+<?xml version="1.0" ?>
+
+<!--
+ 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.
+-->
+
+<!-- This is a "kitchen sink" config file that tests can use.
+     When writting a new test, feel free to add *new* items (plugins,
+     config options, etc...) as long as they don't break any existing
+     tests.  if you need to test something esoteric please add a new
+     "solrconfig-your-esoteric-purpose.xml" config file.
+
+     Note in particular that this test is used by MinimalSchemaTest so
+     Anything added to this file needs to work correctly even if there
+     is now uniqueKey or defaultSearch Field.
+  -->
+
+<config>
+
+  <dataDir>${solr.data.dir:}</dataDir>
+
+  <directoryFactory name="DirectoryFactory"
+                    class="${solr.directoryFactory:solr.NRTCachingDirectoryFactory}"/>
+
+  <luceneMatchVersion>${tests.luceneMatchVersion:LATEST}</luceneMatchVersion>
+
+  <lib dir="${solr.install.dir:../../../..}/dist/" regex="solr-ltr-\d.*\.jar" />
+
+  <requestHandler name="/select" class="solr.SearchHandler">
+    <lst name="defaults">
+      <str name="echoParams">explicit</str>
+      <str name="indent">true</str>
+      <str name="df">text</str>
+    </lst>
+
+  </requestHandler>
+
+  <requestHandler name="/update" class="solr.UpdateRequestHandler"  />
+</config>
+
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestConfigSetsAPI.java b/solr/core/src/test/org/apache/solr/cloud/TestConfigSetsAPI.java
index f4c1ec2..bc5edc3 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestConfigSetsAPI.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestConfigSetsAPI.java
@@ -348,6 +348,7 @@ public class TestConfigSetsAPI extends SolrTestCaseJ4 {
   @Test
   public void testUploadWithScriptUpdateProcessor() throws Exception {
       // Authorization off
+      unprotectConfigsHandler();
       final String untrustedSuffix = "-untrusted";
       uploadConfigSetWithAssertions("with-script-processor", untrustedSuffix, null, null);
       // try to create a collection with the uploaded configset
@@ -369,6 +370,35 @@ public class TestConfigSetsAPI extends SolrTestCaseJ4 {
 
   }
 
+  @Test
+  public void testUploadWithLibDirective() throws Exception {
+    // Authorization off
+    unprotectConfigsHandler();
+    final String untrustedSuffix = "-untrusted";
+    uploadConfigSetWithAssertions("with-lib-directive", untrustedSuffix, null, null);
+    // try to create a collection with the uploaded configset
+    Throwable thrown = expectThrows(HttpSolrClient.RemoteSolrException.class, () -> {
+      createCollection("newcollection3", "with-lib-directive" + untrustedSuffix,
+          1, 1, solrCluster.getSolrClient());
+    });
+
+    assertThat(thrown.getMessage(), containsString("Underlying core creation failed"));
+
+    // Authorization on
+    final String trustedSuffix = "-trusted";
+    protectConfigsHandler();
+    uploadConfigSetWithAssertions("with-lib-directive", trustedSuffix, "solr", "SolrRocks");
+    // try to create a collection with the uploaded configset
+    CollectionAdminResponse resp = createCollection("newcollection3", "with-lib-directive" + trustedSuffix,
+        1, 1, solrCluster.getSolrClient());
+    
+    SolrInputDocument doc = sdoc("id", "4055", "subject", "Solr");
+    solrCluster.getSolrClient().add("newcollection3", doc);
+    solrCluster.getSolrClient().commit("newcollection3");
+    assertEquals("4055", solrCluster.getSolrClient().query("newcollection3",
+        params("q", "*:*")).getResults().get(0).get("id"));
+  }
+
   protected SolrZkClient zkClient() {
     ZkStateReader reader = solrCluster.getSolrClient().getZkStateReader();
     if (reader == null)
@@ -376,6 +406,19 @@ public class TestConfigSetsAPI extends SolrTestCaseJ4 {
     return solrCluster.getSolrClient().getZkStateReader().getZkClient();
   }
 
+  private void unprotectConfigsHandler() throws Exception {
+    HttpClient cl = null;
+    try {
+      cl = HttpClientUtil.createClient(null);
+      zkClient().setData("/security.json", "{}".getBytes(UTF_8), true);
+    } finally {
+      if (cl != null) {
+        HttpClientUtil.close(cl);
+      }
+    }
+    Thread.sleep(5000); // TODO: Without a delay, the test fails. Some problem with Authc/Authz framework?
+  }
+  
   private void protectConfigsHandler() throws Exception {
     String authcPrefix = "/admin/authentication";
     String authzPrefix = "/admin/authorization";
diff --git a/solr/core/src/test/org/apache/solr/core/TestConfig.java b/solr/core/src/test/org/apache/solr/core/TestConfig.java
index 69882c6..38722cf 100644
--- a/solr/core/src/test/org/apache/solr/core/TestConfig.java
+++ b/solr/core/src/test/org/apache/solr/core/TestConfig.java
@@ -110,7 +110,7 @@ public class TestConfig extends SolrTestCaseJ4 {
  @Test
  public void testCacheEnablingDisabling() throws Exception {
    // ensure if cache is not defined in the config then cache is disabled 
-   SolrConfig sc = new SolrConfig(new SolrResourceLoader(TEST_PATH().resolve("collection1")), "solrconfig-defaults.xml", null);
+   SolrConfig sc = new SolrConfig(TEST_PATH().resolve("collection1"), "solrconfig-defaults.xml", null, true);
    assertNull(sc.filterCacheConfig);
    assertNull(sc.queryResultCacheConfig);
    assertNull(sc.documentCacheConfig);
@@ -119,7 +119,7 @@ public class TestConfig extends SolrTestCaseJ4 {
    System.setProperty("filterCache.enabled", "true");
    System.setProperty("queryResultCache.enabled", "true");
    System.setProperty("documentCache.enabled", "true");
-   sc = new SolrConfig(new SolrResourceLoader(TEST_PATH().resolve("collection1")), "solrconfig-cache-enable-disable.xml", null);
+   sc = new SolrConfig(TEST_PATH().resolve("collection1"), "solrconfig-cache-enable-disable.xml", null, true);
    assertNotNull(sc.filterCacheConfig);
    assertNotNull(sc.queryResultCacheConfig);
    assertNotNull(sc.documentCacheConfig);
@@ -128,7 +128,7 @@ public class TestConfig extends SolrTestCaseJ4 {
    System.setProperty("filterCache.enabled", "false");
    System.setProperty("queryResultCache.enabled", "false");
    System.setProperty("documentCache.enabled", "false");
-   sc = new SolrConfig(new SolrResourceLoader(TEST_PATH().resolve("collection1")), "solrconfig-cache-enable-disable.xml", null);
+   sc = new SolrConfig(TEST_PATH().resolve("collection1"), "solrconfig-cache-enable-disable.xml", null, true);
    assertNull(sc.filterCacheConfig);
    assertNull(sc.queryResultCacheConfig);
    assertNull(sc.documentCacheConfig);
@@ -146,7 +146,7 @@ public class TestConfig extends SolrTestCaseJ4 {
     int numDefaultsTested = 0;
     int numNullDefaults = 0;
 
-    SolrConfig sc = new SolrConfig(new SolrResourceLoader(TEST_PATH().resolve("collection1")), "solrconfig-defaults.xml", null);
+    SolrConfig sc = new SolrConfig(TEST_PATH().resolve("collection1"), "solrconfig-defaults.xml", null, true);
     SolrIndexConfig sic = sc.indexConfig;
 
     ++numDefaultsTested; assertEquals("default useCompoundFile", false, sic.useCompoundFile);
@@ -210,7 +210,7 @@ public class TestConfig extends SolrTestCaseJ4 {
 
   @Test
   public void testMaxSizeSettingWithoutAutoCommit() throws Exception {
-    SolrConfig solrConfig = new SolrConfig(new SolrResourceLoader(TEST_PATH().resolve("collection1")), "bad-solrconfig-no-autocommit-tag.xml", null);
+    SolrConfig solrConfig = new SolrConfig(TEST_PATH().resolve("collection1"), "bad-solrconfig-no-autocommit-tag.xml", null, true);
     Assert.assertEquals(-1, solrConfig.getUpdateHandlerInfo().autoCommitMaxSizeBytes);
     Assert.assertEquals(-1, solrConfig.getUpdateHandlerInfo().autoCommmitMaxDocs);
     Assert.assertEquals(-1, solrConfig.getUpdateHandlerInfo().autoCommmitMaxTime);
@@ -219,7 +219,7 @@ public class TestConfig extends SolrTestCaseJ4 {
   // sanity check that sys properties are working as expected
   public void testSanityCheckTestSysPropsAreUsed() throws Exception {
 
-    SolrConfig sc = new SolrConfig(new SolrResourceLoader(TEST_PATH().resolve("collection1")), "solrconfig-basic.xml", null);
+    SolrConfig sc = new SolrConfig(TEST_PATH().resolve("collection1"), "solrconfig-basic.xml", null, true);
     SolrIndexConfig sic = sc.indexConfig;
 
     assertEquals("ramBufferSizeMB sysprop", 
diff --git a/solr/core/src/test/org/apache/solr/schema/DateFieldTest.java b/solr/core/src/test/org/apache/solr/schema/DateFieldTest.java
index 6d03002..8929ed3 100644
--- a/solr/core/src/test/org/apache/solr/schema/DateFieldTest.java
+++ b/solr/core/src/test/org/apache/solr/schema/DateFieldTest.java
@@ -25,7 +25,6 @@ import org.apache.lucene.index.IndexableField;
 import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.common.util.ByteArrayUtf8CharSequence;
 import org.apache.solr.core.SolrConfig;
-import org.apache.solr.core.SolrResourceLoader;
 
 public class DateFieldTest extends SolrTestCaseJ4 {
   private final String testInstanceDir = TEST_HOME() + File.separator + "collection1";
@@ -39,7 +38,7 @@ public class DateFieldTest extends SolrTestCaseJ4 {
     System.setProperty("solr.test.sys.prop1", "propone");
     System.setProperty("solr.test.sys.prop2", "proptwo");
     SolrConfig config = new SolrConfig
-        (new SolrResourceLoader(Paths.get(testInstanceDir)), testConfHome + "solrconfig.xml", null);
+        (Paths.get(testInstanceDir), testConfHome + "solrconfig.xml", null, true);
     IndexSchema schema = IndexSchemaFactory.buildIndexSchema(testConfHome + "schema.xml", config);
     f = Boolean.getBoolean(NUMERIC_POINTS_SYSPROP)
       ? new DatePointField() : new TrieDateField();
diff --git a/solr/core/src/test/org/apache/solr/schema/PrimitiveFieldTypeTest.java b/solr/core/src/test/org/apache/solr/schema/PrimitiveFieldTypeTest.java
index 544b2d4..6d2e8a6 100644
--- a/solr/core/src/test/org/apache/solr/schema/PrimitiveFieldTypeTest.java
+++ b/solr/core/src/test/org/apache/solr/schema/PrimitiveFieldTypeTest.java
@@ -24,7 +24,6 @@ import java.util.Map;
 
 import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.core.SolrConfig;
-import org.apache.solr.core.SolrResourceLoader;
 import org.junit.Test;
 
 /**
@@ -46,7 +45,7 @@ public class PrimitiveFieldTypeTest extends SolrTestCaseJ4 {
     System.setProperty("solr.allow.unsafe.resourceloading", "true");
 
     initMap = new HashMap<>();
-    config = new SolrConfig(new SolrResourceLoader(TEST_PATH().resolve("collection1")), testConfHome + "solrconfig.xml", null);
+    config = new SolrConfig(TEST_PATH().resolve("collection1"), testConfHome + "solrconfig.xml", null, true);
   }
   
   @Override
diff --git a/solr/core/src/test/org/apache/solr/schema/TestManagedSchemaThreadSafety.java b/solr/core/src/test/org/apache/solr/schema/TestManagedSchemaThreadSafety.java
index ae205ac..42d2a77 100644
--- a/solr/core/src/test/org/apache/solr/schema/TestManagedSchemaThreadSafety.java
+++ b/solr/core/src/test/org/apache/solr/schema/TestManagedSchemaThreadSafety.java
@@ -175,7 +175,7 @@ public class TestManagedSchemaThreadSafety extends SolrTestCaseJ4 {
     return () -> {
       try {
         SolrResourceLoader loader = new ZkSolrResourceLoader(loaderPath, configsetName, zkController);
-        SolrConfig solrConfig = SolrConfig.readFromResourceLoader(loader, "solrconfig.xml");
+        SolrConfig solrConfig = SolrConfig.readFromResourceLoader(loader, "solrconfig.xml", true);
 
         ManagedIndexSchemaFactory factory = new ManagedIndexSchemaFactory();
         factory.init(new NamedList());
diff --git a/solr/core/src/test/org/apache/solr/update/SolrIndexConfigTest.java b/solr/core/src/test/org/apache/solr/update/SolrIndexConfigTest.java
index b314293..27138ce 100644
--- a/solr/core/src/test/org/apache/solr/update/SolrIndexConfigTest.java
+++ b/solr/core/src/test/org/apache/solr/update/SolrIndexConfigTest.java
@@ -62,7 +62,7 @@ public class SolrIndexConfigTest extends SolrTestCaseJ4 {
 
   @Test
   public void testFailingSolrIndexConfigCreation() throws Exception {
-    SolrConfig solrConfig = new SolrConfig(instanceDir,"bad-mpf-solrconfig.xml", null);
+    SolrConfig solrConfig = new SolrConfig(instanceDir,"bad-mpf-solrconfig.xml", null, true);
     SolrIndexConfig solrIndexConfig = new SolrIndexConfig(solrConfig, null, null);
     IndexSchema indexSchema = IndexSchemaFactory.buildIndexSchema(schemaFileName, solrConfig);
     h.getCore().setLatestSchema(indexSchema);
@@ -75,7 +75,7 @@ public class SolrIndexConfigTest extends SolrTestCaseJ4 {
   @Test
   public void testTieredMPSolrIndexConfigCreation() throws Exception {
     String solrConfigFileName = solrConfigFileNameTieredMergePolicyFactory;
-    SolrConfig solrConfig = new SolrConfig(instanceDir, solrConfigFileName, null);
+    SolrConfig solrConfig = new SolrConfig(instanceDir, solrConfigFileName, null, true);
     SolrIndexConfig solrIndexConfig = new SolrIndexConfig(solrConfig, null, null);
     IndexSchema indexSchema = IndexSchemaFactory.buildIndexSchema(schemaFileName, solrConfig);
     
@@ -100,7 +100,7 @@ public class SolrIndexConfigTest extends SolrTestCaseJ4 {
   @Test
   public void testConcurrentMergeSchedularSolrIndexConfigCreation() throws Exception {
     String solrConfigFileName = solrConfigFileNameConnMSPolicyFactory;
-    SolrConfig solrConfig = new SolrConfig(instanceDir, solrConfigFileName, null);
+    SolrConfig solrConfig = new SolrConfig(instanceDir, solrConfigFileName, null, true);
     SolrIndexConfig solrIndexConfig = new SolrIndexConfig(solrConfig, null, null);
     IndexSchema indexSchema = IndexSchemaFactory.buildIndexSchema(schemaFileName, solrConfig);
 
@@ -124,7 +124,7 @@ public class SolrIndexConfigTest extends SolrTestCaseJ4 {
     final SortField.Type expectedFieldType = SortField.Type.INT;
     final boolean expectedFieldSortDescending = true;
 
-    SolrConfig solrConfig = new SolrConfig(instanceDir, solrConfigFileNameSortingMergePolicyFactory, null);
+    SolrConfig solrConfig = new SolrConfig(instanceDir, solrConfigFileNameSortingMergePolicyFactory, null, true);
     SolrIndexConfig solrIndexConfig = new SolrIndexConfig(solrConfig, null, null);
     assertNotNull(solrIndexConfig);
     IndexSchema indexSchema = IndexSchemaFactory.buildIndexSchema(schemaFileName, solrConfig);
@@ -142,7 +142,7 @@ public class SolrIndexConfigTest extends SolrTestCaseJ4 {
   }
 
   public void testMergedSegmentWarmerIndexConfigCreation() throws Exception {
-    SolrConfig solrConfig = new SolrConfig(instanceDir, solrConfigFileNameWarmerRandomMergePolicyFactory, null);
+    SolrConfig solrConfig = new SolrConfig(instanceDir, solrConfigFileNameWarmerRandomMergePolicyFactory, null, true);
     SolrIndexConfig solrIndexConfig = new SolrIndexConfig(solrConfig, null, null);
     assertNotNull(solrIndexConfig);
     assertNotNull(solrIndexConfig.mergedSegmentWarmerInfo);
@@ -158,7 +158,7 @@ public class SolrIndexConfigTest extends SolrTestCaseJ4 {
     final String solrConfigFileNameWarmer = solrConfigFileNameWarmerRandomMergePolicyFactory;
     final String solrConfigFileNameTMP = solrConfigFileNameTieredMergePolicyFactory;
     final String solrConfigFileName = (random().nextBoolean() ? solrConfigFileNameWarmer : solrConfigFileNameTMP);
-    SolrConfig solrConfig = new SolrConfig(instanceDir, solrConfigFileName, null);
+    SolrConfig solrConfig = new SolrConfig(instanceDir, solrConfigFileName, null, true);
     SolrIndexConfig solrIndexConfig = new SolrIndexConfig(solrConfig, null, null);
     assertNotNull(solrIndexConfig);
     assertNotNull(solrIndexConfig.mergePolicyFactoryInfo);
diff --git a/solr/test-framework/src/java/org/apache/solr/util/TestHarness.java b/solr/test-framework/src/java/org/apache/solr/util/TestHarness.java
index 529a3cb..1a23da8 100644
--- a/solr/test-framework/src/java/org/apache/solr/util/TestHarness.java
+++ b/solr/test-framework/src/java/org/apache/solr/util/TestHarness.java
@@ -86,7 +86,7 @@ public class TestHarness extends BaseTestHarness {
     System.setProperty("solr.test.sys.prop1", "propone");
     System.setProperty("solr.test.sys.prop2", "proptwo");
     try {
-      return new SolrConfig(solrHome.resolve(coreName), confFile, null);
+      return new SolrConfig(solrHome.resolve(coreName), confFile, null, true);
     } catch (Exception xany) {
       throw new RuntimeException(xany);
     }