|
|
@@ -25,6 +25,9 @@ import org.elasticsearch.Version;
|
|
|
import org.elasticsearch.cluster.ClusterChangedEvent;
|
|
|
import org.elasticsearch.cluster.ClusterName;
|
|
|
import org.elasticsearch.cluster.ClusterState;
|
|
|
+import org.elasticsearch.cluster.ClusterStateTaskConfig;
|
|
|
+import org.elasticsearch.cluster.ClusterStateTaskExecutor;
|
|
|
+import org.elasticsearch.cluster.ClusterStateTaskListener;
|
|
|
import org.elasticsearch.cluster.ClusterStateUpdateTask;
|
|
|
import org.elasticsearch.cluster.NotMasterException;
|
|
|
import org.elasticsearch.cluster.block.ClusterBlocks;
|
|
|
@@ -79,13 +82,11 @@ import java.util.concurrent.atomic.AtomicBoolean;
|
|
|
import java.util.concurrent.atomic.AtomicInteger;
|
|
|
import java.util.concurrent.atomic.AtomicLong;
|
|
|
import java.util.concurrent.atomic.AtomicReference;
|
|
|
+import java.util.function.BiFunction;
|
|
|
import java.util.stream.Collectors;
|
|
|
|
|
|
import static org.elasticsearch.common.unit.TimeValue.timeValueSeconds;
|
|
|
|
|
|
-/**
|
|
|
- *
|
|
|
- */
|
|
|
public class ZenDiscovery extends AbstractLifecycleComponent implements Discovery, PingContextProvider {
|
|
|
|
|
|
public static final Setting<TimeValue> PING_TIMEOUT_SETTING =
|
|
|
@@ -148,6 +149,7 @@ public class ZenDiscovery extends AbstractLifecycleComponent implements Discover
|
|
|
|
|
|
// must initialized in doStart(), when we have the allocationService set
|
|
|
private volatile NodeJoinController nodeJoinController;
|
|
|
+ private volatile NodeRemovalClusterStateTaskExecutor nodeRemovalExecutor;
|
|
|
|
|
|
@Inject
|
|
|
public ZenDiscovery(Settings settings, ThreadPool threadPool,
|
|
|
@@ -216,6 +218,7 @@ public class ZenDiscovery extends AbstractLifecycleComponent implements Discover
|
|
|
joinThreadControl.start();
|
|
|
pingService.start();
|
|
|
this.nodeJoinController = new NodeJoinController(clusterService, allocationService, electMaster, discoverySettings, settings);
|
|
|
+ this.nodeRemovalExecutor = new NodeRemovalClusterStateTaskExecutor(allocationService, electMaster, this::rejoin, logger);
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
@@ -500,43 +503,119 @@ public class ZenDiscovery extends AbstractLifecycleComponent implements Discover
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ // visible for testing
|
|
|
+ static class NodeRemovalClusterStateTaskExecutor implements ClusterStateTaskExecutor<NodeRemovalClusterStateTaskExecutor.Task>, ClusterStateTaskListener {
|
|
|
+
|
|
|
+ private final AllocationService allocationService;
|
|
|
+ private final ElectMasterService electMasterService;
|
|
|
+ private final BiFunction<ClusterState, String, ClusterState> rejoin;
|
|
|
+ private final ESLogger logger;
|
|
|
+
|
|
|
+ static class Task {
|
|
|
+
|
|
|
+ private final DiscoveryNode node;
|
|
|
+ private final String reason;
|
|
|
+
|
|
|
+ public Task(final DiscoveryNode node, final String reason) {
|
|
|
+ this.node = node;
|
|
|
+ this.reason = reason;
|
|
|
+ }
|
|
|
+
|
|
|
+ public DiscoveryNode node() {
|
|
|
+ return node;
|
|
|
+ }
|
|
|
+
|
|
|
+ public String reason() {
|
|
|
+ return reason;
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public String toString() {
|
|
|
+ return node + " " + reason;
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ NodeRemovalClusterStateTaskExecutor(
|
|
|
+ final AllocationService allocationService,
|
|
|
+ final ElectMasterService electMasterService,
|
|
|
+ final BiFunction<ClusterState, String, ClusterState> rejoin,
|
|
|
+ final ESLogger logger) {
|
|
|
+ this.allocationService = allocationService;
|
|
|
+ this.electMasterService = electMasterService;
|
|
|
+ this.rejoin = rejoin;
|
|
|
+ this.logger = logger;
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public BatchResult<Task> execute(final ClusterState currentState, final List<Task> tasks) throws Exception {
|
|
|
+ final DiscoveryNodes.Builder remainingNodesBuilder = DiscoveryNodes.builder(currentState.nodes());
|
|
|
+ boolean removed = false;
|
|
|
+ for (final Task task : tasks) {
|
|
|
+ if (currentState.nodes().nodeExists(task.node())) {
|
|
|
+ remainingNodesBuilder.remove(task.node());
|
|
|
+ removed = true;
|
|
|
+ } else {
|
|
|
+ logger.debug("node [{}] does not exist in cluster state, ignoring", task);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ if (!removed) {
|
|
|
+ // no nodes to remove, keep the current cluster state
|
|
|
+ return BatchResult.<Task>builder().successes(tasks).build(currentState);
|
|
|
+ }
|
|
|
+
|
|
|
+ final ClusterState remainingNodesClusterState = remainingNodesClusterState(currentState, remainingNodesBuilder);
|
|
|
+
|
|
|
+ final BatchResult.Builder<Task> resultBuilder = BatchResult.<Task>builder().successes(tasks);
|
|
|
+ if (!electMasterService.hasEnoughMasterNodes(remainingNodesClusterState.nodes())) {
|
|
|
+ return resultBuilder.build(rejoin.apply(remainingNodesClusterState, "not enough master nodes"));
|
|
|
+ } else {
|
|
|
+ final RoutingAllocation.Result routingResult = allocationService.reroute(remainingNodesClusterState, describeTasks(tasks));
|
|
|
+ return resultBuilder.build(ClusterState.builder(remainingNodesClusterState).routingResult(routingResult).build());
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ // visible for testing
|
|
|
+ // hook is used in testing to ensure that correct cluster state is used to test whether a
|
|
|
+ // rejoin or reroute is needed
|
|
|
+ ClusterState remainingNodesClusterState(final ClusterState currentState, DiscoveryNodes.Builder remainingNodesBuilder) {
|
|
|
+ return ClusterState.builder(currentState).nodes(remainingNodesBuilder).build();
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public void onFailure(final String source, final Exception e) {
|
|
|
+ logger.error("unexpected failure during [{}]", e, source);
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public void onNoLongerMaster(String source) {
|
|
|
+ logger.debug("no longer master while processing node removal [{}]", source);
|
|
|
+ }
|
|
|
+
|
|
|
+ }
|
|
|
+
|
|
|
+ private void removeNode(final DiscoveryNode node, final String source, final String reason) {
|
|
|
+ clusterService.submitStateUpdateTask(
|
|
|
+ source + "(" + node + "), reason(" + reason + ")",
|
|
|
+ new NodeRemovalClusterStateTaskExecutor.Task(node, reason),
|
|
|
+ ClusterStateTaskConfig.build(Priority.IMMEDIATE),
|
|
|
+ nodeRemovalExecutor,
|
|
|
+ nodeRemovalExecutor);
|
|
|
+ }
|
|
|
+
|
|
|
private void handleLeaveRequest(final DiscoveryNode node) {
|
|
|
if (lifecycleState() != Lifecycle.State.STARTED) {
|
|
|
// not started, ignore a node failure
|
|
|
return;
|
|
|
}
|
|
|
if (localNodeMaster()) {
|
|
|
- clusterService.submitStateUpdateTask("zen-disco-node-left(" + node + ")", new ClusterStateUpdateTask(Priority.IMMEDIATE) {
|
|
|
- @Override
|
|
|
- public ClusterState execute(ClusterState currentState) {
|
|
|
- DiscoveryNodes.Builder builder = DiscoveryNodes.builder(currentState.nodes()).remove(node.getId());
|
|
|
- currentState = ClusterState.builder(currentState).nodes(builder).build();
|
|
|
- // check if we have enough master nodes, if not, we need to move into joining the cluster again
|
|
|
- if (!electMaster.hasEnoughMasterNodes(currentState.nodes())) {
|
|
|
- return rejoin(currentState, "not enough master nodes");
|
|
|
- }
|
|
|
- // eagerly run reroute to remove dead nodes from routing table
|
|
|
- RoutingAllocation.Result routingResult = allocationService.reroute(ClusterState.builder(currentState).build(),
|
|
|
- "[" + node + "] left");
|
|
|
- return ClusterState.builder(currentState).routingResult(routingResult).build();
|
|
|
- }
|
|
|
-
|
|
|
- @Override
|
|
|
- public void onNoLongerMaster(String source) {
|
|
|
- // ignoring (already logged)
|
|
|
- }
|
|
|
-
|
|
|
- @Override
|
|
|
- public void onFailure(String source, Exception e) {
|
|
|
- logger.error("unexpected failure during [{}]", e, source);
|
|
|
- }
|
|
|
- });
|
|
|
+ removeNode(node, "zen-disco-node-left", "left");
|
|
|
} else if (node.equals(nodes().getMasterNode())) {
|
|
|
handleMasterGone(node, null, "shut_down");
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- private void handleNodeFailure(final DiscoveryNode node, String reason) {
|
|
|
+ private void handleNodeFailure(final DiscoveryNode node, final String reason) {
|
|
|
if (lifecycleState() != Lifecycle.State.STARTED) {
|
|
|
// not started, ignore a node failure
|
|
|
return;
|
|
|
@@ -545,41 +624,7 @@ public class ZenDiscovery extends AbstractLifecycleComponent implements Discover
|
|
|
// nothing to do here...
|
|
|
return;
|
|
|
}
|
|
|
- clusterService.submitStateUpdateTask("zen-disco-node-failed(" + node + "), reason " + reason,
|
|
|
- new ClusterStateUpdateTask(Priority.IMMEDIATE) {
|
|
|
- @Override
|
|
|
- public ClusterState execute(ClusterState currentState) {
|
|
|
- if (currentState.nodes().nodeExists(node) == false) {
|
|
|
- logger.debug("node [{}] already removed from cluster state. ignoring.", node);
|
|
|
- return currentState;
|
|
|
- }
|
|
|
- DiscoveryNodes.Builder builder = DiscoveryNodes.builder(currentState.nodes()).remove(node);
|
|
|
- currentState = ClusterState.builder(currentState).nodes(builder).build();
|
|
|
- // check if we have enough master nodes, if not, we need to move into joining the cluster again
|
|
|
- if (!electMaster.hasEnoughMasterNodes(currentState.nodes())) {
|
|
|
- return rejoin(currentState, "not enough master nodes");
|
|
|
- }
|
|
|
- // eagerly run reroute to remove dead nodes from routing table
|
|
|
- RoutingAllocation.Result routingResult = allocationService.reroute(
|
|
|
- ClusterState.builder(currentState).build(),
|
|
|
- "[" + node + "] failed");
|
|
|
- return ClusterState.builder(currentState).routingResult(routingResult).build();
|
|
|
- }
|
|
|
-
|
|
|
- @Override
|
|
|
- public void onNoLongerMaster(String source) {
|
|
|
- // already logged
|
|
|
- }
|
|
|
-
|
|
|
- @Override
|
|
|
- public void onFailure(String source, Exception e) {
|
|
|
- logger.error("unexpected failure during [{}]", e, source);
|
|
|
- }
|
|
|
-
|
|
|
- @Override
|
|
|
- public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) {
|
|
|
- }
|
|
|
- });
|
|
|
+ removeNode(node, "zen-disco-node-failed", reason);
|
|
|
}
|
|
|
|
|
|
private void handleMinimumMasterNodesChanged(final int minimumMasterNodes) {
|