You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by yo...@apache.org on 2012/02/26 17:25:32 UTC

svn commit: r1293863 - /lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/ZkController.java

Author: yonik
Date: Sun Feb 26 16:25:32 2012
New Revision: 1293863

URL: http://svn.apache.org/viewvc?rev=1293863&view=rev
Log:
SOLR-3080: do tlog recovery for all nodes, not just leaders

Modified:
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/ZkController.java

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/ZkController.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/ZkController.java?rev=1293863&r1=1293862&r2=1293863&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/ZkController.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/ZkController.java Sun Feb 26 16:25:32 2012
@@ -273,6 +273,13 @@ public final class ZkController {
     return zkStateReader.getCloudState();
   }
 
+  /** @return the CoreState for the core, which may not yet be visible to ZooKeeper or other nodes in the cluster */
+  public CoreState getCoreState(String coreName) {
+    synchronized (coreStates) {
+      return coreStates.get(coreName);
+    }
+  }
+
   /**
    * @param zkConfigName
    * @param fileName
@@ -557,29 +564,28 @@ public final class ZkController {
       try {
         core = cc.getCore(desc.getName());
 
-        if (isLeader) {
-          // 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?
-          // TODO: ensure that a replica that is trying to recover waits until I'm
-          // active (or don't make me the
-          // leader until my local replay is done. But this replay is only needed
-          // on the leader - replicas
-          // will do recovery anyway
-          
-          UpdateLog ulog = core.getUpdateHandler().getUpdateLog();
-          if (!core.isReloaded() && ulog != null) {
-            Future<UpdateLog.RecoveryInfo> recoveryFuture = core.getUpdateHandler()
-                .getUpdateLog().recoverFromLog();
-            if (recoveryFuture != null) {
-              recoveryFuture.get(); // NOTE: this could potentially block for
-                                    // minutes or more!
-              // TODO: public as recovering in the mean time?
-            }
+
+        // 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?
+        // TODO: if I'm the leader, ensure that a replica that is trying to recover waits until I'm
+        // active (or don't make me the
+        // leader until my local replay is done.
+
+        UpdateLog ulog = core.getUpdateHandler().getUpdateLog();
+        if (!core.isReloaded() && ulog != null) {
+          Future<UpdateLog.RecoveryInfo> recoveryFuture = core.getUpdateHandler()
+              .getUpdateLog().recoverFromLog();
+          if (recoveryFuture != null) {
+            recoveryFuture.get(); // NOTE: this could potentially block for
+            // minutes or more!
+            // TODO: public as recovering in the mean time?
+            // TODO: in the future we could do peerync in parallel with recoverFromLog
           }
         }
+
         
         boolean didRecovery = checkRecovery(coreName, desc, recoverReloadedCores, isLeader, cloudDesc,
             collection, coreZkNodeName, shardId, leaderProps, core, cc);