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/02/26 13:19:43 UTC

svn commit: r1662432 - in /lucene/dev/branches/lucene_solr_4_10: ./ solr/ solr/CHANGES.txt solr/core/ solr/core/src/java/org/apache/solr/cloud/ElectionContext.java

Author: shalin
Date: Thu Feb 26 12:19:43 2015
New Revision: 1662432

URL: http://svn.apache.org/r1662432
Log:
SOLR-6763: Shard leader elections should not persist across ZK session expiry

Modified:
    lucene/dev/branches/lucene_solr_4_10/   (props changed)
    lucene/dev/branches/lucene_solr_4_10/solr/   (props changed)
    lucene/dev/branches/lucene_solr_4_10/solr/CHANGES.txt   (contents, props changed)
    lucene/dev/branches/lucene_solr_4_10/solr/core/   (props changed)
    lucene/dev/branches/lucene_solr_4_10/solr/core/src/java/org/apache/solr/cloud/ElectionContext.java

Modified: lucene/dev/branches/lucene_solr_4_10/solr/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene_solr_4_10/solr/CHANGES.txt?rev=1662432&r1=1662431&r2=1662432&view=diff
==============================================================================
--- lucene/dev/branches/lucene_solr_4_10/solr/CHANGES.txt (original)
+++ lucene/dev/branches/lucene_solr_4_10/solr/CHANGES.txt Thu Feb 26 12:19:43 2015
@@ -49,6 +49,9 @@ Bug Fixes
 * SOLR-6579: SnapPuller Replication blocks clean shutdown of tomcat
   (Philip Black-Knight via Noble Paul)
 
+* SOLR-6763: Shard leader elections should not persist across session expiry
+  (Alan Woodward, Mark Miller)
+
 ==================  4.10.3 ==================
 
 Bug Fixes

Modified: lucene/dev/branches/lucene_solr_4_10/solr/core/src/java/org/apache/solr/cloud/ElectionContext.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene_solr_4_10/solr/core/src/java/org/apache/solr/cloud/ElectionContext.java?rev=1662432&r1=1662431&r2=1662432&view=diff
==============================================================================
--- lucene/dev/branches/lucene_solr_4_10/solr/core/src/java/org/apache/solr/cloud/ElectionContext.java (original)
+++ lucene/dev/branches/lucene_solr_4_10/solr/core/src/java/org/apache/solr/cloud/ElectionContext.java Thu Feb 26 12:19:43 2015
@@ -1,9 +1,5 @@
 package org.apache.solr.cloud;
 
-import java.io.Closeable;
-import java.io.IOException;
-import java.util.concurrent.TimeUnit;
-
 import org.apache.lucene.search.MatchAllDocsQuery;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;
@@ -28,6 +24,7 @@ import org.apache.zookeeper.KeeperExcept
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.io.Closeable;
 import java.io.IOException;
 import java.util.List;
 import java.util.concurrent.ExecutorService;
@@ -136,13 +133,12 @@ class ShardLeaderElectionContextBase ext
     try {
       RetryUtil.retryOnThrowable(NodeExistsException.class, 15000, 1000,
           new RetryCmd() {
-            
             @Override
             public void execute() throws Throwable {
-              zkClient.makePath(leaderPath, ZkStateReader.toJSON(leaderProps),
-                  CreateMode.EPHEMERAL, true);
+              zkClient.makePath(leaderPath, ZkStateReader.toJSON(leaderProps), CreateMode.EPHEMERAL, true);
             }
-          });
+          }
+      );
     } catch (Throwable t) {
       if (t instanceof OutOfMemoryError) {
         throw (OutOfMemoryError) t;
@@ -402,13 +398,19 @@ final class ShardLeaderElectionContext e
     
     Slice slices = zkController.getClusterState().getSlice(collection, shardId);
     int cnt = 0;
-    while (true && !isClosed && !cc.isShutDown()) {
+    while (!isClosed && !cc.isShutDown()) {
       // wait for everyone to be up
       if (slices != null) {
         int found = 0;
         try {
           found = zkClient.getChildren(shardsElectZkPath, null, true).size();
         } catch (KeeperException e) {
+          if (e instanceof KeeperException.SessionExpiredException) {
+            // if the session has expired, then another election will be launched, so
+            // quit here
+            throw new SolrException(ErrorCode.SERVER_ERROR,
+                                    "ZK session expired - cancelling election for " + collection + " " + shardId);
+          }
           SolrException.log(log,
               "Error checking for the number of election participants", e);
         }