From ef976f8a29d1cec541ce84b1d809b18bec871616 Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Thu, 9 Aug 2018 14:28:38 +0200 Subject: [PATCH 01/29] Node joining --- .../cluster/coordination/Coordinator.java | 295 ++++++++++++++++++ .../cluster/coordination/JoinRequest.java | 26 ++ .../discovery/zen/NodeJoinController.java | 6 +- 3 files changed, 324 insertions(+), 3 deletions(-) create mode 100644 server/src/main/java/org/elasticsearch/cluster/coordination/Coordinator.java create mode 100644 server/src/main/java/org/elasticsearch/cluster/coordination/JoinRequest.java diff --git a/server/src/main/java/org/elasticsearch/cluster/coordination/Coordinator.java b/server/src/main/java/org/elasticsearch/cluster/coordination/Coordinator.java new file mode 100644 index 0000000000000..c65e69f151417 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/Coordinator.java @@ -0,0 +1,295 @@ +package org.elasticsearch.cluster.coordination; + +import org.apache.lucene.util.SetOnce; +import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.ClusterStateTaskConfig; +import org.elasticsearch.cluster.ClusterStateTaskListener; +import org.elasticsearch.cluster.node.DiscoveryNode; +import org.elasticsearch.cluster.routing.allocation.AllocationService; +import org.elasticsearch.cluster.service.MasterService; +import org.elasticsearch.common.Priority; +import org.elasticsearch.common.component.AbstractLifecycleComponent; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.discovery.zen.ElectMasterService; +import org.elasticsearch.discovery.zen.MembershipAction; +import org.elasticsearch.discovery.zen.NodeJoinController; +import org.elasticsearch.discovery.zen.NodeJoinController.JoinTaskExecutor; +import org.elasticsearch.transport.TransportService; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; + +//TODO: add a test that sends random joins and checks if the join led to success (the fake masterservice fully manages state) +// i.e. similar to NodeJoinControllerTests +public class Coordinator extends AbstractLifecycleComponent { + + private final Object mutex = new Object(); + private Mode mode; + private Optional lastKnownLeader; + private Optional lastJoin; + private final SetOnce consensusState = new SetOnce<>(); + private final TransportService transportService; + private final MasterService masterService; + + + private final JoinTaskExecutor joinTaskExecutor; + + // similar to NodeJoinController.ElectionContext.joinRequestAccumulator, captures joins on election + private final Map joinRequestAccumulator = new HashMap<>(); + + + public Coordinator(Settings settings, TransportService transportService, AllocationService allocationService, + MasterService masterService) { + super(settings); + this.transportService = transportService; + this.masterService = masterService; + this.mode = Mode.CANDIDATE; + lastKnownLeader = Optional.empty(); + lastJoin = Optional.empty(); + // disable minimum_master_nodes check + final ElectMasterService electMasterService = new ElectMasterService(settings) { + + @Override + public boolean hasEnoughMasterNodes(Iterable nodes) { + return true; + } + + @Override + public void logMinimumMasterNodesWarningIfNecessary(ClusterState oldState, ClusterState newState) { + // ignore + } + + }; + + // reuse JoinTaskExecutor implementation from ZenDiscovery, but hack some checks + this.joinTaskExecutor = new JoinTaskExecutor(allocationService, electMasterService, logger) { + + @Override + public ClusterTasksResult execute(ClusterState currentState, List joiningNodes) throws Exception { + // This is called when preparing the next cluster state for publication. There is no guarantee that the term we see here is + // the term under which this state will eventually be published: the current term may be increased after this check due to + // some other activity. That the term is correct is, however, checked properly during publication, so it is sufficient to + // check it here on a best-effort basis. This is fine because a concurrent change indicates the existence of another leader + // in a higher term which will cause this node to stand down. + + final long currentTerm = getCurrentTerm(); // TODO perhaps this can be a volatile read? + if (currentState.term() != currentTerm) { + currentState = ClusterState.builder(currentState).term(currentTerm).build(); + } + return super.execute(currentState, joiningNodes); + } + + }; + } + + // package-visible for testing + long getCurrentTerm() { + synchronized (mutex) { + return consensusState.get().getCurrentTerm(); + } + } + + // package-visible for testing + Mode getMode() { + synchronized (mutex) { + return mode; + } + } + + private Optional ensureTermAtLeast(DiscoveryNode sourceNode, long targetTerm) { + assert Thread.holdsLock(mutex) : "Legislator mutex not held"; + if (getCurrentTerm() < targetTerm) { + return Optional.of(joinLeaderInTerm(new StartJoinRequest(sourceNode, targetTerm))); + } + return Optional.empty(); + } + + private Join joinLeaderInTerm(StartJoinRequest startJoinRequest) { + assert Thread.holdsLock(mutex) : "Legislator mutex not held"; + logger.debug("joinLeaderInTerm: from [{}] with term {}", startJoinRequest.getSourceNode(), startJoinRequest.getTerm()); + Join join = consensusState.get().handleStartJoin(startJoinRequest); + lastJoin = Optional.of(join); + if (mode == Mode.CANDIDATE) { + // refresh required because current term has changed + // TODO: heartbeatRequestResponder = new Legislator.HeartbeatRequestResponder(); + } else { + // becomeCandidate refreshes responders + becomeCandidate("joinLeaderInTerm"); + } + return join; + } + + public void handleJoinRequest(JoinRequest joinRequest, MembershipAction.JoinCallback joinCallback) { + + transportService.connectToNode(joinRequest.getSourceNode()); + + synchronized (mutex) { + handleJoinRequestUnderLock(joinRequest, joinCallback); + } + } + + private void handleJoinRequestUnderLock(JoinRequest joinRequest, MembershipAction.JoinCallback joinCallback) { + assert Thread.holdsLock(mutex) : "Legislator mutex not held"; + logger.trace("handleJoinRequestUnderLock: as {}, handling {}", mode, joinRequest); + if (mode == Mode.LEADER) { + // submit as cluster state update task + masterService.submitStateUpdateTask("zen-disco-node-join", + joinRequest.getSourceNode(), ClusterStateTaskConfig.build(Priority.URGENT), + joinTaskExecutor, new NodeJoinController.JoinTaskListener(joinCallback, logger)); + } else { + MembershipAction.JoinCallback prev = joinRequestAccumulator.put(joinRequest.getSourceNode(), joinCallback); + if (prev != null) { + prev.onFailure(new CoordinationStateRejectedException("received a newer join from " + joinRequest.getSourceNode())); + } + } + + try { + joinRequest.getOptionalJoin().ifPresent(this::handleJoin); + } catch (CoordinationStateRejectedException exception) { + final MembershipAction.JoinCallback callback = joinRequestAccumulator.remove(joinRequest.getSourceNode()); + if (callback == null) { + logger.trace("handleJoinRequestUnderLock: submitted task to master service, but vote was rejected", exception); + } else { + callback.onFailure(exception); + } + } + } + + private void handleJoin(Join join) { + assert Thread.holdsLock(mutex) : "Legislator mutex not held"; + Optional optionalJoinToSelf = ensureTermAtLeast(getLocalNode(), join.getTerm()); + + optionalJoinToSelf.ifPresent(this::handleJoin); // if someone thinks we should be master, let's try to become one + + final CoordinationState state = consensusState.get(); + boolean prevElectionWon = state.electionWon(); + boolean addedJoin = state.handleJoin(join); + assert !prevElectionWon || state.electionWon(); // we cannot go from won to not won + +// if (addedJoin && mode == Legislator.Mode.LEADER && currentPublication.isPresent() == false) { +// scheduleReconfigurationIfSuboptimal(); +// } + + if (prevElectionWon == false && state.electionWon()) { + assert mode == Mode.CANDIDATE : "expected candidate but was " + mode; + + becomeLeader("handleJoin"); + + Map pendingAsTasks = getPendingAsTasks(); + joinRequestAccumulator.clear(); + + final String source = "zen-disco-elected-as-master ([" + pendingAsTasks.size() + "] nodes joined)"; + // noop listener, the election finished listener determines result + pendingAsTasks.put(NodeJoinController.BECOME_MASTER_TASK, (source1, e) -> { + }); + // TODO: should we take any further action when FINISH_ELECTION_TASK fails? + pendingAsTasks.put(NodeJoinController.FINISH_ELECTION_TASK, (source1, e) -> { + }); + + try { + masterService.submitStateUpdateTasks(source, pendingAsTasks, ClusterStateTaskConfig.build(Priority.URGENT), + joinTaskExecutor); + } catch (IllegalStateException e) { + // TODO fix NodeJoinController and remove this + assert e.getMessage().contains("is already queued"); + becomeCandidate("handleJoin"); + } + } + } + + // copied from NodeJoinController.getPendingAsTasks + private Map getPendingAsTasks() { + assert Thread.holdsLock(mutex) : "Legislator mutex not held"; + Map tasks = new HashMap<>(); + joinRequestAccumulator.forEach((key, value) -> tasks.put(key, + new NodeJoinController.JoinTaskListener(value, logger))); + return tasks; + } + + private void clearJoins() { + assert Thread.holdsLock(mutex) : "Legislator mutex not held"; + joinRequestAccumulator.values().forEach( + joinCallback -> joinCallback.onFailure(new CoordinationStateRejectedException("node stepped down as leader"))); + joinRequestAccumulator.clear(); + } + + private void becomeCandidate(String method) { + assert Thread.holdsLock(mutex) : "Legislator mutex not held"; + logger.debug("{}: becoming CANDIDATE (was {}, lastKnownLeader was [{}])", method, mode, lastKnownLeader); + + if (mode != Mode.CANDIDATE) { + mode = Mode.CANDIDATE; + + clearJoins(); + } + } + + private void becomeLeader(String method) { + assert Thread.holdsLock(mutex) : "Legislator mutex not held"; + assert mode != Mode.LEADER; + + logger.debug("{}: becoming LEADER (was {}, lastKnownLeader was [{}])", method, mode, lastKnownLeader); + + mode = Mode.LEADER; + lastKnownLeader = Optional.of(getLocalNode()); + } + + private void becomeFollower(String method, DiscoveryNode leaderNode) { + assert Thread.holdsLock(mutex) : "Legislator mutex not held"; + + if (mode != Mode.FOLLOWER) { + logger.debug("{}: becoming FOLLOWER of [{}] (was {}, lastKnownLeader was [{}])", method, leaderNode, mode, lastKnownLeader); + + mode = Mode.FOLLOWER; + + clearJoins(); + } + + lastKnownLeader = Optional.of(leaderNode); + } + + public DiscoveryNode getLocalNode() { + return transportService.getLocalNode(); + } + + @Override + protected void doStart() { + + } + + public void startInitialJoin() { + synchronized (mutex) { + becomeCandidate("startInitialJoin"); + } + } + + @Override + protected void doStop() { + + } + + @Override + protected void doClose() { + + } + + public void invariant() { + synchronized (mutex) { + if (mode == Mode.LEADER) { + assert consensusState.get().electionWon(); + assert lastKnownLeader.isPresent() && lastKnownLeader.get().equals(getLocalNode()); + } else if (mode == Mode.FOLLOWER) { + assert consensusState.get().electionWon() == false : getLocalNode() + " is FOLLOWER so electionWon() should be false"; + assert lastKnownLeader.isPresent() && (lastKnownLeader.get().equals(getLocalNode()) == false); + } else { + assert mode == Mode.CANDIDATE; + } + } + } + + public enum Mode { + CANDIDATE, LEADER, FOLLOWER + } +} diff --git a/server/src/main/java/org/elasticsearch/cluster/coordination/JoinRequest.java b/server/src/main/java/org/elasticsearch/cluster/coordination/JoinRequest.java new file mode 100644 index 0000000000000..92da254a1118b --- /dev/null +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/JoinRequest.java @@ -0,0 +1,26 @@ +package org.elasticsearch.cluster.coordination; + +import org.elasticsearch.cluster.node.DiscoveryNode; + +import java.util.Optional; + +public class JoinRequest { + + private final DiscoveryNode sourceNode; + + private final Optional optionalJoin; + + public JoinRequest(DiscoveryNode sourceNode, Optional optionalJoin) { + this.sourceNode = sourceNode; + this.optionalJoin = optionalJoin; + } + + public DiscoveryNode getSourceNode() { + return sourceNode; + } + + public Optional getOptionalJoin() { + return optionalJoin; + } + +} diff --git a/server/src/main/java/org/elasticsearch/discovery/zen/NodeJoinController.java b/server/src/main/java/org/elasticsearch/discovery/zen/NodeJoinController.java index 913008b5c6d20..0bf9296aca511 100644 --- a/server/src/main/java/org/elasticsearch/discovery/zen/NodeJoinController.java +++ b/server/src/main/java/org/elasticsearch/discovery/zen/NodeJoinController.java @@ -341,15 +341,15 @@ public void onFailure(String source, Exception e) { } - static class JoinTaskListener implements ClusterStateTaskListener { + public static class JoinTaskListener implements ClusterStateTaskListener { final List callbacks; private final Logger logger; - JoinTaskListener(MembershipAction.JoinCallback callback, Logger logger) { + public JoinTaskListener(MembershipAction.JoinCallback callback, Logger logger) { this(Collections.singletonList(callback), logger); } - JoinTaskListener(List callbacks, Logger logger) { + public JoinTaskListener(List callbacks, Logger logger) { this.callbacks = callbacks; this.logger = logger; } From 348fb4e65cb90ff203d5d2997bc58ec906c61d0a Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Thu, 9 Aug 2018 16:24:03 +0200 Subject: [PATCH 02/29] start testing --- .../cluster/coordination/Coordinator.java | 8 +- .../cluster/coordination/NodeJoinTests.java | 196 ++++++++++++++++++ 2 files changed, 202 insertions(+), 2 deletions(-) create mode 100644 server/src/test/java/org/elasticsearch/cluster/coordination/NodeJoinTests.java 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 c65e69f151417..76cfd2dd738e2 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/Coordinator.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/Coordinator.java @@ -20,6 +20,7 @@ import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.function.Supplier; //TODO: add a test that sends random joins and checks if the join led to success (the fake masterservice fully manages state) // i.e. similar to NodeJoinControllerTests @@ -35,16 +36,18 @@ public class Coordinator extends AbstractLifecycleComponent { private final JoinTaskExecutor joinTaskExecutor; + private final Supplier persistedStateSupplier; // similar to NodeJoinController.ElectionContext.joinRequestAccumulator, captures joins on election private final Map joinRequestAccumulator = new HashMap<>(); public Coordinator(Settings settings, TransportService transportService, AllocationService allocationService, - MasterService masterService) { + MasterService masterService, Supplier persistedStateSupplier) { super(settings); this.transportService = transportService; this.masterService = masterService; + this.persistedStateSupplier = persistedStateSupplier; this.mode = Mode.CANDIDATE; lastKnownLeader = Optional.empty(); lastJoin = Optional.empty(); @@ -256,7 +259,8 @@ public DiscoveryNode getLocalNode() { @Override protected void doStart() { - + CoordinationState.PersistedState persistedState = persistedStateSupplier.get(); + consensusState.set(new CoordinationState(settings, getLocalNode(), persistedState)); } public void startInitialJoin() { diff --git a/server/src/test/java/org/elasticsearch/cluster/coordination/NodeJoinTests.java b/server/src/test/java/org/elasticsearch/cluster/coordination/NodeJoinTests.java new file mode 100644 index 0000000000000..2e01c4c24d5a1 --- /dev/null +++ b/server/src/test/java/org/elasticsearch/cluster/coordination/NodeJoinTests.java @@ -0,0 +1,196 @@ +package org.elasticsearch.cluster.coordination; + +import org.apache.logging.log4j.message.ParameterizedMessage; +import org.elasticsearch.Version; +import org.elasticsearch.cluster.ClusterName; +import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.ClusterState.VotingConfiguration; +import org.elasticsearch.cluster.ESAllocationTestCase; +import org.elasticsearch.cluster.block.ClusterBlocks; +import org.elasticsearch.cluster.node.DiscoveryNode; +import org.elasticsearch.cluster.node.DiscoveryNodes; +import org.elasticsearch.cluster.service.MasterService; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.util.concurrent.BaseFuture; +import org.elasticsearch.discovery.zen.MembershipAction; +import org.elasticsearch.test.ClusterServiceUtils; +import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.threadpool.TestThreadPool; +import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.transport.TransportService; +import org.junit.After; +import org.junit.AfterClass; +import org.junit.BeforeClass; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; + +import static java.util.Collections.emptyMap; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public class NodeJoinTests extends ESTestCase { + + private static ThreadPool threadPool; + + private MasterService masterService; + private Coordinator coordinator; + + @BeforeClass + public static void beforeClass() { + threadPool = new TestThreadPool("NodeJoinControllerTests"); + } + + @AfterClass + public static void afterClass() { + ThreadPool.terminate(threadPool, 30, TimeUnit.SECONDS); + threadPool = null; + } + + @After + public void tearDown() throws Exception { + super.tearDown(); + masterService.close(); + } + + private static ClusterState initialState(boolean withMaster, DiscoveryNode localNode, long term, VotingConfiguration config) { + ClusterState initialClusterState = ClusterState.builder(new ClusterName(ClusterServiceUtils.class.getSimpleName())) + .nodes(DiscoveryNodes.builder() + .add(localNode) + .localNodeId(localNode.getId()) + .masterNodeId(withMaster ? localNode.getId() : null)) + .term(term) + .version(1) + .lastAcceptedConfiguration(config) + .lastCommittedConfiguration(config) + .blocks(ClusterBlocks.EMPTY_CLUSTER_BLOCK).build(); + return initialClusterState; + } + + private void setupMasterServiceAndNodeJoinController(long term, ClusterState initialState) { + if (masterService != null || coordinator != null) { + throw new IllegalStateException("method setupMasterServiceAndNodeJoinController can only be called once"); + } + masterService = ClusterServiceUtils.createMasterService(threadPool, initialState); + TransportService transportService = mock(TransportService.class); + when(transportService.getLocalNode()).thenReturn(initialState.nodes().getLocalNode()); + coordinator = new Coordinator(Settings.EMPTY, + transportService, + ESAllocationTestCase.createAllocationService(Settings.EMPTY), + masterService, + () -> new CoordinationStateTests.InMemoryPersistedState(term, initialState)); + coordinator.start(); + coordinator.startInitialJoin(); + } + + protected DiscoveryNode newNode(int i) { + return newNode(i, randomBoolean()); + } + + protected DiscoveryNode newNode(int i, boolean master) { + Set roles = new HashSet<>(); + if (master) { + roles.add(DiscoveryNode.Role.MASTER); + } + final String prefix = master ? "master_" : "data_"; + return new DiscoveryNode(prefix + i, i + "", buildNewFakeTransportAddress(), emptyMap(), roles, Version.CURRENT); + } + + private DiscoveryNode cloneNode(DiscoveryNode node) { + return new DiscoveryNode(node.getName(), node.getId(), node.getEphemeralId(), node.getHostName(), node.getHostAddress(), + node.getAddress(), node.getAttributes(), node.getRoles(), node.getVersion()); + } + + private void joinNode(final JoinRequest joinRequest) throws InterruptedException, ExecutionException { + joinNodeAsync(joinRequest).get(); + } + + static class SimpleFuture extends BaseFuture { + final String description; + + SimpleFuture(String description) { + this.description = description; + } + + public void markAsDone() { + set(null); + } + + public void markAsFailed(Throwable t) { + setException(t); + } + + @Override + public String toString() { + return "future [" + description + "]"; + } + } + + private SimpleFuture joinNodeAsync(final JoinRequest joinRequest) { + final SimpleFuture future = new SimpleFuture("join of " + joinRequest + "]"); + logger.debug("starting {}", future); + // clone the node before submitting to simulate an incoming join, which is guaranteed to have a new + // disco node object serialized off the network + coordinator.handleJoinRequest(joinRequest, new MembershipAction.JoinCallback() { + @Override + public void onSuccess() { + logger.debug("{} completed", future); + future.markAsDone(); + } + + @Override + public void onFailure(Exception e) { + logger.error(() -> new ParameterizedMessage("unexpected error for {}", future), e); + future.markAsFailed(e); + } + }); + return future; + } + + public void testSimpleJoinAccumulation() throws InterruptedException, ExecutionException { + DiscoveryNode localNode = new DiscoveryNode("node", ESTestCase.buildNewFakeTransportAddress(), Collections.emptyMap(), + new HashSet<>(Arrays.asList(DiscoveryNode.Role.values())), Version.CURRENT); + setupMasterServiceAndNodeJoinController(1L, + initialState(true, localNode, 1L, new VotingConfiguration(Collections.singleton(localNode.getId())))); + List nodes = new ArrayList<>(); + nodes.add(localNode); + + int nodeId = 0; + for (int i = randomInt(5); i > 0; i--) { + DiscoveryNode node = newNode(nodeId++); + nodes.add(node); + JoinRequest joinRequest = new JoinRequest(node, Optional.of(new Join(node, localNode, 2, 2, 3))); + joinNode(joinRequest); + } +// nodeJoinController.startElectionContext(); +// ArrayList> pendingJoins = new ArrayList<>(); +// for (int i = randomInt(5); i > 0; i--) { +// DiscoveryNode node = newNode(nodeId++); +// nodes.add(node); +// pendingJoins.add(joinNodeAsync(node)); +// } +// nodeJoinController.stopElectionContext("test"); +// boolean hadSyncJoin = false; +// for (int i = randomInt(5); i > 0; i--) { +// DiscoveryNode node = newNode(nodeId++); +// nodes.add(node); +// joinNode(node); +// hadSyncJoin = true; +// } +// if (hadSyncJoin) { +// for (Future joinFuture : pendingJoins) { +// assertThat(joinFuture.isDone(), equalTo(true)); +// } +// } +// for (Future joinFuture : pendingJoins) { +// joinFuture.get(); +// } + } +} From 554263a0ee67636c67b06f696e9ec165fbcedef0 Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Thu, 9 Aug 2018 16:38:23 +0200 Subject: [PATCH 03/29] request serialozatono --- .../cluster/coordination/JoinRequest.java | 44 ++++++++++++++++++- .../cluster/coordination/MessagesTests.java | 24 ++++++++++ 2 files changed, 67 insertions(+), 1 deletion(-) diff --git a/server/src/main/java/org/elasticsearch/cluster/coordination/JoinRequest.java b/server/src/main/java/org/elasticsearch/cluster/coordination/JoinRequest.java index 92da254a1118b..1b6170fb040ae 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/JoinRequest.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/JoinRequest.java @@ -1,10 +1,14 @@ package org.elasticsearch.cluster.coordination; import org.elasticsearch.cluster.node.DiscoveryNode; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.transport.TransportRequest; +import java.io.IOException; import java.util.Optional; -public class JoinRequest { +public class JoinRequest extends TransportRequest { private final DiscoveryNode sourceNode; @@ -15,6 +19,19 @@ public JoinRequest(DiscoveryNode sourceNode, Optional optionalJoin) { this.optionalJoin = optionalJoin; } + public JoinRequest(StreamInput in) throws IOException { + super(in); + sourceNode = new DiscoveryNode(in); + optionalJoin = Optional.ofNullable(in.readOptionalWriteable(Join::new)); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + sourceNode.writeTo(out); + out.writeOptionalWriteable(optionalJoin.orElse(null)); + } + public DiscoveryNode getSourceNode() { return sourceNode; } @@ -23,4 +40,29 @@ public Optional getOptionalJoin() { return optionalJoin; } + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (!(o instanceof JoinRequest)) return false; + + JoinRequest that = (JoinRequest) o; + + if (!sourceNode.equals(that.sourceNode)) return false; + return optionalJoin.equals(that.optionalJoin); + } + + @Override + public int hashCode() { + int result = sourceNode.hashCode(); + result = 31 * result + optionalJoin.hashCode(); + return result; + } + + @Override + public String toString() { + return "JoinRequest{" + + "sourceNode=" + sourceNode + + ", optionalJoin=" + optionalJoin + + '}'; + } } diff --git a/server/src/test/java/org/elasticsearch/cluster/coordination/MessagesTests.java b/server/src/test/java/org/elasticsearch/cluster/coordination/MessagesTests.java index 926287503d84e..5914587b91a5f 100644 --- a/server/src/test/java/org/elasticsearch/cluster/coordination/MessagesTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/coordination/MessagesTests.java @@ -168,6 +168,30 @@ public void testApplyCommitEqualsHashCodeSerialization() { }); } + public void testJoinRequestEqualsHashCodeSerialization() { + Join initialJoin = new Join(createNode(randomAlphaOfLength(10)), createNode(randomAlphaOfLength(10)), randomNonNegativeLong(), + randomNonNegativeLong(), randomNonNegativeLong()); + JoinRequest initialJoinRequest = new JoinRequest(initialJoin.getSourceNode(), + randomBoolean() ? Optional.empty() : Optional.of(initialJoin)); + EqualsHashCodeTestUtils.checkEqualsAndHashCode(initialJoinRequest, + joinRequest -> copyWriteable(joinRequest, writableRegistry(), JoinRequest::new), + joinRequest -> { + if (randomBoolean()) { + return new JoinRequest(createNode(randomAlphaOfLength(20)), joinRequest.getOptionalJoin()); + } else { + // change OptionalJoin + final Optional newOptionalJoin; + if (joinRequest.getOptionalJoin().isPresent() && randomBoolean()) { + newOptionalJoin = Optional.empty(); + } else { + newOptionalJoin = Optional.of(new Join(createNode(randomAlphaOfLength(10)), createNode(randomAlphaOfLength(10)), + randomNonNegativeLong(), randomNonNegativeLong(), randomNonNegativeLong())); + } + return new JoinRequest(joinRequest.getSourceNode(), newOptionalJoin); + } + }); + } + public ClusterState randomClusterState() { return CoordinationStateTests.clusterState(randomNonNegativeLong(), randomNonNegativeLong(), createNode(randomAlphaOfLength(10)), new ClusterState.VotingConfiguration(Sets.newHashSet(generateRandomStringArray(10, 10, false))), From 00463fe10f1da656d1fda2cac1a6ee751d9ed22a Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Thu, 9 Aug 2018 17:58:44 +0200 Subject: [PATCH 04/29] more testing --- .../cluster/coordination/Coordinator.java | 69 ++++++++++++++ .../cluster/coordination/NodeJoinTests.java | 91 +++++++++++++++---- 2 files changed, 143 insertions(+), 17 deletions(-) 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 76cfd2dd738e2..f59505d57fcf3 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/Coordinator.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/Coordinator.java @@ -1,5 +1,6 @@ package org.elasticsearch.cluster.coordination; +import org.apache.logging.log4j.message.ParameterizedMessage; import org.apache.lucene.util.SetOnce; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterStateTaskConfig; @@ -9,13 +10,21 @@ import org.elasticsearch.cluster.service.MasterService; import org.elasticsearch.common.Priority; import org.elasticsearch.common.component.AbstractLifecycleComponent; +import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.discovery.zen.ElectMasterService; import org.elasticsearch.discovery.zen.MembershipAction; import org.elasticsearch.discovery.zen.NodeJoinController; import org.elasticsearch.discovery.zen.NodeJoinController.JoinTaskExecutor; +import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.threadpool.ThreadPool.Names; +import org.elasticsearch.transport.TransportException; +import org.elasticsearch.transport.TransportResponse; +import org.elasticsearch.transport.TransportResponse.Empty; +import org.elasticsearch.transport.TransportResponseHandler; import org.elasticsearch.transport.TransportService; +import java.io.IOException; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -26,6 +35,8 @@ // i.e. similar to NodeJoinControllerTests public class Coordinator extends AbstractLifecycleComponent { + public static final String JOIN_ACTION_NAME = "internal:discovery/zen2/join"; + private final Object mutex = new Object(); private Mode mode; private Optional lastKnownLeader; @@ -87,6 +98,64 @@ public ClusterTasksResult execute(ClusterState currentState, List }; } + private void registerTransportActions() { + transportService.registerRequestHandler(JOIN_ACTION_NAME, Names.GENERIC, false, false, + JoinRequest::new, + (request, channel, task) -> handleJoinRequest(request, new MembershipAction.JoinCallback() { + @Override + public void onSuccess() { + try { + channel.sendResponse(Empty.INSTANCE); + } catch (IOException e) { + onFailure(e); + } + } + + @Override + public void onFailure(Exception e) { + try { + channel.sendResponse(e); + } catch (Exception inner) { + inner.addSuppressed(e); + logger.warn("failed to send back failure on join request", inner); + } + } + })); + } + + // TODO: maybe add this method in a follow-up. + private void sendOptionalJoin(DiscoveryNode destination, Optional optionalJoin) { + // No synchronisation required + transportService.sendRequest(destination, JOIN_ACTION_NAME, new JoinRequest(getLocalNode(), optionalJoin), + new TransportResponseHandler() { + @Override + public Empty read(StreamInput in) { + return Empty.INSTANCE; + } + + @Override + public void handleResponse(Empty response) { + // No synchronisation required + logger.debug("SendJoinResponseHandler: successfully joined {}", destination); + } + + @Override + public void handleException(TransportException exp) { + // No synchronisation required + if (exp.getRootCause() instanceof CoordinationStateRejectedException) { + logger.debug("SendJoinResponseHandler: [{}] failed: {}", destination, exp.getRootCause().getMessage()); + } else { + logger.debug(() -> new ParameterizedMessage("SendJoinResponseHandler: [{}] failed", destination), exp); + } + } + + @Override + public String executor() { + return Names.GENERIC; + } + }); + } + // package-visible for testing long getCurrentTerm() { synchronized (mutex) { 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 2e01c4c24d5a1..b8c417ce83ca9 100644 --- a/server/src/test/java/org/elasticsearch/cluster/coordination/NodeJoinTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/coordination/NodeJoinTests.java @@ -10,11 +10,13 @@ import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.cluster.service.MasterService; +import org.elasticsearch.cluster.service.MasterServiceTests; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.concurrent.BaseFuture; import org.elasticsearch.discovery.zen.MembershipAction; import org.elasticsearch.test.ClusterServiceUtils; import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.test.junit.annotations.TestLogging; import org.elasticsearch.threadpool.TestThreadPool; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; @@ -31,11 +33,14 @@ import java.util.Set; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; +import java.util.stream.IntStream; import static java.util.Collections.emptyMap; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; +@TestLogging("org.elasticsearch.discovery.zen:TRACE,org.elasticsearch.cluster.service:TRACE,org.elasticsearch.cluster.coordination:TRACE") public class NodeJoinTests extends ESTestCase { private static ThreadPool threadPool; @@ -60,14 +65,15 @@ public void tearDown() throws Exception { masterService.close(); } - private static ClusterState initialState(boolean withMaster, DiscoveryNode localNode, long term, VotingConfiguration config) { + private static ClusterState initialState(boolean withMaster, DiscoveryNode localNode, long term, long version, + VotingConfiguration config) { ClusterState initialClusterState = ClusterState.builder(new ClusterName(ClusterServiceUtils.class.getSimpleName())) .nodes(DiscoveryNodes.builder() .add(localNode) .localNodeId(localNode.getId()) .masterNodeId(withMaster ? localNode.getId() : null)) .term(term) - .version(1) + .version(version) .lastAcceptedConfiguration(config) .lastCommittedConfiguration(config) .blocks(ClusterBlocks.EMPTY_CLUSTER_BLOCK).build(); @@ -154,21 +160,72 @@ public void onFailure(Exception e) { return future; } - public void testSimpleJoinAccumulation() throws InterruptedException, ExecutionException { - DiscoveryNode localNode = new DiscoveryNode("node", ESTestCase.buildNewFakeTransportAddress(), Collections.emptyMap(), - new HashSet<>(Arrays.asList(DiscoveryNode.Role.values())), Version.CURRENT); - setupMasterServiceAndNodeJoinController(1L, - initialState(true, localNode, 1L, new VotingConfiguration(Collections.singleton(localNode.getId())))); - List nodes = new ArrayList<>(); - nodes.add(localNode); - - int nodeId = 0; - for (int i = randomInt(5); i > 0; i--) { - DiscoveryNode node = newNode(nodeId++); - nodes.add(node); - JoinRequest joinRequest = new JoinRequest(node, Optional.of(new Join(node, localNode, 2, 2, 3))); - joinNode(joinRequest); - } + public void testSuccessfulJoinAccumulation() { + List nodes = IntStream.rangeClosed(1, randomIntBetween(2, 5)) + .mapToObj(nodeId -> newNode(nodeId, true)).collect(Collectors.toList()); + + VotingConfiguration votingConfiguration = new VotingConfiguration( + randomSubsetOf(randomIntBetween(1, nodes.size()), nodes).stream().map(DiscoveryNode::getId).collect(Collectors.toSet())); + + DiscoveryNode localNode = nodes.get(0); + setupMasterServiceAndNodeJoinController(1, initialState(false, localNode, 1, 1, votingConfiguration)); + + // we need at least a quorum of voting nodes with a correct term + + List threads = randomSubsetOf(nodes).stream().map(node -> new Thread(() -> { + JoinRequest joinRequest = new JoinRequest(node, Optional.of(new Join(node, localNode, 2, 1, 1))); + try { + joinNode(joinRequest); + } catch (InterruptedException | ExecutionException e) { + throw new RuntimeException(e); + } + })).collect(Collectors.toList()); + + threads.forEach(Thread::start); + threads.forEach(t -> { + try { + t.join(); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + }); + + +// Thread[] threads = new Thread[3 + randomInt(5)]; +// ArrayList nodes = new ArrayList<>(); +// nodes.add(discoveryState(masterService).nodes().getLocalNode()); +// final CyclicBarrier barrier = new CyclicBarrier(threads.length); +// final List backgroundExceptions = new CopyOnWriteArrayList<>(); +// for (int i = 0; i < threads.length; i++) { +// final DiscoveryNode node = newNode(i); +// final int iterations = rarely() ? randomIntBetween(1, 4) : 1; +// nodes.add(node); +// threads[i] = new Thread(new AbstractRunnable() { +// @Override +// public void onFailure(Exception e) { +// logger.error("unexpected error in join thread", e); +// backgroundExceptions.add(e); +// } +// +// @Override +// protected void doRun() throws Exception { +// barrier.await(); +// for (int i = 0; i < iterations; i++) { +// logger.debug("{} joining", node); +// joinNode(node); +// } +// } +// }, "t_" + i); +// threads[i].start(); +// } +// +// logger.info("--> waiting for joins to complete"); +// for (Thread thread : threads) { +// thread.join(); +// } + + assertTrue(MasterServiceTests.discoveryState(masterService).nodes().isLocalNodeElectedMaster()); + // nodeJoinController.startElectionContext(); // ArrayList> pendingJoins = new ArrayList<>(); // for (int i = randomInt(5); i > 0; i--) { From 391989b5f6d1f683d4ba82ba03c2152405c336e4 Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Wed, 15 Aug 2018 19:27:20 +0200 Subject: [PATCH 05/29] factor out into separate class --- .../cluster/coordination/Coordinator.java | 197 ++---------------- .../cluster/coordination/JoinHelper.java | 192 +++++++++++++++++ .../cluster/coordination/NodeJoinTests.java | 4 +- 3 files changed, 215 insertions(+), 178 deletions(-) create mode 100644 server/src/main/java/org/elasticsearch/cluster/coordination/JoinHelper.java 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 f59505d57fcf3..5a48bb2ba70e0 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/Coordinator.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/Coordinator.java @@ -1,161 +1,47 @@ package org.elasticsearch.cluster.coordination; -import org.apache.logging.log4j.message.ParameterizedMessage; import org.apache.lucene.util.SetOnce; -import org.elasticsearch.cluster.ClusterState; -import org.elasticsearch.cluster.ClusterStateTaskConfig; -import org.elasticsearch.cluster.ClusterStateTaskListener; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.routing.allocation.AllocationService; import org.elasticsearch.cluster.service.MasterService; -import org.elasticsearch.common.Priority; import org.elasticsearch.common.component.AbstractLifecycleComponent; -import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.discovery.zen.ElectMasterService; import org.elasticsearch.discovery.zen.MembershipAction; -import org.elasticsearch.discovery.zen.NodeJoinController; -import org.elasticsearch.discovery.zen.NodeJoinController.JoinTaskExecutor; -import org.elasticsearch.threadpool.ThreadPool; -import org.elasticsearch.threadpool.ThreadPool.Names; -import org.elasticsearch.transport.TransportException; -import org.elasticsearch.transport.TransportResponse; -import org.elasticsearch.transport.TransportResponse.Empty; -import org.elasticsearch.transport.TransportResponseHandler; import org.elasticsearch.transport.TransportService; -import java.io.IOException; -import java.util.HashMap; -import java.util.List; -import java.util.Map; import java.util.Optional; import java.util.function.Supplier; -//TODO: add a test that sends random joins and checks if the join led to success (the fake masterservice fully manages state) -// i.e. similar to NodeJoinControllerTests public class Coordinator extends AbstractLifecycleComponent { - public static final String JOIN_ACTION_NAME = "internal:discovery/zen2/join"; - private final Object mutex = new Object(); - private Mode mode; - private Optional lastKnownLeader; - private Optional lastJoin; - private final SetOnce consensusState = new SetOnce<>(); private final TransportService transportService; - private final MasterService masterService; - - - private final JoinTaskExecutor joinTaskExecutor; private final Supplier persistedStateSupplier; - // similar to NodeJoinController.ElectionContext.joinRequestAccumulator, captures joins on election - private final Map joinRequestAccumulator = new HashMap<>(); + private final SetOnce consensusState = new SetOnce<>(); // initialized on start-up (see doStart) + private volatile Mode mode; + private Optional lastKnownLeader; + private Optional lastJoin; + private final JoinHelper joinHelper; public Coordinator(Settings settings, TransportService transportService, AllocationService allocationService, MasterService masterService, Supplier persistedStateSupplier) { super(settings); this.transportService = transportService; - this.masterService = masterService; this.persistedStateSupplier = persistedStateSupplier; - this.mode = Mode.CANDIDATE; + this.mode = null; lastKnownLeader = Optional.empty(); lastJoin = Optional.empty(); - // disable minimum_master_nodes check - final ElectMasterService electMasterService = new ElectMasterService(settings) { - - @Override - public boolean hasEnoughMasterNodes(Iterable nodes) { - return true; - } + joinHelper = new JoinHelper(settings, allocationService, masterService, transportService, this::getCurrentTerm) { @Override - public void logMinimumMasterNodesWarningIfNecessary(ClusterState oldState, ClusterState newState) { - // ignore + public void handleJoinRequest(JoinRequest joinRequest, MembershipAction.JoinCallback joinCallback) { + Coordinator.this.handleJoinRequest(joinRequest, joinCallback); } - - }; - - // reuse JoinTaskExecutor implementation from ZenDiscovery, but hack some checks - this.joinTaskExecutor = new JoinTaskExecutor(allocationService, electMasterService, logger) { - - @Override - public ClusterTasksResult execute(ClusterState currentState, List joiningNodes) throws Exception { - // This is called when preparing the next cluster state for publication. There is no guarantee that the term we see here is - // the term under which this state will eventually be published: the current term may be increased after this check due to - // some other activity. That the term is correct is, however, checked properly during publication, so it is sufficient to - // check it here on a best-effort basis. This is fine because a concurrent change indicates the existence of another leader - // in a higher term which will cause this node to stand down. - - final long currentTerm = getCurrentTerm(); // TODO perhaps this can be a volatile read? - if (currentState.term() != currentTerm) { - currentState = ClusterState.builder(currentState).term(currentTerm).build(); - } - return super.execute(currentState, joiningNodes); - } - }; } - private void registerTransportActions() { - transportService.registerRequestHandler(JOIN_ACTION_NAME, Names.GENERIC, false, false, - JoinRequest::new, - (request, channel, task) -> handleJoinRequest(request, new MembershipAction.JoinCallback() { - @Override - public void onSuccess() { - try { - channel.sendResponse(Empty.INSTANCE); - } catch (IOException e) { - onFailure(e); - } - } - - @Override - public void onFailure(Exception e) { - try { - channel.sendResponse(e); - } catch (Exception inner) { - inner.addSuppressed(e); - logger.warn("failed to send back failure on join request", inner); - } - } - })); - } - - // TODO: maybe add this method in a follow-up. - private void sendOptionalJoin(DiscoveryNode destination, Optional optionalJoin) { - // No synchronisation required - transportService.sendRequest(destination, JOIN_ACTION_NAME, new JoinRequest(getLocalNode(), optionalJoin), - new TransportResponseHandler() { - @Override - public Empty read(StreamInput in) { - return Empty.INSTANCE; - } - - @Override - public void handleResponse(Empty response) { - // No synchronisation required - logger.debug("SendJoinResponseHandler: successfully joined {}", destination); - } - - @Override - public void handleException(TransportException exp) { - // No synchronisation required - if (exp.getRootCause() instanceof CoordinationStateRejectedException) { - logger.debug("SendJoinResponseHandler: [{}] failed: {}", destination, exp.getRootCause().getMessage()); - } else { - logger.debug(() -> new ParameterizedMessage("SendJoinResponseHandler: [{}] failed", destination), exp); - } - } - - @Override - public String executor() { - return Names.GENERIC; - } - }); - } - // package-visible for testing long getCurrentTerm() { synchronized (mutex) { @@ -165,13 +51,11 @@ long getCurrentTerm() { // package-visible for testing Mode getMode() { - synchronized (mutex) { - return mode; - } + return mode; } private Optional ensureTermAtLeast(DiscoveryNode sourceNode, long targetTerm) { - assert Thread.holdsLock(mutex) : "Legislator mutex not held"; + assert Thread.holdsLock(mutex) : "Coordinator mutex not held"; if (getCurrentTerm() < targetTerm) { return Optional.of(joinLeaderInTerm(new StartJoinRequest(sourceNode, targetTerm))); } @@ -179,7 +63,7 @@ private Optional ensureTermAtLeast(DiscoveryNode sourceNode, long targetTe } private Join joinLeaderInTerm(StartJoinRequest startJoinRequest) { - assert Thread.holdsLock(mutex) : "Legislator mutex not held"; + assert Thread.holdsLock(mutex) : "Coordinator mutex not held"; logger.debug("joinLeaderInTerm: from [{}] with term {}", startJoinRequest.getSourceNode(), startJoinRequest.getTerm()); Join join = consensusState.get().handleStartJoin(startJoinRequest); lastJoin = Optional.of(join); @@ -194,7 +78,7 @@ private Join joinLeaderInTerm(StartJoinRequest startJoinRequest) { } public void handleJoinRequest(JoinRequest joinRequest, MembershipAction.JoinCallback joinCallback) { - + assert Thread.holdsLock(mutex) == false; transportService.connectToNode(joinRequest.getSourceNode()); synchronized (mutex) { @@ -203,34 +87,23 @@ public void handleJoinRequest(JoinRequest joinRequest, MembershipAction.JoinCall } private void handleJoinRequestUnderLock(JoinRequest joinRequest, MembershipAction.JoinCallback joinCallback) { - assert Thread.holdsLock(mutex) : "Legislator mutex not held"; + assert Thread.holdsLock(mutex) : "Coordinator mutex not held"; logger.trace("handleJoinRequestUnderLock: as {}, handling {}", mode, joinRequest); if (mode == Mode.LEADER) { - // submit as cluster state update task - masterService.submitStateUpdateTask("zen-disco-node-join", - joinRequest.getSourceNode(), ClusterStateTaskConfig.build(Priority.URGENT), - joinTaskExecutor, new NodeJoinController.JoinTaskListener(joinCallback, logger)); + joinHelper.joinLeader(joinRequest, joinCallback); } else { - MembershipAction.JoinCallback prev = joinRequestAccumulator.put(joinRequest.getSourceNode(), joinCallback); - if (prev != null) { - prev.onFailure(new CoordinationStateRejectedException("received a newer join from " + joinRequest.getSourceNode())); - } + joinHelper.addJoinCallback(joinRequest, joinCallback); } try { joinRequest.getOptionalJoin().ifPresent(this::handleJoin); } catch (CoordinationStateRejectedException exception) { - final MembershipAction.JoinCallback callback = joinRequestAccumulator.remove(joinRequest.getSourceNode()); - if (callback == null) { - logger.trace("handleJoinRequestUnderLock: submitted task to master service, but vote was rejected", exception); - } else { - callback.onFailure(exception); - } + joinHelper.removeAndFail(joinRequest, exception); } } private void handleJoin(Join join) { - assert Thread.holdsLock(mutex) : "Legislator mutex not held"; + assert Thread.holdsLock(mutex) : "Coordinator mutex not held"; Optional optionalJoinToSelf = ensureTermAtLeast(getLocalNode(), join.getTerm()); optionalJoinToSelf.ifPresent(this::handleJoin); // if someone thinks we should be master, let's try to become one @@ -249,20 +122,8 @@ private void handleJoin(Join join) { becomeLeader("handleJoin"); - Map pendingAsTasks = getPendingAsTasks(); - joinRequestAccumulator.clear(); - - final String source = "zen-disco-elected-as-master ([" + pendingAsTasks.size() + "] nodes joined)"; - // noop listener, the election finished listener determines result - pendingAsTasks.put(NodeJoinController.BECOME_MASTER_TASK, (source1, e) -> { - }); - // TODO: should we take any further action when FINISH_ELECTION_TASK fails? - pendingAsTasks.put(NodeJoinController.FINISH_ELECTION_TASK, (source1, e) -> { - }); - try { - masterService.submitStateUpdateTasks(source, pendingAsTasks, ClusterStateTaskConfig.build(Priority.URGENT), - joinTaskExecutor); + joinHelper.clearAndSendJoins(); } catch (IllegalStateException e) { // TODO fix NodeJoinController and remove this assert e.getMessage().contains("is already queued"); @@ -271,22 +132,6 @@ private void handleJoin(Join join) { } } - // copied from NodeJoinController.getPendingAsTasks - private Map getPendingAsTasks() { - assert Thread.holdsLock(mutex) : "Legislator mutex not held"; - Map tasks = new HashMap<>(); - joinRequestAccumulator.forEach((key, value) -> tasks.put(key, - new NodeJoinController.JoinTaskListener(value, logger))); - return tasks; - } - - private void clearJoins() { - assert Thread.holdsLock(mutex) : "Legislator mutex not held"; - joinRequestAccumulator.values().forEach( - joinCallback -> joinCallback.onFailure(new CoordinationStateRejectedException("node stepped down as leader"))); - joinRequestAccumulator.clear(); - } - private void becomeCandidate(String method) { assert Thread.holdsLock(mutex) : "Legislator mutex not held"; logger.debug("{}: becoming CANDIDATE (was {}, lastKnownLeader was [{}])", method, mode, lastKnownLeader); @@ -294,7 +139,7 @@ private void becomeCandidate(String method) { if (mode != Mode.CANDIDATE) { mode = Mode.CANDIDATE; - clearJoins(); + joinHelper.clearJoins(); } } @@ -316,7 +161,7 @@ private void becomeFollower(String method, DiscoveryNode leaderNode) { mode = Mode.FOLLOWER; - clearJoins(); + joinHelper.clearJoins(); } lastKnownLeader = Optional.of(leaderNode); diff --git a/server/src/main/java/org/elasticsearch/cluster/coordination/JoinHelper.java b/server/src/main/java/org/elasticsearch/cluster/coordination/JoinHelper.java new file mode 100644 index 0000000000000..8b1e082824fef --- /dev/null +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/JoinHelper.java @@ -0,0 +1,192 @@ +package org.elasticsearch.cluster.coordination; + +import org.apache.logging.log4j.message.ParameterizedMessage; +import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.ClusterStateTaskConfig; +import org.elasticsearch.cluster.ClusterStateTaskListener; +import org.elasticsearch.cluster.node.DiscoveryNode; +import org.elasticsearch.cluster.routing.allocation.AllocationService; +import org.elasticsearch.cluster.service.MasterService; +import org.elasticsearch.common.Priority; +import org.elasticsearch.common.component.AbstractComponent; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.discovery.zen.ElectMasterService; +import org.elasticsearch.discovery.zen.MembershipAction.JoinCallback; +import org.elasticsearch.discovery.zen.NodeJoinController; +import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.transport.TransportException; +import org.elasticsearch.transport.TransportResponse; +import org.elasticsearch.transport.TransportResponseHandler; +import org.elasticsearch.transport.TransportService; + +import java.io.IOException; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.function.LongSupplier; + +//TODO: add a test that sends random joins and checks if the join led to success (the fake masterservice fully manages state) +// i.e. similar to NodeJoinControllerTests +public abstract class JoinHelper extends AbstractComponent { + + public static final String JOIN_ACTION_NAME = "internal:discovery/zen2/join"; + + // similar to NodeJoinController.ElectionContext.joinRequestAccumulator, captures joins on election + private final Map joinRequestAccumulator = new HashMap<>(); + + private final NodeJoinController.JoinTaskExecutor joinTaskExecutor; + private final MasterService masterService; + + private final TransportService transportService; + + JoinHelper(Settings settings, AllocationService allocationService, MasterService masterService, TransportService transportService, + LongSupplier currentTermSupplier) { + super(settings); + + this.masterService = masterService; + this.transportService = transportService; + + // disable minimum_master_nodes check + final ElectMasterService electMasterService = new ElectMasterService(settings) { + + @Override + public boolean hasEnoughMasterNodes(Iterable nodes) { + return true; + } + + @Override + public void logMinimumMasterNodesWarningIfNecessary(ClusterState oldState, ClusterState newState) { + // ignore + } + + }; + + // reuse JoinTaskExecutor implementation from ZenDiscovery, but hack some checks + this.joinTaskExecutor = new NodeJoinController.JoinTaskExecutor(allocationService, electMasterService, logger) { + + @Override + public ClusterTasksResult execute(ClusterState currentState, List joiningNodes) throws Exception { + // This is called when preparing the next cluster state for publication. There is no guarantee that the term we see here is + // the term under which this state will eventually be published: the current term may be increased after this check due to + // some other activity. That the term is correct is, however, checked properly during publication, so it is sufficient to + // check it here on a best-effort basis. This is fine because a concurrent change indicates the existence of another leader + // in a higher term which will cause this node to stand down. + + final long currentTerm = currentTermSupplier.getAsLong(); + if (currentState.term() != currentTerm) { + currentState = ClusterState.builder(currentState).term(currentTerm).build(); + } + return super.execute(currentState, joiningNodes); + } + + }; + + transportService.registerRequestHandler(JOIN_ACTION_NAME, ThreadPool.Names.GENERIC, false, false, + JoinRequest::new, + (request, channel, task) -> handleJoinRequest(request, new JoinCallback() { + @Override + public void onSuccess() { + try { + channel.sendResponse(TransportResponse.Empty.INSTANCE); + } catch (IOException e) { + onFailure(e); + } + } + + @Override + public void onFailure(Exception e) { + try { + channel.sendResponse(e); + } catch (Exception inner) { + inner.addSuppressed(e); + logger.warn("failed to send back failure on join request", inner); + } + } + })); + } + + public abstract void handleJoinRequest(JoinRequest joinRequest, JoinCallback joinCallback); + + // TODO: maybe add this method in a follow-up. + public void sendOptionalJoin(DiscoveryNode destination, Optional optionalJoin) { + // No synchronisation required + transportService.sendRequest(destination, JOIN_ACTION_NAME, new JoinRequest(transportService.getLocalNode(), optionalJoin), + new TransportResponseHandler() { + @Override + public TransportResponse.Empty read(StreamInput in) { + return TransportResponse.Empty.INSTANCE; + } + + @Override + public void handleResponse(TransportResponse.Empty response) { + // No synchronisation required + logger.debug("SendJoinResponseHandler: successfully joined {}", destination); + } + + @Override + public void handleException(TransportException exp) { + // No synchronisation required + if (exp.getRootCause() instanceof CoordinationStateRejectedException) { + logger.debug("SendJoinResponseHandler: [{}] failed: {}", destination, exp.getRootCause().getMessage()); + } else { + logger.debug(() -> new ParameterizedMessage("SendJoinResponseHandler: [{}] failed", destination), exp); + } + } + + @Override + public String executor() { + return ThreadPool.Names.GENERIC; + } + }); + } + + public void addJoinCallback(JoinRequest joinRequest, JoinCallback joinCallback) { + JoinCallback prev = joinRequestAccumulator.put(joinRequest.getSourceNode(), joinCallback); + if (prev != null) { + prev.onFailure(new CoordinationStateRejectedException("received a newer join from " + joinRequest.getSourceNode())); + } + } + + public void removeAndFail(JoinRequest joinRequest, CoordinationStateRejectedException exception) { + final JoinCallback callback = joinRequestAccumulator.remove(joinRequest.getSourceNode()); + if (callback == null) { + logger.trace("handleJoinRequestUnderLock: submitted task to master service, but vote was rejected", exception); + } else { + callback.onFailure(exception); + } + } + + public void clearJoins() { + joinRequestAccumulator.values().forEach( + joinCallback -> joinCallback.onFailure(new CoordinationStateRejectedException("node stepped down as leader"))); + joinRequestAccumulator.clear(); + } + + public void clearAndSendJoins() { + final Map pendingAsTasks = new HashMap<>(); + joinRequestAccumulator.forEach((key, value) -> pendingAsTasks.put(key, + new NodeJoinController.JoinTaskListener(value, logger))); + joinRequestAccumulator.clear(); + + final String source = "zen-disco-elected-as-master ([" + pendingAsTasks.size() + "] nodes joined)"; + // noop listener, the election finished listener determines result + pendingAsTasks.put(NodeJoinController.BECOME_MASTER_TASK, (source1, e) -> { + }); + // TODO: should we take any further action when FINISH_ELECTION_TASK fails? + pendingAsTasks.put(NodeJoinController.FINISH_ELECTION_TASK, (source1, e) -> { + }); + + masterService.submitStateUpdateTasks(source, pendingAsTasks, ClusterStateTaskConfig.build(Priority.URGENT), + joinTaskExecutor); + } + + public void joinLeader(JoinRequest joinRequest, JoinCallback joinCallback) { + // submit as cluster state update task + masterService.submitStateUpdateTask("zen-disco-node-join", + joinRequest.getSourceNode(), ClusterStateTaskConfig.build(Priority.URGENT), + joinTaskExecutor, new NodeJoinController.JoinTaskListener(joinCallback, logger)); + } + +} 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 b8c417ce83ca9..d4f2dd0616ade 100644 --- a/server/src/test/java/org/elasticsearch/cluster/coordination/NodeJoinTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/coordination/NodeJoinTests.java @@ -50,7 +50,7 @@ public class NodeJoinTests extends ESTestCase { @BeforeClass public static void beforeClass() { - threadPool = new TestThreadPool("NodeJoinControllerTests"); + threadPool = new TestThreadPool(NodeJoinTests.getTestClass().getName()); } @AfterClass @@ -172,7 +172,7 @@ public void testSuccessfulJoinAccumulation() { // we need at least a quorum of voting nodes with a correct term - List threads = randomSubsetOf(nodes).stream().map(node -> new Thread(() -> { + List threads = nodes.stream().map(node -> new Thread(() -> { JoinRequest joinRequest = new JoinRequest(node, Optional.of(new Join(node, localNode, 2, 1, 1))); try { joinNode(joinRequest); From 7055a704ffa213347b573948310d40bed6203705 Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Sat, 18 Aug 2018 13:21:51 +0200 Subject: [PATCH 06/29] Smaller stuff --- .../coordination/CoordinationState.java | 1 + .../cluster/coordination/Coordinator.java | 79 +++++------- .../cluster/coordination/JoinHelper.java | 115 ++++++++++-------- 3 files changed, 96 insertions(+), 99 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/cluster/coordination/CoordinationState.java b/server/src/main/java/org/elasticsearch/cluster/coordination/CoordinationState.java index 27544405d3c0b..8a6e6d1833258 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/CoordinationState.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/CoordinationState.java @@ -227,6 +227,7 @@ public boolean handleJoin(Join join) { boolean added = joinVotes.addVote(join.getSourceNode()); boolean prevElectionWon = electionWon; electionWon = isElectionQuorum(joinVotes); + assert !prevElectionWon || electionWon; // we cannot go from won to not won logger.debug("handleJoin: added join {} from [{}] for election, electionWon={} lastAcceptedTerm={} lastAcceptedVersion={}", join, join.getSourceNode(), electionWon, lastAcceptedTerm, getLastAcceptedVersion()); 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 5a48bb2ba70e0..bed0dfa0353c4 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/Coordinator.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/Coordinator.java @@ -1,12 +1,12 @@ package org.elasticsearch.cluster.coordination; import org.apache.lucene.util.SetOnce; +import org.elasticsearch.cluster.coordination.JoinHelper.JoinCallback; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.routing.allocation.AllocationService; import org.elasticsearch.cluster.service.MasterService; import org.elasticsearch.common.component.AbstractLifecycleComponent; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.discovery.zen.MembershipAction; import org.elasticsearch.transport.TransportService; import java.util.Optional; @@ -14,44 +14,25 @@ public class Coordinator extends AbstractLifecycleComponent { - private final Object mutex = new Object(); private final TransportService transportService; + private final JoinHelper joinHelper; private final Supplier persistedStateSupplier; - - private final SetOnce consensusState = new SetOnce<>(); // initialized on start-up (see doStart) + private final Object mutex = new Object(); + private final SetOnce coordinationState = new SetOnce<>(); // initialized on start-up (see doStart) private volatile Mode mode; private Optional lastKnownLeader; private Optional lastJoin; - private final JoinHelper joinHelper; public Coordinator(Settings settings, TransportService transportService, AllocationService allocationService, MasterService masterService, Supplier persistedStateSupplier) { super(settings); this.transportService = transportService; + this.joinHelper = new JoinHelper(settings, allocationService, masterService, transportService, this::getCurrentTerm, + this::handleJoinRequest); this.persistedStateSupplier = persistedStateSupplier; - this.mode = null; - lastKnownLeader = Optional.empty(); - lastJoin = Optional.empty(); - - joinHelper = new JoinHelper(settings, allocationService, masterService, transportService, this::getCurrentTerm) { - @Override - public void handleJoinRequest(JoinRequest joinRequest, MembershipAction.JoinCallback joinCallback) { - Coordinator.this.handleJoinRequest(joinRequest, joinCallback); - } - }; - } - - // package-visible for testing - long getCurrentTerm() { - synchronized (mutex) { - return consensusState.get().getCurrentTerm(); - } - } - - // package-visible for testing - Mode getMode() { - return mode; + this.lastKnownLeader = Optional.empty(); + this.lastJoin = Optional.empty(); } private Optional ensureTermAtLeast(DiscoveryNode sourceNode, long targetTerm) { @@ -65,7 +46,7 @@ private Optional ensureTermAtLeast(DiscoveryNode sourceNode, long targetTe private Join joinLeaderInTerm(StartJoinRequest startJoinRequest) { assert Thread.holdsLock(mutex) : "Coordinator mutex not held"; logger.debug("joinLeaderInTerm: from [{}] with term {}", startJoinRequest.getSourceNode(), startJoinRequest.getTerm()); - Join join = consensusState.get().handleStartJoin(startJoinRequest); + Join join = coordinationState.get().handleStartJoin(startJoinRequest); lastJoin = Optional.of(join); if (mode == Mode.CANDIDATE) { // refresh required because current term has changed @@ -77,7 +58,7 @@ private Join joinLeaderInTerm(StartJoinRequest startJoinRequest) { return join; } - public void handleJoinRequest(JoinRequest joinRequest, MembershipAction.JoinCallback joinCallback) { + public void handleJoinRequest(JoinRequest joinRequest, JoinCallback joinCallback) { assert Thread.holdsLock(mutex) == false; transportService.connectToNode(joinRequest.getSourceNode()); @@ -86,7 +67,7 @@ public void handleJoinRequest(JoinRequest joinRequest, MembershipAction.JoinCall } } - private void handleJoinRequestUnderLock(JoinRequest joinRequest, MembershipAction.JoinCallback joinCallback) { + private void handleJoinRequestUnderLock(JoinRequest joinRequest, JoinCallback joinCallback) { assert Thread.holdsLock(mutex) : "Coordinator mutex not held"; logger.trace("handleJoinRequestUnderLock: as {}, handling {}", mode, joinRequest); if (mode == Mode.LEADER) { @@ -108,27 +89,14 @@ private void handleJoin(Join join) { optionalJoinToSelf.ifPresent(this::handleJoin); // if someone thinks we should be master, let's try to become one - final CoordinationState state = consensusState.get(); + final CoordinationState state = coordinationState.get(); boolean prevElectionWon = state.electionWon(); boolean addedJoin = state.handleJoin(join); - assert !prevElectionWon || state.electionWon(); // we cannot go from won to not won - -// if (addedJoin && mode == Legislator.Mode.LEADER && currentPublication.isPresent() == false) { -// scheduleReconfigurationIfSuboptimal(); -// } if (prevElectionWon == false && state.electionWon()) { assert mode == Mode.CANDIDATE : "expected candidate but was " + mode; - becomeLeader("handleJoin"); - - try { - joinHelper.clearAndSendJoins(); - } catch (IllegalStateException e) { - // TODO fix NodeJoinController and remove this - assert e.getMessage().contains("is already queued"); - becomeCandidate("handleJoin"); - } + joinHelper.clearAndSendJoins(); } } @@ -167,14 +135,27 @@ private void becomeFollower(String method, DiscoveryNode leaderNode) { lastKnownLeader = Optional.of(leaderNode); } - public DiscoveryNode getLocalNode() { + // package-visible for testing + long getCurrentTerm() { + synchronized (mutex) { + return coordinationState.get().getCurrentTerm(); + } + } + + // package-visible for testing + Mode getMode() { + return mode; + } + + // package-visible for testing + DiscoveryNode getLocalNode() { return transportService.getLocalNode(); } @Override protected void doStart() { CoordinationState.PersistedState persistedState = persistedStateSupplier.get(); - consensusState.set(new CoordinationState(settings, getLocalNode(), persistedState)); + coordinationState.set(new CoordinationState(settings, getLocalNode(), persistedState)); } public void startInitialJoin() { @@ -196,10 +177,10 @@ protected void doClose() { public void invariant() { synchronized (mutex) { if (mode == Mode.LEADER) { - assert consensusState.get().electionWon(); + assert coordinationState.get().electionWon(); assert lastKnownLeader.isPresent() && lastKnownLeader.get().equals(getLocalNode()); } else if (mode == Mode.FOLLOWER) { - assert consensusState.get().electionWon() == false : getLocalNode() + " is FOLLOWER so electionWon() should be false"; + assert coordinationState.get().electionWon() == false : getLocalNode() + " is FOLLOWER so electionWon() should be false"; assert lastKnownLeader.isPresent() && (lastKnownLeader.get().equals(getLocalNode()) == false); } else { assert mode == Mode.CANDIDATE; diff --git a/server/src/main/java/org/elasticsearch/cluster/coordination/JoinHelper.java b/server/src/main/java/org/elasticsearch/cluster/coordination/JoinHelper.java index 8b1e082824fef..f75cc44bd0823 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/JoinHelper.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/JoinHelper.java @@ -1,3 +1,21 @@ +/* + * 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.apache.logging.log4j.message.ParameterizedMessage; @@ -11,9 +29,6 @@ import org.elasticsearch.common.component.AbstractComponent; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.discovery.zen.ElectMasterService; -import org.elasticsearch.discovery.zen.MembershipAction.JoinCallback; -import org.elasticsearch.discovery.zen.NodeJoinController; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportException; import org.elasticsearch.transport.TransportResponse; @@ -25,49 +40,29 @@ import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.function.BiConsumer; import java.util.function.LongSupplier; -//TODO: add a test that sends random joins and checks if the join led to success (the fake masterservice fully manages state) -// i.e. similar to NodeJoinControllerTests -public abstract class JoinHelper extends AbstractComponent { +public class JoinHelper extends AbstractComponent { public static final String JOIN_ACTION_NAME = "internal:discovery/zen2/join"; - // similar to NodeJoinController.ElectionContext.joinRequestAccumulator, captures joins on election - private final Map joinRequestAccumulator = new HashMap<>(); - - private final NodeJoinController.JoinTaskExecutor joinTaskExecutor; private final MasterService masterService; - private final TransportService transportService; + private final JoinTaskExecutor joinTaskExecutor; + private final Map joinRequestAccumulator = new HashMap<>(); - JoinHelper(Settings settings, AllocationService allocationService, MasterService masterService, TransportService transportService, - LongSupplier currentTermSupplier) { + public JoinHelper(Settings settings, AllocationService allocationService, MasterService masterService, + TransportService transportService, LongSupplier currentTermSupplier, + BiConsumer joinRequestHandler) { super(settings); - this.masterService = masterService; this.transportService = transportService; - - // disable minimum_master_nodes check - final ElectMasterService electMasterService = new ElectMasterService(settings) { - - @Override - public boolean hasEnoughMasterNodes(Iterable nodes) { - return true; - } + this.joinTaskExecutor = new JoinTaskExecutor(allocationService, logger) { @Override - public void logMinimumMasterNodesWarningIfNecessary(ClusterState oldState, ClusterState newState) { - // ignore - } - - }; - - // reuse JoinTaskExecutor implementation from ZenDiscovery, but hack some checks - this.joinTaskExecutor = new NodeJoinController.JoinTaskExecutor(allocationService, electMasterService, logger) { - - @Override - public ClusterTasksResult execute(ClusterState currentState, List joiningNodes) throws Exception { + public ClusterTasksResult execute(ClusterState currentState, List joiningTasks) + throws Exception { // This is called when preparing the next cluster state for publication. There is no guarantee that the term we see here is // the term under which this state will eventually be published: the current term may be increased after this check due to // some other activity. That the term is correct is, however, checked properly during publication, so it is sufficient to @@ -78,14 +73,13 @@ public ClusterTasksResult execute(ClusterState currentState, List if (currentState.term() != currentTerm) { currentState = ClusterState.builder(currentState).term(currentTerm).build(); } - return super.execute(currentState, joiningNodes); + return super.execute(currentState, joiningTasks); } }; - transportService.registerRequestHandler(JOIN_ACTION_NAME, ThreadPool.Names.GENERIC, false, false, - JoinRequest::new, - (request, channel, task) -> handleJoinRequest(request, new JoinCallback() { + transportService.registerRequestHandler(JOIN_ACTION_NAME, ThreadPool.Names.GENERIC, false, false, JoinRequest::new, + (request, channel, task) -> joinRequestHandler.accept(request, new JoinCallback() { @Override public void onSuccess() { try { @@ -107,11 +101,8 @@ public void onFailure(Exception e) { })); } - public abstract void handleJoinRequest(JoinRequest joinRequest, JoinCallback joinCallback); - // TODO: maybe add this method in a follow-up. public void sendOptionalJoin(DiscoveryNode destination, Optional optionalJoin) { - // No synchronisation required transportService.sendRequest(destination, JOIN_ACTION_NAME, new JoinRequest(transportService.getLocalNode(), optionalJoin), new TransportResponseHandler() { @Override @@ -121,13 +112,13 @@ public TransportResponse.Empty read(StreamInput in) { @Override public void handleResponse(TransportResponse.Empty response) { - // No synchronisation required logger.debug("SendJoinResponseHandler: successfully joined {}", destination); } @Override public void handleException(TransportException exp) { - // No synchronisation required + //TODO: log certain failures at info level so that users can easily see why nodes are failing to join + // see ZenDiscovery.joinElectedMaster for comparison if (exp.getRootCause() instanceof CoordinationStateRejectedException) { logger.debug("SendJoinResponseHandler: [{}] failed: {}", destination, exp.getRootCause().getMessage()); } else { @@ -137,7 +128,7 @@ public void handleException(TransportException exp) { @Override public String executor() { - return ThreadPool.Names.GENERIC; + return ThreadPool.Names.SAME; // just logging calls } }); } @@ -165,17 +156,17 @@ public void clearJoins() { } public void clearAndSendJoins() { - final Map pendingAsTasks = new HashMap<>(); - joinRequestAccumulator.forEach((key, value) -> pendingAsTasks.put(key, - new NodeJoinController.JoinTaskListener(value, logger))); + final Map pendingAsTasks = new HashMap<>(); + joinRequestAccumulator.forEach((key, value) -> pendingAsTasks.put(new JoinTaskExecutor.Task(key, "elect master"), + new JoinTaskListener(value))); joinRequestAccumulator.clear(); final String source = "zen-disco-elected-as-master ([" + pendingAsTasks.size() + "] nodes joined)"; // noop listener, the election finished listener determines result - pendingAsTasks.put(NodeJoinController.BECOME_MASTER_TASK, (source1, e) -> { + pendingAsTasks.put(JoinTaskExecutor.BECOME_MASTER_TASK, (source1, e) -> { }); // TODO: should we take any further action when FINISH_ELECTION_TASK fails? - pendingAsTasks.put(NodeJoinController.FINISH_ELECTION_TASK, (source1, e) -> { + pendingAsTasks.put(JoinTaskExecutor.FINISH_ELECTION_TASK, (source1, e) -> { }); masterService.submitStateUpdateTasks(source, pendingAsTasks, ClusterStateTaskConfig.build(Priority.URGENT), @@ -185,8 +176,32 @@ public void clearAndSendJoins() { public void joinLeader(JoinRequest joinRequest, JoinCallback joinCallback) { // submit as cluster state update task masterService.submitStateUpdateTask("zen-disco-node-join", - joinRequest.getSourceNode(), ClusterStateTaskConfig.build(Priority.URGENT), - joinTaskExecutor, new NodeJoinController.JoinTaskListener(joinCallback, logger)); + new JoinTaskExecutor.Task(joinRequest.getSourceNode(), "join existing leader"), ClusterStateTaskConfig.build(Priority.URGENT), + joinTaskExecutor, new JoinTaskListener(joinCallback)); + } + + public interface JoinCallback { + void onSuccess(); + + void onFailure(Exception e); + } + + static class JoinTaskListener implements ClusterStateTaskListener { + private final JoinCallback joinCallback; + + JoinTaskListener(JoinCallback joinCallback) { + this.joinCallback = joinCallback; + } + + @Override + public void onFailure(String source, Exception e) { + joinCallback.onFailure(e); + } + + @Override + public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) { + joinCallback.onSuccess(); + } } } From 36b4b39fec7df2dc08a033a03f4438b15e6f3b9e Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Sun, 19 Aug 2018 16:31:33 +0200 Subject: [PATCH 07/29] own joincallback --- .../cluster/coordination/Coordinator.java | 45 +++++++++---------- .../cluster/coordination/JoinHelper.java | 25 +++++------ .../cluster/coordination/NodeJoinTests.java | 6 +-- 3 files changed, 33 insertions(+), 43 deletions(-) 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 bed0dfa0353c4..61ba9ef9b93a7 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/Coordinator.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/Coordinator.java @@ -48,11 +48,7 @@ private Join joinLeaderInTerm(StartJoinRequest startJoinRequest) { logger.debug("joinLeaderInTerm: from [{}] with term {}", startJoinRequest.getSourceNode(), startJoinRequest.getTerm()); Join join = coordinationState.get().handleStartJoin(startJoinRequest); lastJoin = Optional.of(join); - if (mode == Mode.CANDIDATE) { - // refresh required because current term has changed - // TODO: heartbeatRequestResponder = new Legislator.HeartbeatRequestResponder(); - } else { - // becomeCandidate refreshes responders + if (mode != Mode.CANDIDATE) { becomeCandidate("joinLeaderInTerm"); } return join; @@ -70,16 +66,23 @@ public void handleJoinRequest(JoinRequest joinRequest, JoinCallback joinCallback private void handleJoinRequestUnderLock(JoinRequest joinRequest, JoinCallback joinCallback) { assert Thread.holdsLock(mutex) : "Coordinator mutex not held"; logger.trace("handleJoinRequestUnderLock: as {}, handling {}", mode, joinRequest); - if (mode == Mode.LEADER) { - joinHelper.joinLeader(joinRequest, joinCallback); - } else { - joinHelper.addJoinCallback(joinRequest, joinCallback); - } try { joinRequest.getOptionalJoin().ifPresent(this::handleJoin); - } catch (CoordinationStateRejectedException exception) { - joinHelper.removeAndFail(joinRequest, exception); + } catch (Exception exception) { + joinCallback.onFailure(exception); + } + + if (mode == Mode.FOLLOWER) { + joinCallback.onFailure(new CoordinationStateRejectedException("join target is a follower")); + return; + } + + if (mode == Mode.LEADER) { + joinHelper.joinLeader(joinRequest, joinCallback); + } else { + assert mode == Mode.CANDIDATE; + joinHelper.addPendingJoin(joinRequest, joinCallback); } } @@ -91,12 +94,10 @@ private void handleJoin(Join join) { final CoordinationState state = coordinationState.get(); boolean prevElectionWon = state.electionWon(); - boolean addedJoin = state.handleJoin(join); + state.handleJoin(join); if (prevElectionWon == false && state.electionWon()) { - assert mode == Mode.CANDIDATE : "expected candidate but was " + mode; becomeLeader("handleJoin"); - joinHelper.clearAndSendJoins(); } } @@ -106,30 +107,26 @@ private void becomeCandidate(String method) { if (mode != Mode.CANDIDATE) { mode = Mode.CANDIDATE; - - joinHelper.clearJoins(); } } private void becomeLeader(String method) { assert Thread.holdsLock(mutex) : "Legislator mutex not held"; - assert mode != Mode.LEADER; - + assert mode == Mode.CANDIDATE : "expected candidate but was " + mode; logger.debug("{}: becoming LEADER (was {}, lastKnownLeader was [{}])", method, mode, lastKnownLeader); mode = Mode.LEADER; lastKnownLeader = Optional.of(getLocalNode()); + joinHelper.clearAndSubmitJoins(); } private void becomeFollower(String method, DiscoveryNode leaderNode) { assert Thread.holdsLock(mutex) : "Legislator mutex not held"; + logger.debug("{}: becoming FOLLOWER of [{}] (was {}, lastKnownLeader was [{}])", method, leaderNode, mode, lastKnownLeader); if (mode != Mode.FOLLOWER) { - logger.debug("{}: becoming FOLLOWER of [{}] (was {}, lastKnownLeader was [{}])", method, leaderNode, mode, lastKnownLeader); - mode = Mode.FOLLOWER; - - joinHelper.clearJoins(); + joinHelper.clearAndFailJoins("becoming follower"); } lastKnownLeader = Optional.of(leaderNode); @@ -179,9 +176,11 @@ public void invariant() { if (mode == Mode.LEADER) { assert coordinationState.get().electionWon(); assert lastKnownLeader.isPresent() && lastKnownLeader.get().equals(getLocalNode()); + assert joinHelper.getNumberOfPendingJoins() == 0; } else if (mode == Mode.FOLLOWER) { assert coordinationState.get().electionWon() == false : getLocalNode() + " is FOLLOWER so electionWon() should be false"; assert lastKnownLeader.isPresent() && (lastKnownLeader.get().equals(getLocalNode()) == false); + assert joinHelper.getNumberOfPendingJoins() == 0; } else { assert mode == Mode.CANDIDATE; } diff --git a/server/src/main/java/org/elasticsearch/cluster/coordination/JoinHelper.java b/server/src/main/java/org/elasticsearch/cluster/coordination/JoinHelper.java index f75cc44bd0823..89e1764ec7516 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/JoinHelper.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/JoinHelper.java @@ -45,7 +45,7 @@ public class JoinHelper extends AbstractComponent { - public static final String JOIN_ACTION_NAME = "internal:discovery/zen2/join"; + public static final String JOIN_ACTION_NAME = "internal:discovery/join"; private final MasterService masterService; private final TransportService transportService; @@ -133,35 +133,30 @@ public String executor() { }); } - public void addJoinCallback(JoinRequest joinRequest, JoinCallback joinCallback) { + public void addPendingJoin(JoinRequest joinRequest, JoinCallback joinCallback) { JoinCallback prev = joinRequestAccumulator.put(joinRequest.getSourceNode(), joinCallback); if (prev != null) { prev.onFailure(new CoordinationStateRejectedException("received a newer join from " + joinRequest.getSourceNode())); } } - public void removeAndFail(JoinRequest joinRequest, CoordinationStateRejectedException exception) { - final JoinCallback callback = joinRequestAccumulator.remove(joinRequest.getSourceNode()); - if (callback == null) { - logger.trace("handleJoinRequestUnderLock: submitted task to master service, but vote was rejected", exception); - } else { - callback.onFailure(exception); - } + public int getNumberOfPendingJoins() { + return joinRequestAccumulator.size(); } - public void clearJoins() { + public void clearAndFailJoins(String reason) { joinRequestAccumulator.values().forEach( - joinCallback -> joinCallback.onFailure(new CoordinationStateRejectedException("node stepped down as leader"))); + joinCallback -> joinCallback.onFailure(new CoordinationStateRejectedException(reason))); joinRequestAccumulator.clear(); } - public void clearAndSendJoins() { + public void clearAndSubmitJoins() { final Map pendingAsTasks = new HashMap<>(); - joinRequestAccumulator.forEach((key, value) -> pendingAsTasks.put(new JoinTaskExecutor.Task(key, "elect master"), + joinRequestAccumulator.forEach((key, value) -> pendingAsTasks.put(new JoinTaskExecutor.Task(key, "elect leader"), new JoinTaskListener(value))); joinRequestAccumulator.clear(); - final String source = "zen-disco-elected-as-master ([" + pendingAsTasks.size() + "] nodes joined)"; + final String source = "elected-as-master ([" + pendingAsTasks.size() + "] nodes joined)"; // noop listener, the election finished listener determines result pendingAsTasks.put(JoinTaskExecutor.BECOME_MASTER_TASK, (source1, e) -> { }); @@ -175,7 +170,7 @@ public void clearAndSendJoins() { public void joinLeader(JoinRequest joinRequest, JoinCallback joinCallback) { // submit as cluster state update task - masterService.submitStateUpdateTask("zen-disco-node-join", + masterService.submitStateUpdateTask("node-join", new JoinTaskExecutor.Task(joinRequest.getSourceNode(), "join existing leader"), ClusterStateTaskConfig.build(Priority.URGENT), joinTaskExecutor, new JoinTaskListener(joinCallback)); } 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 d4f2dd0616ade..4508f56845488 100644 --- a/server/src/test/java/org/elasticsearch/cluster/coordination/NodeJoinTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/coordination/NodeJoinTests.java @@ -13,7 +13,6 @@ import org.elasticsearch.cluster.service.MasterServiceTests; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.concurrent.BaseFuture; -import org.elasticsearch.discovery.zen.MembershipAction; import org.elasticsearch.test.ClusterServiceUtils; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.junit.annotations.TestLogging; @@ -24,9 +23,6 @@ import org.junit.AfterClass; import org.junit.BeforeClass; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; import java.util.HashSet; import java.util.List; import java.util.Optional; @@ -144,7 +140,7 @@ private SimpleFuture joinNodeAsync(final JoinRequest joinRequest) { logger.debug("starting {}", future); // clone the node before submitting to simulate an incoming join, which is guaranteed to have a new // disco node object serialized off the network - coordinator.handleJoinRequest(joinRequest, new MembershipAction.JoinCallback() { + coordinator.handleJoinRequest(joinRequest, new JoinHelper.JoinCallback() { @Override public void onSuccess() { logger.debug("{} completed", future); From 6b592002678a1b34100d7188e5bb0f570b64e717 Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Sun, 19 Aug 2018 17:47:59 +0200 Subject: [PATCH 08/29] simplify logic in handleJoinRequestUnderLock --- .../cluster/coordination/Coordinator.java | 71 +++-- .../cluster/coordination/JoinHelper.java | 53 +--- .../cluster/coordination/NodeJoinTests.java | 290 +++++++++++++----- .../cluster/FakeThreadPoolMasterService.java | 2 +- .../org/elasticsearch/test/ESTestCase.java | 2 +- 5 files changed, 255 insertions(+), 163 deletions(-) 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 61ba9ef9b93a7..f73946ee30f64 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/Coordinator.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/Coordinator.java @@ -17,10 +17,10 @@ public class Coordinator extends AbstractLifecycleComponent { private final TransportService transportService; private final JoinHelper joinHelper; private final Supplier persistedStateSupplier; - private final Object mutex = new Object(); - private final SetOnce coordinationState = new SetOnce<>(); // initialized on start-up (see doStart) + final Object mutex = new Object(); + final SetOnce coordinationState = new SetOnce<>(); // initialized on start-up (see doStart) - private volatile Mode mode; + private Mode mode; private Optional lastKnownLeader; private Optional lastJoin; @@ -67,41 +67,43 @@ private void handleJoinRequestUnderLock(JoinRequest joinRequest, JoinCallback jo assert Thread.holdsLock(mutex) : "Coordinator mutex not held"; logger.trace("handleJoinRequestUnderLock: as {}, handling {}", mode, joinRequest); - try { - joinRequest.getOptionalJoin().ifPresent(this::handleJoin); - } catch (Exception exception) { - joinCallback.onFailure(exception); - } - - if (mode == Mode.FOLLOWER) { - joinCallback.onFailure(new CoordinationStateRejectedException("join target is a follower")); - return; + final CoordinationState coordState = coordinationState.get(); + final boolean prevElectionWon = coordState.electionWon(); + + if (joinRequest.getOptionalJoin().isPresent()) { + final Join join = joinRequest.getOptionalJoin().get(); + // if someone thinks we should be master, let's add our vote and try to become one + // note that the following line should never throw an exception + ensureTermAtLeast(getLocalNode(), join.getTerm()).ifPresent(coordState::handleJoin); + + // if we have already won the election, then the actual join does not matter for election purposes + if (coordState.electionWon()) { + // add join on a best-effort basis + try { + coordState.handleJoin(join); + } catch (CoordinationStateRejectedException e) { + logger.trace("failed to add join, ignoring", e); + } + } else { + coordState.handleJoin(join); // this might fail and bubble up the exception + } } - if (mode == Mode.LEADER) { + if (prevElectionWon == false && coordState.electionWon()) { + joinHelper.addPendingJoin(joinRequest, joinCallback); + becomeLeader("handleJoin"); + joinHelper.clearAndSubmitPendingJoins(); + } else if (mode == Mode.LEADER) { joinHelper.joinLeader(joinRequest, joinCallback); + } else if (mode == Mode.FOLLOWER) { + joinCallback.onFailure(new CoordinationStateRejectedException("join target is a follower")); } else { assert mode == Mode.CANDIDATE; joinHelper.addPendingJoin(joinRequest, joinCallback); } } - private void handleJoin(Join join) { - assert Thread.holdsLock(mutex) : "Coordinator mutex not held"; - Optional optionalJoinToSelf = ensureTermAtLeast(getLocalNode(), join.getTerm()); - - optionalJoinToSelf.ifPresent(this::handleJoin); // if someone thinks we should be master, let's try to become one - - final CoordinationState state = coordinationState.get(); - boolean prevElectionWon = state.electionWon(); - state.handleJoin(join); - - if (prevElectionWon == false && state.electionWon()) { - becomeLeader("handleJoin"); - } - } - - private void becomeCandidate(String method) { + void becomeCandidate(String method) { assert Thread.holdsLock(mutex) : "Legislator mutex not held"; logger.debug("{}: becoming CANDIDATE (was {}, lastKnownLeader was [{}])", method, mode, lastKnownLeader); @@ -110,23 +112,22 @@ private void becomeCandidate(String method) { } } - private void becomeLeader(String method) { + void becomeLeader(String method) { assert Thread.holdsLock(mutex) : "Legislator mutex not held"; assert mode == Mode.CANDIDATE : "expected candidate but was " + mode; logger.debug("{}: becoming LEADER (was {}, lastKnownLeader was [{}])", method, mode, lastKnownLeader); mode = Mode.LEADER; lastKnownLeader = Optional.of(getLocalNode()); - joinHelper.clearAndSubmitJoins(); } - private void becomeFollower(String method, DiscoveryNode leaderNode) { + void becomeFollower(String method, DiscoveryNode leaderNode) { assert Thread.holdsLock(mutex) : "Legislator mutex not held"; logger.debug("{}: becoming FOLLOWER of [{}] (was {}, lastKnownLeader was [{}])", method, leaderNode, mode, lastKnownLeader); if (mode != Mode.FOLLOWER) { mode = Mode.FOLLOWER; - joinHelper.clearAndFailJoins("becoming follower"); + joinHelper.clearAndFailJoins("following another master : " + leaderNode); } lastKnownLeader = Optional.of(leaderNode); @@ -141,7 +142,9 @@ long getCurrentTerm() { // package-visible for testing Mode getMode() { - return mode; + synchronized (mutex) { + return mode; + } } // package-visible for testing diff --git a/server/src/main/java/org/elasticsearch/cluster/coordination/JoinHelper.java b/server/src/main/java/org/elasticsearch/cluster/coordination/JoinHelper.java index 89e1764ec7516..be764d29779d0 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/JoinHelper.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/JoinHelper.java @@ -18,7 +18,6 @@ */ package org.elasticsearch.cluster.coordination; -import org.apache.logging.log4j.message.ParameterizedMessage; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterStateTaskConfig; import org.elasticsearch.cluster.ClusterStateTaskListener; @@ -27,25 +26,21 @@ import org.elasticsearch.cluster.service.MasterService; import org.elasticsearch.common.Priority; import org.elasticsearch.common.component.AbstractComponent; -import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.threadpool.ThreadPool; -import org.elasticsearch.transport.TransportException; import org.elasticsearch.transport.TransportResponse; -import org.elasticsearch.transport.TransportResponseHandler; import org.elasticsearch.transport.TransportService; import java.io.IOException; import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.Optional; import java.util.function.BiConsumer; import java.util.function.LongSupplier; public class JoinHelper extends AbstractComponent { - public static final String JOIN_ACTION_NAME = "internal:discovery/join"; + public static final String JOIN_ACTION_NAME = "internal:cluster/coordination/join"; private final MasterService masterService; private final TransportService transportService; @@ -101,38 +96,6 @@ public void onFailure(Exception e) { })); } - // TODO: maybe add this method in a follow-up. - public void sendOptionalJoin(DiscoveryNode destination, Optional optionalJoin) { - transportService.sendRequest(destination, JOIN_ACTION_NAME, new JoinRequest(transportService.getLocalNode(), optionalJoin), - new TransportResponseHandler() { - @Override - public TransportResponse.Empty read(StreamInput in) { - return TransportResponse.Empty.INSTANCE; - } - - @Override - public void handleResponse(TransportResponse.Empty response) { - logger.debug("SendJoinResponseHandler: successfully joined {}", destination); - } - - @Override - public void handleException(TransportException exp) { - //TODO: log certain failures at info level so that users can easily see why nodes are failing to join - // see ZenDiscovery.joinElectedMaster for comparison - if (exp.getRootCause() instanceof CoordinationStateRejectedException) { - logger.debug("SendJoinResponseHandler: [{}] failed: {}", destination, exp.getRootCause().getMessage()); - } else { - logger.debug(() -> new ParameterizedMessage("SendJoinResponseHandler: [{}] failed", destination), exp); - } - } - - @Override - public String executor() { - return ThreadPool.Names.SAME; // just logging calls - } - }); - } - public void addPendingJoin(JoinRequest joinRequest, JoinCallback joinCallback) { JoinCallback prev = joinRequestAccumulator.put(joinRequest.getSourceNode(), joinCallback); if (prev != null) { @@ -150,22 +113,16 @@ public void clearAndFailJoins(String reason) { joinRequestAccumulator.clear(); } - public void clearAndSubmitJoins() { + public void clearAndSubmitPendingJoins() { final Map pendingAsTasks = new HashMap<>(); joinRequestAccumulator.forEach((key, value) -> pendingAsTasks.put(new JoinTaskExecutor.Task(key, "elect leader"), new JoinTaskListener(value))); joinRequestAccumulator.clear(); + pendingAsTasks.put(JoinTaskExecutor.BECOME_MASTER_TASK, (source, e) -> {}); + pendingAsTasks.put(JoinTaskExecutor.FINISH_ELECTION_TASK, (source, e) -> {}); final String source = "elected-as-master ([" + pendingAsTasks.size() + "] nodes joined)"; - // noop listener, the election finished listener determines result - pendingAsTasks.put(JoinTaskExecutor.BECOME_MASTER_TASK, (source1, e) -> { - }); - // TODO: should we take any further action when FINISH_ELECTION_TASK fails? - pendingAsTasks.put(JoinTaskExecutor.FINISH_ELECTION_TASK, (source1, e) -> { - }); - - masterService.submitStateUpdateTasks(source, pendingAsTasks, ClusterStateTaskConfig.build(Priority.URGENT), - joinTaskExecutor); + masterService.submitStateUpdateTasks(source, pendingAsTasks, ClusterStateTaskConfig.build(Priority.URGENT), joinTaskExecutor); } public void joinLeader(JoinRequest joinRequest, JoinCallback joinCallback) { 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 4508f56845488..c2640289aff97 100644 --- a/server/src/test/java/org/elasticsearch/cluster/coordination/NodeJoinTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/coordination/NodeJoinTests.java @@ -13,6 +13,8 @@ import org.elasticsearch.cluster.service.MasterServiceTests; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.concurrent.BaseFuture; +import org.elasticsearch.common.util.concurrent.FutureUtils; +import org.elasticsearch.indices.cluster.FakeThreadPoolMasterService; import org.elasticsearch.test.ClusterServiceUtils; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.junit.annotations.TestLogging; @@ -23,26 +25,32 @@ import org.junit.AfterClass; import org.junit.BeforeClass; +import java.util.ArrayList; +import java.util.Collections; import java.util.HashSet; import java.util.List; import java.util.Optional; import java.util.Set; -import java.util.concurrent.ExecutionException; +import java.util.concurrent.BrokenBarrierException; +import java.util.concurrent.CyclicBarrier; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReference; import java.util.stream.Collectors; import java.util.stream.IntStream; import static java.util.Collections.emptyMap; +import static org.hamcrest.Matchers.containsString; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; -@TestLogging("org.elasticsearch.discovery.zen:TRACE,org.elasticsearch.cluster.service:TRACE,org.elasticsearch.cluster.coordination:TRACE") +@TestLogging("org.elasticsearch.cluster.service:TRACE,org.elasticsearch.cluster.coordination:TRACE") public class NodeJoinTests extends ESTestCase { private static ThreadPool threadPool; private MasterService masterService; private Coordinator coordinator; + private DeterministicTaskQueue deterministicTaskQueue; @BeforeClass public static void beforeClass() { @@ -76,11 +84,28 @@ private static ClusterState initialState(boolean withMaster, DiscoveryNode local return initialClusterState; } - private void setupMasterServiceAndNodeJoinController(long term, ClusterState initialState) { - if (masterService != null || coordinator != null) { - throw new IllegalStateException("method setupMasterServiceAndNodeJoinController can only be called once"); + private void setupFakeMasterServiceAndCoordinator(long term, ClusterState initialState) { + deterministicTaskQueue = new DeterministicTaskQueue(Settings.EMPTY); + FakeThreadPoolMasterService fakeMasterService = new FakeThreadPoolMasterService("test", deterministicTaskQueue::scheduleNow); + AtomicReference currentState = new AtomicReference<>(initialState); + fakeMasterService.setClusterStateSupplier(currentState::get); + fakeMasterService.setClusterStatePublisher((event, publishListener, ackListener) -> { + currentState.set(event.state()); + publishListener.onResponse(null); + }); + fakeMasterService.start(); + setupMasterServiceAndCoordinator(term, initialState, fakeMasterService); + } + + private void setupRealMasterServiceAndCoordinator(long term, ClusterState initialState) { + setupMasterServiceAndCoordinator(term, initialState, ClusterServiceUtils.createMasterService(threadPool, initialState)); + } + + private void setupMasterServiceAndCoordinator(long term, ClusterState initialState, MasterService masterService) { + if (this.masterService != null || coordinator != null) { + throw new IllegalStateException("method setupMasterServiceAndCoordinator can only be called once"); } - masterService = ClusterServiceUtils.createMasterService(threadPool, initialState); + this.masterService = masterService; TransportService transportService = mock(TransportService.class); when(transportService.getLocalNode()).thenReturn(initialState.nodes().getLocalNode()); coordinator = new Coordinator(Settings.EMPTY, @@ -105,15 +130,6 @@ protected DiscoveryNode newNode(int i, boolean master) { return new DiscoveryNode(prefix + i, i + "", buildNewFakeTransportAddress(), emptyMap(), roles, Version.CURRENT); } - private DiscoveryNode cloneNode(DiscoveryNode node) { - return new DiscoveryNode(node.getName(), node.getId(), node.getEphemeralId(), node.getHostName(), node.getHostAddress(), - node.getAddress(), node.getAttributes(), node.getRoles(), node.getVersion()); - } - - private void joinNode(final JoinRequest joinRequest) throws InterruptedException, ExecutionException { - joinNodeAsync(joinRequest).get(); - } - static class SimpleFuture extends BaseFuture { final String description; @@ -156,26 +172,191 @@ public void onFailure(Exception e) { return future; } - public void testSuccessfulJoinAccumulation() { + private void joinNode(final JoinRequest joinRequest) { + FutureUtils.get(joinNodeAsync(joinRequest)); + } + + private void joinNodeAndRun(final JoinRequest joinRequest) { + SimpleFuture fut = joinNodeAsync(joinRequest); + deterministicTaskQueue.runAllTasks(random()); + assertTrue(fut.isDone()); + FutureUtils.get(fut); + } + + public void testJoinWithHigherTermElectsLeader() { + DiscoveryNode node0 = newNode(0, true); + DiscoveryNode node1 = newNode(1, true); + setupFakeMasterServiceAndCoordinator(1, initialState(false, node0, 1, 1, + new VotingConfiguration(Collections.singleton(randomFrom(node0, node1).getId())))); + assertFalse(isLocalNodeElectedMaster()); + joinNodeAndRun(new JoinRequest(node1, Optional.of(new Join(node1, node0, 2, 1, 1)))); + assertTrue(isLocalNodeElectedMaster()); + } + + public void testJoinWithHigherTermButBetterStateGetsRejected() { + DiscoveryNode node0 = newNode(0, true); + DiscoveryNode node1 = newNode(1, true); + setupFakeMasterServiceAndCoordinator(1, initialState(false, node0, 1, 1, + new VotingConfiguration(Collections.singleton(node1.getId())))); + assertFalse(isLocalNodeElectedMaster()); + expectThrows(CoordinationStateRejectedException.class, + () -> joinNodeAndRun(new JoinRequest(node1, Optional.of(new Join(node1, node0, 2, 2, 2))))); + assertFalse(isLocalNodeElectedMaster()); + } + + public void testJoinWithHigherTermButBetterStateStillElectsMasterThroughSelfJoin() { + DiscoveryNode node0 = newNode(0, true); + DiscoveryNode node1 = newNode(1, true); + setupFakeMasterServiceAndCoordinator(1, initialState(false, node0, 1, 1, + new VotingConfiguration(Collections.singleton(node0.getId())))); + assertFalse(isLocalNodeElectedMaster()); + joinNodeAndRun(new JoinRequest(node1, Optional.of(new Join(node1, node0, 2, 2, 2)))); + assertTrue(isLocalNodeElectedMaster()); + } + + public void testJoinElectedLeader() { + DiscoveryNode node0 = newNode(0, true); + DiscoveryNode node1 = newNode(1, true); + setupFakeMasterServiceAndCoordinator(1, initialState(false, node0, 1, 1, + new VotingConfiguration(Collections.singleton(node0.getId())))); + assertFalse(isLocalNodeElectedMaster()); + joinNodeAndRun(new JoinRequest(node0, Optional.of(new Join(node0, node0, 2, 1, 1)))); + assertTrue(isLocalNodeElectedMaster()); + assertFalse(clusterStateHasNode(node1)); + joinNodeAndRun(new JoinRequest(node1, Optional.of(new Join(node1, node0, 2, 1, 1)))); + assertTrue(isLocalNodeElectedMaster()); + assertTrue(clusterStateHasNode(node1)); + } + + public void testJoinAccumulation() { + DiscoveryNode node0 = newNode(0, true); + DiscoveryNode node1 = newNode(1, true); + setupFakeMasterServiceAndCoordinator(1, initialState(false, node0, 1, 1, + new VotingConfiguration(Collections.singleton(node1.getId())))); + assertFalse(isLocalNodeElectedMaster()); + SimpleFuture fut = joinNodeAsync(new JoinRequest(node0, Optional.of(new Join(node0, node0, 2, 1, 1)))); + deterministicTaskQueue.runAllTasks(random()); + assertFalse(fut.isDone()); + assertFalse(isLocalNodeElectedMaster()); + joinNodeAndRun(new JoinRequest(node1, Optional.of(new Join(node1, node0, 2, 1, 1)))); + assertTrue(isLocalNodeElectedMaster()); + assertTrue(clusterStateHasNode(node1)); + FutureUtils.get(fut); + } + + public void testJoinFollowerFails() { + DiscoveryNode node0 = newNode(0, true); + DiscoveryNode node1 = newNode(1, true); + setupFakeMasterServiceAndCoordinator(1, initialState(false, node0, 1, 1, + new VotingConfiguration(Collections.singleton(node0.getId())))); + coordinator.coordinationState.get().handleStartJoin(new StartJoinRequest(node1, 2)); + synchronized (coordinator.mutex) { + coordinator.becomeFollower("test", node1); + } + assertFalse(isLocalNodeElectedMaster()); + assertThat(expectThrows(CoordinationStateRejectedException.class, + () -> joinNodeAndRun(new JoinRequest(node1, Optional.of(new Join(node1, node0, 2, 1, 1))))).getMessage(), + containsString("join target is a follower")); + assertFalse(isLocalNodeElectedMaster()); + } + + public void testBecomeFollowerFailsPendingJoin() { + DiscoveryNode node0 = newNode(0, true); + DiscoveryNode node1 = newNode(1, true); + setupFakeMasterServiceAndCoordinator(1, initialState(false, node0, 1, 1, + new VotingConfiguration(Collections.singleton(node1.getId())))); + SimpleFuture fut = joinNodeAsync(new JoinRequest(node0, Optional.of(new Join(node0, node0, 2, 1, 1)))); + deterministicTaskQueue.runAllTasks(random()); + assertFalse(fut.isDone()); + assertFalse(isLocalNodeElectedMaster()); + synchronized (coordinator.mutex) { + coordinator.becomeFollower("test", node1); + } + assertFalse(isLocalNodeElectedMaster()); + assertThat(expectThrows(CoordinationStateRejectedException.class, + () -> FutureUtils.get(fut)).getMessage(), + containsString("following another master")); + assertFalse(isLocalNodeElectedMaster()); + } + + public void testConcurrentJoining() { List nodes = IntStream.rangeClosed(1, randomIntBetween(2, 5)) .mapToObj(nodeId -> newNode(nodeId, true)).collect(Collectors.toList()); VotingConfiguration votingConfiguration = new VotingConfiguration( randomSubsetOf(randomIntBetween(1, nodes.size()), nodes).stream().map(DiscoveryNode::getId).collect(Collectors.toSet())); + logger.info("Voting configuration: {}", votingConfiguration); + DiscoveryNode localNode = nodes.get(0); - setupMasterServiceAndNodeJoinController(1, initialState(false, localNode, 1, 1, votingConfiguration)); + setupRealMasterServiceAndCoordinator(1, initialState(false, localNode, 1, 1, votingConfiguration)); + + // we need at least a quorum of voting nodes with a correct term and worse state + List successfulNodes; + do { + successfulNodes = randomSubsetOf(nodes); + } while (votingConfiguration.hasQuorum(successfulNodes.stream().map(DiscoveryNode::getId).collect(Collectors.toList())) + == false); + + logger.info("Successful voting nodes: {}", successfulNodes); + + List correctJoinRequests = successfulNodes.stream().map( + node -> new JoinRequest(node, Optional.of(new Join(node, localNode, 2, 1, 1)))).collect(Collectors.toList()); + + List possiblyUnsuccessfulNodes = new ArrayList<>(nodes); + possiblyUnsuccessfulNodes.removeAll(successfulNodes); + + logger.info("Possibly unsuccessful voting nodes: {}", possiblyUnsuccessfulNodes); + + List possiblyFailingJoinRequests = possiblyUnsuccessfulNodes.stream().map(node -> { + if (randomBoolean()) { + // a correct request + return new JoinRequest(node, Optional.of(new Join(node, localNode, 2, 1, 1))); + } else if (randomBoolean()) { + // term too low + return new JoinRequest(node, Optional.of(new Join(node, localNode, 1, 1, 1))); + } else { + // better state + return new JoinRequest(node, Optional.of(new Join(node, localNode, 2, 2, 2))); + } + }).collect(Collectors.toList()); - // we need at least a quorum of voting nodes with a correct term + // duplicate some requests, which will be unsuccessful + possiblyFailingJoinRequests.addAll(randomSubsetOf(possiblyFailingJoinRequests)); - List threads = nodes.stream().map(node -> new Thread(() -> { - JoinRequest joinRequest = new JoinRequest(node, Optional.of(new Join(node, localNode, 2, 1, 1))); + CyclicBarrier barrier = new CyclicBarrier(correctJoinRequests.size() + possiblyFailingJoinRequests.size() + 1); + List threads = new ArrayList<>(); + threads.add(new Thread(() -> { try { + barrier.await(); + } catch (InterruptedException | BrokenBarrierException e) { + throw new RuntimeException(e); + } + for (int i = 0; i < 30; i++) { + coordinator.invariant(); + } + })); + threads.addAll(correctJoinRequests.stream().map(joinRequest -> new Thread( + () -> { + try { + barrier.await(); + } catch (InterruptedException | BrokenBarrierException e) { + throw new RuntimeException(e); + } joinNode(joinRequest); - } catch (InterruptedException | ExecutionException e) { + })).collect(Collectors.toList())); + threads.addAll(possiblyFailingJoinRequests.stream().map(joinRequest -> new Thread(() -> { + try { + barrier.await(); + } catch (InterruptedException | BrokenBarrierException e) { throw new RuntimeException(e); } - })).collect(Collectors.toList()); + try { + joinNode(joinRequest); + } catch (CoordinationStateRejectedException ignore) { + // ignore + } + })).collect(Collectors.toList())); threads.forEach(Thread::start); threads.forEach(t -> { @@ -186,64 +367,15 @@ public void testSuccessfulJoinAccumulation() { } }); - -// Thread[] threads = new Thread[3 + randomInt(5)]; -// ArrayList nodes = new ArrayList<>(); -// nodes.add(discoveryState(masterService).nodes().getLocalNode()); -// final CyclicBarrier barrier = new CyclicBarrier(threads.length); -// final List backgroundExceptions = new CopyOnWriteArrayList<>(); -// for (int i = 0; i < threads.length; i++) { -// final DiscoveryNode node = newNode(i); -// final int iterations = rarely() ? randomIntBetween(1, 4) : 1; -// nodes.add(node); -// threads[i] = new Thread(new AbstractRunnable() { -// @Override -// public void onFailure(Exception e) { -// logger.error("unexpected error in join thread", e); -// backgroundExceptions.add(e); -// } -// -// @Override -// protected void doRun() throws Exception { -// barrier.await(); -// for (int i = 0; i < iterations; i++) { -// logger.debug("{} joining", node); -// joinNode(node); -// } -// } -// }, "t_" + i); -// threads[i].start(); -// } -// -// logger.info("--> waiting for joins to complete"); -// for (Thread thread : threads) { -// thread.join(); -// } - assertTrue(MasterServiceTests.discoveryState(masterService).nodes().isLocalNodeElectedMaster()); + successfulNodes.forEach(node -> assertTrue(clusterStateHasNode(node))); + } + + private boolean isLocalNodeElectedMaster() { + return MasterServiceTests.discoveryState(masterService).nodes().isLocalNodeElectedMaster(); + } -// nodeJoinController.startElectionContext(); -// ArrayList> pendingJoins = new ArrayList<>(); -// for (int i = randomInt(5); i > 0; i--) { -// DiscoveryNode node = newNode(nodeId++); -// nodes.add(node); -// pendingJoins.add(joinNodeAsync(node)); -// } -// nodeJoinController.stopElectionContext("test"); -// boolean hadSyncJoin = false; -// for (int i = randomInt(5); i > 0; i--) { -// DiscoveryNode node = newNode(nodeId++); -// nodes.add(node); -// joinNode(node); -// hadSyncJoin = true; -// } -// if (hadSyncJoin) { -// for (Future joinFuture : pendingJoins) { -// assertThat(joinFuture.isDone(), equalTo(true)); -// } -// } -// for (Future joinFuture : pendingJoins) { -// joinFuture.get(); -// } + private boolean clusterStateHasNode(DiscoveryNode node) { + return node.equals(MasterServiceTests.discoveryState(masterService).nodes().get(node.getId())); } } diff --git a/server/src/test/java/org/elasticsearch/indices/cluster/FakeThreadPoolMasterService.java b/server/src/test/java/org/elasticsearch/indices/cluster/FakeThreadPoolMasterService.java index 519704349e1d7..08a7ea38c729f 100644 --- a/server/src/test/java/org/elasticsearch/indices/cluster/FakeThreadPoolMasterService.java +++ b/server/src/test/java/org/elasticsearch/indices/cluster/FakeThreadPoolMasterService.java @@ -50,7 +50,7 @@ public class FakeThreadPoolMasterService extends MasterService { private boolean taskInProgress = false; private boolean waitForPublish = false; - FakeThreadPoolMasterService(String serviceName, Consumer onTaskAvailableToRun) { + public FakeThreadPoolMasterService(String serviceName, Consumer onTaskAvailableToRun) { super(Settings.EMPTY, createMockThreadPool()); this.name = serviceName; this.onTaskAvailableToRun = onTaskAvailableToRun; diff --git a/test/framework/src/main/java/org/elasticsearch/test/ESTestCase.java b/test/framework/src/main/java/org/elasticsearch/test/ESTestCase.java index 922a6e0d27606..599b42633926e 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/ESTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/test/ESTestCase.java @@ -972,7 +972,7 @@ public static List randomSubsetOf(int size, T... values) { * Returns a random subset of values (including a potential empty list) */ public static List randomSubsetOf(Collection collection) { - return randomSubsetOf(randomInt(Math.max(collection.size() - 1, 0)), collection); + return randomSubsetOf(randomInt(collection.size()), collection); } /** From 5fb85fb9097f1128dce2518865a711be62a3b219 Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Tue, 21 Aug 2018 09:15:36 +0200 Subject: [PATCH 09/29] minor stuff --- .../org/elasticsearch/cluster/coordination/Coordinator.java | 2 +- .../org/elasticsearch/cluster/coordination/JoinHelper.java | 2 +- .../org/elasticsearch/discovery/zen/NodeJoinController.java | 6 +++--- 3 files changed, 5 insertions(+), 5 deletions(-) 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 f73946ee30f64..8576c34525593 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/Coordinator.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/Coordinator.java @@ -127,7 +127,7 @@ void becomeFollower(String method, DiscoveryNode leaderNode) { if (mode != Mode.FOLLOWER) { mode = Mode.FOLLOWER; - joinHelper.clearAndFailJoins("following another master : " + leaderNode); + joinHelper.clearAndFailPendingJoins("following another master : " + leaderNode); } lastKnownLeader = Optional.of(leaderNode); diff --git a/server/src/main/java/org/elasticsearch/cluster/coordination/JoinHelper.java b/server/src/main/java/org/elasticsearch/cluster/coordination/JoinHelper.java index be764d29779d0..858c67c80e412 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/JoinHelper.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/JoinHelper.java @@ -107,7 +107,7 @@ public int getNumberOfPendingJoins() { return joinRequestAccumulator.size(); } - public void clearAndFailJoins(String reason) { + public void clearAndFailPendingJoins(String reason) { joinRequestAccumulator.values().forEach( joinCallback -> joinCallback.onFailure(new CoordinationStateRejectedException(reason))); joinRequestAccumulator.clear(); diff --git a/server/src/main/java/org/elasticsearch/discovery/zen/NodeJoinController.java b/server/src/main/java/org/elasticsearch/discovery/zen/NodeJoinController.java index 0bf9296aca511..913008b5c6d20 100644 --- a/server/src/main/java/org/elasticsearch/discovery/zen/NodeJoinController.java +++ b/server/src/main/java/org/elasticsearch/discovery/zen/NodeJoinController.java @@ -341,15 +341,15 @@ public void onFailure(String source, Exception e) { } - public static class JoinTaskListener implements ClusterStateTaskListener { + static class JoinTaskListener implements ClusterStateTaskListener { final List callbacks; private final Logger logger; - public JoinTaskListener(MembershipAction.JoinCallback callback, Logger logger) { + JoinTaskListener(MembershipAction.JoinCallback callback, Logger logger) { this(Collections.singletonList(callback), logger); } - public JoinTaskListener(List callbacks, Logger logger) { + JoinTaskListener(List callbacks, Logger logger) { this.callbacks = callbacks; this.logger = logger; } From 544560afd70f397a2bc314778345f85a68c8612d Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Tue, 21 Aug 2018 09:43:58 +0200 Subject: [PATCH 10/29] licenses --- .../cluster/coordination/JoinRequest.java | 18 ++++++++++++++++++ .../cluster/coordination/NodeJoinTests.java | 18 ++++++++++++++++++ 2 files changed, 36 insertions(+) diff --git a/server/src/main/java/org/elasticsearch/cluster/coordination/JoinRequest.java b/server/src/main/java/org/elasticsearch/cluster/coordination/JoinRequest.java index 1b6170fb040ae..18717d8eff1ad 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/JoinRequest.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/JoinRequest.java @@ -1,3 +1,21 @@ +/* + * 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.node.DiscoveryNode; 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 c2640289aff97..e89fad4e5860c 100644 --- a/server/src/test/java/org/elasticsearch/cluster/coordination/NodeJoinTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/coordination/NodeJoinTests.java @@ -1,3 +1,21 @@ +/* + * 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.apache.logging.log4j.message.ParameterizedMessage; From 85a272a59f6528dfc285716d0080f980d34ad6d9 Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Tue, 21 Aug 2018 09:52:58 +0200 Subject: [PATCH 11/29] moar licenses --- .../cluster/coordination/Coordinator.java | 18 ++++++++++++++++++ 1 file changed, 18 insertions(+) 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 8576c34525593..89baec09f8f64 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/Coordinator.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/Coordinator.java @@ -1,3 +1,21 @@ +/* + * 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.apache.lucene.util.SetOnce; From 2757c072ecc9e7cc8457de64339fd5ad2172805a Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Wed, 22 Aug 2018 13:08:25 +0200 Subject: [PATCH 12/29] Add assertion about source node to join request --- .../org/elasticsearch/cluster/coordination/JoinRequest.java | 1 + .../org/elasticsearch/cluster/coordination/MessagesTests.java | 4 ++-- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/cluster/coordination/JoinRequest.java b/server/src/main/java/org/elasticsearch/cluster/coordination/JoinRequest.java index 18717d8eff1ad..091a6809c84dc 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/JoinRequest.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/JoinRequest.java @@ -33,6 +33,7 @@ public class JoinRequest extends TransportRequest { private final Optional optionalJoin; public JoinRequest(DiscoveryNode sourceNode, Optional optionalJoin) { + assert optionalJoin.isPresent() == false || optionalJoin.get().getSourceNode().equals(sourceNode); this.sourceNode = sourceNode; this.optionalJoin = optionalJoin; } diff --git a/server/src/test/java/org/elasticsearch/cluster/coordination/MessagesTests.java b/server/src/test/java/org/elasticsearch/cluster/coordination/MessagesTests.java index 5914587b91a5f..f47fec2913eee 100644 --- a/server/src/test/java/org/elasticsearch/cluster/coordination/MessagesTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/coordination/MessagesTests.java @@ -176,7 +176,7 @@ public void testJoinRequestEqualsHashCodeSerialization() { EqualsHashCodeTestUtils.checkEqualsAndHashCode(initialJoinRequest, joinRequest -> copyWriteable(joinRequest, writableRegistry(), JoinRequest::new), joinRequest -> { - if (randomBoolean()) { + if (randomBoolean() && joinRequest.getOptionalJoin().isPresent() == false) { return new JoinRequest(createNode(randomAlphaOfLength(20)), joinRequest.getOptionalJoin()); } else { // change OptionalJoin @@ -184,7 +184,7 @@ public void testJoinRequestEqualsHashCodeSerialization() { if (joinRequest.getOptionalJoin().isPresent() && randomBoolean()) { newOptionalJoin = Optional.empty(); } else { - newOptionalJoin = Optional.of(new Join(createNode(randomAlphaOfLength(10)), createNode(randomAlphaOfLength(10)), + newOptionalJoin = Optional.of(new Join(joinRequest.getSourceNode(), createNode(randomAlphaOfLength(10)), randomNonNegativeLong(), randomNonNegativeLong(), randomNonNegativeLong())); } return new JoinRequest(joinRequest.getSourceNode(), newOptionalJoin); From 1ce4fdc3839d7989eecd18a7c708989d3241cc90 Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Wed, 22 Aug 2018 13:10:11 +0200 Subject: [PATCH 13/29] enhance javadoc of randomSubsetOf --- .../src/main/java/org/elasticsearch/test/ESTestCase.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/test/framework/src/main/java/org/elasticsearch/test/ESTestCase.java b/test/framework/src/main/java/org/elasticsearch/test/ESTestCase.java index 599b42633926e..d8cd22d92dbe5 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/ESTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/test/ESTestCase.java @@ -969,7 +969,7 @@ public static List randomSubsetOf(int size, T... values) { } /** - * Returns a random subset of values (including a potential empty list) + * Returns a random subset of values (including a potential empty list, or the full original list) */ public static List randomSubsetOf(Collection collection) { return randomSubsetOf(randomInt(collection.size()), collection); From 46761160a45ab55236ed31473d23bfefa807726c Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Wed, 22 Aug 2018 13:20:17 +0200 Subject: [PATCH 14/29] add toString() --- .../cluster/coordination/JoinHelper.java | 26 ++++++++++++++----- 1 file changed, 20 insertions(+), 6 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/cluster/coordination/JoinHelper.java b/server/src/main/java/org/elasticsearch/cluster/coordination/JoinHelper.java index 858c67c80e412..2d14020da6810 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/JoinHelper.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/JoinHelper.java @@ -93,6 +93,11 @@ public void onFailure(Exception e) { logger.warn("failed to send back failure on join request", inner); } } + + @Override + public String toString() { + return "JoinCallback{request=" + request + "}"; + } })); } @@ -115,8 +120,10 @@ public void clearAndFailPendingJoins(String reason) { public void clearAndSubmitPendingJoins() { final Map pendingAsTasks = new HashMap<>(); - joinRequestAccumulator.forEach((key, value) -> pendingAsTasks.put(new JoinTaskExecutor.Task(key, "elect leader"), - new JoinTaskListener(value))); + joinRequestAccumulator.forEach((key, value) -> { + final JoinTaskExecutor.Task task = new JoinTaskExecutor.Task(key, "elect leader"); + pendingAsTasks.put(task, new JoinTaskListener(task, value)); + }); joinRequestAccumulator.clear(); pendingAsTasks.put(JoinTaskExecutor.BECOME_MASTER_TASK, (source, e) -> {}); @@ -127,9 +134,9 @@ public void clearAndSubmitPendingJoins() { public void joinLeader(JoinRequest joinRequest, JoinCallback joinCallback) { // submit as cluster state update task - masterService.submitStateUpdateTask("node-join", - new JoinTaskExecutor.Task(joinRequest.getSourceNode(), "join existing leader"), ClusterStateTaskConfig.build(Priority.URGENT), - joinTaskExecutor, new JoinTaskListener(joinCallback)); + final JoinTaskExecutor.Task task = new JoinTaskExecutor.Task(joinRequest.getSourceNode(), "join existing leader"); + masterService.submitStateUpdateTask("node-join", task, ClusterStateTaskConfig.build(Priority.URGENT), + joinTaskExecutor, new JoinTaskListener(task, joinCallback)); } public interface JoinCallback { @@ -139,9 +146,11 @@ public interface JoinCallback { } static class JoinTaskListener implements ClusterStateTaskListener { + private final JoinTaskExecutor.Task task; private final JoinCallback joinCallback; - JoinTaskListener(JoinCallback joinCallback) { + JoinTaskListener(JoinTaskExecutor.Task task, JoinCallback joinCallback) { + this.task = task; this.joinCallback = joinCallback; } @@ -154,6 +163,11 @@ public void onFailure(String source, Exception e) { public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) { joinCallback.onSuccess(); } + + @Override + public String toString() { + return "JoinTaskListener{task=" + task + "}"; + } } } From 7dcd65612010983bdf3c80c1d90bbfe27d0438c0 Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Wed, 22 Aug 2018 13:42:40 +0200 Subject: [PATCH 15/29] pick more variable term and version --- .../cluster/coordination/NodeJoinTests.java | 75 +++++++++++++------ 1 file changed, 53 insertions(+), 22 deletions(-) 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 e89fad4e5860c..5d201598004db 100644 --- a/server/src/test/java/org/elasticsearch/cluster/coordination/NodeJoinTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/coordination/NodeJoinTests.java @@ -204,44 +204,58 @@ private void joinNodeAndRun(final JoinRequest joinRequest) { public void testJoinWithHigherTermElectsLeader() { DiscoveryNode node0 = newNode(0, true); DiscoveryNode node1 = newNode(1, true); - setupFakeMasterServiceAndCoordinator(1, initialState(false, node0, 1, 1, + long initialTerm = randomLongBetween(1, 10); + long initialVersion = randomLongBetween(1, 10); + setupFakeMasterServiceAndCoordinator(initialTerm, initialState(false, node0, initialTerm, initialVersion, new VotingConfiguration(Collections.singleton(randomFrom(node0, node1).getId())))); assertFalse(isLocalNodeElectedMaster()); - joinNodeAndRun(new JoinRequest(node1, Optional.of(new Join(node1, node0, 2, 1, 1)))); + long newTerm = initialTerm + randomLongBetween(1, 10); + joinNodeAndRun(new JoinRequest(node1, Optional.of(new Join(node1, node0, newTerm, initialTerm, initialVersion)))); assertTrue(isLocalNodeElectedMaster()); } public void testJoinWithHigherTermButBetterStateGetsRejected() { DiscoveryNode node0 = newNode(0, true); DiscoveryNode node1 = newNode(1, true); - setupFakeMasterServiceAndCoordinator(1, initialState(false, node0, 1, 1, + long initialTerm = randomLongBetween(1, 10); + long initialVersion = randomLongBetween(1, 10); + setupFakeMasterServiceAndCoordinator(initialTerm, initialState(false, node0, initialTerm, initialVersion, new VotingConfiguration(Collections.singleton(node1.getId())))); assertFalse(isLocalNodeElectedMaster()); + long newTerm = initialTerm + randomLongBetween(1, 10); + long higherVersion = initialVersion + randomLongBetween(1, 10); expectThrows(CoordinationStateRejectedException.class, - () -> joinNodeAndRun(new JoinRequest(node1, Optional.of(new Join(node1, node0, 2, 2, 2))))); + () -> joinNodeAndRun(new JoinRequest(node1, Optional.of(new Join(node1, node0, newTerm, initialTerm, higherVersion))))); assertFalse(isLocalNodeElectedMaster()); } public void testJoinWithHigherTermButBetterStateStillElectsMasterThroughSelfJoin() { DiscoveryNode node0 = newNode(0, true); DiscoveryNode node1 = newNode(1, true); - setupFakeMasterServiceAndCoordinator(1, initialState(false, node0, 1, 1, + long initialTerm = randomLongBetween(1, 10); + long initialVersion = randomLongBetween(1, 10); + setupFakeMasterServiceAndCoordinator(initialTerm, initialState(false, node0, initialTerm, initialVersion, new VotingConfiguration(Collections.singleton(node0.getId())))); assertFalse(isLocalNodeElectedMaster()); - joinNodeAndRun(new JoinRequest(node1, Optional.of(new Join(node1, node0, 2, 2, 2)))); + long newTerm = initialTerm + randomLongBetween(1, 10); + long higherVersion = initialVersion + randomLongBetween(1, 10); + joinNodeAndRun(new JoinRequest(node1, Optional.of(new Join(node1, node0, newTerm, initialTerm, higherVersion)))); assertTrue(isLocalNodeElectedMaster()); } public void testJoinElectedLeader() { DiscoveryNode node0 = newNode(0, true); DiscoveryNode node1 = newNode(1, true); - setupFakeMasterServiceAndCoordinator(1, initialState(false, node0, 1, 1, + long initialTerm = randomLongBetween(1, 10); + long initialVersion = randomLongBetween(1, 10); + setupFakeMasterServiceAndCoordinator(initialTerm, initialState(false, node0, initialTerm, initialVersion, new VotingConfiguration(Collections.singleton(node0.getId())))); assertFalse(isLocalNodeElectedMaster()); - joinNodeAndRun(new JoinRequest(node0, Optional.of(new Join(node0, node0, 2, 1, 1)))); + long newTerm = initialTerm + randomLongBetween(1, 10); + joinNodeAndRun(new JoinRequest(node0, Optional.of(new Join(node0, node0, newTerm, initialTerm, initialVersion)))); assertTrue(isLocalNodeElectedMaster()); assertFalse(clusterStateHasNode(node1)); - joinNodeAndRun(new JoinRequest(node1, Optional.of(new Join(node1, node0, 2, 1, 1)))); + joinNodeAndRun(new JoinRequest(node1, Optional.of(new Join(node1, node0, newTerm, initialTerm, initialVersion)))); assertTrue(isLocalNodeElectedMaster()); assertTrue(clusterStateHasNode(node1)); } @@ -249,14 +263,17 @@ public void testJoinElectedLeader() { public void testJoinAccumulation() { DiscoveryNode node0 = newNode(0, true); DiscoveryNode node1 = newNode(1, true); - setupFakeMasterServiceAndCoordinator(1, initialState(false, node0, 1, 1, + long initialTerm = randomLongBetween(1, 10); + long initialVersion = randomLongBetween(1, 10); + setupFakeMasterServiceAndCoordinator(initialTerm, initialState(false, node0, initialTerm, initialVersion, new VotingConfiguration(Collections.singleton(node1.getId())))); assertFalse(isLocalNodeElectedMaster()); - SimpleFuture fut = joinNodeAsync(new JoinRequest(node0, Optional.of(new Join(node0, node0, 2, 1, 1)))); + long newTerm = initialTerm + randomLongBetween(1, 10); + SimpleFuture fut = joinNodeAsync(new JoinRequest(node0, Optional.of(new Join(node0, node0, newTerm, initialTerm, initialVersion)))); deterministicTaskQueue.runAllTasks(random()); assertFalse(fut.isDone()); assertFalse(isLocalNodeElectedMaster()); - joinNodeAndRun(new JoinRequest(node1, Optional.of(new Join(node1, node0, 2, 1, 1)))); + joinNodeAndRun(new JoinRequest(node1, Optional.of(new Join(node1, node0, newTerm, initialTerm, initialVersion)))); assertTrue(isLocalNodeElectedMaster()); assertTrue(clusterStateHasNode(node1)); FutureUtils.get(fut); @@ -265,15 +282,19 @@ public void testJoinAccumulation() { public void testJoinFollowerFails() { DiscoveryNode node0 = newNode(0, true); DiscoveryNode node1 = newNode(1, true); - setupFakeMasterServiceAndCoordinator(1, initialState(false, node0, 1, 1, + long initialTerm = randomLongBetween(1, 10); + long initialVersion = randomLongBetween(1, 10); + setupFakeMasterServiceAndCoordinator(initialTerm, initialState(false, node0, initialTerm, initialVersion, new VotingConfiguration(Collections.singleton(node0.getId())))); - coordinator.coordinationState.get().handleStartJoin(new StartJoinRequest(node1, 2)); + long newTerm = initialTerm + randomLongBetween(1, 10); + coordinator.coordinationState.get().handleStartJoin(new StartJoinRequest(node1, newTerm)); synchronized (coordinator.mutex) { coordinator.becomeFollower("test", node1); } assertFalse(isLocalNodeElectedMaster()); assertThat(expectThrows(CoordinationStateRejectedException.class, - () -> joinNodeAndRun(new JoinRequest(node1, Optional.of(new Join(node1, node0, 2, 1, 1))))).getMessage(), + () -> joinNodeAndRun(new JoinRequest(node1, + Optional.of(new Join(node1, node0, newTerm, initialTerm, initialVersion))))).getMessage(), containsString("join target is a follower")); assertFalse(isLocalNodeElectedMaster()); } @@ -281,9 +302,12 @@ public void testJoinFollowerFails() { public void testBecomeFollowerFailsPendingJoin() { DiscoveryNode node0 = newNode(0, true); DiscoveryNode node1 = newNode(1, true); - setupFakeMasterServiceAndCoordinator(1, initialState(false, node0, 1, 1, + long initialTerm = randomLongBetween(1, 10); + long initialVersion = randomLongBetween(1, 10); + setupFakeMasterServiceAndCoordinator(initialTerm, initialState(false, node0, initialTerm, initialVersion, new VotingConfiguration(Collections.singleton(node1.getId())))); - SimpleFuture fut = joinNodeAsync(new JoinRequest(node0, Optional.of(new Join(node0, node0, 2, 1, 1)))); + long newTerm = initialTerm + randomLongBetween(1, 10); + SimpleFuture fut = joinNodeAsync(new JoinRequest(node0, Optional.of(new Join(node0, node0, newTerm, initialTerm, initialVersion)))); deterministicTaskQueue.runAllTasks(random()); assertFalse(fut.isDone()); assertFalse(isLocalNodeElectedMaster()); @@ -307,7 +331,10 @@ public void testConcurrentJoining() { logger.info("Voting configuration: {}", votingConfiguration); DiscoveryNode localNode = nodes.get(0); - setupRealMasterServiceAndCoordinator(1, initialState(false, localNode, 1, 1, votingConfiguration)); + long initialTerm = randomLongBetween(1, 10); + long initialVersion = randomLongBetween(1, 10); + setupRealMasterServiceAndCoordinator(initialTerm, initialState(false, localNode, initialTerm, initialVersion, votingConfiguration)); + long newTerm = initialTerm + randomLongBetween(1, 10); // we need at least a quorum of voting nodes with a correct term and worse state List successfulNodes; @@ -319,7 +346,8 @@ public void testConcurrentJoining() { logger.info("Successful voting nodes: {}", successfulNodes); List correctJoinRequests = successfulNodes.stream().map( - node -> new JoinRequest(node, Optional.of(new Join(node, localNode, 2, 1, 1)))).collect(Collectors.toList()); + node -> new JoinRequest(node, Optional.of(new Join(node, localNode, newTerm, initialTerm, initialVersion)))) + .collect(Collectors.toList()); List possiblyUnsuccessfulNodes = new ArrayList<>(nodes); possiblyUnsuccessfulNodes.removeAll(successfulNodes); @@ -329,13 +357,16 @@ public void testConcurrentJoining() { List possiblyFailingJoinRequests = possiblyUnsuccessfulNodes.stream().map(node -> { if (randomBoolean()) { // a correct request - return new JoinRequest(node, Optional.of(new Join(node, localNode, 2, 1, 1))); + return new JoinRequest(node, Optional.of(new Join(node, localNode, + newTerm, initialTerm, initialVersion))); } else if (randomBoolean()) { // term too low - return new JoinRequest(node, Optional.of(new Join(node, localNode, 1, 1, 1))); + return new JoinRequest(node, Optional.of(new Join(node, localNode, + randomLongBetween(0, initialTerm), initialTerm, initialVersion))); } else { // better state - return new JoinRequest(node, Optional.of(new Join(node, localNode, 2, 2, 2))); + return new JoinRequest(node, Optional.of(new Join(node, localNode, + newTerm, initialTerm, initialVersion + randomLongBetween(1, 10)))); } }).collect(Collectors.toList()); From bd242edfea16986877b15300f2e108cdc6cfc007 Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Wed, 22 Aug 2018 14:01:51 +0200 Subject: [PATCH 16/29] simplify cases --- .../cluster/coordination/Coordinator.java | 28 +++++++++++-------- .../cluster/coordination/NodeJoinTests.java | 22 +++++++++++++-- 2 files changed, 37 insertions(+), 13 deletions(-) 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 89baec09f8f64..e6d5de6abcf4f 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/Coordinator.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/Coordinator.java @@ -107,17 +107,23 @@ private void handleJoinRequestUnderLock(JoinRequest joinRequest, JoinCallback jo } } - if (prevElectionWon == false && coordState.electionWon()) { - joinHelper.addPendingJoin(joinRequest, joinCallback); - becomeLeader("handleJoin"); - joinHelper.clearAndSubmitPendingJoins(); - } else if (mode == Mode.LEADER) { - joinHelper.joinLeader(joinRequest, joinCallback); - } else if (mode == Mode.FOLLOWER) { - joinCallback.onFailure(new CoordinationStateRejectedException("join target is a follower")); - } else { - assert mode == Mode.CANDIDATE; - joinHelper.addPendingJoin(joinRequest, joinCallback); + switch (mode) { + case LEADER: + joinHelper.joinLeader(joinRequest, joinCallback); + break; + case FOLLOWER: + assert joinRequest.getOptionalJoin().isPresent() == false : "follower should not have solicited join " + joinRequest; + joinCallback.onFailure(new CoordinationStateRejectedException("join target is a follower")); + break; + case CANDIDATE: + joinHelper.addPendingJoin(joinRequest, joinCallback); + if (prevElectionWon == false && coordState.electionWon()) { + becomeLeader("handleJoin"); + joinHelper.clearAndSubmitPendingJoins(); + } + break; + default: + throw new AssertionError("unexpected mode when handling join: " + mode); } } 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 5d201598004db..5bbfd62be0e8b 100644 --- a/server/src/test/java/org/elasticsearch/cluster/coordination/NodeJoinTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/coordination/NodeJoinTests.java @@ -279,6 +279,25 @@ public void testJoinAccumulation() { FutureUtils.get(fut); } + public void testJoinFollowerWithHigherTerm() { + DiscoveryNode node0 = newNode(0, true); + DiscoveryNode node1 = newNode(1, true); + long initialTerm = randomLongBetween(1, 10); + long initialVersion = randomLongBetween(1, 10); + setupFakeMasterServiceAndCoordinator(initialTerm, initialState(false, node0, initialTerm, initialVersion, + new VotingConfiguration(Collections.singleton(node0.getId())))); + long newTerm = initialTerm + randomLongBetween(1, 10); + coordinator.coordinationState.get().handleStartJoin(new StartJoinRequest(node1, newTerm)); + synchronized (coordinator.mutex) { + coordinator.becomeFollower("test", node1); + } + assertFalse(isLocalNodeElectedMaster()); + long newerTerm = newTerm + randomLongBetween(1, 10); + joinNodeAndRun(new JoinRequest(node1, + Optional.of(new Join(node1, node0, newerTerm, initialTerm, initialVersion)))); + assertTrue(isLocalNodeElectedMaster()); + } + public void testJoinFollowerFails() { DiscoveryNode node0 = newNode(0, true); DiscoveryNode node1 = newNode(1, true); @@ -293,8 +312,7 @@ public void testJoinFollowerFails() { } assertFalse(isLocalNodeElectedMaster()); assertThat(expectThrows(CoordinationStateRejectedException.class, - () -> joinNodeAndRun(new JoinRequest(node1, - Optional.of(new Join(node1, node0, newTerm, initialTerm, initialVersion))))).getMessage(), + () -> joinNodeAndRun(new JoinRequest(node1, Optional.empty()))).getMessage(), containsString("join target is a follower")); assertFalse(isLocalNodeElectedMaster()); } From 6f58a8c1c57e8a3ed17b908eed20c05324cb837c Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Wed, 22 Aug 2018 16:30:53 +0200 Subject: [PATCH 17/29] try moving code --- .../cluster/coordination/Coordinator.java | 21 ++++------------ .../cluster/coordination/JoinHelper.java | 24 ++++++++++++++++--- 2 files changed, 25 insertions(+), 20 deletions(-) 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 e6d5de6abcf4f..6d39f1c61155b 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/Coordinator.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/Coordinator.java @@ -107,24 +107,11 @@ private void handleJoinRequestUnderLock(JoinRequest joinRequest, JoinCallback jo } } - switch (mode) { - case LEADER: - joinHelper.joinLeader(joinRequest, joinCallback); - break; - case FOLLOWER: - assert joinRequest.getOptionalJoin().isPresent() == false : "follower should not have solicited join " + joinRequest; - joinCallback.onFailure(new CoordinationStateRejectedException("join target is a follower")); - break; - case CANDIDATE: - joinHelper.addPendingJoin(joinRequest, joinCallback); - if (prevElectionWon == false && coordState.electionWon()) { - becomeLeader("handleJoin"); - joinHelper.clearAndSubmitPendingJoins(); - } - break; - default: - throw new AssertionError("unexpected mode when handling join: " + mode); + final boolean justBecameLeader = prevElectionWon == false && coordState.electionWon(); + if (justBecameLeader) { + becomeLeader("handleJoin"); } + joinHelper.join(mode, joinRequest, joinCallback, justBecameLeader); } void becomeCandidate(String method) { diff --git a/server/src/main/java/org/elasticsearch/cluster/coordination/JoinHelper.java b/server/src/main/java/org/elasticsearch/cluster/coordination/JoinHelper.java index 2d14020da6810..71b2985d1bb7d 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/JoinHelper.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/JoinHelper.java @@ -21,6 +21,7 @@ import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterStateTaskConfig; import org.elasticsearch.cluster.ClusterStateTaskListener; +import org.elasticsearch.cluster.coordination.Coordinator.Mode; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.routing.allocation.AllocationService; import org.elasticsearch.cluster.service.MasterService; @@ -101,7 +102,7 @@ public String toString() { })); } - public void addPendingJoin(JoinRequest joinRequest, JoinCallback joinCallback) { + private void addPendingJoin(JoinRequest joinRequest, JoinCallback joinCallback) { JoinCallback prev = joinRequestAccumulator.put(joinRequest.getSourceNode(), joinCallback); if (prev != null) { prev.onFailure(new CoordinationStateRejectedException("received a newer join from " + joinRequest.getSourceNode())); @@ -118,7 +119,7 @@ public void clearAndFailPendingJoins(String reason) { joinRequestAccumulator.clear(); } - public void clearAndSubmitPendingJoins() { + private void clearAndSubmitPendingJoins() { final Map pendingAsTasks = new HashMap<>(); joinRequestAccumulator.forEach((key, value) -> { final JoinTaskExecutor.Task task = new JoinTaskExecutor.Task(key, "elect leader"); @@ -132,13 +133,30 @@ public void clearAndSubmitPendingJoins() { masterService.submitStateUpdateTasks(source, pendingAsTasks, ClusterStateTaskConfig.build(Priority.URGENT), joinTaskExecutor); } - public void joinLeader(JoinRequest joinRequest, JoinCallback joinCallback) { + private void joinLeader(JoinRequest joinRequest, JoinCallback joinCallback) { // submit as cluster state update task final JoinTaskExecutor.Task task = new JoinTaskExecutor.Task(joinRequest.getSourceNode(), "join existing leader"); masterService.submitStateUpdateTask("node-join", task, ClusterStateTaskConfig.build(Priority.URGENT), joinTaskExecutor, new JoinTaskListener(task, joinCallback)); } + public void join(Mode mode, JoinRequest joinRequest, JoinCallback joinCallback, boolean justBecameLeader) { + if (mode == Mode.LEADER && justBecameLeader == false) { + joinLeader(joinRequest, joinCallback); + } else if (mode == Mode.FOLLOWER) { + assert joinRequest.getOptionalJoin().isPresent() == false : "follower should not have solicited join " + joinRequest; + joinCallback.onFailure(new CoordinationStateRejectedException("join target is a follower")); + } else { + addPendingJoin(joinRequest, joinCallback); + if (justBecameLeader) { + assert mode == Mode.LEADER; + clearAndSubmitPendingJoins(); + } else { + assert mode == Mode.CANDIDATE; + } + } + } + public interface JoinCallback { void onSuccess(); From a4b63f552f43d3f712d892bef40bd652133e97f1 Mon Sep 17 00:00:00 2001 From: David Turner Date: Wed, 22 Aug 2018 18:33:29 +0100 Subject: [PATCH 18/29] More separation between JoinHelper and Coordinator --- .../cluster/coordination/Coordinator.java | 43 +++-- .../cluster/coordination/JoinHelper.java | 158 ++++++++++++------ .../cluster/coordination/NodeJoinTests.java | 2 +- 3 files changed, 132 insertions(+), 71 deletions(-) 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 6d39f1c61155b..1bb39eee6c79d 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/Coordinator.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/Coordinator.java @@ -47,7 +47,7 @@ public Coordinator(Settings settings, TransportService transportService, Allocat super(settings); this.transportService = transportService; this.joinHelper = new JoinHelper(settings, allocationService, masterService, transportService, this::getCurrentTerm, - this::handleJoinRequest); + this::handleJoin); this.persistedStateSupplier = persistedStateSupplier; this.lastKnownLeader = Optional.empty(); this.lastJoin = Optional.empty(); @@ -72,31 +72,21 @@ private Join joinLeaderInTerm(StartJoinRequest startJoinRequest) { return join; } - public void handleJoinRequest(JoinRequest joinRequest, JoinCallback joinCallback) { + private boolean handleJoin(final Join join) { assert Thread.holdsLock(mutex) == false; - transportService.connectToNode(joinRequest.getSourceNode()); synchronized (mutex) { - handleJoinRequestUnderLock(joinRequest, joinCallback); - } - } + logger.trace("handleJoin: as {}, handling {}", mode, join); - private void handleJoinRequestUnderLock(JoinRequest joinRequest, JoinCallback joinCallback) { - assert Thread.holdsLock(mutex) : "Coordinator mutex not held"; - logger.trace("handleJoinRequestUnderLock: as {}, handling {}", mode, joinRequest); - - final CoordinationState coordState = coordinationState.get(); - final boolean prevElectionWon = coordState.electionWon(); + final CoordinationState coordState = coordinationState.get(); + final boolean prevElectionWon = coordState.electionWon(); - if (joinRequest.getOptionalJoin().isPresent()) { - final Join join = joinRequest.getOptionalJoin().get(); // if someone thinks we should be master, let's add our vote and try to become one // note that the following line should never throw an exception ensureTermAtLeast(getLocalNode(), join.getTerm()).ifPresent(coordState::handleJoin); - // if we have already won the election, then the actual join does not matter for election purposes if (coordState.electionWon()) { - // add join on a best-effort basis + // if we have already won the election then the actual join does not matter for election purposes, so swallow any exception try { coordState.handleJoin(join); } catch (CoordinationStateRejectedException e) { @@ -105,13 +95,14 @@ private void handleJoinRequestUnderLock(JoinRequest joinRequest, JoinCallback jo } else { coordState.handleJoin(join); // this might fail and bubble up the exception } - } - final boolean justBecameLeader = prevElectionWon == false && coordState.electionWon(); - if (justBecameLeader) { - becomeLeader("handleJoin"); + if (prevElectionWon == false && coordState.electionWon()) { + becomeLeader("handleJoin"); + return true; + } } - joinHelper.join(mode, joinRequest, joinCallback, justBecameLeader); + + return false; } void becomeCandidate(String method) { @@ -120,6 +111,7 @@ void becomeCandidate(String method) { if (mode != Mode.CANDIDATE) { mode = Mode.CANDIDATE; + joinHelper.becomeCandidate(); } } @@ -138,7 +130,7 @@ void becomeFollower(String method, DiscoveryNode leaderNode) { if (mode != Mode.FOLLOWER) { mode = Mode.FOLLOWER; - joinHelper.clearAndFailPendingJoins("following another master : " + leaderNode); + joinHelper.becomeFollower(leaderNode); } lastKnownLeader = Optional.of(leaderNode); @@ -190,7 +182,7 @@ public void invariant() { if (mode == Mode.LEADER) { assert coordinationState.get().electionWon(); assert lastKnownLeader.isPresent() && lastKnownLeader.get().equals(getLocalNode()); - assert joinHelper.getNumberOfPendingJoins() == 0; + // assert joinHelper.getNumberOfPendingJoins() == 0; // not true any more, may not have submitted to the master service yet } else if (mode == Mode.FOLLOWER) { assert coordinationState.get().electionWon() == false : getLocalNode() + " is FOLLOWER so electionWon() should be false"; assert lastKnownLeader.isPresent() && (lastKnownLeader.get().equals(getLocalNode()) == false); @@ -201,6 +193,11 @@ public void invariant() { } } + // this is just used because the test doesn't simulate sending the join requests through the transport service - TODO remove it + public void handleJoinRequest(JoinRequest joinRequest, JoinCallback joinCallback) { + joinHelper.handleJoinRequest(joinRequest, joinCallback); + } + public enum Mode { CANDIDATE, LEADER, FOLLOWER } diff --git a/server/src/main/java/org/elasticsearch/cluster/coordination/JoinHelper.java b/server/src/main/java/org/elasticsearch/cluster/coordination/JoinHelper.java index 71b2985d1bb7d..11dc031a84d7f 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/JoinHelper.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/JoinHelper.java @@ -36,8 +36,11 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Optional; +import java.util.concurrent.atomic.AtomicReference; import java.util.function.BiConsumer; import java.util.function.LongSupplier; +import java.util.function.Predicate; public class JoinHelper extends AbstractComponent { @@ -45,15 +48,18 @@ public class JoinHelper extends AbstractComponent { private final MasterService masterService; private final TransportService transportService; + private final Predicate joinHandler; private final JoinTaskExecutor joinTaskExecutor; - private final Map joinRequestAccumulator = new HashMap<>(); + private final Object mutex = new Object(); + private JoinAccumulator joinAccumulator = new CandidateJoinAccumulator(); public JoinHelper(Settings settings, AllocationService allocationService, MasterService masterService, TransportService transportService, LongSupplier currentTermSupplier, - BiConsumer joinRequestHandler) { + Predicate joinHandler) { super(settings); this.masterService = masterService; this.transportService = transportService; + this.joinHandler = joinHandler; this.joinTaskExecutor = new JoinTaskExecutor(allocationService, logger) { @Override @@ -75,7 +81,8 @@ public ClusterTasksResult execute(ClusterState currentSta }; transportService.registerRequestHandler(JOIN_ACTION_NAME, ThreadPool.Names.GENERIC, false, false, JoinRequest::new, - (request, channel, task) -> joinRequestHandler.accept(request, new JoinCallback() { + (request, channel, task) -> handleJoinRequest(request, new JoinCallback() { + @Override public void onSuccess() { try { @@ -102,58 +109,39 @@ public String toString() { })); } - private void addPendingJoin(JoinRequest joinRequest, JoinCallback joinCallback) { - JoinCallback prev = joinRequestAccumulator.put(joinRequest.getSourceNode(), joinCallback); - if (prev != null) { - prev.onFailure(new CoordinationStateRejectedException("received a newer join from " + joinRequest.getSourceNode())); - } - } + void handleJoinRequest(JoinRequest joinRequest, JoinCallback joinCallback) { + transportService.connectToNode(joinRequest.getSourceNode()); - public int getNumberOfPendingJoins() { - return joinRequestAccumulator.size(); - } + final Optional optionalJoin = joinRequest.getOptionalJoin(); + final boolean justBecameLeader = optionalJoin.isPresent() && joinHandler.test(optionalJoin.get()); + + synchronized (mutex) { + joinAccumulator.handleJoinRequest(joinRequest.getSourceNode(), joinCallback); - public void clearAndFailPendingJoins(String reason) { - joinRequestAccumulator.values().forEach( - joinCallback -> joinCallback.onFailure(new CoordinationStateRejectedException(reason))); - joinRequestAccumulator.clear(); + if (justBecameLeader) { + joinAccumulator.clearAndSubmitPendingJoins(); + joinAccumulator = new LeaderJoinAccumulator(); + } + } } - private void clearAndSubmitPendingJoins() { - final Map pendingAsTasks = new HashMap<>(); - joinRequestAccumulator.forEach((key, value) -> { - final JoinTaskExecutor.Task task = new JoinTaskExecutor.Task(key, "elect leader"); - pendingAsTasks.put(task, new JoinTaskListener(task, value)); - }); - joinRequestAccumulator.clear(); - - pendingAsTasks.put(JoinTaskExecutor.BECOME_MASTER_TASK, (source, e) -> {}); - pendingAsTasks.put(JoinTaskExecutor.FINISH_ELECTION_TASK, (source, e) -> {}); - final String source = "elected-as-master ([" + pendingAsTasks.size() + "] nodes joined)"; - masterService.submitStateUpdateTasks(source, pendingAsTasks, ClusterStateTaskConfig.build(Priority.URGENT), joinTaskExecutor); + void becomeCandidate() { + synchronized (mutex) { + joinAccumulator.clearAndFailPendingJoins("becoming candidate"); + joinAccumulator = new CandidateJoinAccumulator(); + } } - private void joinLeader(JoinRequest joinRequest, JoinCallback joinCallback) { - // submit as cluster state update task - final JoinTaskExecutor.Task task = new JoinTaskExecutor.Task(joinRequest.getSourceNode(), "join existing leader"); - masterService.submitStateUpdateTask("node-join", task, ClusterStateTaskConfig.build(Priority.URGENT), - joinTaskExecutor, new JoinTaskListener(task, joinCallback)); + void becomeFollower(DiscoveryNode leaderNode) { + synchronized (mutex) { + joinAccumulator.clearAndFailPendingJoins("started following " + leaderNode); + joinAccumulator = new FollowerJoinAccumulator(); + } } - public void join(Mode mode, JoinRequest joinRequest, JoinCallback joinCallback, boolean justBecameLeader) { - if (mode == Mode.LEADER && justBecameLeader == false) { - joinLeader(joinRequest, joinCallback); - } else if (mode == Mode.FOLLOWER) { - assert joinRequest.getOptionalJoin().isPresent() == false : "follower should not have solicited join " + joinRequest; - joinCallback.onFailure(new CoordinationStateRejectedException("join target is a follower")); - } else { - addPendingJoin(joinRequest, joinCallback); - if (justBecameLeader) { - assert mode == Mode.LEADER; - clearAndSubmitPendingJoins(); - } else { - assert mode == Mode.CANDIDATE; - } + public int getNumberOfPendingJoins() { + synchronized (mutex) { + return joinAccumulator.getNumberOfPendingJoins(); } } @@ -188,4 +176,80 @@ public String toString() { } } + + private interface JoinAccumulator { + void handleJoinRequest(DiscoveryNode sender, JoinCallback joinCallback); + + default void clearAndFailPendingJoins(String reason) { + } + + default void clearAndSubmitPendingJoins() { + } + + default int getNumberOfPendingJoins() { + return 0; + } + } + + private class LeaderJoinAccumulator implements JoinAccumulator { + @Override + public void handleJoinRequest(DiscoveryNode sender, JoinCallback joinCallback) { + final JoinTaskExecutor.Task task = new JoinTaskExecutor.Task(sender, "join existing leader"); + masterService.submitStateUpdateTask("node-join", task, ClusterStateTaskConfig.build(Priority.URGENT), + joinTaskExecutor, new JoinTaskListener(task, joinCallback)); + } + + @Override + public String toString() { + return "LeaderJoinAccumulator"; + } + } + + private static class FollowerJoinAccumulator implements JoinAccumulator { + @Override + public void handleJoinRequest(DiscoveryNode sender, JoinCallback joinCallback) { + joinCallback.onFailure(new CoordinationStateRejectedException("join target is a follower")); + } + + @Override + public String toString() { + return "FollowerJoinAccumulator"; + } + } + + private class CandidateJoinAccumulator implements JoinAccumulator { + + private final Map joinRequestAccumulator = new HashMap<>(); + + @Override + public void handleJoinRequest(DiscoveryNode sender, JoinCallback joinCallback) { + JoinCallback prev = joinRequestAccumulator.put(sender, joinCallback); + if (prev != null) { + prev.onFailure(new CoordinationStateRejectedException("received a newer join from " + sender)); + } + } + + @Override + public void clearAndFailPendingJoins(String reason) { + joinRequestAccumulator.values().forEach( + joinCallback -> joinCallback.onFailure(new CoordinationStateRejectedException(reason))); + joinRequestAccumulator.clear(); + } + + @Override + public void clearAndSubmitPendingJoins() { + final Map pendingAsTasks = new HashMap<>(); + joinRequestAccumulator.forEach((key, value) -> { + final JoinTaskExecutor.Task task = new JoinTaskExecutor.Task(key, "elect leader"); + pendingAsTasks.put(task, new JoinTaskListener(task, value)); + }); + + pendingAsTasks.put(JoinTaskExecutor.BECOME_MASTER_TASK, (source, e) -> { + }); + pendingAsTasks.put(JoinTaskExecutor.FINISH_ELECTION_TASK, (source, e) -> { + }); + final String source = "elected-as-master ([" + pendingAsTasks.size() + "] nodes joined)"; + masterService.submitStateUpdateTasks(source, pendingAsTasks, ClusterStateTaskConfig.build(Priority.URGENT), joinTaskExecutor); + } + } } 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 5bbfd62be0e8b..946162bdaf068 100644 --- a/server/src/test/java/org/elasticsearch/cluster/coordination/NodeJoinTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/coordination/NodeJoinTests.java @@ -335,7 +335,7 @@ public void testBecomeFollowerFailsPendingJoin() { assertFalse(isLocalNodeElectedMaster()); assertThat(expectThrows(CoordinationStateRejectedException.class, () -> FutureUtils.get(fut)).getMessage(), - containsString("following another master")); + containsString("started following")); assertFalse(isLocalNodeElectedMaster()); } From a2e9fc327763b5bfc5a0bfa94da98cbf264a7f54 Mon Sep 17 00:00:00 2001 From: David Turner Date: Thu, 23 Aug 2018 06:18:51 +0100 Subject: [PATCH 19/29] Cleanup --- .../cluster/coordination/JoinHelper.java | 21 +++++++------------ 1 file changed, 8 insertions(+), 13 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/cluster/coordination/JoinHelper.java b/server/src/main/java/org/elasticsearch/cluster/coordination/JoinHelper.java index 11dc031a84d7f..91b4476507e3a 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/JoinHelper.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/JoinHelper.java @@ -21,7 +21,6 @@ import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterStateTaskConfig; import org.elasticsearch.cluster.ClusterStateTaskListener; -import org.elasticsearch.cluster.coordination.Coordinator.Mode; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.routing.allocation.AllocationService; import org.elasticsearch.cluster.service.MasterService; @@ -37,8 +36,6 @@ import java.util.List; import java.util.Map; import java.util.Optional; -import java.util.concurrent.atomic.AtomicReference; -import java.util.function.BiConsumer; import java.util.function.LongSupplier; import java.util.function.Predicate; @@ -119,7 +116,7 @@ void handleJoinRequest(JoinRequest joinRequest, JoinCallback joinCallback) { joinAccumulator.handleJoinRequest(joinRequest.getSourceNode(), joinCallback); if (justBecameLeader) { - joinAccumulator.clearAndSubmitPendingJoins(); + joinAccumulator.submitPendingJoins(); joinAccumulator = new LeaderJoinAccumulator(); } } @@ -127,14 +124,14 @@ void handleJoinRequest(JoinRequest joinRequest, JoinCallback joinCallback) { void becomeCandidate() { synchronized (mutex) { - joinAccumulator.clearAndFailPendingJoins("becoming candidate"); + joinAccumulator.failPendingJoins("becoming candidate"); joinAccumulator = new CandidateJoinAccumulator(); } } void becomeFollower(DiscoveryNode leaderNode) { synchronized (mutex) { - joinAccumulator.clearAndFailPendingJoins("started following " + leaderNode); + joinAccumulator.failPendingJoins("started following " + leaderNode); joinAccumulator = new FollowerJoinAccumulator(); } } @@ -180,10 +177,10 @@ public String toString() { private interface JoinAccumulator { void handleJoinRequest(DiscoveryNode sender, JoinCallback joinCallback); - default void clearAndFailPendingJoins(String reason) { + default void failPendingJoins(String reason) { } - default void clearAndSubmitPendingJoins() { + default void submitPendingJoins() { } default int getNumberOfPendingJoins() { @@ -230,14 +227,12 @@ public void handleJoinRequest(DiscoveryNode sender, JoinCallback joinCallback) { } @Override - public void clearAndFailPendingJoins(String reason) { - joinRequestAccumulator.values().forEach( - joinCallback -> joinCallback.onFailure(new CoordinationStateRejectedException(reason))); - joinRequestAccumulator.clear(); + public void failPendingJoins(String reason) { + joinRequestAccumulator.values().forEach(joinCallback -> joinCallback.onFailure(new CoordinationStateRejectedException(reason))); } @Override - public void clearAndSubmitPendingJoins() { + public void submitPendingJoins() { final Map pendingAsTasks = new HashMap<>(); joinRequestAccumulator.forEach((key, value) -> { final JoinTaskExecutor.Task task = new JoinTaskExecutor.Task(key, "elect leader"); From 5a0efb5a0d4f1044620ab4eebc5bbf6f7912e299 Mon Sep 17 00:00:00 2001 From: David Turner Date: Thu, 23 Aug 2018 06:22:19 +0100 Subject: [PATCH 20/29] toString() --- .../elasticsearch/cluster/coordination/JoinHelper.java | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/cluster/coordination/JoinHelper.java b/server/src/main/java/org/elasticsearch/cluster/coordination/JoinHelper.java index 91b4476507e3a..84130f60396a6 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/JoinHelper.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/JoinHelper.java @@ -239,12 +239,18 @@ public void submitPendingJoins() { pendingAsTasks.put(task, new JoinTaskListener(task, value)); }); + final String stateUpdateSource = "elected-as-master ([" + pendingAsTasks.size() + "] nodes joined)"; + pendingAsTasks.put(JoinTaskExecutor.BECOME_MASTER_TASK, (source, e) -> { }); pendingAsTasks.put(JoinTaskExecutor.FINISH_ELECTION_TASK, (source, e) -> { }); - final String source = "elected-as-master ([" + pendingAsTasks.size() + "] nodes joined)"; - masterService.submitStateUpdateTasks(source, pendingAsTasks, ClusterStateTaskConfig.build(Priority.URGENT), joinTaskExecutor); + masterService.submitStateUpdateTasks(stateUpdateSource, pendingAsTasks, ClusterStateTaskConfig.build(Priority.URGENT), joinTaskExecutor); + } + + @Override + public String toString() { + return "CandidateJoinAccumulator{" + joinRequestAccumulator.keySet() + '}'; } } } From aa9ce2501c64fafbaa14eace4b5605a29fdde2b7 Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Thu, 23 Aug 2018 00:03:20 +0200 Subject: [PATCH 21/29] rename and clean-up --- .../java/org/elasticsearch/cluster/coordination/Coordinator.java | 1 - 1 file changed, 1 deletion(-) 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 1bb39eee6c79d..41e611dedc20d 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/Coordinator.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/Coordinator.java @@ -182,7 +182,6 @@ public void invariant() { if (mode == Mode.LEADER) { assert coordinationState.get().electionWon(); assert lastKnownLeader.isPresent() && lastKnownLeader.get().equals(getLocalNode()); - // assert joinHelper.getNumberOfPendingJoins() == 0; // not true any more, may not have submitted to the master service yet } else if (mode == Mode.FOLLOWER) { assert coordinationState.get().electionWon() == false : getLocalNode() + " is FOLLOWER so electionWon() should be false"; assert lastKnownLeader.isPresent() && (lastKnownLeader.get().equals(getLocalNode()) == false); From 64e3cd4cb0b9650805663300e70e8d6733386c1d Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Thu, 23 Aug 2018 00:44:58 +0200 Subject: [PATCH 22/29] remove handleJoinRequest method from Coordinator --- .../cluster/coordination/Coordinator.java | 7 --- .../cluster/coordination/JoinHelper.java | 12 +--- .../cluster/coordination/NodeJoinTests.java | 59 +++++++++++++++---- 3 files changed, 48 insertions(+), 30 deletions(-) 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 41e611dedc20d..d717705400760 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/Coordinator.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/Coordinator.java @@ -19,7 +19,6 @@ package org.elasticsearch.cluster.coordination; import org.apache.lucene.util.SetOnce; -import org.elasticsearch.cluster.coordination.JoinHelper.JoinCallback; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.routing.allocation.AllocationService; import org.elasticsearch.cluster.service.MasterService; @@ -185,18 +184,12 @@ public void invariant() { } else if (mode == Mode.FOLLOWER) { assert coordinationState.get().electionWon() == false : getLocalNode() + " is FOLLOWER so electionWon() should be false"; assert lastKnownLeader.isPresent() && (lastKnownLeader.get().equals(getLocalNode()) == false); - assert joinHelper.getNumberOfPendingJoins() == 0; } else { assert mode == Mode.CANDIDATE; } } } - // this is just used because the test doesn't simulate sending the join requests through the transport service - TODO remove it - public void handleJoinRequest(JoinRequest joinRequest, JoinCallback joinCallback) { - joinHelper.handleJoinRequest(joinRequest, joinCallback); - } - public enum Mode { CANDIDATE, LEADER, FOLLOWER } diff --git a/server/src/main/java/org/elasticsearch/cluster/coordination/JoinHelper.java b/server/src/main/java/org/elasticsearch/cluster/coordination/JoinHelper.java index 84130f60396a6..d2f81d4f7dfe6 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/JoinHelper.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/JoinHelper.java @@ -106,7 +106,7 @@ public String toString() { })); } - void handleJoinRequest(JoinRequest joinRequest, JoinCallback joinCallback) { + private void handleJoinRequest(JoinRequest joinRequest, JoinCallback joinCallback) { transportService.connectToNode(joinRequest.getSourceNode()); final Optional optionalJoin = joinRequest.getOptionalJoin(); @@ -136,12 +136,6 @@ void becomeFollower(DiscoveryNode leaderNode) { } } - public int getNumberOfPendingJoins() { - synchronized (mutex) { - return joinAccumulator.getNumberOfPendingJoins(); - } - } - public interface JoinCallback { void onSuccess(); @@ -182,10 +176,6 @@ default void failPendingJoins(String reason) { default void submitPendingJoins() { } - - default int getNumberOfPendingJoins() { - return 0; - } } private class LeaderJoinAccumulator implements JoinAccumulator { 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 946162bdaf068..9a1fdb1a16407 100644 --- a/server/src/test/java/org/elasticsearch/cluster/coordination/NodeJoinTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/coordination/NodeJoinTests.java @@ -38,10 +38,15 @@ import org.elasticsearch.test.junit.annotations.TestLogging; import org.elasticsearch.threadpool.TestThreadPool; import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.transport.TransportChannel; +import org.elasticsearch.transport.TransportRequestHandler; +import org.elasticsearch.transport.TransportResponse; +import org.elasticsearch.transport.TransportResponseOptions; import org.elasticsearch.transport.TransportService; import org.junit.After; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.mockito.ArgumentCaptor; import java.util.ArrayList; import java.util.Collections; @@ -58,7 +63,10 @@ import static java.util.Collections.emptyMap; import static org.hamcrest.Matchers.containsString; +import static org.mockito.Matchers.anyObject; +import static org.mockito.Matchers.eq; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; @TestLogging("org.elasticsearch.cluster.service:TRACE,org.elasticsearch.cluster.coordination:TRACE") @@ -69,6 +77,7 @@ public class NodeJoinTests extends ESTestCase { private MasterService masterService; private Coordinator coordinator; private DeterministicTaskQueue deterministicTaskQueue; + private TransportRequestHandler transportRequestHandler; @BeforeClass public static void beforeClass() { @@ -126,11 +135,17 @@ private void setupMasterServiceAndCoordinator(long term, ClusterState initialSta this.masterService = masterService; TransportService transportService = mock(TransportService.class); when(transportService.getLocalNode()).thenReturn(initialState.nodes().getLocalNode()); + @SuppressWarnings("unchecked") + ArgumentCaptor> joinRequestHandler = ArgumentCaptor.forClass( + (Class) TransportRequestHandler.class); coordinator = new Coordinator(Settings.EMPTY, transportService, ESAllocationTestCase.createAllocationService(Settings.EMPTY), masterService, () -> new CoordinationStateTests.InMemoryPersistedState(term, initialState)); + verify(transportService).registerRequestHandler(eq(JoinHelper.JOIN_ACTION_NAME), eq(ThreadPool.Names.GENERIC), eq(false), eq(false), + anyObject(), joinRequestHandler.capture()); + transportRequestHandler = joinRequestHandler.getValue(); coordinator.start(); coordinator.startInitialJoin(); } @@ -174,19 +189,39 @@ private SimpleFuture joinNodeAsync(final JoinRequest joinRequest) { logger.debug("starting {}", future); // clone the node before submitting to simulate an incoming join, which is guaranteed to have a new // disco node object serialized off the network - coordinator.handleJoinRequest(joinRequest, new JoinHelper.JoinCallback() { - @Override - public void onSuccess() { - logger.debug("{} completed", future); - future.markAsDone(); - } + try { + transportRequestHandler.messageReceived(joinRequest, new TransportChannel() { + @Override + public String getProfileName() { + return "dummy"; + } - @Override - public void onFailure(Exception e) { - logger.error(() -> new ParameterizedMessage("unexpected error for {}", future), e); - future.markAsFailed(e); - } - }); + @Override + public String getChannelType() { + return "dummy"; + } + + @Override + public void sendResponse(TransportResponse response) { + logger.debug("{} completed", future); + future.markAsDone(); + } + + @Override + public void sendResponse(TransportResponse response, TransportResponseOptions options) { + sendResponse(response); + } + + @Override + public void sendResponse(Exception e) { + logger.error(() -> new ParameterizedMessage("unexpected error for {}", future), e); + future.markAsFailed(e); + } + }, null); + } catch (Exception e) { + logger.error(() -> new ParameterizedMessage("unexpected error for {}", future), e); + future.markAsFailed(e); + } return future; } From d45cc115b4ffa41a9605dc29cc57b7bc40a97d63 Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Thu, 23 Aug 2018 08:30:21 +0200 Subject: [PATCH 23/29] checkstyle T_T --- .../org/elasticsearch/cluster/coordination/JoinHelper.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/server/src/main/java/org/elasticsearch/cluster/coordination/JoinHelper.java b/server/src/main/java/org/elasticsearch/cluster/coordination/JoinHelper.java index d2f81d4f7dfe6..5902bc789a76e 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/JoinHelper.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/JoinHelper.java @@ -235,7 +235,8 @@ public void submitPendingJoins() { }); pendingAsTasks.put(JoinTaskExecutor.FINISH_ELECTION_TASK, (source, e) -> { }); - masterService.submitStateUpdateTasks(stateUpdateSource, pendingAsTasks, ClusterStateTaskConfig.build(Priority.URGENT), joinTaskExecutor); + masterService.submitStateUpdateTasks(stateUpdateSource, pendingAsTasks, ClusterStateTaskConfig.build(Priority.URGENT), + joinTaskExecutor); } @Override From 49623cb3ee89075de30012e6be7766694e90b18f Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Thu, 23 Aug 2018 12:04:44 +0200 Subject: [PATCH 24/29] explicitly catch exception and handle it --- .../cluster/coordination/JoinHelper.java | 12 +++++++++++- 1 file changed, 11 insertions(+), 1 deletion(-) diff --git a/server/src/main/java/org/elasticsearch/cluster/coordination/JoinHelper.java b/server/src/main/java/org/elasticsearch/cluster/coordination/JoinHelper.java index 5902bc789a76e..cc7b8b0b59086 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/JoinHelper.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/JoinHelper.java @@ -110,7 +110,17 @@ private void handleJoinRequest(JoinRequest joinRequest, JoinCallback joinCallbac transportService.connectToNode(joinRequest.getSourceNode()); final Optional optionalJoin = joinRequest.getOptionalJoin(); - final boolean justBecameLeader = optionalJoin.isPresent() && joinHandler.test(optionalJoin.get()); + final boolean justBecameLeader; + if (optionalJoin.isPresent()) { + try { + justBecameLeader = joinHandler.test(optionalJoin.get()); + } catch (CoordinationStateRejectedException e) { + joinCallback.onFailure(e); + return; + } + } else { + justBecameLeader = false; + } synchronized (mutex) { joinAccumulator.handleJoinRequest(joinRequest.getSourceNode(), joinCallback); From 06ab9e1b5f2010274ee1fb82af03d33d02ee9832 Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Thu, 23 Aug 2018 12:10:30 +0200 Subject: [PATCH 25/29] strengthen test --- .../cluster/coordination/NodeJoinTests.java | 19 ++++++++++++++----- 1 file changed, 14 insertions(+), 5 deletions(-) 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 9a1fdb1a16407..626d330ec4b96 100644 --- a/server/src/test/java/org/elasticsearch/cluster/coordination/NodeJoinTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/coordination/NodeJoinTests.java @@ -298,20 +298,29 @@ public void testJoinElectedLeader() { public void testJoinAccumulation() { DiscoveryNode node0 = newNode(0, true); DiscoveryNode node1 = newNode(1, true); + DiscoveryNode node2 = newNode(2, true); long initialTerm = randomLongBetween(1, 10); long initialVersion = randomLongBetween(1, 10); setupFakeMasterServiceAndCoordinator(initialTerm, initialState(false, node0, initialTerm, initialVersion, - new VotingConfiguration(Collections.singleton(node1.getId())))); + new VotingConfiguration(Collections.singleton(node2.getId())))); assertFalse(isLocalNodeElectedMaster()); long newTerm = initialTerm + randomLongBetween(1, 10); - SimpleFuture fut = joinNodeAsync(new JoinRequest(node0, Optional.of(new Join(node0, node0, newTerm, initialTerm, initialVersion)))); + SimpleFuture futNode0 = joinNodeAsync(new JoinRequest(node0, Optional.of( + new Join(node0, node0, newTerm, initialTerm, initialVersion)))); deterministicTaskQueue.runAllTasks(random()); - assertFalse(fut.isDone()); + assertFalse(futNode0.isDone()); assertFalse(isLocalNodeElectedMaster()); - joinNodeAndRun(new JoinRequest(node1, Optional.of(new Join(node1, node0, newTerm, initialTerm, initialVersion)))); + SimpleFuture futNode1 = joinNodeAsync(new JoinRequest(node1, Optional.of( + new Join(node1, node0, newTerm, initialTerm, initialVersion)))); + deterministicTaskQueue.runAllTasks(random()); + assertFalse(futNode1.isDone()); + assertFalse(isLocalNodeElectedMaster()); + joinNodeAndRun(new JoinRequest(node2, Optional.of(new Join(node2, node0, newTerm, initialTerm, initialVersion)))); assertTrue(isLocalNodeElectedMaster()); assertTrue(clusterStateHasNode(node1)); - FutureUtils.get(fut); + assertTrue(clusterStateHasNode(node2)); + FutureUtils.get(futNode0); + FutureUtils.get(futNode1); } public void testJoinFollowerWithHigherTerm() { From 5b390a89fe62ca9762769265151dfd678010a25b Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Thu, 23 Aug 2018 12:13:33 +0200 Subject: [PATCH 26/29] add TODO comment to coordinator --- .../org/elasticsearch/cluster/coordination/Coordinator.java | 2 ++ 1 file changed, 2 insertions(+) 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 d717705400760..0bbfbc3a31db5 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/Coordinator.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/Coordinator.java @@ -34,6 +34,8 @@ public class Coordinator extends AbstractLifecycleComponent { private final TransportService transportService; private final JoinHelper joinHelper; private final Supplier persistedStateSupplier; + // TODO: the following two fields are package-private as some tests require access to them + // These tests can be rewritten to use public methods once Coordinator is more feature-complete final Object mutex = new Object(); final SetOnce coordinationState = new SetOnce<>(); // initialized on start-up (see doStart) From 1c522ce0d0962ead9edb17c6b9c92e9279619b62 Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Thu, 23 Aug 2018 13:45:36 +0200 Subject: [PATCH 27/29] Move handleJoinRequest to Coordinator --- .../cluster/coordination/Coordinator.java | 54 ++++++++++------- .../cluster/coordination/JoinHelper.java | 60 +++---------------- 2 files changed, 39 insertions(+), 75 deletions(-) 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 0bbfbc3a31db5..7cefa7ea63fd5 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/Coordinator.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/Coordinator.java @@ -42,16 +42,18 @@ public class Coordinator extends AbstractLifecycleComponent { private Mode mode; private Optional lastKnownLeader; private Optional lastJoin; + private JoinHelper.JoinAccumulator joinAccumulator; public Coordinator(Settings settings, TransportService transportService, AllocationService allocationService, MasterService masterService, Supplier persistedStateSupplier) { super(settings); this.transportService = transportService; - this.joinHelper = new JoinHelper(settings, allocationService, masterService, transportService, this::getCurrentTerm, - this::handleJoin); + this.joinHelper = new JoinHelper(settings, allocationService, masterService, transportService, + this::getCurrentTerm, this::handleJoinRequest); this.persistedStateSupplier = persistedStateSupplier; this.lastKnownLeader = Optional.empty(); this.lastJoin = Optional.empty(); + this.joinAccumulator = joinHelper.new CandidateJoinAccumulator(); } private Optional ensureTermAtLeast(DiscoveryNode sourceNode, long targetTerm) { @@ -73,37 +75,41 @@ private Join joinLeaderInTerm(StartJoinRequest startJoinRequest) { return join; } - private boolean handleJoin(final Join join) { + private void handleJoinRequest(JoinRequest joinRequest, JoinHelper.JoinCallback joinCallback) { assert Thread.holdsLock(mutex) == false; + logger.trace("handleJoin: as {}, handling {}", mode, joinRequest); + transportService.connectToNode(joinRequest.getSourceNode()); + final Optional optionalJoin = joinRequest.getOptionalJoin(); synchronized (mutex) { - logger.trace("handleJoin: as {}, handling {}", mode, join); - final CoordinationState coordState = coordinationState.get(); final boolean prevElectionWon = coordState.electionWon(); - // if someone thinks we should be master, let's add our vote and try to become one - // note that the following line should never throw an exception - ensureTermAtLeast(getLocalNode(), join.getTerm()).ifPresent(coordState::handleJoin); - - if (coordState.electionWon()) { - // if we have already won the election then the actual join does not matter for election purposes, so swallow any exception - try { - coordState.handleJoin(join); - } catch (CoordinationStateRejectedException e) { - logger.trace("failed to add join, ignoring", e); + if (optionalJoin.isPresent()) { + Join join = optionalJoin.get(); + // if someone thinks we should be master, let's add our vote and try to become one + // note that the following line should never throw an exception + ensureTermAtLeast(getLocalNode(), join.getTerm()).ifPresent(coordState::handleJoin); + + if (coordState.electionWon()) { + // if we have already won the election then the actual join does not matter for election purposes, + // so swallow any exception + try { + coordState.handleJoin(join); + } catch (CoordinationStateRejectedException e) { + logger.trace("failed to add join, ignoring", e); + } + } else { + coordState.handleJoin(join); // this might fail and bubble up the exception } - } else { - coordState.handleJoin(join); // this might fail and bubble up the exception } + joinAccumulator.handleJoinRequest(joinRequest.getSourceNode(), joinCallback); + if (prevElectionWon == false && coordState.electionWon()) { becomeLeader("handleJoin"); - return true; } } - - return false; } void becomeCandidate(String method) { @@ -112,7 +118,8 @@ void becomeCandidate(String method) { if (mode != Mode.CANDIDATE) { mode = Mode.CANDIDATE; - joinHelper.becomeCandidate(); + joinAccumulator.failPendingJoins("becoming candidate"); + joinAccumulator = joinHelper.new CandidateJoinAccumulator(); } } @@ -123,6 +130,8 @@ void becomeLeader(String method) { mode = Mode.LEADER; lastKnownLeader = Optional.of(getLocalNode()); + joinAccumulator.submitPendingJoins(); + joinAccumulator = joinHelper.new LeaderJoinAccumulator(); } void becomeFollower(String method, DiscoveryNode leaderNode) { @@ -131,7 +140,8 @@ void becomeFollower(String method, DiscoveryNode leaderNode) { if (mode != Mode.FOLLOWER) { mode = Mode.FOLLOWER; - joinHelper.becomeFollower(leaderNode); + joinAccumulator.failPendingJoins("started following " + leaderNode); + joinAccumulator = new JoinHelper.FollowerJoinAccumulator(); } lastKnownLeader = Optional.of(leaderNode); diff --git a/server/src/main/java/org/elasticsearch/cluster/coordination/JoinHelper.java b/server/src/main/java/org/elasticsearch/cluster/coordination/JoinHelper.java index cc7b8b0b59086..bd63ee8428eb7 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/JoinHelper.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/JoinHelper.java @@ -35,9 +35,8 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.Optional; +import java.util.function.BiConsumer; import java.util.function.LongSupplier; -import java.util.function.Predicate; public class JoinHelper extends AbstractComponent { @@ -45,18 +44,14 @@ public class JoinHelper extends AbstractComponent { private final MasterService masterService; private final TransportService transportService; - private final Predicate joinHandler; private final JoinTaskExecutor joinTaskExecutor; - private final Object mutex = new Object(); - private JoinAccumulator joinAccumulator = new CandidateJoinAccumulator(); public JoinHelper(Settings settings, AllocationService allocationService, MasterService masterService, TransportService transportService, LongSupplier currentTermSupplier, - Predicate joinHandler) { + BiConsumer joinHandler) { super(settings); this.masterService = masterService; this.transportService = transportService; - this.joinHandler = joinHandler; this.joinTaskExecutor = new JoinTaskExecutor(allocationService, logger) { @Override @@ -78,7 +73,7 @@ public ClusterTasksResult execute(ClusterState currentSta }; transportService.registerRequestHandler(JOIN_ACTION_NAME, ThreadPool.Names.GENERIC, false, false, JoinRequest::new, - (request, channel, task) -> handleJoinRequest(request, new JoinCallback() { + (request, channel, task) -> joinHandler.accept(request, new JoinCallback() { @Override public void onSuccess() { @@ -106,46 +101,6 @@ public String toString() { })); } - private void handleJoinRequest(JoinRequest joinRequest, JoinCallback joinCallback) { - transportService.connectToNode(joinRequest.getSourceNode()); - - final Optional optionalJoin = joinRequest.getOptionalJoin(); - final boolean justBecameLeader; - if (optionalJoin.isPresent()) { - try { - justBecameLeader = joinHandler.test(optionalJoin.get()); - } catch (CoordinationStateRejectedException e) { - joinCallback.onFailure(e); - return; - } - } else { - justBecameLeader = false; - } - - synchronized (mutex) { - joinAccumulator.handleJoinRequest(joinRequest.getSourceNode(), joinCallback); - - if (justBecameLeader) { - joinAccumulator.submitPendingJoins(); - joinAccumulator = new LeaderJoinAccumulator(); - } - } - } - - void becomeCandidate() { - synchronized (mutex) { - joinAccumulator.failPendingJoins("becoming candidate"); - joinAccumulator = new CandidateJoinAccumulator(); - } - } - - void becomeFollower(DiscoveryNode leaderNode) { - synchronized (mutex) { - joinAccumulator.failPendingJoins("started following " + leaderNode); - joinAccumulator = new FollowerJoinAccumulator(); - } - } - public interface JoinCallback { void onSuccess(); @@ -177,8 +132,7 @@ public String toString() { } } - - private interface JoinAccumulator { + interface JoinAccumulator { void handleJoinRequest(DiscoveryNode sender, JoinCallback joinCallback); default void failPendingJoins(String reason) { @@ -188,7 +142,7 @@ default void submitPendingJoins() { } } - private class LeaderJoinAccumulator implements JoinAccumulator { + class LeaderJoinAccumulator implements JoinAccumulator { @Override public void handleJoinRequest(DiscoveryNode sender, JoinCallback joinCallback) { final JoinTaskExecutor.Task task = new JoinTaskExecutor.Task(sender, "join existing leader"); @@ -202,7 +156,7 @@ public String toString() { } } - private static class FollowerJoinAccumulator implements JoinAccumulator { + static class FollowerJoinAccumulator implements JoinAccumulator { @Override public void handleJoinRequest(DiscoveryNode sender, JoinCallback joinCallback) { joinCallback.onFailure(new CoordinationStateRejectedException("join target is a follower")); @@ -214,7 +168,7 @@ public String toString() { } } - private class CandidateJoinAccumulator implements JoinAccumulator { + class CandidateJoinAccumulator implements JoinAccumulator { private final Map joinRequestAccumulator = new HashMap<>(); From 66825c7f261637194c71438de9beee7ca893dec8 Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Thu, 23 Aug 2018 13:58:53 +0200 Subject: [PATCH 28/29] only use close method --- .../cluster/coordination/Coordinator.java | 6 +-- .../cluster/coordination/JoinHelper.java | 47 ++++++++++--------- .../cluster/coordination/NodeJoinTests.java | 2 +- 3 files changed, 29 insertions(+), 26 deletions(-) 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 7cefa7ea63fd5..029d9d87d6ace 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/Coordinator.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/Coordinator.java @@ -118,7 +118,7 @@ void becomeCandidate(String method) { if (mode != Mode.CANDIDATE) { mode = Mode.CANDIDATE; - joinAccumulator.failPendingJoins("becoming candidate"); + joinAccumulator.close(mode); joinAccumulator = joinHelper.new CandidateJoinAccumulator(); } } @@ -130,7 +130,7 @@ void becomeLeader(String method) { mode = Mode.LEADER; lastKnownLeader = Optional.of(getLocalNode()); - joinAccumulator.submitPendingJoins(); + joinAccumulator.close(mode); joinAccumulator = joinHelper.new LeaderJoinAccumulator(); } @@ -140,7 +140,7 @@ void becomeFollower(String method, DiscoveryNode leaderNode) { if (mode != Mode.FOLLOWER) { mode = Mode.FOLLOWER; - joinAccumulator.failPendingJoins("started following " + leaderNode); + joinAccumulator.close(mode); joinAccumulator = new JoinHelper.FollowerJoinAccumulator(); } diff --git a/server/src/main/java/org/elasticsearch/cluster/coordination/JoinHelper.java b/server/src/main/java/org/elasticsearch/cluster/coordination/JoinHelper.java index bd63ee8428eb7..a235a3656990a 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/JoinHelper.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/JoinHelper.java @@ -21,6 +21,7 @@ import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterStateTaskConfig; import org.elasticsearch.cluster.ClusterStateTaskListener; +import org.elasticsearch.cluster.coordination.Coordinator.Mode; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.routing.allocation.AllocationService; import org.elasticsearch.cluster.service.MasterService; @@ -135,10 +136,8 @@ public String toString() { interface JoinAccumulator { void handleJoinRequest(DiscoveryNode sender, JoinCallback joinCallback); - default void failPendingJoins(String reason) { - } + default void close(Mode newMode) { - default void submitPendingJoins() { } } @@ -181,26 +180,30 @@ public void handleJoinRequest(DiscoveryNode sender, JoinCallback joinCallback) { } @Override - public void failPendingJoins(String reason) { - joinRequestAccumulator.values().forEach(joinCallback -> joinCallback.onFailure(new CoordinationStateRejectedException(reason))); - } + public void close(Mode newMode) { + if (newMode == Mode.LEADER) { + final Map pendingAsTasks = new HashMap<>(); + joinRequestAccumulator.forEach((key, value) -> { + final JoinTaskExecutor.Task task = new JoinTaskExecutor.Task(key, "elect leader"); + pendingAsTasks.put(task, new JoinTaskListener(task, value)); + }); + + final String stateUpdateSource = "elected-as-master ([" + pendingAsTasks.size() + "] nodes joined)"; + + pendingAsTasks.put(JoinTaskExecutor.BECOME_MASTER_TASK, (source, e) -> { + }); + pendingAsTasks.put(JoinTaskExecutor.FINISH_ELECTION_TASK, (source, e) -> { + }); + masterService.submitStateUpdateTasks(stateUpdateSource, pendingAsTasks, ClusterStateTaskConfig.build(Priority.URGENT), + joinTaskExecutor); + } else if (newMode == Mode.FOLLOWER) { + joinRequestAccumulator.values().forEach(joinCallback -> joinCallback.onFailure( + new CoordinationStateRejectedException("became follower"))); + } else { + assert newMode == Mode.CANDIDATE; + assert joinRequestAccumulator.isEmpty() : joinRequestAccumulator.keySet(); + } - @Override - public void submitPendingJoins() { - final Map pendingAsTasks = new HashMap<>(); - joinRequestAccumulator.forEach((key, value) -> { - final JoinTaskExecutor.Task task = new JoinTaskExecutor.Task(key, "elect leader"); - pendingAsTasks.put(task, new JoinTaskListener(task, value)); - }); - - final String stateUpdateSource = "elected-as-master ([" + pendingAsTasks.size() + "] nodes joined)"; - - pendingAsTasks.put(JoinTaskExecutor.BECOME_MASTER_TASK, (source, e) -> { - }); - pendingAsTasks.put(JoinTaskExecutor.FINISH_ELECTION_TASK, (source, e) -> { - }); - masterService.submitStateUpdateTasks(stateUpdateSource, pendingAsTasks, ClusterStateTaskConfig.build(Priority.URGENT), - joinTaskExecutor); } @Override 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 626d330ec4b96..67570bcb186d8 100644 --- a/server/src/test/java/org/elasticsearch/cluster/coordination/NodeJoinTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/coordination/NodeJoinTests.java @@ -379,7 +379,7 @@ public void testBecomeFollowerFailsPendingJoin() { assertFalse(isLocalNodeElectedMaster()); assertThat(expectThrows(CoordinationStateRejectedException.class, () -> FutureUtils.get(fut)).getMessage(), - containsString("started following")); + containsString("became follower")); assertFalse(isLocalNodeElectedMaster()); } From 6566efb1ec2aeb84e0e06e18ca7c41693ec79a92 Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Thu, 23 Aug 2018 16:27:53 +0200 Subject: [PATCH 29/29] add assertions --- .../org/elasticsearch/cluster/coordination/Coordinator.java | 3 +++ .../org/elasticsearch/cluster/coordination/JoinHelper.java | 5 ++++- 2 files changed, 7 insertions(+), 1 deletion(-) 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 029d9d87d6ace..7528050e4df16 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/Coordinator.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/Coordinator.java @@ -193,11 +193,14 @@ public void invariant() { if (mode == Mode.LEADER) { assert coordinationState.get().electionWon(); assert lastKnownLeader.isPresent() && lastKnownLeader.get().equals(getLocalNode()); + assert joinAccumulator instanceof JoinHelper.LeaderJoinAccumulator; } else if (mode == Mode.FOLLOWER) { assert coordinationState.get().electionWon() == false : getLocalNode() + " is FOLLOWER so electionWon() should be false"; assert lastKnownLeader.isPresent() && (lastKnownLeader.get().equals(getLocalNode()) == false); + assert joinAccumulator instanceof JoinHelper.FollowerJoinAccumulator; } else { assert mode == Mode.CANDIDATE; + assert joinAccumulator instanceof JoinHelper.CandidateJoinAccumulator; } } } diff --git a/server/src/main/java/org/elasticsearch/cluster/coordination/JoinHelper.java b/server/src/main/java/org/elasticsearch/cluster/coordination/JoinHelper.java index a235a3656990a..766bdce26daf5 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/JoinHelper.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/JoinHelper.java @@ -170,9 +170,11 @@ public String toString() { class CandidateJoinAccumulator implements JoinAccumulator { private final Map joinRequestAccumulator = new HashMap<>(); + boolean closed; @Override public void handleJoinRequest(DiscoveryNode sender, JoinCallback joinCallback) { + assert closed == false : "CandidateJoinAccumulator closed"; JoinCallback prev = joinRequestAccumulator.put(sender, joinCallback); if (prev != null) { prev.onFailure(new CoordinationStateRejectedException("received a newer join from " + sender)); @@ -181,6 +183,8 @@ public void handleJoinRequest(DiscoveryNode sender, JoinCallback joinCallback) { @Override public void close(Mode newMode) { + assert closed == false : "CandidateJoinAccumulator closed"; + closed = true; if (newMode == Mode.LEADER) { final Map pendingAsTasks = new HashMap<>(); joinRequestAccumulator.forEach((key, value) -> { @@ -203,7 +207,6 @@ public void close(Mode newMode) { assert newMode == Mode.CANDIDATE; assert joinRequestAccumulator.isEmpty() : joinRequestAccumulator.keySet(); } - } @Override