SOLR-13410: Designated overseer wasn't able to rejoin election queue upon restart

This commit is contained in:
Ishan Chattopadhyaya 2019-05-04 00:29:07 +05:30
parent cdd130ccb6
commit 1882a17115
3 changed files with 62 additions and 16 deletions

View File

@ -249,6 +249,9 @@ Bug Fixes
* SOLR-12291: prematurely reporting not yet finished async Collections API call as completed
when collection's replicas are collocated at least at one node (Varun Thacker, Mikhail Khludnev)
* SOLR-13410: Designated overseer wasn't able to rejoin election queue upon restart (Ishan Chattopadhyaya,
Kesharee Nandan Vishwakarma)
Improvements
----------------------

View File

@ -2138,13 +2138,9 @@ public class ZkController implements Closeable {
public void rejoinOverseerElection(String electionNode, boolean joinAtHead) {
try {
if (electionNode != null) {
//this call is from inside the JVM . not from CoreAdminHandler
if (overseerElector.getContext() == null || overseerElector.getContext().leaderSeqPath == null) {
overseerElector.retryElection(new OverseerElectionContext(zkClient,
overseer, getNodeName()), joinAtHead);
return;
}
if (!overseerElector.getContext().leaderSeqPath.endsWith(electionNode)) {
// Check whether we came to this node by mistake
if ( overseerElector.getContext() != null && overseerElector.getContext().leaderSeqPath == null
&& !overseerElector.getContext().leaderSeqPath.endsWith(electionNode)) {
log.warn("Asked to rejoin with wrong election node : {}, current node is {}", electionNode, overseerElector.getContext().leaderSeqPath);
//however delete it . This is possible when the last attempt at deleting the election node failed.
if (electionNode.startsWith(getNodeName())) {
@ -2158,6 +2154,10 @@ public class ZkController implements Closeable {
log.warn("Old election node exists , could not be removed ", e);
}
}
} else { // We're in the right place, now attempt to rejoin
overseerElector.retryElection(new OverseerElectionContext(zkClient,
overseer, getNodeName()), joinAtHead);
return;
}
} else {
overseerElector.retryElection(overseerElector.getContext(), joinAtHead);

View File

@ -46,25 +46,33 @@ public class OverseerRolesTest extends SolrCloudTestCase {
@BeforeClass
public static void setupCluster() throws Exception {
configureCluster(4)
configureCluster(6)
.addConfig("conf", configset("cloud-minimal"))
.configure();
}
private void waitForNewOverseer(int seconds, Predicate<String> state) throws Exception {
private void waitForNewOverseer(int seconds, Predicate<String> state, boolean failOnIntermediateTransition) throws Exception {
TimeOut timeout = new TimeOut(seconds, TimeUnit.SECONDS, TimeSource.NANO_TIME);
String current = null;
while (timeout.hasTimedOut() == false) {
String prev = current;
current = OverseerCollectionConfigSetProcessor.getLeaderNode(zkClient());
if (state.test(current))
return;
else if (failOnIntermediateTransition) {
if (prev != null && !current.equals(prev)) {
fail ("There was an intermediate transition, previous: "+prev+", intermediate transition: "+current);
}
}
Thread.sleep(100);
}
fail("Timed out waiting for overseer state change");
fail("Timed out waiting for overseer state change. The current overseer is: "+current);
}
private void waitForNewOverseer(int seconds, String expected) throws Exception {
waitForNewOverseer(seconds, s -> Objects.equals(s, expected));
private void waitForNewOverseer(int seconds, String expected, boolean failOnIntermediateTransition) throws Exception {
log.info("Expecting node: "+expected);
waitForNewOverseer(seconds, s -> Objects.equals(s, expected), failOnIntermediateTransition);
}
private JettySolrRunner getOverseerJetty() throws Exception {
@ -93,6 +101,10 @@ public class OverseerRolesTest extends SolrCloudTestCase {
logOverseerState();
List<String> nodes = OverseerCollectionConfigSetProcessor.getSortedOverseerNodeNames(zkClient());
// Remove the OVERSEER role, in case it was already assigned by another test in this suite
for (String node: nodes) {
CollectionAdminRequest.removeRole(node, "overseer").process(cluster.getSolrClient());
}
String overseer1 = OverseerCollectionConfigSetProcessor.getLeaderNode(zkClient());
nodes.remove(overseer1);
@ -102,7 +114,7 @@ public class OverseerRolesTest extends SolrCloudTestCase {
CollectionAdminRequest.addRole(overseer2, "overseer").process(cluster.getSolrClient());
waitForNewOverseer(15, overseer2);
waitForNewOverseer(15, overseer2, false);
//add another node as overseer
nodes.remove(overseer2);
@ -117,7 +129,7 @@ public class OverseerRolesTest extends SolrCloudTestCase {
logOverseerState();
leaderJetty.stop();
waitForNewOverseer(10, overseer3);
waitForNewOverseer(10, overseer3, false);
// add another node as overseer
nodes.remove(overseer3);
@ -129,7 +141,7 @@ public class OverseerRolesTest extends SolrCloudTestCase {
// remove the overseer role from the current overseer
CollectionAdminRequest.removeRole(overseer3, "overseer").process(cluster.getSolrClient());
waitForNewOverseer(15, overseer4);
waitForNewOverseer(15, overseer4, false);
// Add it back again - we now have two delegates, 4 and 3
CollectionAdminRequest.addRole(overseer3, "overseer").process(cluster.getSolrClient());
@ -142,7 +154,7 @@ public class OverseerRolesTest extends SolrCloudTestCase {
.offer(Utils.toJSON(new ZkNodeProps(Overseer.QUEUE_OPERATION, OverseerAction.QUIT.toLower(),
"id", leaderId)));
waitForNewOverseer(15, s -> Objects.equals(leader, s) == false);
waitForNewOverseer(15, s -> Objects.equals(leader, s) == false, false);
Thread.sleep(1000);
@ -150,6 +162,37 @@ public class OverseerRolesTest extends SolrCloudTestCase {
assertTrue("The old leader should have rejoined election",
OverseerCollectionConfigSetProcessor.getSortedOverseerNodeNames(zkClient()).contains(leader));
leaderJetty.start(); // starting this back, just for good measure
}
@Test
public void testDesignatedOverseerRestarts() throws Exception {
logOverseerState();
List<String> nodes = OverseerCollectionConfigSetProcessor.getSortedOverseerNodeNames(zkClient());
// Remove the OVERSEER role, in case it was already assigned by another test in this suite
for (String node: nodes) {
CollectionAdminRequest.removeRole(node, "overseer").process(cluster.getSolrClient());
}
String overseer1 = OverseerCollectionConfigSetProcessor.getLeaderNode(zkClient());
nodes.remove(overseer1);
// Setting overseer role to the current overseer
CollectionAdminRequest.addRole(overseer1, "overseer").process(cluster.getSolrClient());
waitForNewOverseer(15, overseer1, false);
// kill the current overseer, and check that the next node in the election queue assumes leadership
JettySolrRunner leaderJetty = getOverseerJetty();
logOverseerState();
leaderJetty.stop();
log.info("Killing designated overseer: "+overseer1);
waitForNewOverseer(10, nodes.get(0), false);
// after 5 seconds, bring back dead designated overseer and assert that it assumes leadership "right away",
// i.e. without any other node assuming leadership before this node becomes leader.
Thread.sleep(5);
logOverseerState();
log.info("Starting back the prioritized overseer..");
leaderJetty.start();
waitForNewOverseer(1500000, overseer1, true); // assert that there is just a single leadership transition
}
}