|
@@ -18,9 +18,13 @@
|
|
|
*/
|
|
|
package org.elasticsearch.discovery.zen;
|
|
|
|
|
|
-import org.elasticsearch.ElasticsearchTimeoutException;
|
|
|
+import org.apache.lucene.store.AlreadyClosedException;
|
|
|
+import org.elasticsearch.Version;
|
|
|
+import org.elasticsearch.cluster.ClusterChangedEvent;
|
|
|
import org.elasticsearch.cluster.ClusterState;
|
|
|
-import org.elasticsearch.cluster.ClusterStateUpdateTask;
|
|
|
+import org.elasticsearch.cluster.ClusterStateTaskConfig;
|
|
|
+import org.elasticsearch.cluster.ClusterStateTaskExecutor;
|
|
|
+import org.elasticsearch.cluster.ClusterStateTaskListener;
|
|
|
import org.elasticsearch.cluster.NotMasterException;
|
|
|
import org.elasticsearch.cluster.block.ClusterBlocks;
|
|
|
import org.elasticsearch.cluster.node.DiscoveryNode;
|
|
@@ -30,21 +34,22 @@ import org.elasticsearch.cluster.routing.allocation.RoutingAllocation;
|
|
|
import org.elasticsearch.cluster.service.ClusterService;
|
|
|
import org.elasticsearch.common.Priority;
|
|
|
import org.elasticsearch.common.component.AbstractComponent;
|
|
|
+import org.elasticsearch.common.logging.ESLogger;
|
|
|
import org.elasticsearch.common.settings.Settings;
|
|
|
+import org.elasticsearch.common.transport.DummyTransportAddress;
|
|
|
import org.elasticsearch.common.unit.TimeValue;
|
|
|
import org.elasticsearch.discovery.DiscoverySettings;
|
|
|
import org.elasticsearch.discovery.zen.elect.ElectMasterService;
|
|
|
import org.elasticsearch.discovery.zen.membership.MembershipAction;
|
|
|
|
|
|
import java.util.ArrayList;
|
|
|
+import java.util.Collections;
|
|
|
import java.util.HashMap;
|
|
|
-import java.util.Iterator;
|
|
|
import java.util.List;
|
|
|
import java.util.Map;
|
|
|
import java.util.concurrent.CountDownLatch;
|
|
|
import java.util.concurrent.TimeUnit;
|
|
|
import java.util.concurrent.atomic.AtomicBoolean;
|
|
|
-import java.util.concurrent.atomic.AtomicReference;
|
|
|
|
|
|
/**
|
|
|
* This class processes incoming join request (passed zia {@link ZenDiscovery}). Incoming nodes
|
|
@@ -52,18 +57,17 @@ import java.util.concurrent.atomic.AtomicReference;
|
|
|
*/
|
|
|
public class NodeJoinController extends AbstractComponent {
|
|
|
|
|
|
- final ClusterService clusterService;
|
|
|
- final RoutingService routingService;
|
|
|
- final ElectMasterService electMaster;
|
|
|
- final DiscoverySettings discoverySettings;
|
|
|
- final AtomicBoolean accumulateJoins = new AtomicBoolean(false);
|
|
|
+ private final ClusterService clusterService;
|
|
|
+ private final RoutingService routingService;
|
|
|
+ private final ElectMasterService electMaster;
|
|
|
+ private final DiscoverySettings discoverySettings;
|
|
|
+ private final JoinTaskExecutor joinTaskExecutor = new JoinTaskExecutor();
|
|
|
|
|
|
- // this is site while trying to become a master
|
|
|
- final AtomicReference<ElectionContext> electionContext = new AtomicReference<>();
|
|
|
+ // this is set while trying to become a master
|
|
|
+ // mutation should be done under lock
|
|
|
+ private ElectionContext electionContext = null;
|
|
|
|
|
|
|
|
|
- protected final Map<DiscoveryNode, List<MembershipAction.JoinCallback>> pendingJoinRequests = new HashMap<>();
|
|
|
-
|
|
|
public NodeJoinController(ClusterService clusterService, RoutingService routingService, ElectMasterService electMaster, DiscoverySettings discoverySettings, Settings settings) {
|
|
|
super(settings);
|
|
|
this.clusterService = clusterService;
|
|
@@ -75,7 +79,7 @@ public class NodeJoinController extends AbstractComponent {
|
|
|
/**
|
|
|
* waits for enough incoming joins from master eligible nodes to complete the master election
|
|
|
* <p>
|
|
|
- * You must start accumulating joins before calling this method. See {@link #startAccumulatingJoins()}
|
|
|
+ * You must start accumulating joins before calling this method. See {@link #startElectionContext()}
|
|
|
* <p>
|
|
|
* The method will return once the local node has been elected as master or some failure/timeout has happened.
|
|
|
* The exact outcome is communicated via the callback parameter, which is guaranteed to be called.
|
|
@@ -86,29 +90,32 @@ public class NodeJoinController extends AbstractComponent {
|
|
|
* object
|
|
|
**/
|
|
|
public void waitToBeElectedAsMaster(int requiredMasterJoins, TimeValue timeValue, final ElectionCallback callback) {
|
|
|
- assert accumulateJoins.get() : "waitToBeElectedAsMaster is called we are not accumulating joins";
|
|
|
-
|
|
|
final CountDownLatch done = new CountDownLatch(1);
|
|
|
- final ElectionContext newContext = new ElectionContext(callback, requiredMasterJoins) {
|
|
|
+ final ElectionCallback wrapperCallback = new ElectionCallback() {
|
|
|
@Override
|
|
|
- void onClose() {
|
|
|
- if (electionContext.compareAndSet(this, null)) {
|
|
|
- stopAccumulatingJoins("election closed");
|
|
|
- } else {
|
|
|
- assert false : "failed to remove current election context";
|
|
|
- }
|
|
|
+ public void onElectedAsMaster(ClusterState state) {
|
|
|
done.countDown();
|
|
|
+ callback.onElectedAsMaster(state);
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public void onFailure(Throwable t) {
|
|
|
+ done.countDown();
|
|
|
+ callback.onFailure(t);
|
|
|
}
|
|
|
};
|
|
|
|
|
|
- if (electionContext.compareAndSet(null, newContext) == false) {
|
|
|
- // should never happen, but be conservative
|
|
|
- failContext(newContext, new IllegalStateException("double waiting for election"));
|
|
|
- return;
|
|
|
- }
|
|
|
+ ElectionContext myElectionContext = null;
|
|
|
+
|
|
|
try {
|
|
|
// check what we have so far..
|
|
|
- checkPendingJoinsAndElectIfNeeded();
|
|
|
+ // capture the context we add the callback to make sure we fail our own
|
|
|
+ synchronized (this) {
|
|
|
+ assert electionContext != null : "waitToBeElectedAsMaster is called we are not accumulating joins";
|
|
|
+ myElectionContext = electionContext;
|
|
|
+ electionContext.onAttemptToBeElected(requiredMasterJoins, wrapperCallback);
|
|
|
+ checkPendingJoinsAndElectIfNeeded();
|
|
|
+ }
|
|
|
|
|
|
try {
|
|
|
if (done.await(timeValue.millis(), TimeUnit.MILLISECONDS)) {
|
|
@@ -119,69 +126,46 @@ public class NodeJoinController extends AbstractComponent {
|
|
|
|
|
|
}
|
|
|
if (logger.isTraceEnabled()) {
|
|
|
- final int pendingNodes;
|
|
|
- synchronized (pendingJoinRequests) {
|
|
|
- pendingNodes = pendingJoinRequests.size();
|
|
|
- }
|
|
|
- logger.trace("timed out waiting to be elected. waited [{}]. pending node joins [{}]", timeValue, pendingNodes);
|
|
|
+ final int pendingNodes = myElectionContext.getPendingMasterJoinsCount();
|
|
|
+ logger.trace("timed out waiting to be elected. waited [{}]. pending master node joins [{}]", timeValue, pendingNodes);
|
|
|
}
|
|
|
- // callback will clear the context, if it's active
|
|
|
- failContext(newContext, new ElasticsearchTimeoutException("timed out waiting to be elected"));
|
|
|
+ failContextIfNeeded(myElectionContext, "timed out waiting to be elected");
|
|
|
} catch (Throwable t) {
|
|
|
logger.error("unexpected failure while waiting for incoming joins", t);
|
|
|
- failContext(newContext, "unexpected failure while waiting for pending joins", t);
|
|
|
+ if (myElectionContext != null) {
|
|
|
+ failContextIfNeeded(myElectionContext, "unexpected failure while waiting for pending joins [" + t.getMessage() + "]");
|
|
|
+ }
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- private void failContext(final ElectionContext context, final Throwable throwable) {
|
|
|
- failContext(context, throwable.getMessage(), throwable);
|
|
|
- }
|
|
|
-
|
|
|
- /** utility method to fail the given election context under the cluster state thread */
|
|
|
- private void failContext(final ElectionContext context, final String reason, final Throwable throwable) {
|
|
|
- clusterService.submitStateUpdateTask("zen-disco-join(failure [" + reason + "])", new ClusterStateUpdateTask(Priority.IMMEDIATE) {
|
|
|
-
|
|
|
- @Override
|
|
|
- public boolean runOnlyOnMaster() {
|
|
|
- return false;
|
|
|
- }
|
|
|
-
|
|
|
- @Override
|
|
|
- public ClusterState execute(ClusterState currentState) throws Exception {
|
|
|
- context.onFailure(throwable);
|
|
|
- return currentState;
|
|
|
- }
|
|
|
-
|
|
|
- @Override
|
|
|
- public void onFailure(String source, Throwable updateFailure) {
|
|
|
- logger.warn("unexpected error while trying to fail election context due to [{}]. original exception [{}]", updateFailure, reason, throwable);
|
|
|
- context.onFailure(updateFailure);
|
|
|
- }
|
|
|
- });
|
|
|
-
|
|
|
+ /**
|
|
|
+ * utility method to fail the given election context under the cluster state thread
|
|
|
+ */
|
|
|
+ private synchronized void failContextIfNeeded(final ElectionContext context, final String reason) {
|
|
|
+ if (electionContext == context) {
|
|
|
+ stopElectionContext(reason);
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
* Accumulates any future incoming join request. Pending join requests will be processed in the final steps of becoming a
|
|
|
- * master or when {@link #stopAccumulatingJoins(String)} is called.
|
|
|
+ * master or when {@link #stopElectionContext(String)} is called.
|
|
|
*/
|
|
|
- public void startAccumulatingJoins() {
|
|
|
- logger.trace("starting to accumulate joins");
|
|
|
- boolean b = accumulateJoins.getAndSet(true);
|
|
|
- assert b == false : "double startAccumulatingJoins() calls";
|
|
|
- assert electionContext.get() == null : "startAccumulatingJoins() called, but there is an ongoing election context";
|
|
|
+ public synchronized void startElectionContext() {
|
|
|
+ logger.trace("starting an election context, will accumulate joins");
|
|
|
+ assert electionContext == null : "double startElectionContext() calls";
|
|
|
+ electionContext = new ElectionContext();
|
|
|
}
|
|
|
|
|
|
- /** Stopped accumulating joins. All pending joins will be processed. Future joins will be processed immediately */
|
|
|
- public void stopAccumulatingJoins(String reason) {
|
|
|
- logger.trace("stopping join accumulation ([{}])", reason);
|
|
|
- assert electionContext.get() == null : "stopAccumulatingJoins() called, but there is an ongoing election context";
|
|
|
- boolean b = accumulateJoins.getAndSet(false);
|
|
|
- assert b : "stopAccumulatingJoins() called but not accumulating";
|
|
|
- synchronized (pendingJoinRequests) {
|
|
|
- if (pendingJoinRequests.size() > 0) {
|
|
|
- processJoins("pending joins after accumulation stop [" + reason + "]");
|
|
|
- }
|
|
|
+ /**
|
|
|
+ * Stopped accumulating joins. All pending joins will be processed. Future joins will be processed immediately
|
|
|
+ */
|
|
|
+ public void stopElectionContext(String reason) {
|
|
|
+ logger.trace("stopping election ([{}])", reason);
|
|
|
+ synchronized (this) {
|
|
|
+ assert electionContext != null : "stopElectionContext() called but not accumulating";
|
|
|
+ electionContext.closeAndProcessPending(reason);
|
|
|
+ electionContext = null;
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -190,19 +174,14 @@ public class NodeJoinController extends AbstractComponent {
|
|
|
* <p>
|
|
|
* Note: doesn't do any validation. This should have been done before.
|
|
|
*/
|
|
|
- public void handleJoinRequest(final DiscoveryNode node, final MembershipAction.JoinCallback callback) {
|
|
|
- synchronized (pendingJoinRequests) {
|
|
|
- List<MembershipAction.JoinCallback> nodeCallbacks = pendingJoinRequests.get(node);
|
|
|
- if (nodeCallbacks == null) {
|
|
|
- nodeCallbacks = new ArrayList<>();
|
|
|
- pendingJoinRequests.put(node, nodeCallbacks);
|
|
|
- }
|
|
|
- nodeCallbacks.add(callback);
|
|
|
- }
|
|
|
- if (accumulateJoins.get() == false) {
|
|
|
- processJoins("join from node[" + node + "]");
|
|
|
- } else {
|
|
|
+ public synchronized void handleJoinRequest(final DiscoveryNode node, final MembershipAction.JoinCallback callback) {
|
|
|
+ if (electionContext != null) {
|
|
|
+ electionContext.addIncomingJoin(node, callback);
|
|
|
checkPendingJoinsAndElectIfNeeded();
|
|
|
+ } else {
|
|
|
+ clusterService.submitStateUpdateTask("zen-disco-join(node " + node + "])",
|
|
|
+ node, ClusterStateTaskConfig.build(Priority.URGENT),
|
|
|
+ joinTaskExecutor, new JoinTaskListener(callback, logger));
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -210,244 +189,288 @@ public class NodeJoinController extends AbstractComponent {
|
|
|
* checks if there is an on going request to become master and if it has enough pending joins. If so, the node will
|
|
|
* become master via a ClusterState update task.
|
|
|
*/
|
|
|
- private void checkPendingJoinsAndElectIfNeeded() {
|
|
|
- assert accumulateJoins.get() : "election check requested but we are not accumulating joins";
|
|
|
- final ElectionContext context = electionContext.get();
|
|
|
- if (context == null) {
|
|
|
- return;
|
|
|
+ private synchronized void checkPendingJoinsAndElectIfNeeded() {
|
|
|
+ assert electionContext != null : "election check requested but no active context";
|
|
|
+ final int pendingMasterJoins = electionContext.getPendingMasterJoinsCount();
|
|
|
+ if (electionContext.isEnoughPendingJoins(pendingMasterJoins) == false) {
|
|
|
+ if (logger.isTraceEnabled()) {
|
|
|
+ logger.trace("not enough joins for election. Got [{}], required [{}]", pendingMasterJoins,
|
|
|
+ electionContext.requiredMasterJoins);
|
|
|
+ }
|
|
|
+ } else {
|
|
|
+ if (logger.isTraceEnabled()) {
|
|
|
+ logger.trace("have enough joins for election. Got [{}], required [{}]", pendingMasterJoins,
|
|
|
+ electionContext.requiredMasterJoins);
|
|
|
+ }
|
|
|
+ electionContext.closeAndBecomeMaster();
|
|
|
+ electionContext = null; // clear this out so future joins won't be accumulated
|
|
|
}
|
|
|
+ }
|
|
|
|
|
|
- int pendingMasterJoins = 0;
|
|
|
- synchronized (pendingJoinRequests) {
|
|
|
- for (DiscoveryNode node : pendingJoinRequests.keySet()) {
|
|
|
+ public interface ElectionCallback {
|
|
|
+ /**
|
|
|
+ * called when the local node is successfully elected as master
|
|
|
+ * Guaranteed to be called on the cluster state update thread
|
|
|
+ **/
|
|
|
+ void onElectedAsMaster(ClusterState state);
|
|
|
+
|
|
|
+ /**
|
|
|
+ * called when the local node failed to be elected as master
|
|
|
+ * Guaranteed to be called on the cluster state update thread
|
|
|
+ **/
|
|
|
+ void onFailure(Throwable t);
|
|
|
+ }
|
|
|
+
|
|
|
+ class ElectionContext {
|
|
|
+ private ElectionCallback callback = null;
|
|
|
+ private int requiredMasterJoins = -1;
|
|
|
+ private final Map<DiscoveryNode, List<MembershipAction.JoinCallback>> joinRequestAccumulator = new HashMap<>();
|
|
|
+
|
|
|
+ final AtomicBoolean closed = new AtomicBoolean();
|
|
|
+
|
|
|
+ public synchronized void onAttemptToBeElected(int requiredMasterJoins, ElectionCallback callback) {
|
|
|
+ ensureOpen();
|
|
|
+ assert this.requiredMasterJoins < 0;
|
|
|
+ assert this.callback == null;
|
|
|
+ this.requiredMasterJoins = requiredMasterJoins;
|
|
|
+ this.callback = callback;
|
|
|
+ }
|
|
|
+
|
|
|
+ public synchronized void addIncomingJoin(DiscoveryNode node, MembershipAction.JoinCallback callback) {
|
|
|
+ ensureOpen();
|
|
|
+ joinRequestAccumulator.computeIfAbsent(node, n -> new ArrayList<>()).add(callback);
|
|
|
+ }
|
|
|
+
|
|
|
+
|
|
|
+ public synchronized boolean isEnoughPendingJoins(int pendingMasterJoins) {
|
|
|
+ final boolean hasEnough;
|
|
|
+ if (requiredMasterJoins < 0) {
|
|
|
+ // requiredMasterNodes is unknown yet, return false and keep on waiting
|
|
|
+ hasEnough = false;
|
|
|
+ } else {
|
|
|
+ assert callback != null : "requiredMasterJoins is set but not the callback";
|
|
|
+ hasEnough = pendingMasterJoins >= requiredMasterJoins;
|
|
|
+ }
|
|
|
+ return hasEnough;
|
|
|
+ }
|
|
|
+
|
|
|
+ private Map<DiscoveryNode, ClusterStateTaskListener> getPendingAsTasks() {
|
|
|
+ Map<DiscoveryNode, ClusterStateTaskListener> tasks = new HashMap<>();
|
|
|
+ joinRequestAccumulator.entrySet().stream().forEach(e -> tasks.put(e.getKey(), new JoinTaskListener(e.getValue(), logger)));
|
|
|
+ return tasks;
|
|
|
+ }
|
|
|
+
|
|
|
+ public synchronized int getPendingMasterJoinsCount() {
|
|
|
+ int pendingMasterJoins = 0;
|
|
|
+ for (DiscoveryNode node : joinRequestAccumulator.keySet()) {
|
|
|
if (node.isMasterNode()) {
|
|
|
pendingMasterJoins++;
|
|
|
}
|
|
|
}
|
|
|
+ return pendingMasterJoins;
|
|
|
}
|
|
|
- if (pendingMasterJoins < context.requiredMasterJoins) {
|
|
|
- if (context.pendingSetAsMasterTask.get() == false) {
|
|
|
- logger.trace("not enough joins for election. Got [{}], required [{}]", pendingMasterJoins, context.requiredMasterJoins);
|
|
|
- }
|
|
|
- return;
|
|
|
- }
|
|
|
- if (context.pendingSetAsMasterTask.getAndSet(true)) {
|
|
|
- logger.trace("elected as master task already submitted, ignoring...");
|
|
|
- return;
|
|
|
+
|
|
|
+ public synchronized void closeAndBecomeMaster() {
|
|
|
+ assert callback != null : "becoming a master but the callback is not yet set";
|
|
|
+ assert isEnoughPendingJoins(getPendingMasterJoinsCount()) : "becoming a master but pending joins of "
|
|
|
+ + getPendingMasterJoinsCount() + " are not enough. needs [" + requiredMasterJoins + "];";
|
|
|
+
|
|
|
+ innerClose();
|
|
|
+
|
|
|
+ Map<DiscoveryNode, ClusterStateTaskListener> tasks = getPendingAsTasks();
|
|
|
+ final String source = "zen-disco-join(elected_as_master, [" + tasks.size() + "] nodes joined)";
|
|
|
+
|
|
|
+ tasks.put(BECOME_MASTER_TASK, joinProcessedListener);
|
|
|
+ clusterService.submitStateUpdateTasks(source, tasks, ClusterStateTaskConfig.build(Priority.URGENT), joinTaskExecutor);
|
|
|
}
|
|
|
|
|
|
- final String source = "zen-disco-join(elected_as_master, [" + pendingMasterJoins + "] joins received)";
|
|
|
- clusterService.submitStateUpdateTask(source, new ProcessJoinsTask(Priority.IMMEDIATE) {
|
|
|
- @Override
|
|
|
- public ClusterState execute(ClusterState currentState) {
|
|
|
- // Take into account the previous known nodes, if they happen not to be available
|
|
|
- // then fault detection will remove these nodes.
|
|
|
-
|
|
|
- if (currentState.nodes().getMasterNode() != null) {
|
|
|
- // TODO can we tie break here? we don't have a remote master cluster state version to decide on
|
|
|
- logger.trace("join thread elected local node as master, but there is already a master in place: {}", currentState.nodes().getMasterNode());
|
|
|
- throw new NotMasterException("Node [" + clusterService.localNode() + "] not master for join request");
|
|
|
- }
|
|
|
+ public synchronized void closeAndProcessPending(String reason) {
|
|
|
+ innerClose();
|
|
|
+ Map<DiscoveryNode, ClusterStateTaskListener> tasks = getPendingAsTasks();
|
|
|
+ final String source = "zen-disco-join(election stopped [" + reason + "] nodes joined";
|
|
|
|
|
|
- DiscoveryNodes.Builder builder = new DiscoveryNodes.Builder(currentState.nodes()).masterNodeId(currentState.nodes().getLocalNode().getId());
|
|
|
- // update the fact that we are the master...
|
|
|
- ClusterBlocks clusterBlocks = ClusterBlocks.builder().blocks(currentState.blocks()).removeGlobalBlock(discoverySettings.getNoMasterBlock()).build();
|
|
|
- currentState = ClusterState.builder(currentState).nodes(builder).blocks(clusterBlocks).build();
|
|
|
+ tasks.put(FINISH_ELECTION_NOT_MASTER_TASK, joinProcessedListener);
|
|
|
+ clusterService.submitStateUpdateTasks(source, tasks, ClusterStateTaskConfig.build(Priority.URGENT), joinTaskExecutor);
|
|
|
+ }
|
|
|
|
|
|
- // reroute now to remove any dead nodes (master may have stepped down when they left and didn't update the routing table)
|
|
|
- RoutingAllocation.Result result = routingService.getAllocationService().reroute(currentState, "nodes joined");
|
|
|
- if (result.changed()) {
|
|
|
- currentState = ClusterState.builder(currentState).routingResult(result).build();
|
|
|
- }
|
|
|
+ private void innerClose() {
|
|
|
+ if (closed.getAndSet(true)) {
|
|
|
+ throw new AlreadyClosedException("election context is already closed");
|
|
|
+ }
|
|
|
+ }
|
|
|
|
|
|
- // Add the incoming join requests.
|
|
|
- // Note: we only do this now (after the reroute) to avoid assigning shards to these nodes.
|
|
|
- return super.execute(currentState);
|
|
|
+ private void ensureOpen() {
|
|
|
+ if (closed.get()) {
|
|
|
+ throw new AlreadyClosedException("election context is already closed");
|
|
|
}
|
|
|
+ }
|
|
|
|
|
|
- @Override
|
|
|
- public boolean runOnlyOnMaster() {
|
|
|
- return false;
|
|
|
+ private synchronized ElectionCallback getCallback() {
|
|
|
+ return callback;
|
|
|
+ }
|
|
|
+
|
|
|
+ private void onElectedAsMaster(ClusterState state) {
|
|
|
+ ClusterService.assertClusterStateThread();
|
|
|
+ assert state.nodes().isLocalNodeElectedMaster() : "onElectedAsMaster called but local node is not master";
|
|
|
+ ElectionCallback callback = getCallback(); // get under lock
|
|
|
+ if (callback != null) {
|
|
|
+ callback.onElectedAsMaster(state);
|
|
|
}
|
|
|
+ }
|
|
|
|
|
|
- @Override
|
|
|
- public void onFailure(String source, Throwable t) {
|
|
|
- super.onFailure(source, t);
|
|
|
- context.onFailure(t);
|
|
|
+ private void onFailure(Throwable t) {
|
|
|
+ ClusterService.assertClusterStateThread();
|
|
|
+ ElectionCallback callback = getCallback(); // get under lock
|
|
|
+ if (callback != null) {
|
|
|
+ callback.onFailure(t);
|
|
|
}
|
|
|
+ }
|
|
|
+
|
|
|
+ private final ClusterStateTaskListener joinProcessedListener = new ClusterStateTaskListener() {
|
|
|
|
|
|
@Override
|
|
|
public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) {
|
|
|
- super.clusterStateProcessed(source, oldState, newState);
|
|
|
- context.onElectedAsMaster(newState);
|
|
|
+ assert newState.nodes().isLocalNodeElectedMaster() : "should have become a master but isn't " + newState.prettyPrint();
|
|
|
+ onElectedAsMaster(newState);
|
|
|
}
|
|
|
- });
|
|
|
- }
|
|
|
-
|
|
|
- /** process all pending joins */
|
|
|
- private void processJoins(String reason) {
|
|
|
- clusterService.submitStateUpdateTask("zen-disco-join(" + reason + ")", new ProcessJoinsTask(Priority.URGENT));
|
|
|
- }
|
|
|
-
|
|
|
|
|
|
- public interface ElectionCallback {
|
|
|
- /**
|
|
|
- * called when the local node is successfully elected as master
|
|
|
- * Guaranteed to be called on the cluster state update thread
|
|
|
- **/
|
|
|
- void onElectedAsMaster(ClusterState state);
|
|
|
+ @Override
|
|
|
+ public void onFailure(String source, Throwable t) {
|
|
|
+ ElectionContext.this.onFailure(t);
|
|
|
+ }
|
|
|
+ };
|
|
|
|
|
|
- /**
|
|
|
- * called when the local node failed to be elected as master
|
|
|
- * Guaranteed to be called on the cluster state update thread
|
|
|
- **/
|
|
|
- void onFailure(Throwable t);
|
|
|
}
|
|
|
|
|
|
- static abstract class ElectionContext implements ElectionCallback {
|
|
|
- private final ElectionCallback callback;
|
|
|
- private final int requiredMasterJoins;
|
|
|
+ static class JoinTaskListener implements ClusterStateTaskListener {
|
|
|
+ final List<MembershipAction.JoinCallback> callbacks;
|
|
|
+ final private ESLogger logger;
|
|
|
|
|
|
- /** set to true after enough joins have been seen and a cluster update task is submitted to become master */
|
|
|
- final AtomicBoolean pendingSetAsMasterTask = new AtomicBoolean();
|
|
|
- final AtomicBoolean closed = new AtomicBoolean();
|
|
|
-
|
|
|
- ElectionContext(ElectionCallback callback, int requiredMasterJoins) {
|
|
|
- this.callback = callback;
|
|
|
- this.requiredMasterJoins = requiredMasterJoins;
|
|
|
+ JoinTaskListener(MembershipAction.JoinCallback callback, ESLogger logger) {
|
|
|
+ this(Collections.singletonList(callback), logger);
|
|
|
}
|
|
|
|
|
|
- abstract void onClose();
|
|
|
+ JoinTaskListener(List<MembershipAction.JoinCallback> callbacks, ESLogger logger) {
|
|
|
+ this.callbacks = callbacks;
|
|
|
+ this.logger = logger;
|
|
|
+ }
|
|
|
|
|
|
@Override
|
|
|
- public void onElectedAsMaster(ClusterState state) {
|
|
|
- assert pendingSetAsMasterTask.get() : "onElectedAsMaster called but pendingSetAsMasterTask is not set";
|
|
|
- ClusterService.assertClusterStateThread();
|
|
|
- assert state.nodes().isLocalNodeElectedMaster() : "onElectedAsMaster called but local node is not master";
|
|
|
- if (closed.compareAndSet(false, true)) {
|
|
|
+ public void onFailure(String source, Throwable t) {
|
|
|
+ for (MembershipAction.JoinCallback callback : callbacks) {
|
|
|
try {
|
|
|
- onClose();
|
|
|
- } finally {
|
|
|
- callback.onElectedAsMaster(state);
|
|
|
+ callback.onFailure(t);
|
|
|
+ } catch (Exception e) {
|
|
|
+ logger.error("error during task failure", e);
|
|
|
}
|
|
|
}
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
- public void onFailure(Throwable t) {
|
|
|
- ClusterService.assertClusterStateThread();
|
|
|
- if (closed.compareAndSet(false, true)) {
|
|
|
+ public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) {
|
|
|
+ for (MembershipAction.JoinCallback callback : callbacks) {
|
|
|
try {
|
|
|
- onClose();
|
|
|
- } finally {
|
|
|
- callback.onFailure(t);
|
|
|
+ callback.onSuccess();
|
|
|
+ } catch (Exception e) {
|
|
|
+ logger.error("unexpected error during [{}]", e, source);
|
|
|
}
|
|
|
}
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ // a task indicated that the current node should become master, if no current master is known
|
|
|
+ private final static DiscoveryNode BECOME_MASTER_TASK = new DiscoveryNode("_BECOME_MASTER_TASK_", DummyTransportAddress.INSTANCE,
|
|
|
+ Collections.emptyMap(), Collections.emptySet(), Version.CURRENT);
|
|
|
|
|
|
- /**
|
|
|
- * Processes any pending joins via a ClusterState update task.
|
|
|
- * Note: this task automatically fails (and fails all pending joins) if the current node is not marked as master
|
|
|
- */
|
|
|
- class ProcessJoinsTask extends ClusterStateUpdateTask {
|
|
|
+ // a task that is used to process pending joins without explicitly becoming a master and listening to the results
|
|
|
+ // this task is used when election is stop without the local node becoming a master per se (though it might
|
|
|
+ private final static DiscoveryNode FINISH_ELECTION_NOT_MASTER_TASK = new DiscoveryNode("_NOT_MASTER_TASK_",
|
|
|
+ DummyTransportAddress.INSTANCE, Collections.emptyMap(), Collections.emptySet(), Version.CURRENT);
|
|
|
|
|
|
- private final List<MembershipAction.JoinCallback> joinCallbacksToRespondTo = new ArrayList<>();
|
|
|
- private boolean nodeAdded = false;
|
|
|
-
|
|
|
- public ProcessJoinsTask(Priority priority) {
|
|
|
- super(priority);
|
|
|
- }
|
|
|
+ class JoinTaskExecutor implements ClusterStateTaskExecutor<DiscoveryNode> {
|
|
|
|
|
|
@Override
|
|
|
- public ClusterState execute(ClusterState currentState) {
|
|
|
- DiscoveryNodes.Builder nodesBuilder;
|
|
|
- synchronized (pendingJoinRequests) {
|
|
|
- if (pendingJoinRequests.isEmpty()) {
|
|
|
- return currentState;
|
|
|
- }
|
|
|
+ public BatchResult<DiscoveryNode> execute(ClusterState currentState, List<DiscoveryNode> joiningNodes) throws Exception {
|
|
|
+ final DiscoveryNodes currentNodes = currentState.nodes();
|
|
|
+ final BatchResult.Builder<DiscoveryNode> results = BatchResult.builder();
|
|
|
+ boolean nodesChanged = false;
|
|
|
+ ClusterState.Builder newState = ClusterState.builder(currentState);
|
|
|
+ DiscoveryNodes.Builder nodesBuilder = DiscoveryNodes.builder(currentNodes);
|
|
|
+
|
|
|
+ if (currentNodes.getMasterNode() == null && joiningNodes.contains(BECOME_MASTER_TASK)) {
|
|
|
+ // use these joins to try and become the master.
|
|
|
+ // Note that we don't have to do any validation of the amount of joining nodes - the commit
|
|
|
+ // during the cluster state publishing guarantees that we have enough
|
|
|
+
|
|
|
+ nodesBuilder.masterNodeId(currentNodes.getLocalNodeId());
|
|
|
+ ClusterBlocks clusterBlocks = ClusterBlocks.builder().blocks(currentState.blocks())
|
|
|
+ .removeGlobalBlock(discoverySettings.getNoMasterBlock()).build();
|
|
|
+ newState.blocks(clusterBlocks);
|
|
|
+ newState.nodes(nodesBuilder);
|
|
|
+ nodesChanged = true;
|
|
|
|
|
|
- nodesBuilder = DiscoveryNodes.builder(currentState.nodes());
|
|
|
- Iterator<Map.Entry<DiscoveryNode, List<MembershipAction.JoinCallback>>> iterator = pendingJoinRequests.entrySet().iterator();
|
|
|
- while (iterator.hasNext()) {
|
|
|
- Map.Entry<DiscoveryNode, List<MembershipAction.JoinCallback>> entry = iterator.next();
|
|
|
- final DiscoveryNode node = entry.getKey();
|
|
|
- joinCallbacksToRespondTo.addAll(entry.getValue());
|
|
|
- iterator.remove();
|
|
|
- if (currentState.nodes().nodeExists(node.getId())) {
|
|
|
- logger.debug("received a join request for an existing node [{}]", node);
|
|
|
- } else {
|
|
|
- nodeAdded = true;
|
|
|
- nodesBuilder.put(node);
|
|
|
- for (DiscoveryNode existingNode : currentState.nodes()) {
|
|
|
- if (node.getAddress().equals(existingNode.getAddress())) {
|
|
|
- nodesBuilder.remove(existingNode.getId());
|
|
|
- logger.warn("received join request from node [{}], but found existing node {} with same address, removing existing node", node, existingNode);
|
|
|
- }
|
|
|
- }
|
|
|
- }
|
|
|
+ // reroute now to remove any dead nodes (master may have stepped down when they left and didn't update the routing table)
|
|
|
+ // Note: also do it now to avoid assigning shards to these nodes. We will have another reroute after the cluster
|
|
|
+ // state is published.
|
|
|
+ // TODO: this publishing of a cluster state with no nodes assigned to joining nodes shouldn't be needed anymore. remove.
|
|
|
+
|
|
|
+ final ClusterState tmpState = newState.build();
|
|
|
+ RoutingAllocation.Result result = routingService.getAllocationService().reroute(tmpState, "nodes joined");
|
|
|
+ newState = ClusterState.builder(tmpState);
|
|
|
+ if (result.changed()) {
|
|
|
+ newState.routingResult(result);
|
|
|
}
|
|
|
+ nodesBuilder = DiscoveryNodes.builder(tmpState.nodes());
|
|
|
}
|
|
|
|
|
|
- // we must return a new cluster state instance to force publishing. This is important
|
|
|
- // for the joining node to finalize it's join and set us as a master
|
|
|
- final ClusterState.Builder newState = ClusterState.builder(currentState);
|
|
|
- if (nodeAdded) {
|
|
|
- newState.nodes(nodesBuilder);
|
|
|
+ if (nodesBuilder.isLocalNodeElectedMaster() == false) {
|
|
|
+ logger.trace("processing node joins, but we are not the master. current master: {}", currentNodes.getMasterNode());
|
|
|
+ throw new NotMasterException("Node [" + currentNodes.getLocalNode() + "] not master for join request");
|
|
|
}
|
|
|
|
|
|
- return newState.build();
|
|
|
- }
|
|
|
-
|
|
|
- @Override
|
|
|
- public void onNoLongerMaster(String source) {
|
|
|
- // we are rejected, so drain all pending task (execute never run)
|
|
|
- synchronized (pendingJoinRequests) {
|
|
|
- Iterator<Map.Entry<DiscoveryNode, List<MembershipAction.JoinCallback>>> iterator = pendingJoinRequests.entrySet().iterator();
|
|
|
- while (iterator.hasNext()) {
|
|
|
- Map.Entry<DiscoveryNode, List<MembershipAction.JoinCallback>> entry = iterator.next();
|
|
|
- joinCallbacksToRespondTo.addAll(entry.getValue());
|
|
|
- iterator.remove();
|
|
|
+ for (final DiscoveryNode node : joiningNodes) {
|
|
|
+ if (node.equals(BECOME_MASTER_TASK) || node.equals(FINISH_ELECTION_NOT_MASTER_TASK)) {
|
|
|
+ // noop
|
|
|
+ } else if (currentNodes.nodeExists(node.getId())) {
|
|
|
+ logger.debug("received a join request for an existing node [{}]", node);
|
|
|
+ } else {
|
|
|
+ nodesChanged = true;
|
|
|
+ nodesBuilder.put(node);
|
|
|
+ for (DiscoveryNode existingNode : currentNodes) {
|
|
|
+ if (node.getAddress().equals(existingNode.getAddress())) {
|
|
|
+ nodesBuilder.remove(existingNode.getId());
|
|
|
+ logger.warn("received join request from node [{}], but found existing node {} with same address, removing existing node", node, existingNode);
|
|
|
+ }
|
|
|
+ }
|
|
|
}
|
|
|
+ results.success(node);
|
|
|
}
|
|
|
- Exception e = new NotMasterException("Node [" + clusterService.localNode() + "] not master for join request");
|
|
|
- innerOnFailure(e);
|
|
|
- }
|
|
|
|
|
|
- void innerOnFailure(Throwable t) {
|
|
|
- for (MembershipAction.JoinCallback callback : joinCallbacksToRespondTo) {
|
|
|
- try {
|
|
|
- callback.onFailure(t);
|
|
|
- } catch (Exception e) {
|
|
|
- logger.error("error during task failure", e);
|
|
|
- }
|
|
|
+ if (nodesChanged) {
|
|
|
+ newState.nodes(nodesBuilder);
|
|
|
}
|
|
|
+
|
|
|
+ // we must return a new cluster state instance to force publishing. This is important
|
|
|
+ // for the joining node to finalize its join and set us as a master
|
|
|
+ return results.build(newState.build());
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
- public void onFailure(String source, Throwable t) {
|
|
|
- logger.error("unexpected failure during [{}]", t, source);
|
|
|
- innerOnFailure(t);
|
|
|
+ public boolean runOnlyOnMaster() {
|
|
|
+ // we validate that we are allowed to change the cluster state during cluster state processing
|
|
|
+ return false;
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
- public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) {
|
|
|
- if (nodeAdded) {
|
|
|
+ public void clusterStatePublished(ClusterChangedEvent event) {
|
|
|
+ if (event.nodesDelta().hasChanges()) {
|
|
|
// we reroute not in the same cluster state update since in certain areas we rely on
|
|
|
// the node to be in the cluster state (sampled from ClusterService#state) to be there, also
|
|
|
// shard transitions need to better be handled in such cases
|
|
|
routingService.reroute("post_node_add");
|
|
|
}
|
|
|
- for (MembershipAction.JoinCallback callback : joinCallbacksToRespondTo) {
|
|
|
- try {
|
|
|
- callback.onSuccess();
|
|
|
- } catch (Exception e) {
|
|
|
- logger.error("unexpected error during [{}]", e, source);
|
|
|
- }
|
|
|
- }
|
|
|
|
|
|
- NodeJoinController.this.electMaster.logMinimumMasterNodesWarningIfNecessary(oldState, newState);
|
|
|
+ NodeJoinController.this.electMaster.logMinimumMasterNodesWarningIfNecessary(event.previousState(), event.state());
|
|
|
}
|
|
|
}
|
|
|
}
|