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 2015/09/10 12:48:19 UTC

svn commit: r1702213 [1/2] - in /lucene/dev/branches/branch_5x: ./ solr/ solr/core/ solr/core/src/java/org/apache/solr/cloud/ solr/core/src/java/org/apache/solr/update/processor/ solr/core/src/test/org/apache/solr/cloud/

Author: shalin
Date: Thu Sep 10 10:48:18 2015
New Revision: 1702213

URL: http://svn.apache.org/r1702213
Log:
SOLR-7819: ZK connection loss or session timeout do not stall indexing threads anymore and LIR activity is moved to a background thread

Added:
    lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/cloud/TestLeaderInitiatedRecoveryThread.java
      - copied unchanged from r1702067, lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/TestLeaderInitiatedRecoveryThread.java
Modified:
    lucene/dev/branches/branch_5x/   (props changed)
    lucene/dev/branches/branch_5x/solr/   (props changed)
    lucene/dev/branches/branch_5x/solr/CHANGES.txt   (contents, props changed)
    lucene/dev/branches/branch_5x/solr/core/   (props changed)
    lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/cloud/ElectionContext.java
    lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/cloud/LeaderInitiatedRecoveryThread.java
    lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/cloud/ZkController.java
    lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java
    lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/cloud/HttpPartitionTest.java
    lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/cloud/ZkControllerTest.java

Modified: lucene/dev/branches/branch_5x/solr/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/CHANGES.txt?rev=1702213&r1=1702212&r2=1702213&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/CHANGES.txt (original)
+++ lucene/dev/branches/branch_5x/solr/CHANGES.txt Thu Sep 10 10:48:18 2015
@@ -94,6 +94,10 @@ Bug Fixes
 * SOLR-8001: Fixed bugs in field(foo,min) and field(foo,max) when some docs have no values
   (David Smiley, hossman)
 
+* SOLR-7819: ZK connection loss or session timeout do not stall indexing threads anymore. All activity
+  related to leader initiated recovery is performed by a dedicated LIR thread in the background.
+  (Ramkumar Aiyengar, shalin)
+
 
 Optimizations
 ----------------------

Modified: lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/cloud/ElectionContext.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/cloud/ElectionContext.java?rev=1702213&r1=1702212&r2=1702213&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/cloud/ElectionContext.java (original)
+++ lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/cloud/ElectionContext.java Thu Sep 10 10:48:18 2015
@@ -4,7 +4,6 @@ import java.io.Closeable;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
-import java.util.concurrent.ExecutorService;
 import java.util.concurrent.TimeUnit;
 
 import org.apache.hadoop.fs.Path;
@@ -468,20 +467,9 @@ final class ShardLeaderElectionContext e
                 }
               }
               
-              LeaderInitiatedRecoveryThread lirThread = 
-                  new LeaderInitiatedRecoveryThread(zkController,
-                                                    cc,
-                                                    collection,
-                                                    shardId,
-                                                    coreNodeProps,
-                                                    120,
-                                                    coreNodeName);
-              zkController.ensureReplicaInLeaderInitiatedRecovery(
+              zkController.ensureReplicaInLeaderInitiatedRecovery(cc,
                   collection, shardId, coreNodeProps, coreNodeName,
-                  false /* forcePublishState */, true /* retryOnConnLoss */);
-
-              ExecutorService executor = cc.getUpdateShardHandler().getUpdateExecutor();
-              executor.execute(lirThread);
+                  false /* forcePublishState */);
             }              
           }
         }

Modified: lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/cloud/LeaderInitiatedRecoveryThread.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/cloud/LeaderInitiatedRecoveryThread.java?rev=1702213&r1=1702212&r2=1702213&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/cloud/LeaderInitiatedRecoveryThread.java (original)
+++ lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/cloud/LeaderInitiatedRecoveryThread.java Thu Sep 10 10:48:18 2015
@@ -8,9 +8,12 @@ import org.apache.solr.common.SolrExcept
 import org.apache.solr.common.SolrException.ErrorCode;
 import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.cloud.ZkCoreNodeProps;
+import org.apache.solr.common.cloud.ZkNodeProps;
 import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.params.CoreAdminParams.CoreAdminAction;
+import org.apache.solr.common.util.Utils;
 import org.apache.solr.core.CoreContainer;
+import org.apache.zookeeper.KeeperException;
 import org.apache.solr.util.RTimer;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -75,24 +78,108 @@ public class LeaderInitiatedRecoveryThre
   
   public void run() {
     RTimer timer = new RTimer();
-    try {
-      sendRecoveryCommandWithRetry();
-    } catch (Exception exc) {
-      log.error(getName()+" failed due to: "+exc, exc);
-      if (exc instanceof SolrException) {
-        throw (SolrException)exc;
-      } else {
-        throw new SolrException(ErrorCode.SERVER_ERROR, exc);
+
+    String replicaCoreName = nodeProps.getCoreName();
+    String replicaCoreNodeName = ((Replica) nodeProps.getNodeProps()).getName();
+    String replicaNodeName = nodeProps.getNodeName();
+    final String replicaUrl = nodeProps.getCoreUrl();
+
+    if (!zkController.isReplicaInRecoveryHandling(replicaUrl)) {
+      throw new SolrException(ErrorCode.INVALID_STATE, "Replica: " + replicaUrl + " should have been marked under leader initiated recovery in ZkController but wasn't.");
+    }
+
+    boolean sendRecoveryCommand = publishDownState(replicaCoreName, replicaCoreNodeName, replicaNodeName, replicaUrl, false);
+
+    if (sendRecoveryCommand)  {
+      try {
+        sendRecoveryCommandWithRetry();
+      } catch (Exception exc) {
+        log.error(getName()+" failed due to: "+exc, exc);
+        if (exc instanceof SolrException) {
+          throw (SolrException)exc;
+        } else {
+          throw new SolrException(ErrorCode.SERVER_ERROR, exc);
+        }
+      } finally {
+        zkController.removeReplicaFromLeaderInitiatedRecoveryHandling(replicaUrl);
       }
+    } else  {
+      // replica is no longer in recovery on this node (may be handled on another node)
+      zkController.removeReplicaFromLeaderInitiatedRecoveryHandling(replicaUrl);
     }
     log.info("{} completed successfully after running for {}ms", getName(), timer.getTime());
   }
-  
+
+  public boolean publishDownState(String replicaCoreName, String replicaCoreNodeName, String replicaNodeName, String replicaUrl, boolean forcePublishState) {
+    boolean sendRecoveryCommand = true;
+    boolean publishDownState = false;
+
+    if (zkController.getZkStateReader().getClusterState().liveNodesContain(replicaNodeName)) {
+      try {
+        // create a znode that requires the replica needs to "ack" to verify it knows it was out-of-sync
+        updateLIRState(replicaCoreNodeName);
+
+        log.info("Put replica core={} coreNodeName={} on " +
+            replicaNodeName + " into leader-initiated recovery.", replicaCoreName, replicaCoreNodeName);
+        publishDownState = true;
+      } catch (Exception e) {
+        Throwable setLirZnodeFailedCause = SolrException.getRootCause(e);
+        log.error("Leader failed to set replica " +
+            nodeProps.getCoreUrl() + " state to DOWN due to: " + setLirZnodeFailedCause, setLirZnodeFailedCause);
+        if (setLirZnodeFailedCause instanceof KeeperException.SessionExpiredException
+            || setLirZnodeFailedCause instanceof KeeperException.ConnectionLossException
+            || setLirZnodeFailedCause instanceof ZkController.NotLeaderException) {
+          // our session is expired, which means our state is suspect, so don't go
+          // putting other replicas in recovery (see SOLR-6511)
+          sendRecoveryCommand = false;
+          forcePublishState = false; // no need to force publish any state in this case
+        } // else will go ahead and try to send the recovery command once after this error
+      }
+    } else  {
+      log.info("Node " + replicaNodeName +
+              " is not live, so skipping leader-initiated recovery for replica: core={} coreNodeName={}",
+          replicaCoreName, replicaCoreNodeName);
+      // publishDownState will be false to avoid publishing the "down" state too many times
+      // as many errors can occur together and will each call into this method (SOLR-6189)
+      forcePublishState = false; // no need to force publish the state because replica is not live
+      sendRecoveryCommand = false; // no need to send recovery messages as well
+    }
+
+    try {
+      if (publishDownState || forcePublishState) {
+        ZkNodeProps m = new ZkNodeProps(Overseer.QUEUE_OPERATION, "state",
+            ZkStateReader.STATE_PROP, Replica.State.DOWN.toString(),
+            ZkStateReader.BASE_URL_PROP, nodeProps.getBaseUrl(),
+            ZkStateReader.CORE_NAME_PROP, nodeProps.getCoreName(),
+            ZkStateReader.NODE_NAME_PROP, nodeProps.getNodeName(),
+            ZkStateReader.SHARD_ID_PROP, shardId,
+            ZkStateReader.COLLECTION_PROP, collection);
+        log.warn("Leader is publishing core={} coreNodeName ={} state={} on behalf of un-reachable replica {}",
+            replicaCoreName, replicaCoreNodeName, Replica.State.DOWN.toString(), replicaUrl);
+        zkController.getOverseerJobQueue().offer(Utils.toJSON(m));
+      }
+    } catch (Exception e) {
+      log.error("Could not publish 'down' state for replicaUrl: {}", replicaUrl, e);
+    }
+
+    return sendRecoveryCommand;
+  }
+
+  /*
+  protected scope for testing purposes
+   */
+  protected void updateLIRState(String replicaCoreNodeName) {
+    zkController.updateLeaderInitiatedRecoveryState(collection,
+        shardId,
+        replicaCoreNodeName, Replica.State.DOWN, leaderCoreNodeName, true);
+  }
+
   protected void sendRecoveryCommandWithRetry() throws Exception {    
     int tries = 0;
     long waitBetweenTriesMs = 5000L;
     boolean continueTrying = true;
-        
+
+    String replicaCoreName = nodeProps.getCoreName();
     String recoveryUrl = nodeProps.getBaseUrl();
     String replicaNodeName = nodeProps.getNodeName();
     String coreNeedingRecovery = nodeProps.getCoreName();
@@ -224,11 +311,8 @@ public class LeaderInitiatedRecoveryThre
                         // OK, so the replica thinks it is active, but it never ack'd the leader initiated recovery
                         // so its state cannot be trusted and it needs to be told to recover again ... and we keep looping here
                         log.warn("Replica core={} coreNodeName={} set to active but the leader thinks it should be in recovery;"
-                            + " forcing it back to down state to re-run the leader-initiated recovery process; props: "+replicaProps.get(0), coreNeedingRecovery, replicaCoreNodeName);
-                        zkController.ensureReplicaInLeaderInitiatedRecovery(
-                            collection, shardId, nodeProps, leaderCoreNodeName,
-                            true /* forcePublishState */, true /* retryOnConnLoss */
-                        );
+                            + " forcing it back to down state to re-run the leader-initiated recovery process; props: " + replicaProps.get(0), coreNeedingRecovery, replicaCoreNodeName);
+                        publishDownState(replicaCoreName, replicaCoreNodeName, replicaNodeName, replicaUrl, true);
                       }
                     }
                     break;

