Prepares allocator decision objects for use with the allocation explain API (#21691)

This commit enhances the allocator decision result objects (namely,
AllocateUnassignedDecision, MoveDecision, and RebalanceDecision)
to enable them to be used directly by the cluster allocation explain API. In
particular, this commit does the following:

- Adds serialization and toXContent methods to the response objects,
which will form the explain API responses.
- Moves the calculation of the final explanation to the response
object itself, removing it from the responsibility of the allocators.
- Adds shard store information to the NodeAllocationResult, so that
store information is available for each node, when explaining a
shard allocation by the PrimaryShardAllocator or the ReplicaShardAllocator.
- Removes RebalanceDecision in favor of using MoveDecision for both
moving and rebalancing shards.
- Removes NodeRebalanceResult in favor of using NodeAllocationResult.
- Changes the notion of weight ranking to be relative to the current node,
instead of an absolute weight that doesn't convey any added value to the
API user and can be confusing.
- Introduces a new enum AllocationDecision to convey the decision type,
which enables conveying unassigned, moving, and rebalancing scenarios
with more detail as opposed to just Decision.Type and AllocationStatus.
This commit is contained in:
Ali Beyad 2016-12-07 17:37:51 -05:00 committed by GitHub
parent 9fbfe540d5
commit e6e7bab58c
28 changed files with 1826 additions and 876 deletions

View File

@ -73,7 +73,7 @@ public class NodeExplanation implements Writeable, ToXContent {
@Override
public void writeTo(StreamOutput out) throws IOException {
node.writeTo(out);
Decision.writeTo(nodeDecision, out);
nodeDecision.writeTo(out);
out.writeFloat(nodeWeight);
if (storeStatus == null) {
out.writeBoolean(false);
@ -108,7 +108,9 @@ public class NodeExplanation implements Writeable, ToXContent {
builder.field("final_decision", finalDecision.toString());
builder.field("final_explanation", finalExplanation);
builder.field("weight", nodeWeight);
builder.startArray("decisions");
nodeDecision.toXContent(builder, params);
builder.endArray();
}
builder.endObject(); // end node <uuid>
return builder;

View File

@ -0,0 +1,169 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.cluster.routing.allocation;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.routing.allocation.decider.Decision.Type;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.io.stream.Writeable;
import org.elasticsearch.common.xcontent.ToXContent;
import org.elasticsearch.common.xcontent.XContentBuilder;
import java.io.IOException;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
/**
* An abstract class for representing various types of allocation decisions.
*/
public abstract class AbstractAllocationDecision implements ToXContent, Writeable {
@Nullable
protected final DiscoveryNode targetNode;
@Nullable
protected final List<NodeAllocationResult> nodeDecisions;
protected AbstractAllocationDecision(@Nullable DiscoveryNode targetNode, @Nullable List<NodeAllocationResult> nodeDecisions) {
this.targetNode = targetNode;
this.nodeDecisions = nodeDecisions != null ? sortNodeDecisions(nodeDecisions) : null;
}
protected AbstractAllocationDecision(StreamInput in) throws IOException {
targetNode = in.readOptionalWriteable(DiscoveryNode::new);
nodeDecisions = in.readBoolean() ? Collections.unmodifiableList(in.readList(NodeAllocationResult::new)) : null;
}
/**
* Returns {@code true} if a decision was taken by the allocator, {@code false} otherwise.
* If no decision was taken, then the rest of the fields in this object cannot be accessed and will
* throw an {@code IllegalStateException}.
*/
public abstract boolean isDecisionTaken();
/**
* Get the node that the allocator will assign the shard to, returning {@code null} if there is no node to
* which the shard will be assigned or moved. If {@link #isDecisionTaken()} returns {@code false}, then
* invoking this method will throw an {@code IllegalStateException}.
*/
@Nullable
public DiscoveryNode getTargetNode() {
checkDecisionState();
return targetNode;
}
/**
* Gets the sorted list of individual node-level decisions that went into making the ultimate decision whether
* to allocate or move the shard. If {@link #isDecisionTaken()} returns {@code false}, then
* invoking this method will throw an {@code IllegalStateException}.
*/
@Nullable
public List<NodeAllocationResult> getNodeDecisions() {
checkDecisionState();
return nodeDecisions;
}
/**
* Gets the explanation for the decision. If {@link #isDecisionTaken()} returns {@code false}, then invoking
* this method will throw an {@code IllegalStateException}.
*/
public abstract String getExplanation();
@Override
public void writeTo(StreamOutput out) throws IOException {
out.writeOptionalWriteable(targetNode);
if (nodeDecisions != null) {
out.writeBoolean(true);
out.writeList(nodeDecisions);
} else {
out.writeBoolean(false);
}
}
protected void checkDecisionState() {
if (isDecisionTaken() == false) {
throw new IllegalStateException("decision was not taken, individual object fields cannot be accessed");
}
}
/**
* Generates X-Content for a {@link DiscoveryNode} that leaves off some of the non-critical fields.
*/
public static XContentBuilder discoveryNodeToXContent(DiscoveryNode node, boolean outerObjectWritten, XContentBuilder builder)
throws IOException {
builder.field(outerObjectWritten ? "id" : "node_id", node.getId());
builder.field(outerObjectWritten ? "name" : "node_name", node.getName());
builder.field("transport_address", node.getAddress().toString());
if (node.getAttributes().isEmpty() == false) {
builder.startObject(outerObjectWritten ? "attributes" : "node_attributes");
for (Map.Entry<String, String> entry : node.getAttributes().entrySet()) {
builder.field(entry.getKey(), entry.getValue());
}
builder.endObject();
}
return builder;
}
/**
* Sorts a list of node level decisions by the decision type, then by weight ranking, and finally by node id.
*/
public List<NodeAllocationResult> sortNodeDecisions(List<NodeAllocationResult> nodeDecisions) {
return Collections.unmodifiableList(nodeDecisions.stream().sorted().collect(Collectors.toList()));
}
/**
* Generates X-Content for the node-level decisions, creating the outer "node_decisions" object
* in which they are serialized.
*/
public XContentBuilder nodeDecisionsToXContent(List<NodeAllocationResult> nodeDecisions, XContentBuilder builder, Params params)
throws IOException {
if (nodeDecisions != null) {
builder.startArray("node_allocation_decisions");
{
for (NodeAllocationResult explanation : nodeDecisions) {
explanation.toXContent(builder, params);
}
}
builder.endArray();
}
return builder;
}
/**
* Returns {@code true} if there is at least one node that returned a {@link Type#YES} decision for allocating this shard.
*/
protected boolean atLeastOneNodeWithYesDecision() {
if (nodeDecisions == null) {
return false;
}
for (NodeAllocationResult result : nodeDecisions) {
if (result.getNodeDecision() == AllocationDecision.YES) {
return true;
}
}
return false;
}
}

View File

@ -19,23 +19,30 @@
package org.elasticsearch.cluster.routing.allocation;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.routing.UnassignedInfo.AllocationStatus;
import org.elasticsearch.cluster.routing.allocation.decider.Decision;
import org.elasticsearch.cluster.routing.allocation.decider.Decision.Type;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.xcontent.XContentBuilder;
import java.io.IOException;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Objects;
/**
* Represents the allocation decision by an allocator for an unassigned shard.
*/
public class AllocateUnassignedDecision {
public class AllocateUnassignedDecision extends AbstractAllocationDecision {
/** a constant representing a shard decision where no decision was taken */
public static final AllocateUnassignedDecision NOT_TAKEN =
new AllocateUnassignedDecision(null, null, null, null, null, null, null);
new AllocateUnassignedDecision(AllocationStatus.NO_ATTEMPT, null, null, null, false, 0L, 0L);
/**
* a map of cached common no/throttle decisions that don't need explanations,
* this helps prevent unnecessary object allocations for the non-explain API case
@ -44,132 +51,120 @@ public class AllocateUnassignedDecision {
static {
Map<AllocationStatus, AllocateUnassignedDecision> cachedDecisions = new HashMap<>();
cachedDecisions.put(AllocationStatus.FETCHING_SHARD_DATA,
new AllocateUnassignedDecision(Type.NO, AllocationStatus.FETCHING_SHARD_DATA, null, null, null, null, null));
new AllocateUnassignedDecision(AllocationStatus.FETCHING_SHARD_DATA, null, null, null, false, 0L, 0L));
cachedDecisions.put(AllocationStatus.NO_VALID_SHARD_COPY,
new AllocateUnassignedDecision(Type.NO, AllocationStatus.NO_VALID_SHARD_COPY, null, null, null, null, null));
new AllocateUnassignedDecision(AllocationStatus.NO_VALID_SHARD_COPY, null, null, null, false, 0L, 0L));
cachedDecisions.put(AllocationStatus.DECIDERS_NO,
new AllocateUnassignedDecision(Type.NO, AllocationStatus.DECIDERS_NO, null, null, null, null, null));
new AllocateUnassignedDecision(AllocationStatus.DECIDERS_NO, null, null, null, false, 0L, 0L));
cachedDecisions.put(AllocationStatus.DECIDERS_THROTTLED,
new AllocateUnassignedDecision(Type.THROTTLE, AllocationStatus.DECIDERS_THROTTLED, null, null, null, null, null));
new AllocateUnassignedDecision(AllocationStatus.DECIDERS_THROTTLED, null, null, null, false, 0L, 0L));
cachedDecisions.put(AllocationStatus.DELAYED_ALLOCATION,
new AllocateUnassignedDecision(Type.NO, AllocationStatus.DELAYED_ALLOCATION, null, null, null, null, null));
new AllocateUnassignedDecision(AllocationStatus.DELAYED_ALLOCATION, null, null, null, false, 0L, 0L));
CACHED_DECISIONS = Collections.unmodifiableMap(cachedDecisions);
}
@Nullable
private final Type finalDecision;
@Nullable
private final AllocationStatus allocationStatus;
@Nullable
private final String finalExplanation;
@Nullable
private final String assignedNodeId;
@Nullable
private final String allocationId;
@Nullable
private final Map<String, NodeAllocationResult> nodeDecisions;
@Nullable
private final Decision shardDecision;
private final boolean reuseStore;
private final long remainingDelayInMillis;
private final long configuredDelayInMillis;
private AllocateUnassignedDecision(Type finalDecision,
AllocationStatus allocationStatus,
String finalExplanation,
String assignedNodeId,
private AllocateUnassignedDecision(AllocationStatus allocationStatus,
DiscoveryNode assignedNode,
String allocationId,
Map<String, NodeAllocationResult> nodeDecisions,
Decision shardDecision) {
assert assignedNodeId != null || finalDecision == null || finalDecision != Type.YES :
List<NodeAllocationResult> nodeDecisions,
boolean reuseStore,
long remainingDelayInMillis,
long configuredDelayInMillis) {
super(assignedNode, nodeDecisions);
assert assignedNode != null || allocationStatus != null :
"a yes decision must have a node to assign the shard to";
assert allocationStatus != null || finalDecision == null || finalDecision == Type.YES :
"only a yes decision should not have an allocation status";
assert allocationId == null || assignedNodeId != null :
assert allocationId == null || assignedNode != null :
"allocation id can only be null if the assigned node is null";
this.finalDecision = finalDecision;
this.allocationStatus = allocationStatus;
this.finalExplanation = finalExplanation;
this.assignedNodeId = assignedNodeId;
this.allocationId = allocationId;
this.nodeDecisions = nodeDecisions != null ? Collections.unmodifiableMap(nodeDecisions) : null;
this.shardDecision = shardDecision;
this.reuseStore = reuseStore;
this.remainingDelayInMillis = remainingDelayInMillis;
this.configuredDelayInMillis = configuredDelayInMillis;
}
public AllocateUnassignedDecision(StreamInput in) throws IOException {
super(in);
allocationStatus = in.readOptionalWriteable(AllocationStatus::readFrom);
allocationId = in.readOptionalString();
reuseStore = in.readBoolean();
remainingDelayInMillis = in.readVLong();
configuredDelayInMillis = in.readVLong();
}
/**
* Returns a NO decision with the given shard-level decision and explanation (if in explain mode).
* Returns a NO decision with the given {@link AllocationStatus}, and the individual node-level
* decisions that comprised the final NO decision if in explain mode.
*/
public static AllocateUnassignedDecision no(Decision shardDecision, @Nullable String explanation) {
if (explanation != null) {
return new AllocateUnassignedDecision(Type.NO, AllocationStatus.DECIDERS_NO, explanation, null, null, null, shardDecision);
} else {
return getCachedDecision(AllocationStatus.DECIDERS_NO);
}
public static AllocateUnassignedDecision no(AllocationStatus allocationStatus, @Nullable List<NodeAllocationResult> decisions) {
return no(allocationStatus, decisions, false);
}
/**
* Returns a NO decision with the given {@link AllocationStatus} and explanation for the NO decision, if in explain mode.
* Returns a NO decision for a delayed shard allocation on a replica shard, with the individual node-level
* decisions that comprised the final NO decision, if in explain mode. Instances created with this
* method will return {@link AllocationStatus#DELAYED_ALLOCATION} for {@link #getAllocationStatus()}.
*/
public static AllocateUnassignedDecision no(AllocationStatus allocationStatus, @Nullable String explanation) {
return no(allocationStatus, explanation, null);
public static AllocateUnassignedDecision delayed(long remainingDelay, long totalDelay,
@Nullable List<NodeAllocationResult> decisions) {
return no(AllocationStatus.DELAYED_ALLOCATION, decisions, false, remainingDelay, totalDelay);
}
/**
* Returns a NO decision with the given {@link AllocationStatus}, and the explanation for the NO decision
* as well as the individual node-level decisions that comprised the final NO decision if in explain mode.
* Returns a NO decision with the given {@link AllocationStatus}, and the individual node-level
* decisions that comprised the final NO decision if in explain mode.
*/
public static AllocateUnassignedDecision no(AllocationStatus allocationStatus, @Nullable String explanation,
@Nullable Map<String, Decision> nodeDecisions) {
Objects.requireNonNull(allocationStatus, "allocationStatus must not be null");
if (explanation != null) {
return new AllocateUnassignedDecision(Type.NO, allocationStatus, explanation, null, null, asExplanations(nodeDecisions), null);
public static AllocateUnassignedDecision no(AllocationStatus allocationStatus, @Nullable List<NodeAllocationResult> decisions,
boolean reuseStore) {
return no(allocationStatus, decisions, reuseStore, 0L, 0L);
}
private static AllocateUnassignedDecision no(AllocationStatus allocationStatus, @Nullable List<NodeAllocationResult> decisions,
boolean reuseStore, long remainingDelay, long totalDelay) {
if (decisions != null) {
return new AllocateUnassignedDecision(allocationStatus, null, null, decisions, reuseStore, remainingDelay, totalDelay);
} else {
return getCachedDecision(allocationStatus);
}
}
/**
* Returns a THROTTLE decision, with the given explanation and individual node-level decisions that
* Returns a THROTTLE decision, with the individual node-level decisions that
* comprised the final THROTTLE decision if in explain mode.
*/
public static AllocateUnassignedDecision throttle(@Nullable String explanation, @Nullable Map<String, Decision> nodeDecisions) {
if (explanation != null) {
return new AllocateUnassignedDecision(Type.THROTTLE, AllocationStatus.DECIDERS_THROTTLED, explanation, null, null,
asExplanations(nodeDecisions), null);
public static AllocateUnassignedDecision throttle(@Nullable List<NodeAllocationResult> decisions) {
if (decisions != null) {
return new AllocateUnassignedDecision(AllocationStatus.DECIDERS_THROTTLED, null, null, decisions, false, 0L, 0L);
} else {
return getCachedDecision(AllocationStatus.DECIDERS_THROTTLED);
}
}
/**
* Creates a YES decision with the given explanation and individual node-level decisions that
* Creates a YES decision with the given individual node-level decisions that
* comprised the final YES decision, along with the node id to which the shard is assigned and
* the allocation id for the shard, if available.
*/
public static AllocateUnassignedDecision yes(String assignedNodeId, @Nullable String explanation, @Nullable String allocationId,
@Nullable Map<String, Decision> nodeDecisions) {
Objects.requireNonNull(assignedNodeId, "assignedNodeId must not be null");
return new AllocateUnassignedDecision(Type.YES, null, explanation, assignedNodeId, allocationId,
asExplanations(nodeDecisions), null);
public static AllocateUnassignedDecision yes(DiscoveryNode assignedNode, @Nullable String allocationId,
@Nullable List<NodeAllocationResult> decisions, boolean reuseStore) {
return new AllocateUnassignedDecision(null, assignedNode, allocationId, decisions, reuseStore, 0L, 0L);
}
/**
* Creates a {@link AllocateUnassignedDecision} from the given {@link Decision} and the assigned node, if any.
*/
public static AllocateUnassignedDecision fromDecision(Decision decision, @Nullable String assignedNodeId, boolean explain,
@Nullable Map<String, NodeAllocationResult> nodeDecisions) {
public static AllocateUnassignedDecision fromDecision(Decision decision, @Nullable DiscoveryNode assignedNode,
@Nullable List<NodeAllocationResult> nodeDecisions) {
final Type decisionType = decision.type();
AllocationStatus allocationStatus = decisionType != Type.YES ? AllocationStatus.fromDecision(decisionType) : null;
String explanation = null;
if (explain) {
if (decision.type() == Type.YES) {
assert assignedNodeId != null;
explanation = "shard assigned to node [" + assignedNodeId + "]";
} else if (decision.type() == Type.THROTTLE) {
assert assignedNodeId != null;
explanation = "shard assignment throttled on node [" + assignedNodeId + "]";
} else {
explanation = "shard cannot be assigned to any node in the cluster";
}
}
return new AllocateUnassignedDecision(decisionType, allocationStatus, explanation, assignedNodeId, null, nodeDecisions, null);
return new AllocateUnassignedDecision(allocationStatus, assignedNode, null, nodeDecisions, false, 0L, 0L);
}
private static AllocateUnassignedDecision getCachedDecision(AllocationStatus allocationStatus) {
@ -177,101 +172,132 @@ public class AllocateUnassignedDecision {
return Objects.requireNonNull(decision, "precomputed decision not found for " + allocationStatus);
}
private static Map<String, NodeAllocationResult> asExplanations(Map<String, Decision> decisionMap) {
if (decisionMap != null) {
Map<String, NodeAllocationResult> explanationMap = new HashMap<>();
for (Map.Entry<String, Decision> entry : decisionMap.entrySet()) {
explanationMap.put(entry.getKey(), new NodeAllocationResult(entry.getValue(), Float.POSITIVE_INFINITY));
}
return explanationMap;
}
return null;
}
/**
* Returns <code>true</code> if a decision was taken by the allocator, {@code false} otherwise.
* If no decision was taken, then the rest of the fields in this object are meaningless and return {@code null}.
*/
@Override
public boolean isDecisionTaken() {
return finalDecision != null;
return this != NOT_TAKEN;
}
/**
* Returns the final decision made by the allocator on whether to assign the shard.
* This value can only be {@code null} if {@link #isDecisionTaken()} returns {@code false}.
* Returns the {@link AllocationDecision} denoting the result of an allocation attempt.
* If {@link #isDecisionTaken()} returns {@code false}, then invoking this method will
* throw an {@code IllegalStateException}.
*/
@Nullable
public Type getFinalDecisionType() {
return finalDecision;
}
/**
* Returns the final decision made by the allocator on whether to assign the shard.
* Only call this method if {@link #isDecisionTaken()} returns {@code true}, otherwise it will
* throw an {@code IllegalArgumentException}.
*/
public Type getFinalDecisionSafe() {
if (isDecisionTaken() == false) {
throw new IllegalArgumentException("decision must have been taken in order to return the final decision");
}
return finalDecision;
public AllocationDecision getAllocationDecision() {
checkDecisionState();
return AllocationDecision.fromAllocationStatus(allocationStatus);
}
/**
* Returns the status of an unsuccessful allocation attempt. This value will be {@code null} if
* no decision was taken or if the decision was {@link Decision.Type#YES}.
* no decision was taken or if the decision was {@link Decision.Type#YES}. If {@link #isDecisionTaken()}
* returns {@code false}, then invoking this method will throw an {@code IllegalStateException}.
*/
@Nullable
public AllocationStatus getAllocationStatus() {
checkDecisionState();
return allocationStatus;
}
/**
* Returns the free-text explanation for the reason behind the decision taken in {@link #getFinalDecisionType()}.
*/
@Nullable
public String getFinalExplanation() {
return finalExplanation;
}
/**
* Get the node id that the allocator will assign the shard to, unless {@link #getFinalDecisionType()} returns
* a value other than {@link Decision.Type#YES}, in which case this returns {@code null}.
*/
@Nullable
public String getAssignedNodeId() {
return assignedNodeId;
}
/**
* Gets the allocation id for the existing shard copy that the allocator is assigning the shard to.
* This method returns a non-null value iff {@link #getAssignedNodeId()} returns a non-null value
* This method returns a non-null value iff {@link #getTargetNode()} returns a non-null value
* and the node on which the shard is assigned already has a shard copy with an in-sync allocation id
* that we can re-use.
* that we can re-use. If {@link #isDecisionTaken()} returns {@code false}, then invoking this method
* will throw an {@code IllegalStateException}.
*/
@Nullable
public String getAllocationId() {
checkDecisionState();
return allocationId;
}
/**
* Gets the individual node-level decisions that went into making the final decision as represented by
* {@link #getFinalDecisionType()}. The map that is returned has the node id as the key and a {@link Decision}
* as the decision for the given node.
* Gets the remaining delay for allocating the replica shard when a node holding the replica left
* the cluster and the deciders are waiting to see if the node returns before allocating the replica
* elsewhere. Only returns a meaningful positive value if {@link #getAllocationStatus()} returns
* {@link AllocationStatus#DELAYED_ALLOCATION}. If {@link #isDecisionTaken()} returns {@code false},
* then invoking this method will throw an {@code IllegalStateException}.
*/
@Nullable
public Map<String, NodeAllocationResult> getNodeDecisions() {
return nodeDecisions;
public long getRemainingDelayInMillis() {
checkDecisionState();
return remainingDelayInMillis;
}
/**
* Gets the decision on allocating a shard, without examining any specific nodes to allocate to
* (e.g. a replica can never be allocated if the primary is not allocated, so this is a shard-level
* decision, not having taken any node into account).
* Gets the total configured delay for allocating the replica shard when a node holding the replica left
* the cluster and the deciders are waiting to see if the node returns before allocating the replica
* elsewhere. Only returns a meaningful positive value if {@link #getAllocationStatus()} returns
* {@link AllocationStatus#DELAYED_ALLOCATION}. If {@link #isDecisionTaken()} returns {@code false},
* then invoking this method will throw an {@code IllegalStateException}.
*/
@Nullable
public Decision getShardDecision() {
return shardDecision;
public long getConfiguredDelayInMillis() {
checkDecisionState();
return configuredDelayInMillis;
}
@Override
public String getExplanation() {
checkDecisionState();
String explanation;
if (allocationStatus == null) {
explanation = "can allocate the shard";
} else if (allocationStatus == AllocationStatus.DECIDERS_THROTTLED) {
explanation = "allocation temporarily throttled";
} else if (allocationStatus == AllocationStatus.FETCHING_SHARD_DATA) {
explanation = "cannot allocate because information about existing shard data is still being retrieved from " +
"some of the nodes";
} else if (allocationStatus == AllocationStatus.NO_VALID_SHARD_COPY) {
if (getNodeDecisions() != null && getNodeDecisions().size() > 0) {
explanation = "cannot allocate because all existing copies of the shard are unreadable";
} else {
explanation = "cannot allocate because a previous copy of the shard existed but could not be found";
}
} else if (allocationStatus == AllocationStatus.DELAYED_ALLOCATION) {
explanation = "cannot allocate because the cluster is still waiting " +
TimeValue.timeValueMillis(remainingDelayInMillis) +
" for the departed node holding a replica to rejoin" +
(atLeastOneNodeWithYesDecision() ?
", despite being allowed to allocate the shard to at least one other node" : "");
} else {
assert allocationStatus == AllocationStatus.DECIDERS_NO;
if (reuseStore) {
explanation = "cannot allocate because allocation is not permitted to any of the nodes that hold an in-sync shard copy";
} else {
explanation = "cannot allocate because allocation is not permitted to any of the nodes";
}
}
return explanation;
}
@Override
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
checkDecisionState();
builder.field("can_allocate", getAllocationDecision());
builder.field("allocate_explanation", getExplanation());
if (targetNode != null) {
builder.startObject("target_node");
discoveryNodeToXContent(targetNode, true, builder);
builder.endObject();
}
if (allocationId != null) {
builder.field("allocation_id", allocationId);
}
if (allocationStatus == AllocationStatus.DELAYED_ALLOCATION) {
builder.timeValueField("configured_delay_in_millis", "configured_delay", TimeValue.timeValueMillis(configuredDelayInMillis));
builder.timeValueField("remaining_delay_in_millis", "remaining_delay", TimeValue.timeValueMillis(remainingDelayInMillis));
}
nodeDecisionsToXContent(nodeDecisions, builder, params);
return builder;
}
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
out.writeOptionalWriteable(allocationStatus);
out.writeOptionalString(allocationId);
out.writeBoolean(reuseStore);
out.writeVLong(remainingDelayInMillis);
out.writeVLong(configuredDelayInMillis);
}
}

View File

@ -0,0 +1,156 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.cluster.routing.allocation;
import org.elasticsearch.cluster.routing.UnassignedInfo.AllocationStatus;
import org.elasticsearch.cluster.routing.allocation.decider.Decision;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.io.stream.Writeable;
import java.io.IOException;
import java.util.Locale;
/**
* An enum which represents the various decision types that can be taken by the
* allocators and deciders for allocating a shard to a node.
*/
public enum AllocationDecision implements Writeable {
/**
* The shard can be allocated to a node.
*/
YES((byte) 0),
/**
* The allocation attempt was throttled for the shard.
*/
THROTTLE((byte) 1),
/**
* The shard cannot be allocated, which can happen for any number of reasons,
* including the allocation deciders gave a NO decision for allocating.
*/
NO((byte) 2),
/**
* The shard could not be rebalanced to another node despite rebalancing
* being allowed, because moving the shard to the other node would not form
* a better cluster balance.
*/
WORSE_BALANCE((byte) 3),
/**
* Waiting on getting shard data from all nodes before making a decision
* about where to allocate the shard.
*/
FETCH_PENDING((byte) 4),
/**
* The allocation decision has been delayed waiting for a replica with a shard copy
* that left the cluster to rejoin.
*/
DELAYED_ALLOCATION((byte) 5),
/**
* The shard was denied allocation because there were no valid shard copies
* found for it amongst the nodes in the cluster.
*/
NO_VALID_SHARD_COPY((byte) 6),
/**
* No attempt was made to allocate the shard
*/
NO_ATTEMPT((byte) 7);
private final byte id;
AllocationDecision(byte id) {
this.id = id;
}
@Override
public void writeTo(StreamOutput out) throws IOException {
out.writeByte(id);
}
public static AllocationDecision readFrom(StreamInput in) throws IOException {
byte id = in.readByte();
switch (id) {
case 0:
return YES;
case 1:
return THROTTLE;
case 2:
return NO;
case 3:
return WORSE_BALANCE;
case 4:
return FETCH_PENDING;
case 5:
return DELAYED_ALLOCATION;
case 6:
return NO_VALID_SHARD_COPY;
case 7:
return NO_ATTEMPT;
default:
throw new IllegalArgumentException("Unknown value [" + id + "]");
}
}
/**
* Gets an {@link AllocationDecision} from a {@link AllocationStatus}.
*/
public static AllocationDecision fromAllocationStatus(AllocationStatus allocationStatus) {
if (allocationStatus == null) {
return YES;
} else {
switch (allocationStatus) {
case DECIDERS_THROTTLED:
return THROTTLE;
case FETCHING_SHARD_DATA:
return FETCH_PENDING;
case DELAYED_ALLOCATION:
return DELAYED_ALLOCATION;
case NO_VALID_SHARD_COPY:
return NO_VALID_SHARD_COPY;
case NO_ATTEMPT:
return NO_ATTEMPT;
default:
assert allocationStatus == AllocationStatus.DECIDERS_NO : "unhandled AllocationStatus type [" + allocationStatus + "]";
return NO;
}
}
}
/**
* Gets an {@link AllocationDecision} from a {@link Decision.Type}
*/
public static AllocationDecision fromDecisionType(Decision.Type type) {
switch (type) {
case YES:
return YES;
case THROTTLE:
return THROTTLE;
default:
assert type == Decision.Type.NO;
return NO;
}
}
@Override
public String toString() {
return super.toString().toLowerCase(Locale.ROOT);
}
}

View File

@ -19,109 +19,267 @@
package org.elasticsearch.cluster.routing.allocation;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.routing.allocation.decider.Decision;
import org.elasticsearch.cluster.routing.allocation.decider.Decision.Type;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.xcontent.XContentBuilder;
import java.util.Collections;
import java.util.Map;
import java.util.Objects;
import java.io.IOException;
import java.util.List;
/**
* Represents a decision to move a started shard because it is no longer allowed to remain on its current node.
* Represents a decision to move a started shard, either because it is no longer allowed to remain on its current node
* or because moving it to another node will form a better cluster balance.
*/
public final class MoveDecision extends RelocationDecision {
public final class MoveDecision extends AbstractAllocationDecision {
/** a constant representing no decision taken */
public static final MoveDecision NOT_TAKEN = new MoveDecision(null, null, null, null, null);
public static final MoveDecision NOT_TAKEN = new MoveDecision(null, null, AllocationDecision.NO_ATTEMPT, null, null, 0);
/** cached decisions so we don't have to recreate objects for common decisions when not in explain mode. */
private static final MoveDecision CACHED_STAY_DECISION = new MoveDecision(Decision.YES, Decision.Type.NO, null, null, null);
private static final MoveDecision CACHED_CANNOT_MOVE_DECISION = new MoveDecision(Decision.NO, Decision.Type.NO, null, null, null);
private static final MoveDecision CACHED_STAY_DECISION =
new MoveDecision(Decision.YES, null, AllocationDecision.NO_ATTEMPT, null, null, 0);
private static final MoveDecision CACHED_CANNOT_MOVE_DECISION =
new MoveDecision(Decision.NO, null, AllocationDecision.NO, null, null, 0);
@Nullable
AllocationDecision allocationDecision;
@Nullable
private final Decision canRemainDecision;
@Nullable
private final Map<String, NodeAllocationResult> nodeDecisions;
private final Decision canRebalanceDecision;
private final int currentNodeRanking;
private MoveDecision(Decision canRemainDecision, Decision.Type finalDecision, String finalExplanation,
String assignedNodeId, Map<String, NodeAllocationResult> nodeDecisions) {
super(finalDecision, finalExplanation, assignedNodeId);
private MoveDecision(Decision canRemainDecision, Decision canRebalanceDecision, AllocationDecision allocationDecision,
DiscoveryNode assignedNode, List<NodeAllocationResult> nodeDecisions, int currentNodeRanking) {
super(assignedNode, nodeDecisions);
this.allocationDecision = allocationDecision;
this.canRemainDecision = canRemainDecision;
this.nodeDecisions = nodeDecisions != null ? Collections.unmodifiableMap(nodeDecisions) : null;
this.canRebalanceDecision = canRebalanceDecision;
this.currentNodeRanking = currentNodeRanking;
}
public MoveDecision(StreamInput in) throws IOException {
super(in);
allocationDecision = in.readOptionalWriteable(AllocationDecision::readFrom);
canRemainDecision = in.readOptionalWriteable(Decision::readFrom);
canRebalanceDecision = in.readOptionalWriteable(Decision::readFrom);
currentNodeRanking = in.readVInt();
}
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
out.writeOptionalWriteable(allocationDecision);
out.writeOptionalWriteable(canRemainDecision);
out.writeOptionalWriteable(canRebalanceDecision);
out.writeVInt(currentNodeRanking);
}
/**
* Creates a move decision for the shard being able to remain on its current node, so not moving.
* Creates a move decision for the shard being able to remain on its current node, so the shard won't
* be forced to move to another node.
*/
public static MoveDecision stay(Decision canRemainDecision, boolean explain) {
assert canRemainDecision.type() != Decision.Type.NO;
if (explain) {
final String explanation;
if (explain) {
explanation = "shard is allowed to remain on its current node, so no reason to move";
} else {
explanation = null;
}
return new MoveDecision(Objects.requireNonNull(canRemainDecision), Decision.Type.NO, explanation, null, null);
public static MoveDecision stay(Decision canRemainDecision) {
if (canRemainDecision != null) {
assert canRemainDecision.type() != Type.NO;
return new MoveDecision(canRemainDecision, null, AllocationDecision.NO_ATTEMPT, null, null, 0);
} else {
return CACHED_STAY_DECISION;
}
}
/**
* Creates a move decision for the shard not being able to remain on its current node.
* Creates a move decision for the shard not being allowed to remain on its current node.
*
* @param canRemainDecision the decision for whether the shard is allowed to remain on its current node
* @param finalDecision the decision of whether to move the shard to another node
* @param explain true if in explain mode
* @param currentNodeId the current node id where the shard is assigned
* @param assignedNodeId the node id for where the shard can move to
* @param allocationDecision the {@link AllocationDecision} for moving the shard to another node
* @param assignedNode the node where the shard should move to
* @param nodeDecisions the node-level decisions that comprised the final decision, non-null iff explain is true
* @return the {@link MoveDecision} for moving the shard to another node
*/
public static MoveDecision decision(Decision canRemainDecision, Decision.Type finalDecision, boolean explain, String currentNodeId,
String assignedNodeId, Map<String, NodeAllocationResult> nodeDecisions) {
public static MoveDecision cannotRemain(Decision canRemainDecision, AllocationDecision allocationDecision, DiscoveryNode assignedNode,
List<NodeAllocationResult> nodeDecisions) {
assert canRemainDecision != null;
assert canRemainDecision.type() != Decision.Type.YES : "create decision with MoveDecision#stay instead";
String finalExplanation = null;
if (explain) {
assert currentNodeId != null;
if (finalDecision == Decision.Type.YES) {
assert assignedNodeId != null;
finalExplanation = "shard cannot remain on node [" + currentNodeId + "], moving to node [" + assignedNodeId + "]";
} else if (finalDecision == Decision.Type.THROTTLE) {
finalExplanation = "shard cannot remain on node [" + currentNodeId + "], throttled on moving to another node";
} else {
finalExplanation = "shard cannot remain on node [" + currentNodeId + "], but cannot be assigned to any other node";
}
}
if (finalExplanation == null && finalDecision == Decision.Type.NO) {
assert canRemainDecision.type() != Type.YES : "create decision with MoveDecision#stay instead";
if (nodeDecisions == null && allocationDecision == AllocationDecision.NO) {
// the final decision is NO (no node to move the shard to) and we are not in explain mode, return a cached version
return CACHED_CANNOT_MOVE_DECISION;
} else {
assert ((assignedNodeId == null) == (finalDecision != Decision.Type.YES));
return new MoveDecision(canRemainDecision, finalDecision, finalExplanation, assignedNodeId, nodeDecisions);
assert ((assignedNode == null) == (allocationDecision != AllocationDecision.YES));
return new MoveDecision(canRemainDecision, null, allocationDecision, assignedNode, nodeDecisions, 0);
}
}
/**
* Returns {@code true} if the shard cannot remain on its current node and can be moved, returns {@code false} otherwise.
* Creates a move decision for when rebalancing the shard is not allowed.
*/
public boolean move() {
return cannotRemain() && getFinalDecisionType() == Decision.Type.YES;
public static MoveDecision cannotRebalance(Decision canRebalanceDecision, AllocationDecision allocationDecision, int currentNodeRanking,
List<NodeAllocationResult> nodeDecisions) {
return new MoveDecision(null, canRebalanceDecision, allocationDecision, null, nodeDecisions, currentNodeRanking);
}
/**
* Returns {@code true} if the shard cannot remain on its current node.
* Creates a decision for whether to move the shard to a different node to form a better cluster balance.
*/
public boolean cannotRemain() {
return isDecisionTaken() && canRemainDecision.type() == Decision.Type.NO;
public static MoveDecision rebalance(Decision canRebalanceDecision, AllocationDecision allocationDecision,
@Nullable DiscoveryNode assignedNode, int currentNodeRanking,
List<NodeAllocationResult> nodeDecisions) {
return new MoveDecision(null, canRebalanceDecision, allocationDecision, assignedNode, nodeDecisions, currentNodeRanking);
}
@Override
public boolean isDecisionTaken() {
return this != NOT_TAKEN;
}
/**
* Gets the individual node-level decisions that went into making the final decision as represented by
* {@link #getFinalDecisionType()}. The map that is returned has the node id as the key and a {@link NodeAllocationResult}.
* Returns {@code true} if the shard cannot remain on its current node and can be moved,
* returns {@code false} otherwise. If {@link #isDecisionTaken()} returns {@code false},
* then invoking this method will throw an {@code IllegalStateException}.
*/
public boolean forceMove() {
checkDecisionState();
return canRemain() == false && allocationDecision == AllocationDecision.YES;
}
/**
* Returns {@code true} if the shard can remain on its current node, returns {@code false} otherwise.
* If {@link #isDecisionTaken()} returns {@code false}, then invoking this method will throw an {@code IllegalStateException}.
*/
public boolean canRemain() {
checkDecisionState();
return canRemainDecision.type() == Type.YES;
}
/**
* Returns the decision for the shard being allowed to remain on its current node. If {@link #isDecisionTaken()}
* returns {@code false}, then invoking this method will throw an {@code IllegalStateException}.
*/
public Decision getCanRemainDecision() {
checkDecisionState();
return canRemainDecision;
}
/**
* Returns {@code true} if the shard is allowed to be rebalanced to another node in the cluster,
* returns {@code false} otherwise. If {@link #getCanRebalanceDecision()} returns {@code null}, then
* the result of this method is meaningless, as no rebalance decision was taken. If {@link #isDecisionTaken()}
* returns {@code false}, then invoking this method will throw an {@code IllegalStateException}.
*/
public boolean canRebalance() {
checkDecisionState();
return canRebalanceDecision.type() == Type.YES;
}
/**
* Returns the decision for being allowed to rebalance the shard. Invoking this method will return
* {@code null} if {@link #canRemain()} ()} returns {@code false}, which means the node is not allowed to
* remain on its current node, so the cluster is forced to attempt to move the shard to a different node,
* as opposed to attempting to rebalance the shard if a better cluster balance is possible by moving it.
* If {@link #isDecisionTaken()} returns {@code false}, then invoking this method will throw an
* {@code IllegalStateException}.
*/
@Nullable
public Map<String, NodeAllocationResult> getNodeDecisions() {
return nodeDecisions;
public Decision getCanRebalanceDecision() {
checkDecisionState();
return canRebalanceDecision;
}
/**
* Returns the {@link AllocationDecision} for moving this shard to another node. If {@link #isDecisionTaken()} returns
* {@code false}, then invoking this method will throw an {@code IllegalStateException}.
*/
@Nullable
public AllocationDecision getAllocationDecision() {
return allocationDecision;
}
/**
* Gets the current ranking of the node to which the shard is currently assigned, relative to the
* other nodes in the cluster as reported in {@link NodeAllocationResult#getWeightRanking()}. The
* ranking will only return a meaningful positive integer if {@link #getCanRebalanceDecision()} returns
* a non-null value; otherwise, 0 will be returned. If {@link #isDecisionTaken()} returns
* {@code false}, then invoking this method will throw an {@code IllegalStateException}.
*/
public int getCurrentNodeRanking() {
checkDecisionState();
return currentNodeRanking;
}
@Override
public String getExplanation() {
checkDecisionState();
String explanation;
if (canRebalanceDecision != null) {
// it was a decision to rebalance the shard, because the shard was allowed to remain on its current node
if (allocationDecision == AllocationDecision.FETCH_PENDING) {
explanation = "cannot rebalance as information about existing copies of this shard in the cluster is still being gathered";
} else if (canRebalanceDecision.type() == Type.NO) {
explanation = "rebalancing is not allowed on the cluster" + (atLeastOneNodeWithYesDecision() ? ", even though there " +
"is at least one node on which the shard can be allocated" : "");
} else if (canRebalanceDecision.type() == Type.THROTTLE) {
explanation = "rebalancing is throttled";
} else {
if (getTargetNode() != null) {
if (allocationDecision == AllocationDecision.THROTTLE) {
explanation = "shard rebalancing throttled";
} else {
explanation = "can rebalance shard";
}
} else {
explanation = "cannot rebalance as no target node exists that can both allocate this shard " +
"and improve the cluster balance";
}
}
} else {
// it was a decision to force move the shard
if (canRemain()) {
explanation = "shard can remain on its current node";
} else if (allocationDecision == AllocationDecision.YES) {
explanation = "shard cannot remain on this node and is force-moved to another node";
} else if (allocationDecision == AllocationDecision.THROTTLE) {
explanation = "shard cannot remain on this node but is throttled on moving to another node";
} else {
assert allocationDecision == AllocationDecision.NO;
explanation = "cannot move shard to another node, even though it is not allowed to remain on its current node";
}
}
return explanation;
}
@Override
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
checkDecisionState();
if (targetNode != null) {
builder.startObject("target_node");
discoveryNodeToXContent(targetNode, true, builder);
builder.endObject();
}
builder.field("can_remain_on_current_node", canRemain() ? "yes" : "no");
if (canRemain() == false && canRemainDecision.getDecisions().isEmpty() == false) {
builder.startArray("can_remain_decisions");
canRemainDecision.toXContent(builder, params);
builder.endArray();
}
if (canRebalanceDecision != null) {
AllocationDecision rebalanceDecision = AllocationDecision.fromDecisionType(canRebalanceDecision.type());
builder.field("can_rebalance_cluster", rebalanceDecision);
if (rebalanceDecision != AllocationDecision.YES && canRebalanceDecision.getDecisions().isEmpty() == false) {
builder.startArray("can_rebalance_cluster_decisions");
canRebalanceDecision.toXContent(builder, params);
builder.endArray();
}
}
if (canRebalanceDecision != null) {
builder.field("can_rebalance_to_other_node", allocationDecision);
} else {
builder.field("can_move_to_other_node", forceMove() ? "yes" : "no");
}
builder.field(canRebalanceDecision != null ? "rebalance_explanation" : "move_explanation", getExplanation());
nodeDecisionsToXContent(nodeDecisions, builder, params);
return builder;
}
}

View File

@ -19,59 +19,256 @@
package org.elasticsearch.cluster.routing.allocation;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.routing.allocation.decider.Decision;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.io.stream.Writeable;
import org.elasticsearch.common.xcontent.ToXContent;
import org.elasticsearch.common.xcontent.XContentBuilder;
import java.util.Objects;
import java.io.IOException;
import java.util.Comparator;
import static org.elasticsearch.cluster.routing.allocation.AbstractAllocationDecision.discoveryNodeToXContent;
/**
* This class represents the shard allocation decision for a single node,
* including the {@link Decision} whether to allocate to the node and other
* information related to obtaining the decision for the node.
* This class represents the shard allocation decision and its explanation for a single node.
*/
public final class NodeAllocationResult {
public class NodeAllocationResult implements ToXContent, Writeable, Comparable<NodeAllocationResult> {
private final Decision decision;
private final float weight;
private static final Comparator<NodeAllocationResult> nodeResultComparator =
Comparator.comparing(NodeAllocationResult::getNodeDecision)
.thenComparingInt(NodeAllocationResult::getWeightRanking)
.thenComparing(r -> r.getNode().getId());
public NodeAllocationResult(Decision decision) {
this.decision = Objects.requireNonNull(decision);
this.weight = Float.POSITIVE_INFINITY;
private final DiscoveryNode node;
@Nullable
private final ShardStoreInfo shardStoreInfo;
private final AllocationDecision nodeDecision;
private final Decision canAllocateDecision;
private final int weightRanking;
public NodeAllocationResult(DiscoveryNode node, ShardStoreInfo shardStoreInfo, Decision decision) {
this.node = node;
this.shardStoreInfo = shardStoreInfo;
this.canAllocateDecision = decision;
this.nodeDecision = AllocationDecision.fromDecisionType(canAllocateDecision.type());
this.weightRanking = 0;
}
public NodeAllocationResult(Decision decision, float weight) {
this.decision = Objects.requireNonNull(decision);
this.weight = Objects.requireNonNull(weight);
public NodeAllocationResult(DiscoveryNode node, AllocationDecision nodeDecision, Decision canAllocate, int weightRanking) {
this.node = node;
this.shardStoreInfo = null;
this.canAllocateDecision = canAllocate;
this.nodeDecision = nodeDecision;
this.weightRanking = weightRanking;
}
/**
* The decision for allocating to the node.
*/
public Decision getDecision() {
return decision;
public NodeAllocationResult(DiscoveryNode node, Decision decision, int weightRanking) {
this.node = node;
this.shardStoreInfo = null;
this.canAllocateDecision = decision;
this.nodeDecision = AllocationDecision.fromDecisionType(decision.type());
this.weightRanking = weightRanking;
}
/**
* The calculated weight for allocating a shard to the node. A value of {@link Float#POSITIVE_INFINITY}
* means the weight was not calculated or factored into the decision.
*/
public float getWeight() {
return weight;
public NodeAllocationResult(StreamInput in) throws IOException {
node = new DiscoveryNode(in);
shardStoreInfo = in.readOptionalWriteable(ShardStoreInfo::new);
canAllocateDecision = Decision.readFrom(in);
nodeDecision = AllocationDecision.readFrom(in);
weightRanking = in.readVInt();
}
@Override
public boolean equals(Object other) {
if (this == other) {
return true;
}
if (other == null || getClass() != other.getClass()) {
return false;
}
NodeAllocationResult that = (NodeAllocationResult) other;
return decision.equals(that.decision) && Float.compare(weight, that.weight) == 0;
public void writeTo(StreamOutput out) throws IOException {
node.writeTo(out);
out.writeOptionalWriteable(shardStoreInfo);
canAllocateDecision.writeTo(out);
nodeDecision.writeTo(out);
out.writeVInt(weightRanking);
}
/**
* Get the node that this decision is for.
*/
public DiscoveryNode getNode() {
return node;
}
/**
* Get the shard store information for the node, if it exists.
*/
@Nullable
public ShardStoreInfo getShardStoreInfo() {
return shardStoreInfo;
}
/**
* The decision details for allocating to this node.
*/
public Decision getCanAllocateDecision() {
return canAllocateDecision;
}
/**
* Is the weight assigned for the node?
*/
public boolean isWeightRanked() {
return weightRanking > 0;
}
/**
* The weight ranking for allocating a shard to the node. Each node will have
* a unique weight ranking that is relative to the other nodes against which the
* deciders ran. For example, suppose there are 3 nodes which the allocation deciders
* decided upon: node1, node2, and node3. If node2 had the best weight for holding the
* shard, followed by node3, followed by node1, then node2's weight will be 1, node3's
* weight will be 2, and node1's weight will be 1. A value of 0 means the weight was
* not calculated or factored into the decision.
*/
public int getWeightRanking() {
return weightRanking;
}
/**
* Gets the {@link AllocationDecision} for allocating to this node.
*/
public AllocationDecision getNodeDecision() {
return nodeDecision;
}
@Override
public int hashCode() {
return Objects.hash(decision, weight);
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
builder.startObject();
{
discoveryNodeToXContent(node, false, builder);
builder.field("node_decision", nodeDecision);
if (shardStoreInfo != null) {
shardStoreInfo.toXContent(builder, params);
}
if (isWeightRanked()) {
builder.field("weight_ranking", getWeightRanking());
}
if (canAllocateDecision.getDecisions().isEmpty() == false) {
builder.startArray("deciders");
canAllocateDecision.toXContent(builder, params);
builder.endArray();
}
}
builder.endObject();
return builder;
}
@Override
public int compareTo(NodeAllocationResult other) {
return nodeResultComparator.compare(this, other);
}
/** A class that captures metadata about a shard store on a node. */
public static final class ShardStoreInfo implements ToXContent, Writeable {
private final boolean inSync;
@Nullable
private final String allocationId;
private final long matchingBytes;
@Nullable
private final Exception storeException;
public ShardStoreInfo(String allocationId, boolean inSync, Exception storeException) {
this.inSync = inSync;
this.allocationId = allocationId;
this.matchingBytes = -1;
this.storeException = storeException;
}
public ShardStoreInfo(long matchingBytes) {
this.inSync = false;
this.allocationId = null;
this.matchingBytes = matchingBytes;
this.storeException = null;
}
public ShardStoreInfo(StreamInput in) throws IOException {
this.inSync = in.readBoolean();
this.allocationId = in.readOptionalString();
this.matchingBytes = in.readLong();
this.storeException = in.readException();
}
/**
* Returns {@code true} if the shard copy is in-sync and contains the latest data.
* Returns {@code false} if the shard copy is stale or if the shard copy being examined
* is for a replica shard allocation.
*/
public boolean isInSync() {
return inSync;
}
/**
* Gets the allocation id for the shard copy, if it exists.
*/
@Nullable
public String getAllocationId() {
return allocationId;
}
/**
* Returns {@code true} if the shard copy has a matching sync id with the primary shard.
* Returns {@code false} if the shard copy does not have a matching sync id with the primary
* shard, or this explanation pertains to the allocation of a primary shard, in which case
* matching sync ids are irrelevant.
*/
public boolean hasMatchingSyncId() {
return matchingBytes == Long.MAX_VALUE;
}
/**
* Gets the number of matching bytes the shard copy has with the primary shard.
* Returns {@code Long.MAX_VALUE} if {@link #hasMatchingSyncId()} returns {@code true}.
* Returns -1 if not applicable (this value only applies to assigning replica shards).
*/
public long getMatchingBytes() {
return matchingBytes;
}
@Override
public void writeTo(StreamOutput out) throws IOException {
out.writeBoolean(inSync);
out.writeOptionalString(allocationId);
out.writeLong(matchingBytes);
out.writeException(storeException);
}
@Override
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
builder.startObject("store");
{
if (matchingBytes < 0) {
// dealing with a primary shard
builder.field("in_sync", inSync);
}
if (allocationId != null) {
builder.field("allocation_id", allocationId);
}
if (matchingBytes >= 0) {
if (hasMatchingSyncId()) {
builder.field("matching_sync_id", true);
} else {
builder.byteSizeField("matching_size_in_bytes", "matching_size", matchingBytes);
}
}
if (storeException != null) {
builder.startObject("store_exception");
ElasticsearchException.toXContent(builder, params, storeException);
builder.endObject();
}
}
builder.endObject();
return builder;
}
}
}

View File

@ -1,88 +0,0 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.cluster.routing.allocation;
import org.elasticsearch.cluster.routing.allocation.decider.Decision;
import java.util.Objects;
/**
* A node-level explanation for the decision to rebalance a shard.
*/
public final class NodeRebalanceResult {
private final Decision.Type nodeDecisionType;
private final Decision canAllocate;
private final boolean betterWeightThanCurrent;
private final boolean deltaAboveThreshold;
private final float currentWeight;
private final float weightWithShardAdded;
public NodeRebalanceResult(Decision.Type nodeDecisionType, Decision canAllocate, boolean betterWeightThanCurrent,
boolean deltaAboveThreshold, float currentWeight, float weightWithShardAdded) {
this.nodeDecisionType = Objects.requireNonNull(nodeDecisionType);
this.canAllocate = Objects.requireNonNull(canAllocate);
this.betterWeightThanCurrent = betterWeightThanCurrent;
this.deltaAboveThreshold = deltaAboveThreshold;
this.currentWeight = currentWeight;
this.weightWithShardAdded = weightWithShardAdded;
}
/**
* Returns the decision to rebalance to the node.
*/
public Decision.Type getNodeDecisionType() {
return nodeDecisionType;
}
/**
* Returns whether the shard is allowed to be allocated to the node.
*/
public Decision getCanAllocateDecision() {
return canAllocate;
}
/**
* Returns whether the weight of the node is better than the weight of the node where the shard currently resides.
*/
public boolean isBetterWeightThanCurrent() {
return betterWeightThanCurrent;
}
/**
* Returns if the weight delta by assigning to this node was above the threshold to warrant a rebalance.
*/
public boolean isDeltaAboveThreshold() {
return deltaAboveThreshold;
}
/**
* Returns the current weight of the node if the shard is not added to the node.
*/
public float getCurrentWeight() {
return currentWeight;
}
/**
* Returns the weight of the node if the shard is added to the node.
*/
public float getWeightWithShardAdded() {
return weightWithShardAdded;
}
}

View File

@ -1,95 +0,0 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.cluster.routing.allocation;
import org.elasticsearch.cluster.routing.allocation.decider.Decision;
import org.elasticsearch.cluster.routing.allocation.decider.Decision.Type;
import org.elasticsearch.common.Nullable;
import java.util.Collections;
import java.util.Map;
/**
* Represents a decision to move a started shard to form a more optimally balanced cluster.
*/
public final class RebalanceDecision extends RelocationDecision {
/** a constant representing no decision taken */
public static final RebalanceDecision NOT_TAKEN = new RebalanceDecision(null, null, null, null, null, Float.POSITIVE_INFINITY);
@Nullable
private final Decision canRebalanceDecision;
@Nullable
private final Map<String, NodeRebalanceResult> nodeDecisions;
private float currentWeight;
public RebalanceDecision(Decision canRebalanceDecision, Type finalDecision, String finalExplanation) {
this(canRebalanceDecision, finalDecision, finalExplanation, null, null, Float.POSITIVE_INFINITY);
}
public RebalanceDecision(Decision canRebalanceDecision, Type finalDecision, String finalExplanation,
String assignedNodeId, Map<String, NodeRebalanceResult> nodeDecisions, float currentWeight) {
super(finalDecision, finalExplanation, assignedNodeId);
this.canRebalanceDecision = canRebalanceDecision;
this.nodeDecisions = nodeDecisions != null ? Collections.unmodifiableMap(nodeDecisions) : null;
this.currentWeight = currentWeight;
}
/**
* Creates a new {@link RebalanceDecision}, computing the explanation based on the decision parameters.
*/
public static RebalanceDecision decision(Decision canRebalanceDecision, Type finalDecision, String assignedNodeId,
Map<String, NodeRebalanceResult> nodeDecisions, float currentWeight, float threshold) {
final String explanation = produceFinalExplanation(finalDecision, assignedNodeId, threshold);
return new RebalanceDecision(canRebalanceDecision, finalDecision, explanation, assignedNodeId, nodeDecisions, currentWeight);
}
/**
* Returns the decision for being allowed to rebalance the shard.
*/
@Nullable
public Decision getCanRebalanceDecision() {
return canRebalanceDecision;
}
/**
* Gets the individual node-level decisions that went into making the final decision as represented by
* {@link #getFinalDecisionType()}. The map that is returned has the node id as the key and a {@link NodeRebalanceResult}.
*/
@Nullable
public Map<String, NodeRebalanceResult> getNodeDecisions() {
return nodeDecisions;
}
private static String produceFinalExplanation(final Type finalDecisionType, final String assignedNodeId, final float threshold) {
final String finalExplanation;
if (assignedNodeId != null) {
if (finalDecisionType == Type.THROTTLE) {
finalExplanation = "throttle moving shard to node [" + assignedNodeId + "], as it is " +
"currently busy with other shard relocations";
} else {
finalExplanation = "moving shard to node [" + assignedNodeId + "] to form a more balanced cluster";
}
} else {
finalExplanation = "cannot rebalance shard, no other node exists that would form a more balanced " +
"cluster within the defined threshold [" + threshold + "]";
}
return finalExplanation;
}
}

View File

@ -1,74 +0,0 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.cluster.routing.allocation;
import org.elasticsearch.cluster.routing.allocation.decider.Decision;
import org.elasticsearch.common.Nullable;
/**
* Represents a decision to relocate a started shard from its current node.
*/
public abstract class RelocationDecision {
@Nullable
private final Decision.Type finalDecision;
@Nullable
private final String finalExplanation;
@Nullable
private final String assignedNodeId;
protected RelocationDecision(Decision.Type finalDecision, String finalExplanation, String assignedNodeId) {
this.finalDecision = finalDecision;
this.finalExplanation = finalExplanation;
this.assignedNodeId = assignedNodeId;
}
/**
* Returns {@code true} if a decision was taken by the allocator, {@code false} otherwise.
* If no decision was taken, then the rest of the fields in this object are meaningless and return {@code null}.
*/
public boolean isDecisionTaken() {
return finalDecision != null;
}
/**
* Returns the final decision made by the allocator on whether to assign the shard, and
* {@code null} if no decision was taken.
*/
public Decision.Type getFinalDecisionType() {
return finalDecision;
}
/**
* Returns the free-text explanation for the reason behind the decision taken in {@link #getFinalDecisionType()}.
*/
@Nullable
public String getFinalExplanation() {
return finalExplanation;
}
/**
* Get the node id that the allocator will assign the shard to, unless {@link #getFinalDecisionType()} returns
* a value other than {@link Decision.Type#YES}, in which case this returns {@code null}.
*/
@Nullable
public String getAssignedNodeId() {
return assignedNodeId;
}
}

View File

@ -20,7 +20,6 @@
package org.elasticsearch.cluster.routing.allocation;
import org.elasticsearch.cluster.routing.allocation.command.AllocationCommand;
import org.elasticsearch.cluster.routing.allocation.command.AllocationCommands;
import org.elasticsearch.cluster.routing.allocation.decider.Decision;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
@ -59,7 +58,7 @@ public class RerouteExplanation implements ToXContent {
public static void writeTo(RerouteExplanation explanation, StreamOutput out) throws IOException {
out.writeNamedWriteable(explanation.command);
Decision.writeTo(explanation.decisions, out);
explanation.decisions.writeTo(out);
}
@Override
@ -67,15 +66,9 @@ public class RerouteExplanation implements ToXContent {
builder.startObject();
builder.field("command", command.name());
builder.field("parameters", command);
// The Decision could be a Multi or Single decision, and they should
// both be encoded the same, so check and wrap in an array if necessary
if (decisions instanceof Decision.Multi) {
decisions.toXContent(builder, params);
} else {
builder.startArray("decisions");
decisions.toXContent(builder, params);
builder.endArray();
}
builder.startArray("decisions");
decisions.toXContent(builder, params);
builder.endArray();
builder.endObject();
return builder;
}

View File

@ -29,13 +29,12 @@ import org.elasticsearch.cluster.routing.RoutingNode;
import org.elasticsearch.cluster.routing.RoutingNodes;
import org.elasticsearch.cluster.routing.ShardRouting;
import org.elasticsearch.cluster.routing.ShardRoutingState;
import org.elasticsearch.cluster.routing.UnassignedInfo;
import org.elasticsearch.cluster.routing.UnassignedInfo.AllocationStatus;
import org.elasticsearch.cluster.routing.allocation.AllocateUnassignedDecision;
import org.elasticsearch.cluster.routing.allocation.AllocationDecision;
import org.elasticsearch.cluster.routing.allocation.MoveDecision;
import org.elasticsearch.cluster.routing.allocation.NodeRebalanceResult;
import org.elasticsearch.cluster.routing.allocation.RebalanceDecision;
import org.elasticsearch.cluster.routing.allocation.RoutingAllocation;
import org.elasticsearch.cluster.routing.allocation.NodeAllocationResult;
import org.elasticsearch.cluster.routing.allocation.RoutingAllocation;
import org.elasticsearch.cluster.routing.allocation.decider.AllocationDeciders;
import org.elasticsearch.cluster.routing.allocation.decider.Decision;
import org.elasticsearch.cluster.routing.allocation.decider.Decision.Type;
@ -49,12 +48,14 @@ import org.elasticsearch.common.settings.Setting.Property;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.gateway.PriorityComparator;
import java.util.ArrayList;
import java.util.Collections;
import java.util.Comparator;
import java.util.HashMap;
import java.util.HashSet;
import java.util.IdentityHashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Set;
@ -134,7 +135,7 @@ public class BalancedShardsAllocator extends AbstractComponent implements Shards
* from the cluster allocation explain API to explain possible rebalancing decisions for a single
* shard.
*/
public RebalanceDecision decideRebalance(final ShardRouting shard, final RoutingAllocation allocation) {
public MoveDecision decideRebalance(final ShardRouting shard, final RoutingAllocation allocation) {
assert allocation.debugDecision() : "debugDecision should be set in explain mode";
return new Balancer(logger, allocation, weightFunction, threshold).decideRebalance(shard);
}
@ -334,24 +335,14 @@ public class BalancedShardsAllocator extends AbstractComponent implements Shards
* optimally balanced cluster. This method is invoked from the cluster allocation
* explain API only.
*/
private RebalanceDecision decideRebalance(final ShardRouting shard) {
private MoveDecision decideRebalance(final ShardRouting shard) {
if (shard.started() == false) {
// cannot rebalance a shard that isn't started
return RebalanceDecision.NOT_TAKEN;
return MoveDecision.NOT_TAKEN;
}
Decision canRebalance = allocation.deciders().canRebalance(shard, allocation);
if (allocation.hasPendingAsyncFetch()) {
return new RebalanceDecision(
canRebalance,
Type.NO,
"cannot rebalance due to in-flight shard store fetches, otherwise allocation may prematurely rebalance a shard to " +
"a node that is soon to receive another shard assignment upon completion of the shard store fetch, " +
"rendering the cluster imbalanced again"
);
}
sorter.reset(shard.getIndexName());
ModelNode[] modelNodes = sorter.modelNodes;
final String currentNodeId = shard.currentNodeId();
@ -369,12 +360,14 @@ public class BalancedShardsAllocator extends AbstractComponent implements Shards
final float currentWeight = sorter.weight(currentNode);
final AllocationDeciders deciders = allocation.deciders();
final String idxName = shard.getIndexName();
Map<String, NodeRebalanceResult> nodeDecisions = new HashMap<>(modelNodes.length - 1);
Type rebalanceDecisionType = Type.NO;
String assignedNodeId = null;
ModelNode assignedNode = null;
List<Tuple<ModelNode, Decision>> betterBalanceNodes = new ArrayList<>();
List<Tuple<ModelNode, Decision>> sameBalanceNodes = new ArrayList<>();
List<Tuple<ModelNode, Decision>> worseBalanceNodes = new ArrayList<>();
for (ModelNode node : modelNodes) {
if (node == currentNode) {
continue; // skip over node we're currently allocated to it
continue; // skip over node we're currently allocated to
}
final Decision canAllocate = deciders.canAllocate(shard, node.getRoutingNode(), allocation);
// the current weight of the node in the cluster, as computed by the weight function;
@ -387,22 +380,21 @@ public class BalancedShardsAllocator extends AbstractComponent implements Shards
// can make the balance of the cluster better, so we check for that here
final boolean betterWeightThanCurrent = nodeWeight <= currentWeight;
boolean rebalanceConditionsMet = false;
boolean deltaAboveThreshold = false;
float weightWithShardAdded = Float.POSITIVE_INFINITY;
if (betterWeightThanCurrent) {
// get the delta between the weights of the node we are checking and the node that holds the shard
final float currentDelta = absDelta(nodeWeight, currentWeight);
float currentDelta = absDelta(nodeWeight, currentWeight);
// checks if the weight delta is above a certain threshold; if it is not above a certain threshold,
// then even though the node we are examining has a better weight and may make the cluster balance
// more even, it doesn't make sense to execute the heavyweight operation of relocating a shard unless
// the gains make it worth it, as defined by the threshold
deltaAboveThreshold = lessThan(currentDelta, threshold) == false;
boolean deltaAboveThreshold = lessThan(currentDelta, threshold) == false;
// simulate the weight of the node if we were to relocate the shard to it
weightWithShardAdded = weight.weightShardAdded(this, node, idxName);
float weightWithShardAdded = weight.weightShardAdded(this, node, idxName);
// calculate the delta of the weights of the two nodes if we were to add the shard to the
// node in question and move it away from the node that currently holds it.
final float proposedDelta = weightWithShardAdded - weight.weightShardRemoved(this, currentNode, idxName);
rebalanceConditionsMet = deltaAboveThreshold && proposedDelta < currentDelta;
float proposedDelta = weightWithShardAdded - weight.weightShardRemoved(this, currentNode, idxName);
boolean betterWeightWithShardAdded = proposedDelta < currentDelta;
rebalanceConditionsMet = deltaAboveThreshold && betterWeightWithShardAdded;
// if the simulated weight delta with the shard moved away is better than the weight delta
// with the shard remaining on the current node, and we are allowed to allocate to the
// node in question, then allow the rebalance
@ -410,26 +402,47 @@ public class BalancedShardsAllocator extends AbstractComponent implements Shards
// rebalance to the node, only will get overwritten if the decision here is to
// THROTTLE and we get a decision with YES on another node
rebalanceDecisionType = canAllocate.type();
assignedNodeId = node.getNodeId();
assignedNode = node;
}
}
nodeDecisions.put(node.getNodeId(), new NodeRebalanceResult(
rebalanceConditionsMet ? canAllocate.type() : Type.NO,
canAllocate,
betterWeightThanCurrent,
deltaAboveThreshold,
nodeWeight,
weightWithShardAdded)
Tuple<ModelNode, Decision> nodeResult = Tuple.tuple(node, canAllocate);
if (rebalanceConditionsMet) {
betterBalanceNodes.add(nodeResult);
} else if (betterWeightThanCurrent) {
sameBalanceNodes.add(nodeResult);
} else {
worseBalanceNodes.add(nodeResult);
}
}
int weightRanking = 0;
List<NodeAllocationResult> nodeDecisions = new ArrayList<>(modelNodes.length - 1);
for (Tuple<ModelNode, Decision> result : betterBalanceNodes) {
nodeDecisions.add(new NodeAllocationResult(
result.v1().routingNode.node(), AllocationDecision.fromDecisionType(result.v2().type()), result.v2(), ++weightRanking)
);
}
int currentNodeWeightRanking = ++weightRanking;
for (Tuple<ModelNode, Decision> result : sameBalanceNodes) {
AllocationDecision nodeDecision = result.v2().type() == Type.NO ? AllocationDecision.NO : AllocationDecision.WORSE_BALANCE;
nodeDecisions.add(new NodeAllocationResult(
result.v1().routingNode.node(), nodeDecision, result.v2(), currentNodeWeightRanking)
);
}
for (Tuple<ModelNode, Decision> result : worseBalanceNodes) {
AllocationDecision nodeDecision = result.v2().type() == Type.NO ? AllocationDecision.NO : AllocationDecision.WORSE_BALANCE;
nodeDecisions.add(new NodeAllocationResult(
result.v1().routingNode.node(), nodeDecision, result.v2(), ++weightRanking)
);
}
if (canRebalance.type() != Type.YES) {
return new RebalanceDecision(canRebalance, canRebalance.type(), "rebalancing is not allowed", null,
nodeDecisions, currentWeight);
if (canRebalance.type() != Type.YES || allocation.hasPendingAsyncFetch()) {
AllocationDecision allocationDecision = allocation.hasPendingAsyncFetch() ? AllocationDecision.FETCH_PENDING :
AllocationDecision.fromDecisionType(canRebalance.type());
return MoveDecision.cannotRebalance(canRebalance, allocationDecision, currentNodeWeightRanking, nodeDecisions);
} else {
return RebalanceDecision.decision(canRebalance, rebalanceDecisionType, assignedNodeId,
nodeDecisions, currentWeight, threshold);
return MoveDecision.rebalance(canRebalance, AllocationDecision.fromDecisionType(rebalanceDecisionType),
assignedNode != null ? assignedNode.routingNode.node() : null, currentNodeWeightRanking, nodeDecisions);
}
}
@ -632,9 +645,9 @@ public class BalancedShardsAllocator extends AbstractComponent implements Shards
for (Iterator<ShardRouting> it = allocation.routingNodes().nodeInterleavedShardIterator(); it.hasNext(); ) {
ShardRouting shardRouting = it.next();
final MoveDecision moveDecision = makeMoveDecision(shardRouting);
if (moveDecision.move()) {
if (moveDecision.isDecisionTaken() && moveDecision.forceMove()) {
final ModelNode sourceNode = nodes.get(shardRouting.currentNodeId());
final ModelNode targetNode = nodes.get(moveDecision.getAssignedNodeId());
final ModelNode targetNode = nodes.get(moveDecision.getTargetNode().getId());
sourceNode.removeShard(shardRouting);
Tuple<ShardRouting, ShardRouting> relocatingShards = routingNodes.relocateShard(shardRouting, targetNode.getNodeId(),
allocation.clusterInfo().getShardSize(shardRouting, ShardRouting.UNAVAILABLE_EXPECTED_SHARD_SIZE), allocation.changes());
@ -642,7 +655,7 @@ public class BalancedShardsAllocator extends AbstractComponent implements Shards
if (logger.isTraceEnabled()) {
logger.trace("Moved shard [{}] to node [{}]", shardRouting, targetNode.getRoutingNode());
}
} else if (moveDecision.cannotRemain()) {
} else if (moveDecision.isDecisionTaken() && moveDecision.canRemain() == false) {
logger.trace("[{}][{}] can't move", shardRouting.index(), shardRouting.id());
}
}
@ -654,11 +667,11 @@ public class BalancedShardsAllocator extends AbstractComponent implements Shards
* 1. If the shard is not started, no decision will be taken and {@link MoveDecision#isDecisionTaken()} will return false.
* 2. If the shard is allowed to remain on its current node, no attempt will be made to move the shard and
* {@link MoveDecision#canRemainDecision} will have a decision type of YES. All other fields in the object will be null.
* 3. If the shard is not allowed to remain on its current node, then {@link MoveDecision#finalDecision} will be populated
* with the decision of moving to another node. If {@link MoveDecision#finalDecision} returns YES, then
* {@link MoveDecision#assignedNodeId} will return a non-null value, otherwise the assignedNodeId will be null.
* 3. If the shard is not allowed to remain on its current node, then {@link MoveDecision#getAllocationDecision()} will be
* populated with the decision of moving to another node. If {@link MoveDecision#forceMove()} ()} returns {@code true}, then
* {@link MoveDecision#targetNode} will return a non-null value, otherwise the assignedNodeId will be null.
* 4. If the method is invoked in explain mode (e.g. from the cluster allocation explain APIs), then
* {@link MoveDecision#finalExplanation} and {@link MoveDecision#nodeDecisions} will have non-null values.
* {@link MoveDecision#nodeDecisions} will have a non-null value.
*/
public MoveDecision makeMoveDecision(final ShardRouting shardRouting) {
if (shardRouting.started() == false) {
@ -672,7 +685,7 @@ public class BalancedShardsAllocator extends AbstractComponent implements Shards
RoutingNode routingNode = sourceNode.getRoutingNode();
Decision canRemain = allocation.deciders().canRemain(shardRouting, routingNode, allocation);
if (canRemain.type() != Decision.Type.NO) {
return MoveDecision.stay(canRemain, explain);
return MoveDecision.stay(canRemain);
}
sorter.reset(shardRouting.getIndexName());
@ -684,14 +697,16 @@ public class BalancedShardsAllocator extends AbstractComponent implements Shards
*/
Type bestDecision = Type.NO;
RoutingNode targetNode = null;
final Map<String, NodeAllocationResult> nodeExplanationMap = explain ? new HashMap<>() : null;
final List<NodeAllocationResult> nodeExplanationMap = explain ? new ArrayList<>() : null;
int weightRanking = 0;
for (ModelNode currentNode : sorter.modelNodes) {
if (currentNode != sourceNode) {
RoutingNode target = currentNode.getRoutingNode();
// don't use canRebalance as we want hard filtering rules to apply. See #17698
Decision allocationDecision = allocation.deciders().canAllocate(shardRouting, target, allocation);
if (explain) {
nodeExplanationMap.put(currentNode.getNodeId(), new NodeAllocationResult(allocationDecision, sorter.weight(currentNode)));
nodeExplanationMap.add(new NodeAllocationResult(
currentNode.getRoutingNode().node(), allocationDecision, ++weightRanking));
}
// TODO maybe we can respect throttling here too?
if (allocationDecision.type().higherThan(bestDecision)) {
@ -708,8 +723,8 @@ public class BalancedShardsAllocator extends AbstractComponent implements Shards
}
}
return MoveDecision.decision(canRemain, bestDecision, explain, shardRouting.currentNodeId(),
targetNode != null ? targetNode.nodeId() : null, nodeExplanationMap);
return MoveDecision.cannotRemain(canRemain, AllocationDecision.fromDecisionType(bestDecision),
targetNode != null ? targetNode.node() : null, nodeExplanationMap);
}
/**
@ -793,11 +808,11 @@ public class BalancedShardsAllocator extends AbstractComponent implements Shards
for (int i = 0; i < primaryLength; i++) {
ShardRouting shard = primary[i];
AllocateUnassignedDecision allocationDecision = decideAllocateUnassigned(shard, throttledNodes);
final Type decisionType = allocationDecision.getFinalDecisionType();
final String assignedNodeId = allocationDecision.getAssignedNodeId();
final String assignedNodeId = allocationDecision.getTargetNode() != null ?
allocationDecision.getTargetNode().getId() : null;
final ModelNode minNode = assignedNodeId != null ? nodes.get(assignedNodeId) : null;
if (decisionType == Type.YES) {
if (allocationDecision.getAllocationDecision() == AllocationDecision.YES) {
if (logger.isTraceEnabled()) {
logger.trace("Assigned shard [{}] to [{}]", shard, minNode.getNodeId());
}
@ -816,12 +831,13 @@ public class BalancedShardsAllocator extends AbstractComponent implements Shards
} else {
// did *not* receive a YES decision
if (logger.isTraceEnabled()) {
logger.trace("No eligible node found to assign shard [{}] decision [{}]", shard, decisionType);
logger.trace("No eligible node found to assign shard [{}] allocation_status [{}]", shard,
allocationDecision.getAllocationStatus());
}
if (minNode != null) {
// throttle decision scenario
assert decisionType == Type.THROTTLE;
assert allocationDecision.getAllocationStatus() == AllocationStatus.DECIDERS_THROTTLED;
final long shardSize = DiskThresholdDecider.getExpectedShardSize(shard, allocation,
ShardRouting.UNAVAILABLE_EXPECTED_SHARD_SIZE);
minNode.addShard(shard.initialize(minNode.getNodeId(), null, shardSize));
@ -829,23 +845,22 @@ public class BalancedShardsAllocator extends AbstractComponent implements Shards
final Decision.Type nodeLevelDecision = deciders.canAllocate(node, allocation).type();
if (nodeLevelDecision != Type.YES) {
if (logger.isTraceEnabled()) {
logger.trace("Can not allocate on node [{}] remove from round decision [{}]", node, decisionType);
logger.trace("Can not allocate on node [{}] remove from round decision [{}]", node,
allocationDecision.getAllocationStatus());
}
assert nodeLevelDecision == Type.NO;
throttledNodes.add(minNode);
}
} else {
assert decisionType == Type.NO;
if (logger.isTraceEnabled()) {
logger.trace("No Node found to assign shard [{}]", shard);
}
}
UnassignedInfo.AllocationStatus allocationStatus = UnassignedInfo.AllocationStatus.fromDecision(decisionType);
unassigned.ignoreShard(shard, allocationStatus, allocation.changes());
unassigned.ignoreShard(shard, allocationDecision.getAllocationStatus(), allocation.changes());
if (!shard.primary()) { // we could not allocate it and we are a replica - check if we can ignore the other replicas
while(i < primaryLength-1 && comparator.compare(primary[i], primary[i+1]) == 0) {
unassigned.ignoreShard(primary[++i], allocationStatus, allocation.changes());
unassigned.ignoreShard(primary[++i], allocationDecision.getAllocationStatus(), allocation.changes());
}
}
}
@ -871,25 +886,26 @@ public class BalancedShardsAllocator extends AbstractComponent implements Shards
return AllocateUnassignedDecision.NOT_TAKEN;
}
final boolean explain = allocation.debugDecision();
Decision shardLevelDecision = allocation.deciders().canAllocate(shard, allocation);
if (shardLevelDecision.type() == Type.NO) {
if (shardLevelDecision.type() == Type.NO && explain == false) {
// NO decision for allocating the shard, irrespective of any particular node, so exit early
return AllocateUnassignedDecision.no(shardLevelDecision, explain("cannot allocate shard in its current state"));
return AllocateUnassignedDecision.no(AllocationStatus.DECIDERS_NO, null);
}
/* find an node with minimal weight we can allocate on*/
float minWeight = Float.POSITIVE_INFINITY;
ModelNode minNode = null;
Decision decision = null;
final boolean explain = allocation.debugDecision();
if (throttledNodes.size() >= nodes.size() && explain == false) {
// all nodes are throttled, so we know we won't be able to allocate this round,
// so if we are not in explain mode, short circuit
return AllocateUnassignedDecision.no(UnassignedInfo.AllocationStatus.DECIDERS_NO, null);
return AllocateUnassignedDecision.no(AllocationStatus.DECIDERS_NO, null);
}
/* Don't iterate over an identity hashset here the
* iteration order is different for each run and makes testing hard */
Map<String, NodeAllocationResult> nodeExplanationMap = explain ? new HashMap<>() : null;
List<Tuple<String, Float>> nodeWeights = explain ? new ArrayList<>() : null;
for (ModelNode node : nodes.values()) {
if ((throttledNodes.contains(node) || node.containsShard(shard)) && explain == false) {
// decision is NO without needing to check anything further, so short circuit
@ -905,7 +921,9 @@ public class BalancedShardsAllocator extends AbstractComponent implements Shards
Decision currentDecision = allocation.deciders().canAllocate(shard, node.getRoutingNode(), allocation);
if (explain) {
nodeExplanationMap.put(node.getNodeId(), new NodeAllocationResult(currentDecision, currentWeight));
nodeExplanationMap.put(node.getNodeId(),
new NodeAllocationResult(node.getRoutingNode().node(), currentDecision, 0));
nodeWeights.add(Tuple.tuple(node.getNodeId(), currentWeight));
}
if (currentDecision.type() == Type.YES || currentDecision.type() == Type.THROTTLE) {
final boolean updateMinNode;
@ -946,23 +964,24 @@ public class BalancedShardsAllocator extends AbstractComponent implements Shards
// decision was not set and a node was not assigned, so treat it as a NO decision
decision = Decision.NO;
}
List<NodeAllocationResult> nodeDecisions = null;
if (explain) {
nodeDecisions = new ArrayList<>();
// fill in the correct weight ranking, once we've been through all nodes
nodeWeights.sort((nodeWeight1, nodeWeight2) -> Float.compare(nodeWeight1.v2(), nodeWeight2.v2()));
int weightRanking = 0;
for (Tuple<String, Float> nodeWeight : nodeWeights) {
NodeAllocationResult current = nodeExplanationMap.get(nodeWeight.v1());
nodeDecisions.add(new NodeAllocationResult(current.getNode(), current.getCanAllocateDecision(), ++weightRanking));
}
}
return AllocateUnassignedDecision.fromDecision(
decision,
minNode != null ? minNode.getNodeId() : null,
explain,
nodeExplanationMap
minNode != null ? minNode.routingNode.node() : null,
nodeDecisions
);
}
// provide an explanation, if in explain mode
private String explain(String explanation) {
if (allocation.debugDecision()) {
return explanation;
} else {
return null;
}
}
/**
* Tries to find a relocation from the max node to the minimal node for an arbitrary shard of the given index on the
* balance model. Iff this method returns a <code>true</code> the relocation has already been executed on the
@ -1223,5 +1242,4 @@ public class BalancedShardsAllocator extends AbstractComponent implements Shards
return weights[weights.length - 1] - weights[0];
}
}
}

View File

@ -22,6 +22,7 @@ package org.elasticsearch.cluster.routing.allocation.decider;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.io.stream.Writeable;
import org.elasticsearch.common.xcontent.ToXContent;
import org.elasticsearch.common.xcontent.XContentBuilder;
@ -38,7 +39,7 @@ import java.util.Objects;
*
* @see AllocationDecider
*/
public abstract class Decision implements ToXContent {
public abstract class Decision implements ToXContent, Writeable {
public static final Decision ALWAYS = new Single(Type.YES);
public static final Decision YES = new Single(Type.YES);
@ -57,26 +58,6 @@ public abstract class Decision implements ToXContent {
return new Single(type, label, explanation, explanationParams);
}
public static void writeTo(Decision decision, StreamOutput out) throws IOException {
if (decision instanceof Multi) {
// Flag specifying whether it is a Multi or Single Decision
out.writeBoolean(true);
out.writeVInt(((Multi) decision).decisions.size());
for (Decision d : ((Multi) decision).decisions) {
writeTo(d, out);
}
} else {
// Flag specifying whether it is a Multi or Single Decision
out.writeBoolean(false);
Single d = ((Single) decision);
Type.writeTo(d.type, out);
out.writeOptionalString(d.label);
// Flatten explanation on serialization, so that explanationParams
// do not need to be serialized
out.writeOptionalString(d.getExplanation());
}
}
public static Decision readFrom(StreamInput in) throws IOException {
// Determine whether to read a Single or Multi Decision
if (in.readBoolean()) {
@ -100,10 +81,16 @@ public abstract class Decision implements ToXContent {
* This enumeration defines the
* possible types of decisions
*/
public enum Type {
YES,
NO,
THROTTLE;
public enum Type implements Writeable {
YES(1),
THROTTLE(2),
NO(0);
private final int id;
Type(int id) {
this.id = id;
}
public static Type resolve(String s) {
return Type.valueOf(s.toUpperCase(Locale.ROOT));
@ -123,20 +110,9 @@ public abstract class Decision implements ToXContent {
}
}
public static void writeTo(Type type, StreamOutput out) throws IOException {
switch (type) {
case NO:
out.writeVInt(0);
break;
case YES:
out.writeVInt(1);
break;
case THROTTLE:
out.writeVInt(2);
break;
default:
throw new IllegalArgumentException("Invalid Type [" + type + "]");
}
@Override
public void writeTo(StreamOutput out) throws IOException {
out.writeVInt(id);
}
public boolean higherThan(Type other) {
@ -275,6 +251,16 @@ public abstract class Decision implements ToXContent {
builder.endObject();
return builder;
}
@Override
public void writeTo(StreamOutput out) throws IOException {
out.writeBoolean(false); // flag specifying its a single decision
type.writeTo(out);
out.writeOptionalString(label);
// Flatten explanation on serialization, so that explanationParams
// do not need to be serialized
out.writeOptionalString(getExplanation());
}
}
/**
@ -351,12 +337,19 @@ public abstract class Decision implements ToXContent {
@Override
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
builder.startArray("decisions");
for (Decision d : decisions) {
d.toXContent(builder, params);
}
builder.endArray();
return builder;
}
@Override
public void writeTo(StreamOutput out) throws IOException {
out.writeBoolean(true); // flag indicating it is a multi decision
out.writeVInt(getDecisions().size());
for (Decision d : getDecisions()) {
d.writeTo(out);
}
}
}
}

View File

@ -157,19 +157,19 @@ public class EnableAllocationDecider extends AllocationDecider {
case ALL:
return allocation.decision(Decision.YES, NAME, "all rebalancing is allowed");
case NONE:
return allocation.decision(Decision.NO, NAME, "no rebalancing is allowed due to {}", setting(enable, usedIndexSetting));
return allocation.decision(Decision.NO, NAME, "no rebalancing is allowed due to %s", setting(enable, usedIndexSetting));
case PRIMARIES:
if (shardRouting.primary()) {
return allocation.decision(Decision.YES, NAME, "primary rebalancing is allowed");
} else {
return allocation.decision(Decision.NO, NAME, "replica rebalancing is forbidden due to {}",
return allocation.decision(Decision.NO, NAME, "replica rebalancing is forbidden due to %s",
setting(enable, usedIndexSetting));
}
case REPLICAS:
if (shardRouting.primary() == false) {
return allocation.decision(Decision.YES, NAME, "replica rebalancing is allowed");
} else {
return allocation.decision(Decision.NO, NAME, "primary rebalancing is forbidden due to {}",
return allocation.decision(Decision.NO, NAME, "primary rebalancing is forbidden due to %s",
setting(enable, usedIndexSetting));
}
default:

View File

@ -63,16 +63,16 @@ public class MaxRetryAllocationDecider extends AllocationDecider {
// if we are called via the _reroute API we ignore the failure counter and try to allocate
// this improves the usability since people don't need to raise the limits to issue retries since a simple _reroute call is
// enough to manually retry.
decision = allocation.decision(Decision.YES, NAME, "shard has already failed allocating ["
+ unassignedInfo.getNumFailedAllocations() + "] times vs. [" + maxRetry + "] retries allowed "
+ unassignedInfo.toString() + " - retrying once on manual allocation");
decision = allocation.decision(Decision.YES, NAME, "shard has exceeded the maximum number of retries [%d] on " +
"failed allocation attempts - retrying once due to a manual reroute command, [%s]",
maxRetry, unassignedInfo.toString());
} else if (unassignedInfo.getNumFailedAllocations() >= maxRetry) {
decision = allocation.decision(Decision.NO, NAME, "shard has already failed allocating ["
+ unassignedInfo.getNumFailedAllocations() + "] times vs. [" + maxRetry + "] retries allowed "
+ unassignedInfo.toString() + " - manually call [/_cluster/reroute?retry_failed=true] to retry");
decision = allocation.decision(Decision.NO, NAME, "shard has exceeded the maximum number of retries [%d] on " +
"failed allocation attempts - manually call [/_cluster/reroute?retry_failed=true] to retry, [%s]",
maxRetry, unassignedInfo.toString());
} else {
decision = allocation.decision(Decision.YES, NAME, "shard has already failed allocating ["
+ unassignedInfo.getNumFailedAllocations() + "] times but [" + maxRetry + "] retries are allowed");
decision = allocation.decision(Decision.YES, NAME, "shard has failed allocating [%d] times but [%d] retries are allowed",
unassignedInfo.getNumFailedAllocations(), maxRetry);
}
} else {
decision = allocation.decision(Decision.YES, NAME, "shard has no previous failures");

View File

@ -67,7 +67,7 @@ public class SameShardAllocationDecider extends AllocationDecider {
shardRouting.toString());
} else {
return allocation.decision(Decision.NO, NAME,
"the shard cannot be allocated to the same node on which a copy of the shard [%s] already exists",
"the shard cannot be allocated to the same node on which a copy of the shard already exists [%s]",
assignedShard.toString());
}
}

View File

@ -23,8 +23,8 @@ import org.apache.logging.log4j.Logger;
import org.elasticsearch.cluster.routing.RoutingNodes;
import org.elasticsearch.cluster.routing.ShardRouting;
import org.elasticsearch.cluster.routing.allocation.AllocateUnassignedDecision;
import org.elasticsearch.cluster.routing.allocation.AllocationDecision;
import org.elasticsearch.cluster.routing.allocation.RoutingAllocation;
import org.elasticsearch.cluster.routing.allocation.decider.Decision;
import org.elasticsearch.common.component.AbstractComponent;
import org.elasticsearch.common.settings.Settings;
@ -60,8 +60,8 @@ public abstract class BaseGatewayShardAllocator extends AbstractComponent {
continue;
}
if (allocateUnassignedDecision.getFinalDecisionSafe() == Decision.Type.YES) {
unassignedIterator.initialize(allocateUnassignedDecision.getAssignedNodeId(),
if (allocateUnassignedDecision.getAllocationDecision() == AllocationDecision.YES) {
unassignedIterator.initialize(allocateUnassignedDecision.getTargetNode().getId(),
allocateUnassignedDecision.getAllocationId(),
shard.primary() ? ShardRouting.UNAVAILABLE_EXPECTED_SHARD_SIZE :
allocation.clusterInfo().getShardSize(shard, ShardRouting.UNAVAILABLE_EXPECTED_SHARD_SIZE),

View File

@ -204,5 +204,10 @@ public class GatewayAllocator extends AbstractComponent {
}
return shardStores;
}
@Override
protected boolean hasInitiatedFetching(ShardRouting shard) {
return asyncFetchStore.get(shard.shardId()) != null;
}
}
}

View File

@ -32,6 +32,8 @@ import org.elasticsearch.cluster.routing.RoutingNodes;
import org.elasticsearch.cluster.routing.ShardRouting;
import org.elasticsearch.cluster.routing.UnassignedInfo.AllocationStatus;
import org.elasticsearch.cluster.routing.allocation.AllocateUnassignedDecision;
import org.elasticsearch.cluster.routing.allocation.NodeAllocationResult;
import org.elasticsearch.cluster.routing.allocation.NodeAllocationResult.ShardStoreInfo;
import org.elasticsearch.cluster.routing.allocation.RoutingAllocation;
import org.elasticsearch.cluster.routing.allocation.decider.Decision;
import org.elasticsearch.cluster.routing.allocation.decider.Decision.Type;
@ -44,15 +46,14 @@ import org.elasticsearch.gateway.TransportNodesListGatewayStartedShards.NodeGate
import org.elasticsearch.index.shard.ShardStateMetaData;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.Comparator;
import java.util.LinkedHashMap;
import java.util.LinkedList;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.function.Function;
import java.util.stream.Collectors;
import java.util.stream.Stream;
/**
* The primary shard allocator allocates unassigned primary shards to nodes that hold
@ -107,7 +108,7 @@ public abstract class PrimaryShardAllocator extends BaseGatewayShardAllocator {
&& shard.unassigned() // must be unassigned
// only handle either an existing store or a snapshot recovery
&& (shard.recoverySource().getType() == RecoverySource.Type.EXISTING_STORE
|| shard.recoverySource().getType() == RecoverySource.Type.SNAPSHOT);
|| shard.recoverySource().getType() == RecoverySource.Type.SNAPSHOT);
}
@Override
@ -123,8 +124,9 @@ public abstract class PrimaryShardAllocator extends BaseGatewayShardAllocator {
final FetchResult<NodeGatewayStartedShards> shardState = fetchData(unassignedShard, allocation);
if (shardState.hasData() == false) {
allocation.setHasPendingAsyncFetch();
return AllocateUnassignedDecision.no(AllocationStatus.FETCHING_SHARD_DATA,
explain ? "still fetching shard state from the nodes in the cluster" : null);
if (explain == false) {
return AllocateUnassignedDecision.no(AllocationStatus.FETCHING_SHARD_DATA, null);
}
}
// don't create a new IndexSetting object for every shard as this could cause a lot of garbage
@ -179,82 +181,80 @@ public abstract class PrimaryShardAllocator extends BaseGatewayShardAllocator {
// this shard will be picked up when the node joins and we do another allocation reroute
logger.debug("[{}][{}]: not allocating, number_of_allocated_shards_found [{}]",
unassignedShard.index(), unassignedShard.id(), nodeShardsResult.allocationsFound);
return AllocateUnassignedDecision.no(AllocationStatus.NO_VALID_SHARD_COPY,
explain ? "shard was previously allocated, but no valid shard copy could be found amongst the nodes in the cluster" : null);
return AllocateUnassignedDecision.no(AllocationStatus.NO_VALID_SHARD_COPY, null);
}
}
final NodesToAllocate nodesToAllocate = buildNodesToAllocate(
NodesToAllocate nodesToAllocate = buildNodesToAllocate(
allocation, nodeShardsResult.orderedAllocationCandidates, unassignedShard, false
);
DiscoveryNode node = null;
String allocationId = null;
boolean throttled = false;
if (nodesToAllocate.yesNodeShards.isEmpty() == false) {
DecidedNode decidedNode = nodesToAllocate.yesNodeShards.get(0);
logger.debug("[{}][{}]: allocating [{}] to [{}] on primary allocation",
unassignedShard.index(), unassignedShard.id(), unassignedShard, decidedNode.nodeShardState.getNode());
final String nodeId = decidedNode.nodeShardState.getNode().getId();
return AllocateUnassignedDecision.yes(nodeId,
"the allocation deciders returned a YES decision to allocate to node [" + nodeId + "]",
decidedNode.nodeShardState.allocationId(),
buildNodeDecisions(nodesToAllocate, explain));
node = decidedNode.nodeShardState.getNode();
allocationId = decidedNode.nodeShardState.allocationId();
} else if (nodesToAllocate.throttleNodeShards.isEmpty() && !nodesToAllocate.noNodeShards.isEmpty()) {
// The deciders returned a NO decision for all nodes with shard copies, so we check if primary shard
// can be force-allocated to one of the nodes.
final NodesToAllocate nodesToForceAllocate = buildNodesToAllocate(
allocation, nodeShardsResult.orderedAllocationCandidates, unassignedShard, true
);
if (nodesToForceAllocate.yesNodeShards.isEmpty() == false) {
final DecidedNode decidedNode = nodesToForceAllocate.yesNodeShards.get(0);
nodesToAllocate = buildNodesToAllocate(allocation, nodeShardsResult.orderedAllocationCandidates, unassignedShard, true);
if (nodesToAllocate.yesNodeShards.isEmpty() == false) {
final DecidedNode decidedNode = nodesToAllocate.yesNodeShards.get(0);
final NodeGatewayStartedShards nodeShardState = decidedNode.nodeShardState;
logger.debug("[{}][{}]: allocating [{}] to [{}] on forced primary allocation",
unassignedShard.index(), unassignedShard.id(), unassignedShard, nodeShardState.getNode());
final String nodeId = nodeShardState.getNode().getId();
return AllocateUnassignedDecision.yes(nodeId,
"allocating the primary shard to node [" + nodeId+ "], which has a complete copy of the shard data",
nodeShardState.allocationId(),
buildNodeDecisions(nodesToForceAllocate, explain));
} else if (nodesToForceAllocate.throttleNodeShards.isEmpty() == false) {
node = nodeShardState.getNode();
allocationId = nodeShardState.allocationId();
} else if (nodesToAllocate.throttleNodeShards.isEmpty() == false) {
logger.debug("[{}][{}]: throttling allocation [{}] to [{}] on forced primary allocation",
unassignedShard.index(), unassignedShard.id(), unassignedShard, nodesToForceAllocate.throttleNodeShards);
return AllocateUnassignedDecision.throttle(
explain ? "allocation throttled as all nodes to which the shard may be force allocated are busy with other recoveries" : null,
buildNodeDecisions(nodesToForceAllocate, explain));
unassignedShard.index(), unassignedShard.id(), unassignedShard, nodesToAllocate.throttleNodeShards);
throttled = true;
} else {
logger.debug("[{}][{}]: forced primary allocation denied [{}]",
unassignedShard.index(), unassignedShard.id(), unassignedShard);
return AllocateUnassignedDecision.no(AllocationStatus.DECIDERS_NO,
explain ? "all nodes that hold a valid shard copy returned a NO decision, and force allocation is not permitted" : null,
buildNodeDecisions(nodesToForceAllocate, explain));
}
} else {
// we are throttling this, since we are allowed to allocate to this node but there are enough allocations
// taking place on the node currently, ignore it for now
logger.debug("[{}][{}]: throttling allocation [{}] to [{}] on primary allocation",
unassignedShard.index(), unassignedShard.id(), unassignedShard, nodesToAllocate.throttleNodeShards);
return AllocateUnassignedDecision.throttle(
explain ? "allocation throttled as all nodes to which the shard may be allocated are busy with other recoveries" : null,
buildNodeDecisions(nodesToAllocate, explain));
throttled = true;
}
List<NodeAllocationResult> nodeResults = null;
if (explain) {
nodeResults = buildNodeDecisions(nodesToAllocate, inSyncAllocationIds);
}
if (allocation.hasPendingAsyncFetch()) {
return AllocateUnassignedDecision.no(AllocationStatus.FETCHING_SHARD_DATA, nodeResults);
} else if (node != null) {
return AllocateUnassignedDecision.yes(node, allocationId, nodeResults, false);
} else if (throttled) {
return AllocateUnassignedDecision.throttle(nodeResults);
} else {
return AllocateUnassignedDecision.no(AllocationStatus.DECIDERS_NO, nodeResults, true);
}
}
/**
* Builds a map of nodes to the corresponding allocation decisions for those nodes.
*/
private static Map<String, Decision> buildNodeDecisions(NodesToAllocate nodesToAllocate, boolean explain) {
if (explain == false) {
// not in explain mode, no need to return node level decisions
return null;
}
Map<String, Decision> nodeDecisions = new LinkedHashMap<>();
for (final DecidedNode decidedNode : nodesToAllocate.yesNodeShards) {
nodeDecisions.put(decidedNode.nodeShardState.getNode().getId(), decidedNode.decision);
}
for (final DecidedNode decidedNode : nodesToAllocate.throttleNodeShards) {
nodeDecisions.put(decidedNode.nodeShardState.getNode().getId(), decidedNode.decision);
}
for (final DecidedNode decidedNode : nodesToAllocate.noNodeShards) {
nodeDecisions.put(decidedNode.nodeShardState.getNode().getId(), decidedNode.decision);
}
return nodeDecisions;
private static List<NodeAllocationResult> buildNodeDecisions(NodesToAllocate nodesToAllocate, Set<String> inSyncAllocationIds) {
return Stream.of(nodesToAllocate.yesNodeShards, nodesToAllocate.throttleNodeShards, nodesToAllocate.noNodeShards)
.flatMap(Collection::stream)
.map(dnode -> new NodeAllocationResult(dnode.nodeShardState.getNode(),
shardStoreInfo(dnode.nodeShardState, inSyncAllocationIds),
dnode.decision))
.collect(Collectors.toList());
}
private static ShardStoreInfo shardStoreInfo(NodeGatewayStartedShards nodeShardState, Set<String> inSyncAllocationIds) {
final Exception storeErr = nodeShardState.storeException();
final boolean inSync = nodeShardState.allocationId() != null && inSyncAllocationIds.contains(nodeShardState.allocationId());
return new ShardStoreInfo(nodeShardState.allocationId(), inSync, storeErr);
}
private static final Comparator<NodeGatewayStartedShards> NO_STORE_EXCEPTION_FIRST_COMPARATOR =

View File

@ -24,6 +24,7 @@ import com.carrotsearch.hppc.ObjectLongMap;
import com.carrotsearch.hppc.cursors.ObjectCursor;
import com.carrotsearch.hppc.cursors.ObjectLongCursor;
import org.apache.logging.log4j.Logger;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.cluster.metadata.MetaData;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.routing.RoutingNode;
@ -32,12 +33,15 @@ import org.elasticsearch.cluster.routing.ShardRouting;
import org.elasticsearch.cluster.routing.UnassignedInfo;
import org.elasticsearch.cluster.routing.UnassignedInfo.AllocationStatus;
import org.elasticsearch.cluster.routing.allocation.AllocateUnassignedDecision;
import org.elasticsearch.cluster.routing.allocation.NodeAllocationResult;
import org.elasticsearch.cluster.routing.allocation.NodeAllocationResult.ShardStoreInfo;
import org.elasticsearch.cluster.routing.allocation.RoutingAllocation;
import org.elasticsearch.cluster.routing.allocation.decider.Decision;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.ByteSizeValue;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.index.store.StoreFileMetaData;
import org.elasticsearch.indices.store.TransportNodesListShardStoreMetaData;
import org.elasticsearch.indices.store.TransportNodesListShardStoreMetaData.NodeStoreFilesMetaData;
@ -48,6 +52,8 @@ import java.util.List;
import java.util.Map;
import java.util.Objects;
import static org.elasticsearch.cluster.routing.UnassignedInfo.INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING;
public abstract class ReplicaShardAllocator extends BaseGatewayShardAllocator {
public ReplicaShardAllocator(Settings settings) {
@ -150,24 +156,36 @@ public abstract class ReplicaShardAllocator extends BaseGatewayShardAllocator {
final RoutingNodes routingNodes = allocation.routingNodes();
final boolean explain = allocation.debugDecision();
// pre-check if it can be allocated to any node that currently exists, so we won't list the store for it for nothing
Tuple<Decision, Map<String, Decision>> allocateDecision = canBeAllocatedToAtLeastOneNode(unassignedShard, allocation, explain);
if (allocateDecision.v1().type() != Decision.Type.YES) {
Tuple<Decision, Map<String, NodeAllocationResult>> result = canBeAllocatedToAtLeastOneNode(unassignedShard, allocation);
Decision allocateDecision = result.v1();
if (allocateDecision.type() != Decision.Type.YES
&& (explain == false || hasInitiatedFetching(unassignedShard) == false)) {
// only return early if we are not in explain mode, or we are in explain mode but we have not
// yet attempted to fetch any shard data
logger.trace("{}: ignoring allocation, can't be allocated on any node", unassignedShard);
return AllocateUnassignedDecision.no(UnassignedInfo.AllocationStatus.fromDecision(allocateDecision.v1().type()),
explain ? "all nodes returned a " + allocateDecision.v1().type() + " decision for allocating the replica shard" : null,
allocateDecision.v2());
return AllocateUnassignedDecision.no(UnassignedInfo.AllocationStatus.fromDecision(allocateDecision.type()),
result.v2() != null ? new ArrayList<>(result.v2().values()) : null);
}
AsyncShardFetch.FetchResult<NodeStoreFilesMetaData> shardStores = fetchData(unassignedShard, allocation);
if (shardStores.hasData() == false) {
logger.trace("{}: ignoring allocation, still fetching shard stores", unassignedShard);
allocation.setHasPendingAsyncFetch();
return AllocateUnassignedDecision.no(AllocationStatus.FETCHING_SHARD_DATA,
explain ? "still fetching shard state from the nodes in the cluster" : null);
List<NodeAllocationResult> nodeDecisions = null;
if (explain) {
nodeDecisions = buildDecisionsForAllNodes(unassignedShard, allocation);
}
return AllocateUnassignedDecision.no(AllocationStatus.FETCHING_SHARD_DATA, nodeDecisions);
}
ShardRouting primaryShard = routingNodes.activePrimary(unassignedShard.shardId());
assert primaryShard != null : "the replica shard can be allocated on at least one node, so there must be an active primary";
if (primaryShard == null) {
assert explain : "primary should only be null here if we are in explain mode, so we didn't " +
"exit early when canBeAllocatedToAtLeastOneNode didn't return a YES decision";
return AllocateUnassignedDecision.no(UnassignedInfo.AllocationStatus.fromDecision(allocateDecision.type()),
new ArrayList<>(result.v2().values()));
}
TransportNodesListShardStoreMetaData.StoreFilesMetaData primaryStore = findStore(primaryShard, allocation, shardStores);
if (primaryStore == null) {
// if we can't find the primary data, it is probably because the primary shard is corrupted (and listing failed)
@ -181,7 +199,10 @@ public abstract class ReplicaShardAllocator extends BaseGatewayShardAllocator {
MatchingNodes matchingNodes = findMatchingNodes(unassignedShard, allocation, primaryStore, shardStores, explain);
assert explain == false || matchingNodes.nodeDecisions != null : "in explain mode, we must have individual node decisions";
if (matchingNodes.getNodeWithHighestMatch() != null) {
List<NodeAllocationResult> nodeDecisions = augmentExplanationsWithStoreInfo(result.v2(), matchingNodes.nodeDecisions);
if (allocateDecision.type() != Decision.Type.YES) {
return AllocateUnassignedDecision.no(UnassignedInfo.AllocationStatus.fromDecision(allocateDecision.type()), nodeDecisions);
} else if (matchingNodes.getNodeWithHighestMatch() != null) {
RoutingNode nodeWithHighestMatch = allocation.routingNodes().node(matchingNodes.getNodeWithHighestMatch().getId());
// we only check on THROTTLE since we checked before before on NO
Decision decision = allocation.deciders().canAllocate(unassignedShard, nodeWithHighestMatch, allocation);
@ -189,44 +210,60 @@ public abstract class ReplicaShardAllocator extends BaseGatewayShardAllocator {
logger.debug("[{}][{}]: throttling allocation [{}] to [{}] in order to reuse its unallocated persistent store",
unassignedShard.index(), unassignedShard.id(), unassignedShard, nodeWithHighestMatch.node());
// we are throttling this, as we have enough other shards to allocate to this node, so ignore it for now
return AllocateUnassignedDecision.throttle(
explain ? "returned a THROTTLE decision on each node that has an existing copy of the shard, so waiting to re-use one of those copies" : null,
matchingNodes.nodeDecisions);
return AllocateUnassignedDecision.throttle(nodeDecisions);
} else {
logger.debug("[{}][{}]: allocating [{}] to [{}] in order to reuse its unallocated persistent store",
unassignedShard.index(), unassignedShard.id(), unassignedShard, nodeWithHighestMatch.node());
// we found a match
return AllocateUnassignedDecision.yes(nodeWithHighestMatch.nodeId(),
"allocating to node [" + nodeWithHighestMatch.nodeId() + "] in order to re-use its unallocated persistent store",
null,
matchingNodes.nodeDecisions);
return AllocateUnassignedDecision.yes(nodeWithHighestMatch.node(), null, nodeDecisions, true);
}
} else if (matchingNodes.hasAnyData() == false && unassignedShard.unassignedInfo().isDelayed()) {
// if we didn't manage to find *any* data (regardless of matching sizes), and the replica is
// unassigned due to a node leaving, so we delay allocation of this replica to see if the
// node with the shard copy will rejoin so we can re-use the copy it has
logger.debug("{}: allocation of [{}] is delayed", unassignedShard.shardId(), unassignedShard);
return AllocateUnassignedDecision.no(AllocationStatus.DELAYED_ALLOCATION,
explain ? "not allocating this shard, no nodes contain data for the replica and allocation is delayed" : null);
long remainingDelayMillis = 0L;
long totalDelayMillis = 0L;
if (explain) {
UnassignedInfo unassignedInfo = unassignedShard.unassignedInfo();
MetaData metadata = allocation.metaData();
IndexMetaData indexMetaData = metadata.index(unassignedShard.index());
totalDelayMillis = INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING.get(indexMetaData.getSettings()).getMillis();
long remainingDelayNanos = unassignedInfo.getRemainingDelay(System.nanoTime(), indexMetaData.getSettings());
remainingDelayMillis = TimeValue.timeValueNanos(remainingDelayNanos).millis();
}
return AllocateUnassignedDecision.delayed(remainingDelayMillis, totalDelayMillis, nodeDecisions);
}
return AllocateUnassignedDecision.NOT_TAKEN;
}
/**
* Builds decisions for all nodes in the cluster, so that the explain API can provide information on
* allocation decisions for each node, while still waiting to allocate the shard (e.g. due to fetching shard data).
*/
private List<NodeAllocationResult> buildDecisionsForAllNodes(ShardRouting shard, RoutingAllocation allocation) {
List<NodeAllocationResult> results = new ArrayList<>();
for (RoutingNode node : allocation.routingNodes()) {
Decision decision = allocation.deciders().canAllocate(shard, node, allocation);
results.add(new NodeAllocationResult(node.node(), null, decision));
}
return results;
}
/**
* Determines if the shard can be allocated on at least one node based on the allocation deciders.
*
* Returns the best allocation decision for allocating the shard on any node (i.e. YES if at least one
* node decided YES, THROTTLE if at least one node decided THROTTLE, and NO if none of the nodes decided
* YES or THROTTLE). If the explain flag is turned on AND the decision is NO or THROTTLE, then this method
* also returns a map of nodes to decisions (second value in the tuple) to use for explanations; if the explain
* flag is off, the second value in the return tuple will be null.
* YES or THROTTLE). If in explain mode, also returns the node-level explanations as the second element
* in the returned tuple.
*/
private Tuple<Decision, Map<String, Decision>> canBeAllocatedToAtLeastOneNode(ShardRouting shard,
RoutingAllocation allocation,
boolean explain) {
private Tuple<Decision, Map<String, NodeAllocationResult>> canBeAllocatedToAtLeastOneNode(ShardRouting shard,
RoutingAllocation allocation) {
Decision madeDecision = Decision.NO;
Map<String, Decision> nodeDecisions = new HashMap<>();
final boolean explain = allocation.debugDecision();
Map<String, NodeAllocationResult> nodeDecisions = explain ? new HashMap<>() : null;
for (ObjectCursor<DiscoveryNode> cursor : allocation.nodes().getDataNodes().values()) {
RoutingNode node = allocation.routingNodes().node(cursor.value.getId());
if (node == null) {
@ -235,16 +272,40 @@ public abstract class ReplicaShardAllocator extends BaseGatewayShardAllocator {
// if we can't allocate it on a node, ignore it, for example, this handles
// cases for only allocating a replica after a primary
Decision decision = allocation.deciders().canAllocate(shard, node, allocation);
if (explain) {
nodeDecisions.put(node.nodeId(), decision);
}
if (decision.type() == Decision.Type.YES) {
return Tuple.tuple(decision, null);
if (decision.type() == Decision.Type.YES && madeDecision.type() != Decision.Type.YES) {
if (explain) {
madeDecision = decision;
} else {
return Tuple.tuple(decision, nodeDecisions);
}
} else if (madeDecision.type() == Decision.Type.NO && decision.type() == Decision.Type.THROTTLE) {
madeDecision = decision;
}
if (explain) {
nodeDecisions.put(node.nodeId(), new NodeAllocationResult(node.node(), null, decision));
}
}
return Tuple.tuple(madeDecision, explain ? nodeDecisions : null);
return Tuple.tuple(madeDecision, nodeDecisions);
}
/**
* Takes the store info for nodes that have a shard store and adds them to the node decisions,
* leaving the node explanations untouched for those nodes that do not have any store information.
*/
private List<NodeAllocationResult> augmentExplanationsWithStoreInfo(Map<String, NodeAllocationResult> nodeDecisions,
Map<String, NodeAllocationResult> withShardStores) {
if (nodeDecisions == null || withShardStores == null) {
return null;
}
List<NodeAllocationResult> augmented = new ArrayList<>();
for (Map.Entry<String, NodeAllocationResult> entry : nodeDecisions.entrySet()) {
if (withShardStores.containsKey(entry.getKey())) {
augmented.add(withShardStores.get(entry.getKey()));
} else {
augmented.add(entry.getValue());
}
}
return augmented;
}
/**
@ -268,7 +329,7 @@ public abstract class ReplicaShardAllocator extends BaseGatewayShardAllocator {
AsyncShardFetch.FetchResult<NodeStoreFilesMetaData> data,
boolean explain) {
ObjectLongMap<DiscoveryNode> nodesToSize = new ObjectLongHashMap<>();
Map<String, Decision> nodeDecisions = new HashMap<>();
Map<String, NodeAllocationResult> nodeDecisions = explain ? new HashMap<>() : null;
for (Map.Entry<DiscoveryNode, NodeStoreFilesMetaData> nodeStoreEntry : data.getData().entrySet()) {
DiscoveryNode discoNode = nodeStoreEntry.getKey();
TransportNodesListShardStoreMetaData.StoreFilesMetaData storeFilesMetaData = nodeStoreEntry.getValue().storeFilesMetaData();
@ -286,46 +347,68 @@ public abstract class ReplicaShardAllocator extends BaseGatewayShardAllocator {
// we only check for NO, since if this node is THROTTLING and it has enough "same data"
// then we will try and assign it next time
Decision decision = allocation.deciders().canAllocate(shard, node, allocation);
long matchingBytes = -1;
if (explain) {
nodeDecisions.put(node.nodeId(), decision);
matchingBytes = computeMatchingBytes(primaryStore, storeFilesMetaData);
ShardStoreInfo shardStoreInfo = new ShardStoreInfo(matchingBytes);
nodeDecisions.put(node.nodeId(), new NodeAllocationResult(discoNode, shardStoreInfo, decision));
}
if (decision.type() == Decision.Type.NO) {
continue;
}
String primarySyncId = primaryStore.syncId();
String replicaSyncId = storeFilesMetaData.syncId();
// see if we have a sync id we can make use of
if (replicaSyncId != null && replicaSyncId.equals(primarySyncId)) {
logger.trace("{}: node [{}] has same sync id {} as primary", shard, discoNode.getName(), replicaSyncId);
nodesToSize.put(discoNode, Long.MAX_VALUE);
} else {
long sizeMatched = 0;
for (StoreFileMetaData storeFileMetaData : storeFilesMetaData) {
String metaDataFileName = storeFileMetaData.name();
if (primaryStore.fileExists(metaDataFileName) && primaryStore.file(metaDataFileName).isSame(storeFileMetaData)) {
sizeMatched += storeFileMetaData.length();
}
if (matchingBytes < 0) {
matchingBytes = computeMatchingBytes(primaryStore, storeFilesMetaData);
}
nodesToSize.put(discoNode, matchingBytes);
if (logger.isTraceEnabled()) {
if (matchingBytes == Long.MAX_VALUE) {
logger.trace("{}: node [{}] has same sync id {} as primary", shard, discoNode.getName(), storeFilesMetaData.syncId());
} else {
logger.trace("{}: node [{}] has [{}/{}] bytes of re-usable data",
shard, discoNode.getName(), new ByteSizeValue(matchingBytes), matchingBytes);
}
logger.trace("{}: node [{}] has [{}/{}] bytes of re-usable data",
shard, discoNode.getName(), new ByteSizeValue(sizeMatched), sizeMatched);
nodesToSize.put(discoNode, sizeMatched);
}
}
return new MatchingNodes(nodesToSize, explain ? nodeDecisions : null);
return new MatchingNodes(nodesToSize, nodeDecisions);
}
private static long computeMatchingBytes(TransportNodesListShardStoreMetaData.StoreFilesMetaData primaryStore,
TransportNodesListShardStoreMetaData.StoreFilesMetaData storeFilesMetaData) {
String primarySyncId = primaryStore.syncId();
String replicaSyncId = storeFilesMetaData.syncId();
// see if we have a sync id we can make use of
if (replicaSyncId != null && replicaSyncId.equals(primarySyncId)) {
return Long.MAX_VALUE;
} else {
long sizeMatched = 0;
for (StoreFileMetaData storeFileMetaData : storeFilesMetaData) {
String metaDataFileName = storeFileMetaData.name();
if (primaryStore.fileExists(metaDataFileName) && primaryStore.file(metaDataFileName).isSame(storeFileMetaData)) {
sizeMatched += storeFileMetaData.length();
}
}
return sizeMatched;
}
}
protected abstract AsyncShardFetch.FetchResult<NodeStoreFilesMetaData> fetchData(ShardRouting shard, RoutingAllocation allocation);
/**
* Returns a boolean indicating whether fetching shard data has been triggered at any point for the given shard.
*/
protected abstract boolean hasInitiatedFetching(ShardRouting shard);
static class MatchingNodes {
private final ObjectLongMap<DiscoveryNode> nodesToSize;
private final DiscoveryNode nodeWithHighestMatch;
@Nullable
private final Map<String, Decision> nodeDecisions;
private final Map<String, NodeAllocationResult> nodeDecisions;
public MatchingNodes(ObjectLongMap<DiscoveryNode> nodesToSize, @Nullable Map<String, Decision> nodeDecisions) {
public MatchingNodes(ObjectLongMap<DiscoveryNode> nodesToSize, @Nullable Map<String, NodeAllocationResult> nodeDecisions) {
this.nodesToSize = nodesToSize;
this.nodeDecisions = nodeDecisions;
@ -360,13 +443,5 @@ public abstract class ReplicaShardAllocator extends BaseGatewayShardAllocator {
public boolean hasAnyData() {
return nodesToSize.isEmpty() == false;
}
/**
* The decisions map for all nodes with a shard copy, if available.
*/
@Nullable
public Map<String, Decision> getNodeDecisions() {
return nodeDecisions;
}
}
}

View File

@ -363,7 +363,7 @@ final class StoreRecovery {
indexShard.finalizeRecovery();
indexShard.postRecovery("post recovery from shard_store");
} catch (EngineException | IOException e) {
throw new IndexShardRecoveryException(shardId, "failed to recovery from gateway", e);
throw new IndexShardRecoveryException(shardId, "failed to recover from gateway", e);
} finally {
store.decRef();
}

View File

@ -37,9 +37,11 @@ import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.test.ESTestCase;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_CREATION_DATE;
@ -201,6 +203,8 @@ public class ClusterStateCreationUtils {
discoBuilder.masterNodeId(newNode(0).getId());
MetaData.Builder metaData = MetaData.builder();
RoutingTable.Builder routingTable = RoutingTable.builder();
List<String> nodesList = new ArrayList<>(nodes);
int currentNodeToAssign = 0;
for (String index : indices) {
IndexMetaData indexMetaData = IndexMetaData.builder(index).settings(Settings.builder()
.put(SETTING_VERSION_CREATED, Version.CURRENT)
@ -215,7 +219,10 @@ public class ClusterStateCreationUtils {
ShardId shardId = new ShardId(indexMetaData.getIndex(), i);
IndexShardRoutingTable.Builder indexShardRoutingBuilder = new IndexShardRoutingTable.Builder(shardId);
indexShardRoutingBuilder.addShard(
TestShardRouting.newShardRouting(shardId, randomFrom(nodes), true, ShardRoutingState.STARTED));
TestShardRouting.newShardRouting(shardId, nodesList.get(currentNodeToAssign++), true, ShardRoutingState.STARTED));
if (currentNodeToAssign == nodesList.size()) {
currentNodeToAssign = 0;
}
indexRoutingTable.addIndexShard(indexShardRoutingBuilder.build());
}

View File

@ -19,98 +19,120 @@
package org.elasticsearch.cluster.routing.allocation;
import org.elasticsearch.Version;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.routing.UnassignedInfo.AllocationStatus;
import org.elasticsearch.cluster.routing.allocation.decider.Decision;
import org.elasticsearch.common.io.stream.BytesStreamOutput;
import org.elasticsearch.test.ESTestCase;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
import static java.util.Collections.emptyList;
import static java.util.Collections.emptyMap;
import static java.util.Collections.emptySet;
import static org.hamcrest.Matchers.startsWith;
/**
* Unit tests for the {@link AllocateUnassignedDecision} class.
*/
public class AllocateUnassignedDecisionTests extends ESTestCase {
private DiscoveryNode node1 = new DiscoveryNode("node1", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT);
private DiscoveryNode node2 = new DiscoveryNode("node2", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT);
public void testDecisionNotTaken() {
AllocateUnassignedDecision allocateUnassignedDecision = AllocateUnassignedDecision.NOT_TAKEN;
assertFalse(allocateUnassignedDecision.isDecisionTaken());
assertNull(allocateUnassignedDecision.getFinalDecisionType());
assertNull(allocateUnassignedDecision.getAllocationStatus());
assertNull(allocateUnassignedDecision.getAllocationId());
assertNull(allocateUnassignedDecision.getAssignedNodeId());
assertNull(allocateUnassignedDecision.getFinalExplanation());
assertNull(allocateUnassignedDecision.getNodeDecisions());
expectThrows(IllegalArgumentException.class, () -> allocateUnassignedDecision.getFinalDecisionSafe());
expectThrows(IllegalStateException.class, () -> allocateUnassignedDecision.getAllocationDecision());
expectThrows(IllegalStateException.class, () -> allocateUnassignedDecision.getAllocationStatus());
expectThrows(IllegalStateException.class, () -> allocateUnassignedDecision.getAllocationId());
expectThrows(IllegalStateException.class, () -> allocateUnassignedDecision.getTargetNode());
expectThrows(IllegalStateException.class, () -> allocateUnassignedDecision.getNodeDecisions());
expectThrows(IllegalStateException.class, () -> allocateUnassignedDecision.getExplanation());
}
public void testNoDecision() {
final AllocationStatus allocationStatus = randomFrom(
AllocationStatus.DELAYED_ALLOCATION, AllocationStatus.NO_VALID_SHARD_COPY, AllocationStatus.FETCHING_SHARD_DATA
);
AllocateUnassignedDecision noDecision = AllocateUnassignedDecision.no(allocationStatus, "something is wrong");
AllocateUnassignedDecision noDecision = AllocateUnassignedDecision.no(allocationStatus, null);
assertTrue(noDecision.isDecisionTaken());
assertEquals(Decision.Type.NO, noDecision.getFinalDecisionType());
assertEquals(AllocationDecision.fromAllocationStatus(allocationStatus), noDecision.getAllocationDecision());
assertEquals(allocationStatus, noDecision.getAllocationStatus());
assertEquals("something is wrong", noDecision.getFinalExplanation());
if (allocationStatus == AllocationStatus.FETCHING_SHARD_DATA) {
assertEquals("cannot allocate because information about existing shard data is still being retrieved from " +
"some of the nodes", noDecision.getExplanation());
} else if (allocationStatus == AllocationStatus.DELAYED_ALLOCATION) {
assertThat(noDecision.getExplanation(), startsWith("cannot allocate because the cluster is still waiting"));
} else {
assertThat(noDecision.getExplanation(),
startsWith("cannot allocate because a previous copy of the shard existed"));
}
assertNull(noDecision.getNodeDecisions());
assertNull(noDecision.getAssignedNodeId());
assertNull(noDecision.getTargetNode());
assertNull(noDecision.getAllocationId());
Map<String, NodeAllocationResult> nodeDecisions = new HashMap<>();
nodeDecisions.put("node1", new NodeAllocationResult(Decision.NO));
nodeDecisions.put("node2", new NodeAllocationResult(Decision.NO));
noDecision = AllocateUnassignedDecision.no(AllocationStatus.DECIDERS_NO, "something is wrong",
nodeDecisions.entrySet().stream().collect(Collectors.toMap(Map.Entry::getKey, e -> e.getValue().getDecision()))
);
List<NodeAllocationResult> nodeDecisions = new ArrayList<>();
nodeDecisions.add(new NodeAllocationResult(node1, Decision.NO, 1));
nodeDecisions.add(new NodeAllocationResult(node2, Decision.NO, 2));
final boolean reuseStore = randomBoolean();
noDecision = AllocateUnassignedDecision.no(AllocationStatus.DECIDERS_NO, nodeDecisions, reuseStore);
assertTrue(noDecision.isDecisionTaken());
assertEquals(Decision.Type.NO, noDecision.getFinalDecisionType());
assertEquals(AllocationDecision.NO, noDecision.getAllocationDecision());
assertEquals(AllocationStatus.DECIDERS_NO, noDecision.getAllocationStatus());
assertEquals("something is wrong", noDecision.getFinalExplanation());
assertEquals(nodeDecisions, noDecision.getNodeDecisions());
assertNull(noDecision.getAssignedNodeId());
if (reuseStore) {
assertEquals("cannot allocate because allocation is not permitted to any of the nodes that hold an in-sync shard copy",
noDecision.getExplanation());
} else {
assertEquals("cannot allocate because allocation is not permitted to any of the nodes", noDecision.getExplanation());
}
assertEquals(nodeDecisions.stream().sorted().collect(Collectors.toList()), noDecision.getNodeDecisions());
// node1 should be sorted first b/c of better weight ranking
assertEquals("node1", noDecision.getNodeDecisions().iterator().next().getNode().getId());
assertNull(noDecision.getTargetNode());
assertNull(noDecision.getAllocationId());
// test bad values
expectThrows(NullPointerException.class, () -> AllocateUnassignedDecision.no((AllocationStatus)null, "a"));
expectThrows(NullPointerException.class, () -> AllocateUnassignedDecision.no(null, null));
}
public void testThrottleDecision() {
Map<String, NodeAllocationResult> nodeDecisions = new HashMap<>();
nodeDecisions.put("node1", new NodeAllocationResult(Decision.NO));
nodeDecisions.put("node2", new NodeAllocationResult(Decision.THROTTLE));
AllocateUnassignedDecision throttleDecision = AllocateUnassignedDecision.throttle("too much happening",
nodeDecisions.entrySet().stream().collect(Collectors.toMap(Map.Entry::getKey, e -> e.getValue().getDecision()))
);
List<NodeAllocationResult> nodeDecisions = new ArrayList<>();
nodeDecisions.add(new NodeAllocationResult(node1, Decision.NO, 1));
nodeDecisions.add(new NodeAllocationResult(node2, Decision.THROTTLE, 2));
AllocateUnassignedDecision throttleDecision = AllocateUnassignedDecision.throttle(nodeDecisions);
assertTrue(throttleDecision.isDecisionTaken());
assertEquals(Decision.Type.THROTTLE, throttleDecision.getFinalDecisionType());
assertEquals(AllocationDecision.THROTTLE, throttleDecision.getAllocationDecision());
assertEquals(AllocationStatus.DECIDERS_THROTTLED, throttleDecision.getAllocationStatus());
assertEquals("too much happening", throttleDecision.getFinalExplanation());
assertEquals(nodeDecisions, throttleDecision.getNodeDecisions());
assertNull(throttleDecision.getAssignedNodeId());
assertThat(throttleDecision.getExplanation(), startsWith("allocation temporarily throttled"));
assertEquals(nodeDecisions.stream().sorted().collect(Collectors.toList()), throttleDecision.getNodeDecisions());
// node2 should be sorted first b/c a THROTTLE is higher than a NO decision
assertEquals("node2", throttleDecision.getNodeDecisions().iterator().next().getNode().getId());
assertNull(throttleDecision.getTargetNode());
assertNull(throttleDecision.getAllocationId());
}
public void testYesDecision() {
Map<String, NodeAllocationResult> nodeDecisions = new HashMap<>();
nodeDecisions.put("node1", new NodeAllocationResult(Decision.YES));
nodeDecisions.put("node2", new NodeAllocationResult(Decision.NO));
List<NodeAllocationResult> nodeDecisions = new ArrayList<>();
nodeDecisions.add(new NodeAllocationResult(node1, Decision.NO, 1));
nodeDecisions.add(new NodeAllocationResult(node2, Decision.YES, 2));
String allocId = randomBoolean() ? "allocId" : null;
AllocateUnassignedDecision yesDecision = AllocateUnassignedDecision.yes(
"node1", "node was very kind", allocId, nodeDecisions.entrySet().stream().collect(
Collectors.toMap(Map.Entry::getKey, e -> e.getValue().getDecision())
)
);
node2, allocId, nodeDecisions, randomBoolean());
assertTrue(yesDecision.isDecisionTaken());
assertEquals(Decision.Type.YES, yesDecision.getFinalDecisionType());
assertEquals(AllocationDecision.YES, yesDecision.getAllocationDecision());
assertNull(yesDecision.getAllocationStatus());
assertEquals("node was very kind", yesDecision.getFinalExplanation());
assertEquals(nodeDecisions, yesDecision.getNodeDecisions());
assertEquals("node1", yesDecision.getAssignedNodeId());
assertEquals("can allocate the shard", yesDecision.getExplanation());
assertEquals(nodeDecisions.stream().sorted().collect(Collectors.toList()), yesDecision.getNodeDecisions());
assertEquals("node2", yesDecision.getTargetNode().getId());
assertEquals(allocId, yesDecision.getAllocationId());
// node1 should be sorted first b/c YES decisions are the highest
assertEquals("node2", yesDecision.getNodeDecisions().iterator().next().getNode().getId());
}
public void testCachedDecisions() {
@ -118,28 +140,58 @@ public class AllocateUnassignedDecisionTests extends ESTestCase {
AllocationStatus.NO_VALID_SHARD_COPY, AllocationStatus.FETCHING_SHARD_DATA, AllocationStatus.DELAYED_ALLOCATION);
for (AllocationStatus allocationStatus : cachableStatuses) {
if (allocationStatus == AllocationStatus.DECIDERS_THROTTLED) {
AllocateUnassignedDecision cached = AllocateUnassignedDecision.throttle(null, null);
AllocateUnassignedDecision another = AllocateUnassignedDecision.throttle(null, null);
AllocateUnassignedDecision cached = AllocateUnassignedDecision.throttle(null);
AllocateUnassignedDecision another = AllocateUnassignedDecision.throttle(null);
assertSame(cached, another);
AllocateUnassignedDecision notCached = AllocateUnassignedDecision.throttle("abc", null);
another = AllocateUnassignedDecision.throttle("abc", null);
AllocateUnassignedDecision notCached = AllocateUnassignedDecision.throttle(new ArrayList<>());
another = AllocateUnassignedDecision.throttle(new ArrayList<>());
assertNotSame(notCached, another);
} else {
AllocateUnassignedDecision cached = AllocateUnassignedDecision.no(allocationStatus, null);
AllocateUnassignedDecision another = AllocateUnassignedDecision.no(allocationStatus, null);
assertSame(cached, another);
AllocateUnassignedDecision notCached = AllocateUnassignedDecision.no(allocationStatus, "abc");
another = AllocateUnassignedDecision.no(allocationStatus, "abc");
AllocateUnassignedDecision notCached = AllocateUnassignedDecision.no(allocationStatus, new ArrayList<>());
another = AllocateUnassignedDecision.no(allocationStatus, new ArrayList<>());
assertNotSame(notCached, another);
}
}
// yes decisions are not precomputed and cached
Map<String, Decision> dummyMap = Collections.emptyMap();
AllocateUnassignedDecision first = AllocateUnassignedDecision.yes("node1", "abc", "alloc1", dummyMap);
AllocateUnassignedDecision second = AllocateUnassignedDecision.yes("node1", "abc", "alloc1", dummyMap);
// same fields for the AllocateUnassignedDecision, but should be different instances
AllocateUnassignedDecision first = AllocateUnassignedDecision.yes(node1, "abc", emptyList(), randomBoolean());
AllocateUnassignedDecision second = AllocateUnassignedDecision.yes(node1, "abc", emptyList(), randomBoolean());
// same fields for the ShardAllocationDecision, but should be different instances
assertNotSame(first, second);
}
public void testSerialization() throws IOException {
DiscoveryNode node1 = new DiscoveryNode("node1", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT);
DiscoveryNode node2 = new DiscoveryNode("node2", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT);
Decision.Type finalDecision = randomFrom(Decision.Type.values());
DiscoveryNode assignedNode = finalDecision == Decision.Type.YES ? node1 : null;
List<NodeAllocationResult> nodeDecisions = new ArrayList<>();
nodeDecisions.add(new NodeAllocationResult(node1, Decision.NO, 2));
nodeDecisions.add(new NodeAllocationResult(node2, finalDecision == Decision.Type.YES ? Decision.YES :
randomFrom(Decision.NO, Decision.THROTTLE, Decision.YES), 1));
AllocateUnassignedDecision decision;
if (finalDecision == Decision.Type.YES) {
decision = AllocateUnassignedDecision.yes(assignedNode, randomBoolean() ? randomAsciiOfLength(5) : null,
nodeDecisions, randomBoolean());
} else {
decision = AllocateUnassignedDecision.no(randomFrom(
AllocationStatus.DELAYED_ALLOCATION, AllocationStatus.NO_VALID_SHARD_COPY, AllocationStatus.FETCHING_SHARD_DATA
), nodeDecisions, randomBoolean());
}
BytesStreamOutput output = new BytesStreamOutput();
decision.writeTo(output);
AllocateUnassignedDecision readDecision = new AllocateUnassignedDecision(output.bytes().streamInput());
assertEquals(decision.getTargetNode(), readDecision.getTargetNode());
assertEquals(decision.getAllocationStatus(), readDecision.getAllocationStatus());
assertEquals(decision.getExplanation(), readDecision.getExplanation());
assertEquals(decision.getNodeDecisions().size(), readDecision.getNodeDecisions().size());
assertEquals(decision.getAllocationId(), readDecision.getAllocationId());
assertEquals(decision.getAllocationDecision(), readDecision.getAllocationDecision());
// node2 should have the highest sort order
assertEquals("node2", readDecision.getNodeDecisions().iterator().next().getNode().getId());
}
}

View File

@ -0,0 +1,105 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.cluster.routing.allocation;
import org.elasticsearch.cluster.routing.UnassignedInfo.AllocationStatus;
import org.elasticsearch.cluster.routing.allocation.decider.Decision.Type;
import org.elasticsearch.common.io.stream.BytesStreamOutput;
import org.elasticsearch.test.ESTestCase;
import java.io.IOException;
import java.util.Arrays;
/**
* Tests for the {@link AllocationDecision} enum.
*/
public class AllocationDecisionTests extends ESTestCase {
/**
* Tests serialization and deserialization.
*/
public void testSerialization() throws IOException {
AllocationDecision allocationDecision = randomFrom(AllocationDecision.values());
BytesStreamOutput output = new BytesStreamOutput();
allocationDecision.writeTo(output);
assertEquals(allocationDecision, AllocationDecision.readFrom(output.bytes().streamInput()));
}
/**
* Tests the order of values in the enum, because we depend on the natural enum sort order for sorting node decisions.
* See {@link AbstractAllocationDecision#getNodeDecisions()}.
*/
public void testValuesOrder() {
assertEquals(0, AllocationDecision.YES.ordinal());
assertEquals(1, AllocationDecision.THROTTLE.ordinal());
assertEquals(2, AllocationDecision.NO.ordinal());
assertEquals(3, AllocationDecision.WORSE_BALANCE.ordinal());
assertEquals(4, AllocationDecision.FETCH_PENDING.ordinal());
assertEquals(5, AllocationDecision.DELAYED_ALLOCATION.ordinal());
assertEquals(6, AllocationDecision.NO_VALID_SHARD_COPY.ordinal());
assertEquals(7, AllocationDecision.NO_ATTEMPT.ordinal());
AllocationDecision[] decisions = AllocationDecision.values();
Arrays.sort(decisions);
assertEquals(AllocationDecision.YES, decisions[0]);
assertEquals(AllocationDecision.THROTTLE, decisions[1]);
assertEquals(AllocationDecision.NO, decisions[2]);
assertEquals(AllocationDecision.WORSE_BALANCE, decisions[3]);
assertEquals(AllocationDecision.FETCH_PENDING, decisions[4]);
assertEquals(AllocationDecision.DELAYED_ALLOCATION, decisions[5]);
assertEquals(AllocationDecision.NO_VALID_SHARD_COPY, decisions[6]);
assertEquals(AllocationDecision.NO_ATTEMPT, decisions[7]);
}
/**
* Tests getting a {@link AllocationDecision} from {@link Type}.
*/
public void testFromDecisionType() {
Type type = randomFrom(Type.values());
AllocationDecision allocationDecision = AllocationDecision.fromDecisionType(type);
AllocationDecision expected = type == Type.NO ? AllocationDecision.NO :
type == Type.THROTTLE ? AllocationDecision.THROTTLE : AllocationDecision.YES;
assertEquals(expected, allocationDecision);
}
/**
* Tests getting a {@link AllocationDecision} from {@link AllocationStatus}.
*/
public void testFromAllocationStatus() {
AllocationStatus allocationStatus = rarely() ? null : randomFrom(AllocationStatus.values());
AllocationDecision allocationDecision = AllocationDecision.fromAllocationStatus(allocationStatus);
AllocationDecision expected;
if (allocationStatus == null) {
expected = AllocationDecision.YES;
} else if (allocationStatus == AllocationStatus.DECIDERS_THROTTLED) {
expected = AllocationDecision.THROTTLE;
} else if (allocationStatus == AllocationStatus.FETCHING_SHARD_DATA) {
expected = AllocationDecision.FETCH_PENDING;
} else if (allocationStatus == AllocationStatus.DELAYED_ALLOCATION) {
expected = AllocationDecision.DELAYED_ALLOCATION;
} else if (allocationStatus == AllocationStatus.NO_VALID_SHARD_COPY) {
expected = AllocationDecision.NO_VALID_SHARD_COPY;
} else if (allocationStatus == AllocationStatus.NO_ATTEMPT) {
expected = AllocationDecision.NO_ATTEMPT;
} else {
expected = AllocationDecision.NO;
}
assertEquals(expected, allocationDecision);
}
}

View File

@ -44,6 +44,10 @@ import java.util.HashSet;
import java.util.List;
import java.util.Set;
import static java.util.Collections.emptySet;
import static org.hamcrest.Matchers.containsString;
import static org.hamcrest.Matchers.greaterThanOrEqualTo;
import static org.hamcrest.Matchers.lessThan;
import static org.hamcrest.Matchers.startsWith;
/**
@ -56,9 +60,9 @@ public class BalancedSingleShardTests extends ESAllocationTestCase {
ClusterState clusterState = ClusterStateCreationUtils.state("idx", randomBoolean(),
randomFrom(ShardRoutingState.INITIALIZING, ShardRoutingState.UNASSIGNED, ShardRoutingState.RELOCATING));
ShardRouting shard = clusterState.routingTable().index("idx").shard(0).primaryShard();
RebalanceDecision rebalanceDecision = allocator.decideRebalance(shard, newRoutingAllocation(
MoveDecision rebalanceDecision = allocator.decideRebalance(shard, newRoutingAllocation(
new AllocationDeciders(Settings.EMPTY, Collections.emptyList()), clusterState));
assertSame(RebalanceDecision.NOT_TAKEN, rebalanceDecision);
assertSame(MoveDecision.NOT_TAKEN, rebalanceDecision);
}
public void testRebalanceNotAllowedDuringPendingAsyncFetch() {
@ -68,12 +72,13 @@ public class BalancedSingleShardTests extends ESAllocationTestCase {
RoutingAllocation routingAllocation = newRoutingAllocation(
new AllocationDeciders(Settings.EMPTY, Collections.emptyList()), clusterState);
routingAllocation.setHasPendingAsyncFetch();
RebalanceDecision rebalanceDecision = allocator.decideRebalance(shard, routingAllocation);
MoveDecision rebalanceDecision = allocator.decideRebalance(shard, routingAllocation);
assertNotNull(rebalanceDecision.getCanRebalanceDecision());
assertEquals(Type.NO, rebalanceDecision.getFinalDecisionType());
assertThat(rebalanceDecision.getFinalExplanation(), startsWith("cannot rebalance due to in-flight shard store fetches"));
assertNull(rebalanceDecision.getNodeDecisions());
assertNull(rebalanceDecision.getAssignedNodeId());
assertEquals(AllocationDecision.FETCH_PENDING, rebalanceDecision.getAllocationDecision());
assertThat(rebalanceDecision.getExplanation(),
startsWith("cannot rebalance as information about existing copies of this shard in the cluster is still being gathered"));
assertEquals(clusterState.nodes().getSize() - 1, rebalanceDecision.getNodeDecisions().size());
assertNull(rebalanceDecision.getTargetNode());
assertAssignedNodeRemainsSame(allocator, routingAllocation, shard);
}
@ -91,12 +96,13 @@ public class BalancedSingleShardTests extends ESAllocationTestCase {
ShardRouting shard = clusterState.routingTable().index("idx").shard(0).primaryShard();
RoutingAllocation routingAllocation = newRoutingAllocation(
new AllocationDeciders(Settings.EMPTY, Collections.singleton(noRebalanceDecider)), clusterState);
RebalanceDecision rebalanceDecision = allocator.decideRebalance(shard, routingAllocation);
MoveDecision rebalanceDecision = allocator.decideRebalance(shard, routingAllocation);
assertEquals(canRebalanceDecision.type(), rebalanceDecision.getCanRebalanceDecision().type());
assertEquals(canRebalanceDecision.type(), rebalanceDecision.getFinalDecisionType());
assertEquals("rebalancing is not allowed", rebalanceDecision.getFinalExplanation());
assertEquals(AllocationDecision.fromDecisionType(canRebalanceDecision.type()), rebalanceDecision.getAllocationDecision());
assertThat(rebalanceDecision.getExplanation(), containsString(canRebalanceDecision.type() == Type.THROTTLE ?
"rebalancing is throttled" : "rebalancing is not allowed"));
assertNotNull(rebalanceDecision.getNodeDecisions());
assertNull(rebalanceDecision.getAssignedNodeId());
assertNull(rebalanceDecision.getTargetNode());
assertEquals(1, rebalanceDecision.getCanRebalanceDecision().getDecisions().size());
for (Decision subDecision : rebalanceDecision.getCanRebalanceDecision().getDecisions()) {
assertEquals("foobar", ((Decision.Single) subDecision).getExplanation());
@ -112,11 +118,11 @@ public class BalancedSingleShardTests extends ESAllocationTestCase {
return Decision.YES;
}
};
Tuple<ClusterState, RebalanceDecision> rebalance = setupStateAndRebalance(canAllocateDecider, Settings.EMPTY, true);
Tuple<ClusterState, MoveDecision> rebalance = setupStateAndRebalance(canAllocateDecider, Settings.EMPTY, true);
ClusterState clusterState = rebalance.v1();
RebalanceDecision rebalanceDecision = rebalance.v2();
MoveDecision rebalanceDecision = rebalance.v2();
assertEquals(Type.YES, rebalanceDecision.getCanRebalanceDecision().type());
assertNotNull(rebalanceDecision.getFinalExplanation());
assertNotNull(rebalanceDecision.getExplanation());
assertEquals(clusterState.nodes().getSize() - 1, rebalanceDecision.getNodeDecisions().size());
}
@ -127,15 +133,20 @@ public class BalancedSingleShardTests extends ESAllocationTestCase {
return Decision.NO;
}
};
Tuple<ClusterState, RebalanceDecision> rebalance = setupStateAndRebalance(canAllocateDecider, Settings.EMPTY, false);
Tuple<ClusterState, MoveDecision> rebalance = setupStateAndRebalance(canAllocateDecider, Settings.EMPTY, false);
ClusterState clusterState = rebalance.v1();
RebalanceDecision rebalanceDecision = rebalance.v2();
MoveDecision rebalanceDecision = rebalance.v2();
assertEquals(Type.YES, rebalanceDecision.getCanRebalanceDecision().type());
assertEquals(Type.NO, rebalanceDecision.getFinalDecisionType());
assertThat(rebalanceDecision.getFinalExplanation(),
startsWith("cannot rebalance shard, no other node exists that would form a more balanced"));
assertEquals(AllocationDecision.NO, rebalanceDecision.getAllocationDecision());
assertThat(rebalanceDecision.getExplanation(), startsWith(
"cannot rebalance as no target node exists that can both allocate this shard and improve the cluster balance"));
assertEquals(clusterState.nodes().getSize() - 1, rebalanceDecision.getNodeDecisions().size());
assertNull(rebalanceDecision.getAssignedNodeId());
assertNull(rebalanceDecision.getTargetNode());
int prevRanking = 0;
for (NodeAllocationResult result : rebalanceDecision.getNodeDecisions()) {
assertThat(result.getWeightRanking(), greaterThanOrEqualTo(prevRanking));
prevRanking = result.getWeightRanking();
}
}
public void testDontBalanceShardWhenThresholdNotMet() {
@ -147,14 +158,19 @@ public class BalancedSingleShardTests extends ESAllocationTestCase {
};
// ridiculously high threshold setting so we won't rebalance
Settings balancerSettings = Settings.builder().put(BalancedShardsAllocator.THRESHOLD_SETTING.getKey(), 1000f).build();
Tuple<ClusterState, RebalanceDecision> rebalance = setupStateAndRebalance(canAllocateDecider, balancerSettings, false);
Tuple<ClusterState, MoveDecision> rebalance = setupStateAndRebalance(canAllocateDecider, balancerSettings, false);
ClusterState clusterState = rebalance.v1();
RebalanceDecision rebalanceDecision = rebalance.v2();
MoveDecision rebalanceDecision = rebalance.v2();
assertEquals(Type.YES, rebalanceDecision.getCanRebalanceDecision().type());
assertEquals(Type.NO, rebalanceDecision.getFinalDecisionType());
assertNotNull(rebalanceDecision.getFinalExplanation());
assertEquals(AllocationDecision.NO, rebalanceDecision.getAllocationDecision());
assertNotNull(rebalanceDecision.getExplanation());
assertEquals(clusterState.nodes().getSize() - 1, rebalanceDecision.getNodeDecisions().size());
assertNull(rebalanceDecision.getAssignedNodeId());
assertNull(rebalanceDecision.getTargetNode());
int prevRanking = 0;
for (NodeAllocationResult result : rebalanceDecision.getNodeDecisions()) {
assertThat(result.getWeightRanking(), greaterThanOrEqualTo(prevRanking));
prevRanking = result.getWeightRanking();
}
}
public void testSingleShardBalanceProducesSameResultsAsBalanceStep() {
@ -216,18 +232,114 @@ public class BalancedSingleShardTests extends ESAllocationTestCase {
routingAllocation = newRoutingAllocation(new AllocationDeciders(Settings.EMPTY, allocationDeciders), clusterState);
routingAllocation.debugDecision(true);
ShardRouting shard = clusterState.getRoutingNodes().activePrimary(shardToRebalance.shardId());
RebalanceDecision rebalanceDecision = allocator.decideRebalance(shard, routingAllocation);
assertEquals(shardToRebalance.relocatingNodeId(), rebalanceDecision.getAssignedNodeId());
MoveDecision rebalanceDecision = allocator.decideRebalance(shard, routingAllocation);
assertEquals(shardToRebalance.relocatingNodeId(), rebalanceDecision.getTargetNode().getId());
// make sure all excluded nodes returned a NO decision
for (String exludedNode : excludeNodes) {
NodeRebalanceResult nodeRebalanceResult = rebalanceDecision.getNodeDecisions().get(exludedNode);
assertEquals(Type.NO, nodeRebalanceResult.getCanAllocateDecision().type());
for (NodeAllocationResult nodeResult : rebalanceDecision.getNodeDecisions()) {
if (excludeNodes.contains(nodeResult.getNode().getId())) {
assertEquals(Type.NO, nodeResult.getCanAllocateDecision().type());
}
}
}
private Tuple<ClusterState, RebalanceDecision> setupStateAndRebalance(AllocationDecider allocationDecider,
Settings balancerSettings,
boolean rebalanceExpected) {
public void testNodeDecisionsRanking() {
// only one shard, so moving it will not create a better balance anywhere, so all node decisions should
// return the same ranking as the current node
ClusterState clusterState = ClusterStateCreationUtils.state(randomIntBetween(1, 10), new String[] { "idx" }, 1);
ShardRouting shardToRebalance = clusterState.routingTable().index("idx").shardsWithState(ShardRoutingState.STARTED).get(0);
MoveDecision decision = executeRebalanceFor(shardToRebalance, clusterState, emptySet(), -1);
int currentRanking = decision.getCurrentNodeRanking();
assertEquals(1, currentRanking);
for (NodeAllocationResult result : decision.getNodeDecisions()) {
assertEquals(1, result.getWeightRanking());
}
// start off with one node and several shards assigned to that node, then add a few nodes to the cluster,
// each of these new nodes should have a better ranking than the current, given a low enough threshold
clusterState = ClusterStateCreationUtils.state(1, new String[] { "idx" }, randomIntBetween(2, 10));
shardToRebalance = clusterState.routingTable().index("idx").shardsWithState(ShardRoutingState.STARTED).get(0);
clusterState = addNodesToClusterState(clusterState, randomIntBetween(1, 10));
decision = executeRebalanceFor(shardToRebalance, clusterState, emptySet(), 0.01f);
for (NodeAllocationResult result : decision.getNodeDecisions()) {
assertThat(result.getWeightRanking(), lessThan(decision.getCurrentNodeRanking()));
}
// start off with 3 nodes and 7 shards, so that one of the 3 nodes will have 3 shards assigned, the remaining 2
// nodes will have 2 shard each. then, add another node. pick a shard on one of the nodes that has only 2 shard
// to rebalance. the new node should have the best ranking (because it has no shards), followed by the node currently
// holding the shard as well as the other node with only 2 shards (they should have the same ranking), followed by the
// node with 3 shards which will have the lowest ranking.
clusterState = ClusterStateCreationUtils.state(3, new String[] { "idx" }, 7);
shardToRebalance = null;
Set<String> nodesWithTwoShards = new HashSet<>();
String nodeWithThreeShards = null;
for (RoutingNode node : clusterState.getRoutingNodes()) {
if (node.numberOfShardsWithState(ShardRoutingState.STARTED) == 2) {
nodesWithTwoShards.add(node.nodeId());
if (shardToRebalance == null) {
shardToRebalance = node.shardsWithState(ShardRoutingState.STARTED).get(0);
}
} else {
assertEquals(3, node.numberOfShardsWithState(ShardRoutingState.STARTED));
assertNull(nodeWithThreeShards); // should only have one of these
nodeWithThreeShards = node.nodeId();
}
}
clusterState = addNodesToClusterState(clusterState, 1);
decision = executeRebalanceFor(shardToRebalance, clusterState, emptySet(), 0.01f);
for (NodeAllocationResult result : decision.getNodeDecisions()) {
if (result.getWeightRanking() < decision.getCurrentNodeRanking()) {
// highest ranked node should not be any of the initial nodes
assertFalse(nodesWithTwoShards.contains(result.getNode().getId()));
assertNotEquals(nodeWithThreeShards, result.getNode().getId());
} else if (result.getWeightRanking() > decision.getCurrentNodeRanking()) {
// worst ranked should be the node with two shards
assertEquals(nodeWithThreeShards, result.getNode().getId());
} else {
assertTrue(nodesWithTwoShards.contains(result.getNode().getId()));
}
}
}
private MoveDecision executeRebalanceFor(final ShardRouting shardRouting, final ClusterState clusterState,
final Set<String> noDecisionNodes, final float threshold) {
Settings settings = Settings.EMPTY;
if (Float.compare(-1.0f, threshold) != 0) {
settings = Settings.builder().put(BalancedShardsAllocator.THRESHOLD_SETTING.getKey(), threshold).build();
}
AllocationDecider allocationDecider = new AllocationDecider(Settings.EMPTY) {
@Override
public Decision canAllocate(ShardRouting shardRouting, RoutingNode node, RoutingAllocation allocation) {
if (noDecisionNodes.contains(node.nodeId())) {
return Decision.NO;
}
return Decision.YES;
}
};
AllocationDecider rebalanceDecider = new AllocationDecider(Settings.EMPTY) {
@Override
public Decision canRebalance(ShardRouting shardRouting, RoutingAllocation allocation) {
return Decision.YES;
}
};
BalancedShardsAllocator allocator = new BalancedShardsAllocator(settings);
RoutingAllocation routingAllocation = newRoutingAllocation(
new AllocationDeciders(Settings.EMPTY, Arrays.asList(allocationDecider, rebalanceDecider)), clusterState);
return allocator.decideRebalance(shardRouting, routingAllocation);
}
private ClusterState addNodesToClusterState(ClusterState clusterState, int numNodesToAdd) {
DiscoveryNodes.Builder nodesBuilder = DiscoveryNodes.builder(clusterState.nodes());
for (int i = 0; i < numNodesToAdd; i++) {
DiscoveryNode discoveryNode = newNode(randomAsciiOfLength(7));
nodesBuilder.add(discoveryNode);
}
return ClusterState.builder(clusterState).nodes(nodesBuilder).build();
}
private Tuple<ClusterState, MoveDecision> setupStateAndRebalance(AllocationDecider allocationDecider,
Settings balancerSettings,
boolean rebalanceExpected) {
AllocationDecider rebalanceDecider = new AllocationDecider(Settings.EMPTY) {
@Override
public Decision canRebalance(ShardRouting shardRouting, RoutingAllocation allocation) {
@ -245,7 +357,7 @@ public class BalancedSingleShardTests extends ESAllocationTestCase {
ShardRouting shard = clusterState.routingTable().index("idx").shard(0).primaryShard();
RoutingAllocation routingAllocation = newRoutingAllocation(
new AllocationDeciders(Settings.EMPTY, allocationDeciders), clusterState);
RebalanceDecision rebalanceDecision = allocator.decideRebalance(shard, routingAllocation);
MoveDecision rebalanceDecision = allocator.decideRebalance(shard, routingAllocation);
if (rebalanceExpected == false) {
assertAssignedNodeRemainsSame(allocator, routingAllocation, shard);

View File

@ -19,12 +19,19 @@
package org.elasticsearch.cluster.routing.allocation;
import org.elasticsearch.Version;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.routing.allocation.decider.Decision;
import org.elasticsearch.cluster.routing.allocation.decider.Decision.Type;
import org.elasticsearch.common.io.stream.BytesStreamOutput;
import org.elasticsearch.test.ESTestCase;
import java.util.HashMap;
import java.util.Map;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import static java.util.Collections.emptyMap;
import static java.util.Collections.emptySet;
/**
* Unit tests for the {@link MoveDecision} class.
@ -33,60 +40,91 @@ public class MoveDecisionTests extends ESTestCase {
public void testCachedDecisions() {
// cached stay decision
MoveDecision stay1 = MoveDecision.stay(Decision.YES, false);
MoveDecision stay2 = MoveDecision.stay(Decision.YES, false);
MoveDecision stay1 = MoveDecision.stay(null);
MoveDecision stay2 = MoveDecision.stay(null);
assertSame(stay1, stay2); // not in explain mode, so should use cached decision
stay1 = MoveDecision.stay(Decision.YES, true);
stay2 = MoveDecision.stay(Decision.YES, true);
stay1 = MoveDecision.stay(Decision.YES);
stay2 = MoveDecision.stay(Decision.YES);
assertNotSame(stay1, stay2);
// cached cannot move decision
stay1 = MoveDecision.decision(Decision.NO, Type.NO, false, null, null, null);
stay2 = MoveDecision.decision(Decision.NO, Type.NO, false, null, null, null);
stay1 = MoveDecision.cannotRemain(Decision.NO, AllocationDecision.NO, null, null);
stay2 = MoveDecision.cannotRemain(Decision.NO, AllocationDecision.NO, null, null);
assertSame(stay1, stay2);
// final decision is YES, so shouldn't use cached decision
stay1 = MoveDecision.decision(Decision.NO, Type.YES, false, null, "node1", null);
stay2 = MoveDecision.decision(Decision.NO, Type.YES, false, null, "node1", null);
DiscoveryNode node1 = new DiscoveryNode("node1", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT);
stay1 = MoveDecision.cannotRemain(Decision.NO, AllocationDecision.YES, node1, null);
stay2 = MoveDecision.cannotRemain(Decision.NO, AllocationDecision.YES, node1, null);
assertNotSame(stay1, stay2);
assertEquals(stay1.getAssignedNodeId(), stay2.getAssignedNodeId());
assertEquals(stay1.getTargetNode(), stay2.getTargetNode());
// final decision is NO, but in explain mode, so shouldn't use cached decision
stay1 = MoveDecision.decision(Decision.NO, Type.NO, true, "node1", null, null);
stay2 = MoveDecision.decision(Decision.NO, Type.NO, true, "node1", null, null);
stay1 = MoveDecision.cannotRemain(Decision.NO, AllocationDecision.NO, null, new ArrayList<>());
stay2 = MoveDecision.cannotRemain(Decision.NO, AllocationDecision.NO, null, new ArrayList<>());
assertNotSame(stay1, stay2);
assertSame(stay1.getFinalDecisionType(), stay2.getFinalDecisionType());
assertNotNull(stay1.getFinalExplanation());
assertEquals(stay1.getFinalExplanation(), stay2.getFinalExplanation());
assertSame(stay1.getAllocationDecision(), stay2.getAllocationDecision());
assertNotNull(stay1.getExplanation());
assertEquals(stay1.getExplanation(), stay2.getExplanation());
}
public void testStayDecision() {
MoveDecision stay = MoveDecision.stay(Decision.YES, true);
assertFalse(stay.cannotRemain());
assertFalse(stay.move());
MoveDecision stay = MoveDecision.stay(Decision.YES);
assertTrue(stay.canRemain());
assertFalse(stay.forceMove());
assertTrue(stay.isDecisionTaken());
assertNull(stay.getNodeDecisions());
assertNotNull(stay.getFinalExplanation());
assertEquals(Type.NO, stay.getFinalDecisionType());
assertNotNull(stay.getExplanation());
assertEquals(AllocationDecision.NO_ATTEMPT, stay.getAllocationDecision());
stay = MoveDecision.stay(Decision.YES, false);
assertFalse(stay.cannotRemain());
assertFalse(stay.move());
stay = MoveDecision.stay(Decision.YES);
assertTrue(stay.canRemain());
assertFalse(stay.forceMove());
assertTrue(stay.isDecisionTaken());
assertNull(stay.getNodeDecisions());
assertNull(stay.getFinalExplanation());
assertEquals(Type.NO, stay.getFinalDecisionType());
assertEquals("shard can remain on its current node", stay.getExplanation());
assertEquals(AllocationDecision.NO_ATTEMPT, stay.getAllocationDecision());
}
public void testDecisionWithExplain() {
Map<String, NodeAllocationResult> nodeDecisions = new HashMap<>();
nodeDecisions.put("node1", new NodeAllocationResult(randomFrom(Decision.NO, Decision.THROTTLE, Decision.YES), randomFloat()));
nodeDecisions.put("node2", new NodeAllocationResult(randomFrom(Decision.NO, Decision.THROTTLE, Decision.YES), randomFloat()));
MoveDecision decision = MoveDecision.decision(Decision.NO, Type.NO, true, "node1", null, nodeDecisions);
assertNotNull(decision.getFinalDecisionType());
assertNotNull(decision.getFinalExplanation());
public void testDecisionWithNodeExplanations() {
DiscoveryNode node1 = new DiscoveryNode("node1", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT);
DiscoveryNode node2 = new DiscoveryNode("node2", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT);
Decision nodeDecision = randomFrom(Decision.NO, Decision.THROTTLE, Decision.YES);
List<NodeAllocationResult> nodeDecisions = new ArrayList<>();
nodeDecisions.add(new NodeAllocationResult(node1, nodeDecision, 2));
nodeDecisions.add(new NodeAllocationResult(node2, nodeDecision, 1));
MoveDecision decision = MoveDecision.cannotRemain(Decision.NO, AllocationDecision.NO, null, nodeDecisions);
assertNotNull(decision.getAllocationDecision());
assertNotNull(decision.getExplanation());
assertNotNull(decision.getNodeDecisions());
assertEquals(2, decision.getNodeDecisions().size());
// both nodes have the same decision type but node2 has a higher weight ranking, so node2 comes first
assertEquals("node2", decision.getNodeDecisions().iterator().next().getNode().getId());
decision = MoveDecision.decision(Decision.NO, Type.YES, true, "node1", "node2", null);
assertEquals("node2", decision.getAssignedNodeId());
decision = MoveDecision.cannotRemain(Decision.NO, AllocationDecision.YES, node2, null);
assertEquals("node2", decision.getTargetNode().getId());
}
public void testSerialization() throws IOException {
List<NodeAllocationResult> nodeDecisions = new ArrayList<>();
DiscoveryNode node1 = new DiscoveryNode("node1", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT);
DiscoveryNode node2 = new DiscoveryNode("node2", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT);
Type finalDecision = randomFrom(Type.values());
DiscoveryNode assignedNode = finalDecision == Type.YES ? node1 : null;
nodeDecisions.add(new NodeAllocationResult(node1, Decision.NO, 2));
nodeDecisions.add(new NodeAllocationResult(node2, finalDecision == Type.YES ? Decision.YES :
randomFrom(Decision.NO, Decision.THROTTLE, Decision.YES), 1));
MoveDecision moveDecision = MoveDecision.cannotRemain(Decision.NO, AllocationDecision.fromDecisionType(finalDecision),
assignedNode, nodeDecisions);
BytesStreamOutput output = new BytesStreamOutput();
moveDecision.writeTo(output);
MoveDecision readDecision = new MoveDecision(output.bytes().streamInput());
assertEquals(moveDecision.canRemain(), readDecision.canRemain());
assertEquals(moveDecision.getExplanation(), readDecision.getExplanation());
assertEquals(moveDecision.forceMove(), readDecision.forceMove());
assertEquals(moveDecision.getNodeDecisions().size(), readDecision.getNodeDecisions().size());
assertEquals(moveDecision.getTargetNode(), readDecision.getTargetNode());
assertEquals(moveDecision.getAllocationDecision(), readDecision.getAllocationDecision());
// node2 should have the highest sort order
assertEquals("node2", readDecision.getNodeDecisions().iterator().next().getNode().getId());
}
}

View File

@ -0,0 +1,91 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.cluster.routing.allocation;
import org.elasticsearch.Version;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.routing.allocation.NodeAllocationResult.ShardStoreInfo;
import org.elasticsearch.cluster.routing.allocation.decider.Decision;
import org.elasticsearch.common.io.stream.BytesStreamOutput;
import org.elasticsearch.test.ESTestCase;
import java.io.IOException;
import static java.util.Collections.emptyMap;
import static java.util.Collections.emptySet;
/**
* Unit tests for the {@link NodeAllocationResult} class.
*/
public class NodeAllocationResultTests extends ESTestCase {
public void testSerialization() throws IOException {
DiscoveryNode node = new DiscoveryNode("node1", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT);
Decision decision = randomFrom(Decision.YES, Decision.THROTTLE, Decision.NO);
NodeAllocationResult explanation = new NodeAllocationResult(node, decision, 1);
BytesStreamOutput output = new BytesStreamOutput();
explanation.writeTo(output);
NodeAllocationResult readExplanation = new NodeAllocationResult(output.bytes().streamInput());
assertNodeExplanationEquals(explanation, readExplanation);
}
public void testShardStore() throws IOException {
DiscoveryNode node = new DiscoveryNode("node1", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT);
Decision decision = randomFrom(Decision.YES, Decision.THROTTLE, Decision.NO);
long matchingBytes = (long) randomIntBetween(1, 1000);
ShardStoreInfo shardStoreInfo = new ShardStoreInfo(matchingBytes);
NodeAllocationResult explanation = new NodeAllocationResult(node, shardStoreInfo, decision);
BytesStreamOutput output = new BytesStreamOutput();
explanation.writeTo(output);
NodeAllocationResult readExplanation = new NodeAllocationResult(output.bytes().streamInput());
assertNodeExplanationEquals(explanation, readExplanation);
assertEquals(matchingBytes, explanation.getShardStoreInfo().getMatchingBytes());
assertNull(explanation.getShardStoreInfo().getAllocationId());
assertFalse(explanation.getShardStoreInfo().isInSync());
assertFalse(explanation.getShardStoreInfo().hasMatchingSyncId());
String allocId = randomAsciiOfLength(5);
boolean inSync = randomBoolean();
shardStoreInfo = new ShardStoreInfo(allocId, inSync, randomBoolean() ? new Exception("bad stuff") : null);
explanation = new NodeAllocationResult(node, shardStoreInfo, decision);
output = new BytesStreamOutput();
explanation.writeTo(output);
readExplanation = new NodeAllocationResult(output.bytes().streamInput());
assertNodeExplanationEquals(explanation, readExplanation);
assertEquals(inSync, explanation.getShardStoreInfo().isInSync());
assertEquals(-1, explanation.getShardStoreInfo().getMatchingBytes());
assertFalse(explanation.getShardStoreInfo().hasMatchingSyncId());
assertEquals(allocId, explanation.getShardStoreInfo().getAllocationId());
}
private void assertNodeExplanationEquals(NodeAllocationResult expl1, NodeAllocationResult expl2) {
assertEquals(expl1.getNode(), expl2.getNode());
assertEquals(expl1.getCanAllocateDecision(), expl2.getCanAllocateDecision());
assertEquals(0, Float.compare(expl1.getWeightRanking(), expl2.getWeightRanking()));
if (expl1.getShardStoreInfo() != null) {
assertEquals(expl1.getShardStoreInfo().isInSync(), expl2.getShardStoreInfo().isInSync());
assertEquals(expl1.getShardStoreInfo().getAllocationId(), expl2.getShardStoreInfo().getAllocationId());
assertEquals(expl1.getShardStoreInfo().getMatchingBytes(), expl2.getShardStoreInfo().getMatchingBytes());
assertEquals(expl1.getShardStoreInfo().hasMatchingSyncId(), expl2.getShardStoreInfo().hasMatchingSyncId());
} else {
assertNull(expl2.getShardStoreInfo());
}
}
}

View File

@ -384,5 +384,10 @@ public class ReplicaShardAllocatorTests extends ESAllocationTestCase {
}
return new AsyncShardFetch.FetchResult<>(shardId, tData, Collections.<String>emptySet(), Collections.<String>emptySet());
}
@Override
protected boolean hasInitiatedFetching(ShardRouting shard) {
return fetchDataCalled.get();
}
}
}

View File

@ -89,6 +89,11 @@ public class TestGatewayAllocator extends GatewayAllocator {
return new AsyncShardFetch.FetchResult<>(shardId, Collections.emptyMap(), Collections.emptySet(),
allocation.getIgnoreNodes(shardId));
}
@Override
protected boolean hasInitiatedFetching(ShardRouting shard) {
return true;
}
};
public TestGatewayAllocator() {