Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Step down as master when configured out of voting configuration #37802

Merged
merged 13 commits into from
Jan 29, 2019
Original file line number Diff line number Diff line change
Expand Up @@ -796,7 +796,7 @@ ClusterState improveConfiguration(ClusterState clusterState) {
.filter(this::hasJoinVoteFrom).filter(discoveryNode -> isZen1Node(discoveryNode) == false).collect(Collectors.toSet());
final VotingConfiguration newConfig = reconfigurator.reconfigure(liveNodes,
clusterState.getVotingConfigExclusions().stream().map(VotingConfigExclusion::getNodeId).collect(Collectors.toSet()),
clusterState.getLastAcceptedConfiguration());
getLocalNode(), clusterState.getLastAcceptedConfiguration());
if (newConfig.equals(clusterState.getLastAcceptedConfiguration()) == false) {
assert coordinationState.get().joinVotesHaveQuorumFor(newConfig);
return ClusterState.builder(clusterState).metaData(MetaData.builder(clusterState.metaData())
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,7 @@
import org.elasticsearch.common.util.set.Sets;

import java.util.Collection;
import java.util.Comparator;
import java.util.Set;
import java.util.TreeSet;
import java.util.stream.Collectors;
Expand Down Expand Up @@ -90,12 +91,16 @@ public String toString() {
* @param retiredNodeIds Nodes that are leaving the cluster and which should not appear in the configuration if possible. Nodes that are
* retired and not in the current configuration will never appear in the resulting configuration; this is useful
* for shifting the vote in a 2-node cluster so one of the nodes can be restarted without harming availability.
* @param currentMaster The current master. Unless retired, we prefer to keep the current master in the config.
* @param currentConfig The current configuration. As far as possible, we prefer to keep the current config as-is.
* @return An optimal configuration, or leave the current configuration unchanged if the optimal configuration has no live quorum.
*/
public VotingConfiguration reconfigure(Set<DiscoveryNode> liveNodes, Set<String> retiredNodeIds, VotingConfiguration currentConfig) {
public VotingConfiguration reconfigure(Set<DiscoveryNode> liveNodes, Set<String> retiredNodeIds, DiscoveryNode currentMaster,
VotingConfiguration currentConfig) {
assert liveNodes.stream().noneMatch(Coordinator::isZen1Node) : liveNodes;
logger.trace("{} reconfiguring {} based on liveNodes={}, retiredNodeIds={}", this, currentConfig, liveNodes, retiredNodeIds);
assert liveNodes.contains(currentMaster) : "liveNodes = " + liveNodes + " master = " + currentMaster;
logger.trace("{} reconfiguring {} based on liveNodes={}, retiredNodeIds={}, currentMaster={}",
this, currentConfig, liveNodes, retiredNodeIds, currentMaster);

/*
* There are three true/false properties of each node in play: live/non-live, retired/non-retired and in-config/not-in-config.
Expand All @@ -122,7 +127,7 @@ public VotingConfiguration reconfigure(Set<DiscoveryNode> liveNodes, Set<String>
final Set<String> nonRetiredInConfigNotLiveIds = new TreeSet<>(inConfigNotLiveIds);
nonRetiredInConfigNotLiveIds.removeAll(retiredNodeIds);

final Set<String> nonRetiredInConfigLiveIds = new TreeSet<>(liveInConfigIds);
final Set<String> nonRetiredInConfigLiveIds = masterFirstTreeSet(liveInConfigIds, currentMaster);
nonRetiredInConfigLiveIds.removeAll(retiredNodeIds);

final Set<String> nonRetiredLiveNotInConfigIds = Sets.sortedDifference(liveNodeIds, currentConfig.getNodeIds());
Expand Down Expand Up @@ -162,4 +167,12 @@ public VotingConfiguration reconfigure(Set<DiscoveryNode> liveNodes, Set<String>
return currentConfig;
}
}

private TreeSet<String> masterFirstTreeSet(Collection<? extends String> items, DiscoveryNode masterNode) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We already have a mechanism for giving nodes different priorities when picking the new configuration:

Stream.of(nonRetiredInConfigLiveIds, nonRetiredLiveNotInConfigIds, nonRetiredInConfigNotLiveIds)

I think I'd prefer having a singleton/empty set containing the master (if it is non-retired/retired respectively) and say

Stream.of(nonRetiredMaster, nonRetiredInConfigLiveIds, nonRetiredLiveNotInConfigIds, nonRetiredInConfigNotLiveIds)

I'd also be happy doing the whole thing with a single treeset.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I've pushed 4e5886c

final String masterNodeId = masterNode.getId();
final TreeSet<String> set = new TreeSet<>(Comparator.<String>comparingInt(s -> s.equals(masterNodeId) ? 0 : 1)
.thenComparing(Comparator.naturalOrder()));
set.addAll(items);
return set;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,7 @@
import org.junit.Before;

import java.util.Arrays;
import java.util.Comparator;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
Expand All @@ -52,6 +53,7 @@ public void testReconfigurationExamples() {

check(nodes("a"), conf("a"), true, conf("a"));
check(nodes("a", "b"), conf("a"), true, conf("a"));
check(nodes("a", "b"), conf("b"), true, conf("b"));
check(nodes("a", "b"), conf("a", "c"), true, conf("a"));
check(nodes("a", "b"), conf("a", "b"), true, conf("a"));
check(nodes("a", "b"), conf("a", "b", "e"), true, conf("a", "b", "e"));
Expand All @@ -64,6 +66,7 @@ public void testReconfigurationExamples() {
check(nodes("a", "b", "c", "d"), conf("a", "b", "e"), true, conf("a", "b", "c"));
check(nodes("a", "b", "c", "d", "e"), conf("a", "f", "g"), true, conf("a", "b", "c", "d", "e"));
check(nodes("a", "b", "c", "d"), conf("a", "b", "c", "d", "e"), true, conf("a", "b", "c"));
check(nodes("e", "a", "b", "c"), retired(), "e", conf("a", "b", "c", "d", "e"), true, conf("a", "b", "e"));
check(nodes("a", "b", "c"), conf("a", "b", "c", "d", "e"), true, conf("a", "b", "c"));

check(nodes("a"), conf("a"), false, conf("a"));
Expand Down Expand Up @@ -124,7 +127,8 @@ public void testAutoShrinking() {

final int quorumSize = Math.max(liveNodes.length / 2 + 1, initialVotingNodes.length < 3 ? 1 : 2);

final VotingConfiguration finalConfig = reconfigurator.reconfigure(liveNodesSet, emptySet(), initialConfig);
final VotingConfiguration finalConfig = reconfigurator.reconfigure(liveNodesSet, emptySet(),
randomFrom(liveNodesSet), initialConfig);

final String description = "reconfigure " + liveNodesSet + " from " + initialConfig + " yielded " + finalConfig;

Expand Down Expand Up @@ -152,7 +156,8 @@ public void testManualShrinking() {

final int quorumSize = Math.max(liveNodes.length, initialVotingNodes.length) / 2 + 1;

final VotingConfiguration finalConfig = reconfigurator.reconfigure(liveNodesSet, emptySet(), initialConfig);
final VotingConfiguration finalConfig = reconfigurator.reconfigure(liveNodesSet, emptySet(), randomFrom(liveNodesSet),
initialConfig);

final String description = "reconfigure " + liveNodesSet + " from " + initialConfig + " yielded " + finalConfig;

Expand Down Expand Up @@ -187,13 +192,20 @@ private void check(Set<DiscoveryNode> liveNodes, VotingConfiguration config, boo

private void check(Set<DiscoveryNode> liveNodes, Set<String> retired, VotingConfiguration config,
boolean autoShrinkVotingConfiguration, VotingConfiguration expectedConfig) {
final DiscoveryNode master = liveNodes.stream().sorted(Comparator.comparing(DiscoveryNode::getId)).findFirst().get();
check(liveNodes, retired, master.getId(), config, autoShrinkVotingConfiguration, expectedConfig);
}

private void check(Set<DiscoveryNode> liveNodes, Set<String> retired, String masterId, VotingConfiguration config,
boolean autoShrinkVotingConfiguration, VotingConfiguration expectedConfig) {
final Reconfigurator reconfigurator = makeReconfigurator(Settings.builder()
.put(CLUSTER_AUTO_SHRINK_VOTING_CONFIGURATION.getKey(), autoShrinkVotingConfiguration)
.build());

final VotingConfiguration adaptedConfig = reconfigurator.reconfigure(liveNodes, retired, config);
assertEquals(new ParameterizedMessage("[liveNodes={}, retired={}, config={}, autoShrinkVotingConfiguration={}]",
liveNodes, retired, config, autoShrinkVotingConfiguration).getFormattedMessage(),
final DiscoveryNode master = liveNodes.stream().filter(n -> n.getId().equals(masterId)).findFirst().get();
final VotingConfiguration adaptedConfig = reconfigurator.reconfigure(liveNodes, retired, master, config);
assertEquals(new ParameterizedMessage("[liveNodes={}, retired={}, master={}, config={}, autoShrinkVotingConfiguration={}]",
liveNodes, retired, master, config, autoShrinkVotingConfiguration).getFormattedMessage(),
expectedConfig, adaptedConfig);
}

Expand All @@ -206,18 +218,24 @@ public void testDynamicSetting() {
final Reconfigurator reconfigurator = new Reconfigurator(Settings.EMPTY, clusterSettings);
final VotingConfiguration initialConfig = conf("a", "b", "c", "d", "e");

Set<DiscoveryNode> twoNodes = nodes("a", "b");
Set<DiscoveryNode> threeNodes = nodes("a", "b", "c");

// default is "true"
assertThat(reconfigurator.reconfigure(nodes("a", "b"), retired(), initialConfig), equalTo(conf("a", "b", "c")));
assertThat(reconfigurator.reconfigure(twoNodes, retired(), randomFrom(twoNodes), initialConfig), equalTo(conf("a", "b", "c")));

// update to "false"
clusterSettings.applySettings(Settings.builder().put(CLUSTER_AUTO_SHRINK_VOTING_CONFIGURATION.getKey(), "false").build());
assertThat(reconfigurator.reconfigure(nodes("a", "b"), retired(), initialConfig), sameInstance(initialConfig)); // no quorum
assertThat(reconfigurator.reconfigure(nodes("a", "b", "c"), retired(), initialConfig), equalTo(conf("a", "b", "c", "d", "e")));
assertThat(reconfigurator.reconfigure(nodes("a", "b", "c"), retired("d"), initialConfig), equalTo(conf("a", "b", "c", "e")));
assertThat(reconfigurator.reconfigure(twoNodes, retired(), randomFrom(twoNodes), initialConfig),
sameInstance(initialConfig)); // no quorum
assertThat(reconfigurator.reconfigure(threeNodes, retired(), randomFrom(threeNodes), initialConfig),
equalTo(conf("a", "b", "c", "d", "e")));
assertThat(reconfigurator.reconfigure(threeNodes, retired("d"), randomFrom(threeNodes), initialConfig),
equalTo(conf("a", "b", "c", "e")));

// explicitly set to "true"
clusterSettings.applySettings(Settings.builder().put(CLUSTER_AUTO_SHRINK_VOTING_CONFIGURATION.getKey(), "true").build());
assertThat(reconfigurator.reconfigure(nodes("a", "b"), retired(), initialConfig), equalTo(conf("a", "b", "c")));
assertThat(reconfigurator.reconfigure(twoNodes, retired(), randomFrom(twoNodes), initialConfig), equalTo(conf("a", "b", "c")));

expectThrows(IllegalArgumentException.class, () ->
clusterSettings.applySettings(Settings.builder().put(CLUSTER_AUTO_SHRINK_VOTING_CONFIGURATION.getKey(), "blah").build()));
Expand Down