Modified: lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/cloud/ZkController.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/cloud/ZkController.java?rev=1702213&r1=1702212&r2=1702213&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/cloud/ZkController.java (original)
+++ lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/cloud/ZkController.java Thu Sep 10 10:48:18 2015
@@ -97,24 +97,24 @@ import org.apache.zookeeper.Watcher;
 import org.apache.zookeeper.data.Stat;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import org.slf4j.MDC;
 
 import static org.apache.solr.common.cloud.ZkStateReader.BASE_URL_PROP;
 import static org.apache.solr.common.cloud.ZkStateReader.COLLECTION_PROP;
 import static org.apache.solr.common.cloud.ZkStateReader.CORE_NAME_PROP;
-import static org.apache.solr.common.cloud.ZkStateReader.CORE_NODE_NAME_PROP;
 import static org.apache.solr.common.cloud.ZkStateReader.ELECTION_NODE_PROP;
 import static org.apache.solr.common.cloud.ZkStateReader.NODE_NAME_PROP;
 import static org.apache.solr.common.cloud.ZkStateReader.REJOIN_AT_HEAD_PROP;
 import static org.apache.solr.common.cloud.ZkStateReader.SHARD_ID_PROP;
+import static org.apache.solr.common.cloud.ZkStateReader.CORE_NODE_NAME_PROP;
 
 /**
  * Handle ZooKeeper interactions.
- * 
+ * <p>
  * notes: loads everything on init, creates what's not there - further updates
  * are prompted with Watches.
- * 
+ * <p>
  * TODO: exceptions during close on attempts to update cloud state
- * 
  */
 public final class ZkController {
 
@@ -122,7 +122,7 @@ public final class ZkController {
   static final int WAIT_DOWN_STATES_TIMEOUT_SECONDS = 60;
 
   private final boolean SKIP_AUTO_RECOVERY = Boolean.getBoolean("solrcloud.skip.autorecovery");
-  
+
   private final DistributedQueue overseerJobQueue;
   private final OverseerTaskQueue overseerCollectionQueue;
   private final OverseerTaskQueue overseerConfigSetQueue;
@@ -131,19 +131,19 @@ public final class ZkController {
   private final DistributedMap overseerCompletedMap;
   private final DistributedMap overseerFailureMap;
 
-  public final static String COLLECTION_PARAM_PREFIX="collection.";
-  public final static String CONFIGNAME_PROP="configName";
+  public final static String COLLECTION_PARAM_PREFIX = "collection.";
+  public final static String CONFIGNAME_PROP = "configName";
 
   static class ContextKey {
 
     private String collection;
     private String coreNodeName;
-    
+
     public ContextKey(String collection, String coreNodeName) {
       this.collection = collection;
       this.coreNodeName = coreNodeName;
     }
-    
+
     @Override
     public int hashCode() {
       final int prime = 31;
@@ -170,8 +170,9 @@ public final class ZkController {
       return true;
     }
   }
+
   private final Map<ContextKey, ElectionContext> electionContexts = Collections.synchronizedMap(new HashMap<ContextKey, ElectionContext>());
-  
+
   private final SolrZkClient zkClient;
   private final ZkCmdExecutor cmdExecutor;
   private final ZkStateReader zkStateReader;
@@ -186,7 +187,7 @@ public final class ZkController {
   private final CloudConfig cloudConfig;
 
   private LeaderElector overseerElector;
-  
+
 
   // for now, this can be null in tests, in which case recovery will be inactive, and other features
   // may accept defaults or use mocks rather than pulling things from a CoreContainer
@@ -196,16 +197,16 @@ public final class ZkController {
 
   private int leaderVoteWait;
   private int leaderConflictResolveWait;
-  
+
   private boolean genericCoreNodeNames;
 
   private int clientTimeout;
 
   private volatile boolean isClosed;
-  
+
   // keeps track of replicas that have been asked to recover by leaders running on this node
-  private final Map<String,String> replicasInLeaderInitiatedRecovery = new HashMap<String,String>();
-  
+  private final Map<String, String> replicasInLeaderInitiatedRecovery = new HashMap<>();
+
   // This is an expert and unsupported development mode that does not create
   // an Overseer or register a /live node. This let's you monitor the cluster
   // and interact with zookeeper via the Solr admin UI on a node outside the cluster,
@@ -235,8 +236,7 @@ public final class ZkController {
   }
 
   public ZkController(final CoreContainer cc, String zkServerAddress, int zkClientConnectTimeout, CloudConfig cloudConfig, final CurrentCoreDescriptorProvider registerOnReconnect)
-      throws InterruptedException, TimeoutException, IOException
-  {
+      throws InterruptedException, TimeoutException, IOException {
 
     if (cc == null) throw new IllegalArgumentException("CoreContainer cannot be null.");
     this.cc = cc;
@@ -264,7 +264,7 @@ public final class ZkController {
     String zkACLProviderClass = cloudConfig.getZkACLProviderClass();
     ZkACLProvider zkACLProvider = null;
     if (zkACLProviderClass != null && zkACLProviderClass.trim().length() > 0) {
-      zkACLProvider  = cc.getResourceLoader().newInstance(zkACLProviderClass, ZkACLProvider.class);
+      zkACLProvider = cc.getResourceLoader().newInstance(zkACLProviderClass, ZkACLProvider.class);
     } else {
       zkACLProvider = new DefaultZkACLProvider();
     }
@@ -351,7 +351,7 @@ public final class ZkController {
                     listener.command();
                   } catch (Exception exc) {
                     // not much we can do here other than warn in the log
-                    log.warn("Error when notifying OnReconnect listener "+listener+" after session re-connected.", exc);
+                    log.warn("Error when notifying OnReconnect listener " + listener + " after session re-connected.", exc);
                   }
                 }
               }
@@ -404,7 +404,7 @@ public final class ZkController {
   public int getLeaderVoteWait() {
     return leaderVoteWait;
   }
-  
+
   public int getLeaderConflictResolveWait() {
     return leaderConflictResolveWait;
   }
@@ -438,7 +438,7 @@ public final class ZkController {
           }
         }
       }
-        
+
       for (CoreDescriptor descriptor : descriptors) {
         // if it looks like we are going to be the leader, we don't
         // want to wait for the following stuff
@@ -446,7 +446,7 @@ public final class ZkController {
         String collection = cloudDesc.getCollectionName();
         String slice = cloudDesc.getShardId();
         try {
-          
+
           int children = zkStateReader
               .getZkClient()
               .getChildren(
@@ -469,7 +469,7 @@ public final class ZkController {
 
         final String coreZkNodeName = descriptor.getCloudDescriptor().getCoreNodeName();
         try {
-          log.debug("calling waitForLeaderToSeeDownState for coreZkNodeName={} collection={} shard={}", new Object[] {coreZkNodeName,  collection, slice});
+          log.debug("calling waitForLeaderToSeeDownState for coreZkNodeName={} collection={} shard={}", new Object[]{coreZkNodeName, collection, slice});
           waitForLeaderToSeeDownState(descriptor, coreZkNodeName);
         } catch (Exception e) {
           SolrException.log(log, "", e);
@@ -559,7 +559,7 @@ public final class ZkController {
         }
       }
     }
-    
+
   }
 
   /**
@@ -590,7 +590,7 @@ public final class ZkController {
       throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR,
           "Config file contains no data:" + zkPath);
     }
-    
+
     return bytes;
   }
 
@@ -628,18 +628,18 @@ public final class ZkController {
       }
       host = hostaddress;
     } else {
-      if(URLUtil.hasScheme(host)) {
+      if (URLUtil.hasScheme(host)) {
         host = URLUtil.removeScheme(host);
       }
     }
 
     return host;
   }
-  
+
   public String getHostName() {
     return hostName;
   }
-  
+
   public int getHostPort() {
     return localHostPort;
   }
@@ -657,8 +657,9 @@ public final class ZkController {
 
   /**
    * Create the zknodes necessary for a cluster to operate
+   *
    * @param zkClient a SolrZkClient
-   * @throws KeeperException if there is a Zookeeper error
+   * @throws KeeperException      if there is a Zookeeper error
    * @throws InterruptedException on interrupt
    */
   public static void createClusterZkNodes(SolrZkClient zkClient) throws KeeperException, InterruptedException {
@@ -676,12 +677,12 @@ public final class ZkController {
     try {
       boolean createdWatchesAndUpdated = false;
       Stat stat = zkClient.exists(ZkStateReader.LIVE_NODES_ZKNODE, null, true);
-      if (stat!= null && stat.getNumChildren()>0) {
+      if (stat != null && stat.getNumChildren() > 0) {
         zkStateReader.createClusterStateWatchersAndUpdate();
         createdWatchesAndUpdated = true;
         publishAndWaitForDownStates();
       }
-      
+
       createClusterZkNodes(zkClient);
 
       createEphemeralLiveNode();
@@ -690,7 +691,7 @@ public final class ZkController {
       UpdateShardHandler updateShardHandler;
       shardHandler = cc.getShardHandlerFactory().getShardHandler();
       updateShardHandler = cc.getUpdateShardHandler();
-      
+
       if (!zkRunOnly) {
         overseerElector = new LeaderElector(zkClient);
         this.overseer = new Overseer(shardHandler, updateShardHandler,
@@ -700,11 +701,11 @@ public final class ZkController {
         overseerElector.setup(context);
         overseerElector.joinElection(context, false);
       }
-      
+
       if (!createdWatchesAndUpdated) {
         zkStateReader.createClusterStateWatchersAndUpdate();
       }
-      
+
     } catch (IOException e) {
       log.error("", e);
       throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
@@ -725,7 +726,7 @@ public final class ZkController {
 
   public void publishAndWaitForDownStates() throws KeeperException,
       InterruptedException {
-    
+
     ClusterState clusterState = zkStateReader.getClusterState();
     Set<String> collections = clusterState.getCollections();
     Set<String> updatedCoreNodeNames = new HashSet<>();
@@ -755,7 +756,7 @@ public final class ZkController {
         }
       }
     }
-    
+
     // now wait till the updates are in our state
     long now = System.nanoTime();
     long timeout = now + TimeUnit.NANOSECONDS.convert(WAIT_DOWN_STATES_TIMEOUT_SECONDS, TimeUnit.SECONDS);
@@ -771,12 +772,12 @@ public final class ZkController {
           for (Replica replica : replicas) {
             if (replica.getState() == Replica.State.DOWN) {
               updatedCoreNodeNames.remove(replica.getName());
-              
+
             }
           }
         }
       }
-      
+
       if (updatedCoreNodeNames.size() == 0) {
         foundStates = true;
         Thread.sleep(1000);
@@ -787,16 +788,16 @@ public final class ZkController {
     if (!foundStates) {
       log.warn("Timed out waiting to see all nodes published as DOWN in our cluster state.");
     }
-    
+
   }
-  
+
   /**
    * Validates if the chroot exists in zk (or if it is successfully created).
    * Optionally, if create is set to true this method will create the path in
    * case it doesn't exist
-   * 
+   *
    * @return true if the path exists or is created false if the path doesn't
-   *         exist and 'create' = false
+   * exist and 'create' = false
    */
   public static boolean checkChrootPath(String zkHost, boolean create)
       throws KeeperException, InterruptedException {
@@ -829,7 +830,7 @@ public final class ZkController {
     String nodeName = getNodeName();
     String nodePath = ZkStateReader.LIVE_NODES_ZKNODE + "/" + nodeName;
     log.info("Register node as live in ZooKeeper:" + nodePath);
-   
+
     try {
       boolean nodeDeleted = true;
       try {
@@ -855,9 +856,9 @@ public final class ZkController {
       if (e.code() != KeeperException.Code.NODEEXISTS) {
         throw e;
       }
-    }    
+    }
   }
-  
+
   public String getNodeName() {
     return nodeName;
   }
@@ -873,17 +874,17 @@ public final class ZkController {
 
   /**
    * Register shard with ZooKeeper.
-   * 
+   *
    * @return the shardId for the SolrCore
    */
-  public String register(String coreName, final CoreDescriptor desc) throws Exception {  
+  public String register(String coreName, final CoreDescriptor desc) throws Exception {
     return register(coreName, desc, false, false);
   }
-  
+
 
   /**
    * Register shard with ZooKeeper.
-   * 
+   *
    * @return the shardId for the SolrCore
    */
   public String register(String coreName, final CoreDescriptor desc, boolean recoverReloadedCores, boolean afterExpiration) throws Exception {
@@ -891,36 +892,34 @@ public final class ZkController {
       MDCLoggingContext.setCore(core);
     }
     try {
-    // pre register has published our down state
-    final String baseUrl = getBaseUrl();
-    
-    final CloudDescriptor cloudDesc = desc.getCloudDescriptor();
-    final String collection = cloudDesc.getCollectionName();
-
-    final String coreZkNodeName = desc.getCloudDescriptor().getCoreNodeName();
-    assert coreZkNodeName != null : "we should have a coreNodeName by now";
-    
-    String shardId = cloudDesc.getShardId();
-
-    Map<String,Object> props = new HashMap<>();
- // we only put a subset of props into the leader node
-    props.put(ZkStateReader.BASE_URL_PROP, baseUrl);
-    props.put(ZkStateReader.CORE_NAME_PROP, coreName);
-    props.put(ZkStateReader.NODE_NAME_PROP, getNodeName());
-
-
-    if (log.isInfoEnabled()) {
-        log.info("Register replica - core:" + coreName + " address:"
-            + baseUrl + " collection:" + cloudDesc.getCollectionName() + " shard:" + shardId);
-    }
-
-    ZkNodeProps leaderProps = new ZkNodeProps(props);
-    
- 
+      // pre register has published our down state
+      final String baseUrl = getBaseUrl();
+      
+      final CloudDescriptor cloudDesc = desc.getCloudDescriptor();
+      final String collection = cloudDesc.getCollectionName();
+      
+      final String coreZkNodeName = desc.getCloudDescriptor().getCoreNodeName();
+      assert coreZkNodeName != null : "we should have a coreNodeName by now";
+      
+      String shardId = cloudDesc.getShardId();
+      Map<String,Object> props = new HashMap<>();
+      // we only put a subset of props into the leader node
+      props.put(ZkStateReader.BASE_URL_PROP, baseUrl);
+      props.put(ZkStateReader.CORE_NAME_PROP, coreName);
+      props.put(ZkStateReader.NODE_NAME_PROP, getNodeName());
+      
+      if (log.isInfoEnabled()) {
+        log.info("Register replica - core:" + coreName + " address:" + baseUrl + " collection:"
+            + cloudDesc.getCollectionName() + " shard:" + shardId);
+      }
+      
+      ZkNodeProps leaderProps = new ZkNodeProps(props);
+      
       try {
         // If we're a preferred leader, insert ourselves at the head of the queue
         boolean joinAtHead = false;
-        Replica replica = zkStateReader.getClusterState().getReplica(desc.getCloudDescriptor().getCollectionName(), coreZkNodeName);
+        Replica replica = zkStateReader.getClusterState().getReplica(desc.getCloudDescriptor().getCollectionName(),
+            coreZkNodeName);
         if (replica != null) {
           joinAtHead = replica.getBool(SliceMutator.PREFERRED_LEADER_PROP, false);
         }
@@ -932,25 +931,24 @@ public final class ZkController {
       } catch (KeeperException | IOException e) {
         throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR, "", e);
       }
-
-
-      // in this case, we want to wait for the leader as long as the leader might 
+      
+      // in this case, we want to wait for the leader as long as the leader might
       // wait for a vote, at least - but also long enough that a large cluster has
       // time to get its act together
       String leaderUrl = getLeader(cloudDesc, leaderVoteWait + 600000);
-
+      
       String ourUrl = ZkCoreNodeProps.getCoreUrl(baseUrl, coreName);
       log.info("We are " + ourUrl + " and leader is " + leaderUrl);
       boolean isLeader = leaderUrl.equals(ourUrl);
-
+      
       try (SolrCore core = cc.getCore(desc.getName())) {
-
+        
         // recover from local transaction log and wait for it to complete before
         // going active
         // TODO: should this be moved to another thread? To recoveryStrat?
         // TODO: should this actually be done earlier, before (or as part of)
         // leader election perhaps?
-
+        
         UpdateLog ulog = core.getUpdateHandler().getUpdateLog();
         
         // we will call register again after zk expiration and on reload
@@ -970,13 +968,13 @@ public final class ZkController {
             }
           }
         }
-        boolean didRecovery = checkRecovery(coreName, desc, recoverReloadedCores, isLeader, cloudDesc,
-            collection, coreZkNodeName, shardId, leaderProps, core, cc, afterExpiration);
+        boolean didRecovery = checkRecovery(coreName, desc, recoverReloadedCores, isLeader, cloudDesc, collection,
+            coreZkNodeName, shardId, leaderProps, core, cc, afterExpiration);
         if (!didRecovery) {
           publish(desc, Replica.State.ACTIVE);
         }
       }
-
+      
       // make sure we have an update cluster state right away
       zkStateReader.updateClusterState();
       return shardId;
@@ -988,7 +986,7 @@ public final class ZkController {
   // timeoutms is the timeout for the first call to get the leader - there is then
   // a longer wait to make sure that leader matches our local state
   private String getLeader(final CloudDescriptor cloudDesc, int timeoutms) {
-    
+
     String collection = cloudDesc.getCollectionName();
     String shardId = cloudDesc.getShardId();
     // rather than look in the cluster state file, we go straight to the zknodes
@@ -998,14 +996,14 @@ public final class ZkController {
     try {
       leaderUrl = getLeaderProps(collection, cloudDesc.getShardId(), timeoutms)
           .getCoreUrl();
-      
+
       // now wait until our currently cloud state contains the latest leader
       String clusterStateLeaderUrl = zkStateReader.getLeaderUrl(collection,
           shardId, timeoutms * 2); // since we found it in zk, we are willing to
-                                   // wait a while to find it in state
+      // wait a while to find it in state
       int tries = 0;
       final long msInSec = 1000L;
-      int maxTries = (int)Math.floor(leaderConflictResolveWait/msInSec);
+      int maxTries = (int) Math.floor(leaderConflictResolveWait / msInSec);
       while (!leaderUrl.equals(clusterStateLeaderUrl)) {
         if (tries > maxTries) {
           throw new SolrException(ErrorCode.SERVER_ERROR,
@@ -1016,7 +1014,7 @@ public final class ZkController {
         tries++;
         if (tries % 30 == 0) {
           String warnMsg = String.format(Locale.ENGLISH, "Still seeing conflicting information about the leader "
-              + "of shard %s for collection %s after %d seconds; our state says %s, but ZooKeeper says %s",
+                  + "of shard %s for collection %s after %d seconds; our state says %s, but ZooKeeper says %s",
               cloudDesc.getShardId(), collection, tries, clusterStateLeaderUrl, leaderUrl);
           log.warn(warnMsg);
         }
@@ -1026,30 +1024,30 @@ public final class ZkController {
         leaderUrl = getLeaderProps(collection, cloudDesc.getShardId(), timeoutms)
             .getCoreUrl();
       }
-      
+
     } catch (Exception e) {
       log.error("Error getting leader from zk", e);
       throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
           "Error getting leader from zk for shard " + shardId, e);
-    } 
+    }
     return leaderUrl;
   }
-  
+
   /**
    * Get leader props directly from zk nodes.
    */
   public ZkCoreNodeProps getLeaderProps(final String collection,
-      final String slice, int timeoutms) throws InterruptedException {
+                                        final String slice, int timeoutms) throws InterruptedException {
     return getLeaderProps(collection, slice, timeoutms, false);
   }
-  
+
   /**
    * Get leader props directly from zk nodes.
-   * 
+   *
    * @return leader props
    */
   public ZkCoreNodeProps getLeaderProps(final String collection,
-      final String slice, int timeoutms, boolean failImmediatelyOnExpiration) throws InterruptedException {
+                                        final String slice, int timeoutms, boolean failImmediatelyOnExpiration) throws InterruptedException {
     int iterCount = timeoutms / 1000;
     Exception exp = null;
     while (iterCount-- > 0) {
@@ -1068,7 +1066,7 @@ public final class ZkController {
         }
         exp = e;
         Thread.sleep(1000);
-      }  catch (Exception e) {
+      } catch (Exception e) {
         exp = e;
         Thread.sleep(1000);
       }
@@ -1085,18 +1083,18 @@ public final class ZkController {
     // look for old context - if we find it, cancel it
     String collection = cd.getCloudDescriptor().getCollectionName();
     final String coreNodeName = cd.getCloudDescriptor().getCoreNodeName();
-    
+
     ContextKey contextKey = new ContextKey(collection, coreNodeName);
-    
+
     ElectionContext prevContext = electionContexts.get(contextKey);
-    
+
     if (prevContext != null) {
       prevContext.cancelElection();
     }
-    
+
     String shardId = cd.getCloudDescriptor().getShardId();
-    
-    Map<String,Object> props = new HashMap<>();
+
+    Map<String, Object> props = new HashMap<>();
     // we only put a subset of props into the leader node
     props.put(ZkStateReader.BASE_URL_PROP, getBaseUrl());
     props.put(ZkStateReader.CORE_NAME_PROP, cd.getName());
@@ -1130,29 +1128,29 @@ public final class ZkController {
     }
     boolean doRecovery = true;
     if (!isLeader) {
-      
+
       if (!afterExpiration && core.isReloaded() && !recoverReloadedCores) {
         doRecovery = false;
       }
-      
+
       if (doRecovery) {
         log.info("Core needs to recover:" + core.getName());
         core.getUpdateHandler().getSolrCoreState().doRecovery(cc, core.getCoreDescriptor());
         return true;
       }
-      
+
       // see if the leader told us to recover
       final Replica.State lirState = getLeaderInitiatedRecoveryState(collection, shardId,
           core.getCoreDescriptor().getCloudDescriptor().getCoreNodeName());
       if (lirState == Replica.State.DOWN) {
-        log.info("Leader marked core "+core.getName()+" down; starting recovery process");
+        log.info("Leader marked core " + core.getName() + " down; starting recovery process");
         core.getUpdateHandler().getSolrCoreState().doRecovery(cc, core.getCoreDescriptor());
-        return true;        
+        return true;
       }
     } else {
       log.info("I am the leader, no recovery necessary");
     }
-    
+
     return false;
   }
 
@@ -1164,11 +1162,11 @@ public final class ZkController {
   public void publish(final CoreDescriptor cd, final Replica.State state) throws KeeperException, InterruptedException {
     publish(cd, state, true);
   }
-  
+
   public void publish(final CoreDescriptor cd, final Replica.State state, boolean updateLastState) throws KeeperException, InterruptedException {
     publish(cd, state, updateLastState, false);
   }
-  
+
   /**
    * Publish core state to overseer.
    */
@@ -1207,7 +1205,7 @@ public final class ZkController {
           if (state == Replica.State.ACTIVE) {
             // trying to become active, so leader-initiated state must be recovering
             if (lirState == Replica.State.RECOVERING) {
-              updateLeaderInitiatedRecoveryState(collection, shardId, coreNodeName, Replica.State.ACTIVE, null);
+              updateLeaderInitiatedRecoveryState(collection, shardId, coreNodeName, Replica.State.ACTIVE, null, true);
             } else if (lirState == Replica.State.DOWN) {
               throw new SolrException(ErrorCode.INVALID_STATE,
                   "Cannot publish state of core '" + cd.getName() + "' as active without recovering first!");
@@ -1215,7 +1213,7 @@ public final class ZkController {
           } else if (state == Replica.State.RECOVERING) {
             // if it is currently DOWN, then trying to enter into recovering state is good
             if (lirState == Replica.State.DOWN) {
-              updateLeaderInitiatedRecoveryState(collection, shardId, coreNodeName, Replica.State.RECOVERING, null);
+              updateLeaderInitiatedRecoveryState(collection, shardId, coreNodeName, Replica.State.RECOVERING, null, true);
             }
           }
         }
@@ -1259,12 +1257,12 @@ public final class ZkController {
       MDCLoggingContext.clear();
     }
   }
-  
+
   private boolean needsToBeAssignedShardId(final CoreDescriptor desc,
-      final ClusterState state, final String coreNodeName) {
+                                           final ClusterState state, final String coreNodeName) {
 
     final CloudDescriptor cloudDesc = desc.getCloudDescriptor();
-    
+
     final String shardId = state.getShardId(getNodeName(), desc.getName());
 
     if (shardId != null) {
@@ -1277,15 +1275,15 @@ public final class ZkController {
   public void unregister(String coreName, CoreDescriptor cd) throws InterruptedException, KeeperException {
     final String coreNodeName = cd.getCloudDescriptor().getCoreNodeName();
     final String collection = cd.getCloudDescriptor().getCollectionName();
-    
+
     if (Strings.isNullOrEmpty(collection)) {
       log.error("No collection was specified.");
       assert false : "No collection was specified [" + collection + "]";
       return;
     }
-    
+
     ElectionContext context = electionContexts.remove(new ContextKey(collection, coreNodeName));
-    
+
     if (context != null) {
       context.cancelElection();
     }
@@ -1313,7 +1311,7 @@ public final class ZkController {
         ZkStateReader.CORE_NODE_NAME_PROP, coreNodeName);
     overseerJobQueue.offer(Utils.toJSON(m));
   }
-  
+
   public void createCollection(String collection) throws KeeperException,
       InterruptedException {
     ZkNodeProps m = new ZkNodeProps(Overseer.QUEUE_OPERATION,
@@ -1329,20 +1327,20 @@ public final class ZkController {
 
   public void createCollectionZkNode(CloudDescriptor cd) {
     String collection = cd.getCollectionName();
-    
+
     log.info("Check for collection zkNode:" + collection);
     String collectionPath = ZkStateReader.COLLECTIONS_ZKNODE + "/" + collection;
-    
+
     try {
-      if(!zkClient.exists(collectionPath, true)) {
+      if (!zkClient.exists(collectionPath, true)) {
         log.info("Creating collection in ZooKeeper:" + collection);
-       SolrParams params = cd.getParams();
+        SolrParams params = cd.getParams();
 
         try {
-          Map<String,Object> collectionProps = new HashMap<>();
+          Map<String, Object> collectionProps = new HashMap<>();
 
           // TODO: if collection.configName isn't set, and there isn't already a conf in zk, just use that?
-          String defaultConfigName = System.getProperty(COLLECTION_PARAM_PREFIX+CONFIGNAME_PROP, collection);
+          String defaultConfigName = System.getProperty(COLLECTION_PARAM_PREFIX + CONFIGNAME_PROP, collection);
 
           // params passed in - currently only done via core admin (create core commmand).
           if (params != null) {
@@ -1359,8 +1357,8 @@ public final class ZkController {
               // TODO: getting the configName from the collectionPath should fail since we already know it doesn't exist?
               getConfName(collection, collectionPath, collectionProps);
             }
-            
-          } else if(System.getProperty("bootstrap_confdir") != null) {
+
+          } else if (System.getProperty("bootstrap_confdir") != null) {
             // if we are bootstrapping a collection, default the config for
             // a new collection to the collection we are bootstrapping
             log.info("Setting config for collection:" + collection + " to " + defaultConfigName);
@@ -1368,17 +1366,17 @@ public final class ZkController {
             Properties sysProps = System.getProperties();
             for (String sprop : System.getProperties().stringPropertyNames()) {
               if (sprop.startsWith(COLLECTION_PARAM_PREFIX)) {
-                collectionProps.put(sprop.substring(COLLECTION_PARAM_PREFIX.length()), sysProps.getProperty(sprop));                
+                collectionProps.put(sprop.substring(COLLECTION_PARAM_PREFIX.length()), sysProps.getProperty(sprop));
               }
             }
-            
+
             // if the config name wasn't passed in, use the default
             if (!collectionProps.containsKey(CONFIGNAME_PROP))
-              collectionProps.put(CONFIGNAME_PROP,  defaultConfigName);
+              collectionProps.put(CONFIGNAME_PROP, defaultConfigName);
 
           } else if (Boolean.getBoolean("bootstrap_conf")) {
             // the conf name should should be the collection name of this core
-            collectionProps.put(CONFIGNAME_PROP,  cd.getCollectionName());
+            collectionProps.put(CONFIGNAME_PROP, cd.getCollectionName());
           } else {
             getConfName(collection, collectionPath, collectionProps);
           }
@@ -1397,24 +1395,23 @@ public final class ZkController {
       } else {
         log.info("Collection zkNode exists");
       }
-      
+
     } catch (KeeperException e) {
       // it's okay if another beats us creating the node
       if (e.code() == KeeperException.Code.NODEEXISTS) {
         return;
       }
       throw new SolrException(ErrorCode.SERVER_ERROR, "Error creating collection node in Zookeeper", e);
-    }
-    catch (InterruptedException e) {
+    } catch (InterruptedException e) {
       Thread.interrupted();
       throw new SolrException(ErrorCode.SERVER_ERROR, "Error creating collection node in Zookeeper", e);
     }
-    
+
   }
 
 
   private void getConfName(String collection, String collectionPath,
-      Map<String,Object> collectionProps) throws KeeperException,
+                           Map<String, Object> collectionProps) throws KeeperException,
       InterruptedException {
     // check for configName
     log.info("Looking for collection configName");
@@ -1428,7 +1425,7 @@ public final class ZkController {
           break;
         }
       }
-     
+
       // if there is only one conf, use that
       try {
         configNames = zkClient.getChildren(ZkConfigManager.CONFIGS_ZKNODE, null,
@@ -1439,16 +1436,16 @@ public final class ZkController {
       if (configNames != null && configNames.size() == 1) {
         // no config set named, but there is only 1 - use it
         log.info("Only one config set found in zk - using it:" + configNames.get(0));
-        collectionProps.put(CONFIGNAME_PROP,  configNames.get(0));
+        collectionProps.put(CONFIGNAME_PROP, configNames.get(0));
         break;
       }
-      
+
       if (configNames != null && configNames.contains(collection)) {
         log.info("Could not find explicit collection configName, but found config name matching collection name - using that set.");
-        collectionProps.put(CONFIGNAME_PROP,  collection);
+        collectionProps.put(CONFIGNAME_PROP, collection);
         break;
       }
-      
+
       log.info("Could not find collection configName - pausing for 3 seconds and trying again - try: " + retry);
       Thread.sleep(3000);
     }
@@ -1459,7 +1456,7 @@ public final class ZkController {
           "Could not find configName for collection " + collection + " found:" + configNames);
     }
   }
-  
+
   public ZkStateReader getZkStateReader() {
     return zkStateReader;
   }
@@ -1480,17 +1477,17 @@ public final class ZkController {
     int retryCount = 320;
     log.info("look for our core node name");
     while (retryCount-- > 0) {
-      Map<String,Slice> slicesMap = zkStateReader.getClusterState()
+      Map<String, Slice> slicesMap = zkStateReader.getClusterState()
           .getSlicesMap(descriptor.getCloudDescriptor().getCollectionName());
       if (slicesMap != null) {
-        
+
         for (Slice slice : slicesMap.values()) {
           for (Replica replica : slice.getReplicas()) {
             // TODO: for really large clusters, we could 'index' on this
-            
+
             String nodeName = replica.getStr(ZkStateReader.NODE_NAME_PROP);
             String core = replica.getStr(ZkStateReader.CORE_NAME_PROP);
-            
+
             String msgNodeName = getNodeName();
             String msgCore = descriptor.getName();
 
@@ -1525,23 +1522,23 @@ public final class ZkController {
         Thread.currentThread().interrupt();
       }
     }
-    
+
     throw new SolrException(ErrorCode.SERVER_ERROR,
         "Could not get shard id for core: " + cd.getName());
   }
 
 
-  public String getCoreNodeName(CoreDescriptor descriptor){
+  public String getCoreNodeName(CoreDescriptor descriptor) {
     String coreNodeName = descriptor.getCloudDescriptor().getCoreNodeName();
     if (coreNodeName == null && !genericCoreNodeNames) {
       // it's the default
       return getNodeName() + "_" + descriptor.getName();
     }
-    
+
     return coreNodeName;
   }
 
-  public void preRegister(CoreDescriptor cd ) {
+  public void preRegister(CoreDescriptor cd) {
 
     String coreNodeName = getCoreNodeName(cd);
 
@@ -1560,7 +1557,7 @@ public final class ZkController {
       publish(cd, Replica.State.DOWN, false, true);
       DocCollection collection = zkStateReader.getClusterState().getCollectionOrNull(cd.getCloudDescriptor().getCollectionName());
       if (collection != null) {
-        log.info("Registering watch for external collection {}", cd.getCloudDescriptor().getCollectionName());
+        log.info("Registering watch for collection {}", cd.getCloudDescriptor().getCollectionName());
         zkStateReader.addCollectionWatch(cd.getCloudDescriptor().getCollectionName());
       }
     } catch (KeeperException e) {
@@ -1571,7 +1568,7 @@ public final class ZkController {
       log.error("", e);
       throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR, "", e);
     }
-    
+
     if (cd.getCloudDescriptor().getShardId() == null && needsToBeAssignedShardId(cd, zkStateReader.getClusterState(), coreNodeName)) {
       doGetShardIdAndNodeNameProcess(cd);
     } else {
@@ -1587,7 +1584,7 @@ public final class ZkController {
       String coreNodeName = cloudDesc.getCoreNodeName();
       assert coreNodeName != null;
       if (cloudDesc.getShardId() == null) {
-        throw new SolrException(ErrorCode.SERVER_ERROR ,"No shard id for :" + cd);
+        throw new SolrException(ErrorCode.SERVER_ERROR, "No shard id for :" + cd);
       }
       long endTime = System.nanoTime() + TimeUnit.NANOSECONDS.convert(3, TimeUnit.SECONDS);
       String errMessage = null;
@@ -1612,7 +1609,7 @@ public final class ZkController {
       if (errMessage == null) {
         errMessage = "replica " + coreNodeName + " is not present in cluster state";
       }
-      throw new SolrException(ErrorCode.SERVER_ERROR, errMessage + ". state : "+ zkStateReader.getClusterState().getCollection(cd.getCollectionName()));
+      throw new SolrException(ErrorCode.SERVER_ERROR, errMessage + ". state : " + zkStateReader.getClusterState().getCollection(cd.getCollectionName()));
     }
   }
 
@@ -1622,7 +1619,7 @@ public final class ZkController {
     String collection = cloudDesc.getCollectionName();
     String shard = cloudDesc.getShardId();
     ZkCoreNodeProps leaderProps = null;
-    
+
     int retries = 6;
     for (int i = 0; i < retries; i++) {
       try {
@@ -1630,7 +1627,7 @@ public final class ZkController {
           throw new SolrException(ErrorCode.SERVICE_UNAVAILABLE,
               "We have been closed");
         }
-        
+
         // go straight to zk, not the cloud state - we must have current info
         leaderProps = getLeaderProps(collection, shard, 30000);
         break;
@@ -1653,28 +1650,28 @@ public final class ZkController {
     String myCoreNodeName = cloudDesc.getCoreNodeName();
     String myCoreName = descriptor.getName();
     String ourUrl = ZkCoreNodeProps.getCoreUrl(getBaseUrl(), myCoreName);
-    
+
     boolean isLeader = leaderProps.getCoreUrl().equals(ourUrl);
     if (!isLeader && !SKIP_AUTO_RECOVERY) {
-      
+
       // detect if this core is in leader-initiated recovery and if so, 
       // then we don't need the leader to wait on seeing the down state
       Replica.State lirState = null;
       try {
         lirState = getLeaderInitiatedRecoveryState(collection, shard, myCoreNodeName);
       } catch (Exception exc) {
-        log.error("Failed to determine if replica "+myCoreNodeName+
-            " is in leader-initiated recovery due to: "+exc, exc);
+        log.error("Failed to determine if replica " + myCoreNodeName +
+            " is in leader-initiated recovery due to: " + exc, exc);
       }
-      
+
       if (lirState != null) {
-        log.info("Replica "+myCoreNodeName+
+        log.info("Replica " + myCoreNodeName +
             " is already in leader-initiated recovery, so not waiting for leader to see down state.");
       } else {
-        
-        log.info("Replica "+myCoreNodeName+
+
+        log.info("Replica " + myCoreNodeName +
             " NOT in leader-initiated recovery, need to wait for leader to see down state.");
-            
+
         try (HttpSolrClient client = new HttpSolrClient(leaderBaseUrl)) {
           client.setConnectionTimeout(15000);
           client.setSoTimeout(120000);
@@ -1683,7 +1680,7 @@ public final class ZkController {
           prepCmd.setNodeName(getNodeName());
           prepCmd.setCoreNodeName(coreZkNodeName);
           prepCmd.setState(Replica.State.DOWN);
-          
+
           // let's retry a couple times - perhaps the leader just went down,
           // or perhaps he is just not quite ready for us yet
           retries = 6;
@@ -1708,8 +1705,8 @@ public final class ZkController {
                 // if there was a communication error talking to the leader, see if the leader is even alive
                 if (!zkStateReader.getClusterState().liveNodesContain(leaderProps.getNodeName())) {
                   throw new SolrException(ErrorCode.SERVICE_UNAVAILABLE,
-                      "Node "+leaderProps.getNodeName()+" hosting leader for "+
-                          shard+" in "+collection+" is not live!");
+                      "Node " + leaderProps.getNodeName() + " hosting leader for " +
+                          shard + " in " + collection + " is not live!");
                 }
               }
 
@@ -1733,7 +1730,7 @@ public final class ZkController {
     }
     return leaderProps;
   }
-  
+
   public static void linkConfSet(SolrZkClient zkClient, String collection, String confSetName) throws KeeperException, InterruptedException {
     String path = ZkStateReader.COLLECTIONS_ZKNODE + "/" + collection;
     if (log.isInfoEnabled()) {
@@ -1763,21 +1760,21 @@ public final class ZkController {
     }
     // we found existing data, let's update it
     ZkNodeProps props = null;
-    if(data != null) {
+    if (data != null) {
       props = ZkNodeProps.load(data);
-      Map<String,Object> newProps = new HashMap<>();
+      Map<String, Object> newProps = new HashMap<>();
       newProps.putAll(props.getProperties());
       newProps.put(CONFIGNAME_PROP, confSetName);
       props = new ZkNodeProps(newProps);
     } else {
       props = new ZkNodeProps(CONFIGNAME_PROP, confSetName);
     }
-    
+
     // TODO: we should consider using version
     zkClient.setData(path, Utils.toJSON(props), true);
 
   }
-  
+
   /**
    * If in SolrCloud mode, upload config sets for each SolrCore in solr.xml.
    */
@@ -1787,7 +1784,7 @@ public final class ZkController {
 
     //List<String> allCoreNames = cfg.getAllCoreNames();
     List<CoreDescriptor> cds = cc.getCoresLocator().discover(cc);
-    
+
     log.info("bootstrapping config for " + cds.size() + " cores into ZooKeeper using solr.xml from " + solrHome);
 
     for (CoreDescriptor cd : cds) {
@@ -1825,7 +1822,7 @@ public final class ZkController {
   public DistributedMap getOverseerFailureMap() {
     return overseerFailureMap;
   }
-  
+
   public int getClientTimeout() {
     return clientTimeout;
   }
@@ -1837,12 +1834,12 @@ public final class ZkController {
   public LeaderElector getOverseerElector() {
     return overseerElector;
   }
-  
+
   /**
    * Returns the nodeName that should be used based on the specified properties.
    *
-   * @param hostName - must not be null or the empty string
-   * @param hostPort - must consist only of digits, must not be null or the empty string
+   * @param hostName    - must not be null or the empty string
+   * @param hostPort    - must consist only of digits, must not be null or the empty string
    * @param hostContext - should not begin or end with a slash (leading/trailin slashes will be ignored), must not be null, may be the empty string to denote the root context
    * @lucene.experimental
    * @see ZkStateReader#getBaseUrlForNodeName
@@ -1851,56 +1848,56 @@ public final class ZkController {
                                  final String hostPort,
                                  final String hostContext) {
     try {
-      return hostName + ':' + hostPort + '_' + 
-        URLEncoder.encode(trimLeadingAndTrailingSlashes(hostContext), "UTF-8");
+      return hostName + ':' + hostPort + '_' +
+          URLEncoder.encode(trimLeadingAndTrailingSlashes(hostContext), "UTF-8");
     } catch (UnsupportedEncodingException e) {
       throw new Error("JVM Does not seem to support UTF-8", e);
     }
   }
-  
+
   /**
-   * Utility method for trimming and leading and/or trailing slashes from 
-   * its input.  May return the empty string.  May return null if and only 
+   * Utility method for trimming and leading and/or trailing slashes from
+   * its input.  May return the empty string.  May return null if and only
    * if the input is null.
    */
   public static String trimLeadingAndTrailingSlashes(final String in) {
     if (null == in) return in;
-    
+
     String out = in;
     if (out.startsWith("/")) {
       out = out.substring(1);
     }
     if (out.endsWith("/")) {
-      out = out.substring(0,out.length()-1);
+      out = out.substring(0, out.length() - 1);
     }
     return out;
   }
 
   public void rejoinOverseerElection(String electionNode, boolean joinAtHead) {
     try {
-      if(electionNode !=null){
+      if (electionNode != null) {
         //this call is from inside the JVM  . not from CoreAdminHandler
-        if(overseerElector.getContext() == null || overseerElector.getContext().leaderSeqPath == null){
+        if (overseerElector.getContext() == null || overseerElector.getContext().leaderSeqPath == null) {
           overseerElector.retryElection(new OverseerElectionContext(zkClient,
               overseer, getNodeName()), joinAtHead);
           return;
         }
-        if(!overseerElector.getContext().leaderSeqPath.endsWith(electionNode)){
-          log.warn("Asked to rejoin with wrong election node : {}, current node is {}",electionNode, overseerElector.getContext().leaderSeqPath);
+        if (!overseerElector.getContext().leaderSeqPath.endsWith(electionNode)) {
+          log.warn("Asked to rejoin with wrong election node : {}, current node is {}", electionNode, overseerElector.getContext().leaderSeqPath);
           //however delete it . This is possible when the last attempt at deleting the election node failed.
-          if(electionNode.startsWith(getNodeName())){
+          if (electionNode.startsWith(getNodeName())) {
             try {
-              zkClient.delete(OverseerElectionContext.PATH+LeaderElector.ELECTION_NODE+"/"+electionNode,-1,true);
+              zkClient.delete(OverseerElectionContext.PATH + LeaderElector.ELECTION_NODE + "/" + electionNode, -1, true);
             } catch (NoNodeException e) {
               //no problem
-            } catch (InterruptedException e){
+            } catch (InterruptedException e) {
               Thread.currentThread().interrupt();
-            } catch(Exception e) {
-              log.warn("Old election node exists , could not be removed ",e);
+            } catch (Exception e) {
+              log.warn("Old election node exists , could not be removed ", e);
             }
           }
         }
-      }else {
+      } else {
         overseerElector.retryElection(overseerElector.getContext(), joinAtHead);
       }
     } catch (Exception e) {
@@ -1915,20 +1912,20 @@ public final class ZkController {
       String collectionName = params.get(COLLECTION_PROP);
       String shardId = params.get(SHARD_ID_PROP);
       String coreNodeName = params.get(CORE_NODE_NAME_PROP);
-      
+
       // back compat for SOLR-7844
       String nodeNameParam = params.get(NODE_NAME_PROP);
       if (nodeNameParam != null) {
         coreNodeName = params.get(NODE_NAME_PROP);
       }
-      
+
       String coreName = params.get(CORE_NAME_PROP);
       String electionNode = params.get(ELECTION_NODE_PROP);
       String baseUrl = params.get(BASE_URL_PROP);
 
       try (SolrCore core = cc.getCore(coreName)) {
         MDCLoggingContext.setCore(core);
-         
+        
         log.info("Rejoin the shard leader election.");
         
         ContextKey contextKey = new ContextKey(collectionName, coreNodeName);
@@ -1957,49 +1954,49 @@ public final class ZkController {
   public void checkOverseerDesignate() {
     try {
       byte[] data = zkClient.getData(ZkStateReader.ROLES, null, new Stat(), true);
-      if(data ==null) return;
+      if (data == null) return;
       Map roles = (Map) Utils.fromJSON(data);
-      if(roles ==null) return;
-      List nodeList= (List) roles.get("overseer");
-      if(nodeList == null) return;
-      if(nodeList.contains(getNodeName())){
+      if (roles == null) return;
+      List nodeList = (List) roles.get("overseer");
+      if (nodeList == null) return;
+      if (nodeList.contains(getNodeName())) {
         ZkNodeProps props = new ZkNodeProps(Overseer.QUEUE_OPERATION, CollectionParams.CollectionAction.ADDROLE.toString().toLowerCase(Locale.ROOT),
             "node", getNodeName(),
             "role", "overseer");
-        log.info("Going to add role {} ",props);
+        log.info("Going to add role {} ", props);
         getOverseerCollectionQueue().offer(Utils.toJSON(props));
       }
-    } catch (NoNodeException nne){
+    } catch (NoNodeException nne) {
       return;
     } catch (Exception e) {
       log.warn("could not read the overseer designate ", e);
     }
   }
 
-  CoreContainer getCoreContainer(){
+  CoreContainer getCoreContainer() {
     return cc;
   }
-      
+
   /**
    * When a leader receives a communication error when trying to send a request to a replica,
    * it calls this method to ensure the replica enters recovery when connectivity is restored.
-   * 
+   * <p>
    * returns true if the node hosting the replica is still considered "live" by ZooKeeper;
    * false means the node is not live either, so no point in trying to send recovery commands
    * to it.
    */
   public boolean ensureReplicaInLeaderInitiatedRecovery(
+      final CoreContainer container,
       final String collection, final String shardId, final ZkCoreNodeProps replicaCoreProps,
-      String leaderCoreNodeName, boolean forcePublishState, boolean retryOnConnLoss)
-          throws KeeperException, InterruptedException 
-  {
+      String leaderCoreNodeName, boolean forcePublishState)
+      throws KeeperException, InterruptedException {
     final String replicaUrl = replicaCoreProps.getCoreUrl();
 
     if (collection == null)
-      throw new IllegalArgumentException("collection parameter cannot be null for starting leader-initiated recovery for replica: "+replicaUrl);
+      throw new IllegalArgumentException("collection parameter cannot be null for starting leader-initiated recovery for replica: " + replicaUrl);
 
     if (shardId == null)
-      throw new IllegalArgumentException("shard parameter cannot be null for starting leader-initiated recovery for replica: "+replicaUrl);
+      throw new IllegalArgumentException("shard parameter cannot be null for starting leader-initiated recovery for replica: " + replicaUrl);
 
     if (replicaUrl == null)
       throw new IllegalArgumentException("replicaUrl parameter cannot be null for starting leader-initiated recovery");
@@ -2009,28 +2006,41 @@ public final class ZkController {
     // about the same replica having trouble and we only need to send the "needs"
     // recovery signal once
     boolean nodeIsLive = true;
-    boolean publishDownState = false;
     String replicaNodeName = replicaCoreProps.getNodeName();
-    String replicaCoreNodeName = ((Replica)replicaCoreProps.getNodeProps()).getName();
-    assert replicaCoreNodeName != null : "No core name for replica "+replicaNodeName;
+    String replicaCoreNodeName = ((Replica) replicaCoreProps.getNodeProps()).getName();
+    assert replicaCoreNodeName != null : "No core name for replica " + replicaNodeName;
     synchronized (replicasInLeaderInitiatedRecovery) {
-      if (replicasInLeaderInitiatedRecovery.containsKey(replicaUrl)) {     
+      if (replicasInLeaderInitiatedRecovery.containsKey(replicaUrl)) {
         if (!forcePublishState) {
           log.debug("Replica {} already in leader-initiated recovery handling.", replicaUrl);
           return false; // already in this recovery process
         }
       }
 
-      // if the replica's state is not DOWN right now, make it so ...
-      // we only really need to try to send the recovery command if the node itself is "live"
+      // we only really need to try to start the LIR process if the node itself is "live"
       if (getZkStateReader().getClusterState().liveNodesContain(replicaNodeName)) {
+
+        LeaderInitiatedRecoveryThread lirThread =
+            new LeaderInitiatedRecoveryThread(this,
+                container,
+                collection,
+                shardId,
+                replicaCoreProps,
+                120,
+                leaderCoreNodeName); // core node name of current leader
+        ExecutorService executor = container.getUpdateShardHandler().getUpdateExecutor();
+        try {
+          MDC.put("DistributedUpdateProcessor.replicaUrlToRecover", replicaCoreProps.getCoreUrl());
+          executor.execute(lirThread);
+        } finally {
+          MDC.remove("DistributedUpdateProcessor.replicaUrlToRecover");
+        }
+
         // create a znode that requires the replica needs to "ack" to verify it knows it was out-of-sync
-        updateLeaderInitiatedRecoveryState(collection, shardId, replicaCoreNodeName, Replica.State.DOWN, leaderCoreNodeName);
         replicasInLeaderInitiatedRecovery.put(replicaUrl,
             getLeaderInitiatedRecoveryZnodePath(collection, shardId, replicaCoreNodeName));
-        log.info("Put replica core={} coreNodeName={} on "+
-          replicaNodeName+" into leader-initiated recovery.", replicaCoreProps.getCoreName(), replicaCoreNodeName);
-        publishDownState = true;        
+        log.info("Put replica core={} coreNodeName={} on " +
+            replicaNodeName + " into leader-initiated recovery.", replicaCoreProps.getCoreName(), replicaCoreNodeName);
       } else {
         nodeIsLive = false; // we really don't need to send the recovery request if the node is NOT live
         log.info("Node " + replicaNodeName +
@@ -2038,23 +2048,9 @@ public final class ZkController {
             replicaCoreProps.getCoreName(), replicaCoreNodeName);
         // publishDownState will be false to avoid publishing the "down" state too many times
         // as many errors can occur together and will each call into this method (SOLR-6189)        
-      }      
-    }    
-    
-    if (publishDownState || forcePublishState) {
-      String replicaCoreName = replicaCoreProps.getCoreName();
-      ZkNodeProps m = new ZkNodeProps(Overseer.QUEUE_OPERATION, "state",
-          ZkStateReader.STATE_PROP, Replica.State.DOWN.toString(),
-          ZkStateReader.BASE_URL_PROP, replicaCoreProps.getBaseUrl(),
-          ZkStateReader.CORE_NAME_PROP, replicaCoreProps.getCoreName(),
-          ZkStateReader.NODE_NAME_PROP, replicaCoreProps.getNodeName(),
-          ZkStateReader.SHARD_ID_PROP, shardId,
-          ZkStateReader.COLLECTION_PROP, collection);
-      log.warn("Leader is publishing core={} coreNodeName ={} state={} on behalf of un-reachable replica {}; forcePublishState? "+forcePublishState,
-          replicaCoreName, replicaCoreNodeName, Replica.State.DOWN.toString(), replicaUrl);
-      overseerJobQueue.offer(Utils.toJSON(m));
+      }
     }
-    
+
     return nodeIsLive;
   }
 
@@ -2065,13 +2061,13 @@ public final class ZkController {
     }
     return exists;
   }
-  
+
   public void removeReplicaFromLeaderInitiatedRecoveryHandling(String replicaUrl) {
-    synchronized(replicasInLeaderInitiatedRecovery) {
-      replicasInLeaderInitiatedRecovery.remove(replicaUrl);           
+    synchronized (replicasInLeaderInitiatedRecovery) {
+      replicasInLeaderInitiatedRecovery.remove(replicaUrl);
     }
-  }  
-  
+  }
+
   public Replica.State getLeaderInitiatedRecoveryState(String collection, String shardId, String coreNodeName) {
     final Map<String, Object> stateObj = getLeaderInitiatedRecoveryStateObject(collection, shardId, coreNodeName);
     if (stateObj == null) {
@@ -2081,11 +2077,11 @@ public final class ZkController {
     return stateStr == null ? null : Replica.State.getState(stateStr);
   }
 
-  public Map<String,Object> getLeaderInitiatedRecoveryStateObject(String collection, String shardId, String coreNodeName) {
+  public Map<String, Object> getLeaderInitiatedRecoveryStateObject(String collection, String shardId, String coreNodeName) {
 
     if (collection == null || shardId == null || coreNodeName == null)
       return null; // if we don't have complete data about a core in cloud mode, return null
-    
+
     String znodePath = getLeaderInitiatedRecoveryZnodePath(collection, shardId, coreNodeName);
     byte[] stateData = null;
     try {
@@ -2095,26 +2091,26 @@ public final class ZkController {
     } catch (ConnectionLossException | SessionExpiredException cle) {
       // sort of safe to ignore ??? Usually these are seen when the core is going down
       // or there are bigger issues to deal with than reading this znode
-      log.warn("Unable to read "+znodePath+" due to: "+cle);
+      log.warn("Unable to read " + znodePath + " due to: " + cle);
     } catch (Exception exc) {
-      log.error("Failed to read data from znode "+znodePath+" due to: "+exc);
+      log.error("Failed to read data from znode " + znodePath + " due to: " + exc);
       if (exc instanceof SolrException) {
-        throw (SolrException)exc;
+        throw (SolrException) exc;
       } else {
-        throw new SolrException(ErrorCode.SERVER_ERROR, 
-            "Failed to read data from znodePath: "+znodePath, exc);
+        throw new SolrException(ErrorCode.SERVER_ERROR,
+            "Failed to read data from znodePath: " + znodePath, exc);
       }
     }
 
-    Map<String,Object> stateObj = null;
+    Map<String, Object> stateObj = null;
     if (stateData != null && stateData.length > 0) {
       // TODO: Remove later ... this is for upgrading from 4.8.x to 4.10.3 (see: SOLR-6732)
-      if (stateData[0] == (byte)'{') {
+      if (stateData[0] == (byte) '{') {
         Object parsedJson = Utils.fromJSON(stateData);
         if (parsedJson instanceof Map) {
-          stateObj = (Map<String,Object>)parsedJson;
+          stateObj = (Map<String, Object>) parsedJson;
         } else {
-          throw new SolrException(ErrorCode.SERVER_ERROR, "Leader-initiated recovery state data is invalid! "+parsedJson);
+          throw new SolrException(ErrorCode.SERVER_ERROR, "Leader-initiated recovery state data is invalid! " + parsedJson);
         }
       } else {
         // old format still in ZK
@@ -2124,9 +2120,9 @@ public final class ZkController {
 
     return stateObj;
   }
-  
-  private void updateLeaderInitiatedRecoveryState(String collection, String shardId, String coreNodeName, 
-      Replica.State state, String leaderCoreNodeName) {
+
+  public void updateLeaderInitiatedRecoveryState(String collection, String shardId, String coreNodeName,
+      Replica.State state, String leaderCoreNodeName, boolean retryOnConnLoss) {
     if (collection == null || shardId == null || coreNodeName == null) {
       log.warn("Cannot set leader-initiated recovery state znode to "
           + state.toString() + " using: collection=" + collection
@@ -2135,47 +2131,53 @@ public final class ZkController {
     }
 
     String znodePath = getLeaderInitiatedRecoveryZnodePath(collection, shardId, coreNodeName);
-    
+
     if (state == Replica.State.ACTIVE) {
       // since we're marking it active, we don't need this znode anymore, so delete instead of update
       try {
-        zkClient.delete(znodePath, -1, false);
+        zkClient.delete(znodePath, -1, retryOnConnLoss);
       } catch (Exception justLogIt) {
         log.warn("Failed to delete znode " + znodePath, justLogIt);
       }
       return;
     }
 
-    Map<String,Object> stateObj = null;
+    Map<String, Object> stateObj = null;
     try {
       stateObj = getLeaderInitiatedRecoveryStateObject(collection, shardId, coreNodeName);
     } catch (Exception exc) {
       log.warn(exc.getMessage(), exc);
     }
-    if (stateObj == null)
+    if (stateObj == null) {
       stateObj = Utils.makeMap();
+    }
 
     stateObj.put(ZkStateReader.STATE_PROP, state.toString());
     // only update the createdBy value if it's not set
-    if (stateObj.get("createdByNodeName") == null)
-      stateObj.put("createdByNodeName", String.valueOf(this.nodeName));
+    if (stateObj.get("createdByNodeName") == null) {
+      stateObj.put("createdByNodeName", this.nodeName);
+    }
+    if (stateObj.get("createdByCoreNodeName") == null && leaderCoreNodeName != null)  {
+      stateObj.put("createdByCoreNodeName", leaderCoreNodeName);
+    }
 
     byte[] znodeData = Utils.toJSON(stateObj);
 
     try {
       if (state == Replica.State.DOWN) {
-        markShardAsDownIfLeader(collection, shardId, leaderCoreNodeName, znodePath, znodeData);
-      } else  {
+        markShardAsDownIfLeader(collection, shardId, leaderCoreNodeName, znodePath, znodeData, retryOnConnLoss);
+      } else {
+        // must retry on conn loss otherwise future election attempts may assume wrong LIR state
         if (zkClient.exists(znodePath, true)) {
-          zkClient.setData(znodePath, znodeData, true);
+          zkClient.setData(znodePath, znodeData, retryOnConnLoss);
         } else {
-          zkClient.makePath(znodePath, znodeData, true);
+          zkClient.makePath(znodePath, znodeData, retryOnConnLoss);
         }
       }
       log.info("Wrote {} to {}", state.toString(), znodePath);
     } catch (Exception exc) {
       if (exc instanceof SolrException) {
-        throw (SolrException)exc;
+        throw (SolrException) exc;
       } else {
         throw new SolrException(ErrorCode.SERVER_ERROR,
             "Failed to update data to " + state.toString() + " for znode: " + znodePath, exc);
@@ -2190,22 +2192,23 @@ public final class ZkController {
    * doesn't let us mark a node as down *after* we've already lost our session
    */
   private void markShardAsDownIfLeader(String collection, String shardId, String leaderCoreNodeName,
-                                       String znodePath, byte[] znodeData) throws KeeperException, InterruptedException {
+                                       String znodePath, byte[] znodeData,
+                                       boolean retryOnConnLoss) throws KeeperException, InterruptedException {
     String leaderSeqPath = getLeaderSeqPath(collection, leaderCoreNodeName);
     if (leaderSeqPath == null) {
-      throw new SolrException(ErrorCode.SERVER_ERROR,
+      throw new NotLeaderException(ErrorCode.SERVER_ERROR,
           "Failed to update data to 'down' for znode: " + znodePath +
               " because the zookeeper leader sequence for leader: " + leaderCoreNodeName + " is null");
     }
-    if (zkClient.exists(znodePath, true)) {
+    if (zkClient.exists(znodePath, retryOnConnLoss)) {
       List<Op> ops = new ArrayList<>(2);
       ops.add(Op.check(leaderSeqPath, -1)); // version doesn't matter, the seq path is unique
       ops.add(Op.setData(znodePath, znodeData, -1));
-      zkClient.multi(ops, true);
+      zkClient.multi(ops, retryOnConnLoss);
     } else {
       String parentZNodePath = getLeaderInitiatedRecoveryZnodePath(collection, shardId);
       try {
-        zkClient.makePath(parentZNodePath, true);
+        zkClient.makePath(parentZNodePath, retryOnConnLoss);
       } catch (KeeperException.NodeExistsException nee) {
         // if it exists, that's great!
       }
@@ -2213,30 +2216,30 @@ public final class ZkController {
       ops.add(Op.check(leaderSeqPath, -1)); // version doesn't matter, the seq path is unique
       ops.add(Op.create(znodePath, znodeData, zkClient.getZkACLProvider().getACLsToAdd(znodePath),
           CreateMode.PERSISTENT));
-      zkClient.multi(ops, true);
+      zkClient.multi(ops, retryOnConnLoss);
     }
   }
 
   public String getLeaderInitiatedRecoveryZnodePath(String collection, String shardId) {
-    return "/collections/"+collection+"/leader_initiated_recovery/"+shardId;
-  }  
-  
+    return "/collections/" + collection + "/leader_initiated_recovery/" + shardId;
+  }
+
   public String getLeaderInitiatedRecoveryZnodePath(String collection, String shardId, String coreNodeName) {
-    return getLeaderInitiatedRecoveryZnodePath(collection, shardId)+"/"+coreNodeName;
+    return getLeaderInitiatedRecoveryZnodePath(collection, shardId) + "/" + coreNodeName;
   }
-  
+
   public void throwErrorIfReplicaReplaced(CoreDescriptor desc) {
-      ClusterState clusterState = getZkStateReader().getClusterState();
-      if (clusterState != null) {
-        DocCollection collection = clusterState.getCollectionOrNull(desc
-            .getCloudDescriptor().getCollectionName());
-        if (collection != null) {
-          boolean autoAddReplicas = ClusterStateUtil.isAutoAddReplicas( getZkStateReader(), collection.getName());   
-          if (autoAddReplicas) {
-            CloudUtil.checkSharedFSFailoverReplaced(cc, desc);
-          }
+    ClusterState clusterState = getZkStateReader().getClusterState();
+    if (clusterState != null) {
+      DocCollection collection = clusterState.getCollectionOrNull(desc
+          .getCloudDescriptor().getCollectionName());
+      if (collection != null) {
+        boolean autoAddReplicas = ClusterStateUtil.isAutoAddReplicas(getZkStateReader(), collection.getName());
+        if (autoAddReplicas) {
+          CloudUtil.checkSharedFSFailoverReplaced(cc, desc);
         }
       }
+    }
   }
 
   /**
@@ -2329,7 +2332,7 @@ public final class ZkController {
     }
   }
 
-  public static  class ResourceModifiedInZkException extends SolrException {
+  public static class ResourceModifiedInZkException extends SolrException {
     public ResourceModifiedInZkException(ErrorCode code, String msg) {
       super(code, msg);
     }
@@ -2350,7 +2353,8 @@ public final class ZkController {
     }
   }
 
-  /**This will give a callback to the listener whenever a child is modified in the
+  /**
+   * This will give a callback to the listener whenever a child is modified in the
    * conf directory. It is the responsibility of the listener to check if the individual
    * item of interest has been modified.  When the last core which was interested in
    * this conf directory is gone the listeners will be removed automatically.
@@ -2388,10 +2392,11 @@ public final class ZkController {
     return confDirListeners;
   }
 
-  private final Map<String , Set<Runnable>> confDirectoryListeners =  new HashMap<>();
+  private final Map<String, Set<Runnable>> confDirectoryListeners = new HashMap<>();
+
+  private class WatcherImpl implements Watcher {
+    private final String zkDir;
 
-  private class WatcherImpl implements Watcher{
-    private final String zkDir ;
     private WatcherImpl(String dir) {
       this.zkDir = dir;
     }
@@ -2413,20 +2418,22 @@ public final class ZkController {
 
       boolean resetWatcher = false;
       try {
-          resetWatcher = fireEventListeners(this.zkDir);
-        } finally {
-          if (Event.EventType.None.equals(event.getType())) {
-            log.info("A node got unwatched for {}", zkDir);
-          } else {
-            if(resetWatcher) setConfWatcher(zkDir, this, stat);
-          }
+        resetWatcher = fireEventListeners(zkDir);
+      } finally {
+        if (Event.EventType.None.equals(event.getType())) {
+          log.info("A node got unwatched for {}", zkDir);
+        } else {
+          if (resetWatcher) setConfWatcher(zkDir, this, stat);
+          else log.info("A node got unwatched for {}", zkDir);
         }
       }
+    }
   }
-  private boolean fireEventListeners(String zkDir) {
+
+  private boolean fireEventListeners(final String zkDir) {
     synchronized (confDirectoryListeners) {
       // if this is not among directories to be watched then don't set the watcher anymore
-      if( !confDirectoryListeners.containsKey(zkDir)) {
+      if (!confDirectoryListeners.containsKey(zkDir)) {
         log.info("Watcher on {} is removed ", zkDir);
         return false;
       }
@@ -2437,6 +2444,7 @@ public final class ZkController {
           // run these in a separate thread because this can be long running
           @Override
           public void run() {
+            log.info("Running listeners for {}", zkDir);
             for (final Runnable listener : listenersCopy) {
               try {
                 listener.run();
@@ -2471,7 +2479,7 @@ public final class ZkController {
     return new OnReconnect() {
       @Override
       public void command() {
-        synchronized (confDirectoryListeners){
+        synchronized (confDirectoryListeners) {
           for (String s : confDirectoryListeners.keySet()) {
             setConfWatcher(s, new WatcherImpl(s), null);
             fireEventListeners(s);
@@ -2486,4 +2494,13 @@ public final class ZkController {
     ElectionContext context = electionContexts.get(key);
     return context != null ? context.leaderSeqPath : null;
   }
+
+  /**
+   * Thrown during leader initiated recovery process if current node is not leader
+   */
+  public static class NotLeaderException extends SolrException  {
+    public NotLeaderException(ErrorCode code, String msg) {
+      super(code, msg);
+    }
+  }
 }

Modified: lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java?rev=1702213&r1=1702212&r2=1702213&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java (original)
+++ lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java Thu Sep 10 10:48:18 2015
@@ -17,6 +17,21 @@ package org.apache.solr.update.processor
  * limitations under the License.
  */
 
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.locks.ReentrantLock;
+
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.CharsRefBuilder;
 import org.apache.solr.client.solrj.request.UpdateRequest;
@@ -49,7 +64,6 @@ import org.apache.solr.common.params.Upd
 import org.apache.solr.common.util.Hash;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.Utils;
-import org.apache.solr.core.CoreContainer;
 import org.apache.solr.core.CoreDescriptor;
 import org.apache.solr.handler.component.RealTimeGetComponent;
 import org.apache.solr.request.SolrQueryRequest;
@@ -72,23 +86,6 @@ import org.apache.solr.update.VersionInf
 import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import org.slf4j.MDC;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.locks.ReentrantLock;
 
 import static org.apache.solr.update.processor.DistributingUpdateProcessorFactory.DISTRIB_UPDATE_PARAM;
 
@@ -836,8 +833,6 @@ public class DistributedUpdateProcessor
         break;
       }
 
-      int maxTries = 1;       
-      boolean sendRecoveryCommand = true;
       String collection = null;
       String shardId = null;
 
@@ -878,33 +873,24 @@ public class DistributedUpdateProcessor
         if (cloudDesc.getCoreNodeName().equals(leaderCoreNodeName) && foundErrorNodeInReplicaList) {
           try {
             // if false, then the node is probably not "live" anymore
-            sendRecoveryCommand =
-                zkController.ensureReplicaInLeaderInitiatedRecovery(collection,
-                    shardId,
-                    stdNode.getNodeProps(),
-                    leaderCoreNodeName,
-                    false /* forcePublishState */,
-                    false /* retryOnConnLoss */
-                );
-
-            // we want to try more than once, ~10 minutes
-            if (sendRecoveryCommand) {
-              maxTries = 120;
-            } // else the node is no longer "live" so no need to send any recovery command
+            // and we do not need to send a recovery message
+            Throwable rootCause = SolrException.getRootCause(error.e);
+            log.error("Setting up to try to start recovery on replica {}", replicaUrl, rootCause);
+            zkController.ensureReplicaInLeaderInitiatedRecovery(
+                req.getCore().getCoreDescriptor().getCoreContainer(),
+                collection,
+                shardId,
+                stdNode.getNodeProps(),
+                leaderCoreNodeName,
+                false /* forcePublishState */
+            );
           } catch (Exception exc) {
             Throwable setLirZnodeFailedCause = SolrException.getRootCause(exc);
             log.error("Leader failed to set replica " +
                 error.req.node.getUrl() + " state to DOWN due to: " + setLirZnodeFailedCause, setLirZnodeFailedCause);
-            if (setLirZnodeFailedCause instanceof KeeperException.SessionExpiredException ||
-                setLirZnodeFailedCause instanceof KeeperException.ConnectionLossException) {
-              // our session is expired, which means our state is suspect, so don't go
-              // putting other replicas in recovery (see SOLR-6511)
-              sendRecoveryCommand = false;
-            } // else will go ahead and try to send the recovery command once after this error
           }
         } else {
           // not the leader anymore maybe or the error'd node is not my replica?
-          sendRecoveryCommand = false;
           if (!foundErrorNodeInReplicaList) {
             log.warn("Core "+cloudDesc.getCoreNodeName()+" belonging to "+collection+" "+
                 shardId+", does not have error'd node " + stdNode.getNodeProps().getCoreUrl() + " as a replica. " +
@@ -914,30 +900,6 @@ public class DistributedUpdateProcessor
                 shardId+", no request recovery command will be sent!");
           }
         }
-      } // else not a StdNode, recovery command still gets sent once
-            
-      if (!sendRecoveryCommand)
-        continue; // the replica is already in recovery handling or is not live   
-
-      Throwable rootCause = SolrException.getRootCause(error.e);
-      log.error("Setting up to try to start recovery on replica {}", replicaUrl, rootCause);
-
-      // try to send the recovery command to the downed replica in a background thread
-      CoreContainer coreContainer = req.getCore().getCoreDescriptor().getCoreContainer();
-      LeaderInitiatedRecoveryThread lirThread =
-          new LeaderInitiatedRecoveryThread(zkController,
-              coreContainer,
-              collection,
-              shardId,
-              error.req.node.getNodeProps(),
-              maxTries,
-              cloudDesc.getCoreNodeName()); // core node name of current leader
-      ExecutorService executor = coreContainer.getUpdateShardHandler().getUpdateExecutor();
-      try {
-        MDC.put("DistributedUpdateProcessor.replicaUrlToRecover", error.req.node.getNodeProps().getCoreUrl());
-        executor.execute(lirThread);
-      } finally {
-        MDC.remove("DistributedUpdateProcessor.replicaUrlToRecover");
       }
     }
 

Modified: lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/cloud/HttpPartitionTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/cloud/HttpPartitionTest.java?rev=1702213&r1=1702212&r2=1702213&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/cloud/HttpPartitionTest.java (original)
+++ lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/cloud/HttpPartitionTest.java Thu Sep 10 10:48:18 2015
@@ -140,14 +140,10 @@ public class HttpPartitionTest extends A
     ZkCoreNodeProps replicaCoreNodeProps = new ZkCoreNodeProps(notLeader);
     String replicaUrl = replicaCoreNodeProps.getCoreUrl();
 
-    assertTrue(!zkController.isReplicaInRecoveryHandling(replicaUrl));
-    assertTrue(zkController.ensureReplicaInLeaderInitiatedRecovery(testCollectionName, shardId, replicaCoreNodeProps, leader.getName(), false, true));
-    assertTrue(zkController.isReplicaInRecoveryHandling(replicaUrl));
+    zkController.updateLeaderInitiatedRecoveryState(testCollectionName, shardId, notLeader.getName(), Replica.State.DOWN, leader.getName(), true);
     Map<String,Object> lirStateMap = zkController.getLeaderInitiatedRecoveryStateObject(testCollectionName, shardId, notLeader.getName());
     assertNotNull(lirStateMap);
     assertSame(Replica.State.DOWN, Replica.State.getState((String) lirStateMap.get(ZkStateReader.STATE_PROP)));
-    zkController.removeReplicaFromLeaderInitiatedRecoveryHandling(replicaUrl);
-    assertTrue(!zkController.isReplicaInRecoveryHandling(replicaUrl));
 
     // test old non-json format handling
     SolrZkClient zkClient = zkController.getZkClient();