You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by an...@apache.org on 2016/04/21 22:39:35 UTC

[1/4] lucene-solr:branch_5x: SOLR-8697: Fix precommit failure

Repository: lucene-solr
Updated Branches:
  refs/heads/branch_5x d2f68e13c -> 0b7b28a04


SOLR-8697: Fix precommit failure


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

Branch: refs/heads/branch_5x
Commit: 4b42c77ac6b045dce914d2244298cd0fea5300e4
Parents: d08169e
Author: Shalin Shekhar Mangar <sh...@apache.org>
Authored: Sat Feb 27 12:19:50 2016 +0530
Committer: Anshum Gupta <an...@apache.org>
Committed: Thu Apr 21 12:58:53 2016 -0700

----------------------------------------------------------------------
 solr/core/src/test/org/apache/solr/cloud/OverseerTest.java | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4b42c77a/solr/core/src/test/org/apache/solr/cloud/OverseerTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/OverseerTest.java b/solr/core/src/test/org/apache/solr/cloud/OverseerTest.java
index 98b04b4..0a5c609 100644
--- a/solr/core/src/test/org/apache/solr/cloud/OverseerTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/OverseerTest.java
@@ -26,6 +26,7 @@ import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
+import java.util.Locale;
 import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.ExecutorService;
@@ -125,7 +126,7 @@ public class OverseerTest extends SolrTestCaseJ4 {
         try {
           ec.cancelElection();
         } catch (Exception e) {
-          log.warn(String.format("Error cancelling election for %s", ec.id), e);
+          log.warn(String.format(Locale.ROOT, "Error cancelling election for %s", ec.id), e);
         }
       }
       deleteNode(ZkStateReader.LIVE_NODES_ZKNODE + "/" + nodeName);


[3/4] lucene-solr:branch_5x: SOLR-8697: Scope ZK election nodes by session to prevent elections from interfering with each other and other small LeaderElector improvements.

Posted by an...@apache.org.
SOLR-8697: Scope ZK election nodes by session to prevent elections from interfering with each other and other small LeaderElector improvements.


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

Branch: refs/heads/branch_5x
Commit: 0dabbadb4dbdabbad4da06a49e2eebd908a4cd62
Parents: d2f68e1
Author: markrmiller <ma...@apache.org>
Authored: Fri Feb 19 17:06:38 2016 -0500
Committer: Anshum Gupta <an...@apache.org>
Committed: Thu Apr 21 12:58:53 2016 -0700

----------------------------------------------------------------------
 solr/CHANGES.txt                                |   3 +
 .../org/apache/solr/cloud/ElectionContext.java  |   4 +-
 .../org/apache/solr/cloud/LeaderElector.java    | 101 ++++++-------------
 .../org/apache/solr/cloud/OverseerTest.java     |   6 ++
 4 files changed, 43 insertions(+), 71 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/0dabbadb/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 930f02b..7ae5757 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -91,6 +91,9 @@ Bug Fixes
 
 * SOLR-8656: PeerSync should use same nUpdates everywhere. (Ramsey Haddad via Mark Miller)
 
