[Zen2] Introduce auto_shrink_voting_configuration setting (#35217)

Today we allow the user to set the minimum size of a voting configuration. On
reflection we would rather this was simply '3' where possible, and we can use
the retirement API to control the removal of nodes more explicitly.

This change replaces the old reconfigurator setting with a new one,
`cluster.auto_shrink_voting_configuration`, which determines whether
Elasticsearch should automatically remove nodes from the voting configuration
or not.
This commit is contained in:
David Turner 2018-11-06 18:10:29 +00:00 committed by GitHub
parent 2fb3d1a465
commit 7e356ac29b
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
7 changed files with 184 additions and 196 deletions

View File

@ -72,7 +72,6 @@ import java.util.stream.Collectors;
import java.util.stream.StreamSupport;
import static java.util.Collections.emptySet;
import static org.elasticsearch.cluster.coordination.Reconfigurator.CLUSTER_MASTER_NODES_FAILURE_TOLERANCE;
import static org.elasticsearch.discovery.DiscoverySettings.NO_MASTER_BLOCK_WRITES;
import static org.elasticsearch.gateway.GatewayService.STATE_NOT_RECOVERED_BLOCK;
@ -593,8 +592,6 @@ public class Coordinator extends AbstractLifecycleComponent implements Discovery
MetaData.Builder metaDataBuilder = MetaData.builder();
// automatically generate a UID for the metadata if we need to
metaDataBuilder.generateClusterUuidIfNeeded(); // TODO generate UUID in bootstrapping tool?
metaDataBuilder.persistentSettings(Settings.builder().put(CLUSTER_MASTER_NODES_FAILURE_TOLERANCE.getKey(),
(votingConfiguration.getNodeIds().size() - 1) / 2).build()); // TODO set this in bootstrapping tool?
builder.metaData(metaDataBuilder);
coordinationState.get().setInitialState(builder.build());
preVoteCollector.update(getPreVoteResponse(), null); // pick up the change to last-accepted version

View File

@ -41,29 +41,32 @@ public class Reconfigurator extends AbstractComponent {
/**
* The cluster usually requires a vote from at least half of the master nodes in order to commit a cluster state update, and to achieve
* this it makes automatic adjustments to the quorum size as master nodes join or leave the cluster. However, if master nodes leave the
* cluster slowly enough then these automatic adjustments can end up with a single master node; if this last node were to fail then the
* cluster would be rendered permanently unavailable. Instead it may be preferable to stop processing cluster state updates and become
* unavailable when the second-last (more generally, n'th-last) node leaves the cluster, so that the cluster is never in a situation
* where a single node's failure can cause permanent unavailability. This setting determines the size of the smallest set of master
* nodes required to process a cluster state update.
* the best resilience it makes automatic adjustments to the voting configuration as master nodes join or leave the cluster. Adjustments
* that fix or increase the size of the voting configuration are always a good idea, but the wisdom of reducing the voting configuration
* size is less clear. For instance, automatically reducing the voting configuration down to a single node means the cluster requires
* this node to operate, which is not resilient: if it broke we could restore every other master-eligible node in the cluster to health
* and still the cluster would be unavailable. However not reducing the voting configuration size can also hamper resilience: in a
* five-node cluster we could lose two nodes and by reducing the voting configuration to the remaining three nodes we could tolerate the
* loss of a further node before failing.
*
* We offer two options: either we auto-shrink the voting configuration as long as it contains more than three nodes, or we don't and we
* require the user to control the voting configuration manually using the retirement API. The former, default, option, guarantees that
* as long as there have been at least three master-eligible nodes in the cluster and no more than one of them is currently unavailable,
* then the cluster will still operate, which is what almost everyone wants. Manual control is for users who want different guarantees.
*/
public static final Setting<Integer> CLUSTER_MASTER_NODES_FAILURE_TOLERANCE =
Setting.intSetting("cluster.master_nodes_failure_tolerance", 0, 0, Property.NodeScope, Property.Dynamic);
// the default is not supposed to be important since we expect to set this setting explicitly at bootstrapping time
// TODO contemplate setting the default to something larger than 0 (1? 1<<30?)
// TODO prevent this being set as a transient or a per-node setting?
public static final Setting<Boolean> CLUSTER_AUTO_SHRINK_VOTING_CONFIGURATION =
Setting.boolSetting("cluster.auto_shrink_voting_configuration", true, Property.NodeScope, Property.Dynamic);
private volatile int masterNodesFailureTolerance;
private volatile boolean autoShrinkVotingConfiguration;
public Reconfigurator(Settings settings, ClusterSettings clusterSettings) {
super(settings);
masterNodesFailureTolerance = CLUSTER_MASTER_NODES_FAILURE_TOLERANCE.get(settings);
clusterSettings.addSettingsUpdateConsumer(CLUSTER_MASTER_NODES_FAILURE_TOLERANCE, this::setMasterNodesFailureTolerance);
autoShrinkVotingConfiguration = CLUSTER_AUTO_SHRINK_VOTING_CONFIGURATION.get(settings);
clusterSettings.addSettingsUpdateConsumer(CLUSTER_AUTO_SHRINK_VOTING_CONFIGURATION, this::setAutoShrinkVotingConfiguration);
}
public void setMasterNodesFailureTolerance(int masterNodesFailureTolerance) {
this.masterNodesFailureTolerance = masterNodesFailureTolerance;
public void setAutoShrinkVotingConfiguration(boolean autoShrinkVotingConfiguration) {
this.autoShrinkVotingConfiguration = autoShrinkVotingConfiguration;
}
private static int roundDownToOdd(int size) {
@ -73,7 +76,7 @@ public class Reconfigurator extends AbstractComponent {
@Override
public String toString() {
return "Reconfigurator{" +
"masterNodesFailureTolerance=" + masterNodesFailureTolerance +
"autoShrinkVotingConfiguration=" + autoShrinkVotingConfiguration +
'}';
}
@ -92,22 +95,26 @@ public class Reconfigurator extends AbstractComponent {
ClusterState.VotingConfiguration currentConfig) {
logger.trace("{} reconfiguring {} based on liveNodes={}, retiredNodeIds={}", this, currentConfig, liveNodes, retiredNodeIds);
final int safeConfigurationSize = 2 * masterNodesFailureTolerance + 1;
if (currentConfig.getNodeIds().size() < safeConfigurationSize) {
throw new AssertionError(currentConfig + " is smaller than expected " + safeConfigurationSize);
// The new configuration may only shrink if CLUSTER_AUTO_SHRINK_VOTING_CONFIGURATION is true, and even then may not shrink
// to fewer than 3 nodes.
final int minimumConfigurationSize;
if (autoShrinkVotingConfiguration) {
minimumConfigurationSize = Math.min(roundDownToOdd(currentConfig.getNodeIds().size()), 3);
} else {
minimumConfigurationSize = currentConfig.getNodeIds().size();
}
/*
* There are three true/false properties of each node in play: live/non-live, retired/non-retired and in-config/not-in-config.
* Firstly we divide the nodes into disjoint sets based on these properties:
*
* - retiredInConfigNotLiveIds
* - nonRetiredInConfigNotLiveIds
* - retiredInConfigLiveIds
* - nonRetiredInConfigLiveIds
* - nonRetiredLiveNotInConfigIds
*
* The other 3 possibilities are not relevant:
* The other 5 possibilities are not relevant:
* - retired, in-config, live -- retired nodes should be removed from the config
* - retired, in-config, non-live -- retired nodes should be removed from the config
* - retired, not-in-config, live -- cannot add a retired node back to the config
* - retired, not-in-config, non-live -- cannot add a retired node back to the config
* - non-retired, non-live, not-in-config -- no evidence this node exists at all
@ -119,15 +126,11 @@ public class Reconfigurator extends AbstractComponent {
liveInConfigIds.retainAll(liveNodeIds);
final Set<String> inConfigNotLiveIds = Sets.sortedDifference(currentConfig.getNodeIds(), liveInConfigIds);
final Set<String> retiredInConfigNotLiveIds = new TreeSet<>(inConfigNotLiveIds);
retiredInConfigNotLiveIds.retainAll(retiredNodeIds);
final Set<String> nonRetiredInConfigNotLiveIds = new TreeSet<>(inConfigNotLiveIds);
nonRetiredInConfigNotLiveIds.removeAll(retiredInConfigNotLiveIds);
nonRetiredInConfigNotLiveIds.removeAll(retiredNodeIds);
final Set<String> retiredInConfigLiveIds = new TreeSet<>(liveInConfigIds);
retiredInConfigLiveIds.retainAll(retiredNodeIds);
final Set<String> nonRetiredInConfigLiveIds = new TreeSet<>(liveInConfigIds);
nonRetiredInConfigLiveIds.removeAll(retiredInConfigLiveIds);
nonRetiredInConfigLiveIds.removeAll(retiredNodeIds);
final Set<String> nonRetiredLiveNotInConfigIds = Sets.sortedDifference(liveNodeIds, currentConfig.getNodeIds());
nonRetiredLiveNotInConfigIds.removeAll(retiredNodeIds);
@ -135,23 +138,28 @@ public class Reconfigurator extends AbstractComponent {
/*
* Now we work out how many nodes should be in the configuration:
*/
final int targetSize;
// ideally we want the configuration to be all the non-retired live nodes ...
final int nonRetiredLiveNodeCount = nonRetiredInConfigLiveIds.size() + nonRetiredLiveNotInConfigIds.size();
// ... except one, if even, because odd configurations are slightly more resilient ...
final int votingNodeCount = roundDownToOdd(nonRetiredLiveNodeCount);
// ... except that the new configuration must satisfy CLUSTER_MASTER_NODES_FAILURE_TOLERANCE too:
final int targetSize = Math.max(votingNodeCount, safeConfigurationSize);
final int nonRetiredConfigSize = nonRetiredInConfigLiveIds.size() + nonRetiredInConfigNotLiveIds.size();
if (autoShrinkVotingConfiguration) {
if (nonRetiredLiveNodeCount >= 3) {
targetSize = roundDownToOdd(nonRetiredLiveNodeCount);
} else {
// only have one or two available nodes; may not shrink below 3 nodes automatically, but if
// the config (excluding retired nodes) is already smaller than 3 then it's ok.
targetSize = nonRetiredConfigSize < 3 ? 1 : 3;
}
} else {
targetSize = Math.max(roundDownToOdd(nonRetiredLiveNodeCount), nonRetiredConfigSize);
}
/*
* The new configuration is formed by taking this many nodes in the following preference order:
*/
final ClusterState.VotingConfiguration newConfig = new ClusterState.VotingConfiguration(
Stream.of(nonRetiredInConfigLiveIds, nonRetiredLiveNotInConfigIds, // live nodes first, preferring the current config
retiredInConfigLiveIds, // if we need more, first use retired nodes that are still alive and haven't been removed yet
nonRetiredInConfigNotLiveIds, retiredInConfigNotLiveIds) // if we need more, use non-live nodes
// live nodes first, preferring the current config, and if we need more then use non-live nodes
Stream.of(nonRetiredInConfigLiveIds, nonRetiredLiveNotInConfigIds, nonRetiredInConfigNotLiveIds)
.flatMap(Collection::stream).limit(targetSize).collect(Collectors.toSet()));
if (newConfig.hasQuorum(liveNodeIds)) {

View File

@ -452,11 +452,10 @@ public final class ClusterSettings extends AbstractScopedSettings {
ElectionSchedulerFactory.ELECTION_INITIAL_TIMEOUT_SETTING,
ElectionSchedulerFactory.ELECTION_BACK_OFF_TIME_SETTING,
ElectionSchedulerFactory.ELECTION_MAX_TIMEOUT_SETTING,
Coordinator.PUBLISH_TIMEOUT_SETTING,
ElectionSchedulerFactory.ELECTION_DURATION_SETTING,
Coordinator.PUBLISH_TIMEOUT_SETTING,
JoinHelper.JOIN_TIMEOUT_SETTING,
Reconfigurator.CLUSTER_MASTER_NODES_FAILURE_TOLERANCE
Reconfigurator.CLUSTER_AUTO_SHRINK_VOTING_CONFIGURATION
)));
public static List<SettingUpgrader<?>> BUILT_IN_SETTING_UPGRADERS = Collections.unmodifiableList(Arrays.asList(

View File

@ -85,7 +85,7 @@ import static org.elasticsearch.cluster.coordination.FollowersChecker.FOLLOWER_C
import static org.elasticsearch.cluster.coordination.LeaderChecker.LEADER_CHECK_INTERVAL_SETTING;
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_MASTER_NODES_FAILURE_TOLERANCE;
import static org.elasticsearch.cluster.coordination.Reconfigurator.CLUSTER_AUTO_SHRINK_VOTING_CONFIGURATION;
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;
@ -141,15 +141,13 @@ public class CoordinatorTests extends ESTestCase {
assertEquals(currentTerm, newTerm);
}
public void testExpandsConfigurationWhenGrowingFromOneToThreeNodesAndShrinksOnFailure() {
public void testExpandsConfigurationWhenGrowingFromOneNodeToThreeButDoesNotShrink() {
final Cluster cluster = new Cluster(1);
cluster.runRandomly();
cluster.stabilise();
final ClusterNode leader = cluster.getAnyLeader();
assertThat(CLUSTER_MASTER_NODES_FAILURE_TOLERANCE.get(leader.getLastAppliedClusterState().metaData().settings()), equalTo(0));
cluster.addNodesAndStabilise(2);
{
@ -167,22 +165,22 @@ public class CoordinatorTests extends ESTestCase {
{
final ClusterNode newLeader = cluster.getAnyLeader();
final VotingConfiguration lastCommittedConfiguration = newLeader.getLastAppliedClusterState().getLastCommittedConfiguration();
assertThat(lastCommittedConfiguration + " should be 1 node", lastCommittedConfiguration.getNodeIds().size(), equalTo(1));
assertFalse(lastCommittedConfiguration.getNodeIds().contains(disconnect1.getId()));
assertThat(lastCommittedConfiguration + " should be all nodes", lastCommittedConfiguration.getNodeIds(),
equalTo(cluster.clusterNodes.stream().map(ClusterNode::getId).collect(Collectors.toSet())));
}
}
public void testExpandsConfigurationWhenGrowingFromThreeToFiveNodesAndShrinksOnFailure() {
public void testExpandsConfigurationWhenGrowingFromThreeToFiveNodesAndShrinksBackToThreeOnFailure() {
final Cluster cluster = new Cluster(3);
cluster.runRandomly();
cluster.stabilise();
final ClusterNode leader = cluster.getAnyLeader();
logger.info("setting fault tolerance to 1");
leader.submitSetMasterNodesFailureTolerance(1);
logger.info("setting auto-shrink reconfiguration to true");
leader.submitSetAutoShrinkVotingConfiguration(true);
cluster.stabilise(DEFAULT_CLUSTER_STATE_UPDATE_DELAY);
assertThat(CLUSTER_MASTER_NODES_FAILURE_TOLERANCE.get(leader.getLastAppliedClusterState().metaData().settings()), equalTo(1));
assertTrue(CLUSTER_AUTO_SHRINK_VOTING_CONFIGURATION.get(leader.getLastAppliedClusterState().metaData().settings()));
cluster.addNodesAndStabilise(2);
@ -259,10 +257,10 @@ public class CoordinatorTests extends ESTestCase {
{
final ClusterNode leader = cluster.getAnyLeader();
logger.info("setting fault tolerance to 2");
leader.submitSetMasterNodesFailureTolerance(2);
logger.info("setting auto-shrink reconfiguration to false");
leader.submitSetAutoShrinkVotingConfiguration(false);
cluster.stabilise(DEFAULT_CLUSTER_STATE_UPDATE_DELAY);
assertThat(CLUSTER_MASTER_NODES_FAILURE_TOLERANCE.get(leader.getLastAppliedClusterState().metaData().settings()), equalTo(2));
assertFalse(CLUSTER_AUTO_SHRINK_VOTING_CONFIGURATION.get(leader.getLastAppliedClusterState().metaData().settings()));
}
final ClusterNode disconnect1 = cluster.getAnyNode();
@ -281,9 +279,10 @@ public class CoordinatorTests extends ESTestCase {
equalTo(cluster.clusterNodes.stream().map(ClusterNode::getId).collect(Collectors.toSet())));
}
leader.submitSetMasterNodesFailureTolerance(1);
logger.info("setting auto-shrink reconfiguration to true");
leader.submitSetAutoShrinkVotingConfiguration(true);
cluster.stabilise(DEFAULT_CLUSTER_STATE_UPDATE_DELAY * 2); // allow for a reconfiguration
assertThat(CLUSTER_MASTER_NODES_FAILURE_TOLERANCE.get(leader.getLastAppliedClusterState().metaData().settings()), equalTo(1));
assertTrue(CLUSTER_AUTO_SHRINK_VOTING_CONFIGURATION.get(leader.getLastAppliedClusterState().metaData().settings()));
{
final VotingConfiguration lastCommittedConfiguration = leader.getLastAppliedClusterState().getLastCommittedConfiguration();
@ -293,52 +292,11 @@ public class CoordinatorTests extends ESTestCase {
}
}
public void testCanShrinkFromThreeNodesToTwo() {
public void testDoesNotShrinkConfigurationBelowThreeNodes() {
final Cluster cluster = new Cluster(3);
cluster.runRandomly();
cluster.stabilise();
{
final ClusterNode leader = cluster.getAnyLeader();
logger.info("setting fault tolerance to 1");
leader.submitSetMasterNodesFailureTolerance(1);
cluster.stabilise(DEFAULT_CLUSTER_STATE_UPDATE_DELAY);
assertThat(CLUSTER_MASTER_NODES_FAILURE_TOLERANCE.get(leader.getLastAppliedClusterState().metaData().settings()), equalTo(1));
}
final ClusterNode disconnect1 = cluster.getAnyNode();
logger.info("--> disconnecting {}", disconnect1);
disconnect1.disconnect();
cluster.stabilise();
final ClusterNode leader = cluster.getAnyLeader();
{
final VotingConfiguration lastCommittedConfiguration = leader.getLastAppliedClusterState().getLastCommittedConfiguration();
assertThat(lastCommittedConfiguration + " should be all nodes", lastCommittedConfiguration.getNodeIds(),
equalTo(cluster.clusterNodes.stream().map(ClusterNode::getId).collect(Collectors.toSet())));
}
leader.submitSetMasterNodesFailureTolerance(0);
cluster.stabilise(DEFAULT_CLUSTER_STATE_UPDATE_DELAY * 2); // allow for a reconfiguration
assertThat(CLUSTER_MASTER_NODES_FAILURE_TOLERANCE.get(leader.getLastAppliedClusterState().metaData().settings()), equalTo(0));
{
final VotingConfiguration lastCommittedConfiguration = leader.getLastAppliedClusterState().getLastCommittedConfiguration();
assertThat(lastCommittedConfiguration + " should be 1 node", lastCommittedConfiguration.getNodeIds().size(), equalTo(1));
assertFalse(lastCommittedConfiguration.getNodeIds().contains(disconnect1.getId()));
}
}
public void testDoesNotShrinkConfigurationDueToLossToleranceConfigurationWithThreeNodes() {
final Cluster cluster = new Cluster(3);
cluster.runRandomly();
cluster.stabilise();
cluster.getAnyLeader().submitSetMasterNodesFailureTolerance(1);
cluster.stabilise(DEFAULT_ELECTION_DELAY);
final ClusterNode disconnect1 = cluster.getAnyNode();
logger.info("--> disconnecting {}", disconnect1);
@ -358,12 +316,12 @@ public class CoordinatorTests extends ESTestCase {
cluster.stabilise(); // would not work if disconnect1 were removed from the configuration
}
public void testDoesNotShrinkConfigurationDueToLossToleranceConfigurationWithFiveNodes() {
public void testDoesNotShrinkConfigurationBelowFiveNodesIfAutoShrinkDisabled() {
final Cluster cluster = new Cluster(5);
cluster.runRandomly();
cluster.stabilise();
cluster.getAnyLeader().submitSetMasterNodesFailureTolerance(2);
cluster.getAnyLeader().submitSetAutoShrinkVotingConfiguration(false);
cluster.stabilise(DEFAULT_ELECTION_DELAY);
final ClusterNode disconnect1 = cluster.getAnyNode();
@ -911,12 +869,12 @@ public class CoordinatorTests extends ESTestCase {
}).run();
} else if (rarely()) {
final ClusterNode clusterNode = getAnyNodePreferringLeaders();
final int masterNodeFailureTolerance = randomIntBetween(0, 2);
final boolean autoShrinkVotingConfiguration = randomBoolean();
onNode(clusterNode.getLocalNode(),
() -> {
logger.debug("----> [runRandomly {}] setting master-node fault tolerance to {} on {}",
thisStep, masterNodeFailureTolerance, clusterNode.getId());
clusterNode.submitSetMasterNodesFailureTolerance(masterNodeFailureTolerance);
logger.debug("----> [runRandomly {}] setting auto-shrink configuration to {} on {}",
thisStep, autoShrinkVotingConfiguration, clusterNode.getId());
clusterNode.submitSetAutoShrinkVotingConfiguration(autoShrinkVotingConfiguration);
}).run();
} else if (rarely()) {
final ClusterNode clusterNode = getAnyNode();
@ -1286,7 +1244,7 @@ public class CoordinatorTests extends ESTestCase {
final ClusterSettings clusterSettings = new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS);
clusterApplier = new FakeClusterApplier(settings, clusterSettings);
masterService = new AckedFakeThreadPoolMasterService("test_node","test",
masterService = new AckedFakeThreadPoolMasterService("test_node", "test",
runnable -> deterministicTaskQueue.scheduleNow(onNode(localNode, runnable)));
transportService = mockTransport.createTransportService(
settings, deterministicTaskQueue.getThreadPool(runnable -> onNode(localNode, runnable)), NOOP_TRANSPORT_INTERCEPTOR,
@ -1333,18 +1291,16 @@ public class CoordinatorTests extends ESTestCase {
return clusterStateApplyResponse;
}
void submitSetMasterNodesFailureTolerance(final int masterNodesFaultTolerance) {
submitUpdateTask("set master nodes failure tolerance [" + masterNodesFaultTolerance + "]", cs ->
cs.getLastAcceptedConfiguration().getNodeIds().size() < 2 * masterNodesFaultTolerance + 1 ? cs :
// TODO this rejects invalid updates, but in fact this should be validated elsewhere. Where?
ClusterState.builder(cs).metaData(
MetaData.builder(cs.metaData())
.persistentSettings(Settings.builder()
.put(cs.metaData().persistentSettings())
.put(CLUSTER_MASTER_NODES_FAILURE_TOLERANCE.getKey(), masterNodesFaultTolerance)
.build())
void submitSetAutoShrinkVotingConfiguration(final boolean autoShrinkVotingConfiguration) {
submitUpdateTask("set master nodes failure tolerance [" + autoShrinkVotingConfiguration + "]", cs ->
ClusterState.builder(cs).metaData(
MetaData.builder(cs.metaData())
.persistentSettings(Settings.builder()
.put(cs.metaData().persistentSettings())
.put(CLUSTER_AUTO_SHRINK_VOTING_CONFIGURATION.getKey(), autoShrinkVotingConfiguration)
.build())
.build());
.build())
.build());
}
AckCollector submitValue(final long value) {

View File

@ -26,6 +26,7 @@ import org.elasticsearch.cluster.ClusterState.VotingConfiguration;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.common.settings.ClusterSettings;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.settings.Settings.Builder;
import org.elasticsearch.common.util.set.Sets;
import org.elasticsearch.test.ESTestCase;
import org.junit.Before;
@ -37,8 +38,7 @@ import java.util.Set;
import java.util.stream.Collectors;
import static java.util.Collections.emptySet;
import static org.elasticsearch.cluster.coordination.Reconfigurator.CLUSTER_MASTER_NODES_FAILURE_TOLERANCE;
import static org.hamcrest.Matchers.containsString;
import static org.elasticsearch.cluster.coordination.Reconfigurator.CLUSTER_AUTO_SHRINK_VOTING_CONFIGURATION;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.sameInstance;
@ -51,45 +51,62 @@ public class ReconfiguratorTests extends ESTestCase {
public void testReconfigurationExamples() {
check(nodes("a"), conf("a"), 0, conf("a"));
check(nodes("a", "b"), conf("a"), 0, conf("a"));
check(nodes("a", "b", "c"), conf("a"), 0, conf("a", "b", "c"));
check(nodes("a", "b", "c"), conf("a", "b"), 0, conf("a", "b", "c"));
check(nodes("a", "b"), conf("a", "b", "e"), 0, conf("a"));
check(nodes("a", "b"), conf("a", "b", "e"), 1, conf("a", "b", "e"));
check(nodes("a"), conf("a"), true, conf("a"));
check(nodes("a", "b"), conf("a"), true, conf("a"));
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"));
check(nodes("a", "b", "c"), conf("a"), true, conf("a", "b", "c"));
check(nodes("a", "b", "c"), conf("a", "b"), true, conf("a", "b", "c"));
check(nodes("a", "b", "c"), conf("a", "b", "c"), true, conf("a", "b", "c"));
check(nodes("a", "b", "c", "d"), conf("a", "b", "c"), true, conf("a", "b", "c"));
check(nodes("a", "b", "c", "d", "e"), conf("a", "b", "c"), true, conf("a", "b", "c", "d", "e"));
check(nodes("a", "b", "c"), conf("a", "b", "e"), true, conf("a", "b", "c"));
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("a", "b", "c"), conf("a", "b", "c", "d", "e"), true, conf("a", "b", "c"));
for (int masterNodesFailureTolerance = 0; masterNodesFailureTolerance <= 1; masterNodesFailureTolerance++) {
check(nodes("a", "b", "c"), conf("a", "b", "c"), masterNodesFailureTolerance, conf("a", "b", "c"));
check(nodes("a", "b", "c", "d"), conf("a", "b", "c"), masterNodesFailureTolerance, conf("a", "b", "c"));
check(nodes("a", "b", "c", "d", "e"), conf("a", "b", "c"), masterNodesFailureTolerance, conf("a", "b", "c", "d", "e"));
check(nodes("a", "b", "c"), conf("a", "b", "e"), masterNodesFailureTolerance, conf("a", "b", "c"));
check(nodes("a", "b", "c", "d"), conf("a", "b", "e"), masterNodesFailureTolerance, conf("a", "b", "c"));
check(nodes("a", "b", "c", "d", "e"), conf("a", "f", "g"), masterNodesFailureTolerance, conf("a", "b", "c", "d", "e"));
check(nodes("a", "b", "c", "d"), conf("a", "b", "c", "d", "e"), masterNodesFailureTolerance, conf("a", "b", "c"));
}
check(nodes("a", "b", "c", "d"), conf("a", "b", "c", "d", "e"), 2, conf("a", "b", "c", "d", "e"));
check(nodes("a"), conf("a"), false, conf("a"));
check(nodes("a", "b"), conf("a"), false, conf("a"));
check(nodes("a", "b"), conf("a", "b"), false, conf("a", "b"));
check(nodes("a", "b", "c"), conf("a"), false, conf("a", "b", "c"));
check(nodes("a", "b", "c"), conf("a", "b"), false, conf("a", "b", "c"));
check(nodes("a", "b"), conf("a", "b", "e"), false, conf("a", "b", "e"));
check(nodes("a", "b"), conf("a", "c"), false, conf("a", "b"));
check(nodes("a", "b"), conf("a", "b", "e"), false, conf("a", "b", "e"));
check(nodes("a", "b", "c"), conf("a", "b", "c"), false, conf("a", "b", "c"));
check(nodes("a", "b", "c", "d"), conf("a", "b", "c"), false, conf("a", "b", "c"));
check(nodes("a", "b", "c", "d", "e"), conf("a", "b", "c"), false, conf("a", "b", "c", "d", "e"));
check(nodes("a", "b", "c"), conf("a", "b", "e"), false, conf("a", "b", "c"));
check(nodes("a", "b", "c", "d"), conf("a", "b", "e"), false, conf("a", "b", "c"));
check(nodes("a", "b", "c", "d", "e"), conf("a", "f", "g"), false, conf("a", "b", "c", "d", "e"));
check(nodes("a", "b", "c", "d"), conf("a", "b", "c", "d", "e"), false, conf("a", "b", "c", "d", "e"));
check(nodes("a", "b", "c"), conf("a", "b", "c", "d", "e"), false, conf("a", "b", "c", "d", "e"));
// Retiring a single node shifts the votes elsewhere if possible.
check(nodes("a", "b"), retired("a"), conf("a"), 0, conf("b"));
check(nodes("a", "b"), retired("a"), conf("a"), true, conf("b"));
check(nodes("a", "b"), retired("a"), conf("a"), false, conf("b"));
// Retiring a node from a three-node cluster drops down to a one-node configuration if failure tolerance is 0
check(nodes("a", "b", "c"), retired("a"), conf("a"), 0, conf("b"));
check(nodes("a", "b", "c"), retired("a"), conf("a", "b", "c"), 0, conf("b"));
// Retiring is prevented in a three-node cluster if failure tolerance is 1
check(nodes("a", "b", "c"), retired("a"), conf("a", "b", "c"), 1, conf("a", "b", "c"));
// Retiring a node from a three-node cluster drops down to a one-node configuration
check(nodes("a", "b", "c"), retired("a"), conf("a"), true, conf("b"));
check(nodes("a", "b", "c"), retired("a"), conf("a", "b", "c"), true, conf("b"));
check(nodes("a", "b", "c"), retired("a"), conf("a"), false, conf("b"));
check(nodes("a", "b", "c"), retired("a"), conf("a", "b", "c"), false, conf("b", "c"));
// 7 nodes, one for each combination of live/retired/current. Ideally we want the config to be the non-retired live nodes.
// Since there are 2 non-retired live nodes we round down to 1 and just use the one that's already in the config.
check(nodes("a", "b", "c", "f"), retired("c", "e", "f", "g"), conf("a", "c", "d", "e"), 0, conf("a"));
// If we want the config to be at least 3 nodes then we don't retire "c" just yet.
check(nodes("a", "b", "c", "f"), retired("c", "e", "f", "g"), conf("a", "c", "d", "e"), 1, conf("a", "b", "c"));
// If we want the config to be at least 5 nodes then we keep "d" and "h".
check(nodes("a", "b", "c", "f"), retired("c", "e", "f", "g"), conf("a", "c", "d", "e", "h"), 2, conf("a", "b", "c", "d", "h"));
check(nodes("a", "b", "c", "f"), retired("c", "e", "f", "g"), conf("a", "c", "d", "e"), true, conf("a"));
// Only two non-retired nodes in the config, so new config does not shrink below 2
check(nodes("a", "b", "c", "f"), retired("c", "e", "f", "g"), conf("a", "c", "d", "e"), false, conf("a", "b"));
// The config has at least three non-retired nodes so does not shrink below 3
check(nodes("a", "b", "c", "f"), retired("c", "e", "f", "g"), conf("a", "c", "d", "e", "h"), true, conf("a", "b", "d"));
// Three non-retired nodes in the config, so new config does not shrink below 3
check(nodes("a", "b", "c", "f"), retired("c", "e", "f", "g"), conf("a", "c", "d", "e", "h"), false, conf("a", "b", "d"));
}
public void testReconfigurationProperty() {
public void testAutoShrinking() {
final String[] allNodes = new String[]{"a", "b", "c", "d", "e", "f", "g"};
final String[] liveNodes = new String[randomIntBetween(1, allNodes.length)];
@ -98,34 +115,53 @@ public class ReconfiguratorTests extends ESTestCase {
final String[] initialVotingNodes = new String[randomIntBetween(1, allNodes.length)];
randomSubsetOf(initialVotingNodes.length, allNodes).toArray(initialVotingNodes);
final int masterNodesFailureTolerance = randomIntBetween(0, 2);
final Reconfigurator reconfigurator = makeReconfigurator(
Settings.builder().put(CLUSTER_MASTER_NODES_FAILURE_TOLERANCE.getKey(), masterNodesFailureTolerance).build());
final Builder settingsBuilder = Settings.builder();
if (randomBoolean()) {
settingsBuilder.put(CLUSTER_AUTO_SHRINK_VOTING_CONFIGURATION.getKey(), true);
}
final Reconfigurator reconfigurator = makeReconfigurator(settingsBuilder.build());
final Set<DiscoveryNode> liveNodesSet = nodes(liveNodes);
final ClusterState.VotingConfiguration initialConfig = conf(initialVotingNodes);
// min configuration size comes from CLUSTER_MASTER_NODES_FAILURE_TOLERANCE as long as there are enough nodes in the current config
final int quorumSize = Math.max(liveNodes.length / 2 + 1, initialVotingNodes.length < 3 ? 1 : 2);
if (initialConfig.getNodeIds().size() >= masterNodesFailureTolerance * 2 + 1) {
// actual size of a quorum: half the configured nodes, which is all the live nodes plus maybe some dead ones to make up numbers
final int quorumSize = Math.max(liveNodes.length / 2 + 1, masterNodesFailureTolerance + 1);
final ClusterState.VotingConfiguration finalConfig = reconfigurator.reconfigure(liveNodesSet, emptySet(), initialConfig);
final ClusterState.VotingConfiguration finalConfig = reconfigurator.reconfigure(liveNodesSet, emptySet(), initialConfig);
final String description = "reconfigure " + liveNodesSet + " from " + initialConfig + " yielded " + finalConfig;
final String description = "reconfigure " + liveNodesSet + " from " + initialConfig + " with failure tolerance of "
+ masterNodesFailureTolerance + " yielded " + finalConfig;
if (quorumSize > liveNodes.length) {
assertFalse(description + " without a live quorum", finalConfig.hasQuorum(Arrays.asList(liveNodes)));
} else {
final List<String> expectedQuorum = randomSubsetOf(quorumSize, liveNodes);
assertTrue(description + " with quorum[" + quorumSize + "] of " + expectedQuorum, finalConfig.hasQuorum(expectedQuorum));
}
if (quorumSize > liveNodes.length) {
assertFalse(description + " without a live quorum", finalConfig.hasQuorum(Arrays.asList(liveNodes)));
} else {
assertThat(expectThrows(AssertionError.class,
() -> reconfigurator.reconfigure(liveNodesSet, emptySet(), initialConfig)).getMessage(),
containsString("is smaller than expected"));
final List<String> expectedQuorum = randomSubsetOf(quorumSize, liveNodes);
assertTrue(description + " with quorum[" + quorumSize + "] of " + expectedQuorum, finalConfig.hasQuorum(expectedQuorum));
}
}
public void testManualShrinking() {
final String[] allNodes = new String[]{"a", "b", "c", "d", "e", "f", "g"};
final String[] liveNodes = new String[randomIntBetween(1, allNodes.length)];
randomSubsetOf(liveNodes.length, allNodes).toArray(liveNodes);
final String[] initialVotingNodes = new String[randomIntBetween(1, allNodes.length)];
randomSubsetOf(initialVotingNodes.length, allNodes).toArray(initialVotingNodes);
final Reconfigurator reconfigurator
= makeReconfigurator(Settings.builder().put(CLUSTER_AUTO_SHRINK_VOTING_CONFIGURATION.getKey(), false).build());
final Set<DiscoveryNode> liveNodesSet = nodes(liveNodes);
final ClusterState.VotingConfiguration initialConfig = conf(initialVotingNodes);
final int quorumSize = Math.max(liveNodes.length, initialVotingNodes.length) / 2 + 1;
final ClusterState.VotingConfiguration finalConfig = reconfigurator.reconfigure(liveNodesSet, emptySet(), initialConfig);
final String description = "reconfigure " + liveNodesSet + " from " + initialConfig + " yielded " + finalConfig;
if (quorumSize > liveNodes.length) {
assertFalse(description + " without a live quorum", finalConfig.hasQuorum(Arrays.asList(liveNodes)));
} else {
final List<String> expectedQuorum = randomSubsetOf(quorumSize, liveNodes);
assertTrue(description + " with quorum[" + quorumSize + "] of " + expectedQuorum, finalConfig.hasQuorum(expectedQuorum));
}
}
@ -145,20 +181,20 @@ public class ReconfiguratorTests extends ESTestCase {
return Arrays.stream(nodes).collect(Collectors.toSet());
}
private void check(Set<DiscoveryNode> liveNodes, ClusterState.VotingConfiguration config, int masterNodesFailureTolerance,
private void check(Set<DiscoveryNode> liveNodes, ClusterState.VotingConfiguration config, boolean autoShrinkVotingConfiguration,
ClusterState.VotingConfiguration expectedConfig) {
check(liveNodes, retired(), config, masterNodesFailureTolerance, expectedConfig);
check(liveNodes, retired(), config, autoShrinkVotingConfiguration, expectedConfig);
}
private void check(Set<DiscoveryNode> liveNodes, Set<String> retired, ClusterState.VotingConfiguration config,
int masterNodesFailureTolerance, ClusterState.VotingConfiguration expectedConfig) {
boolean autoShrinkVotingConfiguration, ClusterState.VotingConfiguration expectedConfig) {
final Reconfigurator reconfigurator = makeReconfigurator(Settings.builder()
.put(CLUSTER_MASTER_NODES_FAILURE_TOLERANCE.getKey(), masterNodesFailureTolerance)
.put(CLUSTER_AUTO_SHRINK_VOTING_CONFIGURATION.getKey(), autoShrinkVotingConfiguration)
.build());
final ClusterState.VotingConfiguration adaptedConfig = reconfigurator.reconfigure(liveNodes, retired, config);
assertEquals(new ParameterizedMessage("[liveNodes={}, retired={}, config={}, masterNodesFailureTolerance={}]",
liveNodes, retired, config, masterNodesFailureTolerance).getFormattedMessage(),
assertEquals(new ParameterizedMessage("[liveNodes={}, retired={}, config={}, autoShrinkVotingConfiguration={}]",
liveNodes, retired, config, autoShrinkVotingConfiguration).getFormattedMessage(),
expectedConfig, adaptedConfig);
}
@ -171,24 +207,20 @@ public class ReconfiguratorTests extends ESTestCase {
final Reconfigurator reconfigurator = new Reconfigurator(Settings.EMPTY, clusterSettings);
final VotingConfiguration initialConfig = conf("a", "b", "c", "d", "e");
// default is "0"
assertThat(reconfigurator.reconfigure(nodes("a"), retired(), initialConfig), equalTo(conf("a")));
// default is "true"
assertThat(reconfigurator.reconfigure(nodes("a", "b"), retired(), initialConfig), equalTo(conf("a", "b", "c")));
// update to "2"
clusterSettings.applySettings(Settings.builder().put(CLUSTER_MASTER_NODES_FAILURE_TOLERANCE.getKey(), "2").build());
assertThat(reconfigurator.reconfigure(nodes("a"), retired(), initialConfig), sameInstance(initialConfig)); // cannot reconfigure
// 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")));
// update to "1"
clusterSettings.applySettings(Settings.builder().put(CLUSTER_MASTER_NODES_FAILURE_TOLERANCE.getKey(), "1").build());
assertThat(reconfigurator.reconfigure(nodes("a"), retired(), initialConfig), sameInstance(initialConfig)); // cannot reconfigure
assertThat(reconfigurator.reconfigure(nodes("a", "b", "c"), retired(), initialConfig), equalTo(conf("a", "b", "c")));
// explicitly set to "0"
clusterSettings.applySettings(Settings.builder().put(CLUSTER_MASTER_NODES_FAILURE_TOLERANCE.getKey(), "0").build());
assertThat(reconfigurator.reconfigure(nodes("a"), retired(), initialConfig), equalTo(conf("a")));
// 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")));
expectThrows(IllegalArgumentException.class, () ->
clusterSettings.applySettings(Settings.builder().put(CLUSTER_MASTER_NODES_FAILURE_TOLERANCE.getKey(), "-1").build()));
clusterSettings.applySettings(Settings.builder().put(CLUSTER_AUTO_SHRINK_VOTING_CONFIGURATION.getKey(), "blah").build()));
}
}

View File

@ -67,7 +67,6 @@ import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.RestoreInProgress;
import org.elasticsearch.cluster.SnapshotDeletionsInProgress;
import org.elasticsearch.cluster.SnapshotsInProgress;
import org.elasticsearch.cluster.coordination.Reconfigurator;
import org.elasticsearch.cluster.health.ClusterHealthStatus;
import org.elasticsearch.cluster.metadata.IndexGraveyard;
import org.elasticsearch.cluster.metadata.IndexMetaData;
@ -554,7 +553,6 @@ public abstract class ESIntegTestCase extends ESTestCase {
MetaData metaData = client().admin().cluster().prepareState().execute().actionGet().getState().getMetaData();
final Set<String> persistentKeys = new HashSet<>(metaData.persistentSettings().keySet());
persistentKeys.remove(Reconfigurator.CLUSTER_MASTER_NODES_FAILURE_TOLERANCE.getKey());
assertThat("test leaves persistent cluster metadata behind", persistentKeys, empty());
final Set<String> transientKeys = new HashSet<>(metaData.transientSettings().keySet());

View File

@ -25,7 +25,6 @@ import org.elasticsearch.action.admin.indices.get.GetIndexResponse;
import org.elasticsearch.client.Client;
import org.elasticsearch.client.Requests;
import org.elasticsearch.cluster.ClusterName;
import org.elasticsearch.cluster.coordination.Reconfigurator;
import org.elasticsearch.cluster.health.ClusterHealthStatus;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.cluster.metadata.MetaData;
@ -125,7 +124,6 @@ public abstract class ESSingleNodeTestCase extends ESTestCase {
assertAcked(client().admin().indices().prepareDelete("*").get());
MetaData metaData = client().admin().cluster().prepareState().get().getState().getMetaData();
Settings.Builder unexpectedPersistentSettingsBuilder = Settings.builder().put(metaData.persistentSettings());
unexpectedPersistentSettingsBuilder.remove(Reconfigurator.CLUSTER_MASTER_NODES_FAILURE_TOLERANCE.getKey());
Settings unexpectedPersistentSettings = unexpectedPersistentSettingsBuilder.build();
assertThat("test leaves persistent cluster metadata behind: " + unexpectedPersistentSettings.keySet(),
unexpectedPersistentSettings.size(), equalTo(0));