Merge remote-tracking branch 'dakrone/explain-add-fetch-in-progress'
This commit is contained in:
commit
bfce901edf
|
@ -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);
|
||||||
|
|
|
@ -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));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -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() {
|
||||||
|
|
|
@ -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));
|
||||||
|
|
||||||
|
|
|
@ -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();
|
||||||
|
|
|
@ -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());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -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" : {
|
||||||
|
|
Loading…
Reference in New Issue