You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by cp...@apache.org on 2016/04/27 19:11:12 UTC

[4/4] lucene-solr:jira/solr-9045: SOLR-9045: configurable RecoveryStrategy support

SOLR-9045: configurable RecoveryStrategy support

objectives:
 * To allow users to change RecoveryStrategy settings such as maxRetries and startingRecoveryDelay.
 * To support configuration of a custom recovery strategy.

patch summary:
 * RecoveryStrategy turned to DefaultRecoveryStrategy extending abstract RecoveryStrategy base class
 * DefaultRecoveryStrategy hard-coded settings exposed via getters/setters
 * DefaultRecoveryStrategyFactory extends abstract RecoveryStrategyFactory
 * solrconfig.xml now supports an optional <recoveryStrategyFactory class="..."> element
   (absence of the new element preserves existing behaviour)
 * CustomRecoveryStrategyFactoryTest using solrconfig-customrecoverystrategyfactory.xml

illustrative solrconfig.xml snippet:
  <recoveryStrategyFactory class="MyCustomRecoveryStrategyFactory">
    <int name="maxRetries">250</int> <!-- DefaultRecoveryStrategy's default is 500. -->
    <str name="settingUsedByCustomBehaviour"></str>
  </recoveryStrategyFactory>


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

Branch: refs/heads/jira/solr-9045
Commit: e3d5a1925bf3cd3feaff6e7694ffa05da6ce64aa
Parents: 928763b
Author: Christine Poerschke <cp...@apache.org>
Authored: Tue Apr 19 17:13:47 2016 +0100
Committer: Christine Poerschke <cp...@apache.org>
Committed: Wed Apr 27 18:10:03 2016 +0100

----------------------------------------------------------------------
 .../solr/cloud/DefaultRecoveryStrategy.java     | 41 +++++++--
 .../cloud/DefaultRecoveryStrategyFactory.java   | 33 ++++++++
 .../org/apache/solr/cloud/RecoveryStrategy.java |  1 +
 .../solr/cloud/RecoveryStrategyFactory.java     | 47 +++++++++++
 .../java/org/apache/solr/core/SolrConfig.java   |  2 +
 .../src/java/org/apache/solr/core/SolrCore.java | 21 ++++-
 .../solr/update/DefaultSolrCoreState.java       | 17 +++-
 .../org/apache/solr/update/SolrCoreState.java   |  6 ++
 ...solrconfig-customrecoverystrategyfactory.xml | 28 ++++++
 .../core/CustomRecoveryStrategyFactoryTest.java | 89 ++++++++++++++++++++
 10 files changed, 277 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e3d5a192/solr/core/src/java/org/apache/solr/cloud/DefaultRecoveryStrategy.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/DefaultRecoveryStrategy.java b/solr/core/src/java/org/apache/solr/cloud/DefaultRecoveryStrategy.java
