Skip to content

Commit d5139e0

Browse files
authored
Only bootstrap and elect node in current voting configuration (#37712)
Adapts bootstrapping and leader election to only trigger on nodes that are actually part of the voting configuration.
1 parent 4ec3a6d commit d5139e0

File tree

4 files changed

+87
-14
lines changed

4 files changed

+87
-14
lines changed

server/src/main/java/org/elasticsearch/cluster/coordination/ClusterBootstrapService.java

Lines changed: 8 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -97,7 +97,7 @@ public static boolean discoveryIsConfigured(Settings settings) {
9797

9898
void onFoundPeersUpdated() {
9999
final Set<DiscoveryNode> nodes = getDiscoveredNodes();
100-
if (transportService.getLocalNode().isMasterNode() && bootstrapRequirements.isEmpty() == false
100+
if (bootstrappingPermitted.get() && transportService.getLocalNode().isMasterNode() && bootstrapRequirements.isEmpty() == false
101101
&& isBootstrappedSupplier.getAsBoolean() == false && nodes.stream().noneMatch(Coordinator::isZen1Node)) {
102102

103103
final Tuple<Set<DiscoveryNode>,List<String>> requirementMatchingResult;
@@ -114,6 +114,13 @@ void onFoundPeersUpdated() {
114114
logger.trace("nodesMatchingRequirements={}, unsatisfiedRequirements={}, bootstrapRequirements={}",
115115
nodesMatchingRequirements, unsatisfiedRequirements, bootstrapRequirements);
116116

117+
if (nodesMatchingRequirements.contains(transportService.getLocalNode()) == false) {
118+
logger.info("skipping cluster bootstrapping as local node does not match bootstrap requirements: {}",
119+
bootstrapRequirements);
120+
bootstrappingPermitted.set(false);
121+
return;
122+
}
123+
117124
if (nodesMatchingRequirements.size() * 2 > bootstrapRequirements.size()) {
118125
startBootstrap(nodesMatchingRequirements, unsatisfiedRequirements);
119126
}

server/src/main/java/org/elasticsearch/cluster/coordination/Coordinator.java

Lines changed: 38 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -348,6 +348,12 @@ private void startElection() {
348348
// The preVoteCollector is only active while we are candidate, but it does not call this method with synchronisation, so we have
349349
// to check our mode again here.
350350
if (mode == Mode.CANDIDATE) {
351+
if (electionQuorumContainsLocalNode(getLastAcceptedState()) == false) {
352+
logger.trace("skip election as local node is not part of election quorum: {}",
353+
getLastAcceptedState().coordinationMetaData());
354+
return;
355+
}
356+
351357
final StartJoinRequest startJoinRequest
352358
= new StartJoinRequest(getLocalNode(), Math.max(getCurrentTerm(), maxTermSeen) + 1);
353359
logger.debug("starting election with {}", startJoinRequest);
@@ -360,6 +366,13 @@ private void startElection() {
360366
}
361367
}
362368

369+
private static boolean electionQuorumContainsLocalNode(ClusterState lastAcceptedState) {
370+
final String localNodeId = lastAcceptedState.nodes().getLocalNodeId();
371+
assert localNodeId != null;
372+
return lastAcceptedState.getLastCommittedConfiguration().getNodeIds().contains(localNodeId)
373+
|| lastAcceptedState.getLastAcceptedConfiguration().getNodeIds().contains(localNodeId);
374+
}
375+
363376
private Optional<Join> ensureTermAtLeast(DiscoveryNode sourceNode, long targetTerm) {
364377
assert Thread.holdsLock(mutex) : "Coordinator mutex not held";
365378
if (getCurrentTerm() < targetTerm) {
@@ -709,10 +722,24 @@ public boolean setInitialConfiguration(final VotingConfiguration votingConfigura
709722
return false;
710723
}
711724

725+
if (getLocalNode().isMasterNode() == false) {
726+
logger.debug("skip setting initial configuration as local node is not a master-eligible node");
727+
throw new CoordinationStateRejectedException(
728+
"this node is not master-eligible, but cluster bootstrapping can only happen on a master-eligible node");
729+
}
730+
731+
if (votingConfiguration.getNodeIds().contains(getLocalNode().getId()) == false) {
732+
logger.debug("skip setting initial configuration as local node is not part of initial configuration");
733+
throw new CoordinationStateRejectedException("local node is not part of initial configuration");
734+
}
735+
712736
final List<DiscoveryNode> knownNodes = new ArrayList<>();
713737
knownNodes.add(getLocalNode());
714738
peerFinder.getFoundPeers().forEach(knownNodes::add);
739+
715740
if (votingConfiguration.hasQuorum(knownNodes.stream().map(DiscoveryNode::getId).collect(Collectors.toList())) == false) {
741+
logger.debug("skip setting initial configuration as not enough nodes discovered to form a quorum in the " +
742+
"initial configuration [knownNodes={}, {}]", knownNodes, votingConfiguration);
716743
throw new CoordinationStateRejectedException("not enough nodes discovered to form a quorum in the initial configuration " +
717744
"[knownNodes=" + knownNodes + ", " + votingConfiguration + "]");
718745
}
@@ -729,6 +756,8 @@ public boolean setInitialConfiguration(final VotingConfiguration votingConfigura
729756
metaDataBuilder.coordinationMetaData(coordinationMetaData);
730757

731758
coordinationState.get().setInitialState(ClusterState.builder(currentState).metaData(metaDataBuilder).build());
759+
assert electionQuorumContainsLocalNode(getLastAcceptedState()) :
760+
"initial state does not have local node in its election quorum: " + getLastAcceptedState().coordinationMetaData();
732761
preVoteCollector.update(getPreVoteResponse(), null); // pick up the change to last-accepted version
733762
startElectionScheduler();
734763
return true;
@@ -1022,12 +1051,20 @@ private void startElectionScheduler() {
10221051
public void run() {
10231052
synchronized (mutex) {
10241053
if (mode == Mode.CANDIDATE) {
1054+
final ClusterState lastAcceptedState = coordinationState.get().getLastAcceptedState();
1055+
1056+
if (electionQuorumContainsLocalNode(lastAcceptedState) == false) {
1057+
logger.trace("skip prevoting as local node is not part of election quorum: {}",
1058+
lastAcceptedState.coordinationMetaData());
1059+
return;
1060+
}
1061+
10251062
if (prevotingRound != null) {
10261063
prevotingRound.close();
10271064
}
1028-
final ClusterState lastAcceptedState = coordinationState.get().getLastAcceptedState();
10291065
final List<DiscoveryNode> discoveredNodes
10301066
= getDiscoveredNodes().stream().filter(n -> isZen1Node(n) == false).collect(Collectors.toList());
1067+
10311068
prevotingRound = preVoteCollector.start(lastAcceptedState, discoveredNodes);
10321069
}
10331070
}

server/src/test/java/org/elasticsearch/cluster/coordination/ClusterBootstrapServiceTests.java

Lines changed: 12 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -328,6 +328,18 @@ public void testDoesNotBootstrapsOnNonMasterNode() {
328328
deterministicTaskQueue.runAllTasks();
329329
}
330330

331+
public void testDoesNotBootstrapsIfLocalNodeNotInInitialMasterNodes() {
332+
ClusterBootstrapService clusterBootstrapService = new ClusterBootstrapService(Settings.builder().putList(
333+
INITIAL_MASTER_NODES_SETTING.getKey(), otherNode1.getName(), otherNode2.getName()).build(),
334+
transportService, () ->
335+
Stream.of(localNode, otherNode1, otherNode2).collect(Collectors.toList()), () -> false, vc -> {
336+
throw new AssertionError("should not be called");
337+
});
338+
transportService.start();
339+
clusterBootstrapService.onFoundPeersUpdated();
340+
deterministicTaskQueue.runAllTasks();
341+
}
342+
331343
public void testDoesNotBootstrapsIfNotConfigured() {
332344
ClusterBootstrapService clusterBootstrapService = new ClusterBootstrapService(
333345
Settings.builder().putList(INITIAL_MASTER_NODES_SETTING.getKey()).build(), transportService,

server/src/test/java/org/elasticsearch/cluster/coordination/CoordinatorTests.java

Lines changed: 29 additions & 12 deletions
Original file line numberDiff line numberDiff line change
@@ -19,7 +19,6 @@
1919
package org.elasticsearch.cluster.coordination;
2020

2121
import com.carrotsearch.randomizedtesting.RandomizedContext;
22-
2322
import org.apache.logging.log4j.CloseableThreadContext;
2423
import org.apache.logging.log4j.LogManager;
2524
import org.apache.logging.log4j.Logger;
@@ -53,6 +52,7 @@
5352
import org.elasticsearch.common.settings.Settings.Builder;
5453
import org.elasticsearch.common.transport.TransportAddress;
5554
import org.elasticsearch.common.unit.TimeValue;
55+
import org.elasticsearch.common.util.set.Sets;
5656
import org.elasticsearch.discovery.zen.PublishClusterStateStats;
5757
import org.elasticsearch.discovery.zen.UnicastHostsProvider.HostsResolver;
5858
import org.elasticsearch.env.NodeEnvironment;
@@ -93,10 +93,10 @@
9393
import static org.elasticsearch.cluster.coordination.CoordinationStateTests.clusterState;
9494
import static org.elasticsearch.cluster.coordination.CoordinationStateTests.setValue;
9595
import static org.elasticsearch.cluster.coordination.CoordinationStateTests.value;
96-
import static org.elasticsearch.cluster.coordination.Coordinator.PUBLISH_TIMEOUT_SETTING;
9796
import static org.elasticsearch.cluster.coordination.Coordinator.Mode.CANDIDATE;
9897
import static org.elasticsearch.cluster.coordination.Coordinator.Mode.FOLLOWER;
9998
import static org.elasticsearch.cluster.coordination.Coordinator.Mode.LEADER;
99+
import static org.elasticsearch.cluster.coordination.Coordinator.PUBLISH_TIMEOUT_SETTING;
100100
import static org.elasticsearch.cluster.coordination.CoordinatorTests.Cluster.DEFAULT_DELAY_VARIABILITY;
101101
import static org.elasticsearch.cluster.coordination.ElectionSchedulerFactory.ELECTION_BACK_OFF_TIME_SETTING;
102102
import static org.elasticsearch.cluster.coordination.ElectionSchedulerFactory.ELECTION_DURATION_SETTING;
@@ -117,7 +117,6 @@
117117
import static org.elasticsearch.transport.TransportService.NOOP_TRANSPORT_INTERCEPTOR;
118118
import static org.hamcrest.Matchers.containsString;
119119
import static org.hamcrest.Matchers.empty;
120-
import static org.hamcrest.Matchers.endsWith;
121120
import static org.hamcrest.Matchers.equalTo;
122121
import static org.hamcrest.Matchers.greaterThan;
123122
import static org.hamcrest.Matchers.greaterThanOrEqualTo;
@@ -745,7 +744,7 @@ public void testSettingInitialConfigurationTriggersElection() {
745744
assertThat(nodeId + " should have found all peers", foundPeers, hasSize(cluster.size()));
746745
}
747746

748-
final ClusterNode bootstrapNode = cluster.getAnyNode();
747+
final ClusterNode bootstrapNode = cluster.getAnyBootstrappableNode();
749748
bootstrapNode.applyInitialConfiguration();
750749
assertTrue(bootstrapNode.getId() + " has been bootstrapped", bootstrapNode.coordinator.isInitialConfigurationSet());
751750

@@ -775,20 +774,30 @@ public void testCannotSetInitialConfigurationTwice() {
775774
public void testCannotSetInitialConfigurationWithoutQuorum() {
776775
final Cluster cluster = new Cluster(randomIntBetween(1, 5));
777776
final Coordinator coordinator = cluster.getAnyNode().coordinator;
778-
final VotingConfiguration unknownNodeConfiguration = new VotingConfiguration(Collections.singleton("unknown-node"));
777+
final VotingConfiguration unknownNodeConfiguration = new VotingConfiguration(
778+
Sets.newHashSet(coordinator.getLocalNode().getId(), "unknown-node"));
779779
final String exceptionMessage = expectThrows(CoordinationStateRejectedException.class,
780780
() -> coordinator.setInitialConfiguration(unknownNodeConfiguration)).getMessage();
781781
assertThat(exceptionMessage,
782782
startsWith("not enough nodes discovered to form a quorum in the initial configuration [knownNodes=["));
783-
assertThat(exceptionMessage,
784-
endsWith("], VotingConfiguration{unknown-node}]"));
783+
assertThat(exceptionMessage, containsString("unknown-node"));
785784
assertThat(exceptionMessage, containsString(coordinator.getLocalNode().toString()));
786785

787786
// This is VERY BAD: setting a _different_ initial configuration. Yet it works if the first attempt will never be a quorum.
788787
assertTrue(coordinator.setInitialConfiguration(new VotingConfiguration(Collections.singleton(coordinator.getLocalNode().getId()))));
789788
cluster.stabilise();
790789
}
791790

791+
public void testCannotSetInitialConfigurationWithoutLocalNode() {
792+
final Cluster cluster = new Cluster(randomIntBetween(1, 5));
793+
final Coordinator coordinator = cluster.getAnyNode().coordinator;
794+
final VotingConfiguration unknownNodeConfiguration = new VotingConfiguration(Sets.newHashSet("unknown-node"));
795+
final String exceptionMessage = expectThrows(CoordinationStateRejectedException.class,
796+
() -> coordinator.setInitialConfiguration(unknownNodeConfiguration)).getMessage();
797+
assertThat(exceptionMessage,
798+
equalTo("local node is not part of initial configuration"));
799+
}
800+
792801
public void testDiffBasedPublishing() {
793802
final Cluster cluster = new Cluster(randomIntBetween(1, 5));
794803
cluster.runRandomly();
@@ -1331,7 +1340,7 @@ void bootstrapIfNecessary() {
13311340
assertThat("setting initial configuration may fail with disconnected nodes", disconnectedNodes, empty());
13321341
assertThat("setting initial configuration may fail with blackholed nodes", blackholedNodes, empty());
13331342
runFor(defaultMillis(DISCOVERY_FIND_PEERS_INTERVAL_SETTING) * 2, "discovery prior to setting initial configuration");
1334-
final ClusterNode bootstrapNode = getAnyMasterEligibleNode();
1343+
final ClusterNode bootstrapNode = getAnyBootstrappableNode();
13351344
bootstrapNode.applyInitialConfiguration();
13361345
} else {
13371346
logger.info("setting initial configuration not required");
@@ -1402,8 +1411,10 @@ boolean nodeExists(DiscoveryNode node) {
14021411
return clusterNodes.stream().anyMatch(cn -> cn.getLocalNode().equals(node));
14031412
}
14041413

1405-
ClusterNode getAnyMasterEligibleNode() {
1406-
return randomFrom(clusterNodes.stream().filter(n -> n.getLocalNode().isMasterNode()).collect(Collectors.toList()));
1414+
ClusterNode getAnyBootstrappableNode() {
1415+
return randomFrom(clusterNodes.stream().filter(n -> n.getLocalNode().isMasterNode())
1416+
.filter(n -> initialConfiguration.getNodeIds().contains(n.getLocalNode().getId()))
1417+
.collect(Collectors.toList()));
14071418
}
14081419

14091420
ClusterNode getAnyNode() {
@@ -1737,8 +1748,14 @@ void applyInitialConfiguration() {
17371748
Stream.generate(() -> BOOTSTRAP_PLACEHOLDER_PREFIX + UUIDs.randomBase64UUID(random()))
17381749
.limit((Math.max(initialConfiguration.getNodeIds().size(), 2) - 1) / 2)
17391750
.forEach(nodeIdsWithPlaceholders::add);
1740-
final VotingConfiguration configurationWithPlaceholders = new VotingConfiguration(new HashSet<>(
1741-
randomSubsetOf(initialConfiguration.getNodeIds().size(), nodeIdsWithPlaceholders)));
1751+
final Set<String> nodeIds = new HashSet<>(
1752+
randomSubsetOf(initialConfiguration.getNodeIds().size(), nodeIdsWithPlaceholders));
1753+
// initial configuration should not have a place holder for local node
1754+
if (initialConfiguration.getNodeIds().contains(localNode.getId()) && nodeIds.contains(localNode.getId()) == false) {
1755+
nodeIds.remove(nodeIds.iterator().next());
1756+
nodeIds.add(localNode.getId());
1757+
}
1758+
final VotingConfiguration configurationWithPlaceholders = new VotingConfiguration(nodeIds);
17421759
try {
17431760
coordinator.setInitialConfiguration(configurationWithPlaceholders);
17441761
logger.info("successfully set initial configuration to {}", configurationWithPlaceholders);

0 commit comments

Comments
 (0)