Integrate LeaderChecker with Coordinator (#34049)
This change ensures that follower nodes periodically check that their leader is healthy, and that they elect a new leader if not.
This commit is contained in:
parent
f886eebd99
commit
d995fc85c6
|
@ -78,10 +78,13 @@ public class Coordinator extends AbstractLifecycleComponent implements Discovery
|
||||||
private final UnicastConfiguredHostsResolver configuredHostsResolver;
|
private final UnicastConfiguredHostsResolver configuredHostsResolver;
|
||||||
private final TimeValue publishTimeout;
|
private final TimeValue publishTimeout;
|
||||||
private final PublicationTransportHandler publicationHandler;
|
private final PublicationTransportHandler publicationHandler;
|
||||||
|
private final LeaderChecker leaderChecker;
|
||||||
@Nullable
|
@Nullable
|
||||||
private Releasable electionScheduler;
|
private Releasable electionScheduler;
|
||||||
@Nullable
|
@Nullable
|
||||||
private Releasable prevotingRound;
|
private Releasable prevotingRound;
|
||||||
|
@Nullable
|
||||||
|
private Releasable leaderCheckScheduler;
|
||||||
private AtomicLong maxTermSeen = new AtomicLong();
|
private AtomicLong maxTermSeen = new AtomicLong();
|
||||||
|
|
||||||
private Mode mode;
|
private Mode mode;
|
||||||
|
@ -108,10 +111,27 @@ public class Coordinator extends AbstractLifecycleComponent implements Discovery
|
||||||
this.peerFinder = new CoordinatorPeerFinder(settings, transportService,
|
this.peerFinder = new CoordinatorPeerFinder(settings, transportService,
|
||||||
new HandshakingTransportAddressConnector(settings, transportService), configuredHostsResolver);
|
new HandshakingTransportAddressConnector(settings, transportService), configuredHostsResolver);
|
||||||
this.publicationHandler = new PublicationTransportHandler(transportService, this::handlePublishRequest, this::handleApplyCommit);
|
this.publicationHandler = new PublicationTransportHandler(transportService, this::handlePublishRequest, this::handleApplyCommit);
|
||||||
|
this.leaderChecker = new LeaderChecker(settings, transportService, getOnLeaderFailure());
|
||||||
|
|
||||||
masterService.setClusterStateSupplier(this::getStateForMasterService);
|
masterService.setClusterStateSupplier(this::getStateForMasterService);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private Runnable getOnLeaderFailure() {
|
||||||
|
return new Runnable() {
|
||||||
|
@Override
|
||||||
|
public void run() {
|
||||||
|
synchronized (mutex) {
|
||||||
|
becomeCandidate("onLeaderFailure");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String toString() {
|
||||||
|
return "notification of leader failure";
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
|
|
||||||
private void handleApplyCommit(ApplyCommitRequest applyCommitRequest) {
|
private void handleApplyCommit(ApplyCommitRequest applyCommitRequest) {
|
||||||
synchronized (mutex) {
|
synchronized (mutex) {
|
||||||
logger.trace("handleApplyCommit: applying commit {}", applyCommitRequest);
|
logger.trace("handleApplyCommit: applying commit {}", applyCommitRequest);
|
||||||
|
@ -233,6 +253,12 @@ public class Coordinator extends AbstractLifecycleComponent implements Discovery
|
||||||
joinAccumulator = joinHelper.new CandidateJoinAccumulator();
|
joinAccumulator = joinHelper.new CandidateJoinAccumulator();
|
||||||
|
|
||||||
peerFinder.activate(coordinationState.get().getLastAcceptedState().nodes());
|
peerFinder.activate(coordinationState.get().getLastAcceptedState().nodes());
|
||||||
|
leaderChecker.setCurrentNodes(DiscoveryNodes.EMPTY_NODES);
|
||||||
|
|
||||||
|
if (leaderCheckScheduler != null) {
|
||||||
|
leaderCheckScheduler.close();
|
||||||
|
leaderCheckScheduler = null;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
preVoteCollector.update(getPreVoteResponse(), null);
|
preVoteCollector.update(getPreVoteResponse(), null);
|
||||||
|
@ -251,16 +277,21 @@ public class Coordinator extends AbstractLifecycleComponent implements Discovery
|
||||||
peerFinder.deactivate(getLocalNode());
|
peerFinder.deactivate(getLocalNode());
|
||||||
closePrevotingAndElectionScheduler();
|
closePrevotingAndElectionScheduler();
|
||||||
preVoteCollector.update(getPreVoteResponse(), getLocalNode());
|
preVoteCollector.update(getPreVoteResponse(), getLocalNode());
|
||||||
|
|
||||||
|
assert leaderCheckScheduler == null : leaderCheckScheduler;
|
||||||
}
|
}
|
||||||
|
|
||||||
void becomeFollower(String method, DiscoveryNode leaderNode) {
|
void becomeFollower(String method, DiscoveryNode leaderNode) {
|
||||||
assert Thread.holdsLock(mutex) : "Coordinator mutex not held";
|
assert Thread.holdsLock(mutex) : "Coordinator mutex not held";
|
||||||
logger.debug("{}: becoming FOLLOWER of [{}] (was {}, lastKnownLeader was [{}])", method, leaderNode, mode, lastKnownLeader);
|
logger.debug("{}: becoming FOLLOWER of [{}] (was {}, lastKnownLeader was [{}])", method, leaderNode, mode, lastKnownLeader);
|
||||||
|
|
||||||
|
final boolean restartLeaderChecker = (mode == Mode.FOLLOWER && Optional.of(leaderNode).equals(lastKnownLeader)) == false;
|
||||||
|
|
||||||
if (mode != Mode.FOLLOWER) {
|
if (mode != Mode.FOLLOWER) {
|
||||||
mode = Mode.FOLLOWER;
|
mode = Mode.FOLLOWER;
|
||||||
joinAccumulator.close(mode);
|
joinAccumulator.close(mode);
|
||||||
joinAccumulator = new JoinHelper.FollowerJoinAccumulator();
|
joinAccumulator = new JoinHelper.FollowerJoinAccumulator();
|
||||||
|
leaderChecker.setCurrentNodes(DiscoveryNodes.EMPTY_NODES);
|
||||||
}
|
}
|
||||||
|
|
||||||
lastKnownLeader = Optional.of(leaderNode);
|
lastKnownLeader = Optional.of(leaderNode);
|
||||||
|
@ -268,6 +299,13 @@ public class Coordinator extends AbstractLifecycleComponent implements Discovery
|
||||||
closePrevotingAndElectionScheduler();
|
closePrevotingAndElectionScheduler();
|
||||||
cancelActivePublication();
|
cancelActivePublication();
|
||||||
preVoteCollector.update(getPreVoteResponse(), leaderNode);
|
preVoteCollector.update(getPreVoteResponse(), leaderNode);
|
||||||
|
|
||||||
|
if (restartLeaderChecker) {
|
||||||
|
if (leaderCheckScheduler != null) {
|
||||||
|
leaderCheckScheduler.close();
|
||||||
|
}
|
||||||
|
leaderCheckScheduler = leaderChecker.startLeaderChecker(leaderNode);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private PreVoteResponse getPreVoteResponse() {
|
private PreVoteResponse getPreVoteResponse() {
|
||||||
|
@ -339,6 +377,7 @@ public class Coordinator extends AbstractLifecycleComponent implements Discovery
|
||||||
assert getStateForMasterService().nodes().getMasterNodeId() != null
|
assert getStateForMasterService().nodes().getMasterNodeId() != null
|
||||||
|| getStateForMasterService().term() != getCurrentTerm() :
|
|| getStateForMasterService().term() != getCurrentTerm() :
|
||||||
getStateForMasterService();
|
getStateForMasterService();
|
||||||
|
assert leaderCheckScheduler == null : leaderCheckScheduler;
|
||||||
} else if (mode == Mode.FOLLOWER) {
|
} else if (mode == Mode.FOLLOWER) {
|
||||||
assert coordinationState.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);
|
assert lastKnownLeader.isPresent() && (lastKnownLeader.get().equals(getLocalNode()) == false);
|
||||||
|
@ -347,12 +386,16 @@ public class Coordinator extends AbstractLifecycleComponent implements Discovery
|
||||||
assert electionScheduler == null : electionScheduler;
|
assert electionScheduler == null : electionScheduler;
|
||||||
assert prevotingRound == null : prevotingRound;
|
assert prevotingRound == null : prevotingRound;
|
||||||
assert getStateForMasterService().nodes().getMasterNodeId() == null : getStateForMasterService();
|
assert getStateForMasterService().nodes().getMasterNodeId() == null : getStateForMasterService();
|
||||||
|
assert leaderChecker.currentNodeIsMaster() == false;
|
||||||
|
assert leaderCheckScheduler != null;
|
||||||
} else {
|
} else {
|
||||||
assert mode == Mode.CANDIDATE;
|
assert mode == Mode.CANDIDATE;
|
||||||
assert joinAccumulator instanceof JoinHelper.CandidateJoinAccumulator;
|
assert joinAccumulator instanceof JoinHelper.CandidateJoinAccumulator;
|
||||||
assert peerFinderLeader.isPresent() == false : peerFinderLeader;
|
assert peerFinderLeader.isPresent() == false : peerFinderLeader;
|
||||||
assert prevotingRound == null || electionScheduler != null;
|
assert prevotingRound == null || electionScheduler != null;
|
||||||
assert getStateForMasterService().nodes().getMasterNodeId() == null : getStateForMasterService();
|
assert getStateForMasterService().nodes().getMasterNodeId() == null : getStateForMasterService();
|
||||||
|
assert leaderChecker.currentNodeIsMaster() == false;
|
||||||
|
assert leaderCheckScheduler == null : leaderCheckScheduler;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -577,6 +620,8 @@ public class Coordinator extends AbstractLifecycleComponent implements Discovery
|
||||||
return "scheduled timeout for " + publication;
|
return "scheduled timeout for " + publication;
|
||||||
}
|
}
|
||||||
});
|
});
|
||||||
|
|
||||||
|
leaderChecker.setCurrentNodes(publishRequest.getAcceptedState().nodes());
|
||||||
publication.start(Collections.emptySet()); // TODO start failure detector and put faultyNodes here
|
publication.start(Collections.emptySet()); // TODO start failure detector and put faultyNodes here
|
||||||
}
|
}
|
||||||
} catch (Exception e) {
|
} catch (Exception e) {
|
||||||
|
|
|
@ -77,7 +77,7 @@ public class LeaderChecker extends AbstractComponent {
|
||||||
private final TransportService transportService;
|
private final TransportService transportService;
|
||||||
private final Runnable onLeaderFailure;
|
private final Runnable onLeaderFailure;
|
||||||
|
|
||||||
private volatile DiscoveryNodes lastPublishedDiscoveryNodes;
|
private volatile DiscoveryNodes discoveryNodes;
|
||||||
|
|
||||||
public LeaderChecker(final Settings settings, final TransportService transportService, final Runnable onLeaderFailure) {
|
public LeaderChecker(final Settings settings, final TransportService transportService, final Runnable onLeaderFailure) {
|
||||||
super(settings);
|
super(settings);
|
||||||
|
@ -111,19 +111,24 @@ public class LeaderChecker extends AbstractComponent {
|
||||||
* isLocalNodeElectedMaster() should reflect whether this node is a leader, and nodeExists()
|
* isLocalNodeElectedMaster() should reflect whether this node is a leader, and nodeExists()
|
||||||
* should indicate whether nodes are known publication targets or not.
|
* should indicate whether nodes are known publication targets or not.
|
||||||
*/
|
*/
|
||||||
public void setLastPublishedDiscoveryNodes(DiscoveryNodes discoveryNodes) {
|
public void setCurrentNodes(DiscoveryNodes discoveryNodes) {
|
||||||
logger.trace("updating last-published nodes: {}", discoveryNodes);
|
logger.trace("setCurrentNodes: {}", discoveryNodes);
|
||||||
lastPublishedDiscoveryNodes = discoveryNodes;
|
this.discoveryNodes = discoveryNodes;
|
||||||
|
}
|
||||||
|
|
||||||
|
// For assertions
|
||||||
|
boolean currentNodeIsMaster() {
|
||||||
|
return discoveryNodes.isLocalNodeElectedMaster();
|
||||||
}
|
}
|
||||||
|
|
||||||
private void handleLeaderCheck(LeaderCheckRequest request, TransportChannel transportChannel, Task task) throws IOException {
|
private void handleLeaderCheck(LeaderCheckRequest request, TransportChannel transportChannel, Task task) throws IOException {
|
||||||
final DiscoveryNodes lastPublishedDiscoveryNodes = this.lastPublishedDiscoveryNodes;
|
final DiscoveryNodes discoveryNodes = this.discoveryNodes;
|
||||||
assert lastPublishedDiscoveryNodes != null;
|
assert discoveryNodes != null;
|
||||||
|
|
||||||
if (lastPublishedDiscoveryNodes.isLocalNodeElectedMaster() == false) {
|
if (discoveryNodes.isLocalNodeElectedMaster() == false) {
|
||||||
logger.debug("non-master handling {}", request);
|
logger.debug("non-master handling {}", request);
|
||||||
transportChannel.sendResponse(new CoordinationStateRejectedException("non-leader rejecting leader check"));
|
transportChannel.sendResponse(new CoordinationStateRejectedException("non-leader rejecting leader check"));
|
||||||
} else if (lastPublishedDiscoveryNodes.nodeExists(request.getSender()) == false) {
|
} else if (discoveryNodes.nodeExists(request.getSender()) == false) {
|
||||||
logger.debug("leader check from unknown node: {}", request);
|
logger.debug("leader check from unknown node: {}", request);
|
||||||
transportChannel.sendResponse(new CoordinationStateRejectedException("leader check from unknown node"));
|
transportChannel.sendResponse(new CoordinationStateRejectedException("leader check from unknown node"));
|
||||||
} else {
|
} else {
|
||||||
|
|
|
@ -1105,6 +1105,9 @@ public class TransportReplicationActionTests extends ESTestCase {
|
||||||
}
|
}
|
||||||
|
|
||||||
static class TestResponse extends ReplicationResponse {
|
static class TestResponse extends ReplicationResponse {
|
||||||
|
TestResponse() {
|
||||||
|
setShardInfo(new ShardInfo());
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private class TestAction extends TransportReplicationAction<Request, Request, TestResponse> {
|
private class TestAction extends TransportReplicationAction<Request, Request, TestResponse> {
|
||||||
|
|
|
@ -41,6 +41,7 @@ import org.elasticsearch.discovery.zen.UnicastHostsProvider.HostsResolver;
|
||||||
import org.elasticsearch.indices.cluster.FakeThreadPoolMasterService;
|
import org.elasticsearch.indices.cluster.FakeThreadPoolMasterService;
|
||||||
import org.elasticsearch.test.ESTestCase;
|
import org.elasticsearch.test.ESTestCase;
|
||||||
import org.elasticsearch.test.disruption.DisruptableMockTransport;
|
import org.elasticsearch.test.disruption.DisruptableMockTransport;
|
||||||
|
import org.elasticsearch.test.disruption.DisruptableMockTransport.ConnectionStatus;
|
||||||
import org.elasticsearch.test.junit.annotations.TestLogging;
|
import org.elasticsearch.test.junit.annotations.TestLogging;
|
||||||
import org.elasticsearch.transport.TransportService;
|
import org.elasticsearch.transport.TransportService;
|
||||||
import org.hamcrest.Matcher;
|
import org.hamcrest.Matcher;
|
||||||
|
@ -59,7 +60,11 @@ import static java.util.Collections.emptySet;
|
||||||
import static org.elasticsearch.cluster.coordination.CoordinationStateTests.clusterState;
|
import static org.elasticsearch.cluster.coordination.CoordinationStateTests.clusterState;
|
||||||
import static org.elasticsearch.cluster.coordination.CoordinationStateTests.setValue;
|
import static org.elasticsearch.cluster.coordination.CoordinationStateTests.setValue;
|
||||||
import static org.elasticsearch.cluster.coordination.CoordinationStateTests.value;
|
import static org.elasticsearch.cluster.coordination.CoordinationStateTests.value;
|
||||||
|
import static org.elasticsearch.cluster.coordination.Coordinator.Mode.CANDIDATE;
|
||||||
import static org.elasticsearch.cluster.coordination.Coordinator.Mode.FOLLOWER;
|
import static org.elasticsearch.cluster.coordination.Coordinator.Mode.FOLLOWER;
|
||||||
|
import static org.elasticsearch.cluster.coordination.LeaderChecker.LEADER_CHECK_INTERVAL_SETTING;
|
||||||
|
import static org.elasticsearch.cluster.coordination.LeaderChecker.LEADER_CHECK_RETRY_COUNT_SETTING;
|
||||||
|
import static org.elasticsearch.cluster.coordination.LeaderChecker.LEADER_CHECK_TIMEOUT_SETTING;
|
||||||
import static org.elasticsearch.node.Node.NODE_NAME_SETTING;
|
import static org.elasticsearch.node.Node.NODE_NAME_SETTING;
|
||||||
import static org.elasticsearch.transport.TransportService.HANDSHAKE_ACTION_NAME;
|
import static org.elasticsearch.transport.TransportService.HANDSHAKE_ACTION_NAME;
|
||||||
import static org.elasticsearch.transport.TransportService.NOOP_TRANSPORT_INTERCEPTOR;
|
import static org.elasticsearch.transport.TransportService.NOOP_TRANSPORT_INTERCEPTOR;
|
||||||
|
@ -68,7 +73,7 @@ import static org.hamcrest.Matchers.equalTo;
|
||||||
import static org.hamcrest.Matchers.is;
|
import static org.hamcrest.Matchers.is;
|
||||||
import static org.hamcrest.Matchers.not;
|
import static org.hamcrest.Matchers.not;
|
||||||
|
|
||||||
@TestLogging("org.elasticsearch.cluster.coordination:TRACE,org.elasticsearch.cluster.discovery:TRACE")
|
@TestLogging("org.elasticsearch.cluster.coordination:TRACE,org.elasticsearch.discovery:TRACE")
|
||||||
public class CoordinatorTests extends ESTestCase {
|
public class CoordinatorTests extends ESTestCase {
|
||||||
|
|
||||||
public void testCanUpdateClusterStateAfterStabilisation() {
|
public void testCanUpdateClusterStateAfterStabilisation() {
|
||||||
|
@ -101,6 +106,40 @@ public class CoordinatorTests extends ESTestCase {
|
||||||
assertEquals(currentTerm, newTerm);
|
assertEquals(currentTerm, newTerm);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public void testLeaderDisconnectionDetectedQuickly() {
|
||||||
|
final Cluster cluster = new Cluster(randomIntBetween(3, 5));
|
||||||
|
cluster.stabilise();
|
||||||
|
|
||||||
|
final ClusterNode originalLeader = cluster.getAnyLeader();
|
||||||
|
logger.info("--> disconnecting leader {}", originalLeader);
|
||||||
|
originalLeader.disconnect();
|
||||||
|
|
||||||
|
synchronized (originalLeader.coordinator.mutex) {
|
||||||
|
originalLeader.coordinator.becomeCandidate("simulated failure detection"); // TODO remove once follower checker is integrated
|
||||||
|
}
|
||||||
|
|
||||||
|
cluster.stabilise();
|
||||||
|
assertThat(cluster.getAnyLeader().getId(), not(equalTo(originalLeader.getId())));
|
||||||
|
}
|
||||||
|
|
||||||
|
public void testUnresponsiveLeaderDetectedEventually() {
|
||||||
|
final Cluster cluster = new Cluster(randomIntBetween(3, 5));
|
||||||
|
cluster.stabilise();
|
||||||
|
|
||||||
|
final ClusterNode originalLeader = cluster.getAnyLeader();
|
||||||
|
logger.info("--> partitioning leader {}", originalLeader);
|
||||||
|
originalLeader.partition();
|
||||||
|
|
||||||
|
synchronized (originalLeader.coordinator.mutex) {
|
||||||
|
originalLeader.coordinator.becomeCandidate("simulated failure detection"); // TODO remove once follower checker is integrated
|
||||||
|
}
|
||||||
|
|
||||||
|
cluster.stabilise(Cluster.DEFAULT_STABILISATION_TIME
|
||||||
|
+ (LEADER_CHECK_INTERVAL_SETTING.get(Settings.EMPTY).millis() + LEADER_CHECK_TIMEOUT_SETTING.get(Settings.EMPTY).millis())
|
||||||
|
* LEADER_CHECK_RETRY_COUNT_SETTING.get(Settings.EMPTY));
|
||||||
|
assertThat(cluster.getAnyLeader().getId(), not(equalTo(originalLeader.getId())));
|
||||||
|
}
|
||||||
|
|
||||||
private static String nodeIdFromIndex(int nodeIndex) {
|
private static String nodeIdFromIndex(int nodeIndex) {
|
||||||
return "node" + nodeIndex;
|
return "node" + nodeIndex;
|
||||||
}
|
}
|
||||||
|
@ -115,6 +154,9 @@ public class CoordinatorTests extends ESTestCase {
|
||||||
Settings.builder().put(NODE_NAME_SETTING.getKey(), "deterministic-task-queue").build());
|
Settings.builder().put(NODE_NAME_SETTING.getKey(), "deterministic-task-queue").build());
|
||||||
private final VotingConfiguration initialConfiguration;
|
private final VotingConfiguration initialConfiguration;
|
||||||
|
|
||||||
|
private final Set<String> disconnectedNodes = new HashSet<>();
|
||||||
|
private final Set<String> blackholedNodes = new HashSet<>();
|
||||||
|
|
||||||
Cluster(int initialNodeCount) {
|
Cluster(int initialNodeCount) {
|
||||||
logger.info("--> creating cluster of {} nodes", initialNodeCount);
|
logger.info("--> creating cluster of {} nodes", initialNodeCount);
|
||||||
|
|
||||||
|
@ -142,8 +184,12 @@ public class CoordinatorTests extends ESTestCase {
|
||||||
}
|
}
|
||||||
|
|
||||||
void stabilise() {
|
void stabilise() {
|
||||||
|
stabilise(DEFAULT_STABILISATION_TIME);
|
||||||
|
}
|
||||||
|
|
||||||
|
void stabilise(long stabilisationTime) {
|
||||||
final long stabilisationStartTime = deterministicTaskQueue.getCurrentTimeMillis();
|
final long stabilisationStartTime = deterministicTaskQueue.getCurrentTimeMillis();
|
||||||
while (deterministicTaskQueue.getCurrentTimeMillis() < stabilisationStartTime + DEFAULT_STABILISATION_TIME) {
|
while (deterministicTaskQueue.getCurrentTimeMillis() < stabilisationStartTime + stabilisationTime) {
|
||||||
|
|
||||||
while (deterministicTaskQueue.hasRunnableTasks()) {
|
while (deterministicTaskQueue.hasRunnableTasks()) {
|
||||||
try {
|
try {
|
||||||
|
@ -182,6 +228,10 @@ public class CoordinatorTests extends ESTestCase {
|
||||||
}
|
}
|
||||||
|
|
||||||
final String nodeId = clusterNode.getId();
|
final String nodeId = clusterNode.getId();
|
||||||
|
|
||||||
|
if (disconnectedNodes.contains(nodeId) || blackholedNodes.contains(nodeId)) {
|
||||||
|
assertThat(nodeId + " is a candidate", clusterNode.coordinator.getMode(), is(CANDIDATE));
|
||||||
|
} else {
|
||||||
assertThat(nodeId + " has the same term as the leader", clusterNode.coordinator.getCurrentTerm(), is(leaderTerm));
|
assertThat(nodeId + " has the same term as the leader", clusterNode.coordinator.getCurrentTerm(), is(leaderTerm));
|
||||||
// TODO assert that all nodes have actually voted for the leader in this term
|
// TODO assert that all nodes have actually voted for the leader in this term
|
||||||
|
|
||||||
|
@ -193,6 +243,7 @@ public class CoordinatorTests extends ESTestCase {
|
||||||
assertThat(clusterNode.coordinator.getLastCommittedState().map(ClusterState::getNodes).map(dn -> dn.nodeExists(nodeId)),
|
assertThat(clusterNode.coordinator.getLastCommittedState().map(ClusterState::getNodes).map(dn -> dn.nodeExists(nodeId)),
|
||||||
equalTo(Optional.of(true)));
|
equalTo(Optional.of(true)));
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
|
||||||
assertThat(leader.coordinator.getLastCommittedState().map(ClusterState::getNodes).map(DiscoveryNodes::getSize),
|
assertThat(leader.coordinator.getLastCommittedState().map(ClusterState::getNodes).map(DiscoveryNodes::getSize),
|
||||||
equalTo(Optional.of(clusterNodes.size())));
|
equalTo(Optional.of(clusterNodes.size())));
|
||||||
|
@ -204,6 +255,18 @@ public class CoordinatorTests extends ESTestCase {
|
||||||
return randomFrom(allLeaders);
|
return randomFrom(allLeaders);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private ConnectionStatus getConnectionStatus(DiscoveryNode sender, DiscoveryNode destination) {
|
||||||
|
ConnectionStatus connectionStatus;
|
||||||
|
if (blackholedNodes.contains(sender.getId()) || blackholedNodes.contains(destination.getId())) {
|
||||||
|
connectionStatus = ConnectionStatus.BLACK_HOLE;
|
||||||
|
} else if (disconnectedNodes.contains(sender.getId()) || disconnectedNodes.contains(destination.getId())) {
|
||||||
|
connectionStatus = ConnectionStatus.DISCONNECTED;
|
||||||
|
} else {
|
||||||
|
connectionStatus = ConnectionStatus.CONNECTED;
|
||||||
|
}
|
||||||
|
return connectionStatus;
|
||||||
|
}
|
||||||
|
|
||||||
class ClusterNode extends AbstractComponent {
|
class ClusterNode extends AbstractComponent {
|
||||||
private final int nodeIndex;
|
private final int nodeIndex;
|
||||||
private Coordinator coordinator;
|
private Coordinator coordinator;
|
||||||
|
@ -241,7 +304,7 @@ public class CoordinatorTests extends ESTestCase {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected ConnectionStatus getConnectionStatus(DiscoveryNode sender, DiscoveryNode destination) {
|
protected ConnectionStatus getConnectionStatus(DiscoveryNode sender, DiscoveryNode destination) {
|
||||||
return ConnectionStatus.CONNECTED;
|
return Cluster.this.getConnectionStatus(sender, destination);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -264,6 +327,17 @@ public class CoordinatorTests extends ESTestCase {
|
||||||
deterministicTaskQueue.scheduleNow(onNode(destination, doDelivery));
|
deterministicTaskQueue.scheduleNow(onNode(destination, doDelivery));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void onBlackholedDuringSend(long requestId, String action, DiscoveryNode destination) {
|
||||||
|
if (action.equals(HANDSHAKE_ACTION_NAME)) {
|
||||||
|
logger.trace("ignoring blackhole and delivering {}", getRequestDescription(requestId, action, destination));
|
||||||
|
// handshakes always have a timeout, and are sent in a blocking fashion, so we must respond with an exception.
|
||||||
|
sendFromTo(destination, getLocalNode(), action, getDisconnectException(requestId, action, destination));
|
||||||
|
} else {
|
||||||
|
super.onBlackholedDuringSend(requestId, action, destination);
|
||||||
|
}
|
||||||
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
masterService = new FakeThreadPoolMasterService("test",
|
masterService = new FakeThreadPoolMasterService("test",
|
||||||
|
@ -290,7 +364,7 @@ public class CoordinatorTests extends ESTestCase {
|
||||||
return localNode.getId();
|
return localNode.getId();
|
||||||
}
|
}
|
||||||
|
|
||||||
public DiscoveryNode getLocalNode() {
|
DiscoveryNode getLocalNode() {
|
||||||
return localNode;
|
return localNode;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -316,6 +390,14 @@ public class CoordinatorTests extends ESTestCase {
|
||||||
public String toString() {
|
public String toString() {
|
||||||
return localNode.toString();
|
return localNode.toString();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
void disconnect() {
|
||||||
|
disconnectedNodes.add(localNode.getId());
|
||||||
|
}
|
||||||
|
|
||||||
|
void partition() {
|
||||||
|
blackholedNodes.add(localNode.getId());
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private List<TransportAddress> provideUnicastHosts(HostsResolver ignored) {
|
private List<TransportAddress> provideUnicastHosts(HostsResolver ignored) {
|
||||||
|
|
|
@ -294,7 +294,7 @@ public class LeaderCheckerTests extends ESTestCase {
|
||||||
= DiscoveryNodes.builder().add(localNode).localNodeId(localNode.getId()).masterNodeId(localNode.getId()).build();
|
= DiscoveryNodes.builder().add(localNode).localNodeId(localNode.getId()).masterNodeId(localNode.getId()).build();
|
||||||
|
|
||||||
{
|
{
|
||||||
leaderChecker.setLastPublishedDiscoveryNodes(discoveryNodes);
|
leaderChecker.setCurrentNodes(discoveryNodes);
|
||||||
|
|
||||||
final CapturingTransportResponseHandler handler = new CapturingTransportResponseHandler();
|
final CapturingTransportResponseHandler handler = new CapturingTransportResponseHandler();
|
||||||
transportService.sendRequest(localNode, LEADER_CHECK_ACTION_NAME, new LeaderCheckRequest(otherNode), handler);
|
transportService.sendRequest(localNode, LEADER_CHECK_ACTION_NAME, new LeaderCheckRequest(otherNode), handler);
|
||||||
|
@ -307,7 +307,7 @@ public class LeaderCheckerTests extends ESTestCase {
|
||||||
}
|
}
|
||||||
|
|
||||||
{
|
{
|
||||||
leaderChecker.setLastPublishedDiscoveryNodes(DiscoveryNodes.builder(discoveryNodes).add(otherNode).build());
|
leaderChecker.setCurrentNodes(DiscoveryNodes.builder(discoveryNodes).add(otherNode).build());
|
||||||
|
|
||||||
final CapturingTransportResponseHandler handler = new CapturingTransportResponseHandler();
|
final CapturingTransportResponseHandler handler = new CapturingTransportResponseHandler();
|
||||||
transportService.sendRequest(localNode, LEADER_CHECK_ACTION_NAME, new LeaderCheckRequest(otherNode), handler);
|
transportService.sendRequest(localNode, LEADER_CHECK_ACTION_NAME, new LeaderCheckRequest(otherNode), handler);
|
||||||
|
@ -318,7 +318,7 @@ public class LeaderCheckerTests extends ESTestCase {
|
||||||
}
|
}
|
||||||
|
|
||||||
{
|
{
|
||||||
leaderChecker.setLastPublishedDiscoveryNodes(DiscoveryNodes.builder(discoveryNodes).add(otherNode).masterNodeId(null).build());
|
leaderChecker.setCurrentNodes(DiscoveryNodes.builder(discoveryNodes).add(otherNode).masterNodeId(null).build());
|
||||||
|
|
||||||
final CapturingTransportResponseHandler handler = new CapturingTransportResponseHandler();
|
final CapturingTransportResponseHandler handler = new CapturingTransportResponseHandler();
|
||||||
transportService.sendRequest(localNode, LEADER_CHECK_ACTION_NAME, new LeaderCheckRequest(otherNode), handler);
|
transportService.sendRequest(localNode, LEADER_CHECK_ACTION_NAME, new LeaderCheckRequest(otherNode), handler);
|
||||||
|
|
|
@ -30,6 +30,7 @@ import org.elasticsearch.cluster.node.DiscoveryNodes;
|
||||||
import org.elasticsearch.cluster.service.MasterService;
|
import org.elasticsearch.cluster.service.MasterService;
|
||||||
import org.elasticsearch.cluster.service.MasterServiceTests;
|
import org.elasticsearch.cluster.service.MasterServiceTests;
|
||||||
import org.elasticsearch.common.Randomness;
|
import org.elasticsearch.common.Randomness;
|
||||||
|
import org.elasticsearch.common.settings.ClusterSettings;
|
||||||
import org.elasticsearch.common.settings.Settings;
|
import org.elasticsearch.common.settings.Settings;
|
||||||
import org.elasticsearch.common.util.concurrent.BaseFuture;
|
import org.elasticsearch.common.util.concurrent.BaseFuture;
|
||||||
import org.elasticsearch.common.util.concurrent.FutureUtils;
|
import org.elasticsearch.common.util.concurrent.FutureUtils;
|
||||||
|
@ -38,17 +39,18 @@ import org.elasticsearch.node.Node;
|
||||||
import org.elasticsearch.test.ClusterServiceUtils;
|
import org.elasticsearch.test.ClusterServiceUtils;
|
||||||
import org.elasticsearch.test.ESTestCase;
|
import org.elasticsearch.test.ESTestCase;
|
||||||
import org.elasticsearch.test.junit.annotations.TestLogging;
|
import org.elasticsearch.test.junit.annotations.TestLogging;
|
||||||
|
import org.elasticsearch.test.transport.CapturingTransport;
|
||||||
import org.elasticsearch.threadpool.TestThreadPool;
|
import org.elasticsearch.threadpool.TestThreadPool;
|
||||||
import org.elasticsearch.threadpool.ThreadPool;
|
import org.elasticsearch.threadpool.ThreadPool;
|
||||||
|
import org.elasticsearch.transport.RequestHandlerRegistry;
|
||||||
import org.elasticsearch.transport.TransportChannel;
|
import org.elasticsearch.transport.TransportChannel;
|
||||||
import org.elasticsearch.transport.TransportRequestHandler;
|
import org.elasticsearch.transport.TransportRequest;
|
||||||
import org.elasticsearch.transport.TransportResponse;
|
import org.elasticsearch.transport.TransportResponse;
|
||||||
import org.elasticsearch.transport.TransportResponseOptions;
|
import org.elasticsearch.transport.TransportResponseOptions;
|
||||||
import org.elasticsearch.transport.TransportService;
|
import org.elasticsearch.transport.TransportService;
|
||||||
import org.junit.After;
|
import org.junit.After;
|
||||||
import org.junit.AfterClass;
|
import org.junit.AfterClass;
|
||||||
import org.junit.BeforeClass;
|
import org.junit.BeforeClass;
|
||||||
import org.mockito.ArgumentCaptor;
|
|
||||||
|
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.Collections;
|
import java.util.Collections;
|
||||||
|
@ -66,12 +68,8 @@ import java.util.stream.IntStream;
|
||||||
|
|
||||||
import static java.util.Collections.emptyList;
|
import static java.util.Collections.emptyList;
|
||||||
import static java.util.Collections.emptyMap;
|
import static java.util.Collections.emptyMap;
|
||||||
|
import static org.elasticsearch.transport.TransportService.HANDSHAKE_ACTION_NAME;
|
||||||
import static org.hamcrest.Matchers.containsString;
|
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")
|
@TestLogging("org.elasticsearch.cluster.service:TRACE,org.elasticsearch.cluster.coordination:TRACE")
|
||||||
public class NodeJoinTests extends ESTestCase {
|
public class NodeJoinTests extends ESTestCase {
|
||||||
|
@ -81,7 +79,7 @@ public class NodeJoinTests extends ESTestCase {
|
||||||
private MasterService masterService;
|
private MasterService masterService;
|
||||||
private Coordinator coordinator;
|
private Coordinator coordinator;
|
||||||
private DeterministicTaskQueue deterministicTaskQueue;
|
private DeterministicTaskQueue deterministicTaskQueue;
|
||||||
private TransportRequestHandler<JoinRequest> transportRequestHandler;
|
private RequestHandlerRegistry<TransportRequest> transportRequestHandler;
|
||||||
|
|
||||||
@BeforeClass
|
@BeforeClass
|
||||||
public static void beforeClass() {
|
public static void beforeClass() {
|
||||||
|
@ -144,20 +142,29 @@ public class NodeJoinTests extends ESTestCase {
|
||||||
throw new IllegalStateException("method setupMasterServiceAndCoordinator can only be called once");
|
throw new IllegalStateException("method setupMasterServiceAndCoordinator can only be called once");
|
||||||
}
|
}
|
||||||
this.masterService = masterService;
|
this.masterService = masterService;
|
||||||
TransportService transportService = mock(TransportService.class);
|
CapturingTransport capturingTransport = new CapturingTransport() {
|
||||||
when(transportService.getLocalNode()).thenReturn(initialState.nodes().getLocalNode());
|
@Override
|
||||||
when(transportService.getThreadPool()).thenReturn(threadPool);
|
protected void onSendRequest(long requestId, String action, TransportRequest request, DiscoveryNode destination) {
|
||||||
@SuppressWarnings("unchecked")
|
if (action.equals(HANDSHAKE_ACTION_NAME)) {
|
||||||
ArgumentCaptor<TransportRequestHandler<JoinRequest>> joinRequestHandler = ArgumentCaptor.forClass(
|
handleResponse(requestId, new TransportService.HandshakeResponse(destination, initialState.getClusterName(),
|
||||||
(Class) TransportRequestHandler.class);
|
destination.getVersion()));
|
||||||
|
} else {
|
||||||
|
super.onSendRequest(requestId, action, request, destination);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
};
|
||||||
|
TransportService transportService = capturingTransport.createTransportService(Settings.EMPTY, threadPool,
|
||||||
|
TransportService.NOOP_TRANSPORT_INTERCEPTOR,
|
||||||
|
x -> initialState.nodes().getLocalNode(),
|
||||||
|
new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS), Collections.emptySet());
|
||||||
coordinator = new Coordinator(Settings.EMPTY,
|
coordinator = new Coordinator(Settings.EMPTY,
|
||||||
transportService,
|
transportService,
|
||||||
ESAllocationTestCase.createAllocationService(Settings.EMPTY),
|
ESAllocationTestCase.createAllocationService(Settings.EMPTY),
|
||||||
masterService,
|
masterService,
|
||||||
() -> new CoordinationStateTests.InMemoryPersistedState(term, initialState), r -> emptyList(), random);
|
() -> new CoordinationStateTests.InMemoryPersistedState(term, initialState), r -> emptyList(), random);
|
||||||
verify(transportService).registerRequestHandler(eq(JoinHelper.JOIN_ACTION_NAME), eq(ThreadPool.Names.GENERIC), eq(false), eq(false),
|
transportService.start();
|
||||||
anyObject(), joinRequestHandler.capture());
|
transportService.acceptIncomingRequests();
|
||||||
transportRequestHandler = joinRequestHandler.getValue();
|
transportRequestHandler = capturingTransport.getRequestHandler(JoinHelper.JOIN_ACTION_NAME);
|
||||||
coordinator.start();
|
coordinator.start();
|
||||||
coordinator.startInitialJoin();
|
coordinator.startInitialJoin();
|
||||||
}
|
}
|
||||||
|
@ -202,7 +209,7 @@ public class NodeJoinTests extends ESTestCase {
|
||||||
// clone the node before submitting to simulate an incoming join, which is guaranteed to have a new
|
// clone the node before submitting to simulate an incoming join, which is guaranteed to have a new
|
||||||
// disco node object serialized off the network
|
// disco node object serialized off the network
|
||||||
try {
|
try {
|
||||||
transportRequestHandler.messageReceived(joinRequest, new TransportChannel() {
|
transportRequestHandler.processMessageReceived(joinRequest, new TransportChannel() {
|
||||||
@Override
|
@Override
|
||||||
public String getProfileName() {
|
public String getProfileName() {
|
||||||
return "dummy";
|
return "dummy";
|
||||||
|
@ -229,7 +236,7 @@ public class NodeJoinTests extends ESTestCase {
|
||||||
logger.error(() -> new ParameterizedMessage("unexpected error for {}", future), e);
|
logger.error(() -> new ParameterizedMessage("unexpected error for {}", future), e);
|
||||||
future.markAsFailed(e);
|
future.markAsFailed(e);
|
||||||
}
|
}
|
||||||
}, null);
|
});
|
||||||
} catch (Exception e) {
|
} catch (Exception e) {
|
||||||
logger.error(() -> new ParameterizedMessage("unexpected error for {}", future), e);
|
logger.error(() -> new ParameterizedMessage("unexpected error for {}", future), e);
|
||||||
future.markAsFailed(e);
|
future.markAsFailed(e);
|
||||||
|
|
|
@ -51,7 +51,7 @@ public abstract class DisruptableMockTransport extends MockTransport {
|
||||||
|
|
||||||
protected abstract void handle(DiscoveryNode sender, DiscoveryNode destination, String action, Runnable doDelivery);
|
protected abstract void handle(DiscoveryNode sender, DiscoveryNode destination, String action, Runnable doDelivery);
|
||||||
|
|
||||||
private void sendFromTo(DiscoveryNode sender, DiscoveryNode destination, String action, Runnable doDelivery) {
|
protected final void sendFromTo(DiscoveryNode sender, DiscoveryNode destination, String action, Runnable doDelivery) {
|
||||||
handle(sender, destination, action, new Runnable() {
|
handle(sender, destination, action, new Runnable() {
|
||||||
@Override
|
@Override
|
||||||
public void run() {
|
public void run() {
|
||||||
|
@ -74,10 +74,33 @@ public abstract class DisruptableMockTransport extends MockTransport {
|
||||||
|
|
||||||
assert destination.equals(getLocalNode()) == false : "non-local message from " + getLocalNode() + " to itself";
|
assert destination.equals(getLocalNode()) == false : "non-local message from " + getLocalNode() + " to itself";
|
||||||
|
|
||||||
final String requestDescription = new ParameterizedMessage("[{}][{}] from {} to {}",
|
sendFromTo(getLocalNode(), destination, action, new Runnable() {
|
||||||
action, requestId, getLocalNode(), destination).getFormattedMessage();
|
@Override
|
||||||
|
public void run() {
|
||||||
|
switch (getConnectionStatus(getLocalNode(), destination)) {
|
||||||
|
case BLACK_HOLE:
|
||||||
|
onBlackholedDuringSend(requestId, action, destination);
|
||||||
|
break;
|
||||||
|
|
||||||
final Runnable returnConnectException = new Runnable() {
|
case DISCONNECTED:
|
||||||
|
onDisconnectedDuringSend(requestId, action, destination);
|
||||||
|
break;
|
||||||
|
|
||||||
|
case CONNECTED:
|
||||||
|
onConnectedDuringSend(requestId, action, request, destination);
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String toString() {
|
||||||
|
return getRequestDescription(requestId, action, destination);
|
||||||
|
}
|
||||||
|
});
|
||||||
|
}
|
||||||
|
|
||||||
|
protected Runnable getDisconnectException(long requestId, String action, DiscoveryNode destination) {
|
||||||
|
return new Runnable() {
|
||||||
@Override
|
@Override
|
||||||
public void run() {
|
public void run() {
|
||||||
handleError(requestId, new ConnectTransportException(destination, "disconnected"));
|
handleError(requestId, new ConnectTransportException(destination, "disconnected"));
|
||||||
|
@ -85,29 +108,33 @@ public abstract class DisruptableMockTransport extends MockTransport {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public String toString() {
|
public String toString() {
|
||||||
return "disconnection response to " + requestDescription;
|
return "disconnection response to " + getRequestDescription(requestId, action, destination);
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
}
|
||||||
|
|
||||||
sendFromTo(getLocalNode(), destination, action, new Runnable() {
|
protected String getRequestDescription(long requestId, String action, DiscoveryNode destination) {
|
||||||
@Override
|
return new ParameterizedMessage("[{}][{}] from {} to {}",
|
||||||
public void run() {
|
action, requestId, getLocalNode(), destination).getFormattedMessage();
|
||||||
switch (getConnectionStatus(getLocalNode(), destination)) {
|
}
|
||||||
case BLACK_HOLE:
|
|
||||||
logger.trace("dropping {}", requestDescription);
|
|
||||||
break;
|
|
||||||
|
|
||||||
case DISCONNECTED:
|
protected void onBlackholedDuringSend(long requestId, String action, DiscoveryNode destination) {
|
||||||
sendFromTo(destination, getLocalNode(), action, returnConnectException);
|
logger.trace("dropping {}", getRequestDescription(requestId, action, destination));
|
||||||
break;
|
}
|
||||||
|
|
||||||
case CONNECTED:
|
protected void onDisconnectedDuringSend(long requestId, String action, DiscoveryNode destination) {
|
||||||
|
sendFromTo(destination, getLocalNode(), action, getDisconnectException(requestId, action, destination));
|
||||||
|
}
|
||||||
|
|
||||||
|
protected void onConnectedDuringSend(long requestId, String action, TransportRequest request, DiscoveryNode destination) {
|
||||||
Optional<DisruptableMockTransport> destinationTransport = getDisruptedCapturingTransport(destination, action);
|
Optional<DisruptableMockTransport> destinationTransport = getDisruptedCapturingTransport(destination, action);
|
||||||
assert destinationTransport.isPresent();
|
assert destinationTransport.isPresent();
|
||||||
|
|
||||||
final RequestHandlerRegistry<TransportRequest> requestHandler =
|
final RequestHandlerRegistry<TransportRequest> requestHandler =
|
||||||
destinationTransport.get().getRequestHandler(action);
|
destinationTransport.get().getRequestHandler(action);
|
||||||
|
|
||||||
|
final String requestDescription = getRequestDescription(requestId, action, destination);
|
||||||
|
|
||||||
final TransportChannel transportChannel = new TransportChannel() {
|
final TransportChannel transportChannel = new TransportChannel() {
|
||||||
@Override
|
@Override
|
||||||
public String getProfileName() {
|
public String getProfileName() {
|
||||||
|
@ -183,14 +210,6 @@ public abstract class DisruptableMockTransport extends MockTransport {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public String toString() {
|
|
||||||
return requestDescription;
|
|
||||||
}
|
|
||||||
});
|
|
||||||
}
|
|
||||||
|
|
||||||
private NamedWriteableRegistry writeableRegistry() {
|
private NamedWriteableRegistry writeableRegistry() {
|
||||||
return new NamedWriteableRegistry(ClusterModule.getNamedWriteables());
|
return new NamedWriteableRegistry(ClusterModule.getNamedWriteables());
|
||||||
|
|
Loading…
Reference in New Issue