You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by sh...@apache.org on 2017/06/07 14:11:13 UTC

[09/50] [abbrv] lucene-solr:feature/autoscaling: SOLR-8668: In solrconfig.xml remove (and related and ) support in favor of the element introduced by SOLR-8621 in Solr 5.5.0. (Christine Poers

SOLR-8668: In solrconfig.xml remove <mergePolicy> (and related <mergeFactor> and <maxMergeDocs>)
support in favor of the <mergePolicyFactory> element introduced by SOLR-8621 in Solr 5.5.0.
(Christine Poerschke, hossman)


Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/c64f9d64
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/c64f9d64
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/c64f9d64

Branch: refs/heads/feature/autoscaling
Commit: c64f9d64b4edc8c3761368befc394e879b2284ff
Parents: 445ceda
Author: Christine Poerschke <cp...@apache.org>
Authored: Fri Jun 2 13:59:08 2017 +0100
Committer: Christine Poerschke <cp...@apache.org>
Committed: Fri Jun 2 13:59:08 2017 +0100

----------------------------------------------------------------------
 solr/CHANGES.txt                                |   7 +
 .../org/apache/solr/update/SolrIndexConfig.java | 146 ++-----------------
 .../solr/collection1/conf/bad-mp-solrconfig.xml |  36 -----
 .../collection1/conf/solrconfig-delpolicy1.xml  |   3 +-
 .../collection1/conf/solrconfig-delpolicy2.xml  |   3 +-
 .../collection1/conf/solrconfig-indexconfig.xml |  31 ----
 .../conf/solrconfig-indexmetrics.xml            |   8 +-
 .../conf/solrconfig-logmergepolicy.xml          |  36 -----
 .../conf/solrconfig-mergepolicy-legacy.xml      |   1 -
 .../conf/solrconfig-mergepolicy-nocfs.xml       |  34 -----
 .../conf/solrconfig-spellcheckcomponent.xml     |   3 +-
 .../conf/solrconfig-tieredmergepolicy.xml       |  48 ------
 .../solr/collection1/conf/solrconfig-warmer.xml |  46 ------
 .../solrconfig.snippet.randomindexconfig.xml    |   3 +-
 .../org/apache/solr/TestGroupingSearch.java     |   3 -
 .../solr/cloud/TestAuthenticationFramework.java |  11 +-
 .../solr/cloud/TestMiniSolrCloudCluster.java    |  11 +-
 .../cloud/TestSolrCloudWithKerberosAlt.java     |  11 +-
 .../org/apache/solr/core/TestBadConfig.java     |   2 -
 .../test/org/apache/solr/core/TestConfig.java   |   9 +-
 .../apache/solr/core/TestMergePolicyConfig.java |  18 +--
 .../test/org/apache/solr/core/TestNRTOpen.java  |   3 -
 .../apache/solr/core/TestSolrIndexConfig.java   |   2 +-
 .../admin/SegmentsInfoRequestHandlerTest.java   |   2 -
 .../solr/schema/TestHalfAndHalfDocValues.java   |   4 -
 .../apache/solr/search/TestIndexSearcher.java   |   3 -
 .../solr/update/DirectUpdateHandlerTest.java    |   3 -
 .../solr/update/SolrCmdDistributorTest.java     |   3 -
 .../apache/solr/update/SolrIndexConfigTest.java |  36 ++---
 .../solr/update/TestInPlaceUpdatesDistrib.java  |   4 -
 .../update/TestInPlaceUpdatesStandalone.java    |   4 -
 .../java/org/apache/solr/SolrTestCaseJ4.java    |  35 -----
 32 files changed, 52 insertions(+), 517 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c64f9d64/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index d8c9d04..195443d 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -86,6 +86,9 @@ Upgrading from Solr 6.x
   this QParser untill Solr 8, you must explicitly register it in your solrconfig.xml:
      '<queryParser name="lucenePlusSort" class="solr.OldLuceneQParserPlugin"/>'
 
+* In solrconfig.xml the deprecated <mergePolicy> and <mergeFactor> and <maxMergeDocs> elements have
+  been removed in favor of the <mergePolicyFactory> element introduced by SOLR-8621 in Solr 5.5.0.
+
 New Features
 ----------------------
 * SOLR-9857, SOLR-9858: Collect aggregated metrics from nodes and shard leaders in overseer. (ab)
@@ -218,6 +221,10 @@ Other Changes
 * LUCENE-7852: Correct copyright year(s) in solr/LICENSE.txt file.
   (Christine Poerschke, Steve Rowe)
 
+* SOLR-8668: In solrconfig.xml remove <mergePolicy> (and related <mergeFactor> and <maxMergeDocs>)
+  support in favor of the <mergePolicyFactory> element introduced by SOLR-8621 in Solr 5.5.0.
+  (Christine Poerschke, hossman)
+
 ==================  6.7.0 ==================
 
 Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this release.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c64f9d64/solr/core/src/java/org/apache/solr/update/SolrIndexConfig.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/update/SolrIndexConfig.java b/solr/core/src/java/org/apache/solr/update/SolrIndexConfig.java
index 26b9839..c663783 100644
--- a/solr/core/src/java/org/apache/solr/update/SolrIndexConfig.java
+++ b/solr/core/src/java/org/apache/solr/update/SolrIndexConfig.java
@@ -27,10 +27,8 @@ import org.apache.lucene.analysis.DelegatingAnalyzerWrapper;
 import org.apache.lucene.index.ConcurrentMergeScheduler;
 import org.apache.lucene.index.IndexWriter.IndexReaderWarmer;
 import org.apache.lucene.index.IndexWriterConfig;
-import org.apache.lucene.index.LogMergePolicy;
 import org.apache.lucene.index.MergePolicy;
 import org.apache.lucene.index.MergeScheduler;
-import org.apache.lucene.index.TieredMergePolicy;
 import org.apache.lucene.search.Sort;
 import org.apache.lucene.util.InfoStream;
 import org.apache.lucene.util.Version;
@@ -66,17 +64,14 @@ public class SolrIndexConfig implements MapSerializable {
   public static final String DEFAULT_MERGE_SCHEDULER_CLASSNAME = ConcurrentMergeScheduler.class.getName();
   public final Version luceneVersion;
 
-  private boolean effectiveUseCompoundFileSetting;
+  public final boolean useCompoundFile;
 
   public final int maxBufferedDocs;
-  public final int maxMergeDocs;
-  public final int mergeFactor;
 
   public final double ramBufferSizeMB;
 
   public final int writeLockTimeout;
   public final String lockType;
-  public final PluginInfo mergePolicyInfo;
   public final PluginInfo mergePolicyFactoryInfo;
   public final PluginInfo mergeSchedulerInfo;
   public final PluginInfo metricsInfo;
@@ -90,14 +85,11 @@ public class SolrIndexConfig implements MapSerializable {
    */
   private SolrIndexConfig(SolrConfig solrConfig) {
     luceneVersion = solrConfig.luceneMatchVersion;
-    effectiveUseCompoundFileSetting = false;
+    useCompoundFile = false;
     maxBufferedDocs = -1;
-    maxMergeDocs = -1;
-    mergeFactor = -1;
     ramBufferSizeMB = 100;
     writeLockTimeout = -1;
     lockType = DirectoryFactory.LOCK_TYPE_NATIVE;
-    mergePolicyInfo = null;
     mergePolicyFactoryInfo = null;
     mergeSchedulerInfo = null;
     mergedSegmentWarmerInfo = null;
@@ -132,17 +124,15 @@ public class SolrIndexConfig implements MapSerializable {
     assertWarnOrFail("The <mergeScheduler>myclass</mergeScheduler> syntax is no longer supported in solrconfig.xml. Please use syntax <mergeScheduler class=\"myclass\"/> instead.",
         !((solrConfig.getNode(prefix + "/mergeScheduler", false) != null) && (solrConfig.get(prefix + "/mergeScheduler/@class", null) == null)),
         true);
-    assertWarnOrFail("The <mergePolicy>myclass</mergePolicy> syntax is no longer supported in solrconfig.xml. Please use syntax <mergePolicy class=\"myclass\"/> instead.",
+    assertWarnOrFail("Beginning with Solr 7.0, <mergePolicy>myclass</mergePolicy> is no longer supported, use <mergePolicyFactory> instead.",
         !((solrConfig.getNode(prefix + "/mergePolicy", false) != null) && (solrConfig.get(prefix + "/mergePolicy/@class", null) == null)),
         true);
     assertWarnOrFail("The <luceneAutoCommit>true|false</luceneAutoCommit> parameter is no longer valid in solrconfig.xml.",
         solrConfig.get(prefix + "/luceneAutoCommit", null) == null,
         true);
 
-    effectiveUseCompoundFileSetting = solrConfig.getBool(prefix+"/useCompoundFile", def.getUseCompoundFile());
+    useCompoundFile = solrConfig.getBool(prefix+"/useCompoundFile", def.useCompoundFile);
     maxBufferedDocs=solrConfig.getInt(prefix+"/maxBufferedDocs",def.maxBufferedDocs);
-    maxMergeDocs=solrConfig.getInt(prefix+"/maxMergeDocs",def.maxMergeDocs);
-    mergeFactor=solrConfig.getInt(prefix+"/mergeFactor",def.mergeFactor);
     ramBufferSizeMB = solrConfig.getDouble(prefix+"/ramBufferSizeMB", def.ramBufferSizeMB);
 
     writeLockTimeout=solrConfig.getInt(prefix+"/writeLockTimeout", def.writeLockTimeout);
@@ -155,24 +145,17 @@ public class SolrIndexConfig implements MapSerializable {
       metricsInfo = infos.get(0);
     }
     mergeSchedulerInfo = getPluginInfo(prefix + "/mergeScheduler", solrConfig, def.mergeSchedulerInfo);
-    mergePolicyInfo = getPluginInfo(prefix + "/mergePolicy", solrConfig, def.mergePolicyInfo);
     mergePolicyFactoryInfo = getPluginInfo(prefix + "/mergePolicyFactory", solrConfig, def.mergePolicyFactoryInfo);
-    if (mergePolicyInfo != null && mergePolicyFactoryInfo != null) {
-      throw new IllegalArgumentException("<mergePolicy> and <mergePolicyFactory> are mutually exclusive.");
-    }
-    if (maxMergeDocs != def.maxMergeDocs && mergePolicyFactoryInfo != null) {
-      throw new IllegalArgumentException("<maxMergeDocs> and <mergePolicyFactory> are mutually exclusive.");
-    }
-    if (mergeFactor != def.mergeFactor && mergePolicyFactoryInfo != null) {
-      throw new IllegalArgumentException("<mergeFactor> and <mergePolicyFactory> are mutually exclusive.");
-    }
 
-    assertWarnOrFail("Beginning with Solr 5.5, <mergePolicy> is deprecated, use <mergePolicyFactory> instead.",
-        (mergePolicyInfo == null), false);
-    assertWarnOrFail("Beginning with Solr 5.5, <maxMergeDocs> is deprecated, configure it on the relevant <mergePolicyFactory> instead.",
-        (maxMergeDocs == def.maxMergeDocs), false);
-    assertWarnOrFail("Beginning with Solr 5.5, <mergeFactor> is deprecated, configure it on the relevant <mergePolicyFactory> instead.",
-        (mergeFactor == def.mergeFactor), false);
+    assertWarnOrFail("Beginning with Solr 7.0, <mergePolicy> is no longer supported, use <mergePolicyFactory> instead.",
+        getPluginInfo(prefix + "/mergePolicy", solrConfig, null) == null,
+        true);
+    assertWarnOrFail("Beginning with Solr 7.0, <maxMergeDocs> is no longer supported, configure it on the relevant <mergePolicyFactory> instead.",
+        solrConfig.getInt(prefix+"/maxMergeDocs", 0) == 0,
+        true);
+    assertWarnOrFail("Beginning with Solr 7.0, <mergeFactor> is no longer supported, configure it on the relevant <mergePolicyFactory> instead.",
+        solrConfig.getInt(prefix+"/mergeFactor", 0) == 0,
+        true);
 
     String val = solrConfig.get(prefix + "/termIndexInterval", null);
     if (val != null) {
@@ -198,10 +181,8 @@ public class SolrIndexConfig implements MapSerializable {
 
   @Override
   public Map<String, Object> toMap(Map<String, Object> map) {
-    Map<String, Object> m = Utils.makeMap("useCompoundFile", effectiveUseCompoundFileSetting,
+    Map<String, Object> m = Utils.makeMap("useCompoundFile", useCompoundFile,
         "maxBufferedDocs", maxBufferedDocs,
-        "maxMergeDocs", maxMergeDocs,
-        "mergeFactor", mergeFactor,
         "ramBufferSizeMB", ramBufferSizeMB,
         "writeLockTimeout", writeLockTimeout,
         "lockType", lockType,
@@ -210,9 +191,7 @@ public class SolrIndexConfig implements MapSerializable {
     if (metricsInfo != null) {
       m.put("metrics", metricsInfo);
     }
-    if (mergePolicyInfo != null) {
-      m.put("mergePolicy", mergePolicyInfo);
-    } else if (mergePolicyFactoryInfo != null) {
+    if (mergePolicyFactoryInfo != null) {
       m.put("mergePolicyFactory", mergePolicyFactoryInfo);
     }
     if(mergedSegmentWarmerInfo != null) m.put("mergedSegmentWarmer",mergedSegmentWarmerInfo);
@@ -259,9 +238,7 @@ public class SolrIndexConfig implements MapSerializable {
       iwc.setIndexSort(indexSort);
     }
 
-    // do this after buildMergePolicy since the backcompat logic 
-    // there may modify the effective useCompoundFile
-    iwc.setUseCompoundFile(getUseCompoundFile());
+    iwc.setUseCompoundFile(useCompoundFile);
 
     if (mergedSegmentWarmerInfo != null) {
       // TODO: add infostream -> normal logging system (there is an issue somewhere)
@@ -276,19 +253,12 @@ public class SolrIndexConfig implements MapSerializable {
     return iwc;
   }
 
-  private boolean useMergePolicyInfo() {
-    return mergePolicyInfo != null || maxMergeDocs != -1 || mergeFactor != -1;
-  }
-
   /**
    * Builds a MergePolicy using the configured MergePolicyFactory
    * or if no factory is configured uses the configured mergePolicy PluginInfo.
    */
   @SuppressWarnings("unchecked")
   private MergePolicy buildMergePolicy(final IndexSchema schema) {
-    if (useMergePolicyInfo()) {
-      return buildMergePolicyFromInfo(schema);
-    }
 
     final String mpfClassName;
     final MergePolicyFactoryArgs mpfArgs;
@@ -311,59 +281,6 @@ public class SolrIndexConfig implements MapSerializable {
     return mpf.getMergePolicy();
   }
 
-  /**
-   * Builds a MergePolicy, may also modify the value returned by
-   * getUseCompoundFile() for use by the IndexWriterConfig if 
-   * "useCompoundFile" is specified as an init arg for 
-   * an out of the box MergePolicy that no longer supports it
-   *
-   * @see #fixUseCFMergePolicyInitArg
-   * @see #getUseCompoundFile
-   */
-  private MergePolicy buildMergePolicyFromInfo(IndexSchema schema) {
-    final MergePolicy policy;
-    if (mergePolicyInfo == null) {
-      final SolrResourceLoader resourceLoader = schema.getResourceLoader();
-      final MergePolicyFactoryArgs mpfArgs = new MergePolicyFactoryArgs();
-      final MergePolicyFactory defaultMergePolicyFactory = resourceLoader.newInstance(
-          DEFAULT_MERGE_POLICY_FACTORY_CLASSNAME,
-          MergePolicyFactory.class,
-          NO_SUB_PACKAGES,
-          new Class[] { SolrResourceLoader.class, MergePolicyFactoryArgs.class, IndexSchema.class },
-          new Object[] { resourceLoader, mpfArgs, schema });
-      policy = defaultMergePolicyFactory.getMergePolicy();
-    } else {
-      policy = schema.getResourceLoader().newInstance(mergePolicyInfo.className, MergePolicy.class);
-    }
-
-    if (policy instanceof LogMergePolicy) {
-      LogMergePolicy logMergePolicy = (LogMergePolicy) policy;
-      fixUseCFMergePolicyInitArg(LogMergePolicy.class);
-
-      if (maxMergeDocs != -1)
-        logMergePolicy.setMaxMergeDocs(maxMergeDocs);
-
-      if (mergeFactor != -1)
-        logMergePolicy.setMergeFactor(mergeFactor);
-    } else if (policy instanceof TieredMergePolicy) {
-      TieredMergePolicy tieredMergePolicy = (TieredMergePolicy) policy;
-      fixUseCFMergePolicyInitArg(TieredMergePolicy.class);
-
-      if (mergeFactor != -1) {
-        tieredMergePolicy.setMaxMergeAtOnce(mergeFactor);
-        tieredMergePolicy.setSegmentsPerTier(mergeFactor);
-      }
-    } else if (mergeFactor != -1) {
-      log.warn("Use of <mergeFactor> cannot be configured if merge policy is not an instance of LogMergePolicy or TieredMergePolicy. The configured policy's defaults will be used.");
-    }
-
-    if (mergePolicyInfo != null) {
-      SolrPluginUtils.invokeSetters(policy, mergePolicyInfo.initArgs);
-    }
-
-    return policy;
-  }
-
   private MergeScheduler buildMergeScheduler(IndexSchema schema) {
     String msClassName = mergeSchedulerInfo == null ? SolrIndexConfig.DEFAULT_MERGE_SCHEDULER_CLASSNAME : mergeSchedulerInfo.className;
     MergeScheduler scheduler = schema.getResourceLoader().newInstance(msClassName, MergeScheduler.class);
@@ -391,35 +308,4 @@ public class SolrIndexConfig implements MapSerializable {
     return scheduler;
   }
 
-  public boolean getUseCompoundFile() {
-    return effectiveUseCompoundFileSetting;
-  }
-
-  /**
-   * Lucene 4.4 removed the setUseCompoundFile(boolean) method from the two 
-   * conrete MergePolicies provided with Lucene/Solr and added it to the 
-   * IndexWriterConfig.  
-   * In the event that users have a value explicitly configured for this 
-   * setter in their MergePolicy init args, we remove it from the MergePolicy 
-   * init args, update the 'effective' useCompoundFile setting used by the 
-   * IndexWriterConfig, and warn about discontinuing to use this init arg.
-   * 
-   * @see #getUseCompoundFile
-   */
-  private void fixUseCFMergePolicyInitArg(Class c) {
-
-    if (null == mergePolicyInfo || null == mergePolicyInfo.initArgs) return;
-
-    Object useCFSArg = mergePolicyInfo.initArgs.remove("useCompoundFile");
-    if (null != useCFSArg) {
-      log.warn("Ignoring 'useCompoundFile' specified as an init arg for the <mergePolicy> since it is no directly longer supported by " + c.getSimpleName());
-      if (useCFSArg instanceof Boolean) {
-        boolean cfs = ((Boolean)useCFSArg).booleanValue();
-        log.warn("Please update your config to specify <useCompoundFile>"+cfs+"</useCompoundFile> directly in your <indexConfig> settings.");
-        effectiveUseCompoundFileSetting = cfs;
-      } else {
-        log.error("MergePolicy's 'useCompoundFile' init arg is not a boolean, can not apply back compat logic to apply to the IndexWriterConfig: " + useCFSArg.toString());
-      }
-    }
-  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c64f9d64/solr/core/src/test-files/solr/collection1/conf/bad-mp-solrconfig.xml
----------------------------------------------------------------------
diff --git a/solr/core/src/test-files/solr/collection1/conf/bad-mp-solrconfig.xml b/solr/core/src/test-files/solr/collection1/conf/bad-mp-solrconfig.xml
deleted file mode 100644
index 4e3504e..0000000
--- a/solr/core/src/test-files/solr/collection1/conf/bad-mp-solrconfig.xml
+++ /dev/null
@@ -1,36 +0,0 @@
-<?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.
--->
-
-<config>
-
-  <directoryFactory name="DirectoryFactory" class="${solr.directoryFactory:solr.RAMDirectoryFactory}"/>
-
-  <luceneMatchVersion>${tests.luceneMatchVersion:LATEST}</luceneMatchVersion>
-
-  <indexConfig>
-    <mergePolicy class="org.apache.solr.update.DummyMergePolicy"/>
-    <mergeFactor>8</mergeFactor>
-  </indexConfig>
-
-  <updateHandler class="solr.DirectUpdateHandler2"/>
-  <requestHandler name="standard" class="solr.StandardRequestHandler"></requestHandler>
-
-  <schemaFactory class="ClassicIndexSchemaFactory"/>
-
-</config>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c64f9d64/solr/core/src/test-files/solr/collection1/conf/solrconfig-delpolicy1.xml
----------------------------------------------------------------------
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-delpolicy1.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-delpolicy1.xml
index 33faf8b..36b4b98 100644
--- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-delpolicy1.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-delpolicy1.xml
@@ -30,8 +30,7 @@
          to configure an explicit deletion policy, but we still wnat to randomize as much 
          as possible. 
     -->
-    <mergePolicy enable="${solr.tests.useMergePolicy:true}" class="${solr.tests.mergePolicy:org.apache.solr.util.RandomMergePolicy}" />
-    <mergePolicyFactory enable="${solr.tests.useMergePolicyFactory:true}" class="${solr.tests.mergePolicyFactory:org.apache.solr.util.RandomMergePolicyFactory}" />
+    <mergePolicyFactory class="${solr.tests.mergePolicyFactory:org.apache.solr.util.RandomMergePolicyFactory}" />
     <useCompoundFile>${useCompoundFile:false}</useCompoundFile>
 
     <maxBufferedDocs>${solr.tests.maxBufferedDocs}</maxBufferedDocs>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c64f9d64/solr/core/src/test-files/solr/collection1/conf/solrconfig-delpolicy2.xml
----------------------------------------------------------------------
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-delpolicy2.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-delpolicy2.xml
index 61adf1f..06352f3 100644
--- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-delpolicy2.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-delpolicy2.xml
@@ -27,8 +27,7 @@
          to configure an explicit deletion policy, but we still wnat to randomize as much 
          as possible. 
     -->
-    <mergePolicy enable="${solr.tests.useMergePolicy:true}" class="${solr.tests.mergePolicy:org.apache.solr.util.RandomMergePolicy}" />
-    <mergePolicyFactory enable="${solr.tests.useMergePolicyFactory:true}" class="${solr.tests.mergePolicyFactory:org.apache.solr.util.RandomMergePolicyFactory}" />
+    <mergePolicyFactory class="${solr.tests.mergePolicyFactory:org.apache.solr.util.RandomMergePolicyFactory}" />
     <useCompoundFile>${useCompoundFile:false}</useCompoundFile>
 
     <maxBufferedDocs>${solr.tests.maxBufferedDocs}</maxBufferedDocs>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c64f9d64/solr/core/src/test-files/solr/collection1/conf/solrconfig-indexconfig.xml
----------------------------------------------------------------------
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-indexconfig.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-indexconfig.xml
deleted file mode 100644
index 52fb28e..0000000
--- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-indexconfig.xml
+++ /dev/null
@@ -1,31 +0,0 @@
-<?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.
--->
-<config>
-  <dataDir>${solr.data.dir:}</dataDir>
-
-  <luceneMatchVersion>${tests.luceneMatchVersion:LATEST}</luceneMatchVersion>
-
-  <indexConfig>
-    <useCompoundFile>${useCompoundFile:false}</useCompoundFile>
-    <infoStream>true</infoStream>
-    <mergePolicy class="org.apache.solr.util.RandomMergePolicy" />
-  </indexConfig>
-
-  <schemaFactory class="ClassicIndexSchemaFactory"/>
-</config>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c64f9d64/solr/core/src/test-files/solr/collection1/conf/solrconfig-indexmetrics.xml
----------------------------------------------------------------------
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-indexmetrics.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-indexmetrics.xml
index 6cf54d5..6238e7d 100644
--- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-indexmetrics.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-indexmetrics.xml
@@ -34,10 +34,14 @@
     </metrics>
     <!-- intentionally set very low values here to trigger multiple flushes and merges.
          DO NOT USE THESE ABSURD VALUES IN PRODUCTION. -->
-    <mergeFactor>3</mergeFactor>
     <maxBufferedDocs>100</maxBufferedDocs>
 
-    <mergePolicy class="org.apache.lucene.index.TieredMergePolicy"/>
+    <mergePolicyFactory class="org.apache.solr.index.TieredMergePolicyFactory">
+      <!-- intentionally set very low values here to trigger multiple flushes and merges.
+           DO NOT USE THESE ABSURD VALUES IN PRODUCTION. -->
+      <int name="maxMergeAtOnce">3</int>
+      <int name="segmentsPerTier">3</int>
+    </mergePolicyFactory>
   </indexConfig>
 
   <updateHandler class="solr.DirectUpdateHandler2">

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c64f9d64/solr/core/src/test-files/solr/collection1/conf/solrconfig-logmergepolicy.xml
----------------------------------------------------------------------
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-logmergepolicy.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-logmergepolicy.xml
deleted file mode 100644
index 49eec42..0000000
--- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-logmergepolicy.xml
+++ /dev/null
@@ -1,36 +0,0 @@
-<?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.
--->
-
-<config>
-  <luceneMatchVersion>${tests.luceneMatchVersion:LATEST}</luceneMatchVersion>
-  <directoryFactory name="DirectoryFactory" class="${solr.directoryFactory:solr.RAMDirectoryFactory}"/>
-  <schemaFactory class="ClassicIndexSchemaFactory"/>
-  <indexConfig>
-    <!-- set some values to -1 to force the use of internal lucene defaults -->
-    <maxBufferedDocs>-1</maxBufferedDocs>
-    <ramBufferSizeMB>-1</ramBufferSizeMB>
-
-    <mergeFactor>11</mergeFactor>
-    <maxMergeDocs>456</maxMergeDocs>
-    <mergePolicy class="${solr.test.log.merge.policy}" />
-  </indexConfig>
-
-  <requestHandler name="standard" class="solr.StandardRequestHandler"></requestHandler>
-
-</config>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c64f9d64/solr/core/src/test-files/solr/collection1/conf/solrconfig-mergepolicy-legacy.xml
----------------------------------------------------------------------
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-mergepolicy-legacy.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-mergepolicy-legacy.xml
index 6e6c6fe..a7361cc 100644
--- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-mergepolicy-legacy.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-mergepolicy-legacy.xml
@@ -23,7 +23,6 @@
   <schemaFactory class="ClassicIndexSchemaFactory"/>
 
   <indexConfig>
-    <mergeFactor>7</mergeFactor>
     <useCompoundFile>${useCompoundFile:false}</useCompoundFile>
   </indexConfig>
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c64f9d64/solr/core/src/test-files/solr/collection1/conf/solrconfig-mergepolicy-nocfs.xml
----------------------------------------------------------------------
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-mergepolicy-nocfs.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-mergepolicy-nocfs.xml
deleted file mode 100644
index b59cdc8..0000000
--- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-mergepolicy-nocfs.xml
+++ /dev/null
@@ -1,34 +0,0 @@
-<?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.
--->
-
-<config>
-  <luceneMatchVersion>${tests.luceneMatchVersion:LATEST}</luceneMatchVersion>
-  <directoryFactory name="DirectoryFactory" class="${solr.directoryFactory:solr.RAMDirectoryFactory}"/>
-  <schemaFactory class="ClassicIndexSchemaFactory"/>
-
-  <indexConfig>
-    <useCompoundFile>${testSetNoCFSMergePolicyConfig.useCompoundFile:false}</useCompoundFile>
-    <mergePolicy class="org.apache.lucene.index.TieredMergePolicy">
-      <double name="noCFSRatio">0.5</double>
-    </mergePolicy>
-  </indexConfig>
-
-  <requestHandler name="standard" class="solr.StandardRequestHandler"></requestHandler>
-
-</config>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c64f9d64/solr/core/src/test-files/solr/collection1/conf/solrconfig-spellcheckcomponent.xml
----------------------------------------------------------------------
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-spellcheckcomponent.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-spellcheckcomponent.xml
index 5324a35..cf923c4 100644
--- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-spellcheckcomponent.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-spellcheckcomponent.xml
@@ -20,8 +20,7 @@
 <config>
   <luceneMatchVersion>${tests.luceneMatchVersion:LATEST}</luceneMatchVersion>
   <indexConfig>
-    <mergePolicy enable="${solr.tests.useMergePolicy:true}" class="${solr.tests.mergePolicy:org.apache.solr.util.RandomMergePolicy}" />
-    <mergePolicyFactory enable="${solr.tests.useMergePolicyFactory:true}" class="${solr.tests.mergePolicyFactory:org.apache.solr.util.RandomMergePolicyFactory}" />
+    <mergePolicyFactory class="${solr.tests.mergePolicyFactory:org.apache.solr.util.RandomMergePolicyFactory}" />
     <useCompoundFile>${useCompoundFile:false}</useCompoundFile>
     <maxBufferedDocs>${solr.tests.maxBufferedDocs}</maxBufferedDocs>
     <ramBufferSizeMB>${solr.tests.ramBufferSizeMB}</ramBufferSizeMB>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c64f9d64/solr/core/src/test-files/solr/collection1/conf/solrconfig-tieredmergepolicy.xml
----------------------------------------------------------------------
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-tieredmergepolicy.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-tieredmergepolicy.xml
deleted file mode 100644
index 026b3c7..0000000
--- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-tieredmergepolicy.xml
+++ /dev/null
@@ -1,48 +0,0 @@
-<?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.
--->
-
-<config>
-  <luceneMatchVersion>${tests.luceneMatchVersion:LATEST}</luceneMatchVersion>
-  <directoryFactory name="DirectoryFactory" class="${solr.directoryFactory:solr.RAMDirectoryFactory}"/>
-  <schemaFactory class="ClassicIndexSchemaFactory"/>
-
-  <indexConfig>
-    <mergeFactor>7</mergeFactor>
-    <mergePolicy class="org.apache.lucene.index.TieredMergePolicy">
-      <int name="maxMergeAtOnceExplicit">19</int>
-      <int name="segmentsPerTier">9</int>
-      <double name="noCFSRatio">0.1</double>
-
-      <!-- Setter for this was moved from the MergePolicies to IndexWriterConfig
-           in Lucene 4.4, so we should treat it the same as a <useCompoundFile>
-           setting and log a warning (instead of failing because the setter is 
-           gone).
-      -->
-      <bool name="useCompoundFile">${useCompoundFile:false}</bool>
-
-    </mergePolicy>
-    <mergeScheduler class="org.apache.lucene.index.ConcurrentMergeScheduler">
-      <int name="maxMergeCount">987</int>
-      <int name="maxThreadCount">42</int>
-    </mergeScheduler>
-  </indexConfig>
-
-  <requestHandler name="standard" class="solr.StandardRequestHandler"></requestHandler>
-
-</config>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c64f9d64/solr/core/src/test-files/solr/collection1/conf/solrconfig-warmer.xml
----------------------------------------------------------------------
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-warmer.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-warmer.xml
deleted file mode 100644
index 691c94b..0000000
--- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-warmer.xml
+++ /dev/null
@@ -1,46 +0,0 @@
-<?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.
--->
-
-<!-- a basic solrconfig that tests can use when they want simple minimal solrconfig/schema
-     DO NOT ADD THINGS TO THIS CONFIG! -->
-<config>
-  <luceneMatchVersion>${tests.luceneMatchVersion:LATEST}</luceneMatchVersion>
-  <dataDir>${solr.data.dir:}</dataDir>
-  <directoryFactory name="DirectoryFactory" class="${solr.directoryFactory:solr.RAMDirectoryFactory}"/>
-  <schemaFactory class="ClassicIndexSchemaFactory"/>
-  <requestHandler name="standard" class="solr.StandardRequestHandler"></requestHandler>
-  
-  <indexConfig>
-    <!-- we don't use solrconfig.snippet.randomindexconfig.xml here
-         because we explicitly test that a mergedSegmentWarmer works, 
-         but we can still test some of the other randomized indexConfig 
-         settings
-    -->
-    <mergedSegmentWarmer class="org.apache.lucene.index.SimpleMergedSegmentWarmer"/>
-    <mergePolicy class="org.apache.solr.util.RandomMergePolicy" />
-  
-    <useCompoundFile>${useCompoundFile}</useCompoundFile>
-    <maxBufferedDocs>${solr.tests.maxBufferedDocs}</maxBufferedDocs>
-    <ramBufferSizeMB>${solr.tests.ramBufferSizeMB}</ramBufferSizeMB>
-    <mergeScheduler class="${solr.tests.mergeScheduler}" />
-    <writeLockTimeout>1000</writeLockTimeout>
-    <commitLockTimeout>10000</commitLockTimeout>
-    <lockType>single</lockType>
-  </indexConfig>
-</config>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c64f9d64/solr/core/src/test-files/solr/collection1/conf/solrconfig.snippet.randomindexconfig.xml
----------------------------------------------------------------------
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig.snippet.randomindexconfig.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig.snippet.randomindexconfig.xml
index ec5f54e..ecf1f14 100644
--- a/solr/core/src/test-files/solr/collection1/conf/solrconfig.snippet.randomindexconfig.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig.snippet.randomindexconfig.xml
@@ -26,8 +26,7 @@ A solrconfig.xml snippet containing indexConfig settings for randomized testing.
        the RandomMergePolicy in all tests - but some tests expect very specific
        Merge behavior, so those tests can set it as needed.
   -->
-  <mergePolicy enable="${solr.tests.useMergePolicy:true}" class="${solr.tests.mergePolicy:org.apache.solr.util.RandomMergePolicy}" />
-  <mergePolicyFactory enable="${solr.tests.useMergePolicyFactory:true}" class="${solr.tests.mergePolicyFactory:org.apache.solr.util.RandomMergePolicyFactory}" />
+  <mergePolicyFactory class="${solr.tests.mergePolicyFactory:org.apache.solr.util.RandomMergePolicyFactory}" />
   
   <useCompoundFile>${useCompoundFile:false}</useCompoundFile>
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c64f9d64/solr/core/src/test/org/apache/solr/TestGroupingSearch.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/TestGroupingSearch.java b/solr/core/src/test/org/apache/solr/TestGroupingSearch.java
index 2d46551..99dc63c 100644
--- a/solr/core/src/test/org/apache/solr/TestGroupingSearch.java
+++ b/solr/core/src/test/org/apache/solr/TestGroupingSearch.java
@@ -30,7 +30,6 @@ import java.util.List;
 import java.util.Map;
 import java.util.TreeMap;
 
-import org.apache.lucene.index.LogDocMergePolicy;
 import org.apache.solr.client.solrj.impl.BinaryResponseParser;
 import org.apache.solr.common.params.CommonParams;
 import org.apache.solr.common.params.GroupParams;
@@ -63,7 +62,6 @@ public class TestGroupingSearch extends SolrTestCaseJ4 {
   public static void beforeTests() throws Exception {
     // force LogDocMergePolicy so that we get a predictable doc order
     // when doing unsorted group collection
-    systemSetPropertySolrTestsMergePolicy(LogDocMergePolicy.class.getName());
     systemSetPropertySolrTestsMergePolicyFactory(LogDocMergePolicyFactory.class.getName());
 
     System.setProperty("enable.update.log", "false"); // schema12 doesn't support _version_
@@ -72,7 +70,6 @@ public class TestGroupingSearch extends SolrTestCaseJ4 {
 
   @AfterClass
   public static void afterTests() {
-    systemClearPropertySolrTestsMergePolicy();
     systemClearPropertySolrTestsMergePolicyFactory();
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c64f9d64/solr/core/src/test/org/apache/solr/cloud/TestAuthenticationFramework.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestAuthenticationFramework.java b/solr/core/src/test/org/apache/solr/cloud/TestAuthenticationFramework.java
index 5fd8e42..df3790d 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestAuthenticationFramework.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestAuthenticationFramework.java
@@ -32,7 +32,6 @@ import org.apache.http.HttpException;
 import org.apache.http.HttpRequest;
 import org.apache.http.HttpRequestInterceptor;
 import org.apache.http.protocol.HttpContext;
-import org.apache.lucene.index.TieredMergePolicy;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.LuceneTestCase.SuppressSysoutChecks;
 import org.apache.solr.SolrTestCaseJ4;
@@ -149,15 +148,7 @@ public class TestAuthenticationFramework extends LuceneTestCase {
     collectionProperties.putIfAbsent("solr.tests.maxBufferedDocs", "100000");
     collectionProperties.putIfAbsent("solr.tests.ramBufferSizeMB", "100");
     // use non-test classes so RandomizedRunner isn't necessary
-    if (random().nextBoolean()) {
-      collectionProperties.putIfAbsent(SolrTestCaseJ4.SYSTEM_PROPERTY_SOLR_TESTS_MERGEPOLICY, TieredMergePolicy.class.getName());
-      collectionProperties.putIfAbsent(SolrTestCaseJ4.SYSTEM_PROPERTY_SOLR_TESTS_USEMERGEPOLICY, "true");
-      collectionProperties.putIfAbsent(SolrTestCaseJ4.SYSTEM_PROPERTY_SOLR_TESTS_USEMERGEPOLICYFACTORY, "false");
-    } else {
-      collectionProperties.putIfAbsent(SolrTestCaseJ4.SYSTEM_PROPERTY_SOLR_TESTS_MERGEPOLICYFACTORY, TieredMergePolicyFactory.class.getName());
-      collectionProperties.putIfAbsent(SolrTestCaseJ4.SYSTEM_PROPERTY_SOLR_TESTS_USEMERGEPOLICYFACTORY, "true");
-      collectionProperties.putIfAbsent(SolrTestCaseJ4.SYSTEM_PROPERTY_SOLR_TESTS_USEMERGEPOLICY, "false");
-    }
+    collectionProperties.putIfAbsent(SolrTestCaseJ4.SYSTEM_PROPERTY_SOLR_TESTS_MERGEPOLICYFACTORY, TieredMergePolicyFactory.class.getName());
     collectionProperties.putIfAbsent("solr.tests.mergeScheduler", "org.apache.lucene.index.ConcurrentMergeScheduler");
     collectionProperties.putIfAbsent("solr.directoryFactory", (persistIndex ? "solr.StandardDirectoryFactory" : "solr.RAMDirectoryFactory"));
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c64f9d64/solr/core/src/test/org/apache/solr/cloud/TestMiniSolrCloudCluster.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestMiniSolrCloudCluster.java b/solr/core/src/test/org/apache/solr/cloud/TestMiniSolrCloudCluster.java
index de18875..a8250f8 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestMiniSolrCloudCluster.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestMiniSolrCloudCluster.java
@@ -27,7 +27,6 @@ import java.util.List;
 import java.util.Map;
 
 import com.carrotsearch.randomizedtesting.rules.SystemPropertiesRestoreRule;
-import org.apache.lucene.index.TieredMergePolicy;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.LuceneTestCase.SuppressSysoutChecks;
 import org.apache.solr.SolrTestCaseJ4;
@@ -102,15 +101,7 @@ public class TestMiniSolrCloudCluster extends LuceneTestCase {
     collectionProperties.putIfAbsent("solr.tests.maxBufferedDocs", "100000");
     collectionProperties.putIfAbsent("solr.tests.ramBufferSizeMB", "100");
     // use non-test classes so RandomizedRunner isn't necessary
-    if (random().nextBoolean()) {
-      collectionProperties.putIfAbsent(SolrTestCaseJ4.SYSTEM_PROPERTY_SOLR_TESTS_MERGEPOLICY, TieredMergePolicy.class.getName());
-      collectionProperties.putIfAbsent(SolrTestCaseJ4.SYSTEM_PROPERTY_SOLR_TESTS_USEMERGEPOLICY, "true");
-      collectionProperties.putIfAbsent(SolrTestCaseJ4.SYSTEM_PROPERTY_SOLR_TESTS_USEMERGEPOLICYFACTORY, "false");
-    } else {
-      collectionProperties.putIfAbsent(SolrTestCaseJ4.SYSTEM_PROPERTY_SOLR_TESTS_MERGEPOLICYFACTORY, TieredMergePolicyFactory.class.getName());
-      collectionProperties.putIfAbsent(SolrTestCaseJ4.SYSTEM_PROPERTY_SOLR_TESTS_USEMERGEPOLICYFACTORY, "true");
-      collectionProperties.putIfAbsent(SolrTestCaseJ4.SYSTEM_PROPERTY_SOLR_TESTS_USEMERGEPOLICY, "false");
-    }
+    collectionProperties.putIfAbsent(SolrTestCaseJ4.SYSTEM_PROPERTY_SOLR_TESTS_MERGEPOLICYFACTORY, TieredMergePolicyFactory.class.getName());
     collectionProperties.putIfAbsent("solr.tests.mergeScheduler", "org.apache.lucene.index.ConcurrentMergeScheduler");
     collectionProperties.putIfAbsent("solr.directoryFactory", (persistIndex ? "solr.StandardDirectoryFactory" : "solr.RAMDirectoryFactory"));
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c64f9d64/solr/core/src/test/org/apache/solr/cloud/TestSolrCloudWithKerberosAlt.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestSolrCloudWithKerberosAlt.java b/solr/core/src/test/org/apache/solr/cloud/TestSolrCloudWithKerberosAlt.java
index 042c111..1758ae0 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestSolrCloudWithKerberosAlt.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestSolrCloudWithKerberosAlt.java
@@ -23,7 +23,6 @@ import java.util.List;
 import java.util.Properties;
 
 import org.apache.commons.io.FileUtils;
-import org.apache.lucene.index.TieredMergePolicy;
 import org.apache.lucene.util.Constants;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.solr.SolrTestCaseJ4;
@@ -180,15 +179,7 @@ public class TestSolrCloudWithKerberosAlt extends LuceneTestCase {
       properties.put("solr.tests.maxBufferedDocs", "100000");
       properties.put("solr.tests.ramBufferSizeMB", "100");
       // use non-test classes so RandomizedRunner isn't necessary
-      if (random().nextBoolean()) {
-        properties.put(SolrTestCaseJ4.SYSTEM_PROPERTY_SOLR_TESTS_MERGEPOLICY, TieredMergePolicy.class.getName());
-        properties.put(SolrTestCaseJ4.SYSTEM_PROPERTY_SOLR_TESTS_USEMERGEPOLICY, "true");
-        properties.put(SolrTestCaseJ4.SYSTEM_PROPERTY_SOLR_TESTS_USEMERGEPOLICYFACTORY, "false");
-      } else {
-        properties.put(SolrTestCaseJ4.SYSTEM_PROPERTY_SOLR_TESTS_MERGEPOLICYFACTORY, TieredMergePolicyFactory.class.getName());
-        properties.put(SolrTestCaseJ4.SYSTEM_PROPERTY_SOLR_TESTS_USEMERGEPOLICYFACTORY, "true");
-        properties.put(SolrTestCaseJ4.SYSTEM_PROPERTY_SOLR_TESTS_USEMERGEPOLICY, "false");
-      }
+      properties.put(SolrTestCaseJ4.SYSTEM_PROPERTY_SOLR_TESTS_MERGEPOLICYFACTORY, TieredMergePolicyFactory.class.getName());
       properties.put("solr.tests.mergeScheduler", "org.apache.lucene.index.ConcurrentMergeScheduler");
       properties.put("solr.directoryFactory", "solr.RAMDirectoryFactory");
       createRequest.setProperties(properties);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c64f9d64/solr/core/src/test/org/apache/solr/core/TestBadConfig.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/core/TestBadConfig.java b/solr/core/src/test/org/apache/solr/core/TestBadConfig.java
index 31361be..db04152 100644
--- a/solr/core/src/test/org/apache/solr/core/TestBadConfig.java
+++ b/solr/core/src/test/org/apache/solr/core/TestBadConfig.java
@@ -76,8 +76,6 @@ public class TestBadConfig extends AbstractBadConfigTestBase {
   }
 
   public void testBogusMergePolicy() throws Exception {
-    assertConfigs("bad-mp-solrconfig.xml", "schema-minimal.xml",
-                  "DummyMergePolicy");
     assertConfigs("bad-mpf-solrconfig.xml", "schema-minimal.xml",
                   "DummyMergePolicyFactory");
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c64f9d64/solr/core/src/test/org/apache/solr/core/TestConfig.java
----------------------------------------------------------------------
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 094f013..87a453f 100644
--- a/solr/core/src/test/org/apache/solr/core/TestConfig.java
+++ b/solr/core/src/test/org/apache/solr/core/TestConfig.java
@@ -148,11 +148,9 @@ public class TestConfig extends SolrTestCaseJ4 {
     SolrConfig sc = new SolrConfig(new SolrResourceLoader(TEST_PATH().resolve("collection1")), "solrconfig-defaults.xml", null);
     SolrIndexConfig sic = sc.indexConfig;
 
-    ++numDefaultsTested; assertEquals("default useCompoundFile", false, sic.getUseCompoundFile());
+    ++numDefaultsTested; assertEquals("default useCompoundFile", false, sic.useCompoundFile);
 
     ++numDefaultsTested; assertEquals("default maxBufferedDocs", -1, sic.maxBufferedDocs);
-    ++numDefaultsTested; assertEquals("default maxMergeDocs", -1, sic.maxMergeDocs);
-    ++numDefaultsTested; assertEquals("default mergeFactor", -1, sic.mergeFactor);
 
     ++numDefaultsTested; assertEquals("default ramBufferSizeMB", 100.0D, sic.ramBufferSizeMB, 0.0D);
     ++numDefaultsTested; assertEquals("default writeLockTimeout", -1, sic.writeLockTimeout);
@@ -162,10 +160,7 @@ public class TestConfig extends SolrTestCaseJ4 {
 
     ++numDefaultsTested; assertNotNull("default metrics", sic.metricsInfo);
 
-    // mergePolicyInfo and mergePolicyFactoryInfo are mutually exclusive
-    // so ++ count them only once for both instead of individually
     ++numDefaultsTested; ++numNullDefaults;
-    assertNull("default mergePolicyInfo", sic.mergePolicyInfo);
     assertNull("default mergePolicyFactoryInfo", sic.mergePolicyFactoryInfo);
 
     ++numDefaultsTested; ++numNullDefaults; assertNull("default mergeSchedulerInfo", sic.mergeSchedulerInfo);
@@ -197,7 +192,7 @@ public class TestConfig extends SolrTestCaseJ4 {
                  Double.parseDouble(System.getProperty("solr.tests.ramBufferSizeMB")), 
                                     sic.ramBufferSizeMB, 0.0D);
     assertEquals("useCompoundFile sysprop", 
-                 Boolean.parseBoolean(System.getProperty("useCompoundFile")), sic.getUseCompoundFile());
+                 Boolean.parseBoolean(System.getProperty("useCompoundFile")), sic.useCompoundFile);
   }
 
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c64f9d64/solr/core/src/test/org/apache/solr/core/TestMergePolicyConfig.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/core/TestMergePolicyConfig.java b/solr/core/src/test/org/apache/solr/core/TestMergePolicyConfig.java
index f8e232a..3de61be 100644
--- a/solr/core/src/test/org/apache/solr/core/TestMergePolicyConfig.java
+++ b/solr/core/src/test/org/apache/solr/core/TestMergePolicyConfig.java
@@ -50,7 +50,7 @@ public class TestMergePolicyConfig extends SolrTestCaseJ4 {
     final boolean useCompoundFile = random().nextBoolean();
     System.setProperty("testSetNoCFSMergePolicyConfig.useCompoundFile", String.valueOf(useCompoundFile));
     try {
-      initCore(random().nextBoolean() ? "solrconfig-mergepolicy-nocfs.xml" : "solrconfig-mergepolicyfactory-nocfs.xml","schema-minimal.xml");
+      initCore("solrconfig-mergepolicyfactory-nocfs.xml","schema-minimal.xml");
       IndexWriterConfig iwc = solrConfig.indexConfig.toIndexWriterConfig(h.getCore());
       assertEquals(useCompoundFile, iwc.getUseCompoundFile());
 
@@ -84,8 +84,8 @@ public class TestMergePolicyConfig extends SolrTestCaseJ4 {
 
     TieredMergePolicy tieredMP = assertAndCast(TieredMergePolicy.class, iwc.getMergePolicy());
 
-    assertEquals(7, tieredMP.getMaxMergeAtOnce());
-    assertEquals(7.0D, tieredMP.getSegmentsPerTier(), 0.0D);
+    assertEquals(10, tieredMP.getMaxMergeAtOnce());
+    assertEquals(10.0D, tieredMP.getSegmentsPerTier(), 0.0D);
 
     assertCommitSomeNewDocs();
     assertCompoundSegments(h.getCore(), expectCFS);
@@ -95,7 +95,7 @@ public class TestMergePolicyConfig extends SolrTestCaseJ4 {
     final boolean expectCFS 
       = Boolean.parseBoolean(System.getProperty("useCompoundFile"));
 
-    initCore(random().nextBoolean() ? "solrconfig-tieredmergepolicy.xml" : "solrconfig-tieredmergepolicyfactory.xml","schema-minimal.xml");
+    initCore("solrconfig-tieredmergepolicyfactory.xml","schema-minimal.xml");
     IndexWriterConfig iwc = solrConfig.indexConfig.toIndexWriterConfig(h.getCore());
     assertEquals(expectCFS, iwc.getUseCompoundFile());
 
@@ -148,16 +148,6 @@ public class TestMergePolicyConfig extends SolrTestCaseJ4 {
     assertEquals(mergePolicy, iwc.getMergePolicy());
   }
 
-  public void testLogMergePolicyConfig() throws Exception {
-    
-    final Class<? extends LogMergePolicy> mpClass = random().nextBoolean()
-      ? LogByteSizeMergePolicy.class : LogDocMergePolicy.class;
-
-    System.setProperty("solr.test.log.merge.policy", mpClass.getName());
-
-    implTestLogMergePolicyConfig("solrconfig-logmergepolicy.xml", mpClass);
-  }
-
   public void testLogMergePolicyFactoryConfig() throws Exception {
 
     final boolean byteSizeMP = random().nextBoolean();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c64f9d64/solr/core/src/test/org/apache/solr/core/TestNRTOpen.java
----------------------------------------------------------------------
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 0970953..910c618 100644
--- a/solr/core/src/test/org/apache/solr/core/TestNRTOpen.java
+++ b/solr/core/src/test/org/apache/solr/core/TestNRTOpen.java
@@ -23,7 +23,6 @@ import java.util.Set;
 
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.DirectoryReader;
-import org.apache.lucene.index.LogDocMergePolicy;
 import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.index.LogDocMergePolicyFactory;
 import org.apache.solr.search.SolrIndexSearcher;
@@ -41,7 +40,6 @@ public class TestNRTOpen extends SolrTestCaseJ4 {
     System.setProperty("solr.test.leavedatadir", "true");
     // set these so that merges won't break the test
     System.setProperty("solr.tests.maxBufferedDocs", "100000");
-    systemSetPropertySolrTestsMergePolicy(LogDocMergePolicy.class.getName());
     systemSetPropertySolrTestsMergePolicyFactory(LogDocMergePolicyFactory.class.getName());
     initCore("solrconfig-basic.xml", "schema-minimal.xml");
     // add a doc
@@ -62,7 +60,6 @@ public class TestNRTOpen extends SolrTestCaseJ4 {
     System.clearProperty("solr.test.leavedatadir");
     System.clearProperty("solr.directoryFactory");
     System.clearProperty("solr.tests.maxBufferedDocs");
-    systemClearPropertySolrTestsMergePolicy();
     systemClearPropertySolrTestsMergePolicyFactory();
   }
   

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c64f9d64/solr/core/src/test/org/apache/solr/core/TestSolrIndexConfig.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/core/TestSolrIndexConfig.java b/solr/core/src/test/org/apache/solr/core/TestSolrIndexConfig.java
index 978ad0f..2a2d8e6 100644
--- a/solr/core/src/test/org/apache/solr/core/TestSolrIndexConfig.java
+++ b/solr/core/src/test/org/apache/solr/core/TestSolrIndexConfig.java
@@ -31,7 +31,7 @@ public class TestSolrIndexConfig extends SolrTestCaseJ4 {
 
   @BeforeClass
   public static void beforeClass() throws Exception {
-    initCore(random().nextBoolean() ? "solrconfig-indexconfig.xml" : "solrconfig-indexconfig-mergepolicyfactory.xml","schema.xml");
+    initCore("solrconfig-indexconfig-mergepolicyfactory.xml","schema.xml");
   }
 
   public void testLiveWriter() throws Exception {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c64f9d64/solr/core/src/test/org/apache/solr/handler/admin/SegmentsInfoRequestHandlerTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/handler/admin/SegmentsInfoRequestHandlerTest.java b/solr/core/src/test/org/apache/solr/handler/admin/SegmentsInfoRequestHandlerTest.java
index 1355e56..6de7835 100644
--- a/solr/core/src/test/org/apache/solr/handler/admin/SegmentsInfoRequestHandlerTest.java
+++ b/solr/core/src/test/org/apache/solr/handler/admin/SegmentsInfoRequestHandlerTest.java
@@ -16,7 +16,6 @@
  */
 package org.apache.solr.handler.admin;
 
-import org.apache.lucene.index.LogDocMergePolicy;
 import org.apache.lucene.util.Version;
 import org.apache.solr.index.LogDocMergePolicyFactory;
 import org.apache.solr.util.AbstractSolrTestCase;
@@ -38,7 +37,6 @@ public class SegmentsInfoRequestHandlerTest extends AbstractSolrTestCase {
     // we need a consistent segmentation to ensure we don't get a random
     // merge that reduces the total num docs in all segments, or the number of deletes
     //
-    systemSetPropertySolrTestsMergePolicy(LogDocMergePolicy.class.getName());
     systemSetPropertySolrTestsMergePolicyFactory(LogDocMergePolicyFactory.class.getName());
     
     System.setProperty("enable.update.log", "false"); // no _version_ in our schema

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c64f9d64/solr/core/src/test/org/apache/solr/schema/TestHalfAndHalfDocValues.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/schema/TestHalfAndHalfDocValues.java b/solr/core/src/test/org/apache/solr/schema/TestHalfAndHalfDocValues.java
index b0c3956..feb9236 100644
--- a/solr/core/src/test/org/apache/solr/schema/TestHalfAndHalfDocValues.java
+++ b/solr/core/src/test/org/apache/solr/schema/TestHalfAndHalfDocValues.java
@@ -42,10 +42,6 @@ public class TestHalfAndHalfDocValues extends SolrTestCaseJ4 {
     // segments with and without docvalues
     systemSetPropertySolrTestsMergePolicyFactory(NoMergePolicyFactory.class.getName());
 
-    // HACK: Don't use a RandomMergePolicy, but only use the mergePolicyFactory that we've just set
-    System.setProperty(SYSTEM_PROPERTY_SOLR_TESTS_USEMERGEPOLICYFACTORY, "true");
-    System.setProperty(SYSTEM_PROPERTY_SOLR_TESTS_USEMERGEPOLICY, "false");
-
     initCore("solrconfig-basic.xml", "schema-docValues.xml");
 
     // sanity check our schema meets our expectations

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c64f9d64/solr/core/src/test/org/apache/solr/search/TestIndexSearcher.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/search/TestIndexSearcher.java b/solr/core/src/test/org/apache/solr/search/TestIndexSearcher.java
index c36066a..457129f 100644
--- a/solr/core/src/test/org/apache/solr/search/TestIndexSearcher.java
+++ b/solr/core/src/test/org/apache/solr/search/TestIndexSearcher.java
@@ -32,7 +32,6 @@ import com.google.common.collect.ImmutableMap;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.IndexReaderContext;
 import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.index.LogDocMergePolicy;
 import org.apache.lucene.index.ReaderUtil;
 import org.apache.lucene.queries.function.FunctionValues;
 import org.apache.lucene.queries.function.ValueSource;
@@ -61,7 +60,6 @@ public class TestIndexSearcher extends SolrTestCaseJ4 {
 
     // we need a consistent segmentation because reopen test validation
     // dependso n merges not happening when it doesn't expect
-    systemSetPropertySolrTestsMergePolicy(LogDocMergePolicy.class.getName());
     systemSetPropertySolrTestsMergePolicyFactory(LogDocMergePolicyFactory.class.getName());
 
     initCore("solrconfig.xml","schema.xml");
@@ -69,7 +67,6 @@ public class TestIndexSearcher extends SolrTestCaseJ4 {
   
   @AfterClass
   public static void afterClass() {
-    systemClearPropertySolrTestsMergePolicy();
     systemClearPropertySolrTestsMergePolicyFactory();
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c64f9d64/solr/core/src/test/org/apache/solr/update/DirectUpdateHandlerTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/update/DirectUpdateHandlerTest.java b/solr/core/src/test/org/apache/solr/update/DirectUpdateHandlerTest.java
index 7d97ee4..90bb6a0 100644
--- a/solr/core/src/test/org/apache/solr/update/DirectUpdateHandlerTest.java
+++ b/solr/core/src/test/org/apache/solr/update/DirectUpdateHandlerTest.java
@@ -27,7 +27,6 @@ import com.codahale.metrics.Gauge;
 import com.codahale.metrics.Meter;
 import com.codahale.metrics.Metric;
 import org.apache.lucene.index.DirectoryReader;
-import org.apache.lucene.index.TieredMergePolicy;
 import org.apache.lucene.store.Directory;
 import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.common.params.CommonParams;
@@ -62,14 +61,12 @@ public class DirectUpdateHandlerTest extends SolrTestCaseJ4 {
     savedFactory = System.getProperty("solr.DirectoryFactory");
     System.setProperty("solr.directoryFactory", "org.apache.solr.core.MockFSDirectoryFactory");
     System.setProperty("enable.update.log", "false"); // schema12 doesn't support _version_
-    systemSetPropertySolrTestsMergePolicy(TieredMergePolicy.class.getName());
     systemSetPropertySolrTestsMergePolicyFactory(TieredMergePolicyFactory.class.getName());
     initCore("solrconfig.xml", "schema12.xml");
   }
   
   @AfterClass
   public static void afterClass() {
-    systemClearPropertySolrTestsMergePolicy();
     systemClearPropertySolrTestsMergePolicyFactory();
     if (savedFactory == null) {
       System.clearProperty("solr.directoryFactory");

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c64f9d64/solr/core/src/test/org/apache/solr/update/SolrCmdDistributorTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/update/SolrCmdDistributorTest.java b/solr/core/src/test/org/apache/solr/update/SolrCmdDistributorTest.java
index cb35e88..20c2f1a 100644
--- a/solr/core/src/test/org/apache/solr/update/SolrCmdDistributorTest.java
+++ b/solr/core/src/test/org/apache/solr/update/SolrCmdDistributorTest.java
@@ -25,7 +25,6 @@ import java.util.Collections;
 import java.util.List;
 import java.util.concurrent.atomic.AtomicInteger;
 
-import org.apache.lucene.index.LogDocMergePolicy;
 import org.apache.solr.BaseDistributedSearchTestCase;
 import org.apache.solr.client.solrj.SolrClient;
 import org.apache.solr.client.solrj.SolrQuery;
@@ -65,13 +64,11 @@ public class SolrCmdDistributorTest extends BaseDistributedSearchTestCase {
   public static void beforeClass() throws Exception {
     // we can't use the Randomized merge policy because the test depends on
     // being able to call optimize to have all deletes expunged.
-    systemSetPropertySolrTestsMergePolicy(LogDocMergePolicy.class.getName());
     systemSetPropertySolrTestsMergePolicyFactory(LogDocMergePolicyFactory.class.getName());
   }
 
   @AfterClass
   public static void afterClass() {
-    systemClearPropertySolrTestsMergePolicy();
     systemClearPropertySolrTestsMergePolicyFactory();
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c64f9d64/solr/core/src/test/org/apache/solr/update/SolrIndexConfigTest.java
----------------------------------------------------------------------
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 0f53f33..ec5719c 100644
--- a/solr/core/src/test/org/apache/solr/update/SolrIndexConfigTest.java
+++ b/solr/core/src/test/org/apache/solr/update/SolrIndexConfigTest.java
@@ -46,9 +46,7 @@ import org.junit.Test;
 public class SolrIndexConfigTest extends SolrTestCaseJ4 {
 
   private static final String solrConfigFileName = "solrconfig.xml";
-  private static final String solrConfigFileNameWarmerRandomMergePolicy = "solrconfig-warmer.xml";
   private static final String solrConfigFileNameWarmerRandomMergePolicyFactory = "solrconfig-warmer-randommergepolicyfactory.xml";
-  private static final String solrConfigFileNameTieredMergePolicy = "solrconfig-tieredmergepolicy.xml";
   private static final String solrConfigFileNameTieredMergePolicyFactory = "solrconfig-tieredmergepolicyfactory.xml";
   private static final String solrConfigFileNameSortingMergePolicyFactory = "solrconfig-sortingmergepolicyfactory.xml";
   private static final String schemaFileName = "schema.xml";
@@ -63,7 +61,7 @@ public class SolrIndexConfigTest extends SolrTestCaseJ4 {
   @Test
   public void testFailingSolrIndexConfigCreation() {
     try {
-      SolrConfig solrConfig = new SolrConfig(random().nextBoolean() ? "bad-mp-solrconfig.xml" : "bad-mpf-solrconfig.xml");
+      SolrConfig solrConfig = new SolrConfig("bad-mpf-solrconfig.xml");
       SolrIndexConfig solrIndexConfig = new SolrIndexConfig(solrConfig, null, null);
       IndexSchema indexSchema = IndexSchemaFactory.buildIndexSchema(schemaFileName, solrConfig);
       h.getCore().setLatestSchema(indexSchema);
@@ -76,7 +74,7 @@ public class SolrIndexConfigTest extends SolrTestCaseJ4 {
 
   @Test
   public void testTieredMPSolrIndexConfigCreation() throws Exception {
-    String solrConfigFileName = random().nextBoolean() ? solrConfigFileNameTieredMergePolicy : solrConfigFileNameTieredMergePolicyFactory;
+    String solrConfigFileName = solrConfigFileNameTieredMergePolicyFactory;
     SolrConfig solrConfig = new SolrConfig(instanceDir, solrConfigFileName, null);
     SolrIndexConfig solrIndexConfig = new SolrIndexConfig(solrConfig, null, null);
     IndexSchema indexSchema = IndexSchemaFactory.buildIndexSchema(schemaFileName, solrConfig);
@@ -121,7 +119,7 @@ public class SolrIndexConfigTest extends SolrTestCaseJ4 {
   }
 
   public void testMergedSegmentWarmerIndexConfigCreation() throws Exception {
-    SolrConfig solrConfig = new SolrConfig(instanceDir, random().nextBoolean() ? solrConfigFileNameWarmerRandomMergePolicy : solrConfigFileNameWarmerRandomMergePolicyFactory, null);
+    SolrConfig solrConfig = new SolrConfig(instanceDir, solrConfigFileNameWarmerRandomMergePolicyFactory, null);
     SolrIndexConfig solrIndexConfig = new SolrIndexConfig(solrConfig, null, null);
     assertNotNull(solrIndexConfig);
     assertNotNull(solrIndexConfig.mergedSegmentWarmerInfo);
@@ -134,20 +132,14 @@ public class SolrIndexConfigTest extends SolrTestCaseJ4 {
   }
 
   public void testToMap() throws Exception {
-    final String solrConfigFileNameWarmer = random().nextBoolean() ? solrConfigFileNameWarmerRandomMergePolicy : solrConfigFileNameWarmerRandomMergePolicyFactory;
-    final String solrConfigFileNameTMP = random().nextBoolean() ? solrConfigFileNameTieredMergePolicy : solrConfigFileNameTieredMergePolicyFactory;
+    final String solrConfigFileNameWarmer = solrConfigFileNameWarmerRandomMergePolicyFactory;
+    final String solrConfigFileNameTMP = solrConfigFileNameTieredMergePolicyFactory;
     final String solrConfigFileName = (random().nextBoolean() ? solrConfigFileNameWarmer : solrConfigFileNameTMP);
     SolrConfig solrConfig = new SolrConfig(instanceDir, solrConfigFileName, null);
     SolrIndexConfig solrIndexConfig = new SolrIndexConfig(solrConfig, null, null);
     assertNotNull(solrIndexConfig);
-    if (solrConfigFileName.equals(solrConfigFileNameTieredMergePolicyFactory) ||
-        solrConfigFileName.equals(solrConfigFileNameWarmerRandomMergePolicyFactory)) {
-      assertNotNull(solrIndexConfig.mergePolicyFactoryInfo);
-    } else {
-      assertNotNull(solrIndexConfig.mergePolicyInfo);
-    }
-    if (solrConfigFileName.equals(solrConfigFileNameWarmerRandomMergePolicy) ||
-        solrConfigFileName.equals(solrConfigFileNameWarmerRandomMergePolicyFactory)) {
+    assertNotNull(solrIndexConfig.mergePolicyFactoryInfo);
+    if (solrConfigFileName.equals(solrConfigFileNameWarmerRandomMergePolicyFactory)) {
       assertNotNull(solrIndexConfig.mergedSegmentWarmerInfo);
     } else {
       assertNull(solrIndexConfig.mergedSegmentWarmerInfo);
@@ -160,8 +152,6 @@ public class SolrIndexConfigTest extends SolrTestCaseJ4 {
     ++mSizeExpected; assertTrue(m.get("useCompoundFile") instanceof Boolean);
 
     ++mSizeExpected; assertTrue(m.get("maxBufferedDocs") instanceof Integer);
-    ++mSizeExpected; assertTrue(m.get("maxMergeDocs") instanceof Integer);
-    ++mSizeExpected; assertTrue(m.get("mergeFactor") instanceof Integer);
 
     ++mSizeExpected; assertTrue(m.get("ramBufferSizeMB") instanceof Double);
 
@@ -183,16 +173,8 @@ public class SolrIndexConfigTest extends SolrTestCaseJ4 {
     }
     
     ++mSizeExpected; assertTrue(m.get("mergeScheduler") instanceof MapSerializable);
-    if (solrConfigFileName.equals(solrConfigFileNameTieredMergePolicyFactory) ||
-        solrConfigFileName.equals(solrConfigFileNameWarmerRandomMergePolicyFactory)) {
-      assertNull(m.get("mergePolicy"));
-      ++mSizeExpected; assertTrue(m.get("mergePolicyFactory") instanceof MapSerializable);
-    } else {
-      ++mSizeExpected; assertTrue(m.get("mergePolicy") instanceof MapSerializable);
-      assertNull(m.get("mergePolicyFactory"));
-    }
-    if (solrConfigFileName.equals(solrConfigFileNameWarmerRandomMergePolicy) ||
-        solrConfigFileName.equals(solrConfigFileNameWarmerRandomMergePolicyFactory)) {
+    ++mSizeExpected; assertTrue(m.get("mergePolicyFactory") instanceof MapSerializable);
+    if (solrConfigFileName.equals(solrConfigFileNameWarmerRandomMergePolicyFactory)) {
       ++mSizeExpected; assertTrue(m.get("mergedSegmentWarmer") instanceof MapSerializable);
     } else {
       assertNull(m.get("mergedSegmentWarmer"));

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c64f9d64/solr/core/src/test/org/apache/solr/update/TestInPlaceUpdatesDistrib.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/update/TestInPlaceUpdatesDistrib.java b/solr/core/src/test/org/apache/solr/update/TestInPlaceUpdatesDistrib.java
index 44b4a4e..588ecce 100644
--- a/solr/core/src/test/org/apache/solr/update/TestInPlaceUpdatesDistrib.java
+++ b/solr/core/src/test/org/apache/solr/update/TestInPlaceUpdatesDistrib.java
@@ -87,10 +87,6 @@ public class TestInPlaceUpdatesDistrib extends AbstractFullDistribZkTestBase {
     // asserting inplace updates happen by checking the internal [docid]
     systemSetPropertySolrTestsMergePolicyFactory(NoMergePolicyFactory.class.getName());
 
-    // HACK: Don't use a RandomMergePolicy, but only use the mergePolicyFactory that we've just set
-    System.setProperty(SYSTEM_PROPERTY_SOLR_TESTS_USEMERGEPOLICYFACTORY, "true");
-    System.setProperty(SYSTEM_PROPERTY_SOLR_TESTS_USEMERGEPOLICY, "false");
-
     initCore(configString, schemaString);
     
     // sanity check that autocommits are disabled

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c64f9d64/solr/core/src/test/org/apache/solr/update/TestInPlaceUpdatesStandalone.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/update/TestInPlaceUpdatesStandalone.java b/solr/core/src/test/org/apache/solr/update/TestInPlaceUpdatesStandalone.java
index aa075a4..88b1b6d 100644
--- a/solr/core/src/test/org/apache/solr/update/TestInPlaceUpdatesStandalone.java
+++ b/solr/core/src/test/org/apache/solr/update/TestInPlaceUpdatesStandalone.java
@@ -75,10 +75,6 @@ public class TestInPlaceUpdatesStandalone extends SolrTestCaseJ4 {
     // asserting inplace updates happen by checking the internal [docid]
     systemSetPropertySolrTestsMergePolicyFactory(NoMergePolicyFactory.class.getName());
 
-    // HACK: Don't use a RandomMergePolicy, but only use the mergePolicyFactory that we've just set
-    System.setProperty(SYSTEM_PROPERTY_SOLR_TESTS_USEMERGEPOLICYFACTORY, "true");
-    System.setProperty(SYSTEM_PROPERTY_SOLR_TESTS_USEMERGEPOLICY, "false");
-
     initCore("solrconfig-tlog.xml", "schema-inplace-updates.xml");
 
     // sanity check that autocommits are disabled

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c64f9d64/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java
----------------------------------------------------------------------
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 419f94f..e9eefb0 100644
--- a/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java
+++ b/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java
@@ -171,16 +171,8 @@ public abstract class SolrTestCaseJ4 extends LuceneTestCase {
   public static final String DEFAULT_TEST_CORENAME = DEFAULT_TEST_COLLECTION_NAME;
   protected static final String CORE_PROPERTIES_FILENAME = "core.properties";
 
-  // keep solr.tests.mergePolicyFactory use i.e. do not remove with SOLR-8668
   public static final String SYSTEM_PROPERTY_SOLR_TESTS_MERGEPOLICYFACTORY = "solr.tests.mergePolicyFactory";
-  @Deprecated // remove solr.tests.mergePolicy use with SOLR-8668
-  public static final String SYSTEM_PROPERTY_SOLR_TESTS_MERGEPOLICY = "solr.tests.mergePolicy";
 
-  @Deprecated // remove solr.tests.useMergePolicyFactory with SOLR-8668
-  public static final String SYSTEM_PROPERTY_SOLR_TESTS_USEMERGEPOLICYFACTORY = "solr.tests.useMergePolicyFactory";
-  @Deprecated // remove solr.tests.useMergePolicy use with SOLR-8668
-  public static final String SYSTEM_PROPERTY_SOLR_TESTS_USEMERGEPOLICY = "solr.tests.useMergePolicy";
-  
   /**
    * The system property {@code "solr.tests.preferPointFields"} can be used to make tests use PointFields when possible. 
    * PointFields will only be used if the schema used by the tests uses "${solr.tests.TYPEClass}" when defining fields. 
@@ -2482,33 +2474,6 @@ public abstract class SolrTestCaseJ4 extends LuceneTestCase {
     waitForWarming(h.getCore());
   }
 
-  @BeforeClass
-  public static void chooseMPForMP() throws Exception {
-    if (random().nextBoolean()) {
-      System.setProperty(SYSTEM_PROPERTY_SOLR_TESTS_USEMERGEPOLICYFACTORY, "true");
-      System.setProperty(SYSTEM_PROPERTY_SOLR_TESTS_USEMERGEPOLICY, "false");
-    } else {
-      System.setProperty(SYSTEM_PROPERTY_SOLR_TESTS_USEMERGEPOLICYFACTORY, "false");
-      System.setProperty(SYSTEM_PROPERTY_SOLR_TESTS_USEMERGEPOLICY, "true");
-    }
-  }
-
-  @AfterClass
-  public static void unchooseMPForMP() {
-    System.clearProperty(SYSTEM_PROPERTY_SOLR_TESTS_USEMERGEPOLICYFACTORY);
-    System.clearProperty(SYSTEM_PROPERTY_SOLR_TESTS_USEMERGEPOLICY);
-  }
-
-  @Deprecated // remove with SOLR-8668
-  protected static void systemSetPropertySolrTestsMergePolicy(String value) {
-    System.setProperty(SYSTEM_PROPERTY_SOLR_TESTS_MERGEPOLICY, value);
-  }
-
-  @Deprecated // remove with SOLR-8668
-  protected static void systemClearPropertySolrTestsMergePolicy() {
-    System.clearProperty(SYSTEM_PROPERTY_SOLR_TESTS_MERGEPOLICY);
-  }
-
   protected static void systemSetPropertySolrTestsMergePolicyFactory(String value) {
     System.setProperty(SYSTEM_PROPERTY_SOLR_TESTS_MERGEPOLICYFACTORY, value);
   }