You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by da...@apache.org on 2018/03/07 02:10:03 UTC

lucene-solr:master: SOLR-12011: FORCELEADER should also remove recovering flag of a replica so it can win the election

Repository: lucene-solr
Updated Branches:
  refs/heads/master 28de8218f -> 3c153ccd0


SOLR-12011: FORCELEADER should also remove recovering flag of a replica so it can win the election


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

Branch: refs/heads/master
Commit: 3c153ccd0e649cec361846a41cd518c7fc64d902
Parents: 28de821
Author: Cao Manh Dat <da...@apache.org>
Authored: Wed Mar 7 09:09:52 2018 +0700
Committer: Cao Manh Dat <da...@apache.org>
Committed: Wed Mar 7 09:09:52 2018 +0700

----------------------------------------------------------------------
 .../java/org/apache/solr/cloud/ZkShardTerms.java |  3 ++-
 .../solr/handler/admin/CollectionsHandler.java   | 12 +++---------
 .../org/apache/solr/cloud/ZkShardTermsTest.java  | 19 +++++++++++++++++++
 3 files changed, 24 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3c153ccd/solr/core/src/java/org/apache/solr/cloud/ZkShardTerms.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/ZkShardTerms.java b/solr/core/src/java/org/apache/solr/cloud/ZkShardTerms.java
index 50b424c..ca79982 100644
--- a/solr/core/src/java/org/apache/solr/cloud/ZkShardTerms.java
+++ b/solr/core/src/java/org/apache/solr/cloud/ZkShardTerms.java
@@ -194,7 +194,7 @@ public class ZkShardTerms implements AutoCloseable{
   }
 
   /**
-   * Set a replica's term equals to leader's term.
+   * Set a replica's term equals to leader's term, and remove recovering flag of a replica.
    * This call should only be used by {@link org.apache.solr.common.params.CollectionParams.CollectionAction#FORCELEADER}
    * @param coreNodeName of the replica
    */
@@ -554,6 +554,7 @@ public class ZkShardTerms implements AutoCloseable{
 
       HashMap<String, Long> newValues = new HashMap<>(values);
       newValues.put(coreNodeName, maxTerm);
+      newValues.remove(coreNodeName+"_recovering");
       return new Terms(newValues, version);
     }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3c153ccd/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java b/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java
index f4855eb..13f0f98 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java
@@ -1148,16 +1148,10 @@ public class CollectionsHandler extends RequestHandlerBase implements Permission
           .noneMatch(rep -> zkShardTerms.registered(rep.getName()) && zkShardTerms.canBecomeLeader(rep.getName()));
       // we won't increase replica's terms if exist a live replica with term equals to leader
       if (shouldIncreaseReplicaTerms) {
-        OptionalLong optionalMaxTerm = liveReplicas.stream()
+        //TODO only increase terms of replicas less out-of-sync
+        liveReplicas.stream()
             .filter(rep -> zkShardTerms.registered(rep.getName()))
-            .mapToLong(rep -> zkShardTerms.getTerm(rep.getName()))
-            .max();
-        // increase terms of replicas less out-of-sync
-        if (optionalMaxTerm.isPresent()) {
-          liveReplicas.stream()
-              .filter(rep -> zkShardTerms.getTerm(rep.getName()) == optionalMaxTerm.getAsLong())
-              .forEach(rep -> zkShardTerms.setTermEqualsToLeader(rep.getName()));
-        }
+            .forEach(rep -> zkShardTerms.setTermEqualsToLeader(rep.getName()));
       }
 
       // Wait till we have an active leader

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3c153ccd/solr/core/src/test/org/apache/solr/cloud/ZkShardTermsTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/ZkShardTermsTest.java b/solr/core/src/test/org/apache/solr/cloud/ZkShardTermsTest.java
index d557b29..c7f1cdb 100644
--- a/solr/core/src/test/org/apache/solr/cloud/ZkShardTermsTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/ZkShardTermsTest.java
@@ -229,6 +229,25 @@ public class ZkShardTermsTest extends SolrCloudTestCase {
     replicaTerms.close();
   }
 
+  public void testSetTermEqualsToLeader() throws InterruptedException {
+    String collection = "setTermEqualsToLeader";
+    ZkShardTerms leaderTerms = new ZkShardTerms(collection, "shard1", cluster.getZkClient());
+    ZkShardTerms replicaTerms = new ZkShardTerms(collection, "shard1", cluster.getZkClient());
+    leaderTerms.registerTerm("leader");
+    replicaTerms.registerTerm("replica");
+
+    leaderTerms.ensureTermsIsHigher("leader", Collections.singleton("replica"));
+    waitFor(false, () -> replicaTerms.canBecomeLeader("replica"));
+    waitFor(true, () -> leaderTerms.skipSendingUpdatesTo("replica"));
+
+    replicaTerms.setTermEqualsToLeader("replica");
+    waitFor(true, () -> replicaTerms.canBecomeLeader("replica"));
+    waitFor(false, () -> leaderTerms.skipSendingUpdatesTo("replica"));
+
+    leaderTerms.close();
+    replicaTerms.close();
+  }
+
   private <T> void waitFor(T expected, Supplier<T> supplier) throws InterruptedException {
     TimeOut timeOut = new TimeOut(10, TimeUnit.SECONDS, new TimeSource.CurrentTimeSource());
     while (!timeOut.hasTimedOut()) {