+* SOLR-8697: Scope ZK election nodes by session to prevent elections from interfering with each other
+  and other small LeaderElector improvements. (Scott Blum via Mark Miller)
+
 ======================= 5.5.0 =======================
 
 Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this release

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/0dabbadb/solr/core/src/java/org/apache/solr/cloud/ElectionContext.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/ElectionContext.java b/solr/core/src/java/org/apache/solr/cloud/ElectionContext.java
index 09cb3ce..871dbcd 100644
--- a/solr/core/src/java/org/apache/solr/cloud/ElectionContext.java
+++ b/solr/core/src/java/org/apache/solr/cloud/ElectionContext.java
@@ -137,6 +137,7 @@ class ShardLeaderElectionContextBase extends ElectionContext {
   
   @Override
   public void cancelElection() throws InterruptedException, KeeperException {
+    super.cancelElection();
     if (leaderZkNodeParentVersion != null) {
       try {
         // We need to be careful and make sure we *only* delete our own leader registration node.
@@ -163,7 +164,6 @@ class ShardLeaderElectionContextBase extends ElectionContext {
     } else {
       log.info("No version found for ephemeral leader parent node, won't remove previous leader registration.");
     }
-    super.cancelElection();
   }
   
   @Override
@@ -179,7 +179,7 @@ class ShardLeaderElectionContextBase extends ElectionContext {
         
         @Override
         public void execute() throws InterruptedException, KeeperException {
-          log.info("Creating leader registration node", leaderPath);
+          log.info("Creating leader registration node {} after winning as {}", leaderPath, leaderSeqPath);
           List<Op> ops = new ArrayList<>(2);
           
           // We use a multi operation to get the parent nodes version, which will

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/0dabbadb/solr/core/src/java/org/apache/solr/cloud/LeaderElector.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/LeaderElector.java b/solr/core/src/java/org/apache/solr/cloud/LeaderElector.java
index 056b4b9..aa1ace2 100644
--- a/solr/core/src/java/org/apache/solr/cloud/LeaderElector.java
+++ b/solr/core/src/java/org/apache/solr/cloud/LeaderElector.java
@@ -21,6 +21,7 @@ import java.lang.invoke.MethodHandles;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.Comparator;
+import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.regex.Matcher;
@@ -105,54 +106,34 @@ public  class LeaderElector {
     // get all other numbers...
     final String holdElectionPath = context.electionPath + ELECTION_NODE;
     List<String> seqs = zkClient.getChildren(holdElectionPath, null, true);
-
     sortSeqs(seqs);
-    List<Integer> intSeqs = getSeqs(seqs);
-    if (intSeqs.size() == 0) {
-      log.warn("Our node is no longer in line to be leader");
-      return;
-    }
-    // We can't really rely on the sequence number stored in the old watcher, it may be stale, thus this check.
 
-    int seq = -1;
-
-    // See if we've already been re-added, and this is an old context. In which case, use our current sequence number.
-    String newLeaderSeq = "";
-    for (String elec : seqs) {
-      if (getNodeName(elec).equals(getNodeName(context.leaderSeqPath)) && seq < getSeq(elec)) {
-        seq = getSeq(elec); // so use the current sequence number.
-        newLeaderSeq = elec;
-        break;
-      }
-    }
-
-    // Now, if we've been re-added, presumably we've also set up watchers and all that kind of thing, so we're done
-    if (StringUtils.isNotBlank(newLeaderSeq) && seq > getSeq(context.leaderSeqPath)) {
-      log.info("Node " + context.leaderSeqPath + " already in queue as " + newLeaderSeq + " nothing to do.");
+    String leaderSeqNodeName = context.leaderSeqPath.substring(context.leaderSeqPath.lastIndexOf('/') + 1);
+    if (!seqs.contains(leaderSeqNodeName)) {
+      log.warn("Our node is no longer in line to be leader");
       return;
     }
 
-    // Fallback in case we're all coming in here fresh and there is no node for this core already in the election queue.
-    if (seq == -1) {
-      seq = getSeq(context.leaderSeqPath);
-    }
-
-    if (seq <= intSeqs.get(0)) {
-      if (seq == intSeqs.get(0) && !context.leaderSeqPath.equals(holdElectionPath + "/" + seqs.get(0))) {//somebody else already  became the leader with the same sequence id , not me
-        log.info("was going to be leader {} , seq(0) {}", context.leaderSeqPath, holdElectionPath + "/" + seqs.get(0));//but someone else jumped the line
-
-        // The problem is that deleting the ZK node that's watched by others
-        // results in an unpredictable sequencing of the events and sometime the context that comes in for checking
-        // this happens to be after the node has already taken over leadership. So just leave out of here.
-        // This caused one of the tests to fail on having two nodes with the same name in the queue. I'm not sure
-        // the assumption that this is a bad state is valid.
-        if (getNodeName(context.leaderSeqPath).equals(getNodeName(seqs.get(0)))) {
-          return;
+    // If any double-registrations exist for me, remove all but this latest one!
+    // TODO: can we even get into this state?
+    String prefix = zkClient.getSolrZooKeeper().getSessionId() + "-" + context.id;
+    Iterator<String> it = seqs.iterator();
+    while (it.hasNext()) {
+      String elec = it.next();
+      if (!elec.equals(leaderSeqNodeName) && elec.startsWith(prefix)) {
+        try {
+          String toDelete = holdElectionPath + "/" + elec;
+          log.warn("Deleting duplicate registration: {}", toDelete);
+          zkClient.delete(toDelete, -1, true);
+        } catch (KeeperException.NoNodeException e) {
+          // ignore
         }
-        retryElection(context, false);//join at the tail again
-        return;
+        it.remove();
       }
+    }
 
+    if (leaderSeqNodeName.equals(seqs.get(0))) {
+      // I am the leader
       try {
         runIamLeaderProcess(context, replacement);
       } catch (KeeperException.NodeExistsException e) {
@@ -162,30 +143,25 @@ public  class LeaderElector {
       }
     } else {
       // I am not the leader - watch the node below me
-      int toWatch = -1;
-      for (int idx = 0; idx < intSeqs.size(); idx++) {
-        if (intSeqs.get(idx) < seq && ! getNodeName(context.leaderSeqPath).equals(getNodeName(seqs.get(idx)))) {
-          toWatch = idx;
-        }
-        if (intSeqs.get(idx) >= seq) {
+      String toWatch = seqs.get(0);
+      for (String node : seqs) {
+        if (leaderSeqNodeName.equals(node)) {
           break;
         }
-      }
-      if (toWatch < 0) {
-        log.warn("Our node is no longer in line to be leader");
-        return;
+        toWatch = node;
       }
       try {
-        String watchedNode = holdElectionPath + "/" + seqs.get(toWatch);
-
-        zkClient.getData(watchedNode, watcher = new ElectionWatcher(context.leaderSeqPath , watchedNode,seq, context) , null, true);
+        String watchedNode = holdElectionPath + "/" + toWatch;
+        zkClient.getData(watchedNode, watcher = new ElectionWatcher(context.leaderSeqPath, watchedNode, getSeq(context.leaderSeqPath), context), null, true);
         log.info("Watching path {} to know if I could be the leader", watchedNode);
       } catch (KeeperException.SessionExpiredException e) {
         throw e;
+      } catch (KeeperException.NoNodeException e) {
+        // the previous node disappeared, check if we are the leader again
+        checkIfIamLeader(context, true);
       } catch (KeeperException e) {
+        // we couldn't set our watch for some other reason, retry
         log.warn("Failed setting watch", e);
-        // we couldn't set our watch - the node before us may already be down?
-        // we need to check if we are the leader again
         checkIfIamLeader(context, true);
       }
     }
@@ -239,18 +215,6 @@ public  class LeaderElector {
 
   }
   
-  /**
-   * Returns int list given list of form n_0000000001, n_0000000003, etc.
-   * 
-   * @return int seqs
-   */
-  private List<Integer> getSeqs(List<String> seqs) {
-    List<Integer> intSeqs = new ArrayList<>(seqs.size());
-    for (String seq : seqs) {
-      intSeqs.add(getSeq(seq));
-    }
-    return intSeqs;
-  }
   public int joinElection(ElectionContext context, boolean replacement) throws KeeperException, InterruptedException, IOException {
     return joinElection(context,replacement, false);
   }
@@ -411,8 +375,7 @@ public  class LeaderElector {
       
       @Override
       public int compare(String o1, String o2) {
-        int i = Integer.valueOf(getSeq(o1)).compareTo(
-            Integer.valueOf(getSeq(o2)));
+        int i = getSeq(o1) - getSeq(o2);
         return i == 0 ? o1.compareTo(o2) : i ;
       }
     });

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/0dabbadb/solr/core/src/test/org/apache/solr/cloud/OverseerTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/OverseerTest.java b/solr/core/src/test/org/apache/solr/cloud/OverseerTest.java
index 379c9d1..108ccbe 100644
--- a/solr/core/src/test/org/apache/solr/cloud/OverseerTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/OverseerTest.java
@@ -157,6 +157,11 @@ public class OverseerTest extends SolrTestCaseJ4 {
         for (int i = 0; i < 120; i++) {
           String shardId = getShardId(collection, coreNodeName);
           if (shardId != null) {
+            ElectionContext prevContext = electionContext.get(coreName);
+            if (prevContext != null) {
+              prevContext.cancelElection();
+            }
+
             try {
               zkClient.makePath("/collections/" + collection + "/leader_elect/"
                   + shardId + "/election", true);
@@ -172,6 +177,7 @@ public class OverseerTest extends SolrTestCaseJ4 {
                 elector, shardId, collection, nodeName + "_" + coreName, props,
                 zkStateReader);
             elector.setup(ctx);
+            electionContext.put(coreName, ctx);
             elector.joinElection(ctx, false);
             return shardId;
           }


[2/4] lucene-solr:branch_5x: SOLR-8697: Add synchronization around registering as leader and canceling.

Posted by an...@apache.org.
SOLR-8697: Add synchronization around registering as leader and canceling.


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

Branch: refs/heads/branch_5x
Commit: d08169e6f6355d9023779cac1303ab48fe2c86e0
Parents: 0dabbad
Author: markrmiller <ma...@apache.org>
Authored: Fri Feb 26 12:32:12 2016 -0500
Committer: Anshum Gupta <an...@apache.org>
Committed: Thu Apr 21 12:58:53 2016 -0700

----------------------------------------------------------------------
 .../org/apache/solr/cloud/ElectionContext.java  | 111 ++++++++++---------
 .../org/apache/solr/cloud/ZkController.java     |   2 +-
 .../org/apache/solr/cloud/OverseerTest.java     |   7 ++
 3 files changed, 69 insertions(+), 51 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d08169e6/solr/core/src/java/org/apache/solr/cloud/ElectionContext.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/ElectionContext.java b/solr/core/src/java/org/apache/solr/cloud/ElectionContext.java
index 871dbcd..d4d02d8 100644
--- a/solr/core/src/java/org/apache/solr/cloud/ElectionContext.java
+++ b/solr/core/src/java/org/apache/solr/cloud/ElectionContext.java
@@ -110,8 +110,11 @@ class ShardLeaderElectionContextBase extends ElectionContext {
   protected String shardId;
   protected String collection;
   protected LeaderElector leaderElector;
-  protected volatile Integer leaderZkNodeParentVersion;
-  
+  private Integer leaderZkNodeParentVersion;
+
+  // Prevents a race between cancelling and becoming leader.
+  private final Object lock = new Object();
+
   public ShardLeaderElectionContextBase(LeaderElector leaderElector,
       final String shardId, final String collection, final String coreNodeName,
       ZkNodeProps props, ZkStateReader zkStateReader) {
@@ -138,31 +141,33 @@ class ShardLeaderElectionContextBase extends ElectionContext {
   @Override
   public void cancelElection() throws InterruptedException, KeeperException {
     super.cancelElection();
-    if (leaderZkNodeParentVersion != null) {
-      try {
-        // We need to be careful and make sure we *only* delete our own leader registration node.
-        // We do this by using a multi and ensuring the parent znode of the leader registration node
-        // matches the version we expect - there is a setData call that increments the parent's znode
-        // version whenever a leader registers.
-        log.info("Removing leader registration node on cancel: {} {}", leaderPath, leaderZkNodeParentVersion);
-        List<Op> ops = new ArrayList<>(2);
-        ops.add(Op.check(new Path(leaderPath).getParent().toString(), leaderZkNodeParentVersion));
-        ops.add(Op.delete(leaderPath, -1));
-        zkClient.multi(ops, true);
-      } catch (KeeperException.NoNodeException nne) {
-        // no problem
-        log.info("No leader registration node found to remove: {}", leaderPath);
-      } catch (KeeperException.BadVersionException bve) {
-        log.info("Cannot remove leader registration node because the current registered node is not ours: {}", leaderPath);
-        // no problem
-      } catch (InterruptedException e) {
-        throw e;
-      } catch (Exception e) {
-        SolrException.log(log, e);
+    synchronized (lock) {
+      if (leaderZkNodeParentVersion != null) {
+        try {
+          // We need to be careful and make sure we *only* delete our own leader registration node.
+          // We do this by using a multi and ensuring the parent znode of the leader registration node
+          // matches the version we expect - there is a setData call that increments the parent's znode
+          // version whenever a leader registers.
+          log.info("Removing leader registration node on cancel: {} {}", leaderPath, leaderZkNodeParentVersion);
+          List<Op> ops = new ArrayList<>(2);
+          ops.add(Op.check(new Path(leaderPath).getParent().toString(), leaderZkNodeParentVersion));
+          ops.add(Op.delete(leaderPath, -1));
+          zkClient.multi(ops, true);
+        } catch (KeeperException.NoNodeException nne) {
+          // no problem
+          log.info("No leader registration node found to remove: {}", leaderPath);
+        } catch (KeeperException.BadVersionException bve) {
+          log.info("Cannot remove leader registration node because the current registered node is not ours: {}", leaderPath);
+          // no problem
+        } catch (InterruptedException e) {
+          throw e;
+        } catch (Exception e) {
+          SolrException.log(log, e);
+        }
+        leaderZkNodeParentVersion = null;
+      } else {
+        log.info("No version found for ephemeral leader parent node, won't remove previous leader registration.");
       }
-      leaderZkNodeParentVersion = null;
-    } else {
-      log.info("No version found for ephemeral leader parent node, won't remove previous leader registration.");
     }
   }
   
@@ -179,31 +184,31 @@ class ShardLeaderElectionContextBase extends ElectionContext {
         
         @Override
         public void execute() throws InterruptedException, KeeperException {
-          log.info("Creating leader registration node {} after winning as {}", leaderPath, leaderSeqPath);
-          List<Op> ops = new ArrayList<>(2);
-          
-          // We use a multi operation to get the parent nodes version, which will
-          // be used to make sure we only remove our own leader registration node.
-          // The setData call used to get the parent version is also the trigger to
-          // increment the version. We also do a sanity check that our leaderSeqPath exists.
-          
-          ops.add(Op.check(leaderSeqPath, -1));
-          ops.add(Op.create(leaderPath, json, zkClient.getZkACLProvider().getACLsToAdd(leaderPath), CreateMode.EPHEMERAL));
-          // we set the json on the parent too for back compat, see SOLR-7844
-          ops.add(Op.setData(parent, json, -1));
-          List<OpResult> results;
-          
-          results = zkClient.multi(ops, true);
-          
-          for (OpResult result : results) {
-            if (result.getType() == ZooDefs.OpCode.setData) {
-              SetDataResult dresult = (SetDataResult) result;
-              Stat stat = dresult.getStat();
-              leaderZkNodeParentVersion = stat.getVersion();
-              return;
+          synchronized (lock) {
+            log.info("Creating leader registration node {} after winning as {}", leaderPath, leaderSeqPath);
+            List<Op> ops = new ArrayList<>(2);
+
+            // We use a multi operation to get the parent nodes version, which will
+            // be used to make sure we only remove our own leader registration node.
+            // The setData call used to get the parent version is also the trigger to
+            // increment the version. We also do a sanity check that our leaderSeqPath exists.
+
+            ops.add(Op.check(leaderSeqPath, -1));
+            ops.add(Op.create(leaderPath, Utils.toJSON(leaderProps), zkClient.getZkACLProvider().getACLsToAdd(leaderPath), CreateMode.EPHEMERAL));
+            ops.add(Op.setData(parent, null, -1));
+            List<OpResult> results;
+
+            results = zkClient.multi(ops, true);
+            for (OpResult result : results) {
+              if (result.getType() == ZooDefs.OpCode.setData) {
+                SetDataResult dresult = (SetDataResult) result;
+                Stat stat = dresult.getStat();
+                leaderZkNodeParentVersion = stat.getVersion();
+                return;
+              }
             }
+            assert leaderZkNodeParentVersion != null;
           }
-          assert leaderZkNodeParentVersion != null;
         }
       });
     } catch (Throwable t) {
@@ -226,7 +231,13 @@ class ShardLeaderElectionContextBase extends ElectionContext {
 
   public LeaderElector getLeaderElector() {
     return leaderElector;
-  }  
+  }
+
+  Integer getLeaderZkNodeParentVersion() {
+    synchronized (lock) {
+      return leaderZkNodeParentVersion;
+    }
+  }
 }
 
 // add core container and stop passing core around...

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d08169e6/solr/core/src/java/org/apache/solr/cloud/ZkController.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/ZkController.java b/solr/core/src/java/org/apache/solr/cloud/ZkController.java
index e7b5f50..cc8f7a2 100644
--- a/solr/core/src/java/org/apache/solr/cloud/ZkController.java
+++ b/solr/core/src/java/org/apache/solr/cloud/ZkController.java
@@ -2194,7 +2194,7 @@ public final class ZkController {
     // we use this version and multi to ensure *only* the current zk registered leader
     // for a shard can put a replica into LIR
     
-    Integer leaderZkNodeParentVersion = ((ShardLeaderElectionContextBase)context).leaderZkNodeParentVersion;
+    Integer leaderZkNodeParentVersion = ((ShardLeaderElectionContextBase)context).getLeaderZkNodeParentVersion();
     
     // TODO: should we do this optimistically to avoid races?
     if (zkClient.exists(znodePath, retryOnConnLoss)) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d08169e6/solr/core/src/test/org/apache/solr/cloud/OverseerTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/OverseerTest.java b/solr/core/src/test/org/apache/solr/cloud/OverseerTest.java
index 108ccbe..98b04b4 100644
--- a/solr/core/src/test/org/apache/solr/cloud/OverseerTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/OverseerTest.java
@@ -121,6 +121,13 @@ public class OverseerTest extends SolrTestCaseJ4 {
     }
 
     public void close() {
+      for (ElectionContext ec : electionContext.values()) {
+        try {
+          ec.cancelElection();
+        } catch (Exception e) {
+          log.warn(String.format("Error cancelling election for %s", ec.id), e);
+        }
+      }
       deleteNode(ZkStateReader.LIVE_NODES_ZKNODE + "/" + nodeName);
       zkClient.close();
     }


[4/4] lucene-solr:branch_5x: SOLR-8837: Fix duplicate election node detection

Posted by an...@apache.org.
SOLR-8837: Fix duplicate election node detection


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

Branch: refs/heads/branch_5x
Commit: 0b7b28a04157f4a74020ea79a29ad74c280f137f
Parents: 4b42c77
Author: Alan Woodward <ro...@apache.org>
Authored: Mon Mar 21 17:39:37 2016 +0000
Committer: Anshum Gupta <an...@apache.org>
Committed: Thu Apr 21 12:59:36 2016 -0700

----------------------------------------------------------------------
 solr/CHANGES.txt                                            | 5 +++--
 solr/core/src/java/org/apache/solr/cloud/LeaderElector.java | 4 +---
 2 files changed, 4 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/0b7b28a0/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 7ae5757..3dd5b15 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -91,8 +91,9 @@ Bug Fixes
 
 * SOLR-8656: PeerSync should use same nUpdates everywhere. (Ramsey Haddad via Mark Miller)
 
-* SOLR-8697: Scope ZK election nodes by session to prevent elections from interfering with each other
-  and other small LeaderElector improvements. (Scott Blum via Mark Miller)
+* SOLR-8697, SOLR-8837: Scope ZK election nodes by session to prevent elections from interfering with each other
+  and other small LeaderElector improvements. (Scott Blum via Mark Miller, Alan
+  Woodward)
 
 ======================= 5.5.0 =======================
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/0b7b28a0/solr/core/src/java/org/apache/solr/cloud/LeaderElector.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/LeaderElector.java b/solr/core/src/java/org/apache/solr/cloud/LeaderElector.java
index aa1ace2..74b3ff1 100644
--- a/solr/core/src/java/org/apache/solr/cloud/LeaderElector.java
+++ b/solr/core/src/java/org/apache/solr/cloud/LeaderElector.java
@@ -18,7 +18,6 @@ package org.apache.solr.cloud;
 
 import java.io.IOException;
 import java.lang.invoke.MethodHandles;
-import java.util.ArrayList;
 import java.util.Collections;
 import java.util.Comparator;
 import java.util.Iterator;
@@ -27,7 +26,6 @@ import java.util.Map;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
-import org.apache.commons.lang.StringUtils;
 import org.apache.solr.cloud.ZkController.ContextKey;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.cloud.SolrZkClient;
@@ -116,7 +114,7 @@ public  class LeaderElector {
 
     // If any double-registrations exist for me, remove all but this latest one!
     // TODO: can we even get into this state?
-    String prefix = zkClient.getSolrZooKeeper().getSessionId() + "-" + context.id;
+    String prefix = zkClient.getSolrZooKeeper().getSessionId() + "-" + context.id + "-";
     Iterator<String> it = seqs.iterator();
     while (it.hasNext()) {
       String elec = it.next();