index 60d2931..fa77d25 100644
--- a/solr/core/src/java/org/apache/solr/cloud/DefaultRecoveryStrategy.java
+++ b/solr/core/src/java/org/apache/solr/cloud/DefaultRecoveryStrategy.java
@@ -68,9 +68,9 @@ public class DefaultRecoveryStrategy extends RecoveryStrategy {
 
   private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
-  private static final int WAIT_FOR_UPDATES_WITH_STALE_STATE_PAUSE = Integer.getInteger("solr.cloud.wait-for-updates-with-stale-state-pause", 7000);
-  private static final int MAX_RETRIES = 500;
-  private static final int STARTING_RECOVERY_DELAY = 5000;
+  private int waitForUpdatesWithStaleStatePauseMilliSeconds = Integer.getInteger("solr.cloud.wait-for-updates-with-stale-state-pause", 7000);
+  private int maxRetries = 500;
+  private int startingRecoveryDelayMilliSeconds = 5000;
 
   private volatile boolean close = false;
 
@@ -97,6 +97,35 @@ public class DefaultRecoveryStrategy extends RecoveryStrategy {
     coreZkNodeName = cd.getCloudDescriptor().getCoreNodeName();
   }
 
+  public int getWaitForUpdatesWithStaleStatePauseMilliSeconds() {
+    return waitForUpdatesWithStaleStatePauseMilliSeconds;
+  }
+
+  public void setWaitForUpdatesWithStaleStatePauseMilliSeconds(int waitForUpdatesWithStaleStatePauseMilliSeconds) {
+    this.waitForUpdatesWithStaleStatePauseMilliSeconds = waitForUpdatesWithStaleStatePauseMilliSeconds;
+  }
+
+  public int getMaxRetries() {
+    return maxRetries;
+  }
+
+  public void setMaxRetries(int maxRetries) {
+    this.maxRetries = maxRetries;
+  }
+
+  public int getStartingRecoveryDelayMilliSeconds() {
+    return startingRecoveryDelayMilliSeconds;
+  }
+
+  public void setStartingRecoveryDelayMilliSeconds(int startingRecoveryDelayMilliSeconds) {
+    this.startingRecoveryDelayMilliSeconds = startingRecoveryDelayMilliSeconds;
+  }
+
+  @Override
+  public boolean getRecoveringAfterStartup() {
+    return recoveringAfterStartup;
+  }
+
   @Override
   public void setRecoveringAfterStartup(boolean recoveringAfterStartup) {
     this.recoveringAfterStartup = recoveringAfterStartup;
@@ -357,7 +386,7 @@ public class DefaultRecoveryStrategy extends RecoveryStrategy {
         // are sure to have finished (see SOLR-7141 for
         // discussion around current value)
         try {
-          Thread.sleep(WAIT_FOR_UPDATES_WITH_STALE_STATE_PAUSE);
+          Thread.sleep(waitForUpdatesWithStaleStatePauseMilliSeconds);
         } catch (InterruptedException e) {
           Thread.currentThread().interrupt();
         }
@@ -476,7 +505,7 @@ public class DefaultRecoveryStrategy extends RecoveryStrategy {
           LOG.error("Recovery failed - trying again... (" + retries + ")");
           
           retries++;
-          if (retries >= MAX_RETRIES) {
+          if (retries >= maxRetries) {
             SolrException.log(LOG, "Recovery failed - max retries exceeded (" + retries + ").");
             try {
               recoveryFailed(core, zkController, baseUrl, coreZkNodeName, core.getCoreDescriptor());
@@ -501,7 +530,7 @@ public class DefaultRecoveryStrategy extends RecoveryStrategy {
               LOG.info("RecoveryStrategy has been closed");
               break; // check if someone closed us
             }
-            Thread.sleep(STARTING_RECOVERY_DELAY);
+            Thread.sleep(startingRecoveryDelayMilliSeconds);
           }
         } catch (InterruptedException e) {
           Thread.currentThread().interrupt();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e3d5a192/solr/core/src/java/org/apache/solr/cloud/DefaultRecoveryStrategyFactory.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/DefaultRecoveryStrategyFactory.java b/solr/core/src/java/org/apache/solr/cloud/DefaultRecoveryStrategyFactory.java
new file mode 100644
index 0000000..af15c1c
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/cloud/DefaultRecoveryStrategyFactory.java
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.solr.cloud;
+
+import org.apache.solr.core.CoreContainer;
+import org.apache.solr.core.CoreDescriptor;
+
+/**
+ * A factory for creating a {@link DefaultRecoveryStrategy}.
+ */
+public class DefaultRecoveryStrategyFactory extends RecoveryStrategyFactory {
+
+  @Override
+  public RecoveryStrategy newRecoveryStrategy(CoreContainer cc, CoreDescriptor cd,
+      RecoveryStrategy.RecoveryListener recoveryListener) {
+    return new DefaultRecoveryStrategy(cc, cd, recoveryListener);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e3d5a192/solr/core/src/java/org/apache/solr/cloud/RecoveryStrategy.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/RecoveryStrategy.java b/solr/core/src/java/org/apache/solr/cloud/RecoveryStrategy.java
index d855a6d..5b18db2 100644
--- a/solr/core/src/java/org/apache/solr/cloud/RecoveryStrategy.java
+++ b/solr/core/src/java/org/apache/solr/cloud/RecoveryStrategy.java
@@ -25,6 +25,7 @@ public abstract class RecoveryStrategy extends Thread implements Closeable {
     public void failed();
   }
 
+  public abstract boolean getRecoveringAfterStartup();
   public abstract void setRecoveringAfterStartup(boolean recoveringAfterStartup);
 
   @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e3d5a192/solr/core/src/java/org/apache/solr/cloud/RecoveryStrategyFactory.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/RecoveryStrategyFactory.java b/solr/core/src/java/org/apache/solr/cloud/RecoveryStrategyFactory.java
new file mode 100644
index 0000000..3748255
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/cloud/RecoveryStrategyFactory.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.solr.cloud;
+
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.core.CoreContainer;
+import org.apache.solr.core.CoreDescriptor;
+import org.apache.solr.util.SolrPluginUtils;
+import org.apache.solr.util.plugin.NamedListInitializedPlugin;
+
+/**
+ * A factory for creating a {@link RecoveryStrategy}.
+ */
+public abstract class RecoveryStrategyFactory implements NamedListInitializedPlugin {
+
+  private NamedList args;
+
+  @Override
+  public void init(NamedList args) {
+    this.args = args;
+  }
+
+  public RecoveryStrategy create(CoreContainer cc, CoreDescriptor cd,
+      RecoveryStrategy.RecoveryListener recoveryListener) {
+    final RecoveryStrategy recoveryStrategy = newRecoveryStrategy(cc, cd, recoveryListener);
+    SolrPluginUtils.invokeSetters(recoveryStrategy, args);
+    return recoveryStrategy;
+  }
+
+  public abstract RecoveryStrategy newRecoveryStrategy(CoreContainer cc, CoreDescriptor cd,
+      RecoveryStrategy.RecoveryListener recoveryListener);
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e3d5a192/solr/core/src/java/org/apache/solr/core/SolrConfig.java
----------------------------------------------------------------------
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 0b4bac3..adc3e0b 100644
--- a/solr/core/src/java/org/apache/solr/core/SolrConfig.java
+++ b/solr/core/src/java/org/apache/solr/core/SolrConfig.java
@@ -36,6 +36,7 @@ import com.google.common.collect.ImmutableList;
 import org.apache.lucene.index.IndexDeletionPolicy;
 import org.apache.lucene.search.BooleanQuery;
 import org.apache.lucene.util.Version;
+import org.apache.solr.cloud.RecoveryStrategyFactory;
 import org.apache.solr.cloud.ZkSolrResourceLoader;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;
@@ -330,6 +331,7 @@ public class SolrConfig extends Config implements MapSerializable {
       .add(new SolrPluginInfo(SolrEventListener.class, "//listener", REQUIRE_CLASS, MULTI_OK, REQUIRE_NAME_IN_OVERLAY))
 
       .add(new SolrPluginInfo(DirectoryFactory.class, "directoryFactory", REQUIRE_CLASS))
+      .add(new SolrPluginInfo(RecoveryStrategyFactory.class, "recoveryStrategyFactory", REQUIRE_CLASS))
       .add(new SolrPluginInfo(IndexDeletionPolicy.class, "indexConfig/deletionPolicy", REQUIRE_CLASS))
       .add(new SolrPluginInfo(CodecFactory.class, "codecFactory", REQUIRE_CLASS))
       .add(new SolrPluginInfo(IndexReaderFactory.class, "indexReaderFactory", REQUIRE_CLASS))

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e3d5a192/solr/core/src/java/org/apache/solr/core/SolrCore.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/SolrCore.java b/solr/core/src/java/org/apache/solr/core/SolrCore.java
index b94b3d8..462ec9c 100644
--- a/solr/core/src/java/org/apache/solr/core/SolrCore.java
+++ b/solr/core/src/java/org/apache/solr/core/SolrCore.java
@@ -52,6 +52,8 @@ import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.store.LockObtainFailedException;
 import org.apache.solr.client.solrj.impl.BinaryResponseParser;
 import org.apache.solr.cloud.CloudDescriptor;
+import org.apache.solr.cloud.DefaultRecoveryStrategyFactory;
+import org.apache.solr.cloud.RecoveryStrategyFactory;
 import org.apache.solr.cloud.ZkSolrResourceLoader;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.cloud.ClusterState;
@@ -158,6 +160,7 @@ public final class SolrCore implements SolrInfoMBean, Closeable {
   private final Map<String, SolrInfoMBean> infoRegistry;
   private final IndexDeletionPolicyWrapper solrDelPolicy;
   private final DirectoryFactory directoryFactory;
+  private final RecoveryStrategyFactory recoveryStrategyFactory;
   private IndexReaderFactory indexReaderFactory;
   private final Codec codec;
   private final MemClassLoader memClassLoader;
@@ -493,6 +496,20 @@ public final class SolrCore implements SolrInfoMBean, Closeable {
     return dirFactory;
   }
 
+  private RecoveryStrategyFactory initRecoveryStrategyFactory() {
+    final PluginInfo info = solrConfig.getPluginInfo(RecoveryStrategyFactory.class.getName());
+    final RecoveryStrategyFactory rsFactory;
+    if (info != null) {
+      log.info(info.className);
+      rsFactory = getResourceLoader().newInstance(info.className, RecoveryStrategyFactory.class);
+      rsFactory.init(info.initArgs);
+    } else {
+      log.info("solr.RecoveryStrategyFactory");
+      rsFactory = new DefaultRecoveryStrategyFactory();
+    }
+    return rsFactory;
+  }
+
   private void initIndexReaderFactory() {
     IndexReaderFactory indexReaderFactory;
     PluginInfo info = solrConfig.getPluginInfo(IndexReaderFactory.class.getName());
@@ -681,10 +698,12 @@ public final class SolrCore implements SolrInfoMBean, Closeable {
 
     if (updateHandler == null) {
       directoryFactory = initDirectoryFactory();
-      solrCoreState = new DefaultSolrCoreState(directoryFactory);
+      recoveryStrategyFactory = initRecoveryStrategyFactory();
+      solrCoreState = new DefaultSolrCoreState(directoryFactory, recoveryStrategyFactory);
     } else {
       solrCoreState = updateHandler.getSolrCoreState();
       directoryFactory = solrCoreState.getDirectoryFactory();
+      recoveryStrategyFactory = solrCoreState.getRecoveryStrategyFactory();
       isReloaded = true;
     }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e3d5a192/solr/core/src/java/org/apache/solr/update/DefaultSolrCoreState.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/update/DefaultSolrCoreState.java b/solr/core/src/java/org/apache/solr/update/DefaultSolrCoreState.java
index 8eab83f..2695cd7 100644
--- a/solr/core/src/java/org/apache/solr/update/DefaultSolrCoreState.java
+++ b/solr/core/src/java/org/apache/solr/update/DefaultSolrCoreState.java
@@ -32,7 +32,9 @@ import org.apache.lucene.index.MergePolicy;
 import org.apache.lucene.index.SortingMergePolicy;
 import org.apache.lucene.search.Sort;
 import org.apache.solr.cloud.ActionThrottle;
+import org.apache.solr.cloud.DefaultRecoveryStrategyFactory;
 import org.apache.solr.cloud.RecoveryStrategy;
+import org.apache.solr.cloud.RecoveryStrategyFactory;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;
 import org.apache.solr.core.CoreContainer;
@@ -63,6 +65,7 @@ public final class DefaultSolrCoreState extends SolrCoreState implements Recover
 
   private SolrIndexWriter indexWriter = null;
   private DirectoryFactory directoryFactory;
+  private final RecoveryStrategyFactory recoveryStrategyFactory;
 
   private volatile RecoveryStrategy recoveryStrat;
 
@@ -76,8 +79,15 @@ public final class DefaultSolrCoreState extends SolrCoreState implements Recover
   
   protected final ReentrantLock commitLock = new ReentrantLock();
 
+  @Deprecated
   public DefaultSolrCoreState(DirectoryFactory directoryFactory) {
+    this(directoryFactory, new DefaultRecoveryStrategyFactory());
+  }
+
+  public DefaultSolrCoreState(DirectoryFactory directoryFactory,
+      RecoveryStrategyFactory recoveryStrategyFactory) {
     this.directoryFactory = directoryFactory;
+    this.recoveryStrategyFactory = recoveryStrategyFactory;
   }
   
   private void closeIndexWriter(IndexWriterCloser closer) {
@@ -263,6 +273,11 @@ public final class DefaultSolrCoreState extends SolrCoreState implements Recover
   }
 
   @Override
+  public RecoveryStrategyFactory getRecoveryStrategyFactory() {
+    return recoveryStrategyFactory;
+  }
+
+  @Override
   public void doRecovery(CoreContainer cc, CoreDescriptor cd) {
     
     Thread thread = new Thread() {
@@ -310,7 +325,7 @@ public final class DefaultSolrCoreState extends SolrCoreState implements Recover
               recoveryThrottle.minimumWaitBetweenActions();
               recoveryThrottle.markAttemptingAction();
               
-              recoveryStrat = new RecoveryStrategy(cc, cd, DefaultSolrCoreState.this);
+              recoveryStrat = recoveryStrategyFactory.create(cc, cd, DefaultSolrCoreState.this);
               recoveryStrat.setRecoveringAfterStartup(recoveringAfterStartup);
               Future<?> future = cc.getUpdateShardHandler().getRecoveryExecutor().submit(recoveryStrat);
               try {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e3d5a192/solr/core/src/java/org/apache/solr/update/SolrCoreState.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/update/SolrCoreState.java b/solr/core/src/java/org/apache/solr/update/SolrCoreState.java
index 89e286a..25ba94a 100644
--- a/solr/core/src/java/org/apache/solr/update/SolrCoreState.java
+++ b/solr/core/src/java/org/apache/solr/update/SolrCoreState.java
@@ -23,6 +23,7 @@ import java.util.concurrent.locks.Lock;
 import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.search.Sort;
 import org.apache.solr.cloud.ActionThrottle;
+import org.apache.solr.cloud.RecoveryStrategyFactory;
 import org.apache.solr.core.CoreContainer;
 import org.apache.solr.core.CoreDescriptor;
 import org.apache.solr.core.DirectoryFactory;
@@ -138,6 +139,11 @@ public abstract class SolrCoreState {
    */
   public abstract DirectoryFactory getDirectoryFactory();
 
+  /**
+   * @return the {@link RecoveryStrategyFactory} that should be used.
+   */
+  public abstract RecoveryStrategyFactory getRecoveryStrategyFactory();
+
 
   public interface IndexWriterCloser {
     void closeWriter(IndexWriter writer) throws IOException;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e3d5a192/solr/core/src/test-files/solr/collection1/conf/solrconfig-customrecoverystrategyfactory.xml
----------------------------------------------------------------------
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-customrecoverystrategyfactory.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-customrecoverystrategyfactory.xml
new file mode 100644
index 0000000..1d0a93c
--- /dev/null
+++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-customrecoverystrategyfactory.xml
@@ -0,0 +1,28 @@
+<?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>
+  <xi:include href="solrconfig.snippet.randomindexconfig.xml" xmlns:xi="http://www.w3.org/2001/XInclude"/>
+  <requestHandler name="standard" class="solr.StandardRequestHandler"></requestHandler>
+  <recoveryStrategyFactory class="org.apache.solr.core.CustomRecoveryStrategyFactoryTest$CustomRecoveryStrategyFactory">
+    <int name="customParameter">42</int>
+  </recoveryStrategyFactory>
+  <schemaFactory class="ClassicIndexSchemaFactory"/>
+</config>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e3d5a192/solr/core/src/test/org/apache/solr/core/CustomRecoveryStrategyFactoryTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/core/CustomRecoveryStrategyFactoryTest.java b/solr/core/src/test/org/apache/solr/core/CustomRecoveryStrategyFactoryTest.java
new file mode 100644
index 0000000..01146a4
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/core/CustomRecoveryStrategyFactoryTest.java
@@ -0,0 +1,89 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.solr.core;
+
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.cloud.RecoveryStrategy;
+import org.apache.solr.cloud.RecoveryStrategyFactory;
+import org.junit.BeforeClass;
+
+/**
+ * test that configs can override the RecoveryStrategyFactory
+ */
+public class CustomRecoveryStrategyFactoryTest extends SolrTestCaseJ4 {
+
+  @BeforeClass
+  public static void beforeClass() throws Exception {
+    initCore("solrconfig-customrecoverystrategyfactory.xml", "schema.xml");
+  }
+
+  public void testFactory() throws Exception {
+    final RecoveryStrategyFactory recoveryStrategyFactory =
+        h.getCore().getSolrCoreState().getRecoveryStrategyFactory();
+    assertNotNull("recoveryStrategyFactory is null", recoveryStrategyFactory);
+    assertEquals("recoveryStrategyFactory is wrong class (name)",
+                 CustomRecoveryStrategyFactoryTest.CustomRecoveryStrategyFactory.class.getName(),
+                 recoveryStrategyFactory.getClass().getName());
+    assertTrue("recoveryStrategyFactory is wrong class (instanceof)",
+        recoveryStrategyFactory instanceof CustomRecoveryStrategyFactory);
+    final CustomRecoveryStrategyFactory customRecoveryStrategyFactory =
+        (CustomRecoveryStrategyFactory)recoveryStrategyFactory;
+}
+
+  public void testCreate() throws Exception {
+    final RecoveryStrategyFactory recoveryStrategyFactory =
+        h.getCore().getSolrCoreState().getRecoveryStrategyFactory();
+    assertNotNull("recoveryStrategyFactory is null", recoveryStrategyFactory);
+
+    final RecoveryStrategy recoveryStrategy =
+        recoveryStrategyFactory.create(null, null, null);
+
+    assertEquals("recoveryStrategy is wrong class (name)",
+                 CustomRecoveryStrategyFactoryTest.CustomRecoveryStrategy.class.getName(),
+                 recoveryStrategy.getClass().getName());
+    assertTrue("recoveryStrategy is wrong class (instanceof)",
+        recoveryStrategy instanceof CustomRecoveryStrategy);
+
+    final CustomRecoveryStrategy customRecoveryStrategy =
+        (CustomRecoveryStrategy)recoveryStrategy;
+    assertEquals(42, customRecoveryStrategy.getCustomParameter());
+}
+
+  static public class CustomRecoveryStrategy extends RecoveryStrategy {
+    public CustomRecoveryStrategy(CoreContainer cc, CoreDescriptor cd,
+        RecoveryListener recoveryListener) {
+    }
+    private int customParameter = random().nextInt();
+    public int getCustomParameter() { return customParameter; }
+    public void setCustomParameter(int customParameter) { this.customParameter = customParameter; }
+    @Override
+    public boolean getRecoveringAfterStartup() { return false; }
+    @Override
+    public void setRecoveringAfterStartup(boolean recoveringAfterStartup) {}
+    @Override
+    public void close() {}
+  }
+
+  static public class CustomRecoveryStrategyFactory extends RecoveryStrategyFactory {
+    @Override
+    public RecoveryStrategy newRecoveryStrategy(CoreContainer cc, CoreDescriptor cd,
+        RecoveryStrategy.RecoveryListener recoveryListener) {
+      return new CustomRecoveryStrategy(cc, cd, recoveryListener);
+    }
+  }
+
+}