@Inject
  public DiskThresholdDecider(
      Settings settings,
      NodeSettingsService nodeSettingsService,
      ClusterInfoService infoService,
      Client client) {
    super(settings);
    String lowWatermark = settings.get(CLUSTER_ROUTING_ALLOCATION_LOW_DISK_WATERMARK, "85%");
    String highWatermark = settings.get(CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK, "90%");

    if (!validWatermarkSetting(lowWatermark, CLUSTER_ROUTING_ALLOCATION_LOW_DISK_WATERMARK)) {
      throw new ElasticsearchParseException("unable to parse low watermark [{}]", lowWatermark);
    }
    if (!validWatermarkSetting(highWatermark, CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK)) {
      throw new ElasticsearchParseException("unable to parse high watermark [{}]", highWatermark);
    }
    // Watermark is expressed in terms of used data, but we need "free" data watermark
    this.freeDiskThresholdLow = 100.0 - thresholdPercentageFromWatermark(lowWatermark);
    this.freeDiskThresholdHigh = 100.0 - thresholdPercentageFromWatermark(highWatermark);

    this.freeBytesThresholdLow =
        thresholdBytesFromWatermark(lowWatermark, CLUSTER_ROUTING_ALLOCATION_LOW_DISK_WATERMARK);
    this.freeBytesThresholdHigh =
        thresholdBytesFromWatermark(highWatermark, CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK);
    this.includeRelocations =
        settings.getAsBoolean(CLUSTER_ROUTING_ALLOCATION_INCLUDE_RELOCATIONS, true);
    this.rerouteInterval =
        settings.getAsTime(
            CLUSTER_ROUTING_ALLOCATION_REROUTE_INTERVAL, TimeValue.timeValueSeconds(60));

    this.enabled = settings.getAsBoolean(CLUSTER_ROUTING_ALLOCATION_DISK_THRESHOLD_ENABLED, true);
    nodeSettingsService.addListener(new ApplySettings());
    infoService.addListener(new DiskListener(client));
  }
Exemplo n.º 2
0
 public RoutingAllocation.Result applyStartedShards(
     ClusterState clusterState, List<? extends ShardRouting> startedShards, boolean withReroute) {
   RoutingNodes routingNodes = clusterState.routingNodes();
   // shuffle the unassigned nodes, just so we won't have things like poison failed shards
   routingNodes.unassigned().shuffle();
   StartedRerouteAllocation allocation =
       new StartedRerouteAllocation(
           allocationDeciders,
           routingNodes,
           clusterState.nodes(),
           startedShards,
           clusterInfoService.getClusterInfo());
   boolean changed = applyStartedShards(routingNodes, startedShards);
   if (!changed) {
     return new RoutingAllocation.Result(false, clusterState.routingTable());
   }
   shardsAllocators.applyStartedShards(allocation);
   if (withReroute) {
     reroute(allocation);
   }
   return new RoutingAllocation.Result(
       true,
       new RoutingTable.Builder()
           .updateNodes(routingNodes)
           .build()
           .validateRaiseException(clusterState.metaData()));
 }
Exemplo n.º 3
0
  /**
   * Only handles reroute but *without* any reassignment of unassigned shards or rebalancing. Does
   * make sure to handle removed nodes, but only moved the shards to UNASSIGNED, does not reassign
   * them.
   */
  public RoutingAllocation.Result rerouteWithNoReassign(ClusterState clusterState, boolean debug) {
    RoutingNodes routingNodes = clusterState.routingNodes();
    // shuffle the unassigned nodes, just so we won't have things like poison failed shards
    routingNodes.unassigned().shuffle();
    RoutingAllocation allocation =
        new RoutingAllocation(
            allocationDeciders,
            routingNodes,
            clusterState.nodes(),
            clusterInfoService.getClusterInfo());
    allocation.debugDecision(debug);
    boolean changed = false;
    // first, clear from the shards any node id they used to belong to that is now dead
    changed |= deassociateDeadNodes(allocation);

    // create a sorted list of from nodes with least number of shards to the maximum ones
    applyNewNodes(allocation);

    // elect primaries *before* allocating unassigned, so backups of primaries that failed
    // will be moved to primary state and not wait for primaries to be allocated and recovered
    // (*from gateway*)
    changed |= electPrimariesAndUnassignedDanglingReplicas(allocation);

    if (!changed) {
      return new RoutingAllocation.Result(false, clusterState.routingTable());
    }
    return new RoutingAllocation.Result(
        true,
        new RoutingTable.Builder()
            .updateNodes(routingNodes)
            .build()
            .validateRaiseException(clusterState.metaData()));
  }
