|  | @@ -20,7 +20,6 @@
 | 
											
												
													
														|  |  package org.elasticsearch.cluster.routing;
 |  |  package org.elasticsearch.cluster.routing;
 | 
											
												
													
														|  |  
 |  |  
 | 
											
												
													
														|  |  import org.elasticsearch.cluster.*;
 |  |  import org.elasticsearch.cluster.*;
 | 
											
												
													
														|  | -import org.elasticsearch.cluster.node.DiscoveryNode;
 |  | 
 | 
											
												
													
														|  |  import org.elasticsearch.cluster.routing.allocation.AllocationService;
 |  |  import org.elasticsearch.cluster.routing.allocation.AllocationService;
 | 
											
												
													
														|  |  import org.elasticsearch.cluster.routing.allocation.RoutingAllocation;
 |  |  import org.elasticsearch.cluster.routing.allocation.RoutingAllocation;
 | 
											
												
													
														|  |  import org.elasticsearch.common.Priority;
 |  |  import org.elasticsearch.common.Priority;
 | 
											
										
											
												
													
														|  | @@ -32,12 +31,9 @@ import org.elasticsearch.common.util.concurrent.AbstractRunnable;
 | 
											
												
													
														|  |  import org.elasticsearch.common.util.concurrent.FutureUtils;
 |  |  import org.elasticsearch.common.util.concurrent.FutureUtils;
 | 
											
												
													
														|  |  import org.elasticsearch.threadpool.ThreadPool;
 |  |  import org.elasticsearch.threadpool.ThreadPool;
 | 
											
												
													
														|  |  
 |  |  
 | 
											
												
													
														|  | -import java.util.concurrent.Future;
 |  | 
 | 
											
												
													
														|  |  import java.util.concurrent.ScheduledFuture;
 |  |  import java.util.concurrent.ScheduledFuture;
 | 
											
												
													
														|  |  import java.util.concurrent.atomic.AtomicBoolean;
 |  |  import java.util.concurrent.atomic.AtomicBoolean;
 | 
											
												
													
														|  |  
 |  |  
 | 
											
												
													
														|  | -import static org.elasticsearch.common.unit.TimeValue.timeValueSeconds;
 |  | 
 | 
											
												
													
														|  | -
 |  | 
 | 
											
												
													
														|  |  /**
 |  |  /**
 | 
											
												
													
														|  |   * A {@link RoutingService} listens to clusters state. When this service
 |  |   * A {@link RoutingService} listens to clusters state. When this service
 | 
											
												
													
														|  |   * receives a {@link ClusterChangedEvent} the cluster state will be verified and
 |  |   * receives a {@link ClusterChangedEvent} the cluster state will be verified and
 | 
											
										
											
												
													
														|  | @@ -52,21 +48,13 @@ import static org.elasticsearch.common.unit.TimeValue.timeValueSeconds;
 | 
											
												
													
														|  |   */
 |  |   */
 | 
											
												
													
														|  |  public class RoutingService extends AbstractLifecycleComponent<RoutingService> implements ClusterStateListener {
 |  |  public class RoutingService extends AbstractLifecycleComponent<RoutingService> implements ClusterStateListener {
 | 
											
												
													
														|  |  
 |  |  
 | 
											
												
													
														|  | -    private static final String CLUSTER_UPDATE_TASK_SOURCE = "routing-table-updater";
 |  | 
 | 
											
												
													
														|  | -
 |  | 
 | 
											
												
													
														|  | -    private final ThreadPool threadPool;
 |  | 
 | 
											
												
													
														|  | 
 |  | +    private static final String CLUSTER_UPDATE_TASK_SOURCE = "cluster_reroute";
 | 
											
												
													
														|  |  
 |  |  
 | 
											
												
													
														|  | 
 |  | +    final ThreadPool threadPool;
 | 
											
												
													
														|  |      private final ClusterService clusterService;
 |  |      private final ClusterService clusterService;
 | 
											
												
													
														|  | -
 |  | 
 | 
											
												
													
														|  |      private final AllocationService allocationService;
 |  |      private final AllocationService allocationService;
 | 
											
												
													
														|  |  
 |  |  
 | 
											
												
													
														|  | -    private final TimeValue schedule;
 |  | 
 | 
											
												
													
														|  | -
 |  | 
 | 
											
												
													
														|  | -    private volatile boolean routingTableDirty = false;
 |  | 
 | 
											
												
													
														|  | -
 |  | 
 | 
											
												
													
														|  | -    private volatile Future scheduledRoutingTableFuture;
 |  | 
 | 
											
												
													
														|  |      private AtomicBoolean rerouting = new AtomicBoolean();
 |  |      private AtomicBoolean rerouting = new AtomicBoolean();
 | 
											
												
													
														|  | -
 |  | 
 | 
											
												
													
														|  |      private volatile long registeredNextDelaySetting = Long.MAX_VALUE;
 |  |      private volatile long registeredNextDelaySetting = Long.MAX_VALUE;
 | 
											
												
													
														|  |      private volatile ScheduledFuture registeredNextDelayFuture;
 |  |      private volatile ScheduledFuture registeredNextDelayFuture;
 | 
											
												
													
														|  |  
 |  |  
 | 
											
										
											
												
													
														|  | @@ -76,8 +64,9 @@ public class RoutingService extends AbstractLifecycleComponent<RoutingService> i
 | 
											
												
													
														|  |          this.threadPool = threadPool;
 |  |          this.threadPool = threadPool;
 | 
											
												
													
														|  |          this.clusterService = clusterService;
 |  |          this.clusterService = clusterService;
 | 
											
												
													
														|  |          this.allocationService = allocationService;
 |  |          this.allocationService = allocationService;
 | 
											
												
													
														|  | -        this.schedule = settings.getAsTime("cluster.routing.schedule", timeValueSeconds(10));
 |  | 
 | 
											
												
													
														|  | -        clusterService.addFirst(this);
 |  | 
 | 
											
												
													
														|  | 
 |  | +        if (clusterService != null) {
 | 
											
												
													
														|  | 
 |  | +            clusterService.addFirst(this);
 | 
											
												
													
														|  | 
 |  | +        }
 | 
											
												
													
														|  |      }
 |  |      }
 | 
											
												
													
														|  |  
 |  |  
 | 
											
												
													
														|  |      @Override
 |  |      @Override
 | 
											
										
											
												
													
														|  | @@ -90,52 +79,27 @@ public class RoutingService extends AbstractLifecycleComponent<RoutingService> i
 | 
											
												
													
														|  |  
 |  |  
 | 
											
												
													
														|  |      @Override
 |  |      @Override
 | 
											
												
													
														|  |      protected void doClose() {
 |  |      protected void doClose() {
 | 
											
												
													
														|  | -        FutureUtils.cancel(scheduledRoutingTableFuture);
 |  | 
 | 
											
												
													
														|  | -        scheduledRoutingTableFuture = null;
 |  | 
 | 
											
												
													
														|  | 
 |  | +        FutureUtils.cancel(registeredNextDelayFuture);
 | 
											
												
													
														|  |          clusterService.remove(this);
 |  |          clusterService.remove(this);
 | 
											
												
													
														|  |      }
 |  |      }
 | 
											
												
													
														|  |  
 |  |  
 | 
											
												
													
														|  | -    /** make sure that a reroute will be done by the next scheduled check */
 |  | 
 | 
											
												
													
														|  | -    public void scheduleReroute() {
 |  | 
 | 
											
												
													
														|  | -        routingTableDirty = true;
 |  | 
 | 
											
												
													
														|  | 
 |  | +    /**
 | 
											
												
													
														|  | 
 |  | +     * Initiates a reroute.
 | 
											
												
													
														|  | 
 |  | +     */
 | 
											
												
													
														|  | 
 |  | +    public final void reroute(String reason) {
 | 
											
												
													
														|  | 
 |  | +        performReroute(reason);
 | 
											
												
													
														|  |      }
 |  |      }
 | 
											
												
													
														|  |  
 |  |  
 | 
											
												
													
														|  |      @Override
 |  |      @Override
 | 
											
												
													
														|  |      public void clusterChanged(ClusterChangedEvent event) {
 |  |      public void clusterChanged(ClusterChangedEvent event) {
 | 
											
												
													
														|  | -        if (event.source().equals(CLUSTER_UPDATE_TASK_SOURCE)) {
 |  | 
 | 
											
												
													
														|  | 
 |  | +        if (event.source().startsWith(CLUSTER_UPDATE_TASK_SOURCE)) {
 | 
											
												
													
														|  |              // that's us, ignore this event
 |  |              // that's us, ignore this event
 | 
											
												
													
														|  |              return;
 |  |              return;
 | 
											
												
													
														|  |          }
 |  |          }
 | 
											
												
													
														|  |          if (event.state().nodes().localNodeMaster()) {
 |  |          if (event.state().nodes().localNodeMaster()) {
 | 
											
												
													
														|  | -            // we are master, schedule the routing table updater
 |  | 
 | 
											
												
													
														|  | -            if (scheduledRoutingTableFuture == null) {
 |  | 
 | 
											
												
													
														|  | -                // a new master (us), make sure we reroute shards
 |  | 
 | 
											
												
													
														|  | -                routingTableDirty = true;
 |  | 
 | 
											
												
													
														|  | -                scheduledRoutingTableFuture = threadPool.scheduleWithFixedDelay(new RoutingTableUpdater(), schedule);
 |  | 
 | 
											
												
													
														|  | -            }
 |  | 
 | 
											
												
													
														|  | -            if (event.nodesRemoved()) {
 |  | 
 | 
											
												
													
														|  | -                // if nodes were removed, we don't want to wait for the scheduled task
 |  | 
 | 
											
												
													
														|  | -                // since we want to get primary election as fast as possible
 |  | 
 | 
											
												
													
														|  | -                routingTableDirty = true;
 |  | 
 | 
											
												
													
														|  | -                reroute();
 |  | 
 | 
											
												
													
														|  | -                // Commented out since we make sure to reroute whenever shards changes state or metadata changes state
 |  | 
 | 
											
												
													
														|  | -//            } else if (event.routingTableChanged()) {
 |  | 
 | 
											
												
													
														|  | -//                routingTableDirty = true;
 |  | 
 | 
											
												
													
														|  | -//                reroute();
 |  | 
 | 
											
												
													
														|  | -            } else {
 |  | 
 | 
											
												
													
														|  | -                if (event.nodesAdded()) {
 |  | 
 | 
											
												
													
														|  | -                    for (DiscoveryNode node : event.nodesDelta().addedNodes()) {
 |  | 
 | 
											
												
													
														|  | -                        if (node.dataNode()) {
 |  | 
 | 
											
												
													
														|  | -                            routingTableDirty = true;
 |  | 
 | 
											
												
													
														|  | -                            break;
 |  | 
 | 
											
												
													
														|  | -                        }
 |  | 
 | 
											
												
													
														|  | -                    }
 |  | 
 | 
											
												
													
														|  | -                }
 |  | 
 | 
											
												
													
														|  | -            }
 |  | 
 | 
											
												
													
														|  | -
 |  | 
 | 
											
												
													
														|  | -             // figure out when the next unassigned allocation need to happen from now. If this is larger or equal
 |  | 
 | 
											
												
													
														|  | -             // then the last time we checked and scheduled, we are guaranteed to have a reroute until then, so no need
 |  | 
 | 
											
												
													
														|  | -             // to schedule again
 |  | 
 | 
											
												
													
														|  | 
 |  | +            // figure out when the next unassigned allocation need to happen from now. If this is larger or equal
 | 
											
												
													
														|  | 
 |  | +            // then the last time we checked and scheduled, we are guaranteed to have a reroute until then, so no need
 | 
											
												
													
														|  | 
 |  | +            // to schedule again
 | 
											
												
													
														|  |              long nextDelaySetting = UnassignedInfo.findSmallestDelayedAllocationSetting(settings, event.state());
 |  |              long nextDelaySetting = UnassignedInfo.findSmallestDelayedAllocationSetting(settings, event.state());
 | 
											
												
													
														|  |              if (nextDelaySetting > 0 && nextDelaySetting < registeredNextDelaySetting) {
 |  |              if (nextDelaySetting > 0 && nextDelaySetting < registeredNextDelaySetting) {
 | 
											
												
													
														|  |                  FutureUtils.cancel(registeredNextDelayFuture);
 |  |                  FutureUtils.cancel(registeredNextDelayFuture);
 | 
											
										
											
												
													
														|  | @@ -145,9 +109,8 @@ public class RoutingService extends AbstractLifecycleComponent<RoutingService> i
 | 
											
												
													
														|  |                  registeredNextDelayFuture = threadPool.schedule(nextDelay, ThreadPool.Names.SAME, new AbstractRunnable() {
 |  |                  registeredNextDelayFuture = threadPool.schedule(nextDelay, ThreadPool.Names.SAME, new AbstractRunnable() {
 | 
											
												
													
														|  |                      @Override
 |  |                      @Override
 | 
											
												
													
														|  |                      protected void doRun() throws Exception {
 |  |                      protected void doRun() throws Exception {
 | 
											
												
													
														|  | -                        routingTableDirty = true;
 |  | 
 | 
											
												
													
														|  |                          registeredNextDelaySetting = Long.MAX_VALUE;
 |  |                          registeredNextDelaySetting = Long.MAX_VALUE;
 | 
											
												
													
														|  | -                        reroute();
 |  | 
 | 
											
												
													
														|  | 
 |  | +                        reroute("assign delayed unassigned shards");
 | 
											
												
													
														|  |                      }
 |  |                      }
 | 
											
												
													
														|  |  
 |  |  
 | 
											
												
													
														|  |                      @Override
 |  |                      @Override
 | 
											
										
											
												
													
														|  | @@ -158,25 +121,26 @@ public class RoutingService extends AbstractLifecycleComponent<RoutingService> i
 | 
											
												
													
														|  |              } else {
 |  |              } else {
 | 
											
												
													
														|  |                  logger.trace("no need to schedule reroute due to delayed unassigned, next_delay_setting [{}], registered [{}]", nextDelaySetting, registeredNextDelaySetting);
 |  |                  logger.trace("no need to schedule reroute due to delayed unassigned, next_delay_setting [{}], registered [{}]", nextDelaySetting, registeredNextDelaySetting);
 | 
											
												
													
														|  |              }
 |  |              }
 | 
											
												
													
														|  | -        } else {
 |  | 
 | 
											
												
													
														|  | -            FutureUtils.cancel(scheduledRoutingTableFuture);
 |  | 
 | 
											
												
													
														|  | -            scheduledRoutingTableFuture = null;
 |  | 
 | 
											
												
													
														|  |          }
 |  |          }
 | 
											
												
													
														|  |      }
 |  |      }
 | 
											
												
													
														|  |  
 |  |  
 | 
											
												
													
														|  | -    private void reroute() {
 |  | 
 | 
											
												
													
														|  | 
 |  | +    // visible for testing
 | 
											
												
													
														|  | 
 |  | +    long getRegisteredNextDelaySetting() {
 | 
											
												
													
														|  | 
 |  | +        return this.registeredNextDelaySetting;
 | 
											
												
													
														|  | 
 |  | +    }
 | 
											
												
													
														|  | 
 |  | +
 | 
											
												
													
														|  | 
 |  | +    // visible for testing
 | 
											
												
													
														|  | 
 |  | +    void performReroute(String reason) {
 | 
											
												
													
														|  |          try {
 |  |          try {
 | 
											
												
													
														|  | -            if (!routingTableDirty) {
 |  | 
 | 
											
												
													
														|  | -                return;
 |  | 
 | 
											
												
													
														|  | -            }
 |  | 
 | 
											
												
													
														|  |              if (lifecycle.stopped()) {
 |  |              if (lifecycle.stopped()) {
 | 
											
												
													
														|  |                  return;
 |  |                  return;
 | 
											
												
													
														|  |              }
 |  |              }
 | 
											
												
													
														|  |              if (rerouting.compareAndSet(false, true) == false) {
 |  |              if (rerouting.compareAndSet(false, true) == false) {
 | 
											
												
													
														|  | -                logger.trace("already has pending reroute, ignoring");
 |  | 
 | 
											
												
													
														|  | 
 |  | +                logger.trace("already has pending reroute, ignoring {}", reason);
 | 
											
												
													
														|  |                  return;
 |  |                  return;
 | 
											
												
													
														|  |              }
 |  |              }
 | 
											
												
													
														|  | -            clusterService.submitStateUpdateTask(CLUSTER_UPDATE_TASK_SOURCE, Priority.HIGH, new ClusterStateUpdateTask() {
 |  | 
 | 
											
												
													
														|  | 
 |  | +            logger.trace("rerouting {}", reason);
 | 
											
												
													
														|  | 
 |  | +            clusterService.submitStateUpdateTask(CLUSTER_UPDATE_TASK_SOURCE + "(" + reason + ")", Priority.HIGH, new ClusterStateUpdateTask() {
 | 
											
												
													
														|  |                  @Override
 |  |                  @Override
 | 
											
												
													
														|  |                  public ClusterState execute(ClusterState currentState) {
 |  |                  public ClusterState execute(ClusterState currentState) {
 | 
											
												
													
														|  |                      rerouting.set(false);
 |  |                      rerouting.set(false);
 | 
											
										
											
												
													
														|  | @@ -205,19 +169,10 @@ public class RoutingService extends AbstractLifecycleComponent<RoutingService> i
 | 
											
												
													
														|  |                      }
 |  |                      }
 | 
											
												
													
														|  |                  }
 |  |                  }
 | 
											
												
													
														|  |              });
 |  |              });
 | 
											
												
													
														|  | -            routingTableDirty = false;
 |  | 
 | 
											
												
													
														|  |          } catch (Throwable e) {
 |  |          } catch (Throwable e) {
 | 
											
												
													
														|  |              rerouting.set(false);
 |  |              rerouting.set(false);
 | 
											
												
													
														|  |              ClusterState state = clusterService.state();
 |  |              ClusterState state = clusterService.state();
 | 
											
												
													
														|  | -            logger.warn("Failed to reroute routing table, current state:\n{}", e, state.prettyPrint());
 |  | 
 | 
											
												
													
														|  | -        }
 |  | 
 | 
											
												
													
														|  | -    }
 |  | 
 | 
											
												
													
														|  | -
 |  | 
 | 
											
												
													
														|  | -    private class RoutingTableUpdater implements Runnable {
 |  | 
 | 
											
												
													
														|  | -
 |  | 
 | 
											
												
													
														|  | -        @Override
 |  | 
 | 
											
												
													
														|  | -        public void run() {
 |  | 
 | 
											
												
													
														|  | -            reroute();
 |  | 
 | 
											
												
													
														|  | 
 |  | +            logger.warn("failed to reroute routing table, current state:\n{}", e, state.prettyPrint());
 | 
											
												
													
														|  |          }
 |  |          }
 | 
											
												
													
														|  |      }
 |  |      }
 | 
											
												
													
														|  |  }
 |  |  }
 |