Only turn to follower when term bumping on follower check (#36449)
Deals with a situation where a follower becomes disconnected from the leader, but only for such a short time where it becomes candidate and puts up a NO_MASTER_BLOCK, but then receives a follower check from the leader. If the leader does not notice the node disconnecting, it is important for the node not to be turned back into a follower but try and join the leader again. We still should prefer the node into a follower on a follower check when this follower check triggers a term bump as this can help during a leader election to quickly have a leader turn all other nodes into followers, even before the leader has had the chance to transfer a possibly very large cluster state. Closes #36428
This commit is contained in:
parent
d8e3d97a7d
commit
71cf6faaad
|
@ -206,8 +206,9 @@ public class Coordinator extends AbstractLifecycleComponent implements Discovery
|
|||
}
|
||||
}
|
||||
|
||||
private void onFollowerCheckRequest(FollowerCheckRequest followerCheckRequest) {
|
||||
void onFollowerCheckRequest(FollowerCheckRequest followerCheckRequest) {
|
||||
synchronized (mutex) {
|
||||
final long previousTerm = getCurrentTerm();
|
||||
ensureTermAtLeast(followerCheckRequest.getSender(), followerCheckRequest.getTerm());
|
||||
|
||||
if (getCurrentTerm() != followerCheckRequest.getTerm()) {
|
||||
|
@ -216,7 +217,9 @@ public class Coordinator extends AbstractLifecycleComponent implements Discovery
|
|||
+ getCurrentTerm() + "], rejecting " + followerCheckRequest);
|
||||
}
|
||||
|
||||
becomeFollower("onFollowerCheckRequest", followerCheckRequest.getSender());
|
||||
if (previousTerm != getCurrentTerm()) {
|
||||
becomeFollower("onFollowerCheckRequest", followerCheckRequest.getSender());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -95,6 +95,7 @@ import static org.elasticsearch.cluster.coordination.LeaderChecker.LEADER_CHECK_
|
|||
import static org.elasticsearch.cluster.coordination.LeaderChecker.LEADER_CHECK_RETRY_COUNT_SETTING;
|
||||
import static org.elasticsearch.cluster.coordination.LeaderChecker.LEADER_CHECK_TIMEOUT_SETTING;
|
||||
import static org.elasticsearch.cluster.coordination.Reconfigurator.CLUSTER_AUTO_SHRINK_VOTING_CONFIGURATION;
|
||||
import static org.elasticsearch.discovery.DiscoverySettings.NO_MASTER_BLOCK_ID;
|
||||
import static org.elasticsearch.discovery.PeerFinder.DISCOVERY_FIND_PEERS_INTERVAL_SETTING;
|
||||
import static org.elasticsearch.node.Node.NODE_NAME_SETTING;
|
||||
import static org.elasticsearch.transport.TransportService.HANDSHAKE_ACTION_NAME;
|
||||
|
@ -110,6 +111,7 @@ import static org.hamcrest.Matchers.is;
|
|||
import static org.hamcrest.Matchers.lessThan;
|
||||
import static org.hamcrest.Matchers.lessThanOrEqualTo;
|
||||
import static org.hamcrest.Matchers.not;
|
||||
import static org.hamcrest.Matchers.nullValue;
|
||||
import static org.hamcrest.Matchers.sameInstance;
|
||||
import static org.hamcrest.Matchers.startsWith;
|
||||
|
||||
|
@ -887,6 +889,31 @@ public class CoordinatorTests extends ESTestCase {
|
|||
postPublishStats.getIncompatibleClusterStateDiffReceivedCount());
|
||||
}
|
||||
|
||||
/**
|
||||
* Simulates a situation where a follower becomes disconnected from the leader, but only for such a short time where
|
||||
* it becomes candidate and puts up a NO_MASTER_BLOCK, but then receives a follower check from the leader. If the leader
|
||||
* does not notice the node disconnecting, it is important for the node not to be turned back into a follower but try
|
||||
* and join the leader again.
|
||||
*/
|
||||
public void testStayCandidateAfterReceivingFollowerCheckFromKnownMaster() {
|
||||
final Cluster cluster = new Cluster(2, false);
|
||||
cluster.runRandomly();
|
||||
cluster.stabilise();
|
||||
|
||||
final ClusterNode leader = cluster.getAnyLeader();
|
||||
final ClusterNode nonLeader = cluster.getAnyNodeExcept(leader);
|
||||
onNode(nonLeader.getLocalNode(), () -> {
|
||||
logger.debug("forcing {} to become candidate", nonLeader.getId());
|
||||
synchronized (nonLeader.coordinator.mutex) {
|
||||
nonLeader.coordinator.becomeCandidate("forced");
|
||||
}
|
||||
logger.debug("simulate follower check coming through from {} to {}", leader.getId(), nonLeader.getId());
|
||||
nonLeader.coordinator.onFollowerCheckRequest(new FollowersChecker.FollowerCheckRequest(leader.coordinator.getCurrentTerm(),
|
||||
leader.getLocalNode()));
|
||||
}).run();
|
||||
cluster.stabilise();
|
||||
}
|
||||
|
||||
private static long defaultMillis(Setting<TimeValue> setting) {
|
||||
return setting.get(Settings.EMPTY).millis() + Cluster.DEFAULT_DELAY_VARIABILITY;
|
||||
}
|
||||
|
@ -1176,8 +1203,15 @@ public class CoordinatorTests extends ESTestCase {
|
|||
assertTrue(nodeId + " is in the latest applied state on " + leaderId,
|
||||
leader.getLastAppliedClusterState().getNodes().nodeExists(nodeId));
|
||||
assertTrue(nodeId + " has been bootstrapped", clusterNode.coordinator.isInitialConfigurationSet());
|
||||
assertThat(nodeId + " has correct master", clusterNode.getLastAppliedClusterState().nodes().getMasterNode(),
|
||||
equalTo(leader.getLocalNode()));
|
||||
assertThat(nodeId + " has no NO_MASTER_BLOCK",
|
||||
clusterNode.getLastAppliedClusterState().blocks().hasGlobalBlock(NO_MASTER_BLOCK_ID), equalTo(false));
|
||||
} else {
|
||||
assertThat(nodeId + " is not following " + leaderId, clusterNode.coordinator.getMode(), is(CANDIDATE));
|
||||
assertThat(nodeId + " has no master", clusterNode.getLastAppliedClusterState().nodes().getMasterNode(), nullValue());
|
||||
assertThat(nodeId + " has NO_MASTER_BLOCK",
|
||||
clusterNode.getLastAppliedClusterState().blocks().hasGlobalBlock(NO_MASTER_BLOCK_ID), equalTo(true));
|
||||
assertFalse(nodeId + " is not in the applied state on " + leaderId,
|
||||
leader.getLastAppliedClusterState().getNodes().nodeExists(nodeId));
|
||||
}
|
||||
|
|
|
@ -286,7 +286,6 @@ public class ClusterDisruptionIT extends AbstractDisruptionTestCase {
|
|||
}
|
||||
|
||||
// simulate handling of sending shard failure during an isolation
|
||||
@AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/36428")
|
||||
public void testSendingShardFailure() throws Exception {
|
||||
List<String> nodes = startCluster(3);
|
||||
String masterNode = internalCluster().getMasterName();
|
||||
|
|
Loading…
Reference in New Issue