Exemplo n.º 4
0
 public RoutingAllocation.Result reroute(
     ClusterState clusterState, AllocationCommands commands, boolean explain)
     throws ElasticsearchException {
   RoutingNodes routingNodes = clusterState.routingNodes();
   // we don't shuffle the unassigned shards here, to try and get as close as possible to
   // a consistent result of the effect the commands have on the routing
   // this allows systems to dry run the commands, see the resulting cluster state, and act on it
   RoutingAllocation allocation =
       new RoutingAllocation(
           allocationDeciders,
           routingNodes,
           clusterState.nodes(),
           clusterInfoService.getClusterInfo());
   // don't short circuit deciders, we want a full explanation
   allocation.debugDecision(true);
   // we ignore disable allocation, because commands are explicit
   allocation.ignoreDisable(true);
   RoutingExplanations explanations = commands.execute(allocation, explain);
   // we revert the ignore disable flag, since when rerouting, we want the original setting to take
   // place
   allocation.ignoreDisable(false);
   // the assumption is that commands will move / act on shards (or fail through exceptions)
   // so, there will always be shard "movements", so no need to check on reroute
   reroute(allocation);
   return new RoutingAllocation.Result(
       true,
       new RoutingTable.Builder()
           .updateNodes(routingNodes)
           .build()
           .validateRaiseException(clusterState.metaData()),
       explanations);
 }
Exemplo n.º 5
0
 /**
  * Applies the failed shards. Note, shards can be called several times within this method.
  *
  * <p>
  *
  * <p>If the same instance of the routing table is returned, then no change has been made.
  */
 public RoutingAllocation.Result applyFailedShards(
     ClusterState clusterState, List<ShardRouting> failedShards) {
   RoutingNodes routingNodes = clusterState.routingNodes();
   // shuffle the unassigned nodes, just so we won't have things like poison failed shards
   routingNodes.unassigned().shuffle();
   FailedRerouteAllocation allocation =
       new FailedRerouteAllocation(
           allocationDeciders,
           routingNodes,
           clusterState.nodes(),
           failedShards,
           clusterInfoService.getClusterInfo());
   boolean changed = false;
   for (ShardRouting failedShard : failedShards) {
     changed |= applyFailedShard(allocation, failedShard, true);
   }
   if (!changed) {
     return new RoutingAllocation.Result(false, clusterState.routingTable());
   }
   shardsAllocators.applyFailedShards(allocation);
   reroute(allocation);
   return new RoutingAllocation.Result(
       true,
       new RoutingTable.Builder()
           .updateNodes(routingNodes)
           .build()
           .validateRaiseException(clusterState.metaData()));
 }
Exemplo n.º 6
0
 /**
  * Reroutes the routing table based on the live nodes.
  *
  * <p>
  *
  * <p>If the same instance of the routing table is returned, then no change has been made.
  */
 public RoutingAllocation.Result reroute(ClusterState clusterState, boolean debug) {
   RoutingNodes routingNodes = clusterState.routingNodes();
   // shuffle the unassigned nodes, just so we won't have things like poison failed shards
   routingNodes.unassigned().shuffle();
   RoutingAllocation allocation =
       new RoutingAllocation(
           allocationDeciders,
           routingNodes,
           clusterState.nodes(),
           clusterInfoService.getClusterInfo());
   allocation.debugDecision(debug);
   if (!reroute(allocation)) {
     return new RoutingAllocation.Result(false, clusterState.routingTable());
   }
   return new RoutingAllocation.Result(
       true,
       new RoutingTable.Builder()
           .updateNodes(routingNodes)
           .build()
           .validateRaiseException(clusterState.metaData()));
 }