|  | @@ -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());
 | 
	
		
			
				|  |  |          }
 | 
	
		
			
				|  |  |      }
 | 
	
		
			
				|  |  |  }
 |