Merge remote-tracking branch 'dakrone/explain-add-fetch-in-progress'

This commit is contained in:
Lee Hinman 2016-05-23 09:43:16 -06:00
commit bfce901edf
7 changed files with 120 additions and 66 deletions

View File

@ -42,15 +42,18 @@ public final class ClusterAllocationExplanation implements ToXContent, Writeable
private final ShardId shard; private final ShardId shard;
private final boolean primary; private final boolean primary;
private final boolean hasPendingAsyncFetch;
private final String assignedNodeId; private final String assignedNodeId;
private final UnassignedInfo unassignedInfo; private final UnassignedInfo unassignedInfo;
private final long remainingDelayMillis; private final long remainingDelayMillis;
private final Map<DiscoveryNode, NodeExplanation> nodeExplanations; private final Map<DiscoveryNode, NodeExplanation> nodeExplanations;
public ClusterAllocationExplanation(ShardId shard, boolean primary, @Nullable String assignedNodeId, long remainingDelayMillis, public ClusterAllocationExplanation(ShardId shard, boolean primary, @Nullable String assignedNodeId, long remainingDelayMillis,
@Nullable UnassignedInfo unassignedInfo, Map<DiscoveryNode, NodeExplanation> nodeExplanations) { @Nullable UnassignedInfo unassignedInfo, boolean hasPendingAsyncFetch,
Map<DiscoveryNode, NodeExplanation> nodeExplanations) {
this.shard = shard; this.shard = shard;
this.primary = primary; this.primary = primary;
this.hasPendingAsyncFetch = hasPendingAsyncFetch;
this.assignedNodeId = assignedNodeId; this.assignedNodeId = assignedNodeId;
this.unassignedInfo = unassignedInfo; this.unassignedInfo = unassignedInfo;
this.remainingDelayMillis = remainingDelayMillis; this.remainingDelayMillis = remainingDelayMillis;
@ -60,6 +63,7 @@ public final class ClusterAllocationExplanation implements ToXContent, Writeable
public ClusterAllocationExplanation(StreamInput in) throws IOException { public ClusterAllocationExplanation(StreamInput in) throws IOException {
this.shard = ShardId.readShardId(in); this.shard = ShardId.readShardId(in);
this.primary = in.readBoolean(); this.primary = in.readBoolean();
this.hasPendingAsyncFetch = in.readBoolean();
this.assignedNodeId = in.readOptionalString(); this.assignedNodeId = in.readOptionalString();
this.unassignedInfo = in.readOptionalWriteable(UnassignedInfo::new); this.unassignedInfo = in.readOptionalWriteable(UnassignedInfo::new);
this.remainingDelayMillis = in.readVLong(); this.remainingDelayMillis = in.readVLong();
@ -77,6 +81,7 @@ public final class ClusterAllocationExplanation implements ToXContent, Writeable
public void writeTo(StreamOutput out) throws IOException { public void writeTo(StreamOutput out) throws IOException {
this.getShard().writeTo(out); this.getShard().writeTo(out);
out.writeBoolean(this.isPrimary()); out.writeBoolean(this.isPrimary());
out.writeBoolean(this.isStillFetchingShardData());
out.writeOptionalString(this.getAssignedNodeId()); out.writeOptionalString(this.getAssignedNodeId());
out.writeOptionalWriteable(this.getUnassignedInfo()); out.writeOptionalWriteable(this.getUnassignedInfo());
out.writeVLong(remainingDelayMillis); out.writeVLong(remainingDelayMillis);
@ -97,6 +102,11 @@ public final class ClusterAllocationExplanation implements ToXContent, Writeable
return this.primary; return this.primary;
} }
/** Return turn if shard data is still being fetched for the allocation */
public boolean isStillFetchingShardData() {
return this.hasPendingAsyncFetch;
}
/** Return turn if the shard is assigned to a node */ /** Return turn if the shard is assigned to a node */
public boolean isAssigned() { public boolean isAssigned() {
return this.assignedNodeId != null; return this.assignedNodeId != null;
@ -138,6 +148,7 @@ public final class ClusterAllocationExplanation implements ToXContent, Writeable
if (assignedNodeId != null) { if (assignedNodeId != null) {
builder.field("assigned_node_id", this.assignedNodeId); builder.field("assigned_node_id", this.assignedNodeId);
} }
builder.field("shard_state_fetch_pending", this.hasPendingAsyncFetch);
// If we have unassigned info, show that // If we have unassigned info, show that
if (unassignedInfo != null) { if (unassignedInfo != null) {
unassignedInfo.toXContent(builder, params); unassignedInfo.toXContent(builder, params);

View File

@ -50,6 +50,7 @@ import org.elasticsearch.common.collect.ImmutableOpenIntMap;
import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.gateway.GatewayAllocator;
import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.TransportService; import org.elasticsearch.transport.TransportService;
@ -69,19 +70,22 @@ public class TransportClusterAllocationExplainAction
private final AllocationDeciders allocationDeciders; private final AllocationDeciders allocationDeciders;
private final ShardsAllocator shardAllocator; private final ShardsAllocator shardAllocator;
private final TransportIndicesShardStoresAction shardStoresAction; private final TransportIndicesShardStoresAction shardStoresAction;
private final GatewayAllocator gatewayAllocator;
@Inject @Inject
public TransportClusterAllocationExplainAction(Settings settings, TransportService transportService, ClusterService clusterService, public TransportClusterAllocationExplainAction(Settings settings, TransportService transportService, ClusterService clusterService,
ThreadPool threadPool, ActionFilters actionFilters, ThreadPool threadPool, ActionFilters actionFilters,
IndexNameExpressionResolver indexNameExpressionResolver, IndexNameExpressionResolver indexNameExpressionResolver,
ClusterInfoService clusterInfoService, AllocationDeciders allocationDeciders, ClusterInfoService clusterInfoService, AllocationDeciders allocationDeciders,
ShardsAllocator shardAllocator, TransportIndicesShardStoresAction shardStoresAction) { ShardsAllocator shardAllocator, TransportIndicesShardStoresAction shardStoresAction,
GatewayAllocator gatewayAllocator) {
super(settings, ClusterAllocationExplainAction.NAME, transportService, clusterService, threadPool, actionFilters, super(settings, ClusterAllocationExplainAction.NAME, transportService, clusterService, threadPool, actionFilters,
indexNameExpressionResolver, ClusterAllocationExplainRequest::new); indexNameExpressionResolver, ClusterAllocationExplainRequest::new);
this.clusterInfoService = clusterInfoService; this.clusterInfoService = clusterInfoService;
this.allocationDeciders = allocationDeciders; this.allocationDeciders = allocationDeciders;
this.shardAllocator = shardAllocator; this.shardAllocator = shardAllocator;
this.shardStoresAction = shardStoresAction; this.shardStoresAction = shardStoresAction;
this.gatewayAllocator = gatewayAllocator;
} }
@Override @Override
@ -130,7 +134,8 @@ public class TransportClusterAllocationExplainAction
Float nodeWeight, Float nodeWeight,
IndicesShardStoresResponse.StoreStatus storeStatus, IndicesShardStoresResponse.StoreStatus storeStatus,
String assignedNodeId, String assignedNodeId,
Set<String> activeAllocationIds) { Set<String> activeAllocationIds,
boolean hasPendingAsyncFetch) {
final ClusterAllocationExplanation.FinalDecision finalDecision; final ClusterAllocationExplanation.FinalDecision finalDecision;
final ClusterAllocationExplanation.StoreCopy storeCopy; final ClusterAllocationExplanation.StoreCopy storeCopy;
final String finalExplanation; final String finalExplanation;
@ -161,6 +166,19 @@ public class TransportClusterAllocationExplainAction
if (node.getId().equals(assignedNodeId)) { if (node.getId().equals(assignedNodeId)) {
finalDecision = ClusterAllocationExplanation.FinalDecision.ALREADY_ASSIGNED; finalDecision = ClusterAllocationExplanation.FinalDecision.ALREADY_ASSIGNED;
finalExplanation = "the shard is already assigned to this node"; finalExplanation = "the shard is already assigned to this node";
} else if (hasPendingAsyncFetch &&
shard.primary() == false &&
shard.unassigned() &&
shard.allocatedPostIndexCreate(indexMetaData) &&
nodeDecision.type() != Decision.Type.YES) {
finalExplanation = "the shard cannot be assigned because allocation deciders return a " + nodeDecision.type().name() +
" decision and the shard's state is still being fetched";
finalDecision = ClusterAllocationExplanation.FinalDecision.NO;
} else if (hasPendingAsyncFetch &&
shard.unassigned() &&
shard.allocatedPostIndexCreate(indexMetaData)) {
finalExplanation = "the shard's state is still being fetched so it cannot be allocated";
finalDecision = ClusterAllocationExplanation.FinalDecision.NO;
} else if (shard.primary() && shard.unassigned() && shard.allocatedPostIndexCreate(indexMetaData) && } else if (shard.primary() && shard.unassigned() && shard.allocatedPostIndexCreate(indexMetaData) &&
storeCopy == ClusterAllocationExplanation.StoreCopy.STALE) { storeCopy == ClusterAllocationExplanation.StoreCopy.STALE) {
finalExplanation = "the copy of the shard is stale, allocation ids do not match"; finalExplanation = "the copy of the shard is stale, allocation ids do not match";
@ -180,6 +198,7 @@ public class TransportClusterAllocationExplainAction
finalDecision = ClusterAllocationExplanation.FinalDecision.NO; finalDecision = ClusterAllocationExplanation.FinalDecision.NO;
finalExplanation = "the shard cannot be assigned because one or more allocation decider returns a 'NO' decision"; finalExplanation = "the shard cannot be assigned because one or more allocation decider returns a 'NO' decision";
} else { } else {
// TODO: handle throttling decision better here
finalDecision = ClusterAllocationExplanation.FinalDecision.YES; finalDecision = ClusterAllocationExplanation.FinalDecision.YES;
if (storeCopy == ClusterAllocationExplanation.StoreCopy.AVAILABLE) { if (storeCopy == ClusterAllocationExplanation.StoreCopy.AVAILABLE) {
finalExplanation = "the shard can be assigned and the node contains a valid copy of the shard data"; finalExplanation = "the shard can be assigned and the node contains a valid copy of the shard data";
@ -198,7 +217,8 @@ public class TransportClusterAllocationExplainAction
*/ */
public static ClusterAllocationExplanation explainShard(ShardRouting shard, RoutingAllocation allocation, RoutingNodes routingNodes, public static ClusterAllocationExplanation explainShard(ShardRouting shard, RoutingAllocation allocation, RoutingNodes routingNodes,
boolean includeYesDecisions, ShardsAllocator shardAllocator, boolean includeYesDecisions, ShardsAllocator shardAllocator,
List<IndicesShardStoresResponse.StoreStatus> shardStores) { List<IndicesShardStoresResponse.StoreStatus> shardStores,
GatewayAllocator gatewayAllocator) {
// don't short circuit deciders, we want a full explanation // don't short circuit deciders, we want a full explanation
allocation.debugDecision(true); allocation.debugDecision(true);
// get the existing unassigned info if available // get the existing unassigned info if available
@ -238,11 +258,12 @@ public class TransportClusterAllocationExplainAction
Float weight = weights.get(node); Float weight = weights.get(node);
IndicesShardStoresResponse.StoreStatus storeStatus = nodeToStatus.get(node); IndicesShardStoresResponse.StoreStatus storeStatus = nodeToStatus.get(node);
NodeExplanation nodeExplanation = calculateNodeExplanation(shard, indexMetaData, node, decision, weight, NodeExplanation nodeExplanation = calculateNodeExplanation(shard, indexMetaData, node, decision, weight,
storeStatus, shard.currentNodeId(), indexMetaData.activeAllocationIds(shard.getId())); storeStatus, shard.currentNodeId(), indexMetaData.activeAllocationIds(shard.getId()),
allocation.hasPendingAsyncFetch());
explanations.put(node, nodeExplanation); explanations.put(node, nodeExplanation);
} }
return new ClusterAllocationExplanation(shard.shardId(), shard.primary(), return new ClusterAllocationExplanation(shard.shardId(), shard.primary(), shard.currentNodeId(),
shard.currentNodeId(), remainingDelayMillis, ui, explanations); remainingDelayMillis, ui, gatewayAllocator.hasFetchPending(shard.shardId(), shard.primary()), explanations);
} }
@Override @Override
@ -297,7 +318,7 @@ public class TransportClusterAllocationExplainAction
shardStoreResponse.getStoreStatuses().get(shardRouting.getIndexName()); shardStoreResponse.getStoreStatuses().get(shardRouting.getIndexName());
List<IndicesShardStoresResponse.StoreStatus> shardStoreStatus = shardStatuses.get(shardRouting.id()); List<IndicesShardStoresResponse.StoreStatus> shardStoreStatus = shardStatuses.get(shardRouting.id());
ClusterAllocationExplanation cae = explainShard(shardRouting, allocation, routingNodes, ClusterAllocationExplanation cae = explainShard(shardRouting, allocation, routingNodes,
request.includeYesDecisions(), shardAllocator, shardStoreStatus); request.includeYesDecisions(), shardAllocator, shardStoreStatus, gatewayAllocator);
listener.onResponse(new ClusterAllocationExplainResponse(cae)); listener.onResponse(new ClusterAllocationExplainResponse(cae));
} }

View File

@ -62,6 +62,24 @@ public class GatewayAllocator extends AbstractComponent {
this.replicaShardAllocator = new InternalReplicaShardAllocator(settings, storeAction); this.replicaShardAllocator = new InternalReplicaShardAllocator(settings, storeAction);
} }
/**
* Returns true if the given shard has an async fetch pending
*/
public boolean hasFetchPending(ShardId shardId, boolean primary) {
if (primary) {
AsyncShardFetch<TransportNodesListGatewayStartedShards.NodeGatewayStartedShards> fetch = asyncFetchStarted.get(shardId);
if (fetch != null) {
return fetch.getNumberOfInFlightFetches() > 0;
}
} else {
AsyncShardFetch<TransportNodesListShardStoreMetaData.NodeStoreFilesMetaData> fetch = asyncFetchStore.get(shardId);
if (fetch != null) {
return fetch.getNumberOfInFlightFetches() > 0;
}
}
return false;
}
public void setReallocation(final ClusterService clusterService, final RoutingService routingService) { public void setReallocation(final ClusterService clusterService, final RoutingService routingService) {
this.routingService = routingService; this.routingService = routingService;
clusterService.add(new ClusterStateListener() { clusterService.add(new ClusterStateListener() {

View File

@ -120,6 +120,7 @@ public final class ClusterAllocationExplainIT extends ESIntegTestCase {
assertThat(cae.getShard().getIndexName(), equalTo("only-foo")); assertThat(cae.getShard().getIndexName(), equalTo("only-foo"));
assertFalse(cae.isPrimary()); assertFalse(cae.isPrimary());
assertFalse(cae.isAssigned()); assertFalse(cae.isAssigned());
assertFalse(cae.isStillFetchingShardData());
assertThat(UnassignedInfo.Reason.INDEX_CREATED, equalTo(cae.getUnassignedInfo().getReason())); assertThat(UnassignedInfo.Reason.INDEX_CREATED, equalTo(cae.getUnassignedInfo().getReason()));
assertThat("expecting no remaining delay: " + cae.getRemainingDelayMillis(), cae.getRemainingDelayMillis(), equalTo(0L)); assertThat("expecting no remaining delay: " + cae.getRemainingDelayMillis(), cae.getRemainingDelayMillis(), equalTo(0L));

View File

@ -42,6 +42,7 @@ public final class ClusterAllocationExplainTests extends ESSingleNodeTestCase {
assertEquals(0, cae.getShard().getId()); assertEquals(0, cae.getShard().getId());
assertEquals(false, cae.isPrimary()); assertEquals(false, cae.isPrimary());
assertNull(cae.getAssignedNodeId()); assertNull(cae.getAssignedNodeId());
assertFalse(cae.isStillFetchingShardData());
assertNotNull(cae.getUnassignedInfo()); assertNotNull(cae.getUnassignedInfo());
NodeExplanation explanation = cae.getNodeExplanations().values().iterator().next(); NodeExplanation explanation = cae.getNodeExplanations().values().iterator().next();
ClusterAllocationExplanation.FinalDecision fd = explanation.getFinalDecision(); ClusterAllocationExplanation.FinalDecision fd = explanation.getFinalDecision();
@ -68,6 +69,7 @@ public final class ClusterAllocationExplainTests extends ESSingleNodeTestCase {
assertEquals("test", cae.getShard().getIndexName()); assertEquals("test", cae.getShard().getIndexName());
assertEquals(0, cae.getShard().getId()); assertEquals(0, cae.getShard().getId());
assertEquals(true, cae.isPrimary()); assertEquals(true, cae.isPrimary());
assertFalse(cae.isStillFetchingShardData());
assertNotNull("shard should have assigned node id", cae.getAssignedNodeId()); assertNotNull("shard should have assigned node id", cae.getAssignedNodeId());
assertNull("assigned shard should not have unassigned info", cae.getUnassignedInfo()); assertNull("assigned shard should not have unassigned info", cae.getUnassignedInfo());
explanation = cae.getNodeExplanations().values().iterator().next(); explanation = cae.getNodeExplanations().values().iterator().next();

View File

@ -81,26 +81,6 @@ public final class ClusterAllocationExplanationTests extends ESTestCase {
} }
private NodeExplanation makeNodeExplanation(boolean primary, boolean isAssigned, boolean hasErr, boolean hasActiveId) {
Float nodeWeight = randomFloat();
Exception e = hasErr ? new ElasticsearchException("stuff's broke, yo") : null;
IndicesShardStoresResponse.StoreStatus storeStatus = new IndicesShardStoresResponse.StoreStatus(node, 42, "eggplant",
IndicesShardStoresResponse.StoreStatus.AllocationStatus.PRIMARY, e);
String assignedNodeId;
if (isAssigned) {
assignedNodeId = "node-0";
} else {
assignedNodeId = "node-9";
}
Set<String> activeAllocationIds = new HashSet<>();
if (hasActiveId) {
activeAllocationIds.add("eggplant");
}
return TransportClusterAllocationExplainAction.calculateNodeExplanation(primary ? primaryShard : replicaShard,
indexMetaData, node, noDecision, nodeWeight, storeStatus, assignedNodeId, activeAllocationIds);
}
private void assertExplanations(NodeExplanation ne, String finalExplanation, ClusterAllocationExplanation.FinalDecision finalDecision, private void assertExplanations(NodeExplanation ne, String finalExplanation, ClusterAllocationExplanation.FinalDecision finalDecision,
ClusterAllocationExplanation.StoreCopy storeCopy) { ClusterAllocationExplanation.StoreCopy storeCopy) {
assertEquals(finalExplanation, ne.getFinalExplanation()); assertEquals(finalExplanation, ne.getFinalExplanation());
@ -117,71 +97,89 @@ public final class ClusterAllocationExplanationTests extends ESTestCase {
ShardRouting primaryStartedShard = ShardRouting.newUnassigned(new ShardId(i, 0), null, true, ShardRouting primaryStartedShard = ShardRouting.newUnassigned(new ShardId(i, 0), null, true,
new UnassignedInfo(UnassignedInfo.Reason.INDEX_REOPENED, "foo")); new UnassignedInfo(UnassignedInfo.Reason.INDEX_REOPENED, "foo"));
assertTrue(primaryStartedShard.allocatedPostIndexCreate(indexMetaData)); assertTrue(primaryStartedShard.allocatedPostIndexCreate(indexMetaData));
ShardRouting replicaStartedShard = ShardRouting.newUnassigned(new ShardId(i, 0), null, false,
new UnassignedInfo(UnassignedInfo.Reason.INDEX_REOPENED, "foo"));
assertTrue(replicaStartedShard.allocatedPostIndexCreate(indexMetaData));
IndicesShardStoresResponse.StoreStatus storeStatus = new IndicesShardStoresResponse.StoreStatus(node, 42, "eggplant", IndicesShardStoresResponse.StoreStatus storeStatus = new IndicesShardStoresResponse.StoreStatus(node, 42, "eggplant",
IndicesShardStoresResponse.StoreStatus.AllocationStatus.PRIMARY, e); IndicesShardStoresResponse.StoreStatus.AllocationStatus.PRIMARY, e);
NodeExplanation ne = TransportClusterAllocationExplainAction.calculateNodeExplanation(primaryShard, indexMetaData, node, NodeExplanation ne = TransportClusterAllocationExplainAction.calculateNodeExplanation(primaryShard, indexMetaData, node,
yesDecision, nodeWeight, storeStatus, "", activeAllocationIds); yesDecision, nodeWeight, storeStatus, "", activeAllocationIds, false);
assertExplanations(ne, "the copy of the shard cannot be read", assertExplanations(ne, "the copy of the shard cannot be read",
ClusterAllocationExplanation.FinalDecision.NO, ClusterAllocationExplanation.StoreCopy.IO_ERROR); ClusterAllocationExplanation.FinalDecision.NO, ClusterAllocationExplanation.StoreCopy.IO_ERROR);
ne = TransportClusterAllocationExplainAction.calculateNodeExplanation(primaryShard, indexMetaData, node, yesDecision, nodeWeight, ne = TransportClusterAllocationExplainAction.calculateNodeExplanation(primaryShard, indexMetaData, node, yesDecision, nodeWeight,
null, "", activeAllocationIds); null, "", activeAllocationIds, false);
assertExplanations(ne, "the shard can be assigned", assertExplanations(ne, "the shard can be assigned",
ClusterAllocationExplanation.FinalDecision.YES, ClusterAllocationExplanation.StoreCopy.NONE); ClusterAllocationExplanation.FinalDecision.YES, ClusterAllocationExplanation.StoreCopy.NONE);
ne = TransportClusterAllocationExplainAction.calculateNodeExplanation(primaryStartedShard, indexMetaData, node, yesDecision, ne = TransportClusterAllocationExplainAction.calculateNodeExplanation(primaryStartedShard, indexMetaData, node, yesDecision,
nodeWeight, null, "", activeAllocationIds); nodeWeight, null, "", activeAllocationIds, false);
assertExplanations(ne, "there is no copy of the shard available", assertExplanations(ne, "there is no copy of the shard available",
ClusterAllocationExplanation.FinalDecision.NO, ClusterAllocationExplanation.StoreCopy.NONE); ClusterAllocationExplanation.FinalDecision.NO, ClusterAllocationExplanation.StoreCopy.NONE);
ne = TransportClusterAllocationExplainAction.calculateNodeExplanation(primaryShard, indexMetaData, node, noDecision, nodeWeight, ne = TransportClusterAllocationExplainAction.calculateNodeExplanation(primaryShard, indexMetaData, node, noDecision, nodeWeight,
null, "", activeAllocationIds); null, "", activeAllocationIds, false);
assertExplanations(ne, "the shard cannot be assigned because one or more allocation decider returns a 'NO' decision", assertExplanations(ne, "the shard cannot be assigned because one or more allocation decider returns a 'NO' decision",
ClusterAllocationExplanation.FinalDecision.NO, ClusterAllocationExplanation.StoreCopy.NONE); ClusterAllocationExplanation.FinalDecision.NO, ClusterAllocationExplanation.StoreCopy.NONE);
storeStatus = new IndicesShardStoresResponse.StoreStatus(node, 42, "eggplant", storeStatus = new IndicesShardStoresResponse.StoreStatus(node, 42, "eggplant",
IndicesShardStoresResponse.StoreStatus.AllocationStatus.PRIMARY, null); IndicesShardStoresResponse.StoreStatus.AllocationStatus.PRIMARY, null);
ne = TransportClusterAllocationExplainAction.calculateNodeExplanation(primaryShard, indexMetaData, node, noDecision, nodeWeight, ne = TransportClusterAllocationExplainAction.calculateNodeExplanation(primaryShard, indexMetaData, node, noDecision, nodeWeight,
storeStatus, "", activeAllocationIds); storeStatus, "", activeAllocationIds, false);
assertExplanations(ne, "the shard cannot be assigned because one or more allocation decider returns a 'NO' decision", assertExplanations(ne, "the shard cannot be assigned because one or more allocation decider returns a 'NO' decision",
ClusterAllocationExplanation.FinalDecision.NO, ClusterAllocationExplanation.StoreCopy.AVAILABLE); ClusterAllocationExplanation.FinalDecision.NO, ClusterAllocationExplanation.StoreCopy.AVAILABLE);
storeStatus = new IndicesShardStoresResponse.StoreStatus(node, 42, "eggplant", storeStatus = new IndicesShardStoresResponse.StoreStatus(node, 42, "eggplant",
IndicesShardStoresResponse.StoreStatus.AllocationStatus.PRIMARY, corruptE); IndicesShardStoresResponse.StoreStatus.AllocationStatus.PRIMARY, corruptE);
ne = TransportClusterAllocationExplainAction.calculateNodeExplanation(primaryShard, indexMetaData, node, yesDecision, nodeWeight, ne = TransportClusterAllocationExplainAction.calculateNodeExplanation(primaryShard, indexMetaData, node, yesDecision, nodeWeight,
storeStatus, "", activeAllocationIds); storeStatus, "", activeAllocationIds, false);
assertExplanations(ne, "the copy of the shard is corrupt", assertExplanations(ne, "the copy of the shard is corrupt",
ClusterAllocationExplanation.FinalDecision.NO, ClusterAllocationExplanation.StoreCopy.CORRUPT); ClusterAllocationExplanation.FinalDecision.NO, ClusterAllocationExplanation.StoreCopy.CORRUPT);
storeStatus = new IndicesShardStoresResponse.StoreStatus(node, 42, "banana", storeStatus = new IndicesShardStoresResponse.StoreStatus(node, 42, "banana",
IndicesShardStoresResponse.StoreStatus.AllocationStatus.PRIMARY, null); IndicesShardStoresResponse.StoreStatus.AllocationStatus.PRIMARY, null);
ne = TransportClusterAllocationExplainAction.calculateNodeExplanation(primaryShard, indexMetaData, node, yesDecision, nodeWeight, ne = TransportClusterAllocationExplainAction.calculateNodeExplanation(primaryShard, indexMetaData, node, yesDecision, nodeWeight,
storeStatus, "", activeAllocationIds); storeStatus, "", activeAllocationIds, false);
assertExplanations(ne, "the shard can be assigned", assertExplanations(ne, "the shard can be assigned",
ClusterAllocationExplanation.FinalDecision.YES, ClusterAllocationExplanation.StoreCopy.STALE); ClusterAllocationExplanation.FinalDecision.YES, ClusterAllocationExplanation.StoreCopy.STALE);
storeStatus = new IndicesShardStoresResponse.StoreStatus(node, 42, "banana", storeStatus = new IndicesShardStoresResponse.StoreStatus(node, 42, "banana",
IndicesShardStoresResponse.StoreStatus.AllocationStatus.PRIMARY, null); IndicesShardStoresResponse.StoreStatus.AllocationStatus.PRIMARY, null);
ne = TransportClusterAllocationExplainAction.calculateNodeExplanation(primaryStartedShard, indexMetaData, node, yesDecision, ne = TransportClusterAllocationExplainAction.calculateNodeExplanation(primaryStartedShard, indexMetaData, node, yesDecision,
nodeWeight, storeStatus, "", activeAllocationIds); nodeWeight, storeStatus, "", activeAllocationIds, false);
assertExplanations(ne, "the copy of the shard is stale, allocation ids do not match", assertExplanations(ne, "the copy of the shard is stale, allocation ids do not match",
ClusterAllocationExplanation.FinalDecision.NO, ClusterAllocationExplanation.StoreCopy.STALE); ClusterAllocationExplanation.FinalDecision.NO, ClusterAllocationExplanation.StoreCopy.STALE);
storeStatus = new IndicesShardStoresResponse.StoreStatus(node, 42, "eggplant", storeStatus = new IndicesShardStoresResponse.StoreStatus(node, 42, "eggplant",
IndicesShardStoresResponse.StoreStatus.AllocationStatus.PRIMARY, null); IndicesShardStoresResponse.StoreStatus.AllocationStatus.PRIMARY, null);
ne = TransportClusterAllocationExplainAction.calculateNodeExplanation(primaryShard, indexMetaData, node, yesDecision, nodeWeight, ne = TransportClusterAllocationExplainAction.calculateNodeExplanation(primaryShard, indexMetaData, node, yesDecision, nodeWeight,
storeStatus, "node-0", activeAllocationIds); storeStatus, "node-0", activeAllocationIds, false);
assertExplanations(ne, "the shard is already assigned to this node", assertExplanations(ne, "the shard is already assigned to this node",
ClusterAllocationExplanation.FinalDecision.ALREADY_ASSIGNED, ClusterAllocationExplanation.StoreCopy.AVAILABLE); ClusterAllocationExplanation.FinalDecision.ALREADY_ASSIGNED, ClusterAllocationExplanation.StoreCopy.AVAILABLE);
storeStatus = new IndicesShardStoresResponse.StoreStatus(node, 42, "eggplant", storeStatus = new IndicesShardStoresResponse.StoreStatus(node, 42, "eggplant",
IndicesShardStoresResponse.StoreStatus.AllocationStatus.PRIMARY, null); IndicesShardStoresResponse.StoreStatus.AllocationStatus.PRIMARY, null);
ne = TransportClusterAllocationExplainAction.calculateNodeExplanation(primaryShard, indexMetaData, node, yesDecision, nodeWeight, ne = TransportClusterAllocationExplainAction.calculateNodeExplanation(primaryShard, indexMetaData, node, yesDecision, nodeWeight,
storeStatus, "", activeAllocationIds); storeStatus, "", activeAllocationIds, false);
assertExplanations(ne, "the shard can be assigned and the node contains a valid copy of the shard data", assertExplanations(ne, "the shard can be assigned and the node contains a valid copy of the shard data",
ClusterAllocationExplanation.FinalDecision.YES, ClusterAllocationExplanation.StoreCopy.AVAILABLE); ClusterAllocationExplanation.FinalDecision.YES, ClusterAllocationExplanation.StoreCopy.AVAILABLE);
}
storeStatus = new IndicesShardStoresResponse.StoreStatus(node, 42, "eggplant",
IndicesShardStoresResponse.StoreStatus.AllocationStatus.PRIMARY, null);
ne = TransportClusterAllocationExplainAction.calculateNodeExplanation(primaryStartedShard, indexMetaData, node, yesDecision,
nodeWeight, storeStatus, "", activeAllocationIds, true);
assertExplanations(ne, "the shard's state is still being fetched so it cannot be allocated",
ClusterAllocationExplanation.FinalDecision.NO, ClusterAllocationExplanation.StoreCopy.AVAILABLE);
storeStatus = new IndicesShardStoresResponse.StoreStatus(node, 42, "eggplant",
IndicesShardStoresResponse.StoreStatus.AllocationStatus.REPLICA, null);
ne = TransportClusterAllocationExplainAction.calculateNodeExplanation(replicaStartedShard, indexMetaData, node, noDecision,
nodeWeight, storeStatus, "", activeAllocationIds, true);
assertExplanations(ne, "the shard cannot be assigned because allocation deciders return a NO " +
"decision and the shard's state is still being fetched",
ClusterAllocationExplanation.FinalDecision.NO, ClusterAllocationExplanation.StoreCopy.AVAILABLE);
}
public void testDecisionEquality() { public void testDecisionEquality() {
Decision.Multi d = new Decision.Multi(); Decision.Multi d = new Decision.Multi();
@ -206,10 +204,10 @@ public final class ClusterAllocationExplanationTests extends ESTestCase {
IndicesShardStoresResponse.StoreStatus storeStatus = new IndicesShardStoresResponse.StoreStatus(node, 42, "eggplant", IndicesShardStoresResponse.StoreStatus storeStatus = new IndicesShardStoresResponse.StoreStatus(node, 42, "eggplant",
IndicesShardStoresResponse.StoreStatus.AllocationStatus.PRIMARY, null); IndicesShardStoresResponse.StoreStatus.AllocationStatus.PRIMARY, null);
NodeExplanation ne = TransportClusterAllocationExplainAction.calculateNodeExplanation(primaryShard, indexMetaData, node, NodeExplanation ne = TransportClusterAllocationExplainAction.calculateNodeExplanation(primaryShard, indexMetaData, node,
yesDecision, nodeWeight, storeStatus, "", activeAllocationIds); yesDecision, nodeWeight, storeStatus, "", activeAllocationIds, false);
nodeExplanations.put(ne.getNode(), ne); nodeExplanations.put(ne.getNode(), ne);
ClusterAllocationExplanation cae = new ClusterAllocationExplanation(shard, true, ClusterAllocationExplanation cae = new ClusterAllocationExplanation(shard, true,
"assignedNode", remainingDelay, null, nodeExplanations); "assignedNode", remainingDelay, null, false, nodeExplanations);
BytesStreamOutput out = new BytesStreamOutput(); BytesStreamOutput out = new BytesStreamOutput();
cae.writeTo(out); cae.writeTo(out);
StreamInput in = StreamInput.wrap(out.bytes()); StreamInput in = StreamInput.wrap(out.bytes());
@ -243,21 +241,21 @@ public final class ClusterAllocationExplanationTests extends ESTestCase {
IndicesShardStoresResponse.StoreStatus storeStatus = new IndicesShardStoresResponse.StoreStatus(node, 42, "eggplant", IndicesShardStoresResponse.StoreStatus storeStatus = new IndicesShardStoresResponse.StoreStatus(node, 42, "eggplant",
IndicesShardStoresResponse.StoreStatus.AllocationStatus.PRIMARY, new ElasticsearchException("stuff's broke, yo")); IndicesShardStoresResponse.StoreStatus.AllocationStatus.PRIMARY, new ElasticsearchException("stuff's broke, yo"));
NodeExplanation ne = TransportClusterAllocationExplainAction.calculateNodeExplanation(primaryShard, indexMetaData, node, NodeExplanation ne = TransportClusterAllocationExplainAction.calculateNodeExplanation(primaryShard, indexMetaData, node,
d, nodeWeight, storeStatus, "node-0", allocationIds); d, nodeWeight, storeStatus, "node-0", allocationIds, false);
Map<DiscoveryNode, NodeExplanation> nodeExplanations = new HashMap<>(1); Map<DiscoveryNode, NodeExplanation> nodeExplanations = new HashMap<>(1);
nodeExplanations.put(ne.getNode(), ne); nodeExplanations.put(ne.getNode(), ne);
ClusterAllocationExplanation cae = new ClusterAllocationExplanation(shardId, true, ClusterAllocationExplanation cae = new ClusterAllocationExplanation(shardId, true,
"assignedNode", remainingDelay, null, nodeExplanations); "assignedNode", remainingDelay, null, false, nodeExplanations);
XContentBuilder builder = XContentFactory.jsonBuilder(); XContentBuilder builder = XContentFactory.jsonBuilder();
cae.toXContent(builder, ToXContent.EMPTY_PARAMS); cae.toXContent(builder, ToXContent.EMPTY_PARAMS);
assertEquals("{\"shard\":{\"index\":\"foo\",\"index_uuid\":\"uuid\",\"id\":0,\"primary\":true},\"assigned\":true," + assertEquals("{\"shard\":{\"index\":\"foo\",\"index_uuid\":\"uuid\",\"id\":0,\"primary\":true},\"assigned\":true," +
"\"assigned_node_id\":\"assignedNode\",\"nodes\":{\"node-0\":{\"node_name\":\"\",\"node_attribute" + "\"assigned_node_id\":\"assignedNode\",\"shard_state_fetch_pending\":false,\"nodes\":{\"node-0\":" +
"s\":{},\"store\":{\"shard_copy\":\"IO_ERROR\",\"store_exception\":\"ElasticsearchException[stuff" + "{\"node_name\":\"\",\"node_attributes\":{},\"store\":{\"shard_copy\":\"IO_ERROR\",\"store_except" +
"'s broke, yo]\"},\"final_decision\":\"ALREADY_ASSIGNED\",\"final_explanation\":\"the shard is al" + "ion\":\"ElasticsearchException[stuff's broke, yo]\"},\"final_decision\":\"ALREADY_ASSIGNED\",\"f" +
"ready assigned to this node\",\"weight\":1.5,\"decisions\":[{\"decider\":\"no label\",\"decision" + "inal_explanation\":\"the shard is already assigned to this node\",\"weight\":1.5,\"decisions\":[" +
"\":\"NO\",\"explanation\":\"because I said no\"},{\"decider\":\"yes label\",\"decision\":\"YES\"" + "{\"decider\":\"no label\",\"decision\":\"NO\",\"explanation\":\"because I said no\"},{\"decider" +
",\"explanation\":\"yes please\"},{\"decider\":\"throttle label\",\"decision\":\"THROTTLE\",\"exp" + "\":\"yes label\",\"decision\":\"YES\",\"explanation\":\"yes please\"},{\"decider\":\"throttle la" +
"lanation\":\"wait a sec\"}]}}}", "bel\",\"decision\":\"THROTTLE\",\"explanation\":\"wait a sec\"}]}}}",
builder.string()); builder.string());
} }
} }

View File

@ -30,25 +30,26 @@ The response looks like:
"primary" : false "primary" : false
}, },
"assigned" : false, <1> "assigned" : false, <1>
"shard_state_fetch_pending": false, <2>
"unassigned_info" : { "unassigned_info" : {
"reason" : "INDEX_CREATED", <2> "reason" : "INDEX_CREATED", <3>
"at" : "2016-03-22T20:04:23.620Z" "at" : "2016-03-22T20:04:23.620Z"
}, },
"allocation_delay_ms" : 0, <3> "allocation_delay_ms" : 0, <4>
"remaining_delay_ms" : 0, <4> "remaining_delay_ms" : 0, <5>
"nodes" : { "nodes" : {
"V-Spi0AyRZ6ZvKbaI3691w" : { "V-Spi0AyRZ6ZvKbaI3691w" : {
"node_name" : "node1", "node_name" : "node1",
"node_attributes" : { <5> "node_attributes" : { <6>
"bar" : "baz" "bar" : "baz"
}, },
"store" : { "store" : {
"shard_copy" : "NONE" <6> "shard_copy" : "NONE" <7>
}, },
"final_decision" : "NO", <7> "final_decision" : "NO", <8>
"final_explanation" : "the shard cannot be assigned because one or more allocation decider returns a 'NO' decision", "final_explanation" : "the shard cannot be assigned because one or more allocation decider returns a 'NO' decision",
"weight" : 0.06666675, <8> "weight" : 0.06666675, <9>
"decisions" : [ { <9> "decisions" : [ { <10>
"decider" : "filter", "decider" : "filter",
"decision" : "NO", "decision" : "NO",
"explanation" : "node does not match index include filters [foo:\"bar\"]" "explanation" : "node does not match index include filters [foo:\"bar\"]"
@ -91,14 +92,15 @@ The response looks like:
} }
-------------------------------------------------- --------------------------------------------------
<1> Whether the shard is assigned or unassigned <1> Whether the shard is assigned or unassigned
<2> Reason for the shard originally becoming unassigned <2> Whether information about the shard is still being fetched
<3> Configured delay before the shard can be allocated <3> Reason for the shard originally becoming unassigned
<4> Remaining delay before the shard can be allocated <4> Configured delay before the shard can be allocated
<5> User-added attributes the node has <5> Remaining delay before the shard can be allocated
<6> The shard copy information for this node and error (if applicable) <6> User-added attributes the node has
<7> Final decision and explanation of whether the shard can be allocated to this node <7> The shard copy information for this node and error (if applicable)
<8> Weight for how much the allocator would like to allocate the shard to this node <8> Final decision and explanation of whether the shard can be allocated to this node
<9> List of node decisions factoring into final decision about the shard <9> Weight for how much the allocator would like to allocate the shard to this node
<10> List of node decisions factoring into final decision about the shard
For a shard that is already assigned, the output looks similar to: For a shard that is already assigned, the output looks similar to:
@ -113,6 +115,7 @@ For a shard that is already assigned, the output looks similar to:
}, },
"assigned" : true, "assigned" : true,
"assigned_node_id" : "Qc6VL8c5RWaw1qXZ0Rg57g", <1> "assigned_node_id" : "Qc6VL8c5RWaw1qXZ0Rg57g", <1>
"shard_state_fetch_pending": false,
"allocation_delay_ms" : 0, "allocation_delay_ms" : 0,
"remaining_delay_ms" : 0, "remaining_delay_ms" : 0,
"nodes" : { "nodes" : {