diff --git a/A b/A new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/docs/reference/cluster.asciidoc b/docs/reference/cluster.asciidoc index cfa2d5a6488d7..da5ce2b410281 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 @@ -46,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: 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. @@ -69,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/cluster/stats.asciidoc b/docs/reference/cluster/stats.asciidoc index 791fc2414f331..259d8ae2bba11 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}" @@ -207,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/] @@ -217,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. diff --git a/docs/reference/modules/node.asciidoc b/docs/reference/modules/node.asciidoc index 6a097bffd7031..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. @@ -134,6 +135,47 @@ 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 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 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] [[data-node]] === Data Node diff --git a/docs/reference/rest-api/info.asciidoc b/docs/reference/rest-api/info.asciidoc index 8ed862b699a27..ad442fc0b99de 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" : { + "available" : true, + "enabled" : true + }, "watcher" : { "available" : true, "enabled" : true 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..c249458dd23cf 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,9 @@ 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"; + = 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/CoordinationState.java b/server/src/main/java/org/elasticsearch/cluster/coordination/CoordinationState.java index ac75c83c19a26..9df6d6e0bbea7 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/CoordinationState.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/CoordinationState.java @@ -24,13 +24,14 @@ 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; 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 @@ -42,6 +43,8 @@ public class CoordinationState { private final DiscoveryNode localNode; + private final ElectionStrategy electionStrategy; + // persisted state private final PersistedState persistedState; @@ -53,11 +56,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(); @@ -100,13 +104,9 @@ public boolean electionWon() { return electionWon; } - public boolean isElectionQuorum(VoteCollection votes) { - return isElectionQuorum(votes, getLastAcceptedState()); - } - - static boolean isElectionQuorum(VoteCollection votes, ClusterState lastAcceptedState) { - return votes.isQuorum(lastAcceptedState.getLastCommittedConfiguration()) - && votes.isQuorum(lastAcceptedState.getLastAcceptedConfiguration()); + public boolean isElectionQuorum(VoteCollection joinVotes) { + return electionStrategy.isElectionQuorum(localNode, getCurrentTerm(), getLastAcceptedTerm(), getLastAcceptedVersion(), + getLastCommittedConfiguration(), getLastAcceptedConfiguration(), joinVotes); } public boolean isPublishQuorum(VoteCollection votes) { @@ -117,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); } @@ -243,7 +248,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 @@ -489,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) { @@ -519,24 +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(); + 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 b88d8ed21ba09..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,6 +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.VoteCollection; import org.elasticsearch.cluster.coordination.FollowersChecker.FollowerCheckRequest; import org.elasticsearch.cluster.coordination.JoinHelper.InitialJoinAccumulator; import org.elasticsearch.cluster.metadata.MetaData; @@ -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); @@ -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(); @@ -1101,11 +1104,10 @@ 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 ClusterState lastAcceptedState = coordinationState.get().getLastAcceptedState(); - final boolean foundQuorum = CoordinationState.isElectionQuorum(expectedVotes, lastAcceptedState); + final boolean foundQuorum = coordinationState.get().isElectionQuorum(expectedVotes); if (foundQuorum) { if (electionScheduler == null) { @@ -1305,6 +1307,18 @@ public void onSuccess(String source) { final List masterCandidates = completedNodes().stream() .filter(DiscoveryNode::isMasterNode) .filter(node -> nodeMayWinElection(state, node)) + .filter(node -> { + // 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 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(), futureVoteCollection); + }) .collect(Collectors.toList()); if (masterCandidates.isEmpty() == false) { abdicateTo(masterCandidates.get(random.nextInt(masterCandidates.size()))); @@ -1345,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/ElectionStrategy.java b/server/src/main/java/org/elasticsearch/cluster/coordination/ElectionStrategy.java new file mode 100644 index 0000000000000..b9e00cd52fc4e --- /dev/null +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/ElectionStrategy.java @@ -0,0 +1,74 @@ +/* + * 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.coordination.CoordinationMetaData.VotingConfiguration; +import org.elasticsearch.cluster.coordination.CoordinationState.VoteCollection; +import org.elasticsearch.cluster.node.DiscoveryNode; + +/** + * 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 abstract class 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() { + + } + + /** + * Whether there is an election quorum from the point of view of the given local node under the provided voting configurations + */ + 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) && + satisfiesAdditionalQuorumConstraints(localNode, localCurrentTerm, localAcceptedTerm, localAcceptedVersion, + lastCommittedConfiguration, lastAcceptedConfiguration, joinVotes); + } + + /** + * 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/server/src/main/java/org/elasticsearch/cluster/coordination/PreVoteCollector.java b/server/src/main/java/org/elasticsearch/cluster/coordination/PreVoteCollector.java index 5001e3be18183..f683057b1fcd1 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/PreVoteCollector.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/PreVoteCollector.java @@ -35,12 +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.cluster.coordination.CoordinationState.isElectionQuorum; -import static org.elasticsearch.common.util.concurrent.ConcurrentCollections.newConcurrentSet; +import static org.elasticsearch.common.util.concurrent.ConcurrentCollections.newConcurrentMap; 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, @@ -128,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; @@ -185,11 +187,20 @@ private void handlePreVoteResponse(final PreVoteResponse response, final Discove return; } - preVotesReceived.add(sender); + preVotesReceived.put(sender, response); + + // create a fake VoteCollection based on the pre-votes and check if there is an election quorum final VoteCollection voteCollection = new VoteCollection(); - preVotesReceived.forEach(voteCollection::addVote); + final DiscoveryNode localNode = clusterState.nodes().getLocalNode(); + final PreVoteResponse localPreVoteResponse = getPreVoteResponse(); + + preVotesReceived.forEach((node, preVoteResponse) -> voteCollection.addJoinVote( + new Join(node, localNode, preVoteResponse.getCurrentTerm(), + preVoteResponse.getLastAcceptedTerm(), preVoteResponse.getLastAcceptedVersion()))); - if (isElectionQuorum(voteCollection, clusterState) == false) { + 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 f175f5d6f4bf2..6e9ad85aec56c 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).sorted().forEach(sb::append); + sb.append('}'); + } if (!attributes.isEmpty()) { sb.append(attributes); } 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..8fb325d594707 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; @@ -378,6 +379,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/main/java/org/elasticsearch/common/settings/ClusterSettings.java b/server/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java index 6e3e3f38feb57..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,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_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 51f14147bee9b..6fba263d66286 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_STRATEGY = "default"; + + public static final Setting ELECTION_STRATEGY_SETTING = + new Setting<>("cluster.election.strategy", DEFAULT_ELECTION_STRATEGY, 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_STRATEGY, ElectionStrategy.DEFAULT_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_STRATEGY_SETTING.get(settings)); + if (electionStrategy == null) { + throw new IllegalArgumentException("Unknown election strategy " + ELECTION_STRATEGY_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..4757386efc5e8 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,11 @@ default Map> getSeedHostProviders(TransportS * {@link IllegalStateException} if the node and the cluster-state are incompatible. */ 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 f16c7a7878ea9..850a1ec0b7abd 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.DEFAULT_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,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).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 +169,30 @@ 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 +222,31 @@ 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,104 +259,109 @@ 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) - .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 + "] 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) - .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) - .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 + "] 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 +381,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..99f04015867f5 100644 --- a/server/src/test/java/org/elasticsearch/cluster/coordination/CoordinationStateTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/coordination/CoordinationStateTests.java @@ -32,23 +32,17 @@ 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; -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 { @@ -240,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)); @@ -322,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())); @@ -766,12 +764,14 @@ 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.DEFAULT_INSTANCE) + .runRandomly(); } 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.DEFAULT_INSTANCE); } public static ClusterState clusterState(long term, long version, DiscoveryNode localNode, VotingConfiguration lastCommittedConfig, @@ -810,181 +810,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; - 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(Settings.EMPTY, localNode, persistedState); - } - - void reboot() { - state = new CoordinationState(Settings.EMPTY, localNode, persistedState); - } - - 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/server/src/test/java/org/elasticsearch/cluster/coordination/CoordinatorTests.java b/server/src/test/java/org/elasticsearch/cluster/coordination/CoordinatorTests.java index 9d654d29f6c81..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,1178 +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 -> {}); - 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/server/src/test/java/org/elasticsearch/cluster/coordination/NodeJoinTests.java b/server/src/test/java/org/elasticsearch/cluster/coordination/NodeJoinTests.java index 46608ab76b2d8..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 -> {}); + 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 78896752bae4d..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 -> { - }); // 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); } @@ -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.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 5fb56cc4aa3a3..5050b12531859 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.DEFAULT_INSTANCE); } final DiscoveryNode localNode; 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/server/src/test/java/org/elasticsearch/snapshots/SnapshotResiliencyTests.java b/server/src/test/java/org/elasticsearch/snapshots/SnapshotResiliencyTests.java index 6c6b136ba2751..fc2d22fb7029b 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.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 new file mode 100644 index 0000000000000..0547412f61500 --- /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.DEFAULT_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/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/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/core/src/main/java/org/elasticsearch/license/XPackLicenseState.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/license/XPackLicenseState.java index de2bb2af491b9..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 @@ -608,6 +608,15 @@ public synchronized boolean isRollupAllowed() { return status.active; } + /** + * 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 isVotingOnlyAllowed() { + 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 1930c05bc38eb..de8b8ecfcae6f 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 @@ -175,6 +175,7 @@ import org.elasticsearch.xpack.core.upgrade.actions.IndexUpgradeAction; import org.elasticsearch.xpack.core.upgrade.actions.IndexUpgradeInfoAction; import org.elasticsearch.xpack.core.vectors.VectorsFeatureSetUsage; +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; @@ -440,7 +441,9 @@ public List getNamedWriteables() { new NamedWriteableRegistry.Entry(Task.Status.class, DataFrameField.TASK_NAME, DataFrameTransformState::new), new NamedWriteableRegistry.Entry(PersistentTaskState.class, DataFrameField.TASK_NAME, DataFrameTransformState::new), // Vectors - new NamedWriteableRegistry.Entry(XPackFeatureSet.Usage.class, XPackField.VECTORS, VectorsFeatureSetUsage::new) + new NamedWriteableRegistry.Entry(XPackFeatureSet.Usage.class, XPackField.VECTORS, VectorsFeatureSetUsage::new), + // Voting Only Node + new NamedWriteableRegistry.Entry(XPackFeatureSet.Usage.class, XPackField.VOTING_ONLY, 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 e52fca72d4ca9..54e0f58ae2867 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 @@ -39,6 +39,8 @@ public final class XPackField { public static final String DATA_FRAME = "data_frame"; /** Name constant for the vectors feature. */ public static final String VECTORS = "vectors"; + /** Name constant for the voting-only-node feature. */ + public static final String VOTING_ONLY = "voting_only"; private XPackField() {} 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 6e52bffd5a5d8..ead19996007e5 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 @@ -34,9 +34,11 @@ public class XPackUsageFeatureAction extends Action { public static final XPackUsageFeatureAction CCR = new XPackUsageFeatureAction(XPackField.CCR); public static final XPackUsageFeatureAction DATA_FRAME = new XPackUsageFeatureAction(XPackField.DATA_FRAME); public static final XPackUsageFeatureAction VECTORS = new XPackUsageFeatureAction(XPackField.VECTORS); + public static final XPackUsageFeatureAction VOTING_ONLY_NODE = new XPackUsageFeatureAction(XPackField.VOTING_ONLY); public static final List ALL = Arrays.asList( - SECURITY, MONITORING, WATCHER, GRAPH, MACHINE_LEARNING, LOGSTASH, SQL, ROLLUP, INDEX_LIFECYCLE, CCR, DATA_FRAME, VECTORS + SECURITY, MONITORING, WATCHER, GRAPH, MACHINE_LEARNING, LOGSTASH, SQL, ROLLUP, INDEX_LIFECYCLE, CCR, DATA_FRAME, VECTORS, + 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..71e94e6d31860 --- /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) { + super(XPackField.VOTING_ONLY, available, true); + } +} 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/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 new file mode 100644 index 0000000000000..1c79c665f2d04 --- /dev/null +++ b/x-pack/plugin/voting-only-node/build.gradle @@ -0,0 +1,22 @@ +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'] +} + +dependencies { + compileOnly project(path: xpackModule('core'), configuration: 'default') + testCompile project(path: xpackModule('core'), configuration: 'testArtifacts') + if (isEclipse) { + testCompile project(path: xpackModule('core-tests'), configuration: 'testArtifacts') + } +} + +// 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/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..ef54d0ea5ccc2 --- /dev/null +++ b/x-pack/plugin/voting-only-node/src/main/java/org/elasticsearch/cluster/coordination/VotingOnlyNodeFeatureSet.java @@ -0,0 +1,71 @@ +/* + * 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; +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.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.action.XPackUsageFeatureAction; +import org.elasticsearch.xpack.core.action.XPackUsageFeatureResponse; +import org.elasticsearch.xpack.core.action.XPackUsageFeatureTransportAction; +import org.elasticsearch.xpack.core.votingonly.VotingOnlyNodeFeatureSetUsage; + +public class VotingOnlyNodeFeatureSet implements XPackFeatureSet { + + private final XPackLicenseState licenseState; + + @Inject + public VotingOnlyNodeFeatureSet(@Nullable XPackLicenseState licenseState) { + this.licenseState = licenseState; + } + + @Override + public String name() { + return XPackField.VOTING_ONLY; + } + + @Override + public boolean available() { + return licenseState != null && licenseState.isVotingOnlyAllowed(); + } + + @Override + public boolean enabled() { + return true; + } + + public static class UsageTransportAction extends XPackUsageFeatureTransportAction { + + private final XPackLicenseState licenseState; + + @Inject + public UsageTransportAction(TransportService transportService, ClusterService clusterService, ThreadPool threadPool, + ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver, + XPackLicenseState licenseState) { + super(XPackUsageFeatureAction.VOTING_ONLY_NODE.name(), transportService, clusterService, + threadPool, actionFilters, indexNameExpressionResolver); + this.licenseState = licenseState; + } + + @Override + protected void masterOperation(XPackUsageRequest request, ClusterState state, ActionListener listener) { + final boolean available = licenseState.isVotingOnlyAllowed(); + final VotingOnlyNodeFeatureSetUsage usage = + 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 new file mode 100644 index 0000000000000..0c062ee1df591 --- /dev/null +++ b/x-pack/plugin/voting-only-node/src/main/java/org/elasticsearch/cluster/coordination/VotingOnlyNodePlugin.java @@ -0,0 +1,219 @@ +/* + * 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.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.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; +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; +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.ActionPlugin; +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 org.elasticsearch.xpack.core.XPackPlugin; +import org.elasticsearch.xpack.core.action.XPackUsageFeatureAction; + +import java.io.IOException; +import java.util.ArrayList; +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; + +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); + + private static final String VOTING_ONLY_ELECTION_STRATEGY = "supports_voting_only"; + + 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; + + 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) { + throw new IllegalStateException("voting-only node must be master-eligible"); + } + return Collections.singleton(VOTING_ONLY_NODE_ROLE); + } + + @Override + 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(); + } + + @Override + public List> getActions() { + return singletonList(new ActionHandler<>(XPackUsageFeatureAction.VOTING_ONLY_NODE, UsageTransportAction.class)); + } + + @Override + public Collection createGuiceModules() { + List modules = new ArrayList<>(); + modules.add(b -> XPackPlugin.bindFeatureSet(b, VotingOnlyNodeFeatureSet.class)); + return modules; + } + + @Override + public Map getElectionStrategies() { + return Collections.singletonMap(VOTING_ONLY_ELECTION_STRATEGY, new VotingOnlyNodeElectionStrategy()); + } + + @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(); + } + } + + @Override + public Settings additionalSettings() { + return Settings.builder().put(DiscoveryModule.ELECTION_STRATEGY_SETTING.getKey(), VOTING_ONLY_ELECTION_STRATEGY).build(); + } + + static class VotingOnlyNodeElectionStrategy extends ElectionStrategy { + + @Override + 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) + 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)) { + return false; + } + } + return true; + } + } + + 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 used purely for state transfer: " + 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); + } + }); + } else { + threadPoolSupplier.get().generic().execute(() -> handler.handleException(new TransportException( + 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 new file mode 100644 index 0000000000000..aaacb6b76edfe --- /dev/null +++ b/x-pack/plugin/voting-only-node/src/test/java/org/elasticsearch/cluster/coordination/LocalStateVotingOnlyNodePlugin.java @@ -0,0 +1,20 @@ +/* + * 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.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); + + plugins.add(new VotingOnlyNodePlugin(settings)); + } +} 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(); + } + +} 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); + } + +} 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 new file mode 100644 index 0000000000000..e6d9cbb3c2d2f --- /dev/null +++ b/x-pack/plugin/voting-only-node/src/test/java/org/elasticsearch/cluster/coordination/VotingOnlyNodePluginTests.java @@ -0,0 +1,93 @@ +/* + * 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.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 java.util.Collection; +import java.util.Collections; + +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) +public class VotingOnlyNodePluginTests extends ESIntegTestCase { + + @Override + protected Collection> nodePlugins() { + return Collections.singleton(LocalStateVotingOnlyNodePlugin.class); + } + + 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 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)); + 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 { + 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))); + final String oldMasterId = client().admin().cluster().prepareState().get().getState().nodes().getMasterNodeId(); + + internalCluster().stopCurrentMasterNode(); + + 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); + } +}