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/04/12 05:28:17 UTC

lucene-solr:master: SOLR-12214: Leader may skip publish itself as ACTIVE even its last published state is DOWN

Repository: lucene-solr
Updated Branches:
  refs/heads/master 8d20fc575 -> 11d54b0cc


SOLR-12214: Leader may skip publish itself as ACTIVE even its last published state is DOWN


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

Branch: refs/heads/master
Commit: 11d54b0cc4fd1df567afe9f4690e0f5a8a55f1ab
Parents: 8d20fc5
Author: Cao Manh Dat <da...@apache.org>
Authored: Thu Apr 12 12:28:00 2018 +0700
Committer: Cao Manh Dat <da...@apache.org>
Committed: Thu Apr 12 12:28:00 2018 +0700

----------------------------------------------------------------------
 solr/CHANGES.txt                                 |  1 +
 .../org/apache/solr/cloud/ElectionContext.java   |  3 ++-
 .../org/apache/solr/cloud/RecoveryStrategy.java  |  2 +-
 .../solr/cloud/LeaderVoteWaitTimeoutTest.java    | 19 ++++++++++---------
 4 files changed, 14 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/11d54b0c/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 360d663..da6642a 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -142,6 +142,7 @@ Bug Fixes
 
 * SOLR-12190: Need to properly escape output in GraphMLResponseWriter. (yonik)
 
+* SOLR-12214: Leader may skip publish itself as ACTIVE when its last published state is DOWN (Cao Manh Dat)
  
 Optimizations
 ----------------------

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/11d54b0c/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 7d64717..ae5c452 100644
--- a/solr/core/src/java/org/apache/solr/cloud/ElectionContext.java
+++ b/solr/core/src/java/org/apache/solr/cloud/ElectionContext.java
@@ -580,7 +580,8 @@ final class ShardLeaderElectionContext extends ShardLeaderElectionContextBase {
         zkStateReader.forceUpdateCollection(collection);
         ClusterState clusterState = zkStateReader.getClusterState();
         Replica rep = getReplica(clusterState, collection, leaderProps.getStr(ZkStateReader.CORE_NODE_NAME_PROP));
-        if (rep != null && rep.getState() != Replica.State.ACTIVE) {
+        if (rep == null) return;
+        if (rep.getState() != Replica.State.ACTIVE || core.getCoreDescriptor().getCloudDescriptor().getLastPublished() != Replica.State.ACTIVE) {
           log.debug("We have become the leader after core registration but are not in an ACTIVE state - publishing ACTIVE");
           zkController.publish(core.getCoreDescriptor(), Replica.State.ACTIVE);
         }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/11d54b0c/solr/core/src/java/org/apache/solr/cloud/RecoveryStrategy.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/RecoveryStrategy.java b/solr/core/src/java/org/apache/solr/cloud/RecoveryStrategy.java
index 023b2e3..c8f5ae8 100644
--- a/solr/core/src/java/org/apache/solr/cloud/RecoveryStrategy.java
+++ b/solr/core/src/java/org/apache/solr/cloud/RecoveryStrategy.java
@@ -749,7 +749,7 @@ public class RecoveryStrategy implements Runnable, Closeable {
     while (true) {
       CloudDescriptor cloudDesc = coreDesc.getCloudDescriptor();
       DocCollection docCollection = zkStateReader.getClusterState().getCollection(cloudDesc.getCollectionName());
-      if (mayPutReplicaAsDown && numTried == 1 &&
+      if (!isClosed() && mayPutReplicaAsDown && numTried == 1 &&
           docCollection.getReplica(coreDesc.getCloudDescriptor().getCoreNodeName()).getState() == Replica.State.ACTIVE) {
         // this operation may take a long time, by putting replica into DOWN state, client won't query this replica
         zkController.publish(coreDesc, Replica.State.DOWN);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/11d54b0c/solr/core/src/test/org/apache/solr/cloud/LeaderVoteWaitTimeoutTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/LeaderVoteWaitTimeoutTest.java b/solr/core/src/test/org/apache/solr/cloud/LeaderVoteWaitTimeoutTest.java
index c9eb336..6d32c3a 100644
--- a/solr/core/src/test/org/apache/solr/cloud/LeaderVoteWaitTimeoutTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/LeaderVoteWaitTimeoutTest.java
@@ -90,7 +90,6 @@ public class LeaderVoteWaitTimeoutTest extends SolrCloudTestCase {
   }
 
   @Test
-  @BadApple(bugUrl="https://issues.apache.org/jira/browse/SOLR-12028") // 17-Mar-2018
   public void basicTest() throws Exception {
     final String collectionName = "basicTest";
     CollectionAdminRequest.createCollection(collectionName, 1, 1)
@@ -127,7 +126,6 @@ public class LeaderVoteWaitTimeoutTest extends SolrCloudTestCase {
   }
 
   @Test
-  @BadApple(bugUrl="https://issues.apache.org/jira/browse/SOLR-12028") // 17-Mar-2018
   public void testMostInSyncReplicasCanWinElection() throws Exception {
     final String collectionName = "collection1";
     CollectionAdminRequest.createCollection(collectionName, 1, 3)
@@ -187,15 +185,18 @@ public class LeaderVoteWaitTimeoutTest extends SolrCloudTestCase {
     proxies.get(cluster.getJettySolrRunner(2)).reopen();
     cluster.getJettySolrRunner(0).stop();
 
-    // even replica2 joined election at the end of the queue, but it is the one with highest term
-    waitForState("Timeout waiting for new leader", collectionName, new CollectionStatePredicate() {
-      @Override
-      public boolean matches(Set<String> liveNodes, DocCollection collectionState) {
+    try {
+      // even replica2 joined election at the end of the queue, but it is the one with highest term
+      waitForState("Timeout waiting for new leader", collectionName, (liveNodes, collectionState) -> {
         Replica newLeader = collectionState.getSlice("shard1").getLeader();
         return newLeader.getName().equals(replica2.getName());
-      }
-    });
-
+      });
+    } catch (Exception e) {
+      List<String> children = zkClient().getChildren("/collections/"+collectionName+"/leader_elect/shard1/election",
+          null, true);
+      LOG.info("{} election nodes:{}", collectionName, children);
+      throw e;
+    }
     cluster.getJettySolrRunner(0).start();
     proxies.get(cluster.getJettySolrRunner(0)).reopen();