From 0567b49cb130bd2faf96291faf11e086437da13c Mon Sep 17 00:00:00 2001 From: David Turner Date: Tue, 18 Jun 2019 14:11:56 +0100 Subject: [PATCH 01/29] Voting-only nodes Adds a plugin that allows some master-eligible nodes to be used as "voting-only" nodes. Such nodes are never elected as the true master of the cluster, but they perform all the tasks needed of a master-eligible node to provide resilience. --- .../ClusterFormationFailureHelper.java | 6 +- .../coordination/CoordinationState.java | 12 +- .../cluster/coordination/Coordinator.java | 26 +-- .../coordination/ElectionStrategy.java | 85 ++++++++++ .../coordination/PreVoteCollector.java | 13 +- .../cluster/coordination/Reconfigurator.java | 7 +- .../common/settings/ClusterSettings.java | 1 + .../discovery/DiscoveryModule.java | 20 ++- .../plugins/DiscoveryPlugin.java | 5 + .../ClusterFormationFailureHelperTests.java | 82 +++++----- .../coordination/CoordinationStateTests.java | 8 +- .../coordination/CoordinatorTests.java | 3 +- .../cluster/coordination/NodeJoinTests.java | 2 +- .../coordination/PreVoteCollectorTests.java | 6 +- .../coordination/PublicationTests.java | 3 +- .../coordination/ReconfiguratorTests.java | 6 +- .../snapshots/SnapshotResiliencyTests.java | 3 +- x-pack/plugin/voting-only-node/build.gradle | 21 +++ .../coordination/VotingOnlyNodePlugin.java | 152 ++++++++++++++++++ .../coordination/VotingOnlyNodePluginIT.java | 33 ++++ 20 files changed, 419 insertions(+), 75 deletions(-) create mode 100644 server/src/main/java/org/elasticsearch/cluster/coordination/ElectionStrategy.java create mode 100644 x-pack/plugin/voting-only-node/build.gradle create mode 100644 x-pack/plugin/voting-only-node/src/main/java/org/elasticsearch/cluster/coordination/VotingOnlyNodePlugin.java create mode 100644 x-pack/plugin/voting-only-node/src/test/java/org/elasticsearch/cluster/coordination/VotingOnlyNodePluginIT.java diff --git a/server/src/main/java/org/elasticsearch/cluster/coordination/ClusterFormationFailureHelper.java b/server/src/main/java/org/elasticsearch/cluster/coordination/ClusterFormationFailureHelper.java index b5bae4bc3f0d4..4d4027d3e9f3a 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/ClusterFormationFailureHelper.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/ClusterFormationFailureHelper.java @@ -121,14 +121,16 @@ static class ClusterFormationState { private final List resolvedAddresses; private final List foundPeers; private final long currentTerm; + private final ElectionStrategy electionStrategy; ClusterFormationState(Settings settings, ClusterState clusterState, List resolvedAddresses, - List foundPeers, long currentTerm) { + List foundPeers, long currentTerm, ElectionStrategy electionStrategy) { this.settings = settings; this.clusterState = clusterState; this.resolvedAddresses = resolvedAddresses; this.foundPeers = foundPeers; this.currentTerm = currentTerm; + this.electionStrategy = electionStrategy; } String getDescription() { @@ -185,7 +187,7 @@ String getDescription() { final VoteCollection voteCollection = new VoteCollection(); foundPeers.forEach(voteCollection::addVote); final String isQuorumOrNot - = CoordinationState.isElectionQuorum(voteCollection, clusterState) ? "is a quorum" : "is not a quorum"; + = CoordinationState.isElectionQuorum(voteCollection, clusterState, electionStrategy) ? "is a quorum" : "is not a quorum"; return String.format(Locale.ROOT, "master not discovered or elected yet, an election requires %s, have discovered %s which %s; %s", diff --git a/server/src/main/java/org/elasticsearch/cluster/coordination/CoordinationState.java b/server/src/main/java/org/elasticsearch/cluster/coordination/CoordinationState.java index ac75c83c19a26..ef35e10832cb2 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/CoordinationState.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/CoordinationState.java @@ -24,7 +24,6 @@ import org.elasticsearch.cluster.coordination.CoordinationMetaData.VotingConfiguration; import org.elasticsearch.cluster.metadata.MetaData; import org.elasticsearch.cluster.node.DiscoveryNode; -import org.elasticsearch.common.settings.Settings; import java.util.Collection; import java.util.Collections; @@ -44,6 +43,7 @@ public class CoordinationState { // persisted state private final PersistedState persistedState; + private final ElectionStrategy electionStrategy; // transient state private VoteCollection joinVotes; @@ -53,11 +53,12 @@ public class CoordinationState { private VotingConfiguration lastPublishedConfiguration; private VoteCollection publishVotes; - public CoordinationState(Settings settings, DiscoveryNode localNode, PersistedState persistedState) { + public CoordinationState(DiscoveryNode localNode, PersistedState persistedState, ElectionStrategy electionStrategy) { this.localNode = localNode; // persisted state this.persistedState = persistedState; + this.electionStrategy = electionStrategy; // transient state this.joinVotes = new VoteCollection(); @@ -101,11 +102,12 @@ public boolean electionWon() { } public boolean isElectionQuorum(VoteCollection votes) { - return isElectionQuorum(votes, getLastAcceptedState()); + return isElectionQuorum(votes, getLastAcceptedState(), electionStrategy); } - static boolean isElectionQuorum(VoteCollection votes, ClusterState lastAcceptedState) { - return votes.isQuorum(lastAcceptedState.getLastCommittedConfiguration()) + static boolean isElectionQuorum(VoteCollection votes, ClusterState lastAcceptedState, ElectionStrategy electionStrategy) { + return electionStrategy.isGoodQuorum(votes.nodes()) + && votes.isQuorum(lastAcceptedState.getLastCommittedConfiguration()) && votes.isQuorum(lastAcceptedState.getLastAcceptedConfiguration()); } diff --git a/server/src/main/java/org/elasticsearch/cluster/coordination/Coordinator.java b/server/src/main/java/org/elasticsearch/cluster/coordination/Coordinator.java index b88d8ed21ba09..43499f1bd9bce 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/Coordinator.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/Coordinator.java @@ -22,6 +22,7 @@ import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; import org.apache.lucene.util.SetOnce; +import org.elasticsearch.ElasticsearchException; import org.elasticsearch.action.ActionListener; import org.elasticsearch.cluster.ClusterChangedEvent; import org.elasticsearch.cluster.ClusterName; @@ -100,6 +101,7 @@ public class Coordinator extends AbstractLifecycleComponent implements Discovery private final Settings settings; private final boolean singleNodeDiscovery; + private final ElectionStrategy electionStrategy; private final TransportService transportService; private final MasterService masterService; private final AllocationService allocationService; @@ -150,13 +152,14 @@ public Coordinator(String nodeName, Settings settings, ClusterSettings clusterSe NamedWriteableRegistry namedWriteableRegistry, AllocationService allocationService, MasterService masterService, Supplier persistedStateSupplier, SeedHostsProvider seedHostsProvider, ClusterApplier clusterApplier, Collection> onJoinValidators, Random random, - Consumer reroute) { + Consumer reroute, ElectionStrategy electionStrategy) { this.settings = settings; this.transportService = transportService; this.masterService = masterService; this.allocationService = allocationService; this.onJoinValidators = JoinTaskExecutor.addBuiltInJoinValidators(onJoinValidators); this.singleNodeDiscovery = DiscoveryModule.SINGLE_NODE_DISCOVERY_TYPE.equals(DiscoveryModule.DISCOVERY_TYPE_SETTING.get(settings)); + this.electionStrategy = electionStrategy; this.joinHelper = new JoinHelper(settings, allocationService, masterService, transportService, this::getCurrentTerm, this::getStateForMasterService, this::handleJoinRequest, this::joinLeaderInTerm, this.onJoinValidators, reroute); @@ -168,7 +171,7 @@ public Coordinator(String nodeName, Settings settings, ClusterSettings clusterSe this.publishTimeout = PUBLISH_TIMEOUT_SETTING.get(settings); this.random = random; this.electionSchedulerFactory = new ElectionSchedulerFactory(settings, random, transportService.getThreadPool()); - this.preVoteCollector = new PreVoteCollector(transportService, this::startElection, this::updateMaxTermSeen); + this.preVoteCollector = new PreVoteCollector(transportService, this::startElection, this::updateMaxTermSeen, electionStrategy); configuredHostsResolver = new SeedHostsResolver(nodeName, settings, transportService, seedHostsProvider); this.peerFinder = new CoordinatorPeerFinder(settings, transportService, new HandshakingTransportAddressConnector(settings, transportService), configuredHostsResolver); @@ -179,7 +182,7 @@ public Coordinator(String nodeName, Settings settings, ClusterSettings clusterSe this.nodeRemovalExecutor = new NodeRemovalClusterStateTaskExecutor(allocationService, logger); this.clusterApplier = clusterApplier; masterService.setClusterStateSupplier(this::getStateForMasterService); - this.reconfigurator = new Reconfigurator(settings, clusterSettings); + this.reconfigurator = new Reconfigurator(settings, clusterSettings, electionStrategy); this.clusterBootstrapService = new ClusterBootstrapService(settings, transportService, this::getFoundPeers, this::isInitialConfigurationSet, this::setInitialConfiguration); this.lagDetector = new LagDetector(settings, transportService.getThreadPool(), n -> removeNode(n, "lagging"), @@ -191,7 +194,7 @@ public Coordinator(String nodeName, Settings settings, ClusterSettings clusterSe private ClusterFormationState getClusterFormationState() { return new ClusterFormationState(settings, getStateForMasterService(), peerFinder.getLastResolvedAddresses(), Stream.concat(Stream.of(getLocalNode()), StreamSupport.stream(peerFinder.getFoundPeers().spliterator(), false)) - .collect(Collectors.toList()), getCurrentTerm()); + .collect(Collectors.toList()), getCurrentTerm(), electionStrategy); } private void onLeaderFailure(Exception e) { @@ -649,7 +652,7 @@ boolean publicationInProgress() { protected void doStart() { synchronized (mutex) { CoordinationState.PersistedState persistedState = persistedStateSupplier.get(); - coordinationState.set(new CoordinationState(settings, getLocalNode(), persistedState)); + coordinationState.set(new CoordinationState(getLocalNode(), persistedState, electionStrategy)); peerFinder.setCurrentTerm(getCurrentTerm()); configuredHostsResolver.start(); final ClusterState lastAcceptedState = coordinationState.get().getLastAcceptedState(); @@ -1104,9 +1107,7 @@ protected void onFoundPeersUpdated() { final CoordinationState.VoteCollection expectedVotes = new CoordinationState.VoteCollection(); foundPeers.forEach(expectedVotes::addVote); expectedVotes.addVote(Coordinator.this.getLocalNode()); - final ClusterState lastAcceptedState = coordinationState.get().getLastAcceptedState(); - final boolean foundQuorum = CoordinationState.isElectionQuorum(expectedVotes, lastAcceptedState); - + final boolean foundQuorum = coordinationState.get().isElectionQuorum(expectedVotes); if (foundQuorum) { if (electionScheduler == null) { startElectionScheduler(); @@ -1305,6 +1306,7 @@ public void onSuccess(String source) { final List masterCandidates = completedNodes().stream() .filter(DiscoveryNode::isMasterNode) .filter(node -> nodeMayWinElection(state, node)) + .filter(electionStrategy::isAbdicationTarget) .collect(Collectors.toList()); if (masterCandidates.isEmpty() == false) { abdicateTo(masterCandidates.get(random.nextInt(masterCandidates.size()))); @@ -1386,7 +1388,13 @@ protected void onMissingJoin(DiscoveryNode discoveryNode) { @Override protected void sendPublishRequest(DiscoveryNode destination, PublishRequest publishRequest, ActionListener responseActionListener) { - publicationContext.sendPublishRequest(destination, publishRequest, wrapWithMutex(responseActionListener)); + if (electionStrategy.shouldReceivePublication(destination)) { + publicationContext.sendPublishRequest(destination, publishRequest, + electionStrategy.wrapPublishResponseHandler(wrapWithMutex(responseActionListener))); + } else { + logger.debug("sendPublishRequest: suppressing publication to [{}]", destination); + responseActionListener.onFailure(new ElasticsearchException("suppressing publication at source")); + } } @Override diff --git a/server/src/main/java/org/elasticsearch/cluster/coordination/ElectionStrategy.java b/server/src/main/java/org/elasticsearch/cluster/coordination/ElectionStrategy.java new file mode 100644 index 0000000000000..b0879b055994a --- /dev/null +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/ElectionStrategy.java @@ -0,0 +1,85 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.elasticsearch.cluster.coordination; + +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.node.DiscoveryNode; + +import java.util.Collection; + +public interface ElectionStrategy { + /** + * Whether this node should abdicate to the given node when standing down as master + */ + boolean isAbdicationTarget(DiscoveryNode discoveryNode); + + /** + * Returns an extra filter on whether a collection of votes is a good quorum for an election + */ + boolean isGoodQuorum(Collection votingNodes); + + /** + * Whether to accept the given pre-vote + */ + boolean acceptPrevote(PreVoteResponse response, DiscoveryNode sender, ClusterState clusterState); + + /** + * Whether the destination node should receive publications of new cluster states + */ + boolean shouldReceivePublication(DiscoveryNode destination); + + /** + * Allows the strategy to modify the {@link PublishWithJoinResponse} received before it is handled by the {@link Coordinator}. + */ + ActionListener wrapPublishResponseHandler(ActionListener listener); + + class DefaultElectionStrategy implements ElectionStrategy { + + public static final ElectionStrategy INSTANCE = new DefaultElectionStrategy(); + + private DefaultElectionStrategy() { + } + + @Override + public boolean isAbdicationTarget(DiscoveryNode discoveryNode) { + return true; + } + + @Override + public boolean isGoodQuorum(Collection votingNodes) { + return true; + } + + @Override + public boolean acceptPrevote(PreVoteResponse response, DiscoveryNode sender, ClusterState clusterState) { + return true; + } + + @Override + public boolean shouldReceivePublication(DiscoveryNode destination) { + return true; + } + + @Override + public ActionListener wrapPublishResponseHandler(ActionListener listener) { + return listener; + } + } +} diff --git a/server/src/main/java/org/elasticsearch/cluster/coordination/PreVoteCollector.java b/server/src/main/java/org/elasticsearch/cluster/coordination/PreVoteCollector.java index 5001e3be18183..0fc714803bc86 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/PreVoteCollector.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/PreVoteCollector.java @@ -39,7 +39,6 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.function.LongConsumer; -import static org.elasticsearch.cluster.coordination.CoordinationState.isElectionQuorum; import static org.elasticsearch.common.util.concurrent.ConcurrentCollections.newConcurrentSet; public class PreVoteCollector { @@ -51,14 +50,17 @@ public class PreVoteCollector { private final TransportService transportService; private final Runnable startElection; private final LongConsumer updateMaxTermSeen; + private final ElectionStrategy electionStrategy; // Tuple for simple atomic updates. null until the first call to `update()`. private volatile Tuple state; // DiscoveryNode component is null if there is currently no known leader. - PreVoteCollector(final TransportService transportService, final Runnable startElection, final LongConsumer updateMaxTermSeen) { + PreVoteCollector(final TransportService transportService, final Runnable startElection, final LongConsumer updateMaxTermSeen, + final ElectionStrategy electionStrategy) { this.transportService = transportService; this.startElection = startElection; this.updateMaxTermSeen = updateMaxTermSeen; + this.electionStrategy = electionStrategy; // TODO does this need to be on the generic threadpool or can it use SAME? transportService.registerRequestHandler(REQUEST_PRE_VOTE_ACTION_NAME, Names.GENERIC, false, false, @@ -185,11 +187,16 @@ private void handlePreVoteResponse(final PreVoteResponse response, final Discove return; } + if (electionStrategy.acceptPrevote(response, sender, clusterState) == false) { + return; + } + preVotesReceived.add(sender); + final VoteCollection voteCollection = new VoteCollection(); preVotesReceived.forEach(voteCollection::addVote); - if (isElectionQuorum(voteCollection, clusterState) == false) { + if (CoordinationState.isElectionQuorum(voteCollection, clusterState, electionStrategy) == false) { logger.debug("{} added {} from {}, no quorum yet", this, response, sender); return; } diff --git a/server/src/main/java/org/elasticsearch/cluster/coordination/Reconfigurator.java b/server/src/main/java/org/elasticsearch/cluster/coordination/Reconfigurator.java index fd9bab3af11ec..23190b2440c57 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/Reconfigurator.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/Reconfigurator.java @@ -59,8 +59,11 @@ public class Reconfigurator { private volatile boolean autoShrinkVotingConfiguration; - public Reconfigurator(Settings settings, ClusterSettings clusterSettings) { + private final ElectionStrategy electionStrategy; + + public Reconfigurator(Settings settings, ClusterSettings clusterSettings, ElectionStrategy electionStrategy) { autoShrinkVotingConfiguration = CLUSTER_AUTO_SHRINK_VOTING_CONFIGURATION.get(settings); + this.electionStrategy = electionStrategy; clusterSettings.addSettingsUpdateConsumer(CLUSTER_AUTO_SHRINK_VOTING_CONFIGURATION, this::setAutoShrinkVotingConfiguration); } @@ -127,7 +130,7 @@ public VotingConfiguration reconfigure(Set liveNodes, Set .collect(Collectors.toSet())); // new configuration should have a quorum - if (newConfig.hasQuorum(liveNodeIds)) { + if (newConfig.hasQuorum(liveNodeIds) && electionStrategy.isGoodQuorum(liveNodes)) { return newConfig; } else { // If there are not enough live nodes to form a quorum in the newly-proposed configuration, it's better to do nothing. diff --git a/server/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java b/server/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java index 6e3e3f38feb57..2437f1a43dcbc 100644 --- a/server/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java +++ b/server/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java @@ -352,6 +352,7 @@ public void apply(Settings value, Settings current, Settings previous) { Node.INITIAL_STATE_TIMEOUT_SETTING, DiscoveryModule.DISCOVERY_TYPE_SETTING, DiscoveryModule.DISCOVERY_SEED_PROVIDERS_SETTING, + DiscoveryModule.ELECTION_TYPE_SETTING, SettingsBasedSeedHostsProvider.DISCOVERY_SEED_HOSTS_SETTING, SeedHostsResolver.DISCOVERY_SEED_RESOLVER_MAX_CONCURRENT_RESOLVERS_SETTING, SeedHostsResolver.DISCOVERY_SEED_RESOLVER_TIMEOUT_SETTING, diff --git a/server/src/main/java/org/elasticsearch/discovery/DiscoveryModule.java b/server/src/main/java/org/elasticsearch/discovery/DiscoveryModule.java index 51f14147bee9b..889790eca2ee4 100644 --- a/server/src/main/java/org/elasticsearch/discovery/DiscoveryModule.java +++ b/server/src/main/java/org/elasticsearch/discovery/DiscoveryModule.java @@ -23,6 +23,7 @@ import org.apache.logging.log4j.Logger; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.coordination.Coordinator; +import org.elasticsearch.cluster.coordination.ElectionStrategy; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.routing.RoutingService; import org.elasticsearch.cluster.routing.allocation.AllocationService; @@ -75,6 +76,11 @@ public class DiscoveryModule { Setting.listSetting("discovery.seed_providers", Collections.emptyList(), Function.identity(), Property.NodeScope); + public static final String DEFAULT_ELECTION_TYPE = "default"; + + public static final Setting ELECTION_TYPE_SETTING = + new Setting<>("cluster.election.type", DEFAULT_ELECTION_TYPE, Function.identity(), Property.NodeScope); + private final Discovery discovery; public DiscoveryModule(Settings settings, ThreadPool threadPool, TransportService transportService, @@ -86,6 +92,8 @@ public DiscoveryModule(Settings settings, ThreadPool threadPool, TransportServic final Map> hostProviders = new HashMap<>(); hostProviders.put("settings", () -> new SettingsBasedSeedHostsProvider(settings, transportService)); hostProviders.put("file", () -> new FileBasedSeedHostsProvider(configFile)); + final Map electionStrategies = new HashMap<>(); + electionStrategies.put(DEFAULT_ELECTION_TYPE, ElectionStrategy.DefaultElectionStrategy.INSTANCE); for (DiscoveryPlugin plugin : plugins) { plugin.getSeedHostProviders(transportService, networkService).forEach((key, value) -> { if (hostProviders.put(key, value) != null) { @@ -96,6 +104,11 @@ public DiscoveryModule(Settings settings, ThreadPool threadPool, TransportServic if (joinValidator != null) { joinValidators.add(joinValidator); } + plugin.getElectionStrategies().forEach((key, value) -> { + if (electionStrategies.put(key, value) != null) { + throw new IllegalArgumentException("Cannot register election strategy [" + key + "] twice"); + } + }); } List seedProviderNames = DISCOVERY_SEED_PROVIDERS_SETTING.get(settings); @@ -126,12 +139,17 @@ public DiscoveryModule(Settings settings, ThreadPool threadPool, TransportServic return Collections.unmodifiableList(addresses); }; + final ElectionStrategy electionStrategy = electionStrategies.get(ELECTION_TYPE_SETTING.get(settings)); + if (electionStrategy == null) { + throw new IllegalArgumentException("Unknown election type " + ELECTION_TYPE_SETTING.get(settings)); + } + if (ZEN2_DISCOVERY_TYPE.equals(discoveryType) || SINGLE_NODE_DISCOVERY_TYPE.equals(discoveryType)) { discovery = new Coordinator(NODE_NAME_SETTING.get(settings), settings, clusterSettings, transportService, namedWriteableRegistry, allocationService, masterService, () -> gatewayMetaState.getPersistedState(settings, (ClusterApplierService) clusterApplier), seedHostsProvider, - clusterApplier, joinValidators, new Random(Randomness.get().nextLong()), routingService::reroute); + clusterApplier, joinValidators, new Random(Randomness.get().nextLong()), routingService::reroute, electionStrategy); } else { throw new IllegalArgumentException("Unknown discovery type [" + discoveryType + "]"); } diff --git a/server/src/main/java/org/elasticsearch/plugins/DiscoveryPlugin.java b/server/src/main/java/org/elasticsearch/plugins/DiscoveryPlugin.java index 21cfbd0e99e8d..97b6308289311 100644 --- a/server/src/main/java/org/elasticsearch/plugins/DiscoveryPlugin.java +++ b/server/src/main/java/org/elasticsearch/plugins/DiscoveryPlugin.java @@ -20,6 +20,7 @@ package org.elasticsearch.plugins; import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.coordination.ElectionStrategy; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.common.network.NetworkService; import org.elasticsearch.common.settings.Settings; @@ -85,4 +86,8 @@ default Map> getSeedHostProviders(TransportS * {@link IllegalStateException} if the node and the cluster-state are incompatible. */ default BiConsumer getJoinValidator() { return null; } + + default Map getElectionStrategies() { + return Collections.emptyMap(); + } } diff --git a/server/src/test/java/org/elasticsearch/cluster/coordination/ClusterFormationFailureHelperTests.java b/server/src/test/java/org/elasticsearch/cluster/coordination/ClusterFormationFailureHelperTests.java index f16c7a7878ea9..eabfb0ed2d6d8 100644 --- a/server/src/test/java/org/elasticsearch/cluster/coordination/ClusterFormationFailureHelperTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/coordination/ClusterFormationFailureHelperTests.java @@ -45,9 +45,12 @@ import static org.elasticsearch.node.Node.NODE_NAME_SETTING; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.is; -import static org.hamcrest.Matchers.isOneOf; +import static org.hamcrest.Matchers.oneOf; public class ClusterFormationFailureHelperTests extends ESTestCase { + + private static final ElectionStrategy electionStrategy = ElectionStrategy.DefaultElectionStrategy.INSTANCE; + public void testScheduling() { final long expectedDelayMillis; final Settings.Builder settingsBuilder = Settings.builder(); @@ -73,7 +76,7 @@ public void testScheduling() { final ClusterFormationFailureHelper clusterFormationFailureHelper = new ClusterFormationFailureHelper(settingsBuilder.build(), () -> { warningCount.incrementAndGet(); - return new ClusterFormationState(Settings.EMPTY, clusterState, emptyList(), emptyList(), 0L); + return new ClusterFormationState(Settings.EMPTY, clusterState, emptyList(), emptyList(), 0L, electionStrategy); }, deterministicTaskQueue.getThreadPool(), logLastFailedJoinAttemptWarningCount::incrementAndGet); @@ -141,17 +144,17 @@ public void testDescriptionOnMasterIneligibleNodes() { final ClusterState clusterState = ClusterState.builder(ClusterName.DEFAULT) .version(12L).nodes(DiscoveryNodes.builder().add(localNode).localNodeId(localNode.getId())).build(); - assertThat(new ClusterFormationState(Settings.EMPTY, clusterState, emptyList(), emptyList(), 15L).getDescription(), + assertThat(new ClusterFormationState(Settings.EMPTY, clusterState, emptyList(), emptyList(), 15L, electionStrategy).getDescription(), is("master not discovered yet: have discovered []; discovery will continue using [] from hosts providers " + "and [] from last-known cluster state; node term 15, last-accepted version 12 in term 0")); final TransportAddress otherAddress = buildNewFakeTransportAddress(); - assertThat(new ClusterFormationState(Settings.EMPTY, clusterState, singletonList(otherAddress), emptyList(), 16L).getDescription(), + assertThat(new ClusterFormationState(Settings.EMPTY, clusterState, singletonList(otherAddress), emptyList(), 16L, electionStrategy).getDescription(), is("master not discovered yet: have discovered []; discovery will continue using [" + otherAddress + "] from hosts providers and [] from last-known cluster state; node term 16, last-accepted version 12 in term 0")); final DiscoveryNode otherNode = new DiscoveryNode("other", buildNewFakeTransportAddress(), Version.CURRENT); - assertThat(new ClusterFormationState(Settings.EMPTY, clusterState, emptyList(), singletonList(otherNode), 17L).getDescription(), + assertThat(new ClusterFormationState(Settings.EMPTY, clusterState, emptyList(), singletonList(otherNode), 17L, electionStrategy).getDescription(), is("master not discovered yet: have discovered [" + otherNode + "]; discovery will continue using [] from hosts providers " + "and [] from last-known cluster state; node term 17, last-accepted version 12 in term 0")); } @@ -163,28 +166,28 @@ public void testDescriptionBeforeBootstrapping() { .metaData(MetaData.builder().coordinationMetaData(CoordinationMetaData.builder().term(4L).build())) .nodes(DiscoveryNodes.builder().add(localNode).localNodeId(localNode.getId())).build(); - assertThat(new ClusterFormationState(Settings.EMPTY, clusterState, emptyList(), emptyList(), 1L).getDescription(), + assertThat(new ClusterFormationState(Settings.EMPTY, clusterState, emptyList(), emptyList(), 1L, electionStrategy).getDescription(), is("master not discovered yet, this node has not previously joined a bootstrapped cluster, and " + "[cluster.initial_master_nodes] is empty on this node: have discovered []; " + "discovery will continue using [] from hosts providers and [" + localNode + "] from last-known cluster state; node term 1, last-accepted version 7 in term 4")); final TransportAddress otherAddress = buildNewFakeTransportAddress(); - assertThat(new ClusterFormationState(Settings.EMPTY, clusterState, singletonList(otherAddress), emptyList(), 2L).getDescription(), + assertThat(new ClusterFormationState(Settings.EMPTY, clusterState, singletonList(otherAddress), emptyList(), 2L, electionStrategy).getDescription(), is("master not discovered yet, this node has not previously joined a bootstrapped cluster, and " + "[cluster.initial_master_nodes] is empty on this node: have discovered []; " + "discovery will continue using [" + otherAddress + "] from hosts providers and [" + localNode + "] from last-known cluster state; node term 2, last-accepted version 7 in term 4")); final DiscoveryNode otherNode = new DiscoveryNode("other", buildNewFakeTransportAddress(), Version.CURRENT); - assertThat(new ClusterFormationState(Settings.EMPTY, clusterState, emptyList(), singletonList(otherNode), 3L).getDescription(), + assertThat(new ClusterFormationState(Settings.EMPTY, clusterState, emptyList(), singletonList(otherNode), 3L, electionStrategy).getDescription(), is("master not discovered yet, this node has not previously joined a bootstrapped cluster, and " + "[cluster.initial_master_nodes] is empty on this node: have discovered [" + otherNode + "]; " + "discovery will continue using [] from hosts providers and [" + localNode + "] from last-known cluster state; node term 3, last-accepted version 7 in term 4")); assertThat(new ClusterFormationState(Settings.builder().putList(INITIAL_MASTER_NODES_SETTING.getKey(), "other").build(), - clusterState, emptyList(), emptyList(), 4L).getDescription(), + clusterState, emptyList(), emptyList(), 4L, electionStrategy).getDescription(), is("master not discovered yet, this node has not previously joined a bootstrapped cluster, and " + "this node must discover master-eligible nodes [other] to bootstrap a cluster: have discovered []; " + "discovery will continue using [] from hosts providers and [" + localNode + @@ -214,28 +217,28 @@ public void testDescriptionAfterDetachCluster() { final ClusterState clusterState = state(localNode, VotingConfiguration.MUST_JOIN_ELECTED_MASTER.getNodeIds().toArray(new String[0])); - assertThat(new ClusterFormationState(Settings.EMPTY, clusterState, emptyList(), emptyList(), 0L).getDescription(), + assertThat(new ClusterFormationState(Settings.EMPTY, clusterState, emptyList(), emptyList(), 0L, electionStrategy).getDescription(), is("master not discovered yet and this node was detached from its previous cluster, " + "have discovered []; " + "discovery will continue using [] from hosts providers and [" + localNode + "] from last-known cluster state; node term 0, last-accepted version 0 in term 0")); final TransportAddress otherAddress = buildNewFakeTransportAddress(); - assertThat(new ClusterFormationState(Settings.EMPTY, clusterState, singletonList(otherAddress), emptyList(), 0L).getDescription(), + assertThat(new ClusterFormationState(Settings.EMPTY, clusterState, singletonList(otherAddress), emptyList(), 0L, electionStrategy).getDescription(), is("master not discovered yet and this node was detached from its previous cluster, " + "have discovered []; " + "discovery will continue using [" + otherAddress + "] from hosts providers and [" + localNode + "] from last-known cluster state; node term 0, last-accepted version 0 in term 0")); final DiscoveryNode otherNode = new DiscoveryNode("otherNode", buildNewFakeTransportAddress(), Version.CURRENT); - assertThat(new ClusterFormationState(Settings.EMPTY, clusterState, emptyList(), singletonList(otherNode), 0L).getDescription(), + assertThat(new ClusterFormationState(Settings.EMPTY, clusterState, emptyList(), singletonList(otherNode), 0L, electionStrategy).getDescription(), is("master not discovered yet and this node was detached from its previous cluster, " + "have discovered [" + otherNode + "]; " + "discovery will continue using [] from hosts providers and [" + localNode + "] from last-known cluster state; node term 0, last-accepted version 0 in term 0")); final DiscoveryNode yetAnotherNode = new DiscoveryNode("yetAnotherNode", buildNewFakeTransportAddress(), Version.CURRENT); - assertThat(new ClusterFormationState(Settings.EMPTY, clusterState, emptyList(), singletonList(yetAnotherNode), 0L).getDescription(), + assertThat(new ClusterFormationState(Settings.EMPTY, clusterState, emptyList(), singletonList(yetAnotherNode), 0L, electionStrategy).getDescription(), is("master not discovered yet and this node was detached from its previous cluster, " + "have discovered [" + yetAnotherNode + "]; " + "discovery will continue using [] from hosts providers and [" + localNode + @@ -248,40 +251,40 @@ public void testDescriptionAfterBootstrapping() { final ClusterState clusterState = state(localNode, "otherNode"); - assertThat(new ClusterFormationState(Settings.EMPTY, clusterState, emptyList(), emptyList(), 0L).getDescription(), + assertThat(new ClusterFormationState(Settings.EMPTY, clusterState, emptyList(), emptyList(), 0L, electionStrategy).getDescription(), is("master not discovered or elected yet, an election requires a node with id [otherNode], " + "have discovered [] which is not a quorum; " + "discovery will continue using [] from hosts providers and [" + localNode + "] from last-known cluster state; node term 0, last-accepted version 0 in term 0")); final TransportAddress otherAddress = buildNewFakeTransportAddress(); - assertThat(new ClusterFormationState(Settings.EMPTY, clusterState, singletonList(otherAddress), emptyList(), 0L).getDescription(), + assertThat(new ClusterFormationState(Settings.EMPTY, clusterState, singletonList(otherAddress), emptyList(), 0L, electionStrategy).getDescription(), is("master not discovered or elected yet, an election requires a node with id [otherNode], " + "have discovered [] which is not a quorum; " + "discovery will continue using [" + otherAddress + "] from hosts providers and [" + localNode + "] from last-known cluster state; node term 0, last-accepted version 0 in term 0")); final DiscoveryNode otherNode = new DiscoveryNode("otherNode", buildNewFakeTransportAddress(), Version.CURRENT); - assertThat(new ClusterFormationState(Settings.EMPTY, clusterState, emptyList(), singletonList(otherNode), 0L).getDescription(), + assertThat(new ClusterFormationState(Settings.EMPTY, clusterState, emptyList(), singletonList(otherNode), 0L, electionStrategy).getDescription(), is("master not discovered or elected yet, an election requires a node with id [otherNode], " + "have discovered [" + otherNode + "] which is a quorum; " + "discovery will continue using [] from hosts providers and [" + localNode + "] from last-known cluster state; node term 0, last-accepted version 0 in term 0")); final DiscoveryNode yetAnotherNode = new DiscoveryNode("yetAnotherNode", buildNewFakeTransportAddress(), Version.CURRENT); - assertThat(new ClusterFormationState(Settings.EMPTY, clusterState, emptyList(), singletonList(yetAnotherNode), 0L).getDescription(), + assertThat(new ClusterFormationState(Settings.EMPTY, clusterState, emptyList(), singletonList(yetAnotherNode), 0L, electionStrategy).getDescription(), is("master not discovered or elected yet, an election requires a node with id [otherNode], " + "have discovered [" + yetAnotherNode + "] which is not a quorum; " + "discovery will continue using [] from hosts providers and [" + localNode + "] from last-known cluster state; node term 0, last-accepted version 0 in term 0")); - assertThat(new ClusterFormationState(Settings.EMPTY, state(localNode, "n1", "n2"), emptyList(), emptyList(), 0L).getDescription(), + assertThat(new ClusterFormationState(Settings.EMPTY, state(localNode, "n1", "n2"), emptyList(), emptyList(), 0L, electionStrategy).getDescription(), is("master not discovered or elected yet, an election requires two nodes with ids [n1, n2], " + "have discovered [] which is not a quorum; " + "discovery will continue using [] from hosts providers and [" + localNode + "] from last-known cluster state; node term 0, last-accepted version 0 in term 0")); - assertThat(new ClusterFormationState(Settings.EMPTY, state(localNode, "n1", "n2", "n3"), emptyList(), emptyList(), 0L) + assertThat(new ClusterFormationState(Settings.EMPTY, state(localNode, "n1", "n2", "n3"), emptyList(), emptyList(), 0L, electionStrategy) .getDescription(), is("master not discovered or elected yet, an election requires at least 2 nodes with ids from [n1, n2, n3], " + "have discovered [] which is not a quorum; " + @@ -289,20 +292,20 @@ public void testDescriptionAfterBootstrapping() { "] from last-known cluster state; node term 0, last-accepted version 0 in term 0")); assertThat(new ClusterFormationState(Settings.EMPTY, state(localNode, "n1", "n2", BOOTSTRAP_PLACEHOLDER_PREFIX + "n3"), - emptyList(), emptyList(), 0L).getDescription(), + emptyList(), emptyList(), 0L, electionStrategy).getDescription(), is("master not discovered or elected yet, an election requires 2 nodes with ids [n1, n2], " + "have discovered [] which is not a quorum; " + "discovery will continue using [] from hosts providers and [" + localNode + "] from last-known cluster state; node term 0, last-accepted version 0 in term 0")); - assertThat(new ClusterFormationState(Settings.EMPTY, state(localNode, "n1", "n2", "n3", "n4"), emptyList(), emptyList(), 0L) + assertThat(new ClusterFormationState(Settings.EMPTY, state(localNode, "n1", "n2", "n3", "n4"), emptyList(), emptyList(), 0L, electionStrategy) .getDescription(), is("master not discovered or elected yet, an election requires at least 3 nodes with ids from [n1, n2, n3, n4], " + "have discovered [] which is not a quorum; " + "discovery will continue using [] from hosts providers and [" + localNode + "] from last-known cluster state; node term 0, last-accepted version 0 in term 0")); - assertThat(new ClusterFormationState(Settings.EMPTY, state(localNode, "n1", "n2", "n3", "n4", "n5"), emptyList(), emptyList(), 0L) + assertThat(new ClusterFormationState(Settings.EMPTY, state(localNode, "n1", "n2", "n3", "n4", "n5"), emptyList(), emptyList(), 0L, electionStrategy) .getDescription(), is("master not discovered or elected yet, an election requires at least 3 nodes with ids from [n1, n2, n3, n4, n5], " + "have discovered [] which is not a quorum; " + @@ -310,42 +313,42 @@ public void testDescriptionAfterBootstrapping() { "] from last-known cluster state; node term 0, last-accepted version 0 in term 0")); assertThat(new ClusterFormationState(Settings.EMPTY, state(localNode, "n1", "n2", "n3", "n4", BOOTSTRAP_PLACEHOLDER_PREFIX + "n5"), - emptyList(), emptyList(), 0L).getDescription(), + emptyList(), emptyList(), 0L, electionStrategy).getDescription(), is("master not discovered or elected yet, an election requires at least 3 nodes with ids from [n1, n2, n3, n4], " + "have discovered [] which is not a quorum; " + "discovery will continue using [] from hosts providers and [" + localNode + "] from last-known cluster state; node term 0, last-accepted version 0 in term 0")); assertThat(new ClusterFormationState(Settings.EMPTY, state(localNode, "n1", "n2", "n3", - BOOTSTRAP_PLACEHOLDER_PREFIX + "n4", BOOTSTRAP_PLACEHOLDER_PREFIX + "n5"), emptyList(), emptyList(), 0L).getDescription(), + BOOTSTRAP_PLACEHOLDER_PREFIX + "n4", BOOTSTRAP_PLACEHOLDER_PREFIX + "n5"), emptyList(), emptyList(), 0L, electionStrategy).getDescription(), is("master not discovered or elected yet, an election requires 3 nodes with ids [n1, n2, n3], " + "have discovered [] which is not a quorum; " + "discovery will continue using [] from hosts providers and [" + localNode + "] from last-known cluster state; node term 0, last-accepted version 0 in term 0")); assertThat(new ClusterFormationState(Settings.EMPTY, state(localNode, new String[]{"n1"}, new String[]{"n1"}), emptyList(), - emptyList(), 0L).getDescription(), + emptyList(), 0L, electionStrategy).getDescription(), is("master not discovered or elected yet, an election requires a node with id [n1], " + "have discovered [] which is not a quorum; " + "discovery will continue using [] from hosts providers and [" + localNode + "] from last-known cluster state; node term 0, last-accepted version 0 in term 0")); assertThat(new ClusterFormationState(Settings.EMPTY, state(localNode, new String[]{"n1"}, new String[]{"n2"}), emptyList(), - emptyList(), 0L).getDescription(), + emptyList(), 0L, electionStrategy).getDescription(), is("master not discovered or elected yet, an election requires a node with id [n1] and a node with id [n2], " + "have discovered [] which is not a quorum; " + "discovery will continue using [] from hosts providers and [" + localNode + "] from last-known cluster state; node term 0, last-accepted version 0 in term 0")); assertThat(new ClusterFormationState(Settings.EMPTY, state(localNode, new String[]{"n1"}, new String[]{"n2", "n3"}), emptyList(), - emptyList(), 0L).getDescription(), + emptyList(), 0L, electionStrategy).getDescription(), is("master not discovered or elected yet, an election requires a node with id [n1] and two nodes with ids [n2, n3], " + "have discovered [] which is not a quorum; " + "discovery will continue using [] from hosts providers and [" + localNode + "] from last-known cluster state; node term 0, last-accepted version 0 in term 0")); assertThat(new ClusterFormationState(Settings.EMPTY, state(localNode, new String[]{"n1"}, new String[]{"n2", "n3", "n4"}), - emptyList(), emptyList(), 0L).getDescription(), + emptyList(), emptyList(), 0L, electionStrategy).getDescription(), is("master not discovered or elected yet, an election requires a node with id [n1] and " + "at least 2 nodes with ids from [n2, n3, n4], " + "have discovered [] which is not a quorum; " + @@ -365,18 +368,19 @@ public void testDescriptionAfterBootstrapping() { .lastAcceptedConfiguration(config(configNodeIds)) .lastCommittedConfiguration(config(configNodeIds)).build())).build(); - assertThat(new ClusterFormationState(Settings.EMPTY, stateWithOtherNodes, emptyList(), emptyList(), 0L).getDescription(), isOneOf( + assertThat( + new ClusterFormationState(Settings.EMPTY, stateWithOtherNodes, emptyList(), emptyList(), 0L, electionStrategy).getDescription(), // nodes from last-known cluster state could be in either order - - "master not discovered or elected yet, an election requires two nodes with ids [n1, n2], " + - "have discovered [] which is not a quorum; " + - "discovery will continue using [] from hosts providers and [" + localNode + ", " + otherMasterNode + - "] from last-known cluster state; node term 0, last-accepted version 0 in term 0", - - "master not discovered or elected yet, an election requires two nodes with ids [n1, n2], " + - "have discovered [] which is not a quorum; " + - "discovery will continue using [] from hosts providers and [" + otherMasterNode + ", " + localNode + - "] from last-known cluster state; node term 0, last-accepted version 0 in term 0")); + is(oneOf( + "master not discovered or elected yet, an election requires two nodes with ids [n1, n2], " + + "have discovered [] which is not a quorum; " + + "discovery will continue using [] from hosts providers and [" + localNode + ", " + otherMasterNode + + "] from last-known cluster state; node term 0, last-accepted version 0 in term 0", + + "master not discovered or elected yet, an election requires two nodes with ids [n1, n2], " + + "have discovered [] which is not a quorum; " + + "discovery will continue using [] from hosts providers and [" + otherMasterNode + ", " + localNode + + "] from last-known cluster state; node term 0, last-accepted version 0 in term 0"))); } } diff --git a/server/src/test/java/org/elasticsearch/cluster/coordination/CoordinationStateTests.java b/server/src/test/java/org/elasticsearch/cluster/coordination/CoordinationStateTests.java index 5bbd3dfd24993..c52a2a36f34c0 100644 --- a/server/src/test/java/org/elasticsearch/cluster/coordination/CoordinationStateTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/coordination/CoordinationStateTests.java @@ -32,7 +32,6 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.transport.TransportAddress; import org.elasticsearch.common.util.set.Sets; -import org.elasticsearch.node.Node; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.EqualsHashCodeTestUtils; import org.junit.Before; @@ -770,8 +769,7 @@ public void testSafety() { } public static CoordinationState createCoordinationState(PersistedState storage, DiscoveryNode localNode) { - final Settings initialSettings = Settings.builder().put(Node.NODE_NAME_SETTING.getKey(), localNode.getId()).build(); - return new CoordinationState(initialSettings, localNode, storage); + return new CoordinationState(localNode, storage, ElectionStrategy.DefaultElectionStrategy.INSTANCE); } public static ClusterState clusterState(long term, long version, DiscoveryNode localNode, VotingConfiguration lastCommittedConfig, @@ -821,11 +819,11 @@ static class ClusterNode { this.localNode = localNode; persistedState = new InMemoryPersistedState(0L, clusterState(0L, 0L, localNode, VotingConfiguration.EMPTY_CONFIG, VotingConfiguration.EMPTY_CONFIG, 0L)); - state = new CoordinationState(Settings.EMPTY, localNode, persistedState); + state = new CoordinationState(localNode, persistedState, ElectionStrategy.DefaultElectionStrategy.INSTANCE); } void reboot() { - state = new CoordinationState(Settings.EMPTY, localNode, persistedState); + state = new CoordinationState(localNode, persistedState, ElectionStrategy.DefaultElectionStrategy.INSTANCE); } void setInitialState(VotingConfiguration initialConfig, long initialValue) { diff --git a/server/src/test/java/org/elasticsearch/cluster/coordination/CoordinatorTests.java b/server/src/test/java/org/elasticsearch/cluster/coordination/CoordinatorTests.java index 9d654d29f6c81..cddb70a23f114 100644 --- a/server/src/test/java/org/elasticsearch/cluster/coordination/CoordinatorTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/coordination/CoordinatorTests.java @@ -1997,7 +1997,8 @@ protected Optional getDisruptableMockTransport(Transpo final AllocationService allocationService = ESAllocationTestCase.createAllocationService(Settings.EMPTY); coordinator = new Coordinator("test_node", settings, clusterSettings, transportService, writableRegistry(), allocationService, masterService, this::getPersistedState, - Cluster.this::provideSeedHosts, clusterApplierService, onJoinValidators, Randomness.get(), s -> {}); + Cluster.this::provideSeedHosts, clusterApplierService, onJoinValidators, Randomness.get(), s -> {}, + ElectionStrategy.DefaultElectionStrategy.INSTANCE); masterService.setClusterStatePublisher(coordinator); final GatewayService gatewayService = new GatewayService(settings, allocationService, clusterService, deterministicTaskQueue.getThreadPool(this::onNode), null, coordinator); diff --git a/server/src/test/java/org/elasticsearch/cluster/coordination/NodeJoinTests.java b/server/src/test/java/org/elasticsearch/cluster/coordination/NodeJoinTests.java index 5bbdd020433b0..65d6af302da1b 100644 --- a/server/src/test/java/org/elasticsearch/cluster/coordination/NodeJoinTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/coordination/NodeJoinTests.java @@ -176,7 +176,7 @@ transportService, writableRegistry(), () -> new InMemoryPersistedState(term, initialState), r -> emptyList(), new NoOpClusterApplier(), Collections.emptyList(), - random, s -> {}); + random, s -> {}, ElectionStrategy.DefaultElectionStrategy.INSTANCE); transportService.start(); transportService.acceptIncomingRequests(); transport = capturingTransport; diff --git a/server/src/test/java/org/elasticsearch/cluster/coordination/PreVoteCollectorTests.java b/server/src/test/java/org/elasticsearch/cluster/coordination/PreVoteCollectorTests.java index 78896752bae4d..ec5d5f16f3b3d 100644 --- a/server/src/test/java/org/elasticsearch/cluster/coordination/PreVoteCollectorTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/coordination/PreVoteCollectorTests.java @@ -114,7 +114,7 @@ public String toString() { assert electionOccurred == false; electionOccurred = true; }, l -> { - }); // TODO need tests that check that the max term seen is updated + }, ElectionStrategy.DefaultElectionStrategy.INSTANCE); // TODO need tests that check that the max term seen is updated preVoteCollector.update(getLocalPreVoteResponse(), null); } @@ -233,8 +233,8 @@ public void testPrevotingIndicatesElectionSuccess() { DiscoveryNode[] votingNodes = votingNodesSet.toArray(new DiscoveryNode[0]); startAndRunCollector(votingNodes); - final CoordinationState coordinationState = new CoordinationState(Settings.EMPTY, localNode, - new InMemoryPersistedState(currentTerm, makeClusterState(votingNodes))); + final CoordinationState coordinationState = new CoordinationState(localNode, + new InMemoryPersistedState(currentTerm, makeClusterState(votingNodes)), ElectionStrategy.DefaultElectionStrategy.INSTANCE); final long newTerm = randomLongBetween(currentTerm + 1, Long.MAX_VALUE); diff --git a/server/src/test/java/org/elasticsearch/cluster/coordination/PublicationTests.java b/server/src/test/java/org/elasticsearch/cluster/coordination/PublicationTests.java index 5fb56cc4aa3a3..c3367d10ac4d8 100644 --- a/server/src/test/java/org/elasticsearch/cluster/coordination/PublicationTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/coordination/PublicationTests.java @@ -72,7 +72,8 @@ class MockNode { this.localNode = localNode; ClusterState initialState = CoordinationStateTests.clusterState(0L, 0L, localNode, CoordinationMetaData.VotingConfiguration.EMPTY_CONFIG, CoordinationMetaData.VotingConfiguration.EMPTY_CONFIG, 0L); - coordinationState = new CoordinationState(settings, localNode, new InMemoryPersistedState(0L, initialState)); + coordinationState = new CoordinationState(localNode, new InMemoryPersistedState(0L, initialState), + ElectionStrategy.DefaultElectionStrategy.INSTANCE); } final DiscoveryNode localNode; diff --git a/server/src/test/java/org/elasticsearch/cluster/coordination/ReconfiguratorTests.java b/server/src/test/java/org/elasticsearch/cluster/coordination/ReconfiguratorTests.java index a1d12d98398ca..8d535a5f1bd29 100644 --- a/server/src/test/java/org/elasticsearch/cluster/coordination/ReconfiguratorTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/coordination/ReconfiguratorTests.java @@ -210,12 +210,14 @@ private void check(Set liveNodes, Set retired, String mas } private Reconfigurator makeReconfigurator(Settings settings) { - return new Reconfigurator(settings, new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS)); + return new Reconfigurator(settings, new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS), + ElectionStrategy.DefaultElectionStrategy.INSTANCE); } public void testDynamicSetting() { final ClusterSettings clusterSettings = new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); - final Reconfigurator reconfigurator = new Reconfigurator(Settings.EMPTY, clusterSettings); + final Reconfigurator reconfigurator = new Reconfigurator(Settings.EMPTY, clusterSettings, + ElectionStrategy.DefaultElectionStrategy.INSTANCE); final VotingConfiguration initialConfig = conf("a", "b", "c", "d", "e"); Set twoNodes = nodes("a", "b"); diff --git a/server/src/test/java/org/elasticsearch/snapshots/SnapshotResiliencyTests.java b/server/src/test/java/org/elasticsearch/snapshots/SnapshotResiliencyTests.java index 6c6b136ba2751..084348ce12717 100644 --- a/server/src/test/java/org/elasticsearch/snapshots/SnapshotResiliencyTests.java +++ b/server/src/test/java/org/elasticsearch/snapshots/SnapshotResiliencyTests.java @@ -87,6 +87,7 @@ import org.elasticsearch.cluster.coordination.Coordinator; import org.elasticsearch.cluster.coordination.CoordinatorTests; import org.elasticsearch.cluster.coordination.DeterministicTaskQueue; +import org.elasticsearch.cluster.coordination.ElectionStrategy; import org.elasticsearch.cluster.coordination.InMemoryPersistedState; import org.elasticsearch.cluster.coordination.MockSinglePrioritizingExecutor; import org.elasticsearch.cluster.metadata.AliasValidator; @@ -1246,7 +1247,7 @@ public void start(ClusterState initialState) { hostsResolver -> testClusterNodes.nodes.values().stream().filter(n -> n.node.isMasterNode()) .map(n -> n.node.getAddress()).collect(Collectors.toList()), clusterService.getClusterApplierService(), Collections.emptyList(), random(), - new RoutingService(clusterService, allocationService)::reroute); + new RoutingService(clusterService, allocationService)::reroute, ElectionStrategy.DefaultElectionStrategy.INSTANCE); masterService.setClusterStatePublisher(coordinator); coordinator.start(); masterService.start(); diff --git a/x-pack/plugin/voting-only-node/build.gradle b/x-pack/plugin/voting-only-node/build.gradle new file mode 100644 index 0000000000000..c1efe638a56ae --- /dev/null +++ b/x-pack/plugin/voting-only-node/build.gradle @@ -0,0 +1,21 @@ +evaluationDependsOn(xpackModule('core')) + +apply plugin: 'elasticsearch.esplugin' +esplugin { + name 'x-pack-voting-only-node' + description 'Elasticsearch Expanded Pack Plugin - Voting-only node' + classname 'org.elasticsearch.cluster.coordination.VotingOnlyNodePlugin' + extendedPlugins = ['x-pack-core'] +} +archivesBaseName = 'x-pack-voting-only-node' + +dependencies { + compileOnly "org.elasticsearch:elasticsearch:${version}" + + // "org.elasticsearch.plugin:x-pack-core:${version}" doesn't work with idea because the testArtifacts are also here + compileOnly project(path: xpackModule('core'), configuration: 'default') + testCompile project(path: xpackModule('core'), configuration: 'testArtifacts') + if (isEclipse) { + testCompile project(path: xpackModule('core-tests'), configuration: 'testArtifacts') + } +} diff --git a/x-pack/plugin/voting-only-node/src/main/java/org/elasticsearch/cluster/coordination/VotingOnlyNodePlugin.java b/x-pack/plugin/voting-only-node/src/main/java/org/elasticsearch/cluster/coordination/VotingOnlyNodePlugin.java new file mode 100644 index 0000000000000..bf8fb2a019dd5 --- /dev/null +++ b/x-pack/plugin/voting-only-node/src/main/java/org/elasticsearch/cluster/coordination/VotingOnlyNodePlugin.java @@ -0,0 +1,152 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License; + * you may not use this file except in compliance with the Elastic License. + */ +package org.elasticsearch.cluster.coordination; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.elasticsearch.ElasticsearchException; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.node.DiscoveryNode; +import org.elasticsearch.cluster.node.DiscoveryNodeRole; +import org.elasticsearch.common.settings.Setting; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.discovery.DiscoveryModule; +import org.elasticsearch.node.Node; +import org.elasticsearch.plugins.DiscoveryPlugin; +import org.elasticsearch.plugins.Plugin; + +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Set; + +public class VotingOnlyNodePlugin extends Plugin implements DiscoveryPlugin { + + private static final Logger logger = LogManager.getLogger(VotingOnlyNodePlugin.class); + + public static final Setting VOTING_ONLY_NODE_SETTING + = Setting.boolSetting("node.voting_only", false, Setting.Property.NodeScope); + + private final Boolean isVotingOnlyNode; + + public VotingOnlyNodePlugin(Settings settings) { + isVotingOnlyNode = VOTING_ONLY_NODE_SETTING.get(settings); + if (isVotingOnlyNode && Node.NODE_MASTER_SETTING.get(settings) == false) { + throw new IllegalStateException("voting-only node must be master-eligible"); + } + } + + @Override + public Set getRoles() { + return Collections.singleton(VOTING_ONLY_NODE_ROLE); + } + + @Override + public List> getSettings() { + return Collections.singletonList(VOTING_ONLY_NODE_SETTING); + } + + private static final String VOTING_ONLY_ELECTION_TYPE = "supports-voting-only"; + + @Override + public Settings additionalSettings() { + return Settings.builder().put(DiscoveryModule.ELECTION_TYPE_SETTING.getKey(), VOTING_ONLY_ELECTION_TYPE).build(); + } + + static DiscoveryNodeRole VOTING_ONLY_NODE_ROLE = new DiscoveryNodeRole("voting-only", "v") { + @Override + protected Setting roleSetting() { + return VOTING_ONLY_NODE_SETTING; + } + }; + + private static boolean isTrueMasterNode(DiscoveryNode discoveryNode) { + return discoveryNode.isMasterNode() && discoveryNode.getRoles().contains(VOTING_ONLY_NODE_ROLE) == false; + } + + private class AvoidsVotingOnlyNodesElectionStrategy implements ElectionStrategy { + @Override + public boolean isAbdicationTarget(DiscoveryNode discoveryNode) { + return isTrueMasterNode(discoveryNode); + } + + @Override + public boolean isGoodQuorum(Collection votingNodes) { + return true; + } + + @Override + public boolean acceptPrevote(PreVoteResponse response, DiscoveryNode sender, ClusterState clusterState) { + return true; + } + + @Override + public boolean shouldReceivePublication(DiscoveryNode destination) { + return true; + } + + @Override + public ActionListener wrapPublishResponseHandler(ActionListener listener) { + return listener; + } + } + + private class VotingOnlyNodeElectionStrategy implements ElectionStrategy { + + @Override + public boolean isAbdicationTarget(DiscoveryNode discoveryNode) { + assert false : "voting-only node cannot abdicate"; + return false; + } + + + @Override + public boolean isGoodQuorum(Collection votingNodes) { + return votingNodes.stream().anyMatch(VotingOnlyNodePlugin::isTrueMasterNode); + } + + @Override + public boolean acceptPrevote(PreVoteResponse response, DiscoveryNode sender, ClusterState clusterState) { + if (sender.getRoles().contains(VOTING_ONLY_NODE_ROLE) == false + && response.getLastAcceptedTerm() == clusterState.term() + && response.getLastAcceptedVersion() == clusterState.version()) { + logger.debug("{} ignoring {} from {} as it has higher voting priority", this, response, sender); + return false; + } + + return true; + } + + @Override + public boolean shouldReceivePublication(DiscoveryNode destination) { + return isTrueMasterNode(destination); + } + + @Override + public ActionListener wrapPublishResponseHandler(ActionListener listener) { + return new ActionListener<>() { + @Override + public void onResponse(PublishWithJoinResponse publishWithJoinResponse) { + listener.onFailure(new ElasticsearchException( + "ignoring successful publish response on voting-only node: " + publishWithJoinResponse)); + } + + @Override + public void onFailure(Exception e) { + listener.onFailure(e); + } + }; + } + } + + @Override + public Map getElectionStrategies() { + return Collections.singletonMap(VOTING_ONLY_ELECTION_TYPE, + isVotingOnlyNode ? new VotingOnlyNodeElectionStrategy() : new AvoidsVotingOnlyNodesElectionStrategy()); + } +} diff --git a/x-pack/plugin/voting-only-node/src/test/java/org/elasticsearch/cluster/coordination/VotingOnlyNodePluginIT.java b/x-pack/plugin/voting-only-node/src/test/java/org/elasticsearch/cluster/coordination/VotingOnlyNodePluginIT.java new file mode 100644 index 0000000000000..c408b2d390b69 --- /dev/null +++ b/x-pack/plugin/voting-only-node/src/test/java/org/elasticsearch/cluster/coordination/VotingOnlyNodePluginIT.java @@ -0,0 +1,33 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License; + * you may not use this file except in compliance with the Elastic License. + */ +package org.elasticsearch.cluster.coordination; + +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.plugins.Plugin; +import org.elasticsearch.test.ESIntegTestCase; + +import java.io.IOException; +import java.util.Collection; +import java.util.Collections; + +@ESIntegTestCase.ClusterScope(numDataNodes = 0) +public class VotingOnlyNodePluginIT extends ESIntegTestCase { + + @Override + protected Collection> nodePlugins() { + return Collections.singleton(VotingOnlyNodePlugin.class); + } + + public void testOneVotingOnlyNode() throws IOException { + internalCluster().startNodes(2); + final String votingOnlyNode + = internalCluster().startNode(Settings.builder().put(VotingOnlyNodePlugin.VOTING_ONLY_NODE_SETTING.getKey(), true)); + + internalCluster().stopCurrentMasterNode(); + + assertNotEquals(votingOnlyNode, internalCluster().getMasterName()); + } +} From 7dd80ce30c59d1aa42bb23e9c3dec117e17980e9 Mon Sep 17 00:00:00 2001 From: David Turner Date: Tue, 18 Jun 2019 17:28:14 +0100 Subject: [PATCH 02/29] Randomise good-quorum calculation in CoordinationStateTests --- .../coordination/CoordinationStateTests.java | 40 ++++++++++++++++++- 1 file changed, 38 insertions(+), 2 deletions(-) diff --git a/server/src/test/java/org/elasticsearch/cluster/coordination/CoordinationStateTests.java b/server/src/test/java/org/elasticsearch/cluster/coordination/CoordinationStateTests.java index c52a2a36f34c0..ff1d5d9541f8e 100644 --- a/server/src/test/java/org/elasticsearch/cluster/coordination/CoordinationStateTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/coordination/CoordinationStateTests.java @@ -20,6 +20,7 @@ import org.elasticsearch.Assertions; import org.elasticsearch.Version; +import org.elasticsearch.action.ActionListener; import org.elasticsearch.cluster.ClusterName; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.coordination.CoordinationMetaData.VotingConfiguration; @@ -37,8 +38,11 @@ import org.junit.Before; import java.util.ArrayList; +import java.util.Collection; import java.util.Collections; +import java.util.HashMap; import java.util.List; +import java.util.Map; import java.util.Optional; import java.util.Set; import java.util.stream.Collectors; @@ -813,17 +817,49 @@ static class ClusterNode { final DiscoveryNode localNode; final PersistedState persistedState; + private final ElectionStrategy electionStrategy; CoordinationState state; ClusterNode(DiscoveryNode localNode) { this.localNode = localNode; persistedState = new InMemoryPersistedState(0L, clusterState(0L, 0L, localNode, VotingConfiguration.EMPTY_CONFIG, VotingConfiguration.EMPTY_CONFIG, 0L)); - state = new CoordinationState(localNode, persistedState, ElectionStrategy.DefaultElectionStrategy.INSTANCE); + electionStrategy = randomBoolean() ? ElectionStrategy.DefaultElectionStrategy.INSTANCE + : new ElectionStrategy() { + + private final Map goodNodes = new HashMap<>(); + + @Override + public boolean isAbdicationTarget(DiscoveryNode discoveryNode) { + throw new AssertionError("irrelevant"); + } + + @Override + public boolean isGoodQuorum(Collection votingNodes) { + votingNodes.forEach(n -> goodNodes.putIfAbsent(n, usually())); + return votingNodes.stream().allMatch(goodNodes::get); + } + + @Override + public boolean acceptPrevote(PreVoteResponse response, DiscoveryNode sender, ClusterState clusterState) { + throw new AssertionError("irrelevant"); + } + + @Override + public boolean shouldReceivePublication(DiscoveryNode destination) { + throw new AssertionError("irrelevant"); + } + + @Override + public ActionListener wrapPublishResponseHandler(ActionListener listener) { + throw new AssertionError("irrelevant"); + } + }; + state = new CoordinationState(localNode, persistedState, electionStrategy); } void reboot() { - state = new CoordinationState(localNode, persistedState, ElectionStrategy.DefaultElectionStrategy.INSTANCE); + state = new CoordinationState(localNode, persistedState, electionStrategy); } void setInitialState(VotingConfiguration initialConfig, long initialValue) { From c298508da11bc71360b960e8d30220579738987a Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Wed, 19 Jun 2019 10:18:28 +0200 Subject: [PATCH 03/29] State transfer only --- .../coordination/CoordinationState.java | 2 +- .../cluster/coordination/Coordinator.java | 18 ++-- .../coordination/ElectionStrategy.java | 54 +----------- .../coordination/PreVoteCollector.java | 6 +- .../cluster/coordination/Reconfigurator.java | 2 +- .../coordination/CoordinationStateTests.java | 36 +------- .../coordination/VotingOnlyNodePlugin.java | 86 +------------------ 7 files changed, 27 insertions(+), 177 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/cluster/coordination/CoordinationState.java b/server/src/main/java/org/elasticsearch/cluster/coordination/CoordinationState.java index ef35e10832cb2..41b2cc36acee0 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/CoordinationState.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/CoordinationState.java @@ -106,7 +106,7 @@ public boolean isElectionQuorum(VoteCollection votes) { } static boolean isElectionQuorum(VoteCollection votes, ClusterState lastAcceptedState, ElectionStrategy electionStrategy) { - return electionStrategy.isGoodQuorum(votes.nodes()) + return votes.nodes().stream().allMatch(electionStrategy::isStateTransferOnly) == false && votes.isQuorum(lastAcceptedState.getLastCommittedConfiguration()) && votes.isQuorum(lastAcceptedState.getLastAcceptedConfiguration()); } diff --git a/server/src/main/java/org/elasticsearch/cluster/coordination/Coordinator.java b/server/src/main/java/org/elasticsearch/cluster/coordination/Coordinator.java index 43499f1bd9bce..fe10ca5dbb776 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/Coordinator.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/Coordinator.java @@ -1306,7 +1306,7 @@ public void onSuccess(String source) { final List masterCandidates = completedNodes().stream() .filter(DiscoveryNode::isMasterNode) .filter(node -> nodeMayWinElection(state, node)) - .filter(electionStrategy::isAbdicationTarget) + .filter(node -> electionStrategy.isStateTransferOnly(node) == false) .collect(Collectors.toList()); if (masterCandidates.isEmpty() == false) { abdicateTo(masterCandidates.get(random.nextInt(masterCandidates.size()))); @@ -1388,12 +1388,18 @@ protected void onMissingJoin(DiscoveryNode discoveryNode) { @Override protected void sendPublishRequest(DiscoveryNode destination, PublishRequest publishRequest, ActionListener responseActionListener) { - if (electionStrategy.shouldReceivePublication(destination)) { - publicationContext.sendPublishRequest(destination, publishRequest, - electionStrategy.wrapPublishResponseHandler(wrapWithMutex(responseActionListener))); + if (electionStrategy.isStateTransferOnly(getLocalNode())) { + if (destination.isMasterNode() && electionStrategy.isStateTransferOnly(destination) == false) { + publicationContext.sendPublishRequest(destination, publishRequest, wrapWithMutex( + ActionListener.map(responseActionListener, resp -> { + throw new ElasticsearchException("ignoring successful publish response for state transfer only: " + resp); + }))); + } else { + logger.debug("sendPublishRequest: suppressing state transfer to [{}]", destination); + responseActionListener.onFailure(new ElasticsearchException("suppressing state transfer at source")); + } } else { - logger.debug("sendPublishRequest: suppressing publication to [{}]", destination); - responseActionListener.onFailure(new ElasticsearchException("suppressing publication at source")); + publicationContext.sendPublishRequest(destination, publishRequest, wrapWithMutex(responseActionListener)); } } diff --git a/server/src/main/java/org/elasticsearch/cluster/coordination/ElectionStrategy.java b/server/src/main/java/org/elasticsearch/cluster/coordination/ElectionStrategy.java index b0879b055994a..76e9786589d57 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/ElectionStrategy.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/ElectionStrategy.java @@ -18,68 +18,22 @@ */ package org.elasticsearch.cluster.coordination; -import org.elasticsearch.action.ActionListener; -import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.node.DiscoveryNode; -import java.util.Collection; - public interface ElectionStrategy { - /** - * Whether this node should abdicate to the given node when standing down as master - */ - boolean isAbdicationTarget(DiscoveryNode discoveryNode); /** - * Returns an extra filter on whether a collection of votes is a good quorum for an election + * Whether the given node is only here to ensure state transfer to another node */ - boolean isGoodQuorum(Collection votingNodes); - - /** - * Whether to accept the given pre-vote - */ - boolean acceptPrevote(PreVoteResponse response, DiscoveryNode sender, ClusterState clusterState); - - /** - * Whether the destination node should receive publications of new cluster states - */ - boolean shouldReceivePublication(DiscoveryNode destination); - - /** - * Allows the strategy to modify the {@link PublishWithJoinResponse} received before it is handled by the {@link Coordinator}. - */ - ActionListener wrapPublishResponseHandler(ActionListener listener); + boolean isStateTransferOnly(DiscoveryNode discoveryNode); class DefaultElectionStrategy implements ElectionStrategy { public static final ElectionStrategy INSTANCE = new DefaultElectionStrategy(); - private DefaultElectionStrategy() { - } - - @Override - public boolean isAbdicationTarget(DiscoveryNode discoveryNode) { - return true; - } - - @Override - public boolean isGoodQuorum(Collection votingNodes) { - return true; - } - - @Override - public boolean acceptPrevote(PreVoteResponse response, DiscoveryNode sender, ClusterState clusterState) { - return true; - } - - @Override - public boolean shouldReceivePublication(DiscoveryNode destination) { - return true; - } - @Override - public ActionListener wrapPublishResponseHandler(ActionListener listener) { - return listener; + public boolean isStateTransferOnly(DiscoveryNode discoveryNode) { + return false; } } } diff --git a/server/src/main/java/org/elasticsearch/cluster/coordination/PreVoteCollector.java b/server/src/main/java/org/elasticsearch/cluster/coordination/PreVoteCollector.java index 0fc714803bc86..ff3a1217e0b61 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/PreVoteCollector.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/PreVoteCollector.java @@ -187,7 +187,11 @@ private void handlePreVoteResponse(final PreVoteResponse response, final Discove return; } - if (electionStrategy.acceptPrevote(response, sender, clusterState) == false) { + if (response.getLastAcceptedTerm() == clusterState.term() && + response.getLastAcceptedVersion() == clusterState.version() && + electionStrategy.isStateTransferOnly(clusterState.nodes().getLocalNode()) && + electionStrategy.isStateTransferOnly(sender) == false) { + logger.debug("{} ignoring {} from {} as it has the same state and is not transfer-only", this, response, sender); return; } diff --git a/server/src/main/java/org/elasticsearch/cluster/coordination/Reconfigurator.java b/server/src/main/java/org/elasticsearch/cluster/coordination/Reconfigurator.java index 23190b2440c57..c6224a3616050 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/Reconfigurator.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/Reconfigurator.java @@ -130,7 +130,7 @@ public VotingConfiguration reconfigure(Set liveNodes, Set .collect(Collectors.toSet())); // new configuration should have a quorum - if (newConfig.hasQuorum(liveNodeIds) && electionStrategy.isGoodQuorum(liveNodes)) { + if (newConfig.hasQuorum(liveNodeIds) && liveNodes.stream().allMatch(electionStrategy::isStateTransferOnly) == false) { return newConfig; } else { // If there are not enough live nodes to form a quorum in the newly-proposed configuration, it's better to do nothing. diff --git a/server/src/test/java/org/elasticsearch/cluster/coordination/CoordinationStateTests.java b/server/src/test/java/org/elasticsearch/cluster/coordination/CoordinationStateTests.java index ff1d5d9541f8e..0757d41efc037 100644 --- a/server/src/test/java/org/elasticsearch/cluster/coordination/CoordinationStateTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/coordination/CoordinationStateTests.java @@ -20,7 +20,6 @@ import org.elasticsearch.Assertions; import org.elasticsearch.Version; -import org.elasticsearch.action.ActionListener; import org.elasticsearch.cluster.ClusterName; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.coordination.CoordinationMetaData.VotingConfiguration; @@ -38,11 +37,8 @@ import org.junit.Before; import java.util.ArrayList; -import java.util.Collection; import java.util.Collections; -import java.util.HashMap; import java.util.List; -import java.util.Map; import java.util.Optional; import java.util.Set; import java.util.stream.Collectors; @@ -824,37 +820,7 @@ static class ClusterNode { this.localNode = localNode; persistedState = new InMemoryPersistedState(0L, clusterState(0L, 0L, localNode, VotingConfiguration.EMPTY_CONFIG, VotingConfiguration.EMPTY_CONFIG, 0L)); - electionStrategy = randomBoolean() ? ElectionStrategy.DefaultElectionStrategy.INSTANCE - : new ElectionStrategy() { - - private final Map goodNodes = new HashMap<>(); - - @Override - public boolean isAbdicationTarget(DiscoveryNode discoveryNode) { - throw new AssertionError("irrelevant"); - } - - @Override - public boolean isGoodQuorum(Collection votingNodes) { - votingNodes.forEach(n -> goodNodes.putIfAbsent(n, usually())); - return votingNodes.stream().allMatch(goodNodes::get); - } - - @Override - public boolean acceptPrevote(PreVoteResponse response, DiscoveryNode sender, ClusterState clusterState) { - throw new AssertionError("irrelevant"); - } - - @Override - public boolean shouldReceivePublication(DiscoveryNode destination) { - throw new AssertionError("irrelevant"); - } - - @Override - public ActionListener wrapPublishResponseHandler(ActionListener listener) { - throw new AssertionError("irrelevant"); - } - }; + electionStrategy = randomBoolean() ? ElectionStrategy.DefaultElectionStrategy.INSTANCE : discoveryNode -> rarely(); state = new CoordinationState(localNode, persistedState, electionStrategy); } diff --git a/x-pack/plugin/voting-only-node/src/main/java/org/elasticsearch/cluster/coordination/VotingOnlyNodePlugin.java b/x-pack/plugin/voting-only-node/src/main/java/org/elasticsearch/cluster/coordination/VotingOnlyNodePlugin.java index bf8fb2a019dd5..7371c10646137 100644 --- a/x-pack/plugin/voting-only-node/src/main/java/org/elasticsearch/cluster/coordination/VotingOnlyNodePlugin.java +++ b/x-pack/plugin/voting-only-node/src/main/java/org/elasticsearch/cluster/coordination/VotingOnlyNodePlugin.java @@ -5,11 +5,6 @@ */ package org.elasticsearch.cluster.coordination; -import org.apache.logging.log4j.LogManager; -import org.apache.logging.log4j.Logger; -import org.elasticsearch.ElasticsearchException; -import org.elasticsearch.action.ActionListener; -import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.node.DiscoveryNodeRole; import org.elasticsearch.common.settings.Setting; @@ -19,7 +14,6 @@ import org.elasticsearch.plugins.DiscoveryPlugin; import org.elasticsearch.plugins.Plugin; -import java.util.Collection; import java.util.Collections; import java.util.List; import java.util.Map; @@ -27,8 +21,6 @@ public class VotingOnlyNodePlugin extends Plugin implements DiscoveryPlugin { - private static final Logger logger = LogManager.getLogger(VotingOnlyNodePlugin.class); - public static final Setting VOTING_ONLY_NODE_SETTING = Setting.boolSetting("node.voting_only", false, Setting.Property.NodeScope); @@ -65,88 +57,16 @@ protected Setting roleSetting() { } }; - private static boolean isTrueMasterNode(DiscoveryNode discoveryNode) { - return discoveryNode.isMasterNode() && discoveryNode.getRoles().contains(VOTING_ONLY_NODE_ROLE) == false; - } - - private class AvoidsVotingOnlyNodesElectionStrategy implements ElectionStrategy { - @Override - public boolean isAbdicationTarget(DiscoveryNode discoveryNode) { - return isTrueMasterNode(discoveryNode); - } - - @Override - public boolean isGoodQuorum(Collection votingNodes) { - return true; - } - - @Override - public boolean acceptPrevote(PreVoteResponse response, DiscoveryNode sender, ClusterState clusterState) { - return true; - } - - @Override - public boolean shouldReceivePublication(DiscoveryNode destination) { - return true; - } - - @Override - public ActionListener wrapPublishResponseHandler(ActionListener listener) { - return listener; - } - } - private class VotingOnlyNodeElectionStrategy implements ElectionStrategy { @Override - public boolean isAbdicationTarget(DiscoveryNode discoveryNode) { - assert false : "voting-only node cannot abdicate"; - return false; - } - - - @Override - public boolean isGoodQuorum(Collection votingNodes) { - return votingNodes.stream().anyMatch(VotingOnlyNodePlugin::isTrueMasterNode); - } - - @Override - public boolean acceptPrevote(PreVoteResponse response, DiscoveryNode sender, ClusterState clusterState) { - if (sender.getRoles().contains(VOTING_ONLY_NODE_ROLE) == false - && response.getLastAcceptedTerm() == clusterState.term() - && response.getLastAcceptedVersion() == clusterState.version()) { - logger.debug("{} ignoring {} from {} as it has higher voting priority", this, response, sender); - return false; - } - - return true; - } - - @Override - public boolean shouldReceivePublication(DiscoveryNode destination) { - return isTrueMasterNode(destination); - } - - @Override - public ActionListener wrapPublishResponseHandler(ActionListener listener) { - return new ActionListener<>() { - @Override - public void onResponse(PublishWithJoinResponse publishWithJoinResponse) { - listener.onFailure(new ElasticsearchException( - "ignoring successful publish response on voting-only node: " + publishWithJoinResponse)); - } - - @Override - public void onFailure(Exception e) { - listener.onFailure(e); - } - }; + public boolean isStateTransferOnly(DiscoveryNode discoveryNode) { + return discoveryNode.isMasterNode() && discoveryNode.getRoles().contains(VOTING_ONLY_NODE_ROLE); } } @Override public Map getElectionStrategies() { - return Collections.singletonMap(VOTING_ONLY_ELECTION_TYPE, - isVotingOnlyNode ? new VotingOnlyNodeElectionStrategy() : new AvoidsVotingOnlyNodesElectionStrategy()); + return Collections.singletonMap(VOTING_ONLY_ELECTION_TYPE, new VotingOnlyNodeElectionStrategy()); } } From 0d3750d9222edfb41b89989520a89ecd60176ed8 Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Wed, 19 Jun 2019 14:39:37 +0200 Subject: [PATCH 04/29] Move to isElectionQuorum --- .../ClusterFormationFailureHelper.java | 16 ++++++-- .../coordination/CoordinationState.java | 38 +++++++++++++------ .../cluster/coordination/Coordinator.java | 27 ++++++++++--- .../coordination/ElectionStrategy.java | 13 +++++++ .../coordination/PreVoteCollector.java | 30 +++++++-------- .../cluster/coordination/Reconfigurator.java | 7 +--- .../coordination/VotingOnlyNodePlugin.java | 35 ++++++++++++++++- 7 files changed, 123 insertions(+), 43 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/cluster/coordination/ClusterFormationFailureHelper.java b/server/src/main/java/org/elasticsearch/cluster/coordination/ClusterFormationFailureHelper.java index 4d4027d3e9f3a..51af568a2761d 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/ClusterFormationFailureHelper.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/ClusterFormationFailureHelper.java @@ -22,7 +22,6 @@ import org.apache.logging.log4j.Logger; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.coordination.CoordinationMetaData.VotingConfiguration; -import org.elasticsearch.cluster.coordination.CoordinationState.VoteCollection; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.settings.Setting; @@ -184,10 +183,19 @@ String getDescription() { + describeQuorum(clusterState.getLastCommittedConfiguration()); } - final VoteCollection voteCollection = new VoteCollection(); - foundPeers.forEach(voteCollection::addVote); + final CoordinationState.JoinVoteCollection expectedVotes = new CoordinationState.JoinVoteCollection(); + long term = clusterState.term(); + long lastAcceptedTerm = clusterState.term(); + long lastAcceptedVersion = clusterState.version(); + foundPeers.forEach(node -> expectedVotes.addJoinVote(new Join(node, clusterState.nodes().getLocalNode(), term, lastAcceptedTerm, + lastAcceptedVersion))); + expectedVotes.addJoinVote(new Join(clusterState.nodes().getLocalNode(), clusterState.nodes().getLocalNode(), term, + lastAcceptedTerm, lastAcceptedVersion)); + final String isQuorumOrNot - = CoordinationState.isElectionQuorum(voteCollection, clusterState, electionStrategy) ? "is a quorum" : "is not a quorum"; + = electionStrategy.isElectionQuorum(clusterState.nodes().getLocalNode(), term, lastAcceptedTerm, lastAcceptedVersion, + clusterState.getLastCommittedConfiguration(), clusterState.getLastAcceptedConfiguration(), expectedVotes) ? + "is a quorum" : "is not a quorum"; return String.format(Locale.ROOT, "master not discovered or elected yet, an election requires %s, have discovered %s which %s; %s", diff --git a/server/src/main/java/org/elasticsearch/cluster/coordination/CoordinationState.java b/server/src/main/java/org/elasticsearch/cluster/coordination/CoordinationState.java index 41b2cc36acee0..05cd397571ff1 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/CoordinationState.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/CoordinationState.java @@ -28,8 +28,10 @@ import java.util.Collection; import java.util.Collections; import java.util.HashMap; +import java.util.HashSet; import java.util.Map; import java.util.Optional; +import java.util.Set; /** * The core class of the cluster state coordination algorithm, directly implementing the @@ -46,7 +48,7 @@ public class CoordinationState { private final ElectionStrategy electionStrategy; // transient state - private VoteCollection joinVotes; + private JoinVoteCollection joinVotes; private boolean startedJoinSinceLastReboot; private boolean electionWon; private long lastPublishedVersion; @@ -61,7 +63,7 @@ public CoordinationState(DiscoveryNode localNode, PersistedState persistedState, this.electionStrategy = electionStrategy; // transient state - this.joinVotes = new VoteCollection(); + this.joinVotes = new JoinVoteCollection(); this.startedJoinSinceLastReboot = false; this.electionWon = false; this.lastPublishedVersion = 0L; @@ -101,14 +103,9 @@ public boolean electionWon() { return electionWon; } - public boolean isElectionQuorum(VoteCollection votes) { - return isElectionQuorum(votes, getLastAcceptedState(), electionStrategy); - } - - static boolean isElectionQuorum(VoteCollection votes, ClusterState lastAcceptedState, ElectionStrategy electionStrategy) { - return votes.nodes().stream().allMatch(electionStrategy::isStateTransferOnly) == false - && votes.isQuorum(lastAcceptedState.getLastCommittedConfiguration()) - && votes.isQuorum(lastAcceptedState.getLastAcceptedConfiguration()); + public boolean isElectionQuorum(JoinVoteCollection joinVotes) { + return electionStrategy.isElectionQuorum(localNode, getCurrentTerm(), getLastAcceptedTerm(), getLastAcceptedVersion(), + getLastCommittedConfiguration(), getLastAcceptedConfiguration(), joinVotes); } public boolean isPublishQuorum(VoteCollection votes) { @@ -190,7 +187,7 @@ public Join handleStartJoin(StartJoinRequest startJoinRequest) { lastPublishedConfiguration = getLastAcceptedConfiguration(); startedJoinSinceLastReboot = true; electionWon = false; - joinVotes = new VoteCollection(); + joinVotes = new JoinVoteCollection(); publishVotes = new VoteCollection(); return new Join(localNode, startJoinRequest.getSourceNode(), getCurrentTerm(), getLastAcceptedTerm(), @@ -245,7 +242,7 @@ public boolean handleJoin(Join join) { throw new CoordinationStateRejectedException("rejecting join since this node has not received its initial configuration yet"); } - boolean added = joinVotes.addVote(join.getSourceNode()); + boolean added = joinVotes.addJoinVote(join); boolean prevElectionWon = electionWon; electionWon = isElectionQuorum(joinVotes); assert !prevElectionWon || electionWon; // we cannot go from won to not won @@ -541,4 +538,21 @@ public int hashCode() { return nodes.hashCode(); } } + + public static class JoinVoteCollection extends VoteCollection { + + private final Set joins = new HashSet<>(); + + public boolean addJoinVote(Join join) { + final boolean added = addVote(join.getSourceNode()); + if (added) { + joins.add(join); + } + return added; + } + + public Set getJoins() { + return Collections.unmodifiableSet(joins); + } + } } diff --git a/server/src/main/java/org/elasticsearch/cluster/coordination/Coordinator.java b/server/src/main/java/org/elasticsearch/cluster/coordination/Coordinator.java index fe10ca5dbb776..5d1e7ae05d549 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/Coordinator.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/Coordinator.java @@ -34,6 +34,7 @@ import org.elasticsearch.cluster.coordination.ClusterFormationFailureHelper.ClusterFormationState; import org.elasticsearch.cluster.coordination.CoordinationMetaData.VotingConfigExclusion; import org.elasticsearch.cluster.coordination.CoordinationMetaData.VotingConfiguration; +import org.elasticsearch.cluster.coordination.CoordinationState.JoinVoteCollection; import org.elasticsearch.cluster.coordination.FollowersChecker.FollowerCheckRequest; import org.elasticsearch.cluster.coordination.JoinHelper.InitialJoinAccumulator; import org.elasticsearch.cluster.metadata.MetaData; @@ -182,7 +183,7 @@ public Coordinator(String nodeName, Settings settings, ClusterSettings clusterSe this.nodeRemovalExecutor = new NodeRemovalClusterStateTaskExecutor(allocationService, logger); this.clusterApplier = clusterApplier; masterService.setClusterStateSupplier(this::getStateForMasterService); - this.reconfigurator = new Reconfigurator(settings, clusterSettings, electionStrategy); + this.reconfigurator = new Reconfigurator(settings, clusterSettings); this.clusterBootstrapService = new ClusterBootstrapService(settings, transportService, this::getFoundPeers, this::isInitialConfigurationSet, this::setInitialConfiguration); this.lagDetector = new LagDetector(settings, transportService.getThreadPool(), n -> removeNode(n, "lagging"), @@ -1104,9 +1105,13 @@ protected void onFoundPeersUpdated() { synchronized (mutex) { final Iterable foundPeers = getFoundPeers(); if (mode == Mode.CANDIDATE) { - final CoordinationState.VoteCollection expectedVotes = new CoordinationState.VoteCollection(); - foundPeers.forEach(expectedVotes::addVote); - expectedVotes.addVote(Coordinator.this.getLocalNode()); + final JoinVoteCollection expectedVotes = new JoinVoteCollection(); + long term = getCurrentTerm(); + long lastAcceptedTerm = getLastAcceptedState().term(); + long lastAcceptedVersion = getLastAcceptedState().version(); + foundPeers.forEach(node -> expectedVotes.addJoinVote(new Join(node, getLocalNode(), term, lastAcceptedTerm, + lastAcceptedVersion))); + expectedVotes.addJoinVote(new Join(getLocalNode(), getLocalNode(), term, lastAcceptedTerm, lastAcceptedVersion)); final boolean foundQuorum = coordinationState.get().isElectionQuorum(expectedVotes); if (foundQuorum) { if (electionScheduler == null) { @@ -1306,7 +1311,19 @@ public void onSuccess(String source) { final List masterCandidates = completedNodes().stream() .filter(DiscoveryNode::isMasterNode) .filter(node -> nodeMayWinElection(state, node)) - .filter(node -> electionStrategy.isStateTransferOnly(node) == false) + .filter(node -> { + // check if candidate would be able to get an election quorum + final JoinVoteCollection fakeJoinVoteCollection = new JoinVoteCollection(); + completedNodes().forEach(completedNode -> { + fakeJoinVoteCollection.addJoinVote(new Join(completedNode, node, state.term(), + state.term(), state.version())); + }); + return electionStrategy.isElectionQuorum(node, + state.term(), state.term(), state.version(), + state.getLastCommittedConfiguration(), state.getLastAcceptedConfiguration(), + fakeJoinVoteCollection + ); + }) .collect(Collectors.toList()); if (masterCandidates.isEmpty() == false) { abdicateTo(masterCandidates.get(random.nextInt(masterCandidates.size()))); diff --git a/server/src/main/java/org/elasticsearch/cluster/coordination/ElectionStrategy.java b/server/src/main/java/org/elasticsearch/cluster/coordination/ElectionStrategy.java index 76e9786589d57..02bc9e0c9c954 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/ElectionStrategy.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/ElectionStrategy.java @@ -18,6 +18,8 @@ */ package org.elasticsearch.cluster.coordination; +import org.elasticsearch.cluster.coordination.CoordinationMetaData.VotingConfiguration; +import org.elasticsearch.cluster.coordination.CoordinationState.JoinVoteCollection; import org.elasticsearch.cluster.node.DiscoveryNode; public interface ElectionStrategy { @@ -27,6 +29,10 @@ public interface ElectionStrategy { */ boolean isStateTransferOnly(DiscoveryNode discoveryNode); + boolean isElectionQuorum(DiscoveryNode localNode, long localCurrentTerm, long localAcceptedTerm, long localAcceptedVersion, + VotingConfiguration lastCommittedConfiguration, VotingConfiguration lastAcceptedConfiguration, + JoinVoteCollection joinVotes); + class DefaultElectionStrategy implements ElectionStrategy { public static final ElectionStrategy INSTANCE = new DefaultElectionStrategy(); @@ -35,5 +41,12 @@ class DefaultElectionStrategy implements ElectionStrategy { public boolean isStateTransferOnly(DiscoveryNode discoveryNode) { return false; } + + @Override + public boolean isElectionQuorum(DiscoveryNode localNode, long localCurrentTerm, long localAcceptedTerm, long localAcceptedVersion, + VotingConfiguration lastCommittedConfiguration, VotingConfiguration lastAcceptedConfiguration, + JoinVoteCollection joinVotes) { + return joinVotes.isQuorum(lastCommittedConfiguration) && joinVotes.isQuorum(lastAcceptedConfiguration); + } } } diff --git a/server/src/main/java/org/elasticsearch/cluster/coordination/PreVoteCollector.java b/server/src/main/java/org/elasticsearch/cluster/coordination/PreVoteCollector.java index ff3a1217e0b61..1ea4d11989cd8 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/PreVoteCollector.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/PreVoteCollector.java @@ -23,7 +23,7 @@ import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; import org.elasticsearch.cluster.ClusterState; -import org.elasticsearch.cluster.coordination.CoordinationState.VoteCollection; +import org.elasticsearch.cluster.coordination.CoordinationState.JoinVoteCollection; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.collect.Tuple; @@ -35,11 +35,11 @@ import org.elasticsearch.transport.TransportService; import java.io.IOException; -import java.util.Set; +import java.util.Map; import java.util.concurrent.atomic.AtomicBoolean; import java.util.function.LongConsumer; -import static org.elasticsearch.common.util.concurrent.ConcurrentCollections.newConcurrentSet; +import static org.elasticsearch.common.util.concurrent.ConcurrentCollections.newConcurrentMap; public class PreVoteCollector { @@ -130,7 +130,7 @@ public String toString() { } private class PreVotingRound implements Releasable { - private final Set preVotesReceived = newConcurrentSet(); + private final Map preVotesReceived = newConcurrentMap(); private final AtomicBoolean electionStarted = new AtomicBoolean(); private final PreVoteRequest preVoteRequest; private final ClusterState clusterState; @@ -187,20 +187,18 @@ private void handlePreVoteResponse(final PreVoteResponse response, final Discove return; } - if (response.getLastAcceptedTerm() == clusterState.term() && - response.getLastAcceptedVersion() == clusterState.version() && - electionStrategy.isStateTransferOnly(clusterState.nodes().getLocalNode()) && - electionStrategy.isStateTransferOnly(sender) == false) { - logger.debug("{} ignoring {} from {} as it has the same state and is not transfer-only", this, response, sender); - return; - } - - preVotesReceived.add(sender); + preVotesReceived.put(sender, response); - final VoteCollection voteCollection = new VoteCollection(); - preVotesReceived.forEach(voteCollection::addVote); + final JoinVoteCollection voteCollection = new JoinVoteCollection(); + preVotesReceived.forEach((node, preVoteResponse) -> { + voteCollection.addJoinVote(new Join(node, clusterState.nodes().getLocalNode(), preVoteResponse.getCurrentTerm(), + preVoteResponse.getLastAcceptedTerm(), preVoteResponse.getLastAcceptedVersion())); + }); - if (CoordinationState.isElectionQuorum(voteCollection, clusterState, electionStrategy) == false) { + final PreVoteResponse localPrevoteResponse = getPreVoteResponse(); + if (electionStrategy.isElectionQuorum(clusterState.nodes().getLocalNode(), localPrevoteResponse.getCurrentTerm(), + localPrevoteResponse.getLastAcceptedTerm(), localPrevoteResponse.getLastAcceptedVersion(), + clusterState.getLastCommittedConfiguration(), clusterState.getLastAcceptedConfiguration(), voteCollection) == false) { logger.debug("{} added {} from {}, no quorum yet", this, response, sender); return; } diff --git a/server/src/main/java/org/elasticsearch/cluster/coordination/Reconfigurator.java b/server/src/main/java/org/elasticsearch/cluster/coordination/Reconfigurator.java index c6224a3616050..fd9bab3af11ec 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/Reconfigurator.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/Reconfigurator.java @@ -59,11 +59,8 @@ public class Reconfigurator { private volatile boolean autoShrinkVotingConfiguration; - private final ElectionStrategy electionStrategy; - - public Reconfigurator(Settings settings, ClusterSettings clusterSettings, ElectionStrategy electionStrategy) { + public Reconfigurator(Settings settings, ClusterSettings clusterSettings) { autoShrinkVotingConfiguration = CLUSTER_AUTO_SHRINK_VOTING_CONFIGURATION.get(settings); - this.electionStrategy = electionStrategy; clusterSettings.addSettingsUpdateConsumer(CLUSTER_AUTO_SHRINK_VOTING_CONFIGURATION, this::setAutoShrinkVotingConfiguration); } @@ -130,7 +127,7 @@ public VotingConfiguration reconfigure(Set liveNodes, Set .collect(Collectors.toSet())); // new configuration should have a quorum - if (newConfig.hasQuorum(liveNodeIds) && liveNodes.stream().allMatch(electionStrategy::isStateTransferOnly) == false) { + if (newConfig.hasQuorum(liveNodeIds)) { return newConfig; } else { // If there are not enough live nodes to form a quorum in the newly-proposed configuration, it's better to do nothing. diff --git a/x-pack/plugin/voting-only-node/src/main/java/org/elasticsearch/cluster/coordination/VotingOnlyNodePlugin.java b/x-pack/plugin/voting-only-node/src/main/java/org/elasticsearch/cluster/coordination/VotingOnlyNodePlugin.java index 7371c10646137..3f5370dc89ca6 100644 --- a/x-pack/plugin/voting-only-node/src/main/java/org/elasticsearch/cluster/coordination/VotingOnlyNodePlugin.java +++ b/x-pack/plugin/voting-only-node/src/main/java/org/elasticsearch/cluster/coordination/VotingOnlyNodePlugin.java @@ -5,6 +5,8 @@ */ package org.elasticsearch.cluster.coordination; +import org.elasticsearch.cluster.coordination.CoordinationMetaData.VotingConfiguration; +import org.elasticsearch.cluster.coordination.CoordinationState.JoinVoteCollection; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.node.DiscoveryNodeRole; import org.elasticsearch.common.settings.Setting; @@ -57,12 +59,43 @@ protected Setting roleSetting() { } }; - private class VotingOnlyNodeElectionStrategy implements ElectionStrategy { + public boolean isVotingOnlyNode(DiscoveryNode discoveryNode) { + return discoveryNode.getRoles().contains(VOTING_ONLY_NODE_ROLE); + } + + public boolean isFullMasterNode(DiscoveryNode discoveryNode) { + return discoveryNode.isMasterNode() && discoveryNode.getRoles().contains(VOTING_ONLY_NODE_ROLE) == false; + } + + private class VotingOnlyNodeElectionStrategy extends ElectionStrategy.DefaultElectionStrategy { @Override public boolean isStateTransferOnly(DiscoveryNode discoveryNode) { return discoveryNode.isMasterNode() && discoveryNode.getRoles().contains(VOTING_ONLY_NODE_ROLE); } + + @Override + public boolean isElectionQuorum(DiscoveryNode localNode, long localCurrentTerm, long localAcceptedTerm, long localAcceptedVersion, + VotingConfiguration lastCommittedConfiguration, VotingConfiguration lastAcceptedConfiguration, + JoinVoteCollection joinVotes) { + // if local node is voting only, have additional checks on election quorum definition + if (isVotingOnlyNode(localNode)) { + // if all votes are from voting only nodes, do not elect as master (no need to transfer state) + if (joinVotes.getJoins().stream().allMatch(join -> isVotingOnlyNode(join.getSourceNode()))) { + return false; + } + // if there's a vote from a full master node with same last accepted term and version, that node should become master + // instead, so we should stand down + if (joinVotes.getJoins().stream().anyMatch(join -> isFullMasterNode(join.getSourceNode()) && + join.getLastAcceptedTerm() == localAcceptedTerm && + join.getLastAcceptedVersion() == localAcceptedVersion)) { + return false; + } + } + // fall back to default election quorum definition + return super.isElectionQuorum(localNode, localCurrentTerm, localAcceptedTerm, localAcceptedVersion, + lastCommittedConfiguration, lastAcceptedConfiguration, joinVotes); + } } @Override From ee52a897df0e5f3b52369f8448683928fb4456a1 Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Wed, 19 Jun 2019 16:23:11 +0200 Subject: [PATCH 05/29] use JoinVoteCollection --- .../coordination/CoordinationState.java | 2 +- .../cluster/coordination/Coordinator.java | 18 ++--- .../coordination/ElectionStrategy.java | 6 +- .../coordination/PreVoteCollector.java | 7 +- .../coordination/VotingOnlyNodePlugin.java | 25 ++++--- .../coordination/VotingOnlyNodePluginIT.java | 69 ++++++++++++++++++- 6 files changed, 98 insertions(+), 29 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/cluster/coordination/CoordinationState.java b/server/src/main/java/org/elasticsearch/cluster/coordination/CoordinationState.java index 05cd397571ff1..a443c5670c0e2 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/CoordinationState.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/CoordinationState.java @@ -103,7 +103,7 @@ public boolean electionWon() { return electionWon; } - public boolean isElectionQuorum(JoinVoteCollection joinVotes) { + public boolean isElectionQuorum(VoteCollection joinVotes) { return electionStrategy.isElectionQuorum(localNode, getCurrentTerm(), getLastAcceptedTerm(), getLastAcceptedVersion(), getLastCommittedConfiguration(), getLastAcceptedConfiguration(), joinVotes); } diff --git a/server/src/main/java/org/elasticsearch/cluster/coordination/Coordinator.java b/server/src/main/java/org/elasticsearch/cluster/coordination/Coordinator.java index 5d1e7ae05d549..0edec587cc5cc 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/Coordinator.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/Coordinator.java @@ -68,6 +68,7 @@ import org.elasticsearch.discovery.SeedHostsResolver; import org.elasticsearch.threadpool.Scheduler; import org.elasticsearch.threadpool.ThreadPool.Names; +import org.elasticsearch.transport.TransportException; import org.elasticsearch.transport.TransportResponse.Empty; import org.elasticsearch.transport.TransportService; @@ -1105,14 +1106,11 @@ protected void onFoundPeersUpdated() { synchronized (mutex) { final Iterable foundPeers = getFoundPeers(); if (mode == Mode.CANDIDATE) { - final JoinVoteCollection expectedVotes = new JoinVoteCollection(); - long term = getCurrentTerm(); - long lastAcceptedTerm = getLastAcceptedState().term(); - long lastAcceptedVersion = getLastAcceptedState().version(); - foundPeers.forEach(node -> expectedVotes.addJoinVote(new Join(node, getLocalNode(), term, lastAcceptedTerm, - lastAcceptedVersion))); - expectedVotes.addJoinVote(new Join(getLocalNode(), getLocalNode(), term, lastAcceptedTerm, lastAcceptedVersion)); + final CoordinationState.VoteCollection expectedVotes = new CoordinationState.VoteCollection(); + foundPeers.forEach(expectedVotes::addVote); + expectedVotes.addVote(Coordinator.this.getLocalNode()); final boolean foundQuorum = coordinationState.get().isElectionQuorum(expectedVotes); + if (foundQuorum) { if (electionScheduler == null) { startElectionScheduler(); @@ -1409,11 +1407,13 @@ protected void sendPublishRequest(DiscoveryNode destination, PublishRequest publ if (destination.isMasterNode() && electionStrategy.isStateTransferOnly(destination) == false) { publicationContext.sendPublishRequest(destination, publishRequest, wrapWithMutex( ActionListener.map(responseActionListener, resp -> { - throw new ElasticsearchException("ignoring successful publish response for state transfer only: " + resp); + throw new TransportException( + new ElasticsearchException("ignoring successful publish response for state transfer only: " + resp)); }))); } else { logger.debug("sendPublishRequest: suppressing state transfer to [{}]", destination); - responseActionListener.onFailure(new ElasticsearchException("suppressing state transfer at source")); + responseActionListener.onFailure(new TransportException( + new ElasticsearchException("suppressing state transfer at source"))); } } else { publicationContext.sendPublishRequest(destination, publishRequest, wrapWithMutex(responseActionListener)); diff --git a/server/src/main/java/org/elasticsearch/cluster/coordination/ElectionStrategy.java b/server/src/main/java/org/elasticsearch/cluster/coordination/ElectionStrategy.java index 02bc9e0c9c954..70bcaeb8d0a2c 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/ElectionStrategy.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/ElectionStrategy.java @@ -19,7 +19,7 @@ package org.elasticsearch.cluster.coordination; import org.elasticsearch.cluster.coordination.CoordinationMetaData.VotingConfiguration; -import org.elasticsearch.cluster.coordination.CoordinationState.JoinVoteCollection; +import org.elasticsearch.cluster.coordination.CoordinationState.VoteCollection; import org.elasticsearch.cluster.node.DiscoveryNode; public interface ElectionStrategy { @@ -31,7 +31,7 @@ public interface ElectionStrategy { boolean isElectionQuorum(DiscoveryNode localNode, long localCurrentTerm, long localAcceptedTerm, long localAcceptedVersion, VotingConfiguration lastCommittedConfiguration, VotingConfiguration lastAcceptedConfiguration, - JoinVoteCollection joinVotes); + VoteCollection joinVotes); class DefaultElectionStrategy implements ElectionStrategy { @@ -45,7 +45,7 @@ public boolean isStateTransferOnly(DiscoveryNode discoveryNode) { @Override public boolean isElectionQuorum(DiscoveryNode localNode, long localCurrentTerm, long localAcceptedTerm, long localAcceptedVersion, VotingConfiguration lastCommittedConfiguration, VotingConfiguration lastAcceptedConfiguration, - JoinVoteCollection joinVotes) { + VoteCollection joinVotes) { return joinVotes.isQuorum(lastCommittedConfiguration) && joinVotes.isQuorum(lastAcceptedConfiguration); } } diff --git a/server/src/main/java/org/elasticsearch/cluster/coordination/PreVoteCollector.java b/server/src/main/java/org/elasticsearch/cluster/coordination/PreVoteCollector.java index 1ea4d11989cd8..11b6e53dd409a 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/PreVoteCollector.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/PreVoteCollector.java @@ -190,10 +190,9 @@ private void handlePreVoteResponse(final PreVoteResponse response, final Discove preVotesReceived.put(sender, response); final JoinVoteCollection voteCollection = new JoinVoteCollection(); - preVotesReceived.forEach((node, preVoteResponse) -> { - voteCollection.addJoinVote(new Join(node, clusterState.nodes().getLocalNode(), preVoteResponse.getCurrentTerm(), - preVoteResponse.getLastAcceptedTerm(), preVoteResponse.getLastAcceptedVersion())); - }); + preVotesReceived.forEach((node, preVoteResponse) -> voteCollection.addJoinVote( + new Join(node, clusterState.nodes().getLocalNode(), preVoteResponse.getCurrentTerm(), + preVoteResponse.getLastAcceptedTerm(), preVoteResponse.getLastAcceptedVersion()))); final PreVoteResponse localPrevoteResponse = getPreVoteResponse(); if (electionStrategy.isElectionQuorum(clusterState.nodes().getLocalNode(), localPrevoteResponse.getCurrentTerm(), diff --git a/x-pack/plugin/voting-only-node/src/main/java/org/elasticsearch/cluster/coordination/VotingOnlyNodePlugin.java b/x-pack/plugin/voting-only-node/src/main/java/org/elasticsearch/cluster/coordination/VotingOnlyNodePlugin.java index 3f5370dc89ca6..4dfc8f0c7013e 100644 --- a/x-pack/plugin/voting-only-node/src/main/java/org/elasticsearch/cluster/coordination/VotingOnlyNodePlugin.java +++ b/x-pack/plugin/voting-only-node/src/main/java/org/elasticsearch/cluster/coordination/VotingOnlyNodePlugin.java @@ -7,6 +7,7 @@ import org.elasticsearch.cluster.coordination.CoordinationMetaData.VotingConfiguration; import org.elasticsearch.cluster.coordination.CoordinationState.JoinVoteCollection; +import org.elasticsearch.cluster.coordination.CoordinationState.VoteCollection; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.node.DiscoveryNodeRole; import org.elasticsearch.common.settings.Setting; @@ -26,17 +27,20 @@ public class VotingOnlyNodePlugin extends Plugin implements DiscoveryPlugin { public static final Setting VOTING_ONLY_NODE_SETTING = Setting.boolSetting("node.voting_only", false, Setting.Property.NodeScope); + private final Settings settings; + private final Boolean isVotingOnlyNode; public VotingOnlyNodePlugin(Settings settings) { + this.settings = settings; isVotingOnlyNode = VOTING_ONLY_NODE_SETTING.get(settings); - if (isVotingOnlyNode && Node.NODE_MASTER_SETTING.get(settings) == false) { - throw new IllegalStateException("voting-only node must be master-eligible"); - } } @Override public Set getRoles() { + if (isVotingOnlyNode && Node.NODE_MASTER_SETTING.get(settings) == false) { + throw new IllegalStateException("voting-only node must be master-eligible"); + } return Collections.singleton(VOTING_ONLY_NODE_ROLE); } @@ -59,11 +63,11 @@ protected Setting roleSetting() { } }; - public boolean isVotingOnlyNode(DiscoveryNode discoveryNode) { + public static boolean isVotingOnlyNode(DiscoveryNode discoveryNode) { return discoveryNode.getRoles().contains(VOTING_ONLY_NODE_ROLE); } - public boolean isFullMasterNode(DiscoveryNode discoveryNode) { + public static boolean isFullMasterNode(DiscoveryNode discoveryNode) { return discoveryNode.isMasterNode() && discoveryNode.getRoles().contains(VOTING_ONLY_NODE_ROLE) == false; } @@ -77,18 +81,19 @@ public boolean isStateTransferOnly(DiscoveryNode discoveryNode) { @Override public boolean isElectionQuorum(DiscoveryNode localNode, long localCurrentTerm, long localAcceptedTerm, long localAcceptedVersion, VotingConfiguration lastCommittedConfiguration, VotingConfiguration lastAcceptedConfiguration, - JoinVoteCollection joinVotes) { + VoteCollection joinVotes) { // if local node is voting only, have additional checks on election quorum definition if (isVotingOnlyNode(localNode)) { // if all votes are from voting only nodes, do not elect as master (no need to transfer state) - if (joinVotes.getJoins().stream().allMatch(join -> isVotingOnlyNode(join.getSourceNode()))) { + if (joinVotes.nodes().stream().filter(DiscoveryNode::isMasterNode).allMatch(VotingOnlyNodePlugin::isVotingOnlyNode)) { return false; } // if there's a vote from a full master node with same last accepted term and version, that node should become master // instead, so we should stand down - if (joinVotes.getJoins().stream().anyMatch(join -> isFullMasterNode(join.getSourceNode()) && - join.getLastAcceptedTerm() == localAcceptedTerm && - join.getLastAcceptedVersion() == localAcceptedVersion)) { + if (joinVotes instanceof JoinVoteCollection && + ((JoinVoteCollection) joinVotes).getJoins().stream().anyMatch(join -> isFullMasterNode(join.getSourceNode()) && + join.getLastAcceptedTerm() == localAcceptedTerm && + join.getLastAcceptedVersion() == localAcceptedVersion)) { return false; } } diff --git a/x-pack/plugin/voting-only-node/src/test/java/org/elasticsearch/cluster/coordination/VotingOnlyNodePluginIT.java b/x-pack/plugin/voting-only-node/src/test/java/org/elasticsearch/cluster/coordination/VotingOnlyNodePluginIT.java index c408b2d390b69..e86789969d3cc 100644 --- a/x-pack/plugin/voting-only-node/src/test/java/org/elasticsearch/cluster/coordination/VotingOnlyNodePluginIT.java +++ b/x-pack/plugin/voting-only-node/src/test/java/org/elasticsearch/cluster/coordination/VotingOnlyNodePluginIT.java @@ -5,15 +5,26 @@ */ package org.elasticsearch.cluster.coordination; +import org.elasticsearch.common.Priority; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.discovery.MasterNotDiscoveredException; +import org.elasticsearch.node.Node; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.test.ESIntegTestCase; +import org.elasticsearch.test.ESIntegTestCase.Scope; +import org.elasticsearch.test.junit.annotations.TestLogging; import java.io.IOException; import java.util.Collection; import java.util.Collections; -@ESIntegTestCase.ClusterScope(numDataNodes = 0) +import static org.hamcrest.CoreMatchers.containsString; +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.Matchers.hasSize; +import static org.hamcrest.Matchers.nullValue; + +@ESIntegTestCase.ClusterScope(scope = Scope.TEST, numDataNodes = 0, autoManageMasterNodes = false) +@TestLogging("_root:DEBUG,org.elasticsearch.cluster:TRACE,org.elasticsearch.discovery:TRACE") public class VotingOnlyNodePluginIT extends ESIntegTestCase { @Override @@ -21,13 +32,67 @@ protected Collection> nodePlugins() { return Collections.singleton(VotingOnlyNodePlugin.class); } - public void testOneVotingOnlyNode() throws IOException { + public void testOneVotingOnlyNode() throws Exception { + internalCluster().setBootstrapMasterNodeIndex(0); internalCluster().startNodes(2); final String votingOnlyNode = internalCluster().startNode(Settings.builder().put(VotingOnlyNodePlugin.VOTING_ONLY_NODE_SETTING.getKey(), true)); + assertBusy(() -> assertThat(client().admin().cluster().prepareState().get().getState().getLastCommittedConfiguration().getNodeIds(), + hasSize(3))); internalCluster().stopCurrentMasterNode(); assertNotEquals(votingOnlyNode, internalCluster().getMasterName()); } + + public void testRequireVotingOnlyNodeToBeMasterEligible() { + internalCluster().setBootstrapMasterNodeIndex(0); + IllegalStateException ise = expectThrows(IllegalStateException.class, () -> internalCluster().startNode(Settings.builder() + .put(Node.NODE_MASTER_SETTING.getKey(), false) + .put(VotingOnlyNodePlugin.VOTING_ONLY_NODE_SETTING.getKey(), true) + .build())); + assertThat(ise.getMessage(), containsString("voting-only node must be master-eligible")); + } + + public void testPreferFullMasterOverVotingOnlyNodes() throws Exception { + internalCluster().setBootstrapMasterNodeIndex(0); + internalCluster().startNodes(2); + internalCluster().startNode(Settings.builder().put(VotingOnlyNodePlugin.VOTING_ONLY_NODE_SETTING.getKey(), true).build()); + internalCluster().startDataOnlyNodes(randomInt(2)); + assertBusy(() -> assertThat( + client().admin().cluster().prepareState().get().getState().getLastCommittedConfiguration().getNodeIds().size(), + equalTo(3))); + final String originalMaster = internalCluster().getMasterName(); + + internalCluster().stopCurrentMasterNode(); + client().admin().cluster().prepareHealth().setWaitForEvents(Priority.LANGUID).get(); + assertNotEquals(originalMaster, internalCluster().getMasterName()); + assertThat( + VotingOnlyNodePlugin.isVotingOnlyNode(client().admin().cluster().prepareState().get().getState().nodes().getMasterNode()), + equalTo(false)); + } + + public void testVotingOnlyNodesCannotBeMasterWithoutFullMasterNodes() throws Exception { + internalCluster().setBootstrapMasterNodeIndex(0); + internalCluster().startNode(); + internalCluster().startNodes(2, Settings.builder().put(VotingOnlyNodePlugin.VOTING_ONLY_NODE_SETTING.getKey(), true).build()); + internalCluster().startDataOnlyNodes(randomInt(2)); + assertBusy(() -> assertThat( + client().admin().cluster().prepareState().get().getState().getLastCommittedConfiguration().getNodeIds().size(), + equalTo(3))); + + internalCluster().stopCurrentMasterNode(); + + try { + assertThat(client().admin().cluster().prepareState().setMasterNodeTimeout("100ms") + .execute().actionGet().getState().nodes().getMasterNodeId(), nullValue()); + fail("should not be able to find master"); + } catch (MasterNotDiscoveredException e) { + // all is well, no master elected + } + + // start a fresh full master node, which will be brought into the cluster as master by the voting-only nodes + final String newMaster = internalCluster().startNode(); + assertEquals(newMaster, internalCluster().getMasterName()); + } } From 1a50872a9bcb32cc2ea74b7ce9c809357c5408a5 Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Wed, 19 Jun 2019 17:05:34 +0200 Subject: [PATCH 06/29] fix test --- .../ClusterFormationFailureHelper.java | 18 +++------ .../cluster/coordination/Coordinator.java | 39 ++++++++++++------- .../cluster/node/DiscoveryNode.java | 5 +++ .../coordination/VotingOnlyNodePlugin.java | 2 +- 4 files changed, 38 insertions(+), 26 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/cluster/coordination/ClusterFormationFailureHelper.java b/server/src/main/java/org/elasticsearch/cluster/coordination/ClusterFormationFailureHelper.java index 51af568a2761d..c249458dd23cf 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/ClusterFormationFailureHelper.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/ClusterFormationFailureHelper.java @@ -22,6 +22,7 @@ import org.apache.logging.log4j.Logger; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.coordination.CoordinationMetaData.VotingConfiguration; +import org.elasticsearch.cluster.coordination.CoordinationState.VoteCollection; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.settings.Setting; @@ -183,19 +184,12 @@ String getDescription() { + describeQuorum(clusterState.getLastCommittedConfiguration()); } - final CoordinationState.JoinVoteCollection expectedVotes = new CoordinationState.JoinVoteCollection(); - long term = clusterState.term(); - long lastAcceptedTerm = clusterState.term(); - long lastAcceptedVersion = clusterState.version(); - foundPeers.forEach(node -> expectedVotes.addJoinVote(new Join(node, clusterState.nodes().getLocalNode(), term, lastAcceptedTerm, - lastAcceptedVersion))); - expectedVotes.addJoinVote(new Join(clusterState.nodes().getLocalNode(), clusterState.nodes().getLocalNode(), term, - lastAcceptedTerm, lastAcceptedVersion)); - + final VoteCollection voteCollection = new VoteCollection(); + foundPeers.forEach(voteCollection::addVote); final String isQuorumOrNot - = electionStrategy.isElectionQuorum(clusterState.nodes().getLocalNode(), term, lastAcceptedTerm, lastAcceptedVersion, - clusterState.getLastCommittedConfiguration(), clusterState.getLastAcceptedConfiguration(), expectedVotes) ? - "is a quorum" : "is not a quorum"; + = electionStrategy.isElectionQuorum(clusterState.nodes().getLocalNode(), currentTerm, clusterState.term(), + clusterState.version(), clusterState.getLastCommittedConfiguration(), clusterState.getLastAcceptedConfiguration(), + voteCollection) ? "is a quorum" : "is not a quorum"; return String.format(Locale.ROOT, "master not discovered or elected yet, an election requires %s, have discovered %s which %s; %s", diff --git a/server/src/main/java/org/elasticsearch/cluster/coordination/Coordinator.java b/server/src/main/java/org/elasticsearch/cluster/coordination/Coordinator.java index 0edec587cc5cc..00443bbd8fc21 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/Coordinator.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/Coordinator.java @@ -55,6 +55,7 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.transport.TransportAddress; import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.common.util.concurrent.AbstractRunnable; import org.elasticsearch.common.util.concurrent.EsExecutors; import org.elasticsearch.common.util.concurrent.ListenableFuture; import org.elasticsearch.common.xcontent.XContentHelper; @@ -1310,17 +1311,16 @@ public void onSuccess(String source) { .filter(DiscoveryNode::isMasterNode) .filter(node -> nodeMayWinElection(state, node)) .filter(node -> { - // check if candidate would be able to get an election quorum - final JoinVoteCollection fakeJoinVoteCollection = new JoinVoteCollection(); - completedNodes().forEach(completedNode -> { - fakeJoinVoteCollection.addJoinVote(new Join(completedNode, node, state.term(), - state.term(), state.version())); - }); - return electionStrategy.isElectionQuorum(node, - state.term(), state.term(), state.version(), - state.getLastCommittedConfiguration(), state.getLastAcceptedConfiguration(), - fakeJoinVoteCollection - ); + // check if master candidate would be able to get an election quorum if we were to + // abdicate to it. Assume that every node that completed the publication can provide + // a vote in that next election and has the latest state. + final long futureElectionTerm = state.term() + 1; + final JoinVoteCollection futureJoinVoteCollection = new JoinVoteCollection(); + completedNodes().forEach(completedNode -> futureJoinVoteCollection.addJoinVote( + new Join(completedNode, node, futureElectionTerm, state.term(), state.version()))); + return electionStrategy.isElectionQuorum(node, futureElectionTerm, + state.term(), state.version(), state.getLastCommittedConfiguration(), + state.getLastAcceptedConfiguration(), futureJoinVoteCollection); }) .collect(Collectors.toList()); if (masterCandidates.isEmpty() == false) { @@ -1405,6 +1405,7 @@ protected void sendPublishRequest(DiscoveryNode destination, PublishRequest publ ActionListener responseActionListener) { if (electionStrategy.isStateTransferOnly(getLocalNode())) { if (destination.isMasterNode() && electionStrategy.isStateTransferOnly(destination) == false) { + logger.debug("sendPublishRequest: state transfer to [{}]", destination); publicationContext.sendPublishRequest(destination, publishRequest, wrapWithMutex( ActionListener.map(responseActionListener, resp -> { throw new TransportException( @@ -1412,8 +1413,20 @@ protected void sendPublishRequest(DiscoveryNode destination, PublishRequest publ }))); } else { logger.debug("sendPublishRequest: suppressing state transfer to [{}]", destination); - responseActionListener.onFailure(new TransportException( - new ElasticsearchException("suppressing state transfer at source"))); + // fork response to different thread to allow sending of other publications + transportService.getThreadPool().generic().execute(new AbstractRunnable() { + @Override + public void onFailure(Exception e) { + wrapWithMutex(responseActionListener).onFailure(e); + } + + @Override + protected void doRun() { + wrapWithMutex(responseActionListener).onFailure( + new TransportException( + new ElasticsearchException("suppressing state transfer at source for " + destination))); + } + }); } } else { publicationContext.sendPublishRequest(destination, publishRequest, wrapWithMutex(responseActionListener)); diff --git a/server/src/main/java/org/elasticsearch/cluster/node/DiscoveryNode.java b/server/src/main/java/org/elasticsearch/cluster/node/DiscoveryNode.java index f175f5d6f4bf2..156eb152c5186 100644 --- a/server/src/main/java/org/elasticsearch/cluster/node/DiscoveryNode.java +++ b/server/src/main/java/org/elasticsearch/cluster/node/DiscoveryNode.java @@ -408,6 +408,11 @@ public String toString() { sb.append('{').append(ephemeralId).append('}'); sb.append('{').append(hostName).append('}'); sb.append('{').append(address).append('}'); + if (roles.isEmpty() == false) { + sb.append('{'); + roles.stream().map(DiscoveryNodeRole::roleNameAbbreviation).forEach(sb::append); + sb.append('}'); + } if (!attributes.isEmpty()) { sb.append(attributes); } diff --git a/x-pack/plugin/voting-only-node/src/main/java/org/elasticsearch/cluster/coordination/VotingOnlyNodePlugin.java b/x-pack/plugin/voting-only-node/src/main/java/org/elasticsearch/cluster/coordination/VotingOnlyNodePlugin.java index 4dfc8f0c7013e..01b310643ee9c 100644 --- a/x-pack/plugin/voting-only-node/src/main/java/org/elasticsearch/cluster/coordination/VotingOnlyNodePlugin.java +++ b/x-pack/plugin/voting-only-node/src/main/java/org/elasticsearch/cluster/coordination/VotingOnlyNodePlugin.java @@ -75,7 +75,7 @@ private class VotingOnlyNodeElectionStrategy extends ElectionStrategy.DefaultEle @Override public boolean isStateTransferOnly(DiscoveryNode discoveryNode) { - return discoveryNode.isMasterNode() && discoveryNode.getRoles().contains(VOTING_ONLY_NODE_ROLE); + return isVotingOnlyNode(discoveryNode); } @Override From 2cb2010d9f9a96d0633c7e675b7572f526be260d Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Wed, 19 Jun 2019 18:21:02 +0200 Subject: [PATCH 07/29] fix build and use transport intercepter --- .../cluster/coordination/Coordinator.java | 32 +--- .../coordination/ElectionStrategy.java | 9 +- .../ClusterFormationFailureHelperTests.java | 51 +++--- .../coordination/CoordinationStateTests.java | 2 +- .../coordination/ReconfiguratorTests.java | 6 +- x-pack/plugin/voting-only-node/build.gradle | 8 +- .../coordination/VotingOnlyNodePlugin.java | 148 ++++++++++++++---- ...IT.java => VotingOnlyNodePluginTests.java} | 3 +- 8 files changed, 161 insertions(+), 98 deletions(-) rename x-pack/plugin/voting-only-node/src/test/java/org/elasticsearch/cluster/coordination/{VotingOnlyNodePluginIT.java => VotingOnlyNodePluginTests.java} (98%) diff --git a/server/src/main/java/org/elasticsearch/cluster/coordination/Coordinator.java b/server/src/main/java/org/elasticsearch/cluster/coordination/Coordinator.java index 00443bbd8fc21..ec610f9ec6aaa 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/Coordinator.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/Coordinator.java @@ -22,7 +22,6 @@ import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; import org.apache.lucene.util.SetOnce; -import org.elasticsearch.ElasticsearchException; import org.elasticsearch.action.ActionListener; import org.elasticsearch.cluster.ClusterChangedEvent; import org.elasticsearch.cluster.ClusterName; @@ -55,7 +54,6 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.transport.TransportAddress; import org.elasticsearch.common.unit.TimeValue; -import org.elasticsearch.common.util.concurrent.AbstractRunnable; import org.elasticsearch.common.util.concurrent.EsExecutors; import org.elasticsearch.common.util.concurrent.ListenableFuture; import org.elasticsearch.common.xcontent.XContentHelper; @@ -69,7 +67,6 @@ import org.elasticsearch.discovery.SeedHostsResolver; import org.elasticsearch.threadpool.Scheduler; import org.elasticsearch.threadpool.ThreadPool.Names; -import org.elasticsearch.transport.TransportException; import org.elasticsearch.transport.TransportResponse.Empty; import org.elasticsearch.transport.TransportService; @@ -1403,34 +1400,7 @@ protected void onMissingJoin(DiscoveryNode discoveryNode) { @Override protected void sendPublishRequest(DiscoveryNode destination, PublishRequest publishRequest, ActionListener responseActionListener) { - if (electionStrategy.isStateTransferOnly(getLocalNode())) { - if (destination.isMasterNode() && electionStrategy.isStateTransferOnly(destination) == false) { - logger.debug("sendPublishRequest: state transfer to [{}]", destination); - publicationContext.sendPublishRequest(destination, publishRequest, wrapWithMutex( - ActionListener.map(responseActionListener, resp -> { - throw new TransportException( - new ElasticsearchException("ignoring successful publish response for state transfer only: " + resp)); - }))); - } else { - logger.debug("sendPublishRequest: suppressing state transfer to [{}]", destination); - // fork response to different thread to allow sending of other publications - transportService.getThreadPool().generic().execute(new AbstractRunnable() { - @Override - public void onFailure(Exception e) { - wrapWithMutex(responseActionListener).onFailure(e); - } - - @Override - protected void doRun() { - wrapWithMutex(responseActionListener).onFailure( - new TransportException( - new ElasticsearchException("suppressing state transfer at source for " + destination))); - } - }); - } - } else { - publicationContext.sendPublishRequest(destination, publishRequest, wrapWithMutex(responseActionListener)); - } + publicationContext.sendPublishRequest(destination, publishRequest, wrapWithMutex(responseActionListener)); } @Override diff --git a/server/src/main/java/org/elasticsearch/cluster/coordination/ElectionStrategy.java b/server/src/main/java/org/elasticsearch/cluster/coordination/ElectionStrategy.java index 70bcaeb8d0a2c..2d392a8e5c977 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/ElectionStrategy.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/ElectionStrategy.java @@ -25,10 +25,8 @@ public interface ElectionStrategy { /** - * Whether the given node is only here to ensure state transfer to another node + * Whether there is an election quorum from the point of view of the provided local node */ - boolean isStateTransferOnly(DiscoveryNode discoveryNode); - boolean isElectionQuorum(DiscoveryNode localNode, long localCurrentTerm, long localAcceptedTerm, long localAcceptedVersion, VotingConfiguration lastCommittedConfiguration, VotingConfiguration lastAcceptedConfiguration, VoteCollection joinVotes); @@ -37,11 +35,6 @@ class DefaultElectionStrategy implements ElectionStrategy { public static final ElectionStrategy INSTANCE = new DefaultElectionStrategy(); - @Override - public boolean isStateTransferOnly(DiscoveryNode discoveryNode) { - return false; - } - @Override public boolean isElectionQuorum(DiscoveryNode localNode, long localCurrentTerm, long localAcceptedTerm, long localAcceptedVersion, VotingConfiguration lastCommittedConfiguration, VotingConfiguration lastAcceptedConfiguration, diff --git a/server/src/test/java/org/elasticsearch/cluster/coordination/ClusterFormationFailureHelperTests.java b/server/src/test/java/org/elasticsearch/cluster/coordination/ClusterFormationFailureHelperTests.java index eabfb0ed2d6d8..db94076e40bb0 100644 --- a/server/src/test/java/org/elasticsearch/cluster/coordination/ClusterFormationFailureHelperTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/coordination/ClusterFormationFailureHelperTests.java @@ -144,17 +144,20 @@ public void testDescriptionOnMasterIneligibleNodes() { final ClusterState clusterState = ClusterState.builder(ClusterName.DEFAULT) .version(12L).nodes(DiscoveryNodes.builder().add(localNode).localNodeId(localNode.getId())).build(); - assertThat(new ClusterFormationState(Settings.EMPTY, clusterState, emptyList(), emptyList(), 15L, electionStrategy).getDescription(), + assertThat(new ClusterFormationState(Settings.EMPTY, clusterState, emptyList(), emptyList(), 15L, electionStrategy) + .getDescription(), is("master not discovered yet: have discovered []; discovery will continue using [] from hosts providers " + "and [] from last-known cluster state; node term 15, last-accepted version 12 in term 0")); final TransportAddress otherAddress = buildNewFakeTransportAddress(); - assertThat(new ClusterFormationState(Settings.EMPTY, clusterState, singletonList(otherAddress), emptyList(), 16L, electionStrategy).getDescription(), + assertThat(new ClusterFormationState(Settings.EMPTY, clusterState, singletonList(otherAddress), emptyList(), 16L, electionStrategy) + .getDescription(), is("master not discovered yet: have discovered []; discovery will continue using [" + otherAddress + "] from hosts providers and [] from last-known cluster state; node term 16, last-accepted version 12 in term 0")); final DiscoveryNode otherNode = new DiscoveryNode("other", buildNewFakeTransportAddress(), Version.CURRENT); - assertThat(new ClusterFormationState(Settings.EMPTY, clusterState, emptyList(), singletonList(otherNode), 17L, electionStrategy).getDescription(), + assertThat(new ClusterFormationState(Settings.EMPTY, clusterState, emptyList(), singletonList(otherNode), 17L, electionStrategy) + .getDescription(), is("master not discovered yet: have discovered [" + otherNode + "]; discovery will continue using [] from hosts providers " + "and [] from last-known cluster state; node term 17, last-accepted version 12 in term 0")); } @@ -173,14 +176,16 @@ public void testDescriptionBeforeBootstrapping() { "] from last-known cluster state; node term 1, last-accepted version 7 in term 4")); final TransportAddress otherAddress = buildNewFakeTransportAddress(); - assertThat(new ClusterFormationState(Settings.EMPTY, clusterState, singletonList(otherAddress), emptyList(), 2L, electionStrategy).getDescription(), + assertThat(new ClusterFormationState(Settings.EMPTY, clusterState, singletonList(otherAddress), emptyList(), 2L, electionStrategy) + .getDescription(), is("master not discovered yet, this node has not previously joined a bootstrapped cluster, and " + "[cluster.initial_master_nodes] is empty on this node: have discovered []; " + "discovery will continue using [" + otherAddress + "] from hosts providers and [" + localNode + "] from last-known cluster state; node term 2, last-accepted version 7 in term 4")); final DiscoveryNode otherNode = new DiscoveryNode("other", buildNewFakeTransportAddress(), Version.CURRENT); - assertThat(new ClusterFormationState(Settings.EMPTY, clusterState, emptyList(), singletonList(otherNode), 3L, electionStrategy).getDescription(), + assertThat(new ClusterFormationState(Settings.EMPTY, clusterState, emptyList(), singletonList(otherNode), 3L, electionStrategy) + .getDescription(), is("master not discovered yet, this node has not previously joined a bootstrapped cluster, and " + "[cluster.initial_master_nodes] is empty on this node: have discovered [" + otherNode + "]; " + "discovery will continue using [] from hosts providers and [" + localNode + @@ -224,21 +229,24 @@ public void testDescriptionAfterDetachCluster() { "] from last-known cluster state; node term 0, last-accepted version 0 in term 0")); final TransportAddress otherAddress = buildNewFakeTransportAddress(); - assertThat(new ClusterFormationState(Settings.EMPTY, clusterState, singletonList(otherAddress), emptyList(), 0L, electionStrategy).getDescription(), + assertThat(new ClusterFormationState(Settings.EMPTY, clusterState, singletonList(otherAddress), emptyList(), 0L, electionStrategy) + .getDescription(), is("master not discovered yet and this node was detached from its previous cluster, " + "have discovered []; " + "discovery will continue using [" + otherAddress + "] from hosts providers and [" + localNode + "] from last-known cluster state; node term 0, last-accepted version 0 in term 0")); final DiscoveryNode otherNode = new DiscoveryNode("otherNode", buildNewFakeTransportAddress(), Version.CURRENT); - assertThat(new ClusterFormationState(Settings.EMPTY, clusterState, emptyList(), singletonList(otherNode), 0L, electionStrategy).getDescription(), + assertThat(new ClusterFormationState(Settings.EMPTY, clusterState, emptyList(), singletonList(otherNode), 0L, electionStrategy) + .getDescription(), is("master not discovered yet and this node was detached from its previous cluster, " + "have discovered [" + otherNode + "]; " + "discovery will continue using [] from hosts providers and [" + localNode + "] from last-known cluster state; node term 0, last-accepted version 0 in term 0")); final DiscoveryNode yetAnotherNode = new DiscoveryNode("yetAnotherNode", buildNewFakeTransportAddress(), Version.CURRENT); - assertThat(new ClusterFormationState(Settings.EMPTY, clusterState, emptyList(), singletonList(yetAnotherNode), 0L, electionStrategy).getDescription(), + assertThat(new ClusterFormationState(Settings.EMPTY, clusterState, emptyList(), singletonList(yetAnotherNode), 0L, electionStrategy) + .getDescription(), is("master not discovered yet and this node was detached from its previous cluster, " + "have discovered [" + yetAnotherNode + "]; " + "discovery will continue using [] from hosts providers and [" + localNode + @@ -258,34 +266,38 @@ public void testDescriptionAfterBootstrapping() { "] from last-known cluster state; node term 0, last-accepted version 0 in term 0")); final TransportAddress otherAddress = buildNewFakeTransportAddress(); - assertThat(new ClusterFormationState(Settings.EMPTY, clusterState, singletonList(otherAddress), emptyList(), 0L, electionStrategy).getDescription(), + assertThat(new ClusterFormationState(Settings.EMPTY, clusterState, singletonList(otherAddress), emptyList(), 0L, electionStrategy) + .getDescription(), is("master not discovered or elected yet, an election requires a node with id [otherNode], " + "have discovered [] which is not a quorum; " + "discovery will continue using [" + otherAddress + "] from hosts providers and [" + localNode + "] from last-known cluster state; node term 0, last-accepted version 0 in term 0")); final DiscoveryNode otherNode = new DiscoveryNode("otherNode", buildNewFakeTransportAddress(), Version.CURRENT); - assertThat(new ClusterFormationState(Settings.EMPTY, clusterState, emptyList(), singletonList(otherNode), 0L, electionStrategy).getDescription(), + assertThat(new ClusterFormationState(Settings.EMPTY, clusterState, emptyList(), singletonList(otherNode), 0L, electionStrategy) + .getDescription(), is("master not discovered or elected yet, an election requires a node with id [otherNode], " + "have discovered [" + otherNode + "] which is a quorum; " + "discovery will continue using [] from hosts providers and [" + localNode + "] from last-known cluster state; node term 0, last-accepted version 0 in term 0")); final DiscoveryNode yetAnotherNode = new DiscoveryNode("yetAnotherNode", buildNewFakeTransportAddress(), Version.CURRENT); - assertThat(new ClusterFormationState(Settings.EMPTY, clusterState, emptyList(), singletonList(yetAnotherNode), 0L, electionStrategy).getDescription(), + assertThat(new ClusterFormationState(Settings.EMPTY, clusterState, emptyList(), singletonList(yetAnotherNode), 0L, electionStrategy) + .getDescription(), is("master not discovered or elected yet, an election requires a node with id [otherNode], " + "have discovered [" + yetAnotherNode + "] which is not a quorum; " + "discovery will continue using [] from hosts providers and [" + localNode + "] from last-known cluster state; node term 0, last-accepted version 0 in term 0")); - assertThat(new ClusterFormationState(Settings.EMPTY, state(localNode, "n1", "n2"), emptyList(), emptyList(), 0L, electionStrategy).getDescription(), + assertThat(new ClusterFormationState(Settings.EMPTY, state(localNode, "n1", "n2"), emptyList(), emptyList(), 0L, electionStrategy) + .getDescription(), is("master not discovered or elected yet, an election requires two nodes with ids [n1, n2], " + "have discovered [] which is not a quorum; " + "discovery will continue using [] from hosts providers and [" + localNode + "] from last-known cluster state; node term 0, last-accepted version 0 in term 0")); - assertThat(new ClusterFormationState(Settings.EMPTY, state(localNode, "n1", "n2", "n3"), emptyList(), emptyList(), 0L, electionStrategy) - .getDescription(), + assertThat(new ClusterFormationState(Settings.EMPTY, state(localNode, "n1", "n2", "n3"), emptyList(), emptyList(), 0L, + electionStrategy).getDescription(), is("master not discovered or elected yet, an election requires at least 2 nodes with ids from [n1, n2, n3], " + "have discovered [] which is not a quorum; " + "discovery will continue using [] from hosts providers and [" + localNode + @@ -298,15 +310,15 @@ public void testDescriptionAfterBootstrapping() { "discovery will continue using [] from hosts providers and [" + localNode + "] from last-known cluster state; node term 0, last-accepted version 0 in term 0")); - assertThat(new ClusterFormationState(Settings.EMPTY, state(localNode, "n1", "n2", "n3", "n4"), emptyList(), emptyList(), 0L, electionStrategy) - .getDescription(), + assertThat(new ClusterFormationState(Settings.EMPTY, state(localNode, "n1", "n2", "n3", "n4"), emptyList(), emptyList(), 0L, + electionStrategy).getDescription(), is("master not discovered or elected yet, an election requires at least 3 nodes with ids from [n1, n2, n3, n4], " + "have discovered [] which is not a quorum; " + "discovery will continue using [] from hosts providers and [" + localNode + "] from last-known cluster state; node term 0, last-accepted version 0 in term 0")); - assertThat(new ClusterFormationState(Settings.EMPTY, state(localNode, "n1", "n2", "n3", "n4", "n5"), emptyList(), emptyList(), 0L, electionStrategy) - .getDescription(), + assertThat(new ClusterFormationState(Settings.EMPTY, state(localNode, "n1", "n2", "n3", "n4", "n5"), emptyList(), emptyList(), 0L, + electionStrategy).getDescription(), is("master not discovered or elected yet, an election requires at least 3 nodes with ids from [n1, n2, n3, n4, n5], " + "have discovered [] which is not a quorum; " + "discovery will continue using [] from hosts providers and [" + localNode + @@ -320,7 +332,8 @@ public void testDescriptionAfterBootstrapping() { "] from last-known cluster state; node term 0, last-accepted version 0 in term 0")); assertThat(new ClusterFormationState(Settings.EMPTY, state(localNode, "n1", "n2", "n3", - BOOTSTRAP_PLACEHOLDER_PREFIX + "n4", BOOTSTRAP_PLACEHOLDER_PREFIX + "n5"), emptyList(), emptyList(), 0L, electionStrategy).getDescription(), + BOOTSTRAP_PLACEHOLDER_PREFIX + "n4", BOOTSTRAP_PLACEHOLDER_PREFIX + "n5"), emptyList(), emptyList(), 0L, electionStrategy) + .getDescription(), is("master not discovered or elected yet, an election requires 3 nodes with ids [n1, n2, n3], " + "have discovered [] which is not a quorum; " + "discovery will continue using [] from hosts providers and [" + localNode + diff --git a/server/src/test/java/org/elasticsearch/cluster/coordination/CoordinationStateTests.java b/server/src/test/java/org/elasticsearch/cluster/coordination/CoordinationStateTests.java index 0757d41efc037..83045057ad140 100644 --- a/server/src/test/java/org/elasticsearch/cluster/coordination/CoordinationStateTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/coordination/CoordinationStateTests.java @@ -820,7 +820,7 @@ static class ClusterNode { this.localNode = localNode; persistedState = new InMemoryPersistedState(0L, clusterState(0L, 0L, localNode, VotingConfiguration.EMPTY_CONFIG, VotingConfiguration.EMPTY_CONFIG, 0L)); - electionStrategy = randomBoolean() ? ElectionStrategy.DefaultElectionStrategy.INSTANCE : discoveryNode -> rarely(); + electionStrategy = ElectionStrategy.DefaultElectionStrategy.INSTANCE; state = new CoordinationState(localNode, persistedState, electionStrategy); } diff --git a/server/src/test/java/org/elasticsearch/cluster/coordination/ReconfiguratorTests.java b/server/src/test/java/org/elasticsearch/cluster/coordination/ReconfiguratorTests.java index 8d535a5f1bd29..a1d12d98398ca 100644 --- a/server/src/test/java/org/elasticsearch/cluster/coordination/ReconfiguratorTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/coordination/ReconfiguratorTests.java @@ -210,14 +210,12 @@ private void check(Set liveNodes, Set retired, String mas } private Reconfigurator makeReconfigurator(Settings settings) { - return new Reconfigurator(settings, new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS), - ElectionStrategy.DefaultElectionStrategy.INSTANCE); + return new Reconfigurator(settings, new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS)); } public void testDynamicSetting() { final ClusterSettings clusterSettings = new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); - final Reconfigurator reconfigurator = new Reconfigurator(Settings.EMPTY, clusterSettings, - ElectionStrategy.DefaultElectionStrategy.INSTANCE); + final Reconfigurator reconfigurator = new Reconfigurator(Settings.EMPTY, clusterSettings); final VotingConfiguration initialConfig = conf("a", "b", "c", "d", "e"); Set twoNodes = nodes("a", "b"); diff --git a/x-pack/plugin/voting-only-node/build.gradle b/x-pack/plugin/voting-only-node/build.gradle index c1efe638a56ae..ab46fe68cfdae 100644 --- a/x-pack/plugin/voting-only-node/build.gradle +++ b/x-pack/plugin/voting-only-node/build.gradle @@ -7,15 +7,13 @@ esplugin { classname 'org.elasticsearch.cluster.coordination.VotingOnlyNodePlugin' extendedPlugins = ['x-pack-core'] } -archivesBaseName = 'x-pack-voting-only-node' dependencies { - compileOnly "org.elasticsearch:elasticsearch:${version}" - - // "org.elasticsearch.plugin:x-pack-core:${version}" doesn't work with idea because the testArtifacts are also here compileOnly project(path: xpackModule('core'), configuration: 'default') testCompile project(path: xpackModule('core'), configuration: 'testArtifacts') if (isEclipse) { - testCompile project(path: xpackModule('core-tests'), configuration: 'testArtifacts') + testCompile project(path: xpackModule('core-tests'), configuration: 'testArtifacts') } } + +integTest.enabled = false \ No newline at end of file diff --git a/x-pack/plugin/voting-only-node/src/main/java/org/elasticsearch/cluster/coordination/VotingOnlyNodePlugin.java b/x-pack/plugin/voting-only-node/src/main/java/org/elasticsearch/cluster/coordination/VotingOnlyNodePlugin.java index 01b310643ee9c..a066dfeab41cd 100644 --- a/x-pack/plugin/voting-only-node/src/main/java/org/elasticsearch/cluster/coordination/VotingOnlyNodePlugin.java +++ b/x-pack/plugin/voting-only-node/src/main/java/org/elasticsearch/cluster/coordination/VotingOnlyNodePlugin.java @@ -5,37 +5,85 @@ */ package org.elasticsearch.cluster.coordination; +import org.apache.lucene.util.SetOnce; +import org.elasticsearch.ElasticsearchException; +import org.elasticsearch.client.Client; import org.elasticsearch.cluster.coordination.CoordinationMetaData.VotingConfiguration; import org.elasticsearch.cluster.coordination.CoordinationState.JoinVoteCollection; import org.elasticsearch.cluster.coordination.CoordinationState.VoteCollection; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.node.DiscoveryNodeRole; +import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.common.io.stream.NamedWriteableRegistry; +import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.util.concurrent.ThreadContext; +import org.elasticsearch.common.xcontent.NamedXContentRegistry; import org.elasticsearch.discovery.DiscoveryModule; +import org.elasticsearch.env.Environment; +import org.elasticsearch.env.NodeEnvironment; import org.elasticsearch.node.Node; import org.elasticsearch.plugins.DiscoveryPlugin; +import org.elasticsearch.plugins.NetworkPlugin; import org.elasticsearch.plugins.Plugin; - +import org.elasticsearch.script.ScriptService; +import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.transport.Transport; +import org.elasticsearch.transport.TransportException; +import org.elasticsearch.transport.TransportInterceptor; +import org.elasticsearch.transport.TransportRequest; +import org.elasticsearch.transport.TransportRequestOptions; +import org.elasticsearch.transport.TransportResponse; +import org.elasticsearch.transport.TransportResponseHandler; +import org.elasticsearch.watcher.ResourceWatcherService; + +import java.io.IOException; +import java.util.Collection; import java.util.Collections; import java.util.List; import java.util.Map; import java.util.Set; +import java.util.function.Supplier; -public class VotingOnlyNodePlugin extends Plugin implements DiscoveryPlugin { +public class VotingOnlyNodePlugin extends Plugin implements DiscoveryPlugin, NetworkPlugin { public static final Setting VOTING_ONLY_NODE_SETTING = Setting.boolSetting("node.voting_only", false, Setting.Property.NodeScope); + private static final String VOTING_ONLY_ELECTION_TYPE = "supports-voting-only"; + + private static DiscoveryNodeRole VOTING_ONLY_NODE_ROLE = new DiscoveryNodeRole("voting-only", "v") { + @Override + protected Setting roleSetting() { + return VOTING_ONLY_NODE_SETTING; + } + }; + private final Settings settings; + private final SetOnce threadPool; - private final Boolean isVotingOnlyNode; + private final boolean isVotingOnlyNode; public VotingOnlyNodePlugin(Settings settings) { this.settings = settings; + threadPool = new SetOnce<>(); isVotingOnlyNode = VOTING_ONLY_NODE_SETTING.get(settings); } + public static boolean isVotingOnlyNode(DiscoveryNode discoveryNode) { + return discoveryNode.getRoles().contains(VOTING_ONLY_NODE_ROLE); + } + + public static boolean isFullMasterNode(DiscoveryNode discoveryNode) { + return discoveryNode.isMasterNode() && discoveryNode.getRoles().contains(VOTING_ONLY_NODE_ROLE) == false; + } + + @Override + public List> getSettings() { + return Collections.singletonList(VOTING_ONLY_NODE_SETTING); + } + @Override public Set getRoles() { if (isVotingOnlyNode && Node.NODE_MASTER_SETTING.get(settings) == false) { @@ -45,38 +93,39 @@ public Set getRoles() { } @Override - public List> getSettings() { - return Collections.singletonList(VOTING_ONLY_NODE_SETTING); + public Collection createComponents(Client client, ClusterService clusterService, ThreadPool threadPool, + ResourceWatcherService resourceWatcherService, ScriptService scriptService, + NamedXContentRegistry xContentRegistry, Environment environment, + NodeEnvironment nodeEnvironment, NamedWriteableRegistry namedWriteableRegistry) { + this.threadPool.set(threadPool); + return Collections.emptyList(); } - private static final String VOTING_ONLY_ELECTION_TYPE = "supports-voting-only"; - @Override - public Settings additionalSettings() { - return Settings.builder().put(DiscoveryModule.ELECTION_TYPE_SETTING.getKey(), VOTING_ONLY_ELECTION_TYPE).build(); + public Map getElectionStrategies() { + return Collections.singletonMap(VOTING_ONLY_ELECTION_TYPE, new VotingOnlyNodeElectionStrategy()); } - static DiscoveryNodeRole VOTING_ONLY_NODE_ROLE = new DiscoveryNodeRole("voting-only", "v") { - @Override - protected Setting roleSetting() { - return VOTING_ONLY_NODE_SETTING; + @Override + public List getTransportInterceptors(NamedWriteableRegistry namedWriteableRegistry, ThreadContext threadContext) { + if (isVotingOnlyNode) { + return Collections.singletonList(new TransportInterceptor() { + @Override + public AsyncSender interceptSender(AsyncSender sender) { + return new VotingOnlyNodeAsyncSender(sender, threadPool::get); + } + }); + } else { + return Collections.emptyList(); } - }; - - public static boolean isVotingOnlyNode(DiscoveryNode discoveryNode) { - return discoveryNode.getRoles().contains(VOTING_ONLY_NODE_ROLE); } - public static boolean isFullMasterNode(DiscoveryNode discoveryNode) { - return discoveryNode.isMasterNode() && discoveryNode.getRoles().contains(VOTING_ONLY_NODE_ROLE) == false; + @Override + public Settings additionalSettings() { + return Settings.builder().put(DiscoveryModule.ELECTION_TYPE_SETTING.getKey(), VOTING_ONLY_ELECTION_TYPE).build(); } - private class VotingOnlyNodeElectionStrategy extends ElectionStrategy.DefaultElectionStrategy { - - @Override - public boolean isStateTransferOnly(DiscoveryNode discoveryNode) { - return isVotingOnlyNode(discoveryNode); - } + static class VotingOnlyNodeElectionStrategy extends ElectionStrategy.DefaultElectionStrategy { @Override public boolean isElectionQuorum(DiscoveryNode localNode, long localCurrentTerm, long localAcceptedTerm, long localAcceptedVersion, @@ -103,8 +152,51 @@ public boolean isElectionQuorum(DiscoveryNode localNode, long localCurrentTerm, } } - @Override - public Map getElectionStrategies() { - return Collections.singletonMap(VOTING_ONLY_ELECTION_TYPE, new VotingOnlyNodeElectionStrategy()); + static class VotingOnlyNodeAsyncSender implements TransportInterceptor.AsyncSender { + private final TransportInterceptor.AsyncSender sender; + private final Supplier threadPoolSupplier; + + VotingOnlyNodeAsyncSender(TransportInterceptor.AsyncSender sender, Supplier threadPoolSupplier) { + this.sender = sender; + this.threadPoolSupplier = threadPoolSupplier; + } + + @Override + public void sendRequest(Transport.Connection connection, String action, TransportRequest request, + TransportRequestOptions options, TransportResponseHandler handler) { + if (action.equals(PublicationTransportHandler.PUBLISH_STATE_ACTION_NAME)) { + final DiscoveryNode destinationNode = connection.getNode(); + if (isFullMasterNode(destinationNode)) { + sender.sendRequest(connection, action, request, options, new TransportResponseHandler<>() { + @Override + public void handleResponse(TransportResponse response) { + handler.handleException(new TransportException( + new ElasticsearchException("ignoring successful publish response for state transfer only: " + response))); + } + + @Override + public void handleException(TransportException exp) { + handler.handleException(exp); + } + + @Override + public String executor() { + return handler.executor(); + } + + @Override + public TransportResponse read(StreamInput in) throws IOException { + return handler.read(in); + } + }); + return; + } else { + threadPoolSupplier.get().generic().execute(() -> handler.handleException(new TransportException( + new ElasticsearchException("voting-only node skipping publication to [" + destinationNode + "]")))); + return; + } + } + sender.sendRequest(connection, action, request, options, handler); + } } } diff --git a/x-pack/plugin/voting-only-node/src/test/java/org/elasticsearch/cluster/coordination/VotingOnlyNodePluginIT.java b/x-pack/plugin/voting-only-node/src/test/java/org/elasticsearch/cluster/coordination/VotingOnlyNodePluginTests.java similarity index 98% rename from x-pack/plugin/voting-only-node/src/test/java/org/elasticsearch/cluster/coordination/VotingOnlyNodePluginIT.java rename to x-pack/plugin/voting-only-node/src/test/java/org/elasticsearch/cluster/coordination/VotingOnlyNodePluginTests.java index e86789969d3cc..dedfa041ecf6d 100644 --- a/x-pack/plugin/voting-only-node/src/test/java/org/elasticsearch/cluster/coordination/VotingOnlyNodePluginIT.java +++ b/x-pack/plugin/voting-only-node/src/test/java/org/elasticsearch/cluster/coordination/VotingOnlyNodePluginTests.java @@ -14,7 +14,6 @@ import org.elasticsearch.test.ESIntegTestCase.Scope; import org.elasticsearch.test.junit.annotations.TestLogging; -import java.io.IOException; import java.util.Collection; import java.util.Collections; @@ -25,7 +24,7 @@ @ESIntegTestCase.ClusterScope(scope = Scope.TEST, numDataNodes = 0, autoManageMasterNodes = false) @TestLogging("_root:DEBUG,org.elasticsearch.cluster:TRACE,org.elasticsearch.discovery:TRACE") -public class VotingOnlyNodePluginIT extends ESIntegTestCase { +public class VotingOnlyNodePluginTests extends ESIntegTestCase { @Override protected Collection> nodePlugins() { From 838382341145b9e3faa77d48b241bc957ddfa0fb Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Wed, 19 Jun 2019 19:06:27 +0200 Subject: [PATCH 08/29] move tests --- A | 0 .../coordination/CoordinationStateTests.java | 189 +------------ .../CoordinationStateTestCluster.java | 264 ++++++++++++++++++ .../coordination/VotingOnlyNodePlugin.java | 2 +- .../VotingOnlyNodeCoordinationStateTests.java | 30 ++ 5 files changed, 299 insertions(+), 186 deletions(-) create mode 100644 A create mode 100644 test/framework/src/main/java/org/elasticsearch/cluster/coordination/CoordinationStateTestCluster.java create mode 100644 x-pack/plugin/voting-only-node/src/test/java/org/elasticsearch/cluster/coordination/VotingOnlyNodeCoordinationStateTests.java diff --git a/A b/A new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/server/src/test/java/org/elasticsearch/cluster/coordination/CoordinationStateTests.java b/server/src/test/java/org/elasticsearch/cluster/coordination/CoordinationStateTests.java index 83045057ad140..ea1706652900e 100644 --- a/server/src/test/java/org/elasticsearch/cluster/coordination/CoordinationStateTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/coordination/CoordinationStateTests.java @@ -36,18 +36,13 @@ import org.elasticsearch.test.EqualsHashCodeTestUtils; import org.junit.Before; -import java.util.ArrayList; import java.util.Collections; -import java.util.List; import java.util.Optional; -import java.util.Set; import java.util.stream.Collectors; import java.util.stream.IntStream; -import static java.util.stream.Collectors.toSet; import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.equalTo; -import static org.hamcrest.Matchers.hasSize; public class CoordinationStateTests extends ESTestCase { @@ -765,7 +760,10 @@ public void testVoteCollection() { } public void testSafety() { - new Cluster(randomIntBetween(1, 5)).runRandomly(); + new CoordinationStateTestCluster(IntStream.range(0, randomIntBetween(1, 5)) + .mapToObj(i -> new DiscoveryNode("node_" + i, buildNewFakeTransportAddress(), Version.CURRENT)) + .collect(Collectors.toList()), ElectionStrategy.DefaultElectionStrategy.INSTANCE) + .runRandomly(); } public static CoordinationState createCoordinationState(PersistedState storage, DiscoveryNode localNode) { @@ -808,183 +806,4 @@ public static ClusterState setValue(ClusterState clusterState, long value) { public static long value(ClusterState clusterState) { return clusterState.metaData().persistentSettings().getAsLong("value", 0L); } - - static class ClusterNode { - - final DiscoveryNode localNode; - final PersistedState persistedState; - private final ElectionStrategy electionStrategy; - CoordinationState state; - - ClusterNode(DiscoveryNode localNode) { - this.localNode = localNode; - persistedState = new InMemoryPersistedState(0L, - clusterState(0L, 0L, localNode, VotingConfiguration.EMPTY_CONFIG, VotingConfiguration.EMPTY_CONFIG, 0L)); - electionStrategy = ElectionStrategy.DefaultElectionStrategy.INSTANCE; - state = new CoordinationState(localNode, persistedState, electionStrategy); - } - - void reboot() { - state = new CoordinationState(localNode, persistedState, electionStrategy); - } - - void setInitialState(VotingConfiguration initialConfig, long initialValue) { - final ClusterState.Builder builder = ClusterState.builder(state.getLastAcceptedState()); - builder.metaData(MetaData.builder() - .coordinationMetaData(CoordinationMetaData.builder() - .lastAcceptedConfiguration(initialConfig) - .lastCommittedConfiguration(initialConfig) - .build())); - state.setInitialState(setValue(builder.build(), initialValue)); - } - } - - static class Cluster { - - final List messages; - final List clusterNodes; - final VotingConfiguration initialConfiguration; - final long initialValue; - - Cluster(int numNodes) { - messages = new ArrayList<>(); - - clusterNodes = IntStream.range(0, numNodes) - .mapToObj(i -> new DiscoveryNode("node_" + i, buildNewFakeTransportAddress(), Version.CURRENT)) - .map(ClusterNode::new) - .collect(Collectors.toList()); - - initialConfiguration = randomVotingConfig(); - initialValue = randomLong(); - } - - static class Message { - final DiscoveryNode sourceNode; - final DiscoveryNode targetNode; - final Object payload; - - Message(DiscoveryNode sourceNode, DiscoveryNode targetNode, Object payload) { - this.sourceNode = sourceNode; - this.targetNode = targetNode; - this.payload = payload; - } - } - - void reply(Message m, Object payload) { - messages.add(new Message(m.targetNode, m.sourceNode, payload)); - } - - void broadcast(DiscoveryNode sourceNode, Object payload) { - messages.addAll(clusterNodes.stream().map(cn -> new Message(sourceNode, cn.localNode, payload)).collect(Collectors.toList())); - } - - Optional getNode(DiscoveryNode node) { - return clusterNodes.stream().filter(cn -> cn.localNode.equals(node)).findFirst(); - } - - VotingConfiguration randomVotingConfig() { - return new VotingConfiguration( - randomSubsetOf(randomIntBetween(1, clusterNodes.size()), clusterNodes).stream() - .map(cn -> cn.localNode.getId()).collect(toSet())); - } - - void applyMessage(Message message) { - final Optional maybeNode = getNode(message.targetNode); - if (maybeNode.isPresent() == false) { - throw new CoordinationStateRejectedException("node not available"); - } else { - final Object payload = message.payload; - if (payload instanceof StartJoinRequest) { - reply(message, maybeNode.get().state.handleStartJoin((StartJoinRequest) payload)); - } else if (payload instanceof Join) { - maybeNode.get().state.handleJoin((Join) payload); - } else if (payload instanceof PublishRequest) { - reply(message, maybeNode.get().state.handlePublishRequest((PublishRequest) payload)); - } else if (payload instanceof PublishResponse) { - maybeNode.get().state.handlePublishResponse(message.sourceNode, (PublishResponse) payload) - .ifPresent(ac -> broadcast(message.targetNode, ac)); - } else if (payload instanceof ApplyCommitRequest) { - maybeNode.get().state.handleCommit((ApplyCommitRequest) payload); - } else { - throw new AssertionError("unknown message type"); - } - } - } - - void runRandomly() { - final int iterations = 10000; - final long maxTerm = 4; - long nextTerm = 1; - for (int i = 0; i < iterations; i++) { - try { - if (rarely() && nextTerm < maxTerm) { - final long term = rarely() ? randomLongBetween(0, maxTerm + 1) : nextTerm++; - final StartJoinRequest startJoinRequest = new StartJoinRequest(randomFrom(clusterNodes).localNode, term); - broadcast(startJoinRequest.getSourceNode(), startJoinRequest); - } else if (rarely()) { - randomFrom(clusterNodes).setInitialState(initialConfiguration, initialValue); - } else if (rarely() && rarely()) { - randomFrom(clusterNodes).reboot(); - } else if (rarely()) { - final List masterNodes = clusterNodes.stream().filter(cn -> cn.state.electionWon()) - .collect(Collectors.toList()); - if (masterNodes.isEmpty() == false) { - final ClusterNode clusterNode = randomFrom(masterNodes); - final long term = rarely() ? randomLongBetween(0, maxTerm + 1) : clusterNode.state.getCurrentTerm(); - final long version = rarely() ? randomIntBetween(0, 5) : clusterNode.state.getLastPublishedVersion() + 1; - final VotingConfiguration acceptedConfig = rarely() ? randomVotingConfig() : - clusterNode.state.getLastAcceptedConfiguration(); - final PublishRequest publishRequest = clusterNode.state.handleClientValue( - clusterState(term, version, clusterNode.localNode, clusterNode.state.getLastCommittedConfiguration(), - acceptedConfig, randomLong())); - broadcast(clusterNode.localNode, publishRequest); - } - } else if (messages.isEmpty() == false) { - applyMessage(randomFrom(messages)); - } - - // check node invariants after each iteration - clusterNodes.forEach(cn -> cn.state.invariant()); - } catch (CoordinationStateRejectedException e) { - // ignore - } - } - - // check system invariants. It's sufficient to do this at the end as these invariants are monotonic. - invariant(); - } - - void invariant() { - // one master per term - messages.stream().filter(m -> m.payload instanceof PublishRequest) - .collect(Collectors.groupingBy(m -> ((PublishRequest) m.payload).getAcceptedState().term())) - .forEach((term, publishMessages) -> { - Set mastersForTerm = publishMessages.stream().collect(Collectors.groupingBy(m -> m.sourceNode)).keySet(); - assertThat("Multiple masters " + mastersForTerm + " for term " + term, mastersForTerm, hasSize(1)); - }); - - // unique cluster state per (term, version) pair - messages.stream().filter(m -> m.payload instanceof PublishRequest) - .map(m -> ((PublishRequest) m.payload).getAcceptedState()) - .collect(Collectors.groupingBy(ClusterState::term)) - .forEach((term, clusterStates) -> { - clusterStates.stream().collect(Collectors.groupingBy(ClusterState::version)) - .forEach((version, clusterStates1) -> { - Set clusterStateUUIDsForTermAndVersion = clusterStates1.stream().collect(Collectors.groupingBy( - ClusterState::stateUUID - )).keySet(); - assertThat("Multiple cluster states " + clusterStates1 + " for term " + term + " and version " + version, - clusterStateUUIDsForTermAndVersion, hasSize(1)); - - Set clusterStateValuesForTermAndVersion = clusterStates1.stream().collect(Collectors.groupingBy( - CoordinationStateTests::value - )).keySet(); - - assertThat("Multiple cluster states " + clusterStates1 + " for term " + term + " and version " + version, - clusterStateValuesForTermAndVersion, hasSize(1)); - }); - }); - } - - } } diff --git a/test/framework/src/main/java/org/elasticsearch/cluster/coordination/CoordinationStateTestCluster.java b/test/framework/src/main/java/org/elasticsearch/cluster/coordination/CoordinationStateTestCluster.java new file mode 100644 index 0000000000000..582ecd1fcface --- /dev/null +++ b/test/framework/src/main/java/org/elasticsearch/cluster/coordination/CoordinationStateTestCluster.java @@ -0,0 +1,264 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.cluster.coordination; + +import org.elasticsearch.cluster.ClusterName; +import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.metadata.MetaData; +import org.elasticsearch.cluster.node.DiscoveryNode; +import org.elasticsearch.cluster.node.DiscoveryNodes; +import org.elasticsearch.common.UUIDs; +import org.elasticsearch.common.settings.Settings; + +import java.util.ArrayList; +import java.util.List; +import java.util.Optional; +import java.util.Set; +import java.util.stream.Collectors; + +import static com.carrotsearch.randomizedtesting.RandomizedTest.rarely; +import static java.util.stream.Collectors.toSet; +import static org.apache.lucene.util.LuceneTestCase.random; +import static org.elasticsearch.test.ESTestCase.randomFrom; +import static org.elasticsearch.test.ESTestCase.randomIntBetween; +import static org.elasticsearch.test.ESTestCase.randomLong; +import static org.elasticsearch.test.ESTestCase.randomLongBetween; +import static org.elasticsearch.test.ESTestCase.randomSubsetOf; +import static org.hamcrest.Matchers.hasSize; +import static org.junit.Assert.assertThat; + +public class CoordinationStateTestCluster { + + public static ClusterState clusterState(long term, long version, DiscoveryNode localNode, + CoordinationMetaData.VotingConfiguration lastCommittedConfig, + CoordinationMetaData.VotingConfiguration lastAcceptedConfig, long value) { + return clusterState(term, version, DiscoveryNodes.builder().add(localNode).localNodeId(localNode.getId()).build(), + lastCommittedConfig, lastAcceptedConfig, value); + } + + public static ClusterState clusterState(long term, long version, DiscoveryNodes discoveryNodes, + CoordinationMetaData.VotingConfiguration lastCommittedConfig, + CoordinationMetaData.VotingConfiguration lastAcceptedConfig, long value) { + return setValue(ClusterState.builder(ClusterName.DEFAULT) + .version(version) + .nodes(discoveryNodes) + .metaData(MetaData.builder() + .clusterUUID(UUIDs.randomBase64UUID(random())) // generate cluster UUID deterministically for repeatable tests + .coordinationMetaData(CoordinationMetaData.builder() + .term(term) + .lastCommittedConfiguration(lastCommittedConfig) + .lastAcceptedConfiguration(lastAcceptedConfig) + .build())) + .stateUUID(UUIDs.randomBase64UUID(random())) // generate cluster state UUID deterministically for repeatable tests + .build(), value); + } + + public static ClusterState setValue(ClusterState clusterState, long value) { + return ClusterState.builder(clusterState).metaData( + MetaData.builder(clusterState.metaData()) + .persistentSettings(Settings.builder() + .put(clusterState.metaData().persistentSettings()) + .put("value", value) + .build()) + .build()) + .build(); + } + + public static long value(ClusterState clusterState) { + return clusterState.metaData().persistentSettings().getAsLong("value", 0L); + } + + static class ClusterNode { + + final DiscoveryNode localNode; + final CoordinationState.PersistedState persistedState; + private final ElectionStrategy electionStrategy; + CoordinationState state; + + ClusterNode(DiscoveryNode localNode, ElectionStrategy electionStrategy) { + this.localNode = localNode; + persistedState = new InMemoryPersistedState(0L, + clusterState(0L, 0L, localNode, CoordinationMetaData.VotingConfiguration.EMPTY_CONFIG, + CoordinationMetaData.VotingConfiguration.EMPTY_CONFIG, 0L)); + this.electionStrategy = electionStrategy; + state = new CoordinationState(localNode, persistedState, electionStrategy); + } + + void reboot() { + state = new CoordinationState(localNode, persistedState, electionStrategy); + } + + void setInitialState(CoordinationMetaData.VotingConfiguration initialConfig, long initialValue) { + final ClusterState.Builder builder = ClusterState.builder(state.getLastAcceptedState()); + builder.metaData(MetaData.builder() + .coordinationMetaData(CoordinationMetaData.builder() + .lastAcceptedConfiguration(initialConfig) + .lastCommittedConfiguration(initialConfig) + .build())); + state.setInitialState(setValue(builder.build(), initialValue)); + } + } + + final ElectionStrategy electionStrategy; + final List messages; + final List clusterNodes; + final CoordinationMetaData.VotingConfiguration initialConfiguration; + final long initialValue; + + CoordinationStateTestCluster(List nodes, ElectionStrategy electionStrategy) { + this.electionStrategy = electionStrategy; + messages = new ArrayList<>(); + + clusterNodes = nodes.stream() + .map(node -> new ClusterNode(node, electionStrategy)) + .collect(Collectors.toList()); + + initialConfiguration = randomVotingConfig(); + initialValue = randomLong(); + } + + static class Message { + final DiscoveryNode sourceNode; + final DiscoveryNode targetNode; + final Object payload; + + Message(DiscoveryNode sourceNode, DiscoveryNode targetNode, Object payload) { + this.sourceNode = sourceNode; + this.targetNode = targetNode; + this.payload = payload; + } + } + + void reply(Message m, Object payload) { + messages.add(new Message(m.targetNode, m.sourceNode, payload)); + } + + void broadcast(DiscoveryNode sourceNode, Object payload) { + messages.addAll(clusterNodes.stream().map(cn -> new Message(sourceNode, cn.localNode, payload)).collect(Collectors.toList())); + } + + Optional getNode(DiscoveryNode node) { + return clusterNodes.stream().filter(cn -> cn.localNode.equals(node)).findFirst(); + } + + CoordinationMetaData.VotingConfiguration randomVotingConfig() { + return new CoordinationMetaData.VotingConfiguration( + randomSubsetOf(randomIntBetween(1, clusterNodes.size()), clusterNodes).stream() + .map(cn -> cn.localNode.getId()).collect(toSet())); + } + + void applyMessage(Message message) { + final Optional maybeNode = getNode(message.targetNode); + if (maybeNode.isPresent() == false) { + throw new CoordinationStateRejectedException("node not available"); + } else { + final Object payload = message.payload; + if (payload instanceof StartJoinRequest) { + reply(message, maybeNode.get().state.handleStartJoin((StartJoinRequest) payload)); + } else if (payload instanceof Join) { + maybeNode.get().state.handleJoin((Join) payload); + } else if (payload instanceof PublishRequest) { + reply(message, maybeNode.get().state.handlePublishRequest((PublishRequest) payload)); + } else if (payload instanceof PublishResponse) { + maybeNode.get().state.handlePublishResponse(message.sourceNode, (PublishResponse) payload) + .ifPresent(ac -> broadcast(message.targetNode, ac)); + } else if (payload instanceof ApplyCommitRequest) { + maybeNode.get().state.handleCommit((ApplyCommitRequest) payload); + } else { + throw new AssertionError("unknown message type"); + } + } + } + + void runRandomly() { + final int iterations = 10000; + final long maxTerm = 4; + long nextTerm = 1; + for (int i = 0; i < iterations; i++) { + try { + if (rarely() && nextTerm < maxTerm) { + final long term = rarely() ? randomLongBetween(0, maxTerm + 1) : nextTerm++; + final StartJoinRequest startJoinRequest = new StartJoinRequest(randomFrom(clusterNodes).localNode, term); + broadcast(startJoinRequest.getSourceNode(), startJoinRequest); + } else if (rarely()) { + randomFrom(clusterNodes).setInitialState(initialConfiguration, initialValue); + } else if (rarely() && rarely()) { + randomFrom(clusterNodes).reboot(); + } else if (rarely()) { + final List masterNodes = clusterNodes.stream().filter(cn -> cn.state.electionWon()) + .collect(Collectors.toList()); + if (masterNodes.isEmpty() == false) { + final ClusterNode clusterNode = randomFrom(masterNodes); + final long term = rarely() ? randomLongBetween(0, maxTerm + 1) : clusterNode.state.getCurrentTerm(); + final long version = rarely() ? randomIntBetween(0, 5) : clusterNode.state.getLastPublishedVersion() + 1; + final CoordinationMetaData.VotingConfiguration acceptedConfig = rarely() ? randomVotingConfig() : + clusterNode.state.getLastAcceptedConfiguration(); + final PublishRequest publishRequest = clusterNode.state.handleClientValue( + clusterState(term, version, clusterNode.localNode, clusterNode.state.getLastCommittedConfiguration(), + acceptedConfig, randomLong())); + broadcast(clusterNode.localNode, publishRequest); + } + } else if (messages.isEmpty() == false) { + applyMessage(randomFrom(messages)); + } + + // check node invariants after each iteration + clusterNodes.forEach(cn -> cn.state.invariant()); + } catch (CoordinationStateRejectedException e) { + // ignore + } + } + + // check system invariants. It's sufficient to do this at the end as these invariants are monotonic. + invariant(); + } + + void invariant() { + // one master per term + messages.stream().filter(m -> m.payload instanceof PublishRequest) + .collect(Collectors.groupingBy(m -> ((PublishRequest) m.payload).getAcceptedState().term())) + .forEach((term, publishMessages) -> { + Set mastersForTerm = publishMessages.stream().collect(Collectors.groupingBy(m -> m.sourceNode)).keySet(); + assertThat("Multiple masters " + mastersForTerm + " for term " + term, mastersForTerm, hasSize(1)); + }); + + // unique cluster state per (term, version) pair + messages.stream().filter(m -> m.payload instanceof PublishRequest) + .map(m -> ((PublishRequest) m.payload).getAcceptedState()) + .collect(Collectors.groupingBy(ClusterState::term)) + .forEach((term, clusterStates) -> { + clusterStates.stream().collect(Collectors.groupingBy(ClusterState::version)) + .forEach((version, clusterStates1) -> { + Set clusterStateUUIDsForTermAndVersion = clusterStates1.stream().collect(Collectors.groupingBy( + ClusterState::stateUUID + )).keySet(); + assertThat("Multiple cluster states " + clusterStates1 + " for term " + term + " and version " + version, + clusterStateUUIDsForTermAndVersion, hasSize(1)); + + Set clusterStateValuesForTermAndVersion = clusterStates1.stream().collect(Collectors.groupingBy( + CoordinationStateTestCluster::value + )).keySet(); + + assertThat("Multiple cluster states " + clusterStates1 + " for term " + term + " and version " + version, + clusterStateValuesForTermAndVersion, hasSize(1)); + }); + }); + } +} diff --git a/x-pack/plugin/voting-only-node/src/main/java/org/elasticsearch/cluster/coordination/VotingOnlyNodePlugin.java b/x-pack/plugin/voting-only-node/src/main/java/org/elasticsearch/cluster/coordination/VotingOnlyNodePlugin.java index a066dfeab41cd..f67324b332b19 100644 --- a/x-pack/plugin/voting-only-node/src/main/java/org/elasticsearch/cluster/coordination/VotingOnlyNodePlugin.java +++ b/x-pack/plugin/voting-only-node/src/main/java/org/elasticsearch/cluster/coordination/VotingOnlyNodePlugin.java @@ -53,7 +53,7 @@ public class VotingOnlyNodePlugin extends Plugin implements DiscoveryPlugin, Net private static final String VOTING_ONLY_ELECTION_TYPE = "supports-voting-only"; - private static DiscoveryNodeRole VOTING_ONLY_NODE_ROLE = new DiscoveryNodeRole("voting-only", "v") { + static DiscoveryNodeRole VOTING_ONLY_NODE_ROLE = new DiscoveryNodeRole("voting-only", "v") { @Override protected Setting roleSetting() { return VOTING_ONLY_NODE_SETTING; diff --git a/x-pack/plugin/voting-only-node/src/test/java/org/elasticsearch/cluster/coordination/VotingOnlyNodeCoordinationStateTests.java b/x-pack/plugin/voting-only-node/src/test/java/org/elasticsearch/cluster/coordination/VotingOnlyNodeCoordinationStateTests.java new file mode 100644 index 0000000000000..414ba350adf40 --- /dev/null +++ b/x-pack/plugin/voting-only-node/src/test/java/org/elasticsearch/cluster/coordination/VotingOnlyNodeCoordinationStateTests.java @@ -0,0 +1,30 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License; + * you may not use this file except in compliance with the Elastic License. + */ +package org.elasticsearch.cluster.coordination; + +import org.elasticsearch.Version; +import org.elasticsearch.cluster.node.DiscoveryNode; +import org.elasticsearch.cluster.node.DiscoveryNodeRole; +import org.elasticsearch.test.ESTestCase; + +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +public class VotingOnlyNodeCoordinationStateTests extends ESTestCase { + + public void testSafety() { + new CoordinationStateTestCluster(IntStream.range(0, randomIntBetween(1, 5)) + .mapToObj(i -> new DiscoveryNode("node_" + i, buildNewFakeTransportAddress(), Map.of(), + randomBoolean() ? DiscoveryNodeRole.BUILT_IN_ROLES : + Set.of(DiscoveryNodeRole.DATA_ROLE, DiscoveryNodeRole.INGEST_ROLE, DiscoveryNodeRole.MASTER_ROLE, + VotingOnlyNodePlugin.VOTING_ONLY_NODE_ROLE), Version.CURRENT)) + .collect(Collectors.toList()), new VotingOnlyNodePlugin.VotingOnlyNodeElectionStrategy()) + .runRandomly(); + } + +} From 9ee9fdc96897e00d354b2fe3f9760d0265cd97bc Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Wed, 19 Jun 2019 20:04:53 +0200 Subject: [PATCH 09/29] move tests --- .../coordination/CoordinatorTests.java | 1296 +--------------- .../AbstractCoordinatorTestCase.java | 1351 +++++++++++++++++ .../coordination/LinearizabilityChecker.java | 0 .../gateway/MockGatewayMetaState.java | 2 - .../cluster/FakeThreadPoolMasterService.java | 0 .../VotingOnlyNodeCoordinatorTests.java | 43 + 6 files changed, 1398 insertions(+), 1294 deletions(-) create mode 100644 test/framework/src/main/java/org/elasticsearch/cluster/coordination/AbstractCoordinatorTestCase.java rename {server/src/test => test/framework/src/main}/java/org/elasticsearch/cluster/coordination/LinearizabilityChecker.java (100%) rename {server/src/test => test/framework/src/main}/java/org/elasticsearch/gateway/MockGatewayMetaState.java (95%) rename {server/src/test => test/framework/src/main}/java/org/elasticsearch/indices/cluster/FakeThreadPoolMasterService.java (100%) create mode 100644 x-pack/plugin/voting-only-node/src/test/java/org/elasticsearch/cluster/coordination/VotingOnlyNodeCoordinatorTests.java diff --git a/server/src/test/java/org/elasticsearch/cluster/coordination/CoordinatorTests.java b/server/src/test/java/org/elasticsearch/cluster/coordination/CoordinatorTests.java index cddb70a23f114..261abfbfac7bc 100644 --- a/server/src/test/java/org/elasticsearch/cluster/coordination/CoordinatorTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/coordination/CoordinatorTests.java @@ -18,109 +18,45 @@ */ package org.elasticsearch.cluster.coordination; -import com.carrotsearch.randomizedtesting.RandomizedContext; -import org.apache.logging.log4j.CloseableThreadContext; import org.apache.logging.log4j.Level; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.core.LogEvent; -import org.apache.logging.log4j.message.ParameterizedMessage; import org.elasticsearch.ElasticsearchException; import org.elasticsearch.Version; import org.elasticsearch.cluster.AbstractDiffable; -import org.elasticsearch.cluster.ClusterModule; import org.elasticsearch.cluster.ClusterState; -import org.elasticsearch.cluster.ClusterStateTaskListener; -import org.elasticsearch.cluster.ClusterStateUpdateTask; -import org.elasticsearch.cluster.ESAllocationTestCase; -import org.elasticsearch.cluster.NodeConnectionsService; import org.elasticsearch.cluster.block.ClusterBlock; -import org.elasticsearch.cluster.coordination.ClusterStatePublisher.AckListener; +import org.elasticsearch.cluster.coordination.AbstractCoordinatorTestCase.Cluster.ClusterNode; import org.elasticsearch.cluster.coordination.CoordinationMetaData.VotingConfiguration; -import org.elasticsearch.cluster.coordination.CoordinationState.PersistedState; import org.elasticsearch.cluster.coordination.Coordinator.Mode; -import org.elasticsearch.cluster.coordination.CoordinatorTests.Cluster.ClusterNode; -import org.elasticsearch.cluster.coordination.LinearizabilityChecker.History; -import org.elasticsearch.cluster.coordination.LinearizabilityChecker.SequentialSpec; -import org.elasticsearch.cluster.metadata.Manifest; import org.elasticsearch.cluster.metadata.MetaData; import org.elasticsearch.cluster.node.DiscoveryNode; -import org.elasticsearch.cluster.node.DiscoveryNodeRole; -import org.elasticsearch.cluster.routing.allocation.AllocationService; -import org.elasticsearch.cluster.service.ClusterApplierService; -import org.elasticsearch.cluster.service.ClusterService; -import org.elasticsearch.common.Nullable; -import org.elasticsearch.common.Randomness; -import org.elasticsearch.common.UUIDs; -import org.elasticsearch.common.collect.Tuple; -import org.elasticsearch.common.io.stream.BytesStreamOutput; -import org.elasticsearch.common.io.stream.NamedWriteableAwareStreamInput; -import org.elasticsearch.common.io.stream.NamedWriteableRegistry; -import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.logging.Loggers; import org.elasticsearch.common.regex.Regex; -import org.elasticsearch.common.settings.ClusterSettings; -import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings.Builder; -import org.elasticsearch.common.transport.TransportAddress; -import org.elasticsearch.common.unit.TimeValue; -import org.elasticsearch.common.util.concurrent.PrioritizedEsThreadPoolExecutor; import org.elasticsearch.common.util.set.Sets; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.discovery.DiscoveryModule; -import org.elasticsearch.discovery.SeedHostsProvider.HostsResolver; -import org.elasticsearch.env.NodeEnvironment; -import org.elasticsearch.gateway.ClusterStateUpdaters; import org.elasticsearch.gateway.GatewayService; -import org.elasticsearch.gateway.MetaStateService; -import org.elasticsearch.gateway.MockGatewayMetaState; -import org.elasticsearch.indices.cluster.FakeThreadPoolMasterService; -import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.MockLogAppender; -import org.elasticsearch.test.disruption.DisruptableMockTransport; -import org.elasticsearch.test.disruption.DisruptableMockTransport.ConnectionStatus; -import org.elasticsearch.transport.TransportService; -import org.hamcrest.Matcher; -import org.hamcrest.core.IsCollectionContaining; -import org.junit.After; -import org.junit.Before; import java.io.IOException; -import java.io.UncheckedIOException; -import java.util.ArrayList; import java.util.Arrays; -import java.util.Collection; import java.util.Collections; -import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; -import java.util.Optional; import java.util.Set; -import java.util.concurrent.Callable; import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.function.BiConsumer; -import java.util.function.Consumer; import java.util.function.Function; -import java.util.function.Supplier; -import java.util.function.UnaryOperator; import java.util.stream.Collectors; -import java.util.stream.Stream; -import static java.util.Collections.emptyList; -import static java.util.Collections.emptySet; -import static org.elasticsearch.cluster.coordination.ClusterBootstrapService.BOOTSTRAP_PLACEHOLDER_PREFIX; -import static org.elasticsearch.cluster.coordination.CoordinationStateTests.clusterState; +import static org.elasticsearch.cluster.coordination.AbstractCoordinatorTestCase.Cluster.DEFAULT_DELAY_VARIABILITY; import static org.elasticsearch.cluster.coordination.Coordinator.Mode.CANDIDATE; -import static org.elasticsearch.cluster.coordination.Coordinator.Mode.FOLLOWER; -import static org.elasticsearch.cluster.coordination.Coordinator.Mode.LEADER; import static org.elasticsearch.cluster.coordination.Coordinator.PUBLISH_TIMEOUT_SETTING; -import static org.elasticsearch.cluster.coordination.CoordinatorTests.Cluster.DEFAULT_DELAY_VARIABILITY; -import static org.elasticsearch.cluster.coordination.ElectionSchedulerFactory.ELECTION_BACK_OFF_TIME_SETTING; -import static org.elasticsearch.cluster.coordination.ElectionSchedulerFactory.ELECTION_DURATION_SETTING; import static org.elasticsearch.cluster.coordination.ElectionSchedulerFactory.ELECTION_INITIAL_TIMEOUT_SETTING; import static org.elasticsearch.cluster.coordination.FollowersChecker.FOLLOWER_CHECK_INTERVAL_SETTING; import static org.elasticsearch.cluster.coordination.FollowersChecker.FOLLOWER_CHECK_RETRY_COUNT_SETTING; @@ -129,71 +65,20 @@ 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.NoMasterBlockService.NO_MASTER_BLOCK_ALL; -import static org.elasticsearch.cluster.coordination.NoMasterBlockService.NO_MASTER_BLOCK_ID; import static org.elasticsearch.cluster.coordination.NoMasterBlockService.NO_MASTER_BLOCK_SETTING; import static org.elasticsearch.cluster.coordination.NoMasterBlockService.NO_MASTER_BLOCK_WRITES; 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.gateway.GatewayService.STATE_NOT_RECOVERED_BLOCK; -import static org.elasticsearch.node.Node.NODE_NAME_SETTING; -import static org.elasticsearch.transport.TransportService.NOOP_TRANSPORT_INTERCEPTOR; import static org.hamcrest.Matchers.containsString; -import static org.hamcrest.Matchers.empty; import static org.hamcrest.Matchers.equalTo; -import static org.hamcrest.Matchers.greaterThan; -import static org.hamcrest.Matchers.greaterThanOrEqualTo; import static org.hamcrest.Matchers.hasItem; import static org.hamcrest.Matchers.hasSize; import static org.hamcrest.Matchers.instanceOf; import static org.hamcrest.Matchers.is; -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; -public class CoordinatorTests extends ESTestCase { - - private final List nodeEnvironments = new ArrayList<>(); - - private final AtomicInteger nextNodeIndex = new AtomicInteger(); - - @Before - public void resetNodeIndexBeforeEachTest() { - nextNodeIndex.set(0); - } - - @After - public void closeNodeEnvironmentsAfterEachTest() { - for (NodeEnvironment nodeEnvironment : nodeEnvironments) { - nodeEnvironment.close(); - } - nodeEnvironments.clear(); - } - - @Before - public void resetPortCounterBeforeEachTest() { - resetPortCounter(); - } - - // check that runRandomly leads to reproducible results - public void testRepeatableTests() throws Exception { - final Callable test = () -> { - resetNodeIndexBeforeEachTest(); - final Cluster cluster = new Cluster(randomIntBetween(1, 5)); - cluster.runRandomly(); - final long afterRunRandomly = value(cluster.getAnyNode().getLastAppliedClusterState()); - cluster.stabilise(); - final long afterStabilisation = value(cluster.getAnyNode().getLastAppliedClusterState()); - return afterRunRandomly ^ afterStabilisation; - }; - final long seed = randomLong(); - logger.info("First run with seed [{}]", seed); - final long result1 = RandomizedContext.current().runWithPrivateRandomness(seed, test); - logger.info("Second run with seed [{}]", seed); - final long result2 = RandomizedContext.current().runWithPrivateRandomness(seed, test); - assertEquals(result1, result2); - } +public class CoordinatorTests extends AbstractCoordinatorTestCase { /** * This test was added to verify that state recovery is properly reset on a node after it has become master and successfully @@ -259,7 +144,7 @@ public void testDoesNotElectNonMasterNode() { cluster.stabilise(); final ClusterNode leader = cluster.getAnyLeader(); - assertTrue(leader.localNode.isMasterNode()); + assertTrue(leader.getLocalNode().isMasterNode()); } public void testNodesJoinAfterStableCluster() { @@ -1332,1179 +1217,6 @@ public void assertMatched() { } } - private static long defaultMillis(Setting setting) { - return setting.get(Settings.EMPTY).millis() + Cluster.DEFAULT_DELAY_VARIABILITY; - } - - private static int defaultInt(Setting setting) { - return setting.get(Settings.EMPTY); - } - - // Updating the cluster state involves up to 7 delays: - // 1. submit the task to the master service - // 2. send PublishRequest - // 3. receive PublishResponse - // 4. send ApplyCommitRequest - // 5. apply committed cluster state - // 6. receive ApplyCommitResponse - // 7. apply committed state on master (last one to apply cluster state) - private static final long DEFAULT_CLUSTER_STATE_UPDATE_DELAY = 7 * DEFAULT_DELAY_VARIABILITY; - - private static final int ELECTION_RETRIES = 10; - - // The time it takes to complete an election - private static final long DEFAULT_ELECTION_DELAY - // Pinging all peers twice should be enough to discover all nodes - = defaultMillis(DISCOVERY_FIND_PEERS_INTERVAL_SETTING) * 2 - // Then wait for an election to be scheduled; we allow enough time for retries to allow for collisions - + defaultMillis(ELECTION_INITIAL_TIMEOUT_SETTING) * ELECTION_RETRIES - + defaultMillis(ELECTION_BACK_OFF_TIME_SETTING) * ELECTION_RETRIES * (ELECTION_RETRIES - 1) / 2 - + defaultMillis(ELECTION_DURATION_SETTING) * ELECTION_RETRIES - // Allow two round-trip for pre-voting and voting - + 4 * DEFAULT_DELAY_VARIABILITY - // Then a commit of the new leader's first cluster state - + DEFAULT_CLUSTER_STATE_UPDATE_DELAY; - - private static final long DEFAULT_STABILISATION_TIME = - // If leader just blackholed, need to wait for this to be detected - (defaultMillis(LEADER_CHECK_INTERVAL_SETTING) + defaultMillis(LEADER_CHECK_TIMEOUT_SETTING)) - * defaultInt(LEADER_CHECK_RETRY_COUNT_SETTING) - // then wait for a follower to be promoted to leader - + DEFAULT_ELECTION_DELAY - // perhaps there is an election collision requiring another publication (which times out) and a term bump - + defaultMillis(PUBLISH_TIMEOUT_SETTING) + DEFAULT_ELECTION_DELAY - // then wait for the new leader to notice that the old leader is unresponsive - + (defaultMillis(FOLLOWER_CHECK_INTERVAL_SETTING) + defaultMillis(FOLLOWER_CHECK_TIMEOUT_SETTING)) - * defaultInt(FOLLOWER_CHECK_RETRY_COUNT_SETTING) - // then wait for the new leader to commit a state without the old leader - + DEFAULT_CLUSTER_STATE_UPDATE_DELAY; - - class Cluster { - - static final long EXTREME_DELAY_VARIABILITY = 10000L; - static final long DEFAULT_DELAY_VARIABILITY = 100L; - - final List clusterNodes; - final DeterministicTaskQueue deterministicTaskQueue = new DeterministicTaskQueue( - // TODO does ThreadPool need a node name any more? - Settings.builder().put(NODE_NAME_SETTING.getKey(), "deterministic-task-queue").build(), random()); - private boolean disruptStorage; - - private final VotingConfiguration initialConfiguration; - - private final Set disconnectedNodes = new HashSet<>(); - private final Set blackholedNodes = new HashSet<>(); - private final Set> blackholedConnections = new HashSet<>(); - private final Map committedStatesByVersion = new HashMap<>(); - private final LinearizabilityChecker linearizabilityChecker = new LinearizabilityChecker(); - private final History history = new History(); - - private final Function defaultPersistedStateSupplier = MockPersistedState::new; - - @Nullable // null means construct a list from all the current nodes - private List seedHostsList; - - Cluster(int initialNodeCount) { - this(initialNodeCount, true, Settings.EMPTY); - } - - Cluster(int initialNodeCount, boolean allNodesMasterEligible, Settings nodeSettings) { - deterministicTaskQueue.setExecutionDelayVariabilityMillis(DEFAULT_DELAY_VARIABILITY); - - assertThat(initialNodeCount, greaterThan(0)); - - final Set masterEligibleNodeIds = new HashSet<>(initialNodeCount); - clusterNodes = new ArrayList<>(initialNodeCount); - for (int i = 0; i < initialNodeCount; i++) { - final ClusterNode clusterNode = new ClusterNode(nextNodeIndex.getAndIncrement(), - allNodesMasterEligible || i == 0 || randomBoolean(), nodeSettings); - clusterNodes.add(clusterNode); - if (clusterNode.getLocalNode().isMasterNode()) { - masterEligibleNodeIds.add(clusterNode.getId()); - } - } - - initialConfiguration = new VotingConfiguration(new HashSet<>( - randomSubsetOf(randomIntBetween(1, masterEligibleNodeIds.size()), masterEligibleNodeIds))); - - logger.info("--> creating cluster of {} nodes (master-eligible nodes: {}) with initial configuration {}", - initialNodeCount, masterEligibleNodeIds, initialConfiguration); - } - - List addNodesAndStabilise(int newNodesCount) { - final List addedNodes = addNodes(newNodesCount); - stabilise( - // The first pinging discovers the master - defaultMillis(DISCOVERY_FIND_PEERS_INTERVAL_SETTING) - // One message delay to send a join - + DEFAULT_DELAY_VARIABILITY - // Commit a new cluster state with the new node(s). Might be split into multiple commits, and each might need a - // followup reconfiguration - + newNodesCount * 2 * DEFAULT_CLUSTER_STATE_UPDATE_DELAY); - // TODO Investigate whether 4 publications is sufficient due to batching? A bound linear in the number of nodes isn't great. - return addedNodes; - } - - List addNodes(int newNodesCount) { - logger.info("--> adding {} nodes", newNodesCount); - - final List addedNodes = new ArrayList<>(); - for (int i = 0; i < newNodesCount; i++) { - final ClusterNode clusterNode = new ClusterNode(nextNodeIndex.getAndIncrement(), true, Settings.EMPTY); - addedNodes.add(clusterNode); - } - clusterNodes.addAll(addedNodes); - return addedNodes; - } - - int size() { - return clusterNodes.size(); - } - - void runRandomly() { - runRandomly(true); - } - - void runRandomly(boolean allowReboots) { - - // TODO supporting (preserving?) existing disruptions needs implementing if needed, for now we just forbid it - assertThat("may reconnect disconnected nodes, probably unexpected", disconnectedNodes, empty()); - assertThat("may reconnect blackholed nodes, probably unexpected", blackholedNodes, empty()); - - final List cleanupActions = new ArrayList<>(); - cleanupActions.add(disconnectedNodes::clear); - cleanupActions.add(blackholedNodes::clear); - cleanupActions.add(() -> disruptStorage = false); - - final int randomSteps = scaledRandomIntBetween(10, 10000); - final int keyRange = randomSteps / 50; // for randomized writes and reads - logger.info("--> start of safety phase of at least [{}] steps", randomSteps); - - deterministicTaskQueue.setExecutionDelayVariabilityMillis(EXTREME_DELAY_VARIABILITY); - disruptStorage = true; - int step = 0; - long finishTime = -1; - - while (finishTime == -1 || deterministicTaskQueue.getCurrentTimeMillis() <= finishTime) { - step++; - final int thisStep = step; // for lambdas - - if (randomSteps <= step && finishTime == -1) { - finishTime = deterministicTaskQueue.getLatestDeferredExecutionTime(); - deterministicTaskQueue.setExecutionDelayVariabilityMillis(DEFAULT_DELAY_VARIABILITY); - logger.debug("----> [runRandomly {}] reducing delay variability and running until [{}ms]", step, finishTime); - } - - try { - if (finishTime == -1 && randomBoolean() && randomBoolean() && randomBoolean()) { - final ClusterNode clusterNode = getAnyNodePreferringLeaders(); - final int key = randomIntBetween(0, keyRange); - final int newValue = randomInt(); - clusterNode.onNode(() -> { - logger.debug("----> [runRandomly {}] proposing new value [{}] to [{}]", - thisStep, newValue, clusterNode.getId()); - clusterNode.submitValue(key, newValue); - }).run(); - } else if (finishTime == -1 && randomBoolean() && randomBoolean() && randomBoolean()) { - final ClusterNode clusterNode = getAnyNodePreferringLeaders(); - final int key = randomIntBetween(0, keyRange); - clusterNode.onNode(() -> { - logger.debug("----> [runRandomly {}] reading value from [{}]", - thisStep, clusterNode.getId()); - clusterNode.readValue(key); - }).run(); - } else if (rarely()) { - final ClusterNode clusterNode = getAnyNodePreferringLeaders(); - final boolean autoShrinkVotingConfiguration = randomBoolean(); - clusterNode.onNode( - () -> { - logger.debug("----> [runRandomly {}] setting auto-shrink configuration to {} on {}", - thisStep, autoShrinkVotingConfiguration, clusterNode.getId()); - clusterNode.submitSetAutoShrinkVotingConfiguration(autoShrinkVotingConfiguration); - }).run(); - } else if (allowReboots && rarely()) { - // reboot random node - final ClusterNode clusterNode = getAnyNode(); - logger.debug("----> [runRandomly {}] rebooting [{}]", thisStep, clusterNode.getId()); - clusterNode.close(); - clusterNodes.forEach( - cn -> deterministicTaskQueue.scheduleNow(cn.onNode( - new Runnable() { - @Override - public void run() { - cn.transportService.disconnectFromNode(clusterNode.getLocalNode()); - } - - @Override - public String toString() { - return "disconnect from " + clusterNode.getLocalNode() + " after shutdown"; - } - }))); - clusterNodes.replaceAll(cn -> cn == clusterNode ? cn.restartedNode() : cn); - } else if (rarely()) { - final ClusterNode clusterNode = getAnyNode(); - clusterNode.onNode(() -> { - logger.debug("----> [runRandomly {}] forcing {} to become candidate", thisStep, clusterNode.getId()); - synchronized (clusterNode.coordinator.mutex) { - clusterNode.coordinator.becomeCandidate("runRandomly"); - } - }).run(); - } else if (rarely()) { - final ClusterNode clusterNode = getAnyNode(); - - switch (randomInt(2)) { - case 0: - if (clusterNode.heal()) { - logger.debug("----> [runRandomly {}] healing {}", step, clusterNode.getId()); - } - break; - case 1: - if (clusterNode.disconnect()) { - logger.debug("----> [runRandomly {}] disconnecting {}", step, clusterNode.getId()); - } - break; - case 2: - if (clusterNode.blackhole()) { - logger.debug("----> [runRandomly {}] blackholing {}", step, clusterNode.getId()); - } - break; - } - } else if (rarely()) { - final ClusterNode clusterNode = getAnyNode(); - logger.debug("----> [runRandomly {}] applying initial configuration on {}", step, clusterNode.getId()); - clusterNode.applyInitialConfiguration(); - } else { - if (deterministicTaskQueue.hasDeferredTasks() && randomBoolean()) { - deterministicTaskQueue.advanceTime(); - } else if (deterministicTaskQueue.hasRunnableTasks()) { - deterministicTaskQueue.runRandomTask(); - } - } - - // TODO other random steps: - // - reboot a node - // - abdicate leadership - - } catch (CoordinationStateRejectedException | UncheckedIOException ignored) { - // This is ok: it just means a message couldn't currently be handled. - } - - assertConsistentStates(); - } - - logger.debug("running {} cleanup actions", cleanupActions.size()); - cleanupActions.forEach(Runnable::run); - logger.debug("finished running cleanup actions"); - } - - private void assertConsistentStates() { - for (final ClusterNode clusterNode : clusterNodes) { - clusterNode.coordinator.invariant(); - } - updateCommittedStates(); - } - - private void updateCommittedStates() { - for (final ClusterNode clusterNode : clusterNodes) { - ClusterState applierState = clusterNode.coordinator.getApplierState(); - ClusterState storedState = committedStatesByVersion.get(applierState.getVersion()); - if (storedState == null) { - committedStatesByVersion.put(applierState.getVersion(), applierState); - } else { - assertEquals("expected " + applierState + " but got " + storedState, - value(applierState), value(storedState)); - } - } - } - - void stabilise() { - stabilise(DEFAULT_STABILISATION_TIME); - } - - void stabilise(long stabilisationDurationMillis) { - assertThat("stabilisation requires default delay variability (and proper cleanup of raised variability)", - deterministicTaskQueue.getExecutionDelayVariabilityMillis(), lessThanOrEqualTo(DEFAULT_DELAY_VARIABILITY)); - assertFalse("stabilisation requires stable storage", disruptStorage); - - bootstrapIfNecessary(); - - runFor(stabilisationDurationMillis, "stabilising"); - - final ClusterNode leader = getAnyLeader(); - final long leaderTerm = leader.coordinator.getCurrentTerm(); - - final int pendingTaskCount = leader.masterService.getFakeMasterServicePendingTaskCount(); - runFor((pendingTaskCount + 1) * DEFAULT_CLUSTER_STATE_UPDATE_DELAY, "draining task queue"); - - final Matcher isEqualToLeaderVersion = equalTo(leader.coordinator.getLastAcceptedState().getVersion()); - final String leaderId = leader.getId(); - - assertTrue(leaderId + " has been bootstrapped", leader.coordinator.isInitialConfigurationSet()); - assertTrue(leaderId + " exists in its last-applied state", leader.getLastAppliedClusterState().getNodes().nodeExists(leaderId)); - assertThat(leaderId + " has no NO_MASTER_BLOCK", - leader.getLastAppliedClusterState().blocks().hasGlobalBlockWithId(NO_MASTER_BLOCK_ID), equalTo(false)); - assertThat(leaderId + " has no STATE_NOT_RECOVERED_BLOCK", - leader.getLastAppliedClusterState().blocks().hasGlobalBlock(STATE_NOT_RECOVERED_BLOCK), equalTo(false)); - assertThat(leaderId + " has applied its state ", leader.getLastAppliedClusterState().getVersion(), isEqualToLeaderVersion); - - for (final ClusterNode clusterNode : clusterNodes) { - final String nodeId = clusterNode.getId(); - assertFalse(nodeId + " should not have an active publication", clusterNode.coordinator.publicationInProgress()); - - if (clusterNode == leader) { - assertThat(nodeId + " is still the leader", clusterNode.coordinator.getMode(), is(LEADER)); - assertThat(nodeId + " did not change term", clusterNode.coordinator.getCurrentTerm(), is(leaderTerm)); - continue; - } - - if (isConnectedPair(leader, clusterNode)) { - assertThat(nodeId + " is a follower of " + leaderId, clusterNode.coordinator.getMode(), is(FOLLOWER)); - assertThat(nodeId + " has the same term as " + leaderId, clusterNode.coordinator.getCurrentTerm(), is(leaderTerm)); - assertTrue(nodeId + " has voted for " + leaderId, leader.coordinator.hasJoinVoteFrom(clusterNode.getLocalNode())); - assertThat(nodeId + " has the same accepted state as " + leaderId, - clusterNode.coordinator.getLastAcceptedState().getVersion(), isEqualToLeaderVersion); - if (clusterNode.getClusterStateApplyResponse() == ClusterStateApplyResponse.SUCCEED) { - assertThat(nodeId + " has the same applied state as " + leaderId, - clusterNode.getLastAppliedClusterState().getVersion(), isEqualToLeaderVersion); - assertTrue(nodeId + " is in its own latest applied state", - clusterNode.getLastAppliedClusterState().getNodes().nodeExists(nodeId)); - } - 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().hasGlobalBlockWithId(NO_MASTER_BLOCK_ID), equalTo(false)); - assertThat(nodeId + " has no STATE_NOT_RECOVERED_BLOCK", - clusterNode.getLastAppliedClusterState().blocks().hasGlobalBlock(STATE_NOT_RECOVERED_BLOCK), 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().hasGlobalBlockWithId(NO_MASTER_BLOCK_ID), equalTo(true)); - assertFalse(nodeId + " is not in the applied state on " + leaderId, - leader.getLastAppliedClusterState().getNodes().nodeExists(nodeId)); - } - } - - final Set connectedNodeIds - = clusterNodes.stream().filter(n -> isConnectedPair(leader, n)).map(ClusterNode::getId).collect(Collectors.toSet()); - - assertThat(leader.getLastAppliedClusterState().getNodes().getSize(), equalTo(connectedNodeIds.size())); - - final ClusterState lastAcceptedState = leader.coordinator.getLastAcceptedState(); - final VotingConfiguration lastCommittedConfiguration = lastAcceptedState.getLastCommittedConfiguration(); - assertTrue(connectedNodeIds + " should be a quorum of " + lastCommittedConfiguration, - lastCommittedConfiguration.hasQuorum(connectedNodeIds)); - assertThat("leader " + leader.getLocalNode() + " should be part of voting configuration " + lastCommittedConfiguration, - lastCommittedConfiguration.getNodeIds(), IsCollectionContaining.hasItem(leader.getLocalNode().getId())); - - assertThat("no reconfiguration is in progress", - lastAcceptedState.getLastCommittedConfiguration(), equalTo(lastAcceptedState.getLastAcceptedConfiguration())); - assertThat("current configuration is already optimal", - leader.improveConfiguration(lastAcceptedState), sameInstance(lastAcceptedState)); - - logger.info("checking linearizability of history with size {}: {}", history.size(), history); - assertTrue("history not linearizable: " + history, linearizabilityChecker.isLinearizable(spec, history, i -> null)); - logger.info("linearizability check completed"); - } - - void bootstrapIfNecessary() { - if (clusterNodes.stream().allMatch(ClusterNode::isNotUsefullyBootstrapped)) { - assertThat("setting initial configuration may fail with disconnected nodes", disconnectedNodes, empty()); - assertThat("setting initial configuration may fail with blackholed nodes", blackholedNodes, empty()); - runFor(defaultMillis(DISCOVERY_FIND_PEERS_INTERVAL_SETTING) * 2, "discovery prior to setting initial configuration"); - final ClusterNode bootstrapNode = getAnyBootstrappableNode(); - bootstrapNode.applyInitialConfiguration(); - } else { - logger.info("setting initial configuration not required"); - } - } - - void runFor(long runDurationMillis, String description) { - final long endTime = deterministicTaskQueue.getCurrentTimeMillis() + runDurationMillis; - logger.info("--> runFor({}ms) running until [{}ms]: {}", runDurationMillis, endTime, description); - - while (deterministicTaskQueue.getCurrentTimeMillis() < endTime) { - - while (deterministicTaskQueue.hasRunnableTasks()) { - try { - deterministicTaskQueue.runRandomTask(); - } catch (CoordinationStateRejectedException e) { - logger.debug("ignoring benign exception thrown when stabilising", e); - } - for (final ClusterNode clusterNode : clusterNodes) { - clusterNode.coordinator.invariant(); - } - updateCommittedStates(); - } - - if (deterministicTaskQueue.hasDeferredTasks() == false) { - // A 1-node cluster has no need for fault detection etc so will eventually run out of things to do. - assert clusterNodes.size() == 1 : clusterNodes.size(); - break; - } - - deterministicTaskQueue.advanceTime(); - } - - logger.info("--> runFor({}ms) completed run until [{}ms]: {}", runDurationMillis, endTime, description); - } - - private boolean isConnectedPair(ClusterNode n1, ClusterNode n2) { - return n1 == n2 || - (getConnectionStatus(n1.getLocalNode(), n2.getLocalNode()) == ConnectionStatus.CONNECTED - && getConnectionStatus(n2.getLocalNode(), n1.getLocalNode()) == ConnectionStatus.CONNECTED); - } - - ClusterNode getAnyLeader() { - List allLeaders = clusterNodes.stream().filter(ClusterNode::isLeader).collect(Collectors.toList()); - assertThat("leaders", allLeaders, not(empty())); - return randomFrom(allLeaders); - } - - private final ConnectionStatus preferredUnknownNodeConnectionStatus = - randomFrom(ConnectionStatus.DISCONNECTED, ConnectionStatus.BLACK_HOLE); - - private ConnectionStatus getConnectionStatus(DiscoveryNode sender, DiscoveryNode destination) { - ConnectionStatus connectionStatus; - if (blackholedNodes.contains(sender.getId()) || blackholedNodes.contains(destination.getId())) { - connectionStatus = ConnectionStatus.BLACK_HOLE; - } else if (disconnectedNodes.contains(sender.getId()) || disconnectedNodes.contains(destination.getId())) { - connectionStatus = ConnectionStatus.DISCONNECTED; - } else if (blackholedConnections.contains(Tuple.tuple(sender.getId(), destination.getId()))) { - connectionStatus = ConnectionStatus.BLACK_HOLE_REQUESTS_ONLY; - } else if (nodeExists(sender) && nodeExists(destination)) { - connectionStatus = ConnectionStatus.CONNECTED; - } else { - connectionStatus = usually() ? preferredUnknownNodeConnectionStatus : - randomFrom(ConnectionStatus.DISCONNECTED, ConnectionStatus.BLACK_HOLE); - } - return connectionStatus; - } - - boolean nodeExists(DiscoveryNode node) { - return clusterNodes.stream().anyMatch(cn -> cn.getLocalNode().equals(node)); - } - - ClusterNode getAnyBootstrappableNode() { - return randomFrom(clusterNodes.stream().filter(n -> n.getLocalNode().isMasterNode()) - .filter(n -> initialConfiguration.getNodeIds().contains(n.getLocalNode().getId())) - .collect(Collectors.toList())); - } - - ClusterNode getAnyNode() { - return getAnyNodeExcept(); - } - - ClusterNode getAnyNodeExcept(ClusterNode... clusterNodes) { - List filteredNodes = getAllNodesExcept(clusterNodes); - assert filteredNodes.isEmpty() == false; - return randomFrom(filteredNodes); - } - - List getAllNodesExcept(ClusterNode... clusterNodes) { - Set forbiddenIds = Arrays.stream(clusterNodes).map(ClusterNode::getId).collect(Collectors.toSet()); - List acceptableNodes - = this.clusterNodes.stream().filter(n -> forbiddenIds.contains(n.getId()) == false).collect(Collectors.toList()); - return acceptableNodes; - } - - ClusterNode getAnyNodePreferringLeaders() { - for (int i = 0; i < 3; i++) { - ClusterNode clusterNode = getAnyNode(); - if (clusterNode.coordinator.getMode() == LEADER) { - return clusterNode; - } - } - return getAnyNode(); - } - - void setEmptySeedHostsList() { - seedHostsList = emptyList(); - } - - void blackholeConnectionsFrom(ClusterNode sender, ClusterNode destination) { - blackholedConnections.add(Tuple.tuple(sender.getId(), destination.getId())); - } - - void clearBlackholedConnections() { - blackholedConnections.clear(); - } - - class MockPersistedState implements PersistedState { - private final PersistedState delegate; - private final NodeEnvironment nodeEnvironment; - - MockPersistedState(DiscoveryNode localNode) { - try { - if (rarely()) { - nodeEnvironment = newNodeEnvironment(); - nodeEnvironments.add(nodeEnvironment); - delegate = new MockGatewayMetaState(Settings.EMPTY, nodeEnvironment, xContentRegistry(), localNode) - .getPersistedState(Settings.EMPTY, null); - } else { - nodeEnvironment = null; - delegate = new InMemoryPersistedState(0L, - ClusterStateUpdaters.addStateNotRecoveredBlock( - clusterState(0L, 0L, localNode, VotingConfiguration.EMPTY_CONFIG, VotingConfiguration.EMPTY_CONFIG, 0L))); - } - } catch (IOException e) { - throw new UncheckedIOException("Unable to create MockPersistedState", e); - } - } - - MockPersistedState(DiscoveryNode newLocalNode, MockPersistedState oldState, - Function adaptGlobalMetaData, Function adaptCurrentTerm) { - try { - if (oldState.nodeEnvironment != null) { - nodeEnvironment = oldState.nodeEnvironment; - final MetaStateService metaStateService = new MetaStateService(nodeEnvironment, xContentRegistry()); - final MetaData updatedMetaData = adaptGlobalMetaData.apply(oldState.getLastAcceptedState().metaData()); - if (updatedMetaData != oldState.getLastAcceptedState().metaData()) { - metaStateService.writeGlobalStateAndUpdateManifest("update global state", updatedMetaData); - } - final long updatedTerm = adaptCurrentTerm.apply(oldState.getCurrentTerm()); - if (updatedTerm != oldState.getCurrentTerm()) { - final Manifest manifest = metaStateService.loadManifestOrEmpty(); - metaStateService.writeManifestAndCleanup("update term", - new Manifest(updatedTerm, manifest.getClusterStateVersion(), manifest.getGlobalGeneration(), - manifest.getIndexGenerations())); - } - delegate = new MockGatewayMetaState(Settings.EMPTY, nodeEnvironment, xContentRegistry(), newLocalNode) - .getPersistedState(Settings.EMPTY, null); - } else { - nodeEnvironment = null; - BytesStreamOutput outStream = new BytesStreamOutput(); - outStream.setVersion(Version.CURRENT); - final MetaData updatedMetaData = adaptGlobalMetaData.apply(oldState.getLastAcceptedState().metaData()); - final ClusterState clusterState; - if (updatedMetaData != oldState.getLastAcceptedState().metaData()) { - clusterState = ClusterState.builder(oldState.getLastAcceptedState()).metaData(updatedMetaData).build(); - } else { - clusterState = oldState.getLastAcceptedState(); - } - clusterState.writeTo(outStream); - StreamInput inStream = new NamedWriteableAwareStreamInput(outStream.bytes().streamInput(), - new NamedWriteableRegistry(ClusterModule.getNamedWriteables())); - // adapt cluster state to new localNode instance and add blocks - delegate = new InMemoryPersistedState(adaptCurrentTerm.apply(oldState.getCurrentTerm()), - ClusterStateUpdaters.addStateNotRecoveredBlock(ClusterState.readFrom(inStream, newLocalNode))); - } - } catch (IOException e) { - throw new UncheckedIOException("Unable to create MockPersistedState", e); - } - } - - private void possiblyFail(String description) { - if (disruptStorage && rarely()) { - logger.trace("simulating IO exception [{}]", description); - // In the real-life IOError might be thrown, for example if state fsync fails. - // This will require node restart and we're not emulating it here. - throw new UncheckedIOException(new IOException("simulated IO exception [" + description + ']')); - } - } - - @Override - public long getCurrentTerm() { - return delegate.getCurrentTerm(); - } - - @Override - public ClusterState getLastAcceptedState() { - return delegate.getLastAcceptedState(); - } - - @Override - public void setCurrentTerm(long currentTerm) { - possiblyFail("before writing term of " + currentTerm); - delegate.setCurrentTerm(currentTerm); - } - - @Override - public void setLastAcceptedState(ClusterState clusterState) { - possiblyFail("before writing last-accepted state of term=" + clusterState.term() + ", version=" + clusterState.version()); - delegate.setLastAcceptedState(clusterState); - } - } - - class ClusterNode { - private final Logger logger = LogManager.getLogger(ClusterNode.class); - - private final int nodeIndex; - private Coordinator coordinator; - private final DiscoveryNode localNode; - private final MockPersistedState persistedState; - private final Settings nodeSettings; - private AckedFakeThreadPoolMasterService masterService; - private DisruptableClusterApplierService clusterApplierService; - private ClusterService clusterService; - private TransportService transportService; - private DisruptableMockTransport mockTransport; - private List> extraJoinValidators = new ArrayList<>(); - - ClusterNode(int nodeIndex, boolean masterEligible, Settings nodeSettings) { - this(nodeIndex, createDiscoveryNode(nodeIndex, masterEligible), defaultPersistedStateSupplier, nodeSettings); - } - - ClusterNode(int nodeIndex, DiscoveryNode localNode, Function persistedStateSupplier, - Settings nodeSettings) { - this.nodeIndex = nodeIndex; - this.localNode = localNode; - this.nodeSettings = nodeSettings; - persistedState = persistedStateSupplier.apply(localNode); - onNodeLog(localNode, this::setUp).run(); - } - - private void setUp() { - mockTransport = new DisruptableMockTransport(localNode, logger) { - @Override - protected void execute(Runnable runnable) { - deterministicTaskQueue.scheduleNow(onNode(runnable)); - } - - @Override - protected ConnectionStatus getConnectionStatus(DiscoveryNode destination) { - return Cluster.this.getConnectionStatus(getLocalNode(), destination); - } - - @Override - protected Optional getDisruptableMockTransport(TransportAddress address) { - return clusterNodes.stream().map(cn -> cn.mockTransport) - .filter(transport -> transport.getLocalNode().getAddress().equals(address)).findAny(); - } - }; - - final Settings settings = nodeSettings.hasValue(DiscoveryModule.DISCOVERY_TYPE_SETTING.getKey()) ? - nodeSettings : Settings.builder().put(nodeSettings) - .putList(ClusterBootstrapService.INITIAL_MASTER_NODES_SETTING.getKey(), - ClusterBootstrapService.INITIAL_MASTER_NODES_SETTING.get(Settings.EMPTY)).build(); // suppress auto-bootstrap - transportService = mockTransport.createTransportService( - settings, deterministicTaskQueue.getThreadPool(this::onNode), NOOP_TRANSPORT_INTERCEPTOR, - a -> localNode, null, emptySet()); - masterService = new AckedFakeThreadPoolMasterService(localNode.getId(), "test", - runnable -> deterministicTaskQueue.scheduleNow(onNode(runnable))); - final ClusterSettings clusterSettings = new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); - clusterApplierService = new DisruptableClusterApplierService(localNode.getId(), settings, clusterSettings, - deterministicTaskQueue, this::onNode); - clusterService = new ClusterService(settings, clusterSettings, masterService, clusterApplierService); - clusterService.setNodeConnectionsService( - new NodeConnectionsService(clusterService.getSettings(), deterministicTaskQueue.getThreadPool(this::onNode), - transportService)); - final Collection> onJoinValidators = - Collections.singletonList((dn, cs) -> extraJoinValidators.forEach(validator -> validator.accept(dn, cs))); - final AllocationService allocationService = ESAllocationTestCase.createAllocationService(Settings.EMPTY); - coordinator = new Coordinator("test_node", settings, clusterSettings, transportService, writableRegistry(), - allocationService, masterService, this::getPersistedState, - Cluster.this::provideSeedHosts, clusterApplierService, onJoinValidators, Randomness.get(), s -> {}, - ElectionStrategy.DefaultElectionStrategy.INSTANCE); - masterService.setClusterStatePublisher(coordinator); - final GatewayService gatewayService = new GatewayService(settings, allocationService, clusterService, - deterministicTaskQueue.getThreadPool(this::onNode), null, coordinator); - - logger.trace("starting up [{}]", localNode); - transportService.start(); - transportService.acceptIncomingRequests(); - coordinator.start(); - gatewayService.start(); - clusterService.start(); - coordinator.startInitialJoin(); - } - - void close() { - onNode(() -> { - logger.trace("taking down [{}]", localNode); - coordinator.stop(); - clusterService.stop(); - //transportService.stop(); // does blocking stuff :/ - clusterService.close(); - coordinator.close(); - //transportService.close(); // does blocking stuff :/ - }); - } - - ClusterNode restartedNode() { - return restartedNode(Function.identity(), Function.identity(), nodeSettings); - } - - ClusterNode restartedNode(Function adaptGlobalMetaData, Function adaptCurrentTerm, - Settings nodeSettings) { - final TransportAddress address = randomBoolean() ? buildNewFakeTransportAddress() : localNode.getAddress(); - final DiscoveryNode newLocalNode = new DiscoveryNode(localNode.getName(), localNode.getId(), - UUIDs.randomBase64UUID(random()), // generated deterministically for repeatable tests - address.address().getHostString(), address.getAddress(), address, Collections.emptyMap(), - localNode.isMasterNode() ? DiscoveryNodeRole.BUILT_IN_ROLES : emptySet(), Version.CURRENT); - return new ClusterNode(nodeIndex, newLocalNode, - node -> new MockPersistedState(newLocalNode, persistedState, adaptGlobalMetaData, adaptCurrentTerm), nodeSettings); - } - - private PersistedState getPersistedState() { - return persistedState; - } - - String getId() { - return localNode.getId(); - } - - DiscoveryNode getLocalNode() { - return localNode; - } - - boolean isLeader() { - return coordinator.getMode() == LEADER; - } - - boolean isCandidate() { - return coordinator.getMode() == CANDIDATE; - } - - ClusterState improveConfiguration(ClusterState currentState) { - synchronized (coordinator.mutex) { - return coordinator.improveConfiguration(currentState); - } - } - - void setClusterStateApplyResponse(ClusterStateApplyResponse clusterStateApplyResponse) { - clusterApplierService.clusterStateApplyResponse = clusterStateApplyResponse; - } - - ClusterStateApplyResponse getClusterStateApplyResponse() { - return clusterApplierService.clusterStateApplyResponse; - } - - Runnable onNode(Runnable runnable) { - final Runnable wrapped = onNodeLog(localNode, runnable); - return new Runnable() { - @Override - public void run() { - if (clusterNodes.contains(ClusterNode.this) == false) { - logger.trace("ignoring runnable {} from node {} as node has been removed from cluster", runnable, localNode); - return; - } - wrapped.run(); - } - - @Override - public String toString() { - return wrapped.toString(); - } - }; - } - - 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(), (source, e) -> {}); - } - - AckCollector submitValue(final long value) { - return submitValue(0, value); - } - - AckCollector submitValue(final int key, final long value) { - final int eventId = history.invoke(new Tuple<>(key, value)); - return submitUpdateTask("new value [" + value + "]", cs -> setValue(cs, key, value), new ClusterStateTaskListener() { - @Override - public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) { - history.respond(eventId, value(oldState, key)); - } - - @Override - public void onNoLongerMaster(String source) { - // in this case, we know for sure that event was not processed by the system and will not change history - // remove event to help avoid bloated history and state space explosion in linearizability checker - history.remove(eventId); - } - - @Override - public void onFailure(String source, Exception e) { - // do not remove event from history, the write might still take place - // instead, complete history when checking for linearizability - } - }); - } - - void readValue(int key) { - final int eventId = history.invoke(new Tuple<>(key, null)); - submitUpdateTask("read value", cs -> ClusterState.builder(cs).build(), new ClusterStateTaskListener() { - @Override - public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) { - history.respond(eventId, value(newState, key)); - } - - @Override - public void onFailure(String source, Exception e) { - // reads do not change state - // remove event to help avoid bloated history and state space explosion in linearizability checker - history.remove(eventId); - } - }); - } - - AckCollector submitUpdateTask(String source, UnaryOperator clusterStateUpdate, - ClusterStateTaskListener taskListener) { - final AckCollector ackCollector = new AckCollector(); - onNode(() -> { - logger.trace("[{}] submitUpdateTask: enqueueing [{}]", localNode.getId(), source); - final long submittedTerm = coordinator.getCurrentTerm(); - masterService.submitStateUpdateTask(source, - new ClusterStateUpdateTask() { - @Override - public ClusterState execute(ClusterState currentState) { - assertThat(currentState.term(), greaterThanOrEqualTo(submittedTerm)); - masterService.nextAckCollector = ackCollector; - return clusterStateUpdate.apply(currentState); - } - - @Override - public void onFailure(String source, Exception e) { - logger.debug(() -> new ParameterizedMessage("failed to publish: [{}]", source), e); - taskListener.onFailure(source, e); - } - - @Override - public void onNoLongerMaster(String source) { - logger.trace("no longer master: [{}]", source); - taskListener.onNoLongerMaster(source); - } - - @Override - public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) { - updateCommittedStates(); - ClusterState state = committedStatesByVersion.get(newState.version()); - assertNotNull("State not committed : " + newState.toString(), state); - assertStateEquals(state, newState); - logger.trace("successfully published: [{}]", newState); - taskListener.clusterStateProcessed(source, oldState, newState); - } - }); - }).run(); - return ackCollector; - } - - @Override - public String toString() { - return localNode.toString(); - } - - boolean heal() { - boolean unBlackholed = blackholedNodes.remove(localNode.getId()); - boolean unDisconnected = disconnectedNodes.remove(localNode.getId()); - assert unBlackholed == false || unDisconnected == false; - return unBlackholed || unDisconnected; - } - - boolean disconnect() { - boolean unBlackholed = blackholedNodes.remove(localNode.getId()); - boolean disconnected = disconnectedNodes.add(localNode.getId()); - assert disconnected || unBlackholed == false; - return disconnected; - } - - boolean blackhole() { - boolean unDisconnected = disconnectedNodes.remove(localNode.getId()); - boolean blackholed = blackholedNodes.add(localNode.getId()); - assert blackholed || unDisconnected == false; - return blackholed; - } - - void onDisconnectEventFrom(ClusterNode clusterNode) { - transportService.disconnectFromNode(clusterNode.localNode); - } - - ClusterState getLastAppliedClusterState() { - return clusterApplierService.state(); - } - - void applyInitialConfiguration() { - onNode(() -> { - final Set nodeIdsWithPlaceholders = new HashSet<>(initialConfiguration.getNodeIds()); - Stream.generate(() -> BOOTSTRAP_PLACEHOLDER_PREFIX + UUIDs.randomBase64UUID(random())) - .limit((Math.max(initialConfiguration.getNodeIds().size(), 2) - 1) / 2) - .forEach(nodeIdsWithPlaceholders::add); - final Set nodeIds = new HashSet<>( - randomSubsetOf(initialConfiguration.getNodeIds().size(), nodeIdsWithPlaceholders)); - // initial configuration should not have a place holder for local node - if (initialConfiguration.getNodeIds().contains(localNode.getId()) && nodeIds.contains(localNode.getId()) == false) { - nodeIds.remove(nodeIds.iterator().next()); - nodeIds.add(localNode.getId()); - } - final VotingConfiguration configurationWithPlaceholders = new VotingConfiguration(nodeIds); - try { - coordinator.setInitialConfiguration(configurationWithPlaceholders); - logger.info("successfully set initial configuration to {}", configurationWithPlaceholders); - } catch (CoordinationStateRejectedException e) { - logger.info(new ParameterizedMessage("failed to set initial configuration to {}", - configurationWithPlaceholders), e); - } - }).run(); - } - - private boolean isNotUsefullyBootstrapped() { - return getLocalNode().isMasterNode() == false || coordinator.isInitialConfigurationSet() == false; - } - } - - private List provideSeedHosts(HostsResolver ignored) { - return seedHostsList != null ? seedHostsList - : clusterNodes.stream().map(ClusterNode::getLocalNode).map(DiscoveryNode::getAddress).collect(Collectors.toList()); - } - } - - private static final String NODE_ID_LOG_CONTEXT_KEY = "nodeId"; - - private static String getNodeIdForLogContext(DiscoveryNode node) { - return "{" + node.getId() + "}{" + node.getEphemeralId() + "}"; - } - - public static Runnable onNodeLog(DiscoveryNode node, Runnable runnable) { - final String nodeId = getNodeIdForLogContext(node); - return new Runnable() { - @Override - public void run() { - try (CloseableThreadContext.Instance ignored = CloseableThreadContext.put(NODE_ID_LOG_CONTEXT_KEY, nodeId)) { - runnable.run(); - } - } - - @Override - public String toString() { - return nodeId + ": " + runnable.toString(); - } - }; - } - - static class AckCollector implements AckListener { - - private final Set ackedNodes = new HashSet<>(); - private final List successfulNodes = new ArrayList<>(); - private final List unsuccessfulNodes = new ArrayList<>(); - - @Override - public void onCommit(TimeValue commitTime) { - // TODO we only currently care about per-node acks - } - - @Override - public void onNodeAck(DiscoveryNode node, Exception e) { - assertTrue("duplicate ack from " + node, ackedNodes.add(node)); - if (e == null) { - successfulNodes.add(node); - } else { - unsuccessfulNodes.add(node); - } - } - - boolean hasAckedSuccessfully(ClusterNode clusterNode) { - return successfulNodes.contains(clusterNode.localNode); - } - - boolean hasAckedUnsuccessfully(ClusterNode clusterNode) { - return unsuccessfulNodes.contains(clusterNode.localNode); - } - - boolean hasAcked(ClusterNode clusterNode) { - return ackedNodes.contains(clusterNode.localNode); - } - - int getSuccessfulAckIndex(ClusterNode clusterNode) { - assert successfulNodes.contains(clusterNode.localNode) : "get index of " + clusterNode; - return successfulNodes.indexOf(clusterNode.localNode); - } - } - - static class AckedFakeThreadPoolMasterService extends FakeThreadPoolMasterService { - - AckCollector nextAckCollector = new AckCollector(); - - AckedFakeThreadPoolMasterService(String nodeName, String serviceName, Consumer onTaskAvailableToRun) { - super(nodeName, serviceName, onTaskAvailableToRun); - } - - @Override - protected AckListener wrapAckListener(AckListener ackListener) { - final AckCollector ackCollector = nextAckCollector; - nextAckCollector = new AckCollector(); - return new AckListener() { - @Override - public void onCommit(TimeValue commitTime) { - ackCollector.onCommit(commitTime); - ackListener.onCommit(commitTime); - } - - @Override - public void onNodeAck(DiscoveryNode node, Exception e) { - ackCollector.onNodeAck(node, e); - ackListener.onNodeAck(node, e); - } - }; - } - } - static class DisruptableClusterApplierService extends ClusterApplierService { - private final String nodeName; - private final DeterministicTaskQueue deterministicTaskQueue; - ClusterStateApplyResponse clusterStateApplyResponse = ClusterStateApplyResponse.SUCCEED; - - DisruptableClusterApplierService(String nodeName, Settings settings, ClusterSettings clusterSettings, - DeterministicTaskQueue deterministicTaskQueue, Function runnableWrapper) { - super(nodeName, settings, clusterSettings, deterministicTaskQueue.getThreadPool(runnableWrapper)); - this.nodeName = nodeName; - this.deterministicTaskQueue = deterministicTaskQueue; - addStateApplier(event -> { - switch (clusterStateApplyResponse) { - case SUCCEED: - case HANG: - final ClusterState oldClusterState = event.previousState(); - final ClusterState newClusterState = event.state(); - assert oldClusterState.version() <= newClusterState.version() : "updating cluster state from version " - + oldClusterState.version() + " to stale version " + newClusterState.version(); - break; - case FAIL: - throw new ElasticsearchException("simulated cluster state applier failure"); - } - }); - } - - @Override - protected PrioritizedEsThreadPoolExecutor createThreadPoolExecutor() { - return new MockSinglePrioritizingExecutor(nodeName, deterministicTaskQueue); - } - - @Override - public void onNewClusterState(String source, Supplier clusterStateSupplier, ClusterApplyListener listener) { - if (clusterStateApplyResponse == ClusterStateApplyResponse.HANG) { - if (randomBoolean()) { - // apply cluster state, but don't notify listener - super.onNewClusterState(source, clusterStateSupplier, (source1, e) -> { - // ignore result - }); - } - } else { - super.onNewClusterState(source, clusterStateSupplier, listener); - } - } - - @Override - protected void connectToNodesAndWait(ClusterState newClusterState) { - // don't do anything, and don't block - } - } - - private static DiscoveryNode createDiscoveryNode(int nodeIndex, boolean masterEligible) { - final TransportAddress address = buildNewFakeTransportAddress(); - return new DiscoveryNode("", "node" + nodeIndex, - UUIDs.randomBase64UUID(random()), // generated deterministically for repeatable tests - address.address().getHostString(), address.getAddress(), address, Collections.emptyMap(), - masterEligible ? DiscoveryNodeRole.BUILT_IN_ROLES : emptySet(), Version.CURRENT); - } - - /** - * How to behave with a new cluster state - */ - enum ClusterStateApplyResponse { - /** - * Apply the state (default) - */ - SUCCEED, - - /** - * Reject the state with an exception. - */ - FAIL, - - /** - * Never respond either way. - */ - HANG, - } - - public ClusterState setValue(ClusterState clusterState, int key, long value) { - return ClusterState.builder(clusterState).metaData( - MetaData.builder(clusterState.metaData()) - .persistentSettings(Settings.builder() - .put(clusterState.metaData().persistentSettings()) - .put("value_" + key, value) - .build()) - .build()) - .build(); - } - - public long value(ClusterState clusterState) { - return value(clusterState, 0); - } - - public long value(ClusterState clusterState, int key) { - return clusterState.metaData().persistentSettings().getAsLong("value_" + key, 0L); - } - - public void assertStateEquals(ClusterState clusterState1, ClusterState clusterState2) { - assertEquals(clusterState1.version(), clusterState2.version()); - assertEquals(clusterState1.term(), clusterState2.term()); - assertEquals(keySet(clusterState1), keySet(clusterState2)); - for (int key : keySet(clusterState1)) { - assertEquals(value(clusterState1, key), value(clusterState2, key)); - } - } - - public Set keySet(ClusterState clusterState) { - return clusterState.metaData().persistentSettings().keySet().stream() - .filter(s -> s.startsWith("value_")).map(s -> Integer.valueOf(s.substring("value_".length()))).collect(Collectors.toSet()); - } - - /** - * Simple register model. Writes are modeled by providing an integer input. Reads are modeled by providing null as input. - * Responses that time out are modeled by returning null. Successful writes return the previous value of the register. - */ - private final SequentialSpec spec = new LinearizabilityChecker.KeyedSpec() { - @Override - public Object getKey(Object value) { - return ((Tuple) value).v1(); - } - - @Override - public Object getValue(Object value) { - return ((Tuple) value).v2(); - } - - @Override - public Object initialState() { - return 0L; - } - - @Override - public Optional nextState(Object currentState, Object input, Object output) { - // null input is read, non-null is write - if (input == null) { - // history is completed with null, simulating timeout, which assumes that read went through - if (output == null || currentState.equals(output)) { - return Optional.of(currentState); - } - return Optional.empty(); - } else { - if (output == null || currentState.equals(output)) { - // history is completed with null, simulating timeout, which assumes that write went through - return Optional.of(input); - } - return Optional.empty(); - } - } - }; - - public void testRegisterSpecConsistency() { - assertThat(spec.initialState(), equalTo(0L)); - assertThat(spec.nextState(7, 42, 7), equalTo(Optional.of(42))); // successful write 42 returns previous value 7 - assertThat(spec.nextState(7, 42, null), equalTo(Optional.of(42))); // write 42 times out - assertThat(spec.nextState(7, null, 7), equalTo(Optional.of(7))); // successful read - assertThat(spec.nextState(7, null, null), equalTo(Optional.of(7))); // read times out - assertThat(spec.nextState(7, null, 42), equalTo(Optional.empty())); - } } diff --git a/test/framework/src/main/java/org/elasticsearch/cluster/coordination/AbstractCoordinatorTestCase.java b/test/framework/src/main/java/org/elasticsearch/cluster/coordination/AbstractCoordinatorTestCase.java new file mode 100644 index 0000000000000..a6f1ab34384d9 --- /dev/null +++ b/test/framework/src/main/java/org/elasticsearch/cluster/coordination/AbstractCoordinatorTestCase.java @@ -0,0 +1,1351 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.elasticsearch.cluster.coordination; + +import com.carrotsearch.randomizedtesting.RandomizedContext; +import org.apache.logging.log4j.CloseableThreadContext; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.apache.logging.log4j.message.ParameterizedMessage; +import org.elasticsearch.ElasticsearchException; +import org.elasticsearch.Version; +import org.elasticsearch.cluster.ClusterModule; +import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.ClusterStateTaskListener; +import org.elasticsearch.cluster.ClusterStateUpdateTask; +import org.elasticsearch.cluster.ESAllocationTestCase; +import org.elasticsearch.cluster.NodeConnectionsService; +import org.elasticsearch.cluster.coordination.AbstractCoordinatorTestCase.Cluster.ClusterNode; +import org.elasticsearch.cluster.coordination.CoordinationMetaData.VotingConfiguration; +import org.elasticsearch.cluster.coordination.LinearizabilityChecker.History; +import org.elasticsearch.cluster.coordination.LinearizabilityChecker.SequentialSpec; +import org.elasticsearch.cluster.metadata.Manifest; +import org.elasticsearch.cluster.metadata.MetaData; +import org.elasticsearch.cluster.node.DiscoveryNode; +import org.elasticsearch.cluster.node.DiscoveryNodeRole; +import org.elasticsearch.cluster.routing.allocation.AllocationService; +import org.elasticsearch.cluster.service.ClusterApplierService; +import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.common.Nullable; +import org.elasticsearch.common.Randomness; +import org.elasticsearch.common.UUIDs; +import org.elasticsearch.common.collect.Tuple; +import org.elasticsearch.common.io.stream.BytesStreamOutput; +import org.elasticsearch.common.io.stream.NamedWriteableAwareStreamInput; +import org.elasticsearch.common.io.stream.NamedWriteableRegistry; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.settings.ClusterSettings; +import org.elasticsearch.common.settings.Setting; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.transport.TransportAddress; +import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.common.util.concurrent.PrioritizedEsThreadPoolExecutor; +import org.elasticsearch.discovery.DiscoveryModule; +import org.elasticsearch.discovery.SeedHostsProvider; +import org.elasticsearch.env.NodeEnvironment; +import org.elasticsearch.gateway.ClusterStateUpdaters; +import org.elasticsearch.gateway.GatewayService; +import org.elasticsearch.gateway.MetaStateService; +import org.elasticsearch.gateway.MockGatewayMetaState; +import org.elasticsearch.indices.cluster.FakeThreadPoolMasterService; +import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.test.disruption.DisruptableMockTransport; +import org.elasticsearch.test.disruption.DisruptableMockTransport.ConnectionStatus; +import org.elasticsearch.transport.TransportService; +import org.hamcrest.Matcher; +import org.hamcrest.core.IsCollectionContaining; +import org.junit.After; +import org.junit.Before; + +import java.io.IOException; +import java.io.UncheckedIOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.Callable; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.BiConsumer; +import java.util.function.Consumer; +import java.util.function.Function; +import java.util.function.Supplier; +import java.util.function.UnaryOperator; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static java.util.Collections.emptyList; +import static java.util.Collections.emptySet; +import static org.elasticsearch.cluster.coordination.AbstractCoordinatorTestCase.Cluster.DEFAULT_DELAY_VARIABILITY; +import static org.elasticsearch.cluster.coordination.ClusterBootstrapService.BOOTSTRAP_PLACEHOLDER_PREFIX; +import static org.elasticsearch.cluster.coordination.CoordinationStateTestCluster.clusterState; +import static org.elasticsearch.cluster.coordination.Coordinator.Mode.CANDIDATE; +import static org.elasticsearch.cluster.coordination.Coordinator.Mode.FOLLOWER; +import static org.elasticsearch.cluster.coordination.Coordinator.Mode.LEADER; +import static org.elasticsearch.cluster.coordination.Coordinator.PUBLISH_TIMEOUT_SETTING; +import static org.elasticsearch.cluster.coordination.ElectionSchedulerFactory.ELECTION_BACK_OFF_TIME_SETTING; +import static org.elasticsearch.cluster.coordination.ElectionSchedulerFactory.ELECTION_DURATION_SETTING; +import static org.elasticsearch.cluster.coordination.ElectionSchedulerFactory.ELECTION_INITIAL_TIMEOUT_SETTING; +import static org.elasticsearch.cluster.coordination.FollowersChecker.FOLLOWER_CHECK_INTERVAL_SETTING; +import static org.elasticsearch.cluster.coordination.FollowersChecker.FOLLOWER_CHECK_RETRY_COUNT_SETTING; +import static org.elasticsearch.cluster.coordination.FollowersChecker.FOLLOWER_CHECK_TIMEOUT_SETTING; +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.NoMasterBlockService.NO_MASTER_BLOCK_ID; +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.gateway.GatewayService.STATE_NOT_RECOVERED_BLOCK; +import static org.elasticsearch.node.Node.NODE_NAME_SETTING; +import static org.elasticsearch.transport.TransportService.NOOP_TRANSPORT_INTERCEPTOR; +import static org.hamcrest.Matchers.empty; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.greaterThan; +import static org.hamcrest.Matchers.greaterThanOrEqualTo; +import static org.hamcrest.Matchers.is; +import static org.hamcrest.Matchers.lessThanOrEqualTo; +import static org.hamcrest.Matchers.not; +import static org.hamcrest.Matchers.nullValue; +import static org.hamcrest.Matchers.sameInstance; + +public class AbstractCoordinatorTestCase extends ESTestCase { + + protected final List nodeEnvironments = new ArrayList<>(); + + protected final AtomicInteger nextNodeIndex = new AtomicInteger(); + + @Before + public void resetNodeIndexBeforeEachTest() { + nextNodeIndex.set(0); + } + + @After + public void closeNodeEnvironmentsAfterEachTest() { + for (NodeEnvironment nodeEnvironment : nodeEnvironments) { + nodeEnvironment.close(); + } + nodeEnvironments.clear(); + } + + @Before + public void resetPortCounterBeforeEachTest() { + resetPortCounter(); + } + + // check that runRandomly leads to reproducible results + public void testRepeatableTests() throws Exception { + final Callable test = () -> { + resetNodeIndexBeforeEachTest(); + final Cluster cluster = new Cluster(randomIntBetween(1, 5)); + cluster.runRandomly(); + final long afterRunRandomly = value(cluster.getAnyNode().getLastAppliedClusterState()); + cluster.stabilise(); + final long afterStabilisation = value(cluster.getAnyNode().getLastAppliedClusterState()); + return afterRunRandomly ^ afterStabilisation; + }; + final long seed = randomLong(); + logger.info("First run with seed [{}]", seed); + final long result1 = RandomizedContext.current().runWithPrivateRandomness(seed, test); + logger.info("Second run with seed [{}]", seed); + final long result2 = RandomizedContext.current().runWithPrivateRandomness(seed, test); + assertEquals(result1, result2); + } + + protected static long defaultMillis(Setting setting) { + return setting.get(Settings.EMPTY).millis() + DEFAULT_DELAY_VARIABILITY; + } + + protected static int defaultInt(Setting setting) { + return setting.get(Settings.EMPTY); + } + + // Updating the cluster state involves up to 7 delays: + // 1. submit the task to the master service + // 2. send PublishRequest + // 3. receive PublishResponse + // 4. send ApplyCommitRequest + // 5. apply committed cluster state + // 6. receive ApplyCommitResponse + // 7. apply committed state on master (last one to apply cluster state) + public static final long DEFAULT_CLUSTER_STATE_UPDATE_DELAY = 7 * DEFAULT_DELAY_VARIABILITY; + + private static final int ELECTION_RETRIES = 10; + + // The time it takes to complete an election + public static final long DEFAULT_ELECTION_DELAY + // Pinging all peers twice should be enough to discover all nodes + = defaultMillis(DISCOVERY_FIND_PEERS_INTERVAL_SETTING) * 2 + // Then wait for an election to be scheduled; we allow enough time for retries to allow for collisions + + defaultMillis(ELECTION_INITIAL_TIMEOUT_SETTING) * ELECTION_RETRIES + + defaultMillis(ELECTION_BACK_OFF_TIME_SETTING) * ELECTION_RETRIES * (ELECTION_RETRIES - 1) / 2 + + defaultMillis(ELECTION_DURATION_SETTING) * ELECTION_RETRIES + // Allow two round-trip for pre-voting and voting + + 4 * DEFAULT_DELAY_VARIABILITY + // Then a commit of the new leader's first cluster state + + DEFAULT_CLUSTER_STATE_UPDATE_DELAY; + + public static final long DEFAULT_STABILISATION_TIME = + // If leader just blackholed, need to wait for this to be detected + (defaultMillis(LEADER_CHECK_INTERVAL_SETTING) + defaultMillis(LEADER_CHECK_TIMEOUT_SETTING)) + * defaultInt(LEADER_CHECK_RETRY_COUNT_SETTING) + // then wait for a follower to be promoted to leader + + DEFAULT_ELECTION_DELAY + // perhaps there is an election collision requiring another publication (which times out) and a term bump + + defaultMillis(PUBLISH_TIMEOUT_SETTING) + DEFAULT_ELECTION_DELAY + // then wait for the new leader to notice that the old leader is unresponsive + + (defaultMillis(FOLLOWER_CHECK_INTERVAL_SETTING) + defaultMillis(FOLLOWER_CHECK_TIMEOUT_SETTING)) + * defaultInt(FOLLOWER_CHECK_RETRY_COUNT_SETTING) + // then wait for the new leader to commit a state without the old leader + + DEFAULT_CLUSTER_STATE_UPDATE_DELAY; + + class Cluster { + + static final long EXTREME_DELAY_VARIABILITY = 10000L; + static final long DEFAULT_DELAY_VARIABILITY = 100L; + + final List clusterNodes; + final DeterministicTaskQueue deterministicTaskQueue = new DeterministicTaskQueue( + // TODO does ThreadPool need a node name any more? + Settings.builder().put(NODE_NAME_SETTING.getKey(), "deterministic-task-queue").build(), random()); + private boolean disruptStorage; + + final VotingConfiguration initialConfiguration; + + private final Set disconnectedNodes = new HashSet<>(); + private final Set blackholedNodes = new HashSet<>(); + private final Set> blackholedConnections = new HashSet<>(); + private final Map committedStatesByVersion = new HashMap<>(); + private final LinearizabilityChecker linearizabilityChecker = new LinearizabilityChecker(); + private final History history = new History(); + + private final Function defaultPersistedStateSupplier = MockPersistedState::new; + + @Nullable // null means construct a list from all the current nodes + private List seedHostsList; + + Cluster(int initialNodeCount) { + this(initialNodeCount, true, Settings.EMPTY); + } + + Cluster(int initialNodeCount, boolean allNodesMasterEligible, Settings nodeSettings) { + deterministicTaskQueue.setExecutionDelayVariabilityMillis(DEFAULT_DELAY_VARIABILITY); + + assertThat(initialNodeCount, greaterThan(0)); + + final Set masterEligibleNodeIds = new HashSet<>(initialNodeCount); + clusterNodes = new ArrayList<>(initialNodeCount); + for (int i = 0; i < initialNodeCount; i++) { + final ClusterNode clusterNode = new ClusterNode(nextNodeIndex.getAndIncrement(), + allNodesMasterEligible || i == 0 || randomBoolean(), nodeSettings); + clusterNodes.add(clusterNode); + if (clusterNode.getLocalNode().isMasterNode()) { + masterEligibleNodeIds.add(clusterNode.getId()); + } + } + + initialConfiguration = new VotingConfiguration(new HashSet<>( + randomSubsetOf(randomIntBetween(1, masterEligibleNodeIds.size()), masterEligibleNodeIds))); + + logger.info("--> creating cluster of {} nodes (master-eligible nodes: {}) with initial configuration {}", + initialNodeCount, masterEligibleNodeIds, initialConfiguration); + } + + List addNodesAndStabilise(int newNodesCount) { + final List addedNodes = addNodes(newNodesCount); + stabilise( + // The first pinging discovers the master + defaultMillis(DISCOVERY_FIND_PEERS_INTERVAL_SETTING) + // One message delay to send a join + + DEFAULT_DELAY_VARIABILITY + // Commit a new cluster state with the new node(s). Might be split into multiple commits, and each might need a + // followup reconfiguration + + newNodesCount * 2 * DEFAULT_CLUSTER_STATE_UPDATE_DELAY); + // TODO Investigate whether 4 publications is sufficient due to batching? A bound linear in the number of nodes isn't great. + return addedNodes; + } + + List addNodes(int newNodesCount) { + logger.info("--> adding {} nodes", newNodesCount); + + final List addedNodes = new ArrayList<>(); + for (int i = 0; i < newNodesCount; i++) { + final ClusterNode clusterNode = new ClusterNode(nextNodeIndex.getAndIncrement(), true, Settings.EMPTY); + addedNodes.add(clusterNode); + } + clusterNodes.addAll(addedNodes); + return addedNodes; + } + + int size() { + return clusterNodes.size(); + } + + void runRandomly() { + runRandomly(true); + } + + void runRandomly(boolean allowReboots) { + + // TODO supporting (preserving?) existing disruptions needs implementing if needed, for now we just forbid it + assertThat("may reconnect disconnected nodes, probably unexpected", disconnectedNodes, empty()); + assertThat("may reconnect blackholed nodes, probably unexpected", blackholedNodes, empty()); + + final List cleanupActions = new ArrayList<>(); + cleanupActions.add(disconnectedNodes::clear); + cleanupActions.add(blackholedNodes::clear); + cleanupActions.add(() -> disruptStorage = false); + + final int randomSteps = scaledRandomIntBetween(10, 10000); + final int keyRange = randomSteps / 50; // for randomized writes and reads + logger.info("--> start of safety phase of at least [{}] steps", randomSteps); + + deterministicTaskQueue.setExecutionDelayVariabilityMillis(EXTREME_DELAY_VARIABILITY); + disruptStorage = true; + int step = 0; + long finishTime = -1; + + while (finishTime == -1 || deterministicTaskQueue.getCurrentTimeMillis() <= finishTime) { + step++; + final int thisStep = step; // for lambdas + + if (randomSteps <= step && finishTime == -1) { + finishTime = deterministicTaskQueue.getLatestDeferredExecutionTime(); + deterministicTaskQueue.setExecutionDelayVariabilityMillis(DEFAULT_DELAY_VARIABILITY); + logger.debug("----> [runRandomly {}] reducing delay variability and running until [{}ms]", step, finishTime); + } + + try { + if (finishTime == -1 && randomBoolean() && randomBoolean() && randomBoolean()) { + final ClusterNode clusterNode = getAnyNodePreferringLeaders(); + final int key = randomIntBetween(0, keyRange); + final int newValue = randomInt(); + clusterNode.onNode(() -> { + logger.debug("----> [runRandomly {}] proposing new value [{}] to [{}]", + thisStep, newValue, clusterNode.getId()); + clusterNode.submitValue(key, newValue); + }).run(); + } else if (finishTime == -1 && randomBoolean() && randomBoolean() && randomBoolean()) { + final ClusterNode clusterNode = getAnyNodePreferringLeaders(); + final int key = randomIntBetween(0, keyRange); + clusterNode.onNode(() -> { + logger.debug("----> [runRandomly {}] reading value from [{}]", + thisStep, clusterNode.getId()); + clusterNode.readValue(key); + }).run(); + } else if (rarely()) { + final ClusterNode clusterNode = getAnyNodePreferringLeaders(); + final boolean autoShrinkVotingConfiguration = randomBoolean(); + clusterNode.onNode( + () -> { + logger.debug("----> [runRandomly {}] setting auto-shrink configuration to {} on {}", + thisStep, autoShrinkVotingConfiguration, clusterNode.getId()); + clusterNode.submitSetAutoShrinkVotingConfiguration(autoShrinkVotingConfiguration); + }).run(); + } else if (allowReboots && rarely()) { + // reboot random node + final ClusterNode clusterNode = getAnyNode(); + logger.debug("----> [runRandomly {}] rebooting [{}]", thisStep, clusterNode.getId()); + clusterNode.close(); + clusterNodes.forEach( + cn -> deterministicTaskQueue.scheduleNow(cn.onNode( + new Runnable() { + @Override + public void run() { + cn.transportService.disconnectFromNode(clusterNode.getLocalNode()); + } + + @Override + public String toString() { + return "disconnect from " + clusterNode.getLocalNode() + " after shutdown"; + } + }))); + clusterNodes.replaceAll(cn -> cn == clusterNode ? cn.restartedNode() : cn); + } else if (rarely()) { + final ClusterNode clusterNode = getAnyNode(); + clusterNode.onNode(() -> { + logger.debug("----> [runRandomly {}] forcing {} to become candidate", thisStep, clusterNode.getId()); + synchronized (clusterNode.coordinator.mutex) { + clusterNode.coordinator.becomeCandidate("runRandomly"); + } + }).run(); + } else if (rarely()) { + final ClusterNode clusterNode = getAnyNode(); + + switch (randomInt(2)) { + case 0: + if (clusterNode.heal()) { + logger.debug("----> [runRandomly {}] healing {}", step, clusterNode.getId()); + } + break; + case 1: + if (clusterNode.disconnect()) { + logger.debug("----> [runRandomly {}] disconnecting {}", step, clusterNode.getId()); + } + break; + case 2: + if (clusterNode.blackhole()) { + logger.debug("----> [runRandomly {}] blackholing {}", step, clusterNode.getId()); + } + break; + } + } else if (rarely()) { + final ClusterNode clusterNode = getAnyNode(); + logger.debug("----> [runRandomly {}] applying initial configuration on {}", step, clusterNode.getId()); + clusterNode.applyInitialConfiguration(); + } else { + if (deterministicTaskQueue.hasDeferredTasks() && randomBoolean()) { + deterministicTaskQueue.advanceTime(); + } else if (deterministicTaskQueue.hasRunnableTasks()) { + deterministicTaskQueue.runRandomTask(); + } + } + + // TODO other random steps: + // - reboot a node + // - abdicate leadership + + } catch (CoordinationStateRejectedException | UncheckedIOException ignored) { + // This is ok: it just means a message couldn't currently be handled. + } + + assertConsistentStates(); + } + + logger.debug("running {} cleanup actions", cleanupActions.size()); + cleanupActions.forEach(Runnable::run); + logger.debug("finished running cleanup actions"); + } + + private void assertConsistentStates() { + for (final ClusterNode clusterNode : clusterNodes) { + clusterNode.coordinator.invariant(); + } + updateCommittedStates(); + } + + private void updateCommittedStates() { + for (final ClusterNode clusterNode : clusterNodes) { + ClusterState applierState = clusterNode.coordinator.getApplierState(); + ClusterState storedState = committedStatesByVersion.get(applierState.getVersion()); + if (storedState == null) { + committedStatesByVersion.put(applierState.getVersion(), applierState); + } else { + assertEquals("expected " + applierState + " but got " + storedState, + value(applierState), value(storedState)); + } + } + } + + void stabilise() { + stabilise(DEFAULT_STABILISATION_TIME); + } + + void stabilise(long stabilisationDurationMillis) { + assertThat("stabilisation requires default delay variability (and proper cleanup of raised variability)", + deterministicTaskQueue.getExecutionDelayVariabilityMillis(), lessThanOrEqualTo(DEFAULT_DELAY_VARIABILITY)); + assertFalse("stabilisation requires stable storage", disruptStorage); + + bootstrapIfNecessary(); + + runFor(stabilisationDurationMillis, "stabilising"); + + final ClusterNode leader = getAnyLeader(); + final long leaderTerm = leader.coordinator.getCurrentTerm(); + + final int pendingTaskCount = leader.masterService.getFakeMasterServicePendingTaskCount(); + runFor((pendingTaskCount + 1) * DEFAULT_CLUSTER_STATE_UPDATE_DELAY, "draining task queue"); + + final Matcher isEqualToLeaderVersion = equalTo(leader.coordinator.getLastAcceptedState().getVersion()); + final String leaderId = leader.getId(); + + assertTrue(leaderId + " has been bootstrapped", leader.coordinator.isInitialConfigurationSet()); + assertTrue(leaderId + " exists in its last-applied state", leader.getLastAppliedClusterState().getNodes().nodeExists(leaderId)); + assertThat(leaderId + " has no NO_MASTER_BLOCK", + leader.getLastAppliedClusterState().blocks().hasGlobalBlockWithId(NO_MASTER_BLOCK_ID), equalTo(false)); + assertThat(leaderId + " has no STATE_NOT_RECOVERED_BLOCK", + leader.getLastAppliedClusterState().blocks().hasGlobalBlock(STATE_NOT_RECOVERED_BLOCK), equalTo(false)); + assertThat(leaderId + " has applied its state ", leader.getLastAppliedClusterState().getVersion(), isEqualToLeaderVersion); + + for (final ClusterNode clusterNode : clusterNodes) { + final String nodeId = clusterNode.getId(); + assertFalse(nodeId + " should not have an active publication", clusterNode.coordinator.publicationInProgress()); + + if (clusterNode == leader) { + assertThat(nodeId + " is still the leader", clusterNode.coordinator.getMode(), is(LEADER)); + assertThat(nodeId + " did not change term", clusterNode.coordinator.getCurrentTerm(), is(leaderTerm)); + continue; + } + + if (isConnectedPair(leader, clusterNode)) { + assertThat(nodeId + " is a follower of " + leaderId, clusterNode.coordinator.getMode(), is(FOLLOWER)); + assertThat(nodeId + " has the same term as " + leaderId, clusterNode.coordinator.getCurrentTerm(), is(leaderTerm)); + assertTrue(nodeId + " has voted for " + leaderId, leader.coordinator.hasJoinVoteFrom(clusterNode.getLocalNode())); + assertThat(nodeId + " has the same accepted state as " + leaderId, + clusterNode.coordinator.getLastAcceptedState().getVersion(), isEqualToLeaderVersion); + if (clusterNode.getClusterStateApplyResponse() == ClusterStateApplyResponse.SUCCEED) { + assertThat(nodeId + " has the same applied state as " + leaderId, + clusterNode.getLastAppliedClusterState().getVersion(), isEqualToLeaderVersion); + assertTrue(nodeId + " is in its own latest applied state", + clusterNode.getLastAppliedClusterState().getNodes().nodeExists(nodeId)); + } + 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().hasGlobalBlockWithId(NO_MASTER_BLOCK_ID), equalTo(false)); + assertThat(nodeId + " has no STATE_NOT_RECOVERED_BLOCK", + clusterNode.getLastAppliedClusterState().blocks().hasGlobalBlock(STATE_NOT_RECOVERED_BLOCK), 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().hasGlobalBlockWithId(NO_MASTER_BLOCK_ID), equalTo(true)); + assertFalse(nodeId + " is not in the applied state on " + leaderId, + leader.getLastAppliedClusterState().getNodes().nodeExists(nodeId)); + } + } + + final Set connectedNodeIds + = clusterNodes.stream().filter(n -> isConnectedPair(leader, n)).map(ClusterNode::getId).collect(Collectors.toSet()); + + assertThat(leader.getLastAppliedClusterState().getNodes().getSize(), equalTo(connectedNodeIds.size())); + + final ClusterState lastAcceptedState = leader.coordinator.getLastAcceptedState(); + final VotingConfiguration lastCommittedConfiguration = lastAcceptedState.getLastCommittedConfiguration(); + assertTrue(connectedNodeIds + " should be a quorum of " + lastCommittedConfiguration, + lastCommittedConfiguration.hasQuorum(connectedNodeIds)); + assertThat("leader " + leader.getLocalNode() + " should be part of voting configuration " + lastCommittedConfiguration, + lastCommittedConfiguration.getNodeIds(), IsCollectionContaining.hasItem(leader.getLocalNode().getId())); + + assertThat("no reconfiguration is in progress", + lastAcceptedState.getLastCommittedConfiguration(), equalTo(lastAcceptedState.getLastAcceptedConfiguration())); + assertThat("current configuration is already optimal", + leader.improveConfiguration(lastAcceptedState), sameInstance(lastAcceptedState)); + + logger.info("checking linearizability of history with size {}: {}", history.size(), history); + assertTrue("history not linearizable: " + history, linearizabilityChecker.isLinearizable(spec, history, i -> null)); + logger.info("linearizability check completed"); + } + + void bootstrapIfNecessary() { + if (clusterNodes.stream().allMatch(ClusterNode::isNotUsefullyBootstrapped)) { + assertThat("setting initial configuration may fail with disconnected nodes", disconnectedNodes, empty()); + assertThat("setting initial configuration may fail with blackholed nodes", blackholedNodes, empty()); + runFor(defaultMillis(DISCOVERY_FIND_PEERS_INTERVAL_SETTING) * 2, "discovery prior to setting initial configuration"); + final ClusterNode bootstrapNode = getAnyBootstrappableNode(); + bootstrapNode.applyInitialConfiguration(); + } else { + logger.info("setting initial configuration not required"); + } + } + + void runFor(long runDurationMillis, String description) { + final long endTime = deterministicTaskQueue.getCurrentTimeMillis() + runDurationMillis; + logger.info("--> runFor({}ms) running until [{}ms]: {}", runDurationMillis, endTime, description); + + while (deterministicTaskQueue.getCurrentTimeMillis() < endTime) { + + while (deterministicTaskQueue.hasRunnableTasks()) { + try { + deterministicTaskQueue.runRandomTask(); + } catch (CoordinationStateRejectedException e) { + logger.debug("ignoring benign exception thrown when stabilising", e); + } + for (final ClusterNode clusterNode : clusterNodes) { + clusterNode.coordinator.invariant(); + } + updateCommittedStates(); + } + + if (deterministicTaskQueue.hasDeferredTasks() == false) { + // A 1-node cluster has no need for fault detection etc so will eventually run out of things to do. + assert clusterNodes.size() == 1 : clusterNodes.size(); + break; + } + + deterministicTaskQueue.advanceTime(); + } + + logger.info("--> runFor({}ms) completed run until [{}ms]: {}", runDurationMillis, endTime, description); + } + + private boolean isConnectedPair(ClusterNode n1, ClusterNode n2) { + return n1 == n2 || + (getConnectionStatus(n1.getLocalNode(), n2.getLocalNode()) == ConnectionStatus.CONNECTED + && getConnectionStatus(n2.getLocalNode(), n1.getLocalNode()) == ConnectionStatus.CONNECTED); + } + + ClusterNode getAnyLeader() { + List allLeaders = clusterNodes.stream().filter(ClusterNode::isLeader).collect(Collectors.toList()); + assertThat("leaders", allLeaders, not(empty())); + return randomFrom(allLeaders); + } + + private final ConnectionStatus preferredUnknownNodeConnectionStatus = + randomFrom(ConnectionStatus.DISCONNECTED, ConnectionStatus.BLACK_HOLE); + + private ConnectionStatus getConnectionStatus(DiscoveryNode sender, DiscoveryNode destination) { + ConnectionStatus connectionStatus; + if (blackholedNodes.contains(sender.getId()) || blackholedNodes.contains(destination.getId())) { + connectionStatus = ConnectionStatus.BLACK_HOLE; + } else if (disconnectedNodes.contains(sender.getId()) || disconnectedNodes.contains(destination.getId())) { + connectionStatus = ConnectionStatus.DISCONNECTED; + } else if (blackholedConnections.contains(Tuple.tuple(sender.getId(), destination.getId()))) { + connectionStatus = ConnectionStatus.BLACK_HOLE_REQUESTS_ONLY; + } else if (nodeExists(sender) && nodeExists(destination)) { + connectionStatus = ConnectionStatus.CONNECTED; + } else { + connectionStatus = usually() ? preferredUnknownNodeConnectionStatus : + randomFrom(ConnectionStatus.DISCONNECTED, ConnectionStatus.BLACK_HOLE); + } + return connectionStatus; + } + + boolean nodeExists(DiscoveryNode node) { + return clusterNodes.stream().anyMatch(cn -> cn.getLocalNode().equals(node)); + } + + ClusterNode getAnyBootstrappableNode() { + return randomFrom(clusterNodes.stream().filter(n -> n.getLocalNode().isMasterNode()) + .filter(n -> initialConfiguration.getNodeIds().contains(n.getLocalNode().getId())) + .collect(Collectors.toList())); + } + + ClusterNode getAnyNode() { + return getAnyNodeExcept(); + } + + ClusterNode getAnyNodeExcept(ClusterNode... clusterNodes) { + List filteredNodes = getAllNodesExcept(clusterNodes); + assert filteredNodes.isEmpty() == false; + return randomFrom(filteredNodes); + } + + List getAllNodesExcept(ClusterNode... clusterNodes) { + Set forbiddenIds = Arrays.stream(clusterNodes).map(ClusterNode::getId).collect(Collectors.toSet()); + List acceptableNodes + = this.clusterNodes.stream().filter(n -> forbiddenIds.contains(n.getId()) == false).collect(Collectors.toList()); + return acceptableNodes; + } + + ClusterNode getAnyNodePreferringLeaders() { + for (int i = 0; i < 3; i++) { + ClusterNode clusterNode = getAnyNode(); + if (clusterNode.coordinator.getMode() == LEADER) { + return clusterNode; + } + } + return getAnyNode(); + } + + void setEmptySeedHostsList() { + seedHostsList = emptyList(); + } + + void blackholeConnectionsFrom(ClusterNode sender, ClusterNode destination) { + blackholedConnections.add(Tuple.tuple(sender.getId(), destination.getId())); + } + + void clearBlackholedConnections() { + blackholedConnections.clear(); + } + + class MockPersistedState implements CoordinationState.PersistedState { + private final CoordinationState.PersistedState delegate; + private final NodeEnvironment nodeEnvironment; + + MockPersistedState(DiscoveryNode localNode) { + try { + if (rarely()) { + nodeEnvironment = newNodeEnvironment(); + nodeEnvironments.add(nodeEnvironment); + delegate = new MockGatewayMetaState(Settings.EMPTY, nodeEnvironment, xContentRegistry(), localNode) + .getPersistedState(Settings.EMPTY, null); + } else { + nodeEnvironment = null; + delegate = new InMemoryPersistedState(0L, + ClusterStateUpdaters.addStateNotRecoveredBlock( + clusterState(0L, 0L, localNode, VotingConfiguration.EMPTY_CONFIG, + VotingConfiguration.EMPTY_CONFIG, 0L))); + } + } catch (IOException e) { + throw new UncheckedIOException("Unable to create MockPersistedState", e); + } + } + + MockPersistedState(DiscoveryNode newLocalNode, MockPersistedState oldState, + Function adaptGlobalMetaData, Function adaptCurrentTerm) { + try { + if (oldState.nodeEnvironment != null) { + nodeEnvironment = oldState.nodeEnvironment; + final MetaStateService metaStateService = new MetaStateService(nodeEnvironment, xContentRegistry()); + final MetaData updatedMetaData = adaptGlobalMetaData.apply(oldState.getLastAcceptedState().metaData()); + if (updatedMetaData != oldState.getLastAcceptedState().metaData()) { + metaStateService.writeGlobalStateAndUpdateManifest("update global state", updatedMetaData); + } + final long updatedTerm = adaptCurrentTerm.apply(oldState.getCurrentTerm()); + if (updatedTerm != oldState.getCurrentTerm()) { + final Manifest manifest = metaStateService.loadManifestOrEmpty(); + metaStateService.writeManifestAndCleanup("update term", + new Manifest(updatedTerm, manifest.getClusterStateVersion(), manifest.getGlobalGeneration(), + manifest.getIndexGenerations())); + } + delegate = new MockGatewayMetaState(Settings.EMPTY, nodeEnvironment, xContentRegistry(), newLocalNode) + .getPersistedState(Settings.EMPTY, null); + } else { + nodeEnvironment = null; + BytesStreamOutput outStream = new BytesStreamOutput(); + outStream.setVersion(Version.CURRENT); + final MetaData updatedMetaData = adaptGlobalMetaData.apply(oldState.getLastAcceptedState().metaData()); + final ClusterState clusterState; + if (updatedMetaData != oldState.getLastAcceptedState().metaData()) { + clusterState = ClusterState.builder(oldState.getLastAcceptedState()).metaData(updatedMetaData).build(); + } else { + clusterState = oldState.getLastAcceptedState(); + } + clusterState.writeTo(outStream); + StreamInput inStream = new NamedWriteableAwareStreamInput(outStream.bytes().streamInput(), + new NamedWriteableRegistry(ClusterModule.getNamedWriteables())); + // adapt cluster state to new localNode instance and add blocks + delegate = new InMemoryPersistedState(adaptCurrentTerm.apply(oldState.getCurrentTerm()), + ClusterStateUpdaters.addStateNotRecoveredBlock(ClusterState.readFrom(inStream, newLocalNode))); + } + } catch (IOException e) { + throw new UncheckedIOException("Unable to create MockPersistedState", e); + } + } + + private void possiblyFail(String description) { + if (disruptStorage && rarely()) { + logger.trace("simulating IO exception [{}]", description); + // In the real-life IOError might be thrown, for example if state fsync fails. + // This will require node restart and we're not emulating it here. + throw new UncheckedIOException(new IOException("simulated IO exception [" + description + ']')); + } + } + + @Override + public long getCurrentTerm() { + return delegate.getCurrentTerm(); + } + + @Override + public ClusterState getLastAcceptedState() { + return delegate.getLastAcceptedState(); + } + + @Override + public void setCurrentTerm(long currentTerm) { + possiblyFail("before writing term of " + currentTerm); + delegate.setCurrentTerm(currentTerm); + } + + @Override + public void setLastAcceptedState(ClusterState clusterState) { + possiblyFail("before writing last-accepted state of term=" + clusterState.term() + ", version=" + clusterState.version()); + delegate.setLastAcceptedState(clusterState); + } + } + + class ClusterNode { + private final Logger logger = LogManager.getLogger(ClusterNode.class); + + private final int nodeIndex; + Coordinator coordinator; + private final DiscoveryNode localNode; + final MockPersistedState persistedState; + final Settings nodeSettings; + private AckedFakeThreadPoolMasterService masterService; + private DisruptableClusterApplierService clusterApplierService; + private ClusterService clusterService; + TransportService transportService; + private DisruptableMockTransport mockTransport; + List> extraJoinValidators = new ArrayList<>(); + + ClusterNode(int nodeIndex, boolean masterEligible, Settings nodeSettings) { + this(nodeIndex, createDiscoveryNode(nodeIndex, masterEligible), defaultPersistedStateSupplier, nodeSettings); + } + + ClusterNode(int nodeIndex, DiscoveryNode localNode, Function persistedStateSupplier, + Settings nodeSettings) { + this.nodeIndex = nodeIndex; + this.localNode = localNode; + this.nodeSettings = nodeSettings; + persistedState = persistedStateSupplier.apply(localNode); + onNodeLog(localNode, this::setUp).run(); + } + + private void setUp() { + mockTransport = new DisruptableMockTransport(localNode, logger) { + @Override + protected void execute(Runnable runnable) { + deterministicTaskQueue.scheduleNow(onNode(runnable)); + } + + @Override + protected ConnectionStatus getConnectionStatus(DiscoveryNode destination) { + return Cluster.this.getConnectionStatus(getLocalNode(), destination); + } + + @Override + protected Optional getDisruptableMockTransport(TransportAddress address) { + return clusterNodes.stream().map(cn -> cn.mockTransport) + .filter(transport -> transport.getLocalNode().getAddress().equals(address)).findAny(); + } + }; + + final Settings settings = nodeSettings.hasValue(DiscoveryModule.DISCOVERY_TYPE_SETTING.getKey()) ? + nodeSettings : Settings.builder().put(nodeSettings) + .putList(ClusterBootstrapService.INITIAL_MASTER_NODES_SETTING.getKey(), + ClusterBootstrapService.INITIAL_MASTER_NODES_SETTING.get(Settings.EMPTY)).build(); // suppress auto-bootstrap + transportService = mockTransport.createTransportService( + settings, deterministicTaskQueue.getThreadPool(this::onNode), NOOP_TRANSPORT_INTERCEPTOR, + a -> localNode, null, emptySet()); + masterService = new AckedFakeThreadPoolMasterService(localNode.getId(), "test", + runnable -> deterministicTaskQueue.scheduleNow(onNode(runnable))); + final ClusterSettings clusterSettings = new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); + clusterApplierService = new DisruptableClusterApplierService(localNode.getId(), settings, clusterSettings, + deterministicTaskQueue, this::onNode); + clusterService = new ClusterService(settings, clusterSettings, masterService, clusterApplierService); + clusterService.setNodeConnectionsService( + new NodeConnectionsService(clusterService.getSettings(), deterministicTaskQueue.getThreadPool(this::onNode), + transportService)); + final Collection> onJoinValidators = + Collections.singletonList((dn, cs) -> extraJoinValidators.forEach(validator -> validator.accept(dn, cs))); + final AllocationService allocationService = ESAllocationTestCase.createAllocationService(Settings.EMPTY); + coordinator = new Coordinator("test_node", settings, clusterSettings, transportService, writableRegistry(), + allocationService, masterService, this::getPersistedState, + Cluster.this::provideSeedHosts, clusterApplierService, onJoinValidators, Randomness.get(), s -> {}, + ElectionStrategy.DefaultElectionStrategy.INSTANCE); + masterService.setClusterStatePublisher(coordinator); + final GatewayService gatewayService = new GatewayService(settings, allocationService, clusterService, + deterministicTaskQueue.getThreadPool(this::onNode), null, coordinator); + + logger.trace("starting up [{}]", localNode); + transportService.start(); + transportService.acceptIncomingRequests(); + coordinator.start(); + gatewayService.start(); + clusterService.start(); + coordinator.startInitialJoin(); + } + + void close() { + onNode(() -> { + logger.trace("taking down [{}]", localNode); + coordinator.stop(); + clusterService.stop(); + //transportService.stop(); // does blocking stuff :/ + clusterService.close(); + coordinator.close(); + //transportService.close(); // does blocking stuff :/ + }); + } + + ClusterNode restartedNode() { + return restartedNode(Function.identity(), Function.identity(), nodeSettings); + } + + ClusterNode restartedNode(Function adaptGlobalMetaData, Function adaptCurrentTerm, + Settings nodeSettings) { + final TransportAddress address = randomBoolean() ? buildNewFakeTransportAddress() : localNode.getAddress(); + final DiscoveryNode newLocalNode = new DiscoveryNode(localNode.getName(), localNode.getId(), + UUIDs.randomBase64UUID(random()), // generated deterministically for repeatable tests + address.address().getHostString(), address.getAddress(), address, Collections.emptyMap(), + localNode.isMasterNode() ? DiscoveryNodeRole.BUILT_IN_ROLES : emptySet(), Version.CURRENT); + return new ClusterNode(nodeIndex, newLocalNode, + node -> new MockPersistedState(newLocalNode, persistedState, adaptGlobalMetaData, adaptCurrentTerm), nodeSettings); + } + + private CoordinationState.PersistedState getPersistedState() { + return persistedState; + } + + String getId() { + return localNode.getId(); + } + + DiscoveryNode getLocalNode() { + return localNode; + } + + boolean isLeader() { + return coordinator.getMode() == LEADER; + } + + boolean isCandidate() { + return coordinator.getMode() == CANDIDATE; + } + + ClusterState improveConfiguration(ClusterState currentState) { + synchronized (coordinator.mutex) { + return coordinator.improveConfiguration(currentState); + } + } + + void setClusterStateApplyResponse(ClusterStateApplyResponse clusterStateApplyResponse) { + clusterApplierService.clusterStateApplyResponse = clusterStateApplyResponse; + } + + ClusterStateApplyResponse getClusterStateApplyResponse() { + return clusterApplierService.clusterStateApplyResponse; + } + + Runnable onNode(Runnable runnable) { + final Runnable wrapped = onNodeLog(localNode, runnable); + return new Runnable() { + @Override + public void run() { + if (clusterNodes.contains(ClusterNode.this) == false) { + logger.trace("ignoring runnable {} from node {} as node has been removed from cluster", runnable, localNode); + return; + } + wrapped.run(); + } + + @Override + public String toString() { + return wrapped.toString(); + } + }; + } + + 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(), (source, e) -> {}); + } + + AckCollector submitValue(final long value) { + return submitValue(0, value); + } + + AckCollector submitValue(final int key, final long value) { + final int eventId = history.invoke(new Tuple<>(key, value)); + return submitUpdateTask("new value [" + value + "]", cs -> setValue(cs, key, value), new ClusterStateTaskListener() { + @Override + public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) { + history.respond(eventId, value(oldState, key)); + } + + @Override + public void onNoLongerMaster(String source) { + // in this case, we know for sure that event was not processed by the system and will not change history + // remove event to help avoid bloated history and state space explosion in linearizability checker + history.remove(eventId); + } + + @Override + public void onFailure(String source, Exception e) { + // do not remove event from history, the write might still take place + // instead, complete history when checking for linearizability + } + }); + } + + void readValue(int key) { + final int eventId = history.invoke(new Tuple<>(key, null)); + submitUpdateTask("read value", cs -> ClusterState.builder(cs).build(), new ClusterStateTaskListener() { + @Override + public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) { + history.respond(eventId, value(newState, key)); + } + + @Override + public void onFailure(String source, Exception e) { + // reads do not change state + // remove event to help avoid bloated history and state space explosion in linearizability checker + history.remove(eventId); + } + }); + } + + AckCollector submitUpdateTask(String source, UnaryOperator clusterStateUpdate, + ClusterStateTaskListener taskListener) { + final AckCollector ackCollector = new AckCollector(); + onNode(() -> { + logger.trace("[{}] submitUpdateTask: enqueueing [{}]", localNode.getId(), source); + final long submittedTerm = coordinator.getCurrentTerm(); + masterService.submitStateUpdateTask(source, + new ClusterStateUpdateTask() { + @Override + public ClusterState execute(ClusterState currentState) { + assertThat(currentState.term(), greaterThanOrEqualTo(submittedTerm)); + masterService.nextAckCollector = ackCollector; + return clusterStateUpdate.apply(currentState); + } + + @Override + public void onFailure(String source, Exception e) { + logger.debug(() -> new ParameterizedMessage("failed to publish: [{}]", source), e); + taskListener.onFailure(source, e); + } + + @Override + public void onNoLongerMaster(String source) { + logger.trace("no longer master: [{}]", source); + taskListener.onNoLongerMaster(source); + } + + @Override + public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) { + updateCommittedStates(); + ClusterState state = committedStatesByVersion.get(newState.version()); + assertNotNull("State not committed : " + newState.toString(), state); + assertStateEquals(state, newState); + logger.trace("successfully published: [{}]", newState); + taskListener.clusterStateProcessed(source, oldState, newState); + } + }); + }).run(); + return ackCollector; + } + + @Override + public String toString() { + return localNode.toString(); + } + + boolean heal() { + boolean unBlackholed = blackholedNodes.remove(localNode.getId()); + boolean unDisconnected = disconnectedNodes.remove(localNode.getId()); + assert unBlackholed == false || unDisconnected == false; + return unBlackholed || unDisconnected; + } + + boolean disconnect() { + boolean unBlackholed = blackholedNodes.remove(localNode.getId()); + boolean disconnected = disconnectedNodes.add(localNode.getId()); + assert disconnected || unBlackholed == false; + return disconnected; + } + + boolean blackhole() { + boolean unDisconnected = disconnectedNodes.remove(localNode.getId()); + boolean blackholed = blackholedNodes.add(localNode.getId()); + assert blackholed || unDisconnected == false; + return blackholed; + } + + void onDisconnectEventFrom(ClusterNode clusterNode) { + transportService.disconnectFromNode(clusterNode.localNode); + } + + ClusterState getLastAppliedClusterState() { + return clusterApplierService.state(); + } + + void applyInitialConfiguration() { + onNode(() -> { + final Set nodeIdsWithPlaceholders = new HashSet<>(initialConfiguration.getNodeIds()); + Stream.generate(() -> BOOTSTRAP_PLACEHOLDER_PREFIX + UUIDs.randomBase64UUID(random())) + .limit((Math.max(initialConfiguration.getNodeIds().size(), 2) - 1) / 2) + .forEach(nodeIdsWithPlaceholders::add); + final Set nodeIds = new HashSet<>( + randomSubsetOf(initialConfiguration.getNodeIds().size(), nodeIdsWithPlaceholders)); + // initial configuration should not have a place holder for local node + if (initialConfiguration.getNodeIds().contains(localNode.getId()) && nodeIds.contains(localNode.getId()) == false) { + nodeIds.remove(nodeIds.iterator().next()); + nodeIds.add(localNode.getId()); + } + final VotingConfiguration configurationWithPlaceholders = new VotingConfiguration(nodeIds); + try { + coordinator.setInitialConfiguration(configurationWithPlaceholders); + logger.info("successfully set initial configuration to {}", configurationWithPlaceholders); + } catch (CoordinationStateRejectedException e) { + logger.info(new ParameterizedMessage("failed to set initial configuration to {}", + configurationWithPlaceholders), e); + } + }).run(); + } + + private boolean isNotUsefullyBootstrapped() { + return getLocalNode().isMasterNode() == false || coordinator.isInitialConfigurationSet() == false; + } + } + + private List provideSeedHosts(SeedHostsProvider.HostsResolver ignored) { + return seedHostsList != null ? seedHostsList + : clusterNodes.stream().map(ClusterNode::getLocalNode).map(DiscoveryNode::getAddress).collect(Collectors.toList()); + } + } + + public static final String NODE_ID_LOG_CONTEXT_KEY = "nodeId"; + + protected static String getNodeIdForLogContext(DiscoveryNode node) { + return "{" + node.getId() + "}{" + node.getEphemeralId() + "}"; + } + + public static Runnable onNodeLog(DiscoveryNode node, Runnable runnable) { + final String nodeId = getNodeIdForLogContext(node); + return new Runnable() { + @Override + public void run() { + try (CloseableThreadContext.Instance ignored = CloseableThreadContext.put(NODE_ID_LOG_CONTEXT_KEY, nodeId)) { + runnable.run(); + } + } + + @Override + public String toString() { + return nodeId + ": " + runnable.toString(); + } + }; + } + + static class AckCollector implements ClusterStatePublisher.AckListener { + + private final Set ackedNodes = new HashSet<>(); + private final List successfulNodes = new ArrayList<>(); + private final List unsuccessfulNodes = new ArrayList<>(); + + @Override + public void onCommit(TimeValue commitTime) { + // TODO we only currently care about per-node acks + } + + @Override + public void onNodeAck(DiscoveryNode node, Exception e) { + assertTrue("duplicate ack from " + node, ackedNodes.add(node)); + if (e == null) { + successfulNodes.add(node); + } else { + unsuccessfulNodes.add(node); + } + } + + boolean hasAckedSuccessfully(ClusterNode clusterNode) { + return successfulNodes.contains(clusterNode.localNode); + } + + boolean hasAckedUnsuccessfully(ClusterNode clusterNode) { + return unsuccessfulNodes.contains(clusterNode.localNode); + } + + boolean hasAcked(ClusterNode clusterNode) { + return ackedNodes.contains(clusterNode.localNode); + } + + int getSuccessfulAckIndex(ClusterNode clusterNode) { + assert successfulNodes.contains(clusterNode.localNode) : "get index of " + clusterNode; + return successfulNodes.indexOf(clusterNode.localNode); + } + } + + static class AckedFakeThreadPoolMasterService extends FakeThreadPoolMasterService { + + AckCollector nextAckCollector = new AckCollector(); + + AckedFakeThreadPoolMasterService(String nodeName, String serviceName, Consumer onTaskAvailableToRun) { + super(nodeName, serviceName, onTaskAvailableToRun); + } + + @Override + protected ClusterStatePublisher.AckListener wrapAckListener(ClusterStatePublisher.AckListener ackListener) { + final AckCollector ackCollector = nextAckCollector; + nextAckCollector = new AckCollector(); + return new ClusterStatePublisher.AckListener() { + @Override + public void onCommit(TimeValue commitTime) { + ackCollector.onCommit(commitTime); + ackListener.onCommit(commitTime); + } + + @Override + public void onNodeAck(DiscoveryNode node, Exception e) { + ackCollector.onNodeAck(node, e); + ackListener.onNodeAck(node, e); + } + }; + } + } + + static class DisruptableClusterApplierService extends ClusterApplierService { + private final String nodeName; + private final DeterministicTaskQueue deterministicTaskQueue; + ClusterStateApplyResponse clusterStateApplyResponse = ClusterStateApplyResponse.SUCCEED; + + DisruptableClusterApplierService(String nodeName, Settings settings, ClusterSettings clusterSettings, + DeterministicTaskQueue deterministicTaskQueue, Function runnableWrapper) { + super(nodeName, settings, clusterSettings, deterministicTaskQueue.getThreadPool(runnableWrapper)); + this.nodeName = nodeName; + this.deterministicTaskQueue = deterministicTaskQueue; + addStateApplier(event -> { + switch (clusterStateApplyResponse) { + case SUCCEED: + case HANG: + final ClusterState oldClusterState = event.previousState(); + final ClusterState newClusterState = event.state(); + assert oldClusterState.version() <= newClusterState.version() : "updating cluster state from version " + + oldClusterState.version() + " to stale version " + newClusterState.version(); + break; + case FAIL: + throw new ElasticsearchException("simulated cluster state applier failure"); + } + }); + } + + @Override + protected PrioritizedEsThreadPoolExecutor createThreadPoolExecutor() { + return new MockSinglePrioritizingExecutor(nodeName, deterministicTaskQueue); + } + + @Override + public void onNewClusterState(String source, Supplier clusterStateSupplier, ClusterApplyListener listener) { + if (clusterStateApplyResponse == ClusterStateApplyResponse.HANG) { + if (randomBoolean()) { + // apply cluster state, but don't notify listener + super.onNewClusterState(source, clusterStateSupplier, (source1, e) -> { + // ignore result + }); + } + } else { + super.onNewClusterState(source, clusterStateSupplier, listener); + } + } + + @Override + protected void connectToNodesAndWait(ClusterState newClusterState) { + // don't do anything, and don't block + } + } + + protected DiscoveryNode createDiscoveryNode(int nodeIndex, boolean masterEligible) { + final TransportAddress address = buildNewFakeTransportAddress(); + return new DiscoveryNode("", "node" + nodeIndex, + UUIDs.randomBase64UUID(random()), // generated deterministically for repeatable tests + address.address().getHostString(), address.getAddress(), address, Collections.emptyMap(), + masterEligible ? DiscoveryNodeRole.BUILT_IN_ROLES : emptySet(), Version.CURRENT); + } + + /** + * How to behave with a new cluster state + */ + enum ClusterStateApplyResponse { + /** + * Apply the state (default) + */ + SUCCEED, + + /** + * Reject the state with an exception. + */ + FAIL, + + /** + * Never respond either way. + */ + HANG, + } + + public ClusterState setValue(ClusterState clusterState, int key, long value) { + return ClusterState.builder(clusterState).metaData( + MetaData.builder(clusterState.metaData()) + .persistentSettings(Settings.builder() + .put(clusterState.metaData().persistentSettings()) + .put("value_" + key, value) + .build()) + .build()) + .build(); + } + + public long value(ClusterState clusterState) { + return value(clusterState, 0); + } + + public long value(ClusterState clusterState, int key) { + return clusterState.metaData().persistentSettings().getAsLong("value_" + key, 0L); + } + + public void assertStateEquals(ClusterState clusterState1, ClusterState clusterState2) { + assertEquals(clusterState1.version(), clusterState2.version()); + assertEquals(clusterState1.term(), clusterState2.term()); + assertEquals(keySet(clusterState1), keySet(clusterState2)); + for (int key : keySet(clusterState1)) { + assertEquals(value(clusterState1, key), value(clusterState2, key)); + } + } + + public Set keySet(ClusterState clusterState) { + return clusterState.metaData().persistentSettings().keySet().stream() + .filter(s -> s.startsWith("value_")).map(s -> Integer.valueOf(s.substring("value_".length()))).collect(Collectors.toSet()); + } + + /** + * Simple register model. Writes are modeled by providing an integer input. Reads are modeled by providing null as input. + * Responses that time out are modeled by returning null. Successful writes return the previous value of the register. + */ + private final SequentialSpec spec = new LinearizabilityChecker.KeyedSpec() { + @Override + public Object getKey(Object value) { + return ((Tuple) value).v1(); + } + + @Override + public Object getValue(Object value) { + return ((Tuple) value).v2(); + } + + @Override + public Object initialState() { + return 0L; + } + + @Override + public Optional nextState(Object currentState, Object input, Object output) { + // null input is read, non-null is write + if (input == null) { + // history is completed with null, simulating timeout, which assumes that read went through + if (output == null || currentState.equals(output)) { + return Optional.of(currentState); + } + return Optional.empty(); + } else { + if (output == null || currentState.equals(output)) { + // history is completed with null, simulating timeout, which assumes that write went through + return Optional.of(input); + } + return Optional.empty(); + } + } + }; + + public void testRegisterSpecConsistency() { + assertThat(spec.initialState(), equalTo(0L)); + assertThat(spec.nextState(7, 42, 7), equalTo(Optional.of(42))); // successful write 42 returns previous value 7 + assertThat(spec.nextState(7, 42, null), equalTo(Optional.of(42))); // write 42 times out + assertThat(spec.nextState(7, null, 7), equalTo(Optional.of(7))); // successful read + assertThat(spec.nextState(7, null, null), equalTo(Optional.of(7))); // read times out + assertThat(spec.nextState(7, null, 42), equalTo(Optional.empty())); + } + +} diff --git a/server/src/test/java/org/elasticsearch/cluster/coordination/LinearizabilityChecker.java b/test/framework/src/main/java/org/elasticsearch/cluster/coordination/LinearizabilityChecker.java similarity index 100% rename from server/src/test/java/org/elasticsearch/cluster/coordination/LinearizabilityChecker.java rename to test/framework/src/main/java/org/elasticsearch/cluster/coordination/LinearizabilityChecker.java diff --git a/server/src/test/java/org/elasticsearch/gateway/MockGatewayMetaState.java b/test/framework/src/main/java/org/elasticsearch/gateway/MockGatewayMetaState.java similarity index 95% rename from server/src/test/java/org/elasticsearch/gateway/MockGatewayMetaState.java rename to test/framework/src/main/java/org/elasticsearch/gateway/MockGatewayMetaState.java index 7749a0edc37b8..317a9d1b7ba0f 100644 --- a/server/src/test/java/org/elasticsearch/gateway/MockGatewayMetaState.java +++ b/test/framework/src/main/java/org/elasticsearch/gateway/MockGatewayMetaState.java @@ -37,8 +37,6 @@ * It's not always easy / convenient to construct these dependencies. * This class constructor takes far fewer dependencies and constructs usable {@link GatewayMetaState} with 2 restrictions: * no metadata upgrade will be performed and no cluster state updaters will be run. This is sufficient for most of the tests. - * Metadata upgrade is tested in {@link GatewayMetaStateTests} and different {@link ClusterStateUpdaters} in - * {@link ClusterStateUpdatersTests}. */ public class MockGatewayMetaState extends GatewayMetaState { private final DiscoveryNode localNode; diff --git a/server/src/test/java/org/elasticsearch/indices/cluster/FakeThreadPoolMasterService.java b/test/framework/src/main/java/org/elasticsearch/indices/cluster/FakeThreadPoolMasterService.java similarity index 100% rename from server/src/test/java/org/elasticsearch/indices/cluster/FakeThreadPoolMasterService.java rename to test/framework/src/main/java/org/elasticsearch/indices/cluster/FakeThreadPoolMasterService.java diff --git a/x-pack/plugin/voting-only-node/src/test/java/org/elasticsearch/cluster/coordination/VotingOnlyNodeCoordinatorTests.java b/x-pack/plugin/voting-only-node/src/test/java/org/elasticsearch/cluster/coordination/VotingOnlyNodeCoordinatorTests.java new file mode 100644 index 0000000000000..2c8d44f63175d --- /dev/null +++ b/x-pack/plugin/voting-only-node/src/test/java/org/elasticsearch/cluster/coordination/VotingOnlyNodeCoordinatorTests.java @@ -0,0 +1,43 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License; + * you may not use this file except in compliance with the Elastic License. + */ +package org.elasticsearch.cluster.coordination; + +import org.elasticsearch.Version; +import org.elasticsearch.cluster.node.DiscoveryNode; +import org.elasticsearch.cluster.node.DiscoveryNodeRole; +import org.elasticsearch.common.UUIDs; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.transport.TransportAddress; + +import java.util.Collections; +import java.util.Set; + +import static java.util.Collections.emptySet; + +public class VotingOnlyNodeCoordinatorTests extends AbstractCoordinatorTestCase { + + public void testDoesNotElectVotingOnlyMasterNode() { + final Cluster cluster = new Cluster(randomIntBetween(1, 5), false, Settings.EMPTY); + cluster.runRandomly(); + cluster.stabilise(); + + final Cluster.ClusterNode leader = cluster.getAnyLeader(); + assertTrue(leader.getLocalNode().isMasterNode()); + assertFalse(VotingOnlyNodePlugin.isVotingOnlyNode(leader.getLocalNode())); + } + + @Override + protected DiscoveryNode createDiscoveryNode(int nodeIndex, boolean masterEligible) { + final TransportAddress address = buildNewFakeTransportAddress(); + return new DiscoveryNode("", "node" + nodeIndex, + UUIDs.randomBase64UUID(random()), // generated deterministically for repeatable tests + address.address().getHostString(), address.getAddress(), address, Collections.emptyMap(), + masterEligible ? DiscoveryNodeRole.BUILT_IN_ROLES : + randomBoolean() ? emptySet() : Set.of(DiscoveryNodeRole.DATA_ROLE, DiscoveryNodeRole.INGEST_ROLE, + DiscoveryNodeRole.MASTER_ROLE, VotingOnlyNodePlugin.VOTING_ONLY_NODE_ROLE), Version.CURRENT); + } + +} From 77c252f6f183c16cfe99d8298e040c8a8aafb4ac Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Wed, 19 Jun 2019 20:38:43 +0200 Subject: [PATCH 10/29] rest test --- .../test/voting_only_node/10_basic.yml | 16 ++++++++++++++++ x-pack/plugin/voting-only-node/build.gradle | 3 +++ .../coordination/VotingOnlyNodePluginTests.java | 8 ++++++++ 3 files changed, 27 insertions(+) create mode 100644 x-pack/plugin/src/test/resources/rest-api-spec/test/voting_only_node/10_basic.yml diff --git a/x-pack/plugin/src/test/resources/rest-api-spec/test/voting_only_node/10_basic.yml b/x-pack/plugin/src/test/resources/rest-api-spec/test/voting_only_node/10_basic.yml new file mode 100644 index 0000000000000..bba86b2ac0cc2 --- /dev/null +++ b/x-pack/plugin/src/test/resources/rest-api-spec/test/voting_only_node/10_basic.yml @@ -0,0 +1,16 @@ +--- +"cluster stats with voting only node stats": + + - skip: + version: " - 7.99.99" + reason: "voting only nodes are added in v8.0.0" + + - do: + cluster.stats: {} + + - gte: { nodes.count.total: 1} + - gte: { nodes.count.master: 1} + - gte: { nodes.count.data: 1} + - gte: { nodes.count.ingest: 0} + - gte: { nodes.count.coordinating_only: 0} + - gte: { nodes.count.voting_only: 0} diff --git a/x-pack/plugin/voting-only-node/build.gradle b/x-pack/plugin/voting-only-node/build.gradle index ab46fe68cfdae..1c79c665f2d04 100644 --- a/x-pack/plugin/voting-only-node/build.gradle +++ b/x-pack/plugin/voting-only-node/build.gradle @@ -16,4 +16,7 @@ dependencies { } } +// xpack modules are installed in real clusters as the meta plugin, so +// installing them as individual plugins for integ tests doesn't make sense, +// so we disable integ tests integTest.enabled = false \ No newline at end of file diff --git a/x-pack/plugin/voting-only-node/src/test/java/org/elasticsearch/cluster/coordination/VotingOnlyNodePluginTests.java b/x-pack/plugin/voting-only-node/src/test/java/org/elasticsearch/cluster/coordination/VotingOnlyNodePluginTests.java index dedfa041ecf6d..de3675e0ac3ff 100644 --- a/x-pack/plugin/voting-only-node/src/test/java/org/elasticsearch/cluster/coordination/VotingOnlyNodePluginTests.java +++ b/x-pack/plugin/voting-only-node/src/test/java/org/elasticsearch/cluster/coordination/VotingOnlyNodePluginTests.java @@ -44,6 +44,14 @@ public void testOneVotingOnlyNode() throws Exception { assertNotEquals(votingOnlyNode, internalCluster().getMasterName()); } + public void testVotingOnlyNodeStats() { + internalCluster().setBootstrapMasterNodeIndex(0); + internalCluster().startNodes(2); + internalCluster().startNode(Settings.builder().put(VotingOnlyNodePlugin.VOTING_ONLY_NODE_SETTING.getKey(), true)); + assertThat(client().admin().cluster().prepareClusterStats().get().getNodesStats().getCounts().getRoles().get( + VotingOnlyNodePlugin.VOTING_ONLY_NODE_ROLE.roleName()).intValue(), equalTo(1)); + } + public void testRequireVotingOnlyNodeToBeMasterEligible() { internalCluster().setBootstrapMasterNodeIndex(0); IllegalStateException ise = expectThrows(IllegalStateException.class, () -> internalCluster().startNode(Settings.builder() From 9de65ea50aaa7bd4e8e709d871de1fcc17fcd9d4 Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Wed, 19 Jun 2019 23:03:59 +0200 Subject: [PATCH 11/29] Add x-pack feature set --- .../license/XPackLicenseState.java | 9 +++ .../xpack/core/XPackClientPlugin.java | 6 +- .../elasticsearch/xpack/core/XPackField.java | 2 + .../xpack/core/XPackSettings.java | 4 + .../core/action/XPackUsageFeatureAction.java | 3 +- .../VotingOnlyNodeFeatureSetUsage.java | 22 ++++++ .../VotingOnlyNodeFeatureSet.java | 79 +++++++++++++++++++ .../coordination/VotingOnlyNodePlugin.java | 30 +++++-- 8 files changed, 148 insertions(+), 7 deletions(-) create mode 100644 x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/votingonly/VotingOnlyNodeFeatureSetUsage.java create mode 100644 x-pack/plugin/voting-only-node/src/main/java/org/elasticsearch/cluster/coordination/VotingOnlyNodeFeatureSet.java diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/license/XPackLicenseState.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/license/XPackLicenseState.java index 32f163a773a8e..d789517d4c4ac 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/license/XPackLicenseState.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/license/XPackLicenseState.java @@ -608,6 +608,15 @@ public synchronized boolean isRollupAllowed() { return status.active; } + /** + * Voting only nodes are always available as long as there is a valid license + * + * @return true if the license is active + */ + public synchronized boolean isVotingOnlyNodeAllowed() { + return status.active; + } + /** * Logstash is allowed as long as there is an active license of type TRIAL, STANDARD, GOLD or PLATINUM * @return {@code true} as long as there is a valid license diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackClientPlugin.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackClientPlugin.java index e40d3f6b2e802..cee85ffad9350 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackClientPlugin.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackClientPlugin.java @@ -174,6 +174,7 @@ import org.elasticsearch.xpack.core.ssl.action.GetCertificateInfoAction; import org.elasticsearch.xpack.core.upgrade.actions.IndexUpgradeAction; import org.elasticsearch.xpack.core.upgrade.actions.IndexUpgradeInfoAction; +import org.elasticsearch.xpack.core.votingonly.VotingOnlyNodeFeatureSetUsage; import org.elasticsearch.xpack.core.watcher.WatcherFeatureSetUsage; import org.elasticsearch.xpack.core.watcher.WatcherMetaData; import org.elasticsearch.xpack.core.watcher.transport.actions.ack.AckWatchAction; @@ -437,7 +438,10 @@ public List getNamedWriteables() { new NamedWriteableRegistry.Entry(XPackFeatureSet.Usage.class, XPackField.DATA_FRAME, DataFrameFeatureSetUsage::new), new NamedWriteableRegistry.Entry(PersistentTaskParams.class, DataFrameField.TASK_NAME, DataFrameTransform::new), new NamedWriteableRegistry.Entry(Task.Status.class, DataFrameField.TASK_NAME, DataFrameTransformState::new), - new NamedWriteableRegistry.Entry(PersistentTaskState.class, DataFrameField.TASK_NAME, DataFrameTransformState::new) + new NamedWriteableRegistry.Entry(PersistentTaskState.class, DataFrameField.TASK_NAME, DataFrameTransformState::new), + // Voting Only Node + new NamedWriteableRegistry.Entry(XPackFeatureSet.Usage.class, XPackField.VOTING_ONLY_NODE, + VotingOnlyNodeFeatureSetUsage::new) ); } diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackField.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackField.java index a5baf4d4f9382..68e2a670e3122 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackField.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackField.java @@ -37,6 +37,8 @@ public final class XPackField { public static final String CCR = "ccr"; /** Name constant for the data frame feature. */ public static final String DATA_FRAME = "data_frame"; + /** Name constant for the voting-only-node feature. */ + public static final String VOTING_ONLY_NODE = "voting_only_node"; private XPackField() {} diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackSettings.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackSettings.java index 848aacd621111..42501124669bf 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackSettings.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackSettings.java @@ -114,6 +114,10 @@ private XPackSettings() { /** Setting for enabling or disabling sql. Defaults to true. */ public static final Setting SQL_ENABLED = Setting.boolSetting("xpack.sql.enabled", true, Setting.Property.NodeScope); + /** Setting for enabling or disabling voting-only-node. Defaults to true. */ + public static final Setting VOTING_ONLY_NODE_ENABLED = Setting.boolSetting("xpack.voting_only_node.enabled", true, + Setting.Property.NodeScope); + /* * SSL settings. These are the settings that are specifically registered for SSL. Many are private as we do not explicitly use them * but instead parse based on a prefix (eg *.ssl.*) diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/action/XPackUsageFeatureAction.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/action/XPackUsageFeatureAction.java index f5b7c4c012758..5448bdf55283c 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/action/XPackUsageFeatureAction.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/action/XPackUsageFeatureAction.java @@ -33,9 +33,10 @@ public class XPackUsageFeatureAction extends Action { public static final XPackUsageFeatureAction INDEX_LIFECYCLE = new XPackUsageFeatureAction(XPackField.INDEX_LIFECYCLE); public static final XPackUsageFeatureAction CCR = new XPackUsageFeatureAction(XPackField.CCR); public static final XPackUsageFeatureAction DATA_FRAME = new XPackUsageFeatureAction(XPackField.DATA_FRAME); + public static final XPackUsageFeatureAction VOTING_ONLY_NODE = new XPackUsageFeatureAction(XPackField.VOTING_ONLY_NODE); public static final List ALL = Arrays.asList( - SECURITY, MONITORING, WATCHER, GRAPH, MACHINE_LEARNING, LOGSTASH, SQL, ROLLUP, INDEX_LIFECYCLE, CCR, DATA_FRAME + SECURITY, MONITORING, WATCHER, GRAPH, MACHINE_LEARNING, LOGSTASH, SQL, ROLLUP, INDEX_LIFECYCLE, CCR, DATA_FRAME, VOTING_ONLY_NODE ); private XPackUsageFeatureAction(String name) { diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/votingonly/VotingOnlyNodeFeatureSetUsage.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/votingonly/VotingOnlyNodeFeatureSetUsage.java new file mode 100644 index 0000000000000..ebe2f3a580569 --- /dev/null +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/votingonly/VotingOnlyNodeFeatureSetUsage.java @@ -0,0 +1,22 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License; + * you may not use this file except in compliance with the Elastic License. + */ +package org.elasticsearch.xpack.core.votingonly; + +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.xpack.core.XPackFeatureSet; +import org.elasticsearch.xpack.core.XPackField; + +import java.io.IOException; + +public class VotingOnlyNodeFeatureSetUsage extends XPackFeatureSet.Usage { + public VotingOnlyNodeFeatureSetUsage(StreamInput input) throws IOException { + super(input); + } + + public VotingOnlyNodeFeatureSetUsage(boolean available, boolean enabled) { + super(XPackField.VOTING_ONLY_NODE, available, enabled); + } +} diff --git a/x-pack/plugin/voting-only-node/src/main/java/org/elasticsearch/cluster/coordination/VotingOnlyNodeFeatureSet.java b/x-pack/plugin/voting-only-node/src/main/java/org/elasticsearch/cluster/coordination/VotingOnlyNodeFeatureSet.java new file mode 100644 index 0000000000000..3c1f98feaac58 --- /dev/null +++ b/x-pack/plugin/voting-only-node/src/main/java/org/elasticsearch/cluster/coordination/VotingOnlyNodeFeatureSet.java @@ -0,0 +1,79 @@ +package org.elasticsearch.cluster.coordination; + +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.support.ActionFilters; +import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; +import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.common.Nullable; +import org.elasticsearch.common.inject.Inject; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.license.XPackLicenseState; +import org.elasticsearch.protocol.xpack.XPackUsageRequest; +import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.transport.TransportService; +import org.elasticsearch.xpack.core.XPackFeatureSet; +import org.elasticsearch.xpack.core.XPackField; +import org.elasticsearch.xpack.core.XPackSettings; +import org.elasticsearch.xpack.core.action.XPackUsageFeatureAction; +import org.elasticsearch.xpack.core.action.XPackUsageFeatureResponse; +import org.elasticsearch.xpack.core.action.XPackUsageFeatureTransportAction; +import org.elasticsearch.xpack.core.votingonly.VotingOnlyNodeFeatureSetUsage; + +import java.util.Map; + +public class VotingOnlyNodeFeatureSet implements XPackFeatureSet { + + private final boolean enabled; + private final XPackLicenseState licenseState; + + @Inject + public VotingOnlyNodeFeatureSet(Settings settings, @Nullable XPackLicenseState licenseState) { + this.enabled = XPackSettings.VOTING_ONLY_NODE_ENABLED.get(settings); + this.licenseState = licenseState; + } + + @Override + public String name() { + return XPackField.VOTING_ONLY_NODE; + } + + @Override + public boolean available() { + return licenseState != null && licenseState.isVotingOnlyNodeAllowed(); + } + + @Override + public boolean enabled() { + return enabled; + } + + @Override + public Map nativeCodeInfo() { + return null; + } + + public static class UsageTransportAction extends XPackUsageFeatureTransportAction { + + private final Settings settings; + private final XPackLicenseState licenseState; + + @Inject + public UsageTransportAction(TransportService transportService, ClusterService clusterService, ThreadPool threadPool, + ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver, + Settings settings, XPackLicenseState licenseState) { + super(XPackUsageFeatureAction.VOTING_ONLY_NODE.name(), transportService, clusterService, + threadPool, actionFilters, indexNameExpressionResolver); + this.settings = settings; + this.licenseState = licenseState; + } + + @Override + protected void masterOperation(XPackUsageRequest request, ClusterState state, ActionListener listener) { + final boolean available = licenseState.isVotingOnlyNodeAllowed(); + final VotingOnlyNodeFeatureSetUsage usage = + new VotingOnlyNodeFeatureSetUsage(available, XPackSettings.VOTING_ONLY_NODE_ENABLED.get(settings)); + listener.onResponse(new XPackUsageFeatureResponse(usage)); + } + } +} diff --git a/x-pack/plugin/voting-only-node/src/main/java/org/elasticsearch/cluster/coordination/VotingOnlyNodePlugin.java b/x-pack/plugin/voting-only-node/src/main/java/org/elasticsearch/cluster/coordination/VotingOnlyNodePlugin.java index f67324b332b19..0b32cf0c28e73 100644 --- a/x-pack/plugin/voting-only-node/src/main/java/org/elasticsearch/cluster/coordination/VotingOnlyNodePlugin.java +++ b/x-pack/plugin/voting-only-node/src/main/java/org/elasticsearch/cluster/coordination/VotingOnlyNodePlugin.java @@ -14,6 +14,7 @@ import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.node.DiscoveryNodeRole; import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.common.inject.Module; import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.settings.Setting; @@ -37,8 +38,11 @@ import org.elasticsearch.transport.TransportResponse; import org.elasticsearch.transport.TransportResponseHandler; import org.elasticsearch.watcher.ResourceWatcherService; +import org.elasticsearch.xpack.core.XPackPlugin; +import org.elasticsearch.xpack.core.XPackSettings; import java.io.IOException; +import java.util.ArrayList; import java.util.Collection; import java.util.Collections; import java.util.List; @@ -51,9 +55,9 @@ public class VotingOnlyNodePlugin extends Plugin implements DiscoveryPlugin, Net public static final Setting VOTING_ONLY_NODE_SETTING = Setting.boolSetting("node.voting_only", false, Setting.Property.NodeScope); - private static final String VOTING_ONLY_ELECTION_TYPE = "supports-voting-only"; + private static final String VOTING_ONLY_ELECTION_TYPE = "supports_voting_only"; - static DiscoveryNodeRole VOTING_ONLY_NODE_ROLE = new DiscoveryNodeRole("voting-only", "v") { + static DiscoveryNodeRole VOTING_ONLY_NODE_ROLE = new DiscoveryNodeRole("voting_only", "v") { @Override protected Setting roleSetting() { return VOTING_ONLY_NODE_SETTING; @@ -86,6 +90,10 @@ public List> getSettings() { @Override public Set getRoles() { + if (VOTING_ONLY_NODE_SETTING.exists(settings) && XPackSettings.VOTING_ONLY_NODE_ENABLED.get(settings) == false) { + throw new IllegalStateException(XPackSettings.VOTING_ONLY_NODE_ENABLED.getKey() + " must be set to true to use the " + + VOTING_ONLY_NODE_SETTING.getKey() + " setting"); + } if (isVotingOnlyNode && Node.NODE_MASTER_SETTING.get(settings) == false) { throw new IllegalStateException("voting-only node must be master-eligible"); } @@ -101,8 +109,18 @@ public Collection createComponents(Client client, ClusterService cluster return Collections.emptyList(); } + @Override + public Collection createGuiceModules() { + List modules = new ArrayList<>(); + modules.add(b -> XPackPlugin.bindFeatureSet(b, VotingOnlyNodeFeatureSet.class)); + return modules; + } + @Override public Map getElectionStrategies() { + if (XPackSettings.VOTING_ONLY_NODE_ENABLED.get(settings) == false) { + return Collections.emptyMap(); + } return Collections.singletonMap(VOTING_ONLY_ELECTION_TYPE, new VotingOnlyNodeElectionStrategy()); } @@ -122,6 +140,9 @@ public AsyncSender interceptSender(AsyncSender sender) { @Override public Settings additionalSettings() { + if (XPackSettings.VOTING_ONLY_NODE_ENABLED.get(settings) == false) { + return Settings.EMPTY; + } return Settings.builder().put(DiscoveryModule.ELECTION_TYPE_SETTING.getKey(), VOTING_ONLY_ELECTION_TYPE).build(); } @@ -189,14 +210,13 @@ public TransportResponse read(StreamInput in) throws IOException { return handler.read(in); } }); - return; } else { threadPoolSupplier.get().generic().execute(() -> handler.handleException(new TransportException( new ElasticsearchException("voting-only node skipping publication to [" + destinationNode + "]")))); - return; } + } else { + sender.sendRequest(connection, action, request, options, handler); } - sender.sendRequest(connection, action, request, options, handler); } } } From 03d249fdb8d81c5d32aaaa16b8abf8f95390d3b1 Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Wed, 19 Jun 2019 23:16:13 +0200 Subject: [PATCH 12/29] add docs --- docs/reference/cluster.asciidoc | 14 +++++++------ docs/reference/modules/node.asciidoc | 31 ++++++++++++++++++++++++++++ 2 files changed, 39 insertions(+), 6 deletions(-) diff --git a/docs/reference/cluster.asciidoc b/docs/reference/cluster.asciidoc index cfa2d5a6488d7..4727067f78b3c 100644 --- a/docs/reference/cluster.asciidoc +++ b/docs/reference/cluster.asciidoc @@ -22,12 +22,14 @@ one of the following: * an IP address or hostname, to add all matching nodes to the subset. * a pattern, using `*` wildcards, which adds all nodes to the subset whose name, address or hostname matches the pattern. -* `master:true`, `data:true`, `ingest:true` or `coordinating_only:true`, which - respectively add to the subset all master-eligible nodes, all data nodes, - all ingest nodes, and all coordinating-only nodes. -* `master:false`, `data:false`, `ingest:false` or `coordinating_only:false`, - which respectively remove from the subset all master-eligible nodes, all data - nodes, all ingest nodes, and all coordinating-only nodes. +* `master:true`, `data:true`, `ingest:true`, `voting_only:true` or + `coordinating_only:true`, which respectively add to the subset all + master-eligible nodes, all data nodes, all ingest nodes, all voting-only + nodes, and all coordinating-only nodes. +* `master:false`, `data:false`, `ingest:false`, `voting_only:true`, or + `coordinating_only:false`, which respectively remove from the subset all + master-eligible nodes, all data nodes, all ingest nodes, all voting-only + nodes and all coordinating-only nodes. * a pair of patterns, using `*` wildcards, of the form `attrname:attrvalue`, which adds to the subset all nodes with a custom node attribute whose name and value match the respective patterns. Custom node attributes are diff --git a/docs/reference/modules/node.asciidoc b/docs/reference/modules/node.asciidoc index 6a097bffd7031..bc9606d978275 100644 --- a/docs/reference/modules/node.asciidoc +++ b/docs/reference/modules/node.asciidoc @@ -134,6 +134,37 @@ cluster.remote.connect: false <4> <3> Disable the `node.ingest` role (enabled by default). <4> Disable {ccs} (enabled by default). +[float] +[[voting-only-node]] +==== Voting-only master-eligible node + +A voting-only master-eligible node is a node that can participate in master +elections but will not act as a master in the cluster. In particular, a +voting-only node can help elect another master-eligible node as master, and +can serve as a tiebreaker in elections. To mark a master-eligible node as +voting-only, set: + +[source,yaml] +------------------- +node.voting_only: true <1> +------------------- +<1> The `node.voting_only` role is disabled by default. + +IMPORTANT: If you use the {oss-dist}, do not set `node.voting_only`. Otherwise, +the node fails to start. Also note that only master-eligible nodes can be +marked as voting-only. + +High availability (HA) clusters require at least three master-eligible nodes, +so that if one of the three nodes is down, then the remaining two can still +elect a master amongst them-selves. This only requires one of the two remaining +nodes to have the capability to act as master, but both need to have voting +powers. This means that one of the three master-eligible nodes can be made as +voting-only. If this voting-only node is a dedicated master, a less powerful +machine or a smaller heap-size can be chosen for this node. Alternatively, +a voting-only non-dedicated master node can play the role of the third +master-eligible node, which allows running an HA cluster with only two +dedicated master nodes. + [float] [[data-node]] === Data Node From b170c9ddf5b1cad7405937306a06c52f416b5e13 Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Wed, 19 Jun 2019 23:47:37 +0200 Subject: [PATCH 13/29] fixup --- .../core/LocalStateCompositeXPackPlugin.java | 19 +++++++++++++ .../VotingOnlyNodeFeatureSet.java | 5 ++++ .../LocalStateVotingOnlyNodePlugin.java | 27 +++++++++++++++++++ .../VotingOnlyNodePluginTests.java | 6 +++-- 4 files changed, 55 insertions(+), 2 deletions(-) create mode 100644 x-pack/plugin/voting-only-node/src/test/java/org/elasticsearch/cluster/coordination/LocalStateVotingOnlyNodePlugin.java diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/LocalStateCompositeXPackPlugin.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/LocalStateCompositeXPackPlugin.java index 2157cbd38cad3..350840883a7fe 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/LocalStateCompositeXPackPlugin.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/LocalStateCompositeXPackPlugin.java @@ -14,9 +14,11 @@ import org.elasticsearch.bootstrap.BootstrapCheck; import org.elasticsearch.client.Client; import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.coordination.ElectionStrategy; import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; import org.elasticsearch.cluster.metadata.IndexTemplateMetaData; import org.elasticsearch.cluster.node.DiscoveryNode; +import org.elasticsearch.cluster.node.DiscoveryNodeRole; import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.inject.Module; @@ -76,10 +78,12 @@ import java.util.Collection; import java.util.Collections; import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Objects; import java.util.Optional; +import java.util.Set; import java.util.function.BiConsumer; import java.util.function.Function; import java.util.function.Predicate; @@ -333,6 +337,7 @@ public List> getExecutorBuilders(final Settings settings) { filterPlugins(Plugin.class).stream().forEach(p -> builders.addAll(p.getExecutorBuilders(settings))); return builders; } + @Override public UnaryOperator> getIndexTemplateMetaDataUpgrader() { return templates -> { @@ -343,6 +348,20 @@ public UnaryOperator> getIndexTemplateMetaDat }; } + @Override + public Map getElectionStrategies() { + Map electionStrategies = new HashMap<>(); + filterPlugins(DiscoveryPlugin.class).stream().forEach(p -> electionStrategies.putAll(p.getElectionStrategies())); + return electionStrategies; + } + + @Override + public Set getRoles() { + Set roles = new HashSet<>(); + filterPlugins(Plugin.class).stream().forEach(p -> roles.addAll(p.getRoles())); + return roles; + } + @Override public Map> getTokenizers() { Map> tokenizers = new HashMap<>(); diff --git a/x-pack/plugin/voting-only-node/src/main/java/org/elasticsearch/cluster/coordination/VotingOnlyNodeFeatureSet.java b/x-pack/plugin/voting-only-node/src/main/java/org/elasticsearch/cluster/coordination/VotingOnlyNodeFeatureSet.java index 3c1f98feaac58..eb127d732fef8 100644 --- a/x-pack/plugin/voting-only-node/src/main/java/org/elasticsearch/cluster/coordination/VotingOnlyNodeFeatureSet.java +++ b/x-pack/plugin/voting-only-node/src/main/java/org/elasticsearch/cluster/coordination/VotingOnlyNodeFeatureSet.java @@ -1,3 +1,8 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License; + * you may not use this file except in compliance with the Elastic License. + */ package org.elasticsearch.cluster.coordination; import org.elasticsearch.action.ActionListener; diff --git a/x-pack/plugin/voting-only-node/src/test/java/org/elasticsearch/cluster/coordination/LocalStateVotingOnlyNodePlugin.java b/x-pack/plugin/voting-only-node/src/test/java/org/elasticsearch/cluster/coordination/LocalStateVotingOnlyNodePlugin.java new file mode 100644 index 0000000000000..9e7fafc2499eb --- /dev/null +++ b/x-pack/plugin/voting-only-node/src/test/java/org/elasticsearch/cluster/coordination/LocalStateVotingOnlyNodePlugin.java @@ -0,0 +1,27 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License; + * you may not use this file except in compliance with the Elastic License. + */ +package org.elasticsearch.cluster.coordination; + +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.license.XPackLicenseState; +import org.elasticsearch.xpack.core.LocalStateCompositeXPackPlugin; + +import java.nio.file.Path; + +public class LocalStateVotingOnlyNodePlugin extends LocalStateCompositeXPackPlugin { + + public LocalStateVotingOnlyNodePlugin(final Settings settings, final Path configPath) throws Exception { + super(settings, configPath); + @SuppressWarnings("resource") + LocalStateVotingOnlyNodePlugin thisVar = this; + + plugins.add(new VotingOnlyNodePlugin(settings) { + protected XPackLicenseState getLicenseState() { + return LocalStateVotingOnlyNodePlugin.this.getLicenseState(); + } + }); + } +} diff --git a/x-pack/plugin/voting-only-node/src/test/java/org/elasticsearch/cluster/coordination/VotingOnlyNodePluginTests.java b/x-pack/plugin/voting-only-node/src/test/java/org/elasticsearch/cluster/coordination/VotingOnlyNodePluginTests.java index de3675e0ac3ff..542e93df0be12 100644 --- a/x-pack/plugin/voting-only-node/src/test/java/org/elasticsearch/cluster/coordination/VotingOnlyNodePluginTests.java +++ b/x-pack/plugin/voting-only-node/src/test/java/org/elasticsearch/cluster/coordination/VotingOnlyNodePluginTests.java @@ -28,7 +28,7 @@ public class VotingOnlyNodePluginTests extends ESIntegTestCase { @Override protected Collection> nodePlugins() { - return Collections.singleton(VotingOnlyNodePlugin.class); + return Collections.singleton(LocalStateVotingOnlyNodePlugin.class); } public void testOneVotingOnlyNode() throws Exception { @@ -44,10 +44,12 @@ public void testOneVotingOnlyNode() throws Exception { assertNotEquals(votingOnlyNode, internalCluster().getMasterName()); } - public void testVotingOnlyNodeStats() { + public void testVotingOnlyNodeStats() throws Exception { internalCluster().setBootstrapMasterNodeIndex(0); internalCluster().startNodes(2); internalCluster().startNode(Settings.builder().put(VotingOnlyNodePlugin.VOTING_ONLY_NODE_SETTING.getKey(), true)); + assertBusy(() -> assertThat(client().admin().cluster().prepareState().get().getState().getLastCommittedConfiguration().getNodeIds(), + hasSize(3))); assertThat(client().admin().cluster().prepareClusterStats().get().getNodesStats().getCounts().getRoles().get( VotingOnlyNodePlugin.VOTING_ONLY_NODE_ROLE.roleName()).intValue(), equalTo(1)); } From c11b2e642f6ace6b7fef4a29c093867e3b76d5ea Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Thu, 20 Jun 2019 00:15:46 +0200 Subject: [PATCH 14/29] fix docs tests --- docs/reference/cluster/stats.asciidoc | 3 ++- docs/reference/rest-api/info.asciidoc | 4 ++++ 2 files changed, 6 insertions(+), 1 deletion(-) diff --git a/docs/reference/cluster/stats.asciidoc b/docs/reference/cluster/stats.asciidoc index 791fc2414f331..b671a15348ff6 100644 --- a/docs/reference/cluster/stats.asciidoc +++ b/docs/reference/cluster/stats.asciidoc @@ -109,7 +109,8 @@ Will return, for example: "data": 1, "coordinating_only": 0, "master": 1, - "ingest": 1 + "ingest": 1, + "voting_only": 0 }, "versions": [ "{version}" diff --git a/docs/reference/rest-api/info.asciidoc b/docs/reference/rest-api/info.asciidoc index db1add3d0310f..89144b4982589 100644 --- a/docs/reference/rest-api/info.asciidoc +++ b/docs/reference/rest-api/info.asciidoc @@ -107,6 +107,10 @@ Example response: "available" : true, "enabled" : true }, + "voting_only_node" : { + "available" : true, + "enabled" : true + }, "watcher" : { "available" : true, "enabled" : true From d10968635bb1f23df40f91cbeb72006d1f34fd79 Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Thu, 20 Jun 2019 08:58:04 +0200 Subject: [PATCH 15/29] Register usage action --- .../cluster/coordination/VotingOnlyNodePlugin.java | 14 +++++++++++++- 1 file changed, 13 insertions(+), 1 deletion(-) diff --git a/x-pack/plugin/voting-only-node/src/main/java/org/elasticsearch/cluster/coordination/VotingOnlyNodePlugin.java b/x-pack/plugin/voting-only-node/src/main/java/org/elasticsearch/cluster/coordination/VotingOnlyNodePlugin.java index 0b32cf0c28e73..43df24e2ecfcb 100644 --- a/x-pack/plugin/voting-only-node/src/main/java/org/elasticsearch/cluster/coordination/VotingOnlyNodePlugin.java +++ b/x-pack/plugin/voting-only-node/src/main/java/org/elasticsearch/cluster/coordination/VotingOnlyNodePlugin.java @@ -7,10 +7,13 @@ import org.apache.lucene.util.SetOnce; import org.elasticsearch.ElasticsearchException; +import org.elasticsearch.action.ActionRequest; +import org.elasticsearch.action.ActionResponse; import org.elasticsearch.client.Client; import org.elasticsearch.cluster.coordination.CoordinationMetaData.VotingConfiguration; import org.elasticsearch.cluster.coordination.CoordinationState.JoinVoteCollection; import org.elasticsearch.cluster.coordination.CoordinationState.VoteCollection; +import org.elasticsearch.cluster.coordination.VotingOnlyNodeFeatureSet.UsageTransportAction; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.node.DiscoveryNodeRole; import org.elasticsearch.cluster.service.ClusterService; @@ -25,6 +28,7 @@ import org.elasticsearch.env.Environment; import org.elasticsearch.env.NodeEnvironment; import org.elasticsearch.node.Node; +import org.elasticsearch.plugins.ActionPlugin; import org.elasticsearch.plugins.DiscoveryPlugin; import org.elasticsearch.plugins.NetworkPlugin; import org.elasticsearch.plugins.Plugin; @@ -40,6 +44,7 @@ import org.elasticsearch.watcher.ResourceWatcherService; import org.elasticsearch.xpack.core.XPackPlugin; import org.elasticsearch.xpack.core.XPackSettings; +import org.elasticsearch.xpack.core.action.XPackUsageFeatureAction; import java.io.IOException; import java.util.ArrayList; @@ -50,7 +55,9 @@ import java.util.Set; import java.util.function.Supplier; -public class VotingOnlyNodePlugin extends Plugin implements DiscoveryPlugin, NetworkPlugin { +import static java.util.Collections.singletonList; + +public class VotingOnlyNodePlugin extends Plugin implements DiscoveryPlugin, NetworkPlugin, ActionPlugin { public static final Setting VOTING_ONLY_NODE_SETTING = Setting.boolSetting("node.voting_only", false, Setting.Property.NodeScope); @@ -109,6 +116,11 @@ public Collection createComponents(Client client, ClusterService cluster return Collections.emptyList(); } + @Override + public List> getActions() { + return singletonList(new ActionHandler<>(XPackUsageFeatureAction.VOTING_ONLY_NODE, UsageTransportAction.class)); + } + @Override public Collection createGuiceModules() { List modules = new ArrayList<>(); From e41012187427b9361c2c61589046e1fd0396b7df Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Thu, 20 Jun 2019 09:28:06 +0200 Subject: [PATCH 16/29] more fixups --- .../xpack/core/XPackSettings.java | 1 + .../VotingOnlyNodePluginTests.java | 36 +++++++++---------- 2 files changed, 17 insertions(+), 20 deletions(-) diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackSettings.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackSettings.java index 42501124669bf..a9973833195af 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackSettings.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackSettings.java @@ -208,6 +208,7 @@ public static List> getAllSettings() { settings.add(PASSWORD_HASHING_ALGORITHM); settings.add(INDEX_LIFECYCLE_ENABLED); settings.add(DATA_FRAME_ENABLED); + settings.add(VOTING_ONLY_NODE_ENABLED); return Collections.unmodifiableList(settings); } diff --git a/x-pack/plugin/voting-only-node/src/test/java/org/elasticsearch/cluster/coordination/VotingOnlyNodePluginTests.java b/x-pack/plugin/voting-only-node/src/test/java/org/elasticsearch/cluster/coordination/VotingOnlyNodePluginTests.java index 542e93df0be12..1c44fec64c8f5 100644 --- a/x-pack/plugin/voting-only-node/src/test/java/org/elasticsearch/cluster/coordination/VotingOnlyNodePluginTests.java +++ b/x-pack/plugin/voting-only-node/src/test/java/org/elasticsearch/cluster/coordination/VotingOnlyNodePluginTests.java @@ -12,7 +12,7 @@ import org.elasticsearch.plugins.Plugin; import org.elasticsearch.test.ESIntegTestCase; import org.elasticsearch.test.ESIntegTestCase.Scope; -import org.elasticsearch.test.junit.annotations.TestLogging; +import org.elasticsearch.xpack.core.XPackSettings; import java.util.Collection; import java.util.Collections; @@ -23,7 +23,6 @@ import static org.hamcrest.Matchers.nullValue; @ESIntegTestCase.ClusterScope(scope = Scope.TEST, numDataNodes = 0, autoManageMasterNodes = false) -@TestLogging("_root:DEBUG,org.elasticsearch.cluster:TRACE,org.elasticsearch.discovery:TRACE") public class VotingOnlyNodePluginTests extends ESIntegTestCase { @Override @@ -31,17 +30,23 @@ protected Collection> nodePlugins() { return Collections.singleton(LocalStateVotingOnlyNodePlugin.class); } - public void testOneVotingOnlyNode() throws Exception { + public void testRequireVotingOnlyNodeToBeMasterEligible() { internalCluster().setBootstrapMasterNodeIndex(0); - internalCluster().startNodes(2); - final String votingOnlyNode - = internalCluster().startNode(Settings.builder().put(VotingOnlyNodePlugin.VOTING_ONLY_NODE_SETTING.getKey(), true)); - assertBusy(() -> assertThat(client().admin().cluster().prepareState().get().getState().getLastCommittedConfiguration().getNodeIds(), - hasSize(3))); - - internalCluster().stopCurrentMasterNode(); + IllegalStateException ise = expectThrows(IllegalStateException.class, () -> internalCluster().startNode(Settings.builder() + .put(Node.NODE_MASTER_SETTING.getKey(), false) + .put(VotingOnlyNodePlugin.VOTING_ONLY_NODE_SETTING.getKey(), true) + .build())); + assertThat(ise.getMessage(), containsString("voting-only node must be master-eligible")); + } - assertNotEquals(votingOnlyNode, internalCluster().getMasterName()); + public void testRequireVotingOnlyNodeToHaveXPackSettingEnabled() { + internalCluster().setBootstrapMasterNodeIndex(0); + IllegalStateException ise = expectThrows(IllegalStateException.class, () -> internalCluster().startNode(Settings.builder() + .put(VotingOnlyNodePlugin.VOTING_ONLY_NODE_SETTING.getKey(), true) + .put(XPackSettings.VOTING_ONLY_NODE_ENABLED.getKey(), false) + .build())); + assertThat(ise.getMessage(), containsString(XPackSettings.VOTING_ONLY_NODE_ENABLED.getKey() + " must be set to true to use the " + + VotingOnlyNodePlugin.VOTING_ONLY_NODE_SETTING.getKey() + " setting")); } public void testVotingOnlyNodeStats() throws Exception { @@ -54,15 +59,6 @@ public void testVotingOnlyNodeStats() throws Exception { VotingOnlyNodePlugin.VOTING_ONLY_NODE_ROLE.roleName()).intValue(), equalTo(1)); } - public void testRequireVotingOnlyNodeToBeMasterEligible() { - internalCluster().setBootstrapMasterNodeIndex(0); - IllegalStateException ise = expectThrows(IllegalStateException.class, () -> internalCluster().startNode(Settings.builder() - .put(Node.NODE_MASTER_SETTING.getKey(), false) - .put(VotingOnlyNodePlugin.VOTING_ONLY_NODE_SETTING.getKey(), true) - .build())); - assertThat(ise.getMessage(), containsString("voting-only node must be master-eligible")); - } - public void testPreferFullMasterOverVotingOnlyNodes() throws Exception { internalCluster().setBootstrapMasterNodeIndex(0); internalCluster().startNodes(2); From 5e260d5d0a7e6bb0c330a7762145d8482452d91c Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Thu, 20 Jun 2019 09:56:01 +0200 Subject: [PATCH 17/29] more fixes --- .../cluster/node/DiscoveryNodes.java | 13 ++++++++++ .../cluster/node/DiscoveryNodesTests.java | 24 +++++++++++++++++-- .../xpack/core/XPackSettings.java | 16 +++++-------- .../VotingOnlyNodeFeatureSet.java | 9 ++----- .../coordination/VotingOnlyNodePlugin.java | 8 +++---- .../VotingOnlyNodePluginTests.java | 6 +++-- 6 files changed, 51 insertions(+), 25 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/cluster/node/DiscoveryNodes.java b/server/src/main/java/org/elasticsearch/cluster/node/DiscoveryNodes.java index 57eb2ac7541dc..57ceb6da47c0c 100644 --- a/server/src/main/java/org/elasticsearch/cluster/node/DiscoveryNodes.java +++ b/server/src/main/java/org/elasticsearch/cluster/node/DiscoveryNodes.java @@ -32,6 +32,7 @@ import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.regex.Regex; import org.elasticsearch.common.transport.TransportAddress; +import org.elasticsearch.common.util.set.Sets; import java.io.IOException; import java.util.ArrayList; @@ -40,6 +41,7 @@ import java.util.List; import java.util.Map; import java.util.Objects; +import java.util.Set; import java.util.stream.Stream; import java.util.stream.StreamSupport; @@ -378,6 +380,17 @@ public String[] resolveNodes(String... nodes) { resolvedNodesIds.removeAll(getCoordinatingOnlyNodes().keys()); } } else { + for (DiscoveryNode node : this) { + for (DiscoveryNodeRole role : Sets.difference(node.getRoles(), DiscoveryNodeRole.BUILT_IN_ROLES)) { + if (role.roleName().equals(matchAttrName)) { + if (Booleans.parseBoolean(matchAttrValue, true)) { + resolvedNodesIds.add(node.getId()); + } else { + resolvedNodesIds.remove(node.getId()); + } + } + } + } for (DiscoveryNode node : this) { for (Map.Entry entry : node.getAttributes().entrySet()) { String attrName = entry.getKey(); diff --git a/server/src/test/java/org/elasticsearch/cluster/node/DiscoveryNodesTests.java b/server/src/test/java/org/elasticsearch/cluster/node/DiscoveryNodesTests.java index 9a5186618b333..2cdbd95fc6358 100644 --- a/server/src/test/java/org/elasticsearch/cluster/node/DiscoveryNodesTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/node/DiscoveryNodesTests.java @@ -21,6 +21,7 @@ import com.carrotsearch.randomizedtesting.generators.RandomPicks; import org.elasticsearch.Version; +import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.transport.TransportAddress; import org.elasticsearch.test.ESTestCase; @@ -240,8 +241,16 @@ private static List randomNodes(final int numNodes) { if (frequently()) { attributes.put("custom", randomBoolean() ? "match" : randomAlphaOfLengthBetween(3, 5)); } - final DiscoveryNode node = newNode(idGenerator.getAndIncrement(), attributes, - new HashSet<>(randomSubsetOf(DiscoveryNodeRole.BUILT_IN_ROLES))); + final Set roles = new HashSet<>(randomSubsetOf(DiscoveryNodeRole.BUILT_IN_ROLES)); + if (frequently()) { + roles.add(new DiscoveryNodeRole("custom_role", "cr") { + @Override + protected Setting roleSetting() { + return null; + } + }); + } + final DiscoveryNode node = newNode(idGenerator.getAndIncrement(), attributes, roles); nodesList.add(node); } return nodesList; @@ -314,6 +323,17 @@ Set matchingNodeIds(DiscoveryNodes nodes) { }); return ids; } + }, CUSTOM_ROLE("custom_role:true") { + @Override + Set matchingNodeIds(DiscoveryNodes nodes) { + Set ids = new HashSet<>(); + nodes.getNodes().valuesIt().forEachRemaining(node -> { + if (node.getRoles().stream().anyMatch(role -> role.roleName().equals("custom_role"))) { + ids.add(node.getId()); + } + }); + return ids; + } }; private final String selector; diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackSettings.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackSettings.java index 738d20d96de85..8d4f636f9e160 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackSettings.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackSettings.java @@ -114,14 +114,13 @@ private XPackSettings() { /** Setting for enabling or disabling sql. Defaults to true. */ public static final Setting SQL_ENABLED = Setting.boolSetting("xpack.sql.enabled", true, Setting.Property.NodeScope); -<<<<<<< HEAD - /** Setting for enabling or disabling voting-only-node. Defaults to true. */ - public static final Setting VOTING_ONLY_NODE_ENABLED = Setting.boolSetting("xpack.voting_only_node.enabled", true, - Setting.Property.NodeScope); -======= /** Setting for enabling or disabling vectors. Defaults to true. */ public static final Setting VECTORS_ENABLED = Setting.boolSetting("xpack.vectors.enabled", true, Setting.Property.NodeScope); ->>>>>>> elastic/master + + /** Setting for enabling or disabling the voting-only-node functionality. Needs to be enabled on both voting-only nodes and regular + * master-eligible nodes for the voting-only functionality to work correctly. Defaults to true. */ + public static final Setting VOTING_ONLY_ENABLED = Setting.boolSetting("xpack.voting_only.enabled", true, + Setting.Property.NodeScope); /* * SSL settings. These are the settings that are specifically registered for SSL. Many are private as we do not explicitly use them @@ -213,11 +212,8 @@ public static List> getAllSettings() { settings.add(PASSWORD_HASHING_ALGORITHM); settings.add(INDEX_LIFECYCLE_ENABLED); settings.add(DATA_FRAME_ENABLED); -<<<<<<< HEAD - settings.add(VOTING_ONLY_NODE_ENABLED); -======= settings.add(VECTORS_ENABLED); ->>>>>>> elastic/master + settings.add(VOTING_ONLY_ENABLED); return Collections.unmodifiableList(settings); } diff --git a/x-pack/plugin/voting-only-node/src/main/java/org/elasticsearch/cluster/coordination/VotingOnlyNodeFeatureSet.java b/x-pack/plugin/voting-only-node/src/main/java/org/elasticsearch/cluster/coordination/VotingOnlyNodeFeatureSet.java index a5c2530eff0a1..0bc26bbfc029b 100644 --- a/x-pack/plugin/voting-only-node/src/main/java/org/elasticsearch/cluster/coordination/VotingOnlyNodeFeatureSet.java +++ b/x-pack/plugin/voting-only-node/src/main/java/org/elasticsearch/cluster/coordination/VotingOnlyNodeFeatureSet.java @@ -34,7 +34,7 @@ public class VotingOnlyNodeFeatureSet implements XPackFeatureSet { @Inject public VotingOnlyNodeFeatureSet(Settings settings, @Nullable XPackLicenseState licenseState) { - this.enabled = XPackSettings.VOTING_ONLY_NODE_ENABLED.get(settings); + this.enabled = XPackSettings.VOTING_ONLY_ENABLED.get(settings); this.licenseState = licenseState; } @@ -53,11 +53,6 @@ public boolean enabled() { return enabled; } - @Override - public Map nativeCodeInfo() { - return null; - } - public static class UsageTransportAction extends XPackUsageFeatureTransportAction { private final Settings settings; @@ -77,7 +72,7 @@ public UsageTransportAction(TransportService transportService, ClusterService cl protected void masterOperation(XPackUsageRequest request, ClusterState state, ActionListener listener) { final boolean available = licenseState.isVotingOnlyNodeAllowed(); final VotingOnlyNodeFeatureSetUsage usage = - new VotingOnlyNodeFeatureSetUsage(available, XPackSettings.VOTING_ONLY_NODE_ENABLED.get(settings)); + new VotingOnlyNodeFeatureSetUsage(available, XPackSettings.VOTING_ONLY_ENABLED.get(settings)); listener.onResponse(new XPackUsageFeatureResponse(usage)); } } diff --git a/x-pack/plugin/voting-only-node/src/main/java/org/elasticsearch/cluster/coordination/VotingOnlyNodePlugin.java b/x-pack/plugin/voting-only-node/src/main/java/org/elasticsearch/cluster/coordination/VotingOnlyNodePlugin.java index 43df24e2ecfcb..c2a1c00b1b707 100644 --- a/x-pack/plugin/voting-only-node/src/main/java/org/elasticsearch/cluster/coordination/VotingOnlyNodePlugin.java +++ b/x-pack/plugin/voting-only-node/src/main/java/org/elasticsearch/cluster/coordination/VotingOnlyNodePlugin.java @@ -97,8 +97,8 @@ public List> getSettings() { @Override public Set getRoles() { - if (VOTING_ONLY_NODE_SETTING.exists(settings) && XPackSettings.VOTING_ONLY_NODE_ENABLED.get(settings) == false) { - throw new IllegalStateException(XPackSettings.VOTING_ONLY_NODE_ENABLED.getKey() + " must be set to true to use the " + + if (VOTING_ONLY_NODE_SETTING.exists(settings) && XPackSettings.VOTING_ONLY_ENABLED.get(settings) == false) { + throw new IllegalStateException(XPackSettings.VOTING_ONLY_ENABLED.getKey() + " must be set to true to use the " + VOTING_ONLY_NODE_SETTING.getKey() + " setting"); } if (isVotingOnlyNode && Node.NODE_MASTER_SETTING.get(settings) == false) { @@ -130,7 +130,7 @@ public Collection createGuiceModules() { @Override public Map getElectionStrategies() { - if (XPackSettings.VOTING_ONLY_NODE_ENABLED.get(settings) == false) { + if (XPackSettings.VOTING_ONLY_ENABLED.get(settings) == false) { return Collections.emptyMap(); } return Collections.singletonMap(VOTING_ONLY_ELECTION_TYPE, new VotingOnlyNodeElectionStrategy()); @@ -152,7 +152,7 @@ public AsyncSender interceptSender(AsyncSender sender) { @Override public Settings additionalSettings() { - if (XPackSettings.VOTING_ONLY_NODE_ENABLED.get(settings) == false) { + if (XPackSettings.VOTING_ONLY_ENABLED.get(settings) == false) { return Settings.EMPTY; } return Settings.builder().put(DiscoveryModule.ELECTION_TYPE_SETTING.getKey(), VOTING_ONLY_ELECTION_TYPE).build(); diff --git a/x-pack/plugin/voting-only-node/src/test/java/org/elasticsearch/cluster/coordination/VotingOnlyNodePluginTests.java b/x-pack/plugin/voting-only-node/src/test/java/org/elasticsearch/cluster/coordination/VotingOnlyNodePluginTests.java index 1c44fec64c8f5..903606606e6f8 100644 --- a/x-pack/plugin/voting-only-node/src/test/java/org/elasticsearch/cluster/coordination/VotingOnlyNodePluginTests.java +++ b/x-pack/plugin/voting-only-node/src/test/java/org/elasticsearch/cluster/coordination/VotingOnlyNodePluginTests.java @@ -43,9 +43,9 @@ public void testRequireVotingOnlyNodeToHaveXPackSettingEnabled() { internalCluster().setBootstrapMasterNodeIndex(0); IllegalStateException ise = expectThrows(IllegalStateException.class, () -> internalCluster().startNode(Settings.builder() .put(VotingOnlyNodePlugin.VOTING_ONLY_NODE_SETTING.getKey(), true) - .put(XPackSettings.VOTING_ONLY_NODE_ENABLED.getKey(), false) + .put(XPackSettings.VOTING_ONLY_ENABLED.getKey(), false) .build())); - assertThat(ise.getMessage(), containsString(XPackSettings.VOTING_ONLY_NODE_ENABLED.getKey() + " must be set to true to use the " + + assertThat(ise.getMessage(), containsString(XPackSettings.VOTING_ONLY_ENABLED.getKey() + " must be set to true to use the " + VotingOnlyNodePlugin.VOTING_ONLY_NODE_SETTING.getKey() + " setting")); } @@ -57,6 +57,8 @@ public void testVotingOnlyNodeStats() throws Exception { hasSize(3))); assertThat(client().admin().cluster().prepareClusterStats().get().getNodesStats().getCounts().getRoles().get( VotingOnlyNodePlugin.VOTING_ONLY_NODE_ROLE.roleName()).intValue(), equalTo(1)); + assertThat(client().admin().cluster().prepareNodesStats("voting_only:true").get().getNodes(), hasSize(1)); + assertThat(client().admin().cluster().prepareNodesStats("master:true", "voting_only:false").get().getNodes(), hasSize(2)); } public void testPreferFullMasterOverVotingOnlyNodes() throws Exception { From 65fda75b8bafc143f6277b8b394d8aa5e5100946 Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Thu, 20 Jun 2019 10:26:40 +0200 Subject: [PATCH 18/29] More fixups --- .../cluster/coordination/CoordinationState.java | 11 ++++++++++- .../cluster/coordination/ElectionStrategy.java | 6 +++++- .../cluster/coordination/PreVoteCollector.java | 11 +++++++---- .../org/elasticsearch/cluster/node/DiscoveryNode.java | 2 +- .../elasticsearch/cluster/node/DiscoveryNodes.java | 1 - .../cluster/coordination/CoordinationStateTests.java | 4 ++++ .../org/elasticsearch/license/XPackLicenseState.java | 4 ++-- .../coordination/VotingOnlyNodeFeatureSet.java | 6 ++---- .../cluster/coordination/VotingOnlyNodePlugin.java | 6 +++--- .../coordination/LocalStateVotingOnlyNodePlugin.java | 9 +-------- 10 files changed, 35 insertions(+), 25 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/cluster/coordination/CoordinationState.java b/server/src/main/java/org/elasticsearch/cluster/coordination/CoordinationState.java index a443c5670c0e2..9eb22f491c825 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/CoordinationState.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/CoordinationState.java @@ -43,9 +43,10 @@ public class CoordinationState { private final DiscoveryNode localNode; + private final ElectionStrategy electionStrategy; + // persisted state private final PersistedState persistedState; - private final ElectionStrategy electionStrategy; // transient state private JoinVoteCollection joinVotes; @@ -116,6 +117,11 @@ public boolean containsJoinVoteFor(DiscoveryNode node) { return joinVotes.containsVoteFor(node); } + // used for tests + boolean containsJoin(Join join) { + return joinVotes.getJoins().contains(join); + } + public boolean joinVotesHaveQuorumFor(VotingConfiguration votingConfiguration) { return joinVotes.isQuorum(votingConfiguration); } @@ -539,6 +545,9 @@ public int hashCode() { } } + /** + * A collection of votes, extending {@link VoteCollection}, which additionally records the Joins + */ public static class JoinVoteCollection extends VoteCollection { private final Set joins = new HashSet<>(); diff --git a/server/src/main/java/org/elasticsearch/cluster/coordination/ElectionStrategy.java b/server/src/main/java/org/elasticsearch/cluster/coordination/ElectionStrategy.java index 2d392a8e5c977..e94f7f8991ccb 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/ElectionStrategy.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/ElectionStrategy.java @@ -22,10 +22,14 @@ import org.elasticsearch.cluster.coordination.CoordinationState.VoteCollection; import org.elasticsearch.cluster.node.DiscoveryNode; +/** + * Allows plugging in a custom election strategy. Note that, in order to guarantee safety of the system, custom election strategies should + * only be more restrictive than the behavior that's provided by {@link DefaultElectionStrategy}. + */ public interface ElectionStrategy { /** - * Whether there is an election quorum from the point of view of the provided local node + * Whether there is an election quorum from the point of view of the given local node under the provided voting configurations */ boolean isElectionQuorum(DiscoveryNode localNode, long localCurrentTerm, long localAcceptedTerm, long localAcceptedVersion, VotingConfiguration lastCommittedConfiguration, VotingConfiguration lastAcceptedConfiguration, diff --git a/server/src/main/java/org/elasticsearch/cluster/coordination/PreVoteCollector.java b/server/src/main/java/org/elasticsearch/cluster/coordination/PreVoteCollector.java index 11b6e53dd409a..1b184e1890140 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/PreVoteCollector.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/PreVoteCollector.java @@ -189,14 +189,17 @@ private void handlePreVoteResponse(final PreVoteResponse response, final Discove preVotesReceived.put(sender, response); + // create a fake JoinVoteCollection based on the pre-votes and check if there is an election quorum final JoinVoteCollection voteCollection = new JoinVoteCollection(); + final DiscoveryNode localNode = clusterState.nodes().getLocalNode(); + final PreVoteResponse localPreVoteResponse = getPreVoteResponse(); + preVotesReceived.forEach((node, preVoteResponse) -> voteCollection.addJoinVote( - new Join(node, clusterState.nodes().getLocalNode(), preVoteResponse.getCurrentTerm(), + new Join(node, localNode, preVoteResponse.getCurrentTerm(), preVoteResponse.getLastAcceptedTerm(), preVoteResponse.getLastAcceptedVersion()))); - final PreVoteResponse localPrevoteResponse = getPreVoteResponse(); - if (electionStrategy.isElectionQuorum(clusterState.nodes().getLocalNode(), localPrevoteResponse.getCurrentTerm(), - localPrevoteResponse.getLastAcceptedTerm(), localPrevoteResponse.getLastAcceptedVersion(), + if (electionStrategy.isElectionQuorum(clusterState.nodes().getLocalNode(), localPreVoteResponse.getCurrentTerm(), + localPreVoteResponse.getLastAcceptedTerm(), localPreVoteResponse.getLastAcceptedVersion(), clusterState.getLastCommittedConfiguration(), clusterState.getLastAcceptedConfiguration(), voteCollection) == false) { logger.debug("{} added {} from {}, no quorum yet", this, response, sender); return; diff --git a/server/src/main/java/org/elasticsearch/cluster/node/DiscoveryNode.java b/server/src/main/java/org/elasticsearch/cluster/node/DiscoveryNode.java index 156eb152c5186..6e9ad85aec56c 100644 --- a/server/src/main/java/org/elasticsearch/cluster/node/DiscoveryNode.java +++ b/server/src/main/java/org/elasticsearch/cluster/node/DiscoveryNode.java @@ -410,7 +410,7 @@ public String toString() { sb.append('{').append(address).append('}'); if (roles.isEmpty() == false) { sb.append('{'); - roles.stream().map(DiscoveryNodeRole::roleNameAbbreviation).forEach(sb::append); + roles.stream().map(DiscoveryNodeRole::roleNameAbbreviation).sorted().forEach(sb::append); sb.append('}'); } if (!attributes.isEmpty()) { diff --git a/server/src/main/java/org/elasticsearch/cluster/node/DiscoveryNodes.java b/server/src/main/java/org/elasticsearch/cluster/node/DiscoveryNodes.java index 57ceb6da47c0c..8fb325d594707 100644 --- a/server/src/main/java/org/elasticsearch/cluster/node/DiscoveryNodes.java +++ b/server/src/main/java/org/elasticsearch/cluster/node/DiscoveryNodes.java @@ -41,7 +41,6 @@ import java.util.List; import java.util.Map; import java.util.Objects; -import java.util.Set; import java.util.stream.Stream; import java.util.stream.StreamSupport; diff --git a/server/src/test/java/org/elasticsearch/cluster/coordination/CoordinationStateTests.java b/server/src/test/java/org/elasticsearch/cluster/coordination/CoordinationStateTests.java index ea1706652900e..b27747f9ecdc5 100644 --- a/server/src/test/java/org/elasticsearch/cluster/coordination/CoordinationStateTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/coordination/CoordinationStateTests.java @@ -234,6 +234,7 @@ public void testJoinWithLowerLastAcceptedTermWinsElection() { assertTrue(cs1.handleJoin(join)); assertTrue(cs1.electionWon()); assertTrue(cs1.containsJoinVoteFor(node1)); + assertTrue(cs1.containsJoin(join)); assertFalse(cs1.containsJoinVoteFor(node2)); assertEquals(cs1.getLastPublishedVersion(), cs1.getLastAcceptedVersion()); assertFalse(cs1.handleJoin(join)); @@ -316,7 +317,10 @@ public void testHandleClientValue() { Join v2 = cs2.handleStartJoin(startJoinRequest1); assertTrue(cs1.handleJoin(v1)); assertTrue(cs1.electionWon()); + assertTrue(cs1.containsJoin(v1)); + assertFalse(cs1.containsJoin(v2)); assertTrue(cs1.handleJoin(v2)); + assertTrue(cs1.containsJoin(v2)); VotingConfiguration newConfig = new VotingConfiguration(Collections.singleton(node2.getId())); diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/license/XPackLicenseState.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/license/XPackLicenseState.java index bdcf329a1e3eb..99c1e2e91715e 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/license/XPackLicenseState.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/license/XPackLicenseState.java @@ -609,11 +609,11 @@ public synchronized boolean isRollupAllowed() { } /** - * Voting only nodes are always available as long as there is a valid license + * Voting only node functionality is always available as long as there is a valid license * * @return true if the license is active */ - public synchronized boolean isVotingOnlyNodeAllowed() { + public synchronized boolean isVotingOnlyAllowed() { return status.active; } diff --git a/x-pack/plugin/voting-only-node/src/main/java/org/elasticsearch/cluster/coordination/VotingOnlyNodeFeatureSet.java b/x-pack/plugin/voting-only-node/src/main/java/org/elasticsearch/cluster/coordination/VotingOnlyNodeFeatureSet.java index 0bc26bbfc029b..3c38f60fd33ed 100644 --- a/x-pack/plugin/voting-only-node/src/main/java/org/elasticsearch/cluster/coordination/VotingOnlyNodeFeatureSet.java +++ b/x-pack/plugin/voting-only-node/src/main/java/org/elasticsearch/cluster/coordination/VotingOnlyNodeFeatureSet.java @@ -25,8 +25,6 @@ import org.elasticsearch.xpack.core.action.XPackUsageFeatureTransportAction; import org.elasticsearch.xpack.core.votingonly.VotingOnlyNodeFeatureSetUsage; -import java.util.Map; - public class VotingOnlyNodeFeatureSet implements XPackFeatureSet { private final boolean enabled; @@ -45,7 +43,7 @@ public String name() { @Override public boolean available() { - return licenseState != null && licenseState.isVotingOnlyNodeAllowed(); + return licenseState != null && licenseState.isVotingOnlyAllowed(); } @Override @@ -70,7 +68,7 @@ public UsageTransportAction(TransportService transportService, ClusterService cl @Override protected void masterOperation(XPackUsageRequest request, ClusterState state, ActionListener listener) { - final boolean available = licenseState.isVotingOnlyNodeAllowed(); + final boolean available = licenseState.isVotingOnlyAllowed(); final VotingOnlyNodeFeatureSetUsage usage = new VotingOnlyNodeFeatureSetUsage(available, XPackSettings.VOTING_ONLY_ENABLED.get(settings)); listener.onResponse(new XPackUsageFeatureResponse(usage)); diff --git a/x-pack/plugin/voting-only-node/src/main/java/org/elasticsearch/cluster/coordination/VotingOnlyNodePlugin.java b/x-pack/plugin/voting-only-node/src/main/java/org/elasticsearch/cluster/coordination/VotingOnlyNodePlugin.java index c2a1c00b1b707..487077f331d6d 100644 --- a/x-pack/plugin/voting-only-node/src/main/java/org/elasticsearch/cluster/coordination/VotingOnlyNodePlugin.java +++ b/x-pack/plugin/voting-only-node/src/main/java/org/elasticsearch/cluster/coordination/VotingOnlyNodePlugin.java @@ -203,8 +203,8 @@ public void sendRequest(Transport.Connection conne sender.sendRequest(connection, action, request, options, new TransportResponseHandler<>() { @Override public void handleResponse(TransportResponse response) { - handler.handleException(new TransportException( - new ElasticsearchException("ignoring successful publish response for state transfer only: " + response))); + handler.handleException(new TransportException(new ElasticsearchException( + "ignoring successful publish response used purely for state transfer: " + response))); } @Override @@ -224,7 +224,7 @@ public TransportResponse read(StreamInput in) throws IOException { }); } else { threadPoolSupplier.get().generic().execute(() -> handler.handleException(new TransportException( - new ElasticsearchException("voting-only node skipping publication to [" + destinationNode + "]")))); + new ElasticsearchException("voting-only node skipping publication to " + destinationNode)))); } } else { sender.sendRequest(connection, action, request, options, handler); diff --git a/x-pack/plugin/voting-only-node/src/test/java/org/elasticsearch/cluster/coordination/LocalStateVotingOnlyNodePlugin.java b/x-pack/plugin/voting-only-node/src/test/java/org/elasticsearch/cluster/coordination/LocalStateVotingOnlyNodePlugin.java index 9e7fafc2499eb..aaacb6b76edfe 100644 --- a/x-pack/plugin/voting-only-node/src/test/java/org/elasticsearch/cluster/coordination/LocalStateVotingOnlyNodePlugin.java +++ b/x-pack/plugin/voting-only-node/src/test/java/org/elasticsearch/cluster/coordination/LocalStateVotingOnlyNodePlugin.java @@ -6,7 +6,6 @@ package org.elasticsearch.cluster.coordination; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.license.XPackLicenseState; import org.elasticsearch.xpack.core.LocalStateCompositeXPackPlugin; import java.nio.file.Path; @@ -15,13 +14,7 @@ public class LocalStateVotingOnlyNodePlugin extends LocalStateCompositeXPackPlug public LocalStateVotingOnlyNodePlugin(final Settings settings, final Path configPath) throws Exception { super(settings, configPath); - @SuppressWarnings("resource") - LocalStateVotingOnlyNodePlugin thisVar = this; - plugins.add(new VotingOnlyNodePlugin(settings) { - protected XPackLicenseState getLicenseState() { - return LocalStateVotingOnlyNodePlugin.this.getLicenseState(); - } - }); + plugins.add(new VotingOnlyNodePlugin(settings)); } } From fbba2c31c7b9685d45eea0ce1b5caa561069c244 Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Thu, 20 Jun 2019 12:26:45 +0200 Subject: [PATCH 19/29] fix docs tests on OSS distrib --- docs/reference/cluster/stats.asciidoc | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/docs/reference/cluster/stats.asciidoc b/docs/reference/cluster/stats.asciidoc index b671a15348ff6..259d8ae2bba11 100644 --- a/docs/reference/cluster/stats.asciidoc +++ b/docs/reference/cluster/stats.asciidoc @@ -208,6 +208,7 @@ Will return, for example: // TESTRESPONSE[s/"plugins": \[[^\]]*\]/"plugins": $body.$_path/] // TESTRESPONSE[s/"network_types": \{[^\}]*\}/"network_types": $body.$_path/] // TESTRESPONSE[s/"discovery_types": \{[^\}]*\}/"discovery_types": $body.$_path/] +// TESTRESPONSE[s/"count": \{[^\}]*\}/"count": $body.$_path/] // TESTRESPONSE[s/"packaging_types": \[[^\]]*\]/"packaging_types": $body.$_path/] // TESTRESPONSE[s/: true|false/: $body.$_path/] // TESTRESPONSE[s/: (\-)?[0-9]+/: $body.$_path/] @@ -218,7 +219,10 @@ Will return, for example: // see an exhaustive list anyway. // 2. Similarly, ignore the contents of `network_types`, `discovery_types`, and // `packaging_types`. -// 3. All of the numbers and strings on the right hand side of *every* field in +// 3. Ignore the contents of the (nodes) count object, as what's shown here +// depends on the license. Voting-only nodes are e.g. only shown when this +// test runs with a basic license. +// 4. All of the numbers and strings on the right hand side of *every* field in // the response are ignored. So we're really only asserting things about the // the shape of this response, not the values in it. From e7c325ed1ad40753930d8792930b5a4a45035c43 Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Fri, 21 Jun 2019 18:04:05 +0200 Subject: [PATCH 20/29] Fold JoinVoteCollection into VoteCollection --- .../coordination/CoordinationState.java | 53 +++++++++---------- .../cluster/coordination/Coordinator.java | 12 ++--- .../coordination/PreVoteCollector.java | 6 +-- .../coordination/VotingOnlyNodePlugin.java | 8 ++- 4 files changed, 37 insertions(+), 42 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/cluster/coordination/CoordinationState.java b/server/src/main/java/org/elasticsearch/cluster/coordination/CoordinationState.java index 9eb22f491c825..9df6d6e0bbea7 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/CoordinationState.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/CoordinationState.java @@ -49,7 +49,7 @@ public class CoordinationState { private final PersistedState persistedState; // transient state - private JoinVoteCollection joinVotes; + private VoteCollection joinVotes; private boolean startedJoinSinceLastReboot; private boolean electionWon; private long lastPublishedVersion; @@ -64,7 +64,7 @@ public CoordinationState(DiscoveryNode localNode, PersistedState persistedState, this.electionStrategy = electionStrategy; // transient state - this.joinVotes = new JoinVoteCollection(); + this.joinVotes = new VoteCollection(); this.startedJoinSinceLastReboot = false; this.electionWon = false; this.lastPublishedVersion = 0L; @@ -193,7 +193,7 @@ public Join handleStartJoin(StartJoinRequest startJoinRequest) { lastPublishedConfiguration = getLastAcceptedConfiguration(); startedJoinSinceLastReboot = true; electionWon = false; - joinVotes = new JoinVoteCollection(); + joinVotes = new VoteCollection(); publishVotes = new VoteCollection(); return new Join(localNode, startJoinRequest.getSourceNode(), getCurrentTerm(), getLastAcceptedTerm(), @@ -494,18 +494,28 @@ default void markLastAcceptedStateAsCommitted() { } /** - * A collection of votes, used to calculate quorums. + * A collection of votes, used to calculate quorums. Optionally records the Joins as well. */ public static class VoteCollection { private final Map nodes; + private final Set joins; public boolean addVote(DiscoveryNode sourceNode) { return nodes.put(sourceNode.getId(), sourceNode) == null; } + public boolean addJoinVote(Join join) { + final boolean added = addVote(join.getSourceNode()); + if (added) { + joins.add(join); + } + return added; + } + public VoteCollection() { nodes = new HashMap<>(); + joins = new HashSet<>(); } public boolean isQuorum(VotingConfiguration configuration) { @@ -524,44 +534,31 @@ public Collection nodes() { return Collections.unmodifiableCollection(nodes.values()); } + public Set getJoins() { + return Collections.unmodifiableSet(joins); + } + @Override public String toString() { - return "VoteCollection{" + String.join(",", nodes.keySet()) + "}"; + return "VoteCollection{votes=" + nodes.keySet() + ", joins=" + joins + "}"; } @Override public boolean equals(Object o) { if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; + if (!(o instanceof VoteCollection)) return false; VoteCollection that = (VoteCollection) o; - return nodes.equals(that.nodes); + if (!nodes.equals(that.nodes)) return false; + return joins.equals(that.joins); } @Override public int hashCode() { - return nodes.hashCode(); - } - } - - /** - * A collection of votes, extending {@link VoteCollection}, which additionally records the Joins - */ - public static class JoinVoteCollection extends VoteCollection { - - private final Set joins = new HashSet<>(); - - public boolean addJoinVote(Join join) { - final boolean added = addVote(join.getSourceNode()); - if (added) { - joins.add(join); - } - return added; - } - - public Set getJoins() { - return Collections.unmodifiableSet(joins); + int result = nodes.hashCode(); + result = 31 * result + joins.hashCode(); + return result; } } } diff --git a/server/src/main/java/org/elasticsearch/cluster/coordination/Coordinator.java b/server/src/main/java/org/elasticsearch/cluster/coordination/Coordinator.java index ec610f9ec6aaa..ce805f8b84a39 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/Coordinator.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/Coordinator.java @@ -33,7 +33,7 @@ import org.elasticsearch.cluster.coordination.ClusterFormationFailureHelper.ClusterFormationState; import org.elasticsearch.cluster.coordination.CoordinationMetaData.VotingConfigExclusion; import org.elasticsearch.cluster.coordination.CoordinationMetaData.VotingConfiguration; -import org.elasticsearch.cluster.coordination.CoordinationState.JoinVoteCollection; +import org.elasticsearch.cluster.coordination.CoordinationState.VoteCollection; import org.elasticsearch.cluster.coordination.FollowersChecker.FollowerCheckRequest; import org.elasticsearch.cluster.coordination.JoinHelper.InitialJoinAccumulator; import org.elasticsearch.cluster.metadata.MetaData; @@ -1104,7 +1104,7 @@ protected void onFoundPeersUpdated() { synchronized (mutex) { final Iterable foundPeers = getFoundPeers(); if (mode == Mode.CANDIDATE) { - final CoordinationState.VoteCollection expectedVotes = new CoordinationState.VoteCollection(); + final VoteCollection expectedVotes = new VoteCollection(); foundPeers.forEach(expectedVotes::addVote); expectedVotes.addVote(Coordinator.this.getLocalNode()); final boolean foundQuorum = coordinationState.get().isElectionQuorum(expectedVotes); @@ -1312,12 +1312,12 @@ public void onSuccess(String source) { // abdicate to it. Assume that every node that completed the publication can provide // a vote in that next election and has the latest state. final long futureElectionTerm = state.term() + 1; - final JoinVoteCollection futureJoinVoteCollection = new JoinVoteCollection(); - completedNodes().forEach(completedNode -> futureJoinVoteCollection.addJoinVote( + final VoteCollection futureVoteCollection = new VoteCollection(); + completedNodes().forEach(completedNode -> futureVoteCollection.addJoinVote( new Join(completedNode, node, futureElectionTerm, state.term(), state.version()))); return electionStrategy.isElectionQuorum(node, futureElectionTerm, state.term(), state.version(), state.getLastCommittedConfiguration(), - state.getLastAcceptedConfiguration(), futureJoinVoteCollection); + state.getLastAcceptedConfiguration(), futureVoteCollection); }) .collect(Collectors.toList()); if (masterCandidates.isEmpty() == false) { @@ -1359,7 +1359,7 @@ private void handleAssociatedJoin(Join join) { } @Override - protected boolean isPublishQuorum(CoordinationState.VoteCollection votes) { + protected boolean isPublishQuorum(VoteCollection votes) { assert Thread.holdsLock(mutex) : "Coordinator mutex not held"; return coordinationState.get().isPublishQuorum(votes); } diff --git a/server/src/main/java/org/elasticsearch/cluster/coordination/PreVoteCollector.java b/server/src/main/java/org/elasticsearch/cluster/coordination/PreVoteCollector.java index 1b184e1890140..f683057b1fcd1 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/PreVoteCollector.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/PreVoteCollector.java @@ -23,7 +23,7 @@ import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; import org.elasticsearch.cluster.ClusterState; -import org.elasticsearch.cluster.coordination.CoordinationState.JoinVoteCollection; +import org.elasticsearch.cluster.coordination.CoordinationState.VoteCollection; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.collect.Tuple; @@ -189,8 +189,8 @@ private void handlePreVoteResponse(final PreVoteResponse response, final Discove preVotesReceived.put(sender, response); - // create a fake JoinVoteCollection based on the pre-votes and check if there is an election quorum - final JoinVoteCollection voteCollection = new JoinVoteCollection(); + // create a fake VoteCollection based on the pre-votes and check if there is an election quorum + final VoteCollection voteCollection = new VoteCollection(); final DiscoveryNode localNode = clusterState.nodes().getLocalNode(); final PreVoteResponse localPreVoteResponse = getPreVoteResponse(); diff --git a/x-pack/plugin/voting-only-node/src/main/java/org/elasticsearch/cluster/coordination/VotingOnlyNodePlugin.java b/x-pack/plugin/voting-only-node/src/main/java/org/elasticsearch/cluster/coordination/VotingOnlyNodePlugin.java index 487077f331d6d..866352fca462f 100644 --- a/x-pack/plugin/voting-only-node/src/main/java/org/elasticsearch/cluster/coordination/VotingOnlyNodePlugin.java +++ b/x-pack/plugin/voting-only-node/src/main/java/org/elasticsearch/cluster/coordination/VotingOnlyNodePlugin.java @@ -11,7 +11,6 @@ import org.elasticsearch.action.ActionResponse; import org.elasticsearch.client.Client; import org.elasticsearch.cluster.coordination.CoordinationMetaData.VotingConfiguration; -import org.elasticsearch.cluster.coordination.CoordinationState.JoinVoteCollection; import org.elasticsearch.cluster.coordination.CoordinationState.VoteCollection; import org.elasticsearch.cluster.coordination.VotingOnlyNodeFeatureSet.UsageTransportAction; import org.elasticsearch.cluster.node.DiscoveryNode; @@ -172,10 +171,9 @@ public boolean isElectionQuorum(DiscoveryNode localNode, long localCurrentTerm, } // if there's a vote from a full master node with same last accepted term and version, that node should become master // instead, so we should stand down - if (joinVotes instanceof JoinVoteCollection && - ((JoinVoteCollection) joinVotes).getJoins().stream().anyMatch(join -> isFullMasterNode(join.getSourceNode()) && - join.getLastAcceptedTerm() == localAcceptedTerm && - join.getLastAcceptedVersion() == localAcceptedVersion)) { + if (joinVotes.getJoins().stream().anyMatch(join -> isFullMasterNode(join.getSourceNode()) && + join.getLastAcceptedTerm() == localAcceptedTerm && + join.getLastAcceptedVersion() == localAcceptedVersion)) { return false; } } From 1caa1b67da354e4b1c33476ca48a3360740220ae Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Fri, 21 Jun 2019 18:20:09 +0200 Subject: [PATCH 21/29] s/election type/election strategy/ --- .../common/settings/ClusterSettings.java | 2 +- .../org/elasticsearch/discovery/DiscoveryModule.java | 12 ++++++------ .../cluster/coordination/VotingOnlyNodePlugin.java | 6 +++--- 3 files changed, 10 insertions(+), 10 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java b/server/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java index 2437f1a43dcbc..b1bb84cff4d6f 100644 --- a/server/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java +++ b/server/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java @@ -352,7 +352,7 @@ public void apply(Settings value, Settings current, Settings previous) { Node.INITIAL_STATE_TIMEOUT_SETTING, DiscoveryModule.DISCOVERY_TYPE_SETTING, DiscoveryModule.DISCOVERY_SEED_PROVIDERS_SETTING, - DiscoveryModule.ELECTION_TYPE_SETTING, + DiscoveryModule.ELECTION_STRATEGY_SETTING, SettingsBasedSeedHostsProvider.DISCOVERY_SEED_HOSTS_SETTING, SeedHostsResolver.DISCOVERY_SEED_RESOLVER_MAX_CONCURRENT_RESOLVERS_SETTING, SeedHostsResolver.DISCOVERY_SEED_RESOLVER_TIMEOUT_SETTING, diff --git a/server/src/main/java/org/elasticsearch/discovery/DiscoveryModule.java b/server/src/main/java/org/elasticsearch/discovery/DiscoveryModule.java index 889790eca2ee4..63b24537c2dd9 100644 --- a/server/src/main/java/org/elasticsearch/discovery/DiscoveryModule.java +++ b/server/src/main/java/org/elasticsearch/discovery/DiscoveryModule.java @@ -76,10 +76,10 @@ public class DiscoveryModule { Setting.listSetting("discovery.seed_providers", Collections.emptyList(), Function.identity(), Property.NodeScope); - public static final String DEFAULT_ELECTION_TYPE = "default"; + public static final String DEFAULT_ELECTION_STRATEGY = "default"; - public static final Setting ELECTION_TYPE_SETTING = - new Setting<>("cluster.election.type", DEFAULT_ELECTION_TYPE, Function.identity(), Property.NodeScope); + public static final Setting ELECTION_STRATEGY_SETTING = + new Setting<>("cluster.election.strategy", DEFAULT_ELECTION_STRATEGY, Function.identity(), Property.NodeScope); private final Discovery discovery; @@ -93,7 +93,7 @@ public DiscoveryModule(Settings settings, ThreadPool threadPool, TransportServic hostProviders.put("settings", () -> new SettingsBasedSeedHostsProvider(settings, transportService)); hostProviders.put("file", () -> new FileBasedSeedHostsProvider(configFile)); final Map electionStrategies = new HashMap<>(); - electionStrategies.put(DEFAULT_ELECTION_TYPE, ElectionStrategy.DefaultElectionStrategy.INSTANCE); + electionStrategies.put(DEFAULT_ELECTION_STRATEGY, ElectionStrategy.DefaultElectionStrategy.INSTANCE); for (DiscoveryPlugin plugin : plugins) { plugin.getSeedHostProviders(transportService, networkService).forEach((key, value) -> { if (hostProviders.put(key, value) != null) { @@ -139,9 +139,9 @@ public DiscoveryModule(Settings settings, ThreadPool threadPool, TransportServic return Collections.unmodifiableList(addresses); }; - final ElectionStrategy electionStrategy = electionStrategies.get(ELECTION_TYPE_SETTING.get(settings)); + final ElectionStrategy electionStrategy = electionStrategies.get(ELECTION_STRATEGY_SETTING.get(settings)); if (electionStrategy == null) { - throw new IllegalArgumentException("Unknown election type " + ELECTION_TYPE_SETTING.get(settings)); + throw new IllegalArgumentException("Unknown election strategy " + ELECTION_STRATEGY_SETTING.get(settings)); } if (ZEN2_DISCOVERY_TYPE.equals(discoveryType) || SINGLE_NODE_DISCOVERY_TYPE.equals(discoveryType)) { diff --git a/x-pack/plugin/voting-only-node/src/main/java/org/elasticsearch/cluster/coordination/VotingOnlyNodePlugin.java b/x-pack/plugin/voting-only-node/src/main/java/org/elasticsearch/cluster/coordination/VotingOnlyNodePlugin.java index 866352fca462f..dcb2eef290199 100644 --- a/x-pack/plugin/voting-only-node/src/main/java/org/elasticsearch/cluster/coordination/VotingOnlyNodePlugin.java +++ b/x-pack/plugin/voting-only-node/src/main/java/org/elasticsearch/cluster/coordination/VotingOnlyNodePlugin.java @@ -61,7 +61,7 @@ public class VotingOnlyNodePlugin extends Plugin implements DiscoveryPlugin, Net public static final Setting VOTING_ONLY_NODE_SETTING = Setting.boolSetting("node.voting_only", false, Setting.Property.NodeScope); - private static final String VOTING_ONLY_ELECTION_TYPE = "supports_voting_only"; + private static final String VOTING_ONLY_ELECTION_STRATEGY = "supports_voting_only"; static DiscoveryNodeRole VOTING_ONLY_NODE_ROLE = new DiscoveryNodeRole("voting_only", "v") { @Override @@ -132,7 +132,7 @@ public Map getElectionStrategies() { if (XPackSettings.VOTING_ONLY_ENABLED.get(settings) == false) { return Collections.emptyMap(); } - return Collections.singletonMap(VOTING_ONLY_ELECTION_TYPE, new VotingOnlyNodeElectionStrategy()); + return Collections.singletonMap(VOTING_ONLY_ELECTION_STRATEGY, new VotingOnlyNodeElectionStrategy()); } @Override @@ -154,7 +154,7 @@ public Settings additionalSettings() { if (XPackSettings.VOTING_ONLY_ENABLED.get(settings) == false) { return Settings.EMPTY; } - return Settings.builder().put(DiscoveryModule.ELECTION_TYPE_SETTING.getKey(), VOTING_ONLY_ELECTION_TYPE).build(); + return Settings.builder().put(DiscoveryModule.ELECTION_STRATEGY_SETTING.getKey(), VOTING_ONLY_ELECTION_STRATEGY).build(); } static class VotingOnlyNodeElectionStrategy extends ElectionStrategy.DefaultElectionStrategy { From 6907d06104bbfbaf7b01867b22ba459879113735 Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Fri, 21 Jun 2019 18:25:05 +0200 Subject: [PATCH 22/29] test adjustment --- .../coordination/VotingOnlyNodePluginTests.java | 13 ++++++------- 1 file changed, 6 insertions(+), 7 deletions(-) diff --git a/x-pack/plugin/voting-only-node/src/test/java/org/elasticsearch/cluster/coordination/VotingOnlyNodePluginTests.java b/x-pack/plugin/voting-only-node/src/test/java/org/elasticsearch/cluster/coordination/VotingOnlyNodePluginTests.java index 903606606e6f8..4ef2081b990fd 100644 --- a/x-pack/plugin/voting-only-node/src/test/java/org/elasticsearch/cluster/coordination/VotingOnlyNodePluginTests.java +++ b/x-pack/plugin/voting-only-node/src/test/java/org/elasticsearch/cluster/coordination/VotingOnlyNodePluginTests.java @@ -87,19 +87,18 @@ public void testVotingOnlyNodesCannotBeMasterWithoutFullMasterNodes() throws Exc assertBusy(() -> assertThat( client().admin().cluster().prepareState().get().getState().getLastCommittedConfiguration().getNodeIds().size(), equalTo(3))); + final String oldMasterId = client().admin().cluster().prepareState().get().getState().nodes().getMasterNodeId(); internalCluster().stopCurrentMasterNode(); - try { - assertThat(client().admin().cluster().prepareState().setMasterNodeTimeout("100ms") - .execute().actionGet().getState().nodes().getMasterNodeId(), nullValue()); - fail("should not be able to find master"); - } catch (MasterNotDiscoveredException e) { - // all is well, no master elected - } + expectThrows(MasterNotDiscoveredException.class, () -> + assertThat(client().admin().cluster().prepareState().setMasterNodeTimeout("100ms").execute().actionGet() + .getState().nodes().getMasterNodeId(), nullValue())); // start a fresh full master node, which will be brought into the cluster as master by the voting-only nodes final String newMaster = internalCluster().startNode(); assertEquals(newMaster, internalCluster().getMasterName()); + final String newMasterId = client().admin().cluster().prepareState().get().getState().nodes().getMasterNodeId(); + assertNotEquals(oldMasterId, newMasterId); } } From e284426a95a1ccc8929b2c007f0c0a6672d9de35 Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Fri, 21 Jun 2019 18:51:40 +0200 Subject: [PATCH 23/29] Move ElectionStrategy from interface to class --- .../coordination/ElectionStrategy.java | 36 ++++++++++--------- .../discovery/DiscoveryModule.java | 2 +- .../plugins/DiscoveryPlugin.java | 3 ++ .../ClusterFormationFailureHelperTests.java | 2 +- .../coordination/CoordinationStateTests.java | 4 +-- .../cluster/coordination/NodeJoinTests.java | 2 +- .../coordination/PreVoteCollectorTests.java | 4 +-- .../coordination/PublicationTests.java | 2 +- .../snapshots/SnapshotResiliencyTests.java | 2 +- .../AbstractCoordinatorTestCase.java | 2 +- .../coordination/VotingOnlyNodePlugin.java | 12 +++---- 11 files changed, 38 insertions(+), 33 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/cluster/coordination/ElectionStrategy.java b/server/src/main/java/org/elasticsearch/cluster/coordination/ElectionStrategy.java index e94f7f8991ccb..0693ae690a597 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/ElectionStrategy.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/ElectionStrategy.java @@ -23,27 +23,31 @@ import org.elasticsearch.cluster.node.DiscoveryNode; /** - * Allows plugging in a custom election strategy. Note that, in order to guarantee safety of the system, custom election strategies should - * only be more restrictive than the behavior that's provided by {@link DefaultElectionStrategy}. + * Allows plugging in a custom election strategy, restricting the notion of an election quorum. */ -public interface ElectionStrategy { +public class ElectionStrategy { + + public static final ElectionStrategy DEFAULT_INSTANCE = new ElectionStrategy(); + + protected ElectionStrategy() { + + } /** * Whether there is an election quorum from the point of view of the given local node under the provided voting configurations */ - boolean isElectionQuorum(DiscoveryNode localNode, long localCurrentTerm, long localAcceptedTerm, long localAcceptedVersion, - VotingConfiguration lastCommittedConfiguration, VotingConfiguration lastAcceptedConfiguration, - VoteCollection joinVotes); - - class DefaultElectionStrategy implements ElectionStrategy { - - public static final ElectionStrategy INSTANCE = new DefaultElectionStrategy(); + public final boolean isElectionQuorum(DiscoveryNode localNode, long localCurrentTerm, long localAcceptedTerm, long localAcceptedVersion, + VotingConfiguration lastCommittedConfiguration, VotingConfiguration lastAcceptedConfiguration, + VoteCollection joinVotes) { + return joinVotes.isQuorum(lastCommittedConfiguration) && + joinVotes.isQuorum(lastAcceptedConfiguration) && + isCustomElectionQuorum(localNode, localCurrentTerm, localAcceptedTerm, localAcceptedVersion, lastCommittedConfiguration, + lastAcceptedConfiguration, joinVotes); + } - @Override - public boolean isElectionQuorum(DiscoveryNode localNode, long localCurrentTerm, long localAcceptedTerm, long localAcceptedVersion, - VotingConfiguration lastCommittedConfiguration, VotingConfiguration lastAcceptedConfiguration, - VoteCollection joinVotes) { - return joinVotes.isQuorum(lastCommittedConfiguration) && joinVotes.isQuorum(lastAcceptedConfiguration); - } + protected boolean isCustomElectionQuorum(DiscoveryNode localNode, long localCurrentTerm, long localAcceptedTerm, + long localAcceptedVersion, VotingConfiguration lastCommittedConfiguration, + VotingConfiguration lastAcceptedConfiguration, VoteCollection joinVotes) { + return true; } } diff --git a/server/src/main/java/org/elasticsearch/discovery/DiscoveryModule.java b/server/src/main/java/org/elasticsearch/discovery/DiscoveryModule.java index 63b24537c2dd9..6fba263d66286 100644 --- a/server/src/main/java/org/elasticsearch/discovery/DiscoveryModule.java +++ b/server/src/main/java/org/elasticsearch/discovery/DiscoveryModule.java @@ -93,7 +93,7 @@ public DiscoveryModule(Settings settings, ThreadPool threadPool, TransportServic hostProviders.put("settings", () -> new SettingsBasedSeedHostsProvider(settings, transportService)); hostProviders.put("file", () -> new FileBasedSeedHostsProvider(configFile)); final Map electionStrategies = new HashMap<>(); - electionStrategies.put(DEFAULT_ELECTION_STRATEGY, ElectionStrategy.DefaultElectionStrategy.INSTANCE); + electionStrategies.put(DEFAULT_ELECTION_STRATEGY, ElectionStrategy.DEFAULT_INSTANCE); for (DiscoveryPlugin plugin : plugins) { plugin.getSeedHostProviders(transportService, networkService).forEach((key, value) -> { if (hostProviders.put(key, value) != null) { diff --git a/server/src/main/java/org/elasticsearch/plugins/DiscoveryPlugin.java b/server/src/main/java/org/elasticsearch/plugins/DiscoveryPlugin.java index 97b6308289311..4757386efc5e8 100644 --- a/server/src/main/java/org/elasticsearch/plugins/DiscoveryPlugin.java +++ b/server/src/main/java/org/elasticsearch/plugins/DiscoveryPlugin.java @@ -87,6 +87,9 @@ default Map> getSeedHostProviders(TransportS */ default BiConsumer getJoinValidator() { return null; } + /** + * Allows plugging in election strategies (see {@link ElectionStrategy}) that define a customized notion of an election quorum. + */ default Map getElectionStrategies() { return Collections.emptyMap(); } diff --git a/server/src/test/java/org/elasticsearch/cluster/coordination/ClusterFormationFailureHelperTests.java b/server/src/test/java/org/elasticsearch/cluster/coordination/ClusterFormationFailureHelperTests.java index db94076e40bb0..850a1ec0b7abd 100644 --- a/server/src/test/java/org/elasticsearch/cluster/coordination/ClusterFormationFailureHelperTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/coordination/ClusterFormationFailureHelperTests.java @@ -49,7 +49,7 @@ public class ClusterFormationFailureHelperTests extends ESTestCase { - private static final ElectionStrategy electionStrategy = ElectionStrategy.DefaultElectionStrategy.INSTANCE; + private static final ElectionStrategy electionStrategy = ElectionStrategy.DEFAULT_INSTANCE; public void testScheduling() { final long expectedDelayMillis; diff --git a/server/src/test/java/org/elasticsearch/cluster/coordination/CoordinationStateTests.java b/server/src/test/java/org/elasticsearch/cluster/coordination/CoordinationStateTests.java index b27747f9ecdc5..99f04015867f5 100644 --- a/server/src/test/java/org/elasticsearch/cluster/coordination/CoordinationStateTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/coordination/CoordinationStateTests.java @@ -766,12 +766,12 @@ public void testVoteCollection() { public void testSafety() { new CoordinationStateTestCluster(IntStream.range(0, randomIntBetween(1, 5)) .mapToObj(i -> new DiscoveryNode("node_" + i, buildNewFakeTransportAddress(), Version.CURRENT)) - .collect(Collectors.toList()), ElectionStrategy.DefaultElectionStrategy.INSTANCE) + .collect(Collectors.toList()), ElectionStrategy.DEFAULT_INSTANCE) .runRandomly(); } public static CoordinationState createCoordinationState(PersistedState storage, DiscoveryNode localNode) { - return new CoordinationState(localNode, storage, ElectionStrategy.DefaultElectionStrategy.INSTANCE); + return new CoordinationState(localNode, storage, ElectionStrategy.DEFAULT_INSTANCE); } public static ClusterState clusterState(long term, long version, DiscoveryNode localNode, VotingConfiguration lastCommittedConfig, diff --git a/server/src/test/java/org/elasticsearch/cluster/coordination/NodeJoinTests.java b/server/src/test/java/org/elasticsearch/cluster/coordination/NodeJoinTests.java index 4416912de0cc4..8f7648130e8da 100644 --- a/server/src/test/java/org/elasticsearch/cluster/coordination/NodeJoinTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/coordination/NodeJoinTests.java @@ -174,7 +174,7 @@ transportService, writableRegistry(), () -> new InMemoryPersistedState(term, initialState), r -> emptyList(), new NoOpClusterApplier(), Collections.emptyList(), - random, s -> {}, ElectionStrategy.DefaultElectionStrategy.INSTANCE); + random, s -> {}, ElectionStrategy.DEFAULT_INSTANCE); transportService.start(); transportService.acceptIncomingRequests(); transport = capturingTransport; diff --git a/server/src/test/java/org/elasticsearch/cluster/coordination/PreVoteCollectorTests.java b/server/src/test/java/org/elasticsearch/cluster/coordination/PreVoteCollectorTests.java index ec5d5f16f3b3d..4d6578e5c5d12 100644 --- a/server/src/test/java/org/elasticsearch/cluster/coordination/PreVoteCollectorTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/coordination/PreVoteCollectorTests.java @@ -114,7 +114,7 @@ public String toString() { assert electionOccurred == false; electionOccurred = true; }, l -> { - }, ElectionStrategy.DefaultElectionStrategy.INSTANCE); // TODO need tests that check that the max term seen is updated + }, ElectionStrategy.DEFAULT_INSTANCE); // TODO need tests that check that the max term seen is updated preVoteCollector.update(getLocalPreVoteResponse(), null); } @@ -234,7 +234,7 @@ public void testPrevotingIndicatesElectionSuccess() { startAndRunCollector(votingNodes); final CoordinationState coordinationState = new CoordinationState(localNode, - new InMemoryPersistedState(currentTerm, makeClusterState(votingNodes)), ElectionStrategy.DefaultElectionStrategy.INSTANCE); + new InMemoryPersistedState(currentTerm, makeClusterState(votingNodes)), ElectionStrategy.DEFAULT_INSTANCE); final long newTerm = randomLongBetween(currentTerm + 1, Long.MAX_VALUE); diff --git a/server/src/test/java/org/elasticsearch/cluster/coordination/PublicationTests.java b/server/src/test/java/org/elasticsearch/cluster/coordination/PublicationTests.java index c3367d10ac4d8..5050b12531859 100644 --- a/server/src/test/java/org/elasticsearch/cluster/coordination/PublicationTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/coordination/PublicationTests.java @@ -73,7 +73,7 @@ class MockNode { ClusterState initialState = CoordinationStateTests.clusterState(0L, 0L, localNode, CoordinationMetaData.VotingConfiguration.EMPTY_CONFIG, CoordinationMetaData.VotingConfiguration.EMPTY_CONFIG, 0L); coordinationState = new CoordinationState(localNode, new InMemoryPersistedState(0L, initialState), - ElectionStrategy.DefaultElectionStrategy.INSTANCE); + ElectionStrategy.DEFAULT_INSTANCE); } final DiscoveryNode localNode; diff --git a/server/src/test/java/org/elasticsearch/snapshots/SnapshotResiliencyTests.java b/server/src/test/java/org/elasticsearch/snapshots/SnapshotResiliencyTests.java index 084348ce12717..fc2d22fb7029b 100644 --- a/server/src/test/java/org/elasticsearch/snapshots/SnapshotResiliencyTests.java +++ b/server/src/test/java/org/elasticsearch/snapshots/SnapshotResiliencyTests.java @@ -1247,7 +1247,7 @@ public void start(ClusterState initialState) { hostsResolver -> testClusterNodes.nodes.values().stream().filter(n -> n.node.isMasterNode()) .map(n -> n.node.getAddress()).collect(Collectors.toList()), clusterService.getClusterApplierService(), Collections.emptyList(), random(), - new RoutingService(clusterService, allocationService)::reroute, ElectionStrategy.DefaultElectionStrategy.INSTANCE); + new RoutingService(clusterService, allocationService)::reroute, ElectionStrategy.DEFAULT_INSTANCE); masterService.setClusterStatePublisher(coordinator); coordinator.start(); masterService.start(); diff --git a/test/framework/src/main/java/org/elasticsearch/cluster/coordination/AbstractCoordinatorTestCase.java b/test/framework/src/main/java/org/elasticsearch/cluster/coordination/AbstractCoordinatorTestCase.java index a6f1ab34384d9..0547412f61500 100644 --- a/test/framework/src/main/java/org/elasticsearch/cluster/coordination/AbstractCoordinatorTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/cluster/coordination/AbstractCoordinatorTestCase.java @@ -839,7 +839,7 @@ protected Optional getDisruptableMockTransport(Transpo coordinator = new Coordinator("test_node", settings, clusterSettings, transportService, writableRegistry(), allocationService, masterService, this::getPersistedState, Cluster.this::provideSeedHosts, clusterApplierService, onJoinValidators, Randomness.get(), s -> {}, - ElectionStrategy.DefaultElectionStrategy.INSTANCE); + ElectionStrategy.DEFAULT_INSTANCE); masterService.setClusterStatePublisher(coordinator); final GatewayService gatewayService = new GatewayService(settings, allocationService, clusterService, deterministicTaskQueue.getThreadPool(this::onNode), null, coordinator); diff --git a/x-pack/plugin/voting-only-node/src/main/java/org/elasticsearch/cluster/coordination/VotingOnlyNodePlugin.java b/x-pack/plugin/voting-only-node/src/main/java/org/elasticsearch/cluster/coordination/VotingOnlyNodePlugin.java index dcb2eef290199..4da2d0a1b1875 100644 --- a/x-pack/plugin/voting-only-node/src/main/java/org/elasticsearch/cluster/coordination/VotingOnlyNodePlugin.java +++ b/x-pack/plugin/voting-only-node/src/main/java/org/elasticsearch/cluster/coordination/VotingOnlyNodePlugin.java @@ -157,12 +157,12 @@ public Settings additionalSettings() { return Settings.builder().put(DiscoveryModule.ELECTION_STRATEGY_SETTING.getKey(), VOTING_ONLY_ELECTION_STRATEGY).build(); } - static class VotingOnlyNodeElectionStrategy extends ElectionStrategy.DefaultElectionStrategy { + static class VotingOnlyNodeElectionStrategy extends ElectionStrategy { @Override - public boolean isElectionQuorum(DiscoveryNode localNode, long localCurrentTerm, long localAcceptedTerm, long localAcceptedVersion, - VotingConfiguration lastCommittedConfiguration, VotingConfiguration lastAcceptedConfiguration, - VoteCollection joinVotes) { + public boolean isCustomElectionQuorum(DiscoveryNode localNode, long localCurrentTerm, long localAcceptedTerm, + long localAcceptedVersion, VotingConfiguration lastCommittedConfiguration, + VotingConfiguration lastAcceptedConfiguration, VoteCollection joinVotes) { // if local node is voting only, have additional checks on election quorum definition if (isVotingOnlyNode(localNode)) { // if all votes are from voting only nodes, do not elect as master (no need to transfer state) @@ -177,9 +177,7 @@ public boolean isElectionQuorum(DiscoveryNode localNode, long localCurrentTerm, return false; } } - // fall back to default election quorum definition - return super.isElectionQuorum(localNode, localCurrentTerm, localAcceptedTerm, localAcceptedVersion, - lastCommittedConfiguration, lastAcceptedConfiguration, joinVotes); + return true; } } From bcd6ec282782235f1458a35a8d9c797c90c65b78 Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Fri, 21 Jun 2019 18:57:34 +0200 Subject: [PATCH 24/29] Have VotingOnlyNodePlugin always enabled --- .../elasticsearch/xpack/core/XPackSettings.java | 6 ------ .../votingonly/VotingOnlyNodeFeatureSetUsage.java | 4 ++-- .../coordination/VotingOnlyNodeFeatureSet.java | 14 ++++---------- .../cluster/coordination/VotingOnlyNodePlugin.java | 11 ----------- .../coordination/VotingOnlyNodePluginTests.java | 10 ---------- 5 files changed, 6 insertions(+), 39 deletions(-) diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackSettings.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackSettings.java index 8d4f636f9e160..95184d02e1ead 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackSettings.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackSettings.java @@ -117,11 +117,6 @@ private XPackSettings() { /** Setting for enabling or disabling vectors. Defaults to true. */ public static final Setting VECTORS_ENABLED = Setting.boolSetting("xpack.vectors.enabled", true, Setting.Property.NodeScope); - /** Setting for enabling or disabling the voting-only-node functionality. Needs to be enabled on both voting-only nodes and regular - * master-eligible nodes for the voting-only functionality to work correctly. Defaults to true. */ - public static final Setting VOTING_ONLY_ENABLED = Setting.boolSetting("xpack.voting_only.enabled", true, - Setting.Property.NodeScope); - /* * SSL settings. These are the settings that are specifically registered for SSL. Many are private as we do not explicitly use them * but instead parse based on a prefix (eg *.ssl.*) @@ -213,7 +208,6 @@ public static List> getAllSettings() { settings.add(INDEX_LIFECYCLE_ENABLED); settings.add(DATA_FRAME_ENABLED); settings.add(VECTORS_ENABLED); - settings.add(VOTING_ONLY_ENABLED); return Collections.unmodifiableList(settings); } diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/votingonly/VotingOnlyNodeFeatureSetUsage.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/votingonly/VotingOnlyNodeFeatureSetUsage.java index 9b62d9c240a91..71e94e6d31860 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/votingonly/VotingOnlyNodeFeatureSetUsage.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/votingonly/VotingOnlyNodeFeatureSetUsage.java @@ -16,7 +16,7 @@ public VotingOnlyNodeFeatureSetUsage(StreamInput input) throws IOException { super(input); } - public VotingOnlyNodeFeatureSetUsage(boolean available, boolean enabled) { - super(XPackField.VOTING_ONLY, available, enabled); + public VotingOnlyNodeFeatureSetUsage(boolean available) { + super(XPackField.VOTING_ONLY, available, true); } } diff --git a/x-pack/plugin/voting-only-node/src/main/java/org/elasticsearch/cluster/coordination/VotingOnlyNodeFeatureSet.java b/x-pack/plugin/voting-only-node/src/main/java/org/elasticsearch/cluster/coordination/VotingOnlyNodeFeatureSet.java index 3c38f60fd33ed..ef54d0ea5ccc2 100644 --- a/x-pack/plugin/voting-only-node/src/main/java/org/elasticsearch/cluster/coordination/VotingOnlyNodeFeatureSet.java +++ b/x-pack/plugin/voting-only-node/src/main/java/org/elasticsearch/cluster/coordination/VotingOnlyNodeFeatureSet.java @@ -12,14 +12,12 @@ import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.inject.Inject; -import org.elasticsearch.common.settings.Settings; import org.elasticsearch.license.XPackLicenseState; import org.elasticsearch.protocol.xpack.XPackUsageRequest; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; import org.elasticsearch.xpack.core.XPackFeatureSet; import org.elasticsearch.xpack.core.XPackField; -import org.elasticsearch.xpack.core.XPackSettings; import org.elasticsearch.xpack.core.action.XPackUsageFeatureAction; import org.elasticsearch.xpack.core.action.XPackUsageFeatureResponse; import org.elasticsearch.xpack.core.action.XPackUsageFeatureTransportAction; @@ -27,12 +25,10 @@ public class VotingOnlyNodeFeatureSet implements XPackFeatureSet { - private final boolean enabled; private final XPackLicenseState licenseState; @Inject - public VotingOnlyNodeFeatureSet(Settings settings, @Nullable XPackLicenseState licenseState) { - this.enabled = XPackSettings.VOTING_ONLY_ENABLED.get(settings); + public VotingOnlyNodeFeatureSet(@Nullable XPackLicenseState licenseState) { this.licenseState = licenseState; } @@ -48,21 +44,19 @@ public boolean available() { @Override public boolean enabled() { - return enabled; + return true; } public static class UsageTransportAction extends XPackUsageFeatureTransportAction { - private final Settings settings; private final XPackLicenseState licenseState; @Inject public UsageTransportAction(TransportService transportService, ClusterService clusterService, ThreadPool threadPool, ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver, - Settings settings, XPackLicenseState licenseState) { + XPackLicenseState licenseState) { super(XPackUsageFeatureAction.VOTING_ONLY_NODE.name(), transportService, clusterService, threadPool, actionFilters, indexNameExpressionResolver); - this.settings = settings; this.licenseState = licenseState; } @@ -70,7 +64,7 @@ public UsageTransportAction(TransportService transportService, ClusterService cl protected void masterOperation(XPackUsageRequest request, ClusterState state, ActionListener listener) { final boolean available = licenseState.isVotingOnlyAllowed(); final VotingOnlyNodeFeatureSetUsage usage = - new VotingOnlyNodeFeatureSetUsage(available, XPackSettings.VOTING_ONLY_ENABLED.get(settings)); + new VotingOnlyNodeFeatureSetUsage(available); listener.onResponse(new XPackUsageFeatureResponse(usage)); } } diff --git a/x-pack/plugin/voting-only-node/src/main/java/org/elasticsearch/cluster/coordination/VotingOnlyNodePlugin.java b/x-pack/plugin/voting-only-node/src/main/java/org/elasticsearch/cluster/coordination/VotingOnlyNodePlugin.java index 4da2d0a1b1875..69a1fce602f1c 100644 --- a/x-pack/plugin/voting-only-node/src/main/java/org/elasticsearch/cluster/coordination/VotingOnlyNodePlugin.java +++ b/x-pack/plugin/voting-only-node/src/main/java/org/elasticsearch/cluster/coordination/VotingOnlyNodePlugin.java @@ -42,7 +42,6 @@ import org.elasticsearch.transport.TransportResponseHandler; import org.elasticsearch.watcher.ResourceWatcherService; import org.elasticsearch.xpack.core.XPackPlugin; -import org.elasticsearch.xpack.core.XPackSettings; import org.elasticsearch.xpack.core.action.XPackUsageFeatureAction; import java.io.IOException; @@ -96,10 +95,6 @@ public List> getSettings() { @Override public Set getRoles() { - if (VOTING_ONLY_NODE_SETTING.exists(settings) && XPackSettings.VOTING_ONLY_ENABLED.get(settings) == false) { - throw new IllegalStateException(XPackSettings.VOTING_ONLY_ENABLED.getKey() + " must be set to true to use the " + - VOTING_ONLY_NODE_SETTING.getKey() + " setting"); - } if (isVotingOnlyNode && Node.NODE_MASTER_SETTING.get(settings) == false) { throw new IllegalStateException("voting-only node must be master-eligible"); } @@ -129,9 +124,6 @@ public Collection createGuiceModules() { @Override public Map getElectionStrategies() { - if (XPackSettings.VOTING_ONLY_ENABLED.get(settings) == false) { - return Collections.emptyMap(); - } return Collections.singletonMap(VOTING_ONLY_ELECTION_STRATEGY, new VotingOnlyNodeElectionStrategy()); } @@ -151,9 +143,6 @@ public AsyncSender interceptSender(AsyncSender sender) { @Override public Settings additionalSettings() { - if (XPackSettings.VOTING_ONLY_ENABLED.get(settings) == false) { - return Settings.EMPTY; - } return Settings.builder().put(DiscoveryModule.ELECTION_STRATEGY_SETTING.getKey(), VOTING_ONLY_ELECTION_STRATEGY).build(); } diff --git a/x-pack/plugin/voting-only-node/src/test/java/org/elasticsearch/cluster/coordination/VotingOnlyNodePluginTests.java b/x-pack/plugin/voting-only-node/src/test/java/org/elasticsearch/cluster/coordination/VotingOnlyNodePluginTests.java index 4ef2081b990fd..591dc1bb02ddc 100644 --- a/x-pack/plugin/voting-only-node/src/test/java/org/elasticsearch/cluster/coordination/VotingOnlyNodePluginTests.java +++ b/x-pack/plugin/voting-only-node/src/test/java/org/elasticsearch/cluster/coordination/VotingOnlyNodePluginTests.java @@ -39,16 +39,6 @@ public void testRequireVotingOnlyNodeToBeMasterEligible() { assertThat(ise.getMessage(), containsString("voting-only node must be master-eligible")); } - public void testRequireVotingOnlyNodeToHaveXPackSettingEnabled() { - internalCluster().setBootstrapMasterNodeIndex(0); - IllegalStateException ise = expectThrows(IllegalStateException.class, () -> internalCluster().startNode(Settings.builder() - .put(VotingOnlyNodePlugin.VOTING_ONLY_NODE_SETTING.getKey(), true) - .put(XPackSettings.VOTING_ONLY_ENABLED.getKey(), false) - .build())); - assertThat(ise.getMessage(), containsString(XPackSettings.VOTING_ONLY_ENABLED.getKey() + " must be set to true to use the " + - VotingOnlyNodePlugin.VOTING_ONLY_NODE_SETTING.getKey() + " setting")); - } - public void testVotingOnlyNodeStats() throws Exception { internalCluster().setBootstrapMasterNodeIndex(0); internalCluster().startNodes(2); From 264e5a3276dfcb2bdf93ff119ad050ec06f7bcd0 Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Fri, 21 Jun 2019 19:15:34 +0200 Subject: [PATCH 25/29] Ryan feedback --- .../coordination/ElectionStrategy.java | 39 ++++++++++++++----- .../coordination/VotingOnlyNodePlugin.java | 6 +-- 2 files changed, 33 insertions(+), 12 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/cluster/coordination/ElectionStrategy.java b/server/src/main/java/org/elasticsearch/cluster/coordination/ElectionStrategy.java index 0693ae690a597..b9e00cd52fc4e 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/ElectionStrategy.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/ElectionStrategy.java @@ -24,10 +24,18 @@ /** * Allows plugging in a custom election strategy, restricting the notion of an election quorum. + * Custom additional quorum restrictions can be defined by implementing the {@link #satisfiesAdditionalQuorumConstraints} method. */ -public class ElectionStrategy { +public abstract class ElectionStrategy { - public static final ElectionStrategy DEFAULT_INSTANCE = new ElectionStrategy(); + public static final ElectionStrategy DEFAULT_INSTANCE = new ElectionStrategy() { + @Override + protected boolean satisfiesAdditionalQuorumConstraints(DiscoveryNode localNode, long localCurrentTerm, long localAcceptedTerm, + long localAcceptedVersion, VotingConfiguration lastCommittedConfiguration, + VotingConfiguration lastAcceptedConfiguration, VoteCollection joinVotes) { + return true; + } + }; protected ElectionStrategy() { @@ -41,13 +49,26 @@ public final boolean isElectionQuorum(DiscoveryNode localNode, long localCurrent VoteCollection joinVotes) { return joinVotes.isQuorum(lastCommittedConfiguration) && joinVotes.isQuorum(lastAcceptedConfiguration) && - isCustomElectionQuorum(localNode, localCurrentTerm, localAcceptedTerm, localAcceptedVersion, lastCommittedConfiguration, - lastAcceptedConfiguration, joinVotes); + satisfiesAdditionalQuorumConstraints(localNode, localCurrentTerm, localAcceptedTerm, localAcceptedVersion, + lastCommittedConfiguration, lastAcceptedConfiguration, joinVotes); } - protected boolean isCustomElectionQuorum(DiscoveryNode localNode, long localCurrentTerm, long localAcceptedTerm, - long localAcceptedVersion, VotingConfiguration lastCommittedConfiguration, - VotingConfiguration lastAcceptedConfiguration, VoteCollection joinVotes) { - return true; - } + /** + * The extension point to be overridden by plugins. Defines additional constraints on the election quorum. + * @param localNode the local node for the election quorum + * @param localCurrentTerm the current term of the local node + * @param localAcceptedTerm the last accepted term of the local node + * @param localAcceptedVersion the last accepted version of the local node + * @param lastCommittedConfiguration the last committed configuration for the election quorum + * @param lastAcceptedConfiguration the last accepted configuration for the election quorum + * @param joinVotes the votes that were provided so far + * @return true iff the additional quorum constraints are satisfied + */ + protected abstract boolean satisfiesAdditionalQuorumConstraints(DiscoveryNode localNode, + long localCurrentTerm, + long localAcceptedTerm, + long localAcceptedVersion, + VotingConfiguration lastCommittedConfiguration, + VotingConfiguration lastAcceptedConfiguration, + VoteCollection joinVotes); } diff --git a/x-pack/plugin/voting-only-node/src/main/java/org/elasticsearch/cluster/coordination/VotingOnlyNodePlugin.java b/x-pack/plugin/voting-only-node/src/main/java/org/elasticsearch/cluster/coordination/VotingOnlyNodePlugin.java index 69a1fce602f1c..0c062ee1df591 100644 --- a/x-pack/plugin/voting-only-node/src/main/java/org/elasticsearch/cluster/coordination/VotingOnlyNodePlugin.java +++ b/x-pack/plugin/voting-only-node/src/main/java/org/elasticsearch/cluster/coordination/VotingOnlyNodePlugin.java @@ -149,9 +149,9 @@ public Settings additionalSettings() { static class VotingOnlyNodeElectionStrategy extends ElectionStrategy { @Override - public boolean isCustomElectionQuorum(DiscoveryNode localNode, long localCurrentTerm, long localAcceptedTerm, - long localAcceptedVersion, VotingConfiguration lastCommittedConfiguration, - VotingConfiguration lastAcceptedConfiguration, VoteCollection joinVotes) { + public boolean satisfiesAdditionalQuorumConstraints(DiscoveryNode localNode, long localCurrentTerm, long localAcceptedTerm, + long localAcceptedVersion, VotingConfiguration lastCommittedConfiguration, + VotingConfiguration lastAcceptedConfiguration, VoteCollection joinVotes) { // if local node is voting only, have additional checks on election quorum definition if (isVotingOnlyNode(localNode)) { // if all votes are from voting only nodes, do not elect as master (no need to transfer state) From d8a0b9a796988c06d2a287e456e8c3736c6a1bf5 Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Fri, 21 Jun 2019 19:42:51 +0200 Subject: [PATCH 26/29] checkstyle --- .../cluster/coordination/VotingOnlyNodePluginTests.java | 1 - 1 file changed, 1 deletion(-) diff --git a/x-pack/plugin/voting-only-node/src/test/java/org/elasticsearch/cluster/coordination/VotingOnlyNodePluginTests.java b/x-pack/plugin/voting-only-node/src/test/java/org/elasticsearch/cluster/coordination/VotingOnlyNodePluginTests.java index 591dc1bb02ddc..e6d9cbb3c2d2f 100644 --- a/x-pack/plugin/voting-only-node/src/test/java/org/elasticsearch/cluster/coordination/VotingOnlyNodePluginTests.java +++ b/x-pack/plugin/voting-only-node/src/test/java/org/elasticsearch/cluster/coordination/VotingOnlyNodePluginTests.java @@ -12,7 +12,6 @@ import org.elasticsearch.plugins.Plugin; import org.elasticsearch.test.ESIntegTestCase; import org.elasticsearch.test.ESIntegTestCase.Scope; -import org.elasticsearch.xpack.core.XPackSettings; import java.util.Collection; import java.util.Collections; From ccdb483eaeb14105ce2c10468744f2621b54cce3 Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Fri, 21 Jun 2019 21:15:25 +0200 Subject: [PATCH 27/29] doc changes --- docs/reference/modules/node.asciidoc | 13 +++++++++---- 1 file changed, 9 insertions(+), 4 deletions(-) diff --git a/docs/reference/modules/node.asciidoc b/docs/reference/modules/node.asciidoc index bc9606d978275..d8ebf9f962e5c 100644 --- a/docs/reference/modules/node.asciidoc +++ b/docs/reference/modules/node.asciidoc @@ -159,12 +159,17 @@ so that if one of the three nodes is down, then the remaining two can still elect a master amongst them-selves. This only requires one of the two remaining nodes to have the capability to act as master, but both need to have voting powers. This means that one of the three master-eligible nodes can be made as -voting-only. If this voting-only node is a dedicated master, a less powerful -machine or a smaller heap-size can be chosen for this node. Alternatively, -a voting-only non-dedicated master node can play the role of the third -master-eligible node, which allows running an HA cluster with only two +voting-only. If this voting-only node is a dedicated master, a machine with +a less powerful CPU and a smaller heap-size can be chosen for this node. +Alternatively, a voting-only non-dedicated master node can play the role of the +third master-eligible node, which allows running an HA cluster with only two dedicated master nodes. +As any master-eligible node, voting-only nodes should have fast disks and a +reliable low-latency connection to the other nodes in the cluster, as they +are on the critical path for +<>. + [float] [[data-node]] === Data Node From 28efcf0532aea9da3301aa4600736701e824e947 Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Mon, 24 Jun 2019 10:05:14 +0200 Subject: [PATCH 28/29] Add note about voting-only in default distrib --- docs/reference/cluster.asciidoc | 3 +++ 1 file changed, 3 insertions(+) diff --git a/docs/reference/cluster.asciidoc b/docs/reference/cluster.asciidoc index 4727067f78b3c..da23027843ba6 100644 --- a/docs/reference/cluster.asciidoc +++ b/docs/reference/cluster.asciidoc @@ -48,6 +48,9 @@ means that filters such as `master:false` which remove nodes from the chosen subset are only useful if they come after some other filters. When used on its own, `master:false` selects no nodes. +NOTE: Designating nodes as `voting_only` and using `voting_only` in node +filters requires the default distribution of Elasticsearch. + Here are some examples of the use of node filters with the <> APIs. From 7bb6fa28654e4965bb44d5ddaf33fbd3320f8b98 Mon Sep 17 00:00:00 2001 From: David Turner Date: Tue, 25 Jun 2019 11:13:34 +0100 Subject: [PATCH 29/29] Reword docs * Generalise the note on the node filters page to talk about the `voting_only` role, since it's not clear on this page what "designating" means. * Contrast `default-dist` to `oss-dist`, because in isolation the phrase "default distribution" is kinda mysterious to the general public. * Qualify statement about "any master eligible node may be elected" * Add excuse for the confusing terminology * Shrink explanation about HA clusters, because it involved concepts like "voting powers" and "capability to act as master" that aren't really defined. * Remove suggestion to use a data node as a voting-only node: although this might work in many cases, I think if we specifically mention it here we will see too many clusters with overloaded data nodes that can't reasonably be a master. I think this is best left implicit. --- docs/reference/cluster.asciidoc | 5 ++- docs/reference/modules/node.asciidoc | 60 +++++++++++++++------------- 2 files changed, 36 insertions(+), 29 deletions(-) diff --git a/docs/reference/cluster.asciidoc b/docs/reference/cluster.asciidoc index da23027843ba6..da5ce2b410281 100644 --- a/docs/reference/cluster.asciidoc +++ b/docs/reference/cluster.asciidoc @@ -48,8 +48,8 @@ means that filters such as `master:false` which remove nodes from the chosen subset are only useful if they come after some other filters. When used on its own, `master:false` selects no nodes. -NOTE: Designating nodes as `voting_only` and using `voting_only` in node -filters requires the default distribution of Elasticsearch. +NOTE: The `voting_only` role requires the {default-dist} of Elasticsearch and +is not supported in the {oss-dist}. Here are some examples of the use of node filters with the <> APIs. @@ -74,6 +74,7 @@ GET /_nodes/10.0.0.* GET /_nodes/_all,master:false GET /_nodes/data:true,ingest:true GET /_nodes/coordinating_only:true +GET /_nodes/master:true,voting_only:false # Select nodes by custom attribute (e.g. with something like `node.attr.rack: 2` in the configuration file) GET /_nodes/rack:2 GET /_nodes/ra*:2 diff --git a/docs/reference/modules/node.asciidoc b/docs/reference/modules/node.asciidoc index d8ebf9f962e5c..7f47031988fd0 100644 --- a/docs/reference/modules/node.asciidoc +++ b/docs/reference/modules/node.asciidoc @@ -84,8 +84,9 @@ creating or deleting an index, tracking which nodes are part of the cluster, and deciding which shards to allocate to which nodes. It is important for cluster health to have a stable master node. -Any master-eligible node (all nodes by default) may be elected to become the -master node by the <>. +Any master-eligible node that is not a <> may +be elected to become the master node by the <>. IMPORTANT: Master nodes must have access to the `data/` directory (just like `data` nodes) as this is where the cluster state is persisted between node restarts. @@ -138,36 +139,41 @@ cluster.remote.connect: false <4> [[voting-only-node]] ==== Voting-only master-eligible node -A voting-only master-eligible node is a node that can participate in master -elections but will not act as a master in the cluster. In particular, a -voting-only node can help elect another master-eligible node as master, and -can serve as a tiebreaker in elections. To mark a master-eligible node as -voting-only, set: +A voting-only master-eligible node is a node that participates in +<> but which will not act as the cluster's +elected master node. In particular, a voting-only node can serve as a tiebreaker +in elections. + +It may seem confusing to use the term "master-eligible" to describe a +voting-only node since such a node is not actually eligible to become the master +at all. This terminology is an unfortunate consequence of history: +master-eligible nodes are those nodes that participate in elections and perform +certain tasks during cluster state publications, and voting-only nodes have the +same responsibilities even if they can never become the elected master. + +To configure a master-eligible node as a voting-only node, set the following +setting: [source,yaml] ------------------- node.voting_only: true <1> ------------------- -<1> The `node.voting_only` role is disabled by default. - -IMPORTANT: If you use the {oss-dist}, do not set `node.voting_only`. Otherwise, -the node fails to start. Also note that only master-eligible nodes can be -marked as voting-only. - -High availability (HA) clusters require at least three master-eligible nodes, -so that if one of the three nodes is down, then the remaining two can still -elect a master amongst them-selves. This only requires one of the two remaining -nodes to have the capability to act as master, but both need to have voting -powers. This means that one of the three master-eligible nodes can be made as -voting-only. If this voting-only node is a dedicated master, a machine with -a less powerful CPU and a smaller heap-size can be chosen for this node. -Alternatively, a voting-only non-dedicated master node can play the role of the -third master-eligible node, which allows running an HA cluster with only two -dedicated master nodes. - -As any master-eligible node, voting-only nodes should have fast disks and a -reliable low-latency connection to the other nodes in the cluster, as they -are on the critical path for +<1> The default for `node.voting_only` is `false`. + +IMPORTANT: The `voting_only` role requires the {default-dist} of Elasticsearch +and is not supported in the {oss-dist}. If you use the {oss-dist} and set +`node.voting_only` then the node will fail to start. Also note that only +master-eligible nodes can be marked as voting-only. + +High availability (HA) clusters require at least three master-eligible nodes, at +least two of which are not voting-only nodes. Such a cluster will be able to +elect a master node even if one of the nodes fails. + +Since voting-only nodes never act as the cluster's elected master, they may +require require less heap and a less powerful CPU than the true master nodes. +However all master-eligible nodes, including voting-only nodes, require +reasonably fast persistent storage and a reliable and low-latency network +connection to the rest of the cluster, since they are on the critical path for <>. [float]