|
@@ -23,12 +23,14 @@ import com.carrotsearch.hppc.ObjectIntHashMap;
|
|
|
import com.carrotsearch.hppc.cursors.ObjectCursor;
|
|
|
import org.apache.lucene.util.CollectionUtil;
|
|
|
import org.elasticsearch.cluster.ClusterState;
|
|
|
+import org.elasticsearch.cluster.metadata.IndexMetaData;
|
|
|
import org.elasticsearch.cluster.metadata.MetaData;
|
|
|
import org.elasticsearch.cluster.node.DiscoveryNode;
|
|
|
import org.elasticsearch.cluster.routing.UnassignedInfo.AllocationStatus;
|
|
|
import org.elasticsearch.common.Nullable;
|
|
|
import org.elasticsearch.common.Randomness;
|
|
|
import org.elasticsearch.common.collect.Tuple;
|
|
|
+import org.elasticsearch.common.logging.ESLogger;
|
|
|
import org.elasticsearch.index.Index;
|
|
|
import org.elasticsearch.index.shard.ShardId;
|
|
|
|
|
@@ -48,8 +50,17 @@ import java.util.Set;
|
|
|
import java.util.function.Predicate;
|
|
|
|
|
|
/**
|
|
|
- * {@link RoutingNodes} represents a copy the routing information contained in
|
|
|
- * the {@link ClusterState cluster state}.
|
|
|
+ * {@link RoutingNodes} represents a copy the routing information contained in the {@link ClusterState cluster state}.
|
|
|
+ * It can be either initialized as mutable or immutable (see {@link #RoutingNodes(ClusterState, boolean)}), allowing
|
|
|
+ * or disallowing changes to its elements.
|
|
|
+ *
|
|
|
+ * The main methods used to update routing entries are:
|
|
|
+ * <ul>
|
|
|
+ * <li> {@link #initializeShard} initializes an unassigned shard.
|
|
|
+ * <li> {@link #startShard} starts an initializing shard / completes relocation of a shard.
|
|
|
+ * <li> {@link #relocateShard} starts relocation of a started shard.
|
|
|
+ * <li> {@link #failShard} fails/cancels an assigned shard.
|
|
|
+ * </ul>
|
|
|
*/
|
|
|
public class RoutingNodes implements Iterable<RoutingNode> {
|
|
|
|
|
@@ -212,6 +223,7 @@ public class RoutingNodes implements Iterable<RoutingNode> {
|
|
|
}
|
|
|
|
|
|
public Iterator<RoutingNode> mutableIterator() {
|
|
|
+ ensureMutable();
|
|
|
return nodesToShards.values().iterator();
|
|
|
}
|
|
|
|
|
@@ -396,10 +408,11 @@ public class RoutingNodes implements Iterable<RoutingNode> {
|
|
|
* @param existingAllocationId allocation id to use. If null, a fresh allocation id is generated.
|
|
|
* @return the initialized shard
|
|
|
*/
|
|
|
- public ShardRouting initialize(ShardRouting shard, String nodeId, @Nullable String existingAllocationId, long expectedSize) {
|
|
|
+ public ShardRouting initializeShard(ShardRouting unassignedShard, String nodeId, @Nullable String existingAllocationId,
|
|
|
+ long expectedSize) {
|
|
|
ensureMutable();
|
|
|
- assert shard.unassigned() : "expected an unassigned shard " + shard;
|
|
|
- ShardRouting initializedShard = shard.initialize(nodeId, existingAllocationId, expectedSize);
|
|
|
+ assert unassignedShard.unassigned() : "expected an unassigned shard " + unassignedShard;
|
|
|
+ ShardRouting initializedShard = unassignedShard.initialize(nodeId, existingAllocationId, expectedSize);
|
|
|
node(nodeId).add(initializedShard);
|
|
|
inactiveShardCount++;
|
|
|
if (initializedShard.primary()) {
|
|
@@ -416,12 +429,12 @@ public class RoutingNodes implements Iterable<RoutingNode> {
|
|
|
*
|
|
|
* @return pair of source relocating and target initializing shards.
|
|
|
*/
|
|
|
- public Tuple<ShardRouting,ShardRouting> relocate(ShardRouting shard, String nodeId, long expectedShardSize) {
|
|
|
+ public Tuple<ShardRouting,ShardRouting> relocateShard(ShardRouting startedShard, String nodeId, long expectedShardSize) {
|
|
|
ensureMutable();
|
|
|
relocatingShards++;
|
|
|
- ShardRouting source = shard.relocate(nodeId, expectedShardSize);
|
|
|
+ ShardRouting source = startedShard.relocate(nodeId, expectedShardSize);
|
|
|
ShardRouting target = source.getTargetRelocatingShard();
|
|
|
- updateAssigned(shard, source);
|
|
|
+ updateAssigned(startedShard, source);
|
|
|
node(target.currentNodeId()).add(target);
|
|
|
assignedShardsAdd(target);
|
|
|
addRecovery(target);
|
|
@@ -429,12 +442,151 @@ public class RoutingNodes implements Iterable<RoutingNode> {
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * Mark a shard as started and adjusts internal statistics.
|
|
|
+ * Applies the relevant logic to start an initializing shard.
|
|
|
+ *
|
|
|
+ * Moves the initializing shard to started. If the shard is a relocation target, also removes the relocation source.
|
|
|
*
|
|
|
* @return the started shard
|
|
|
*/
|
|
|
- public ShardRouting started(ShardRouting shard) {
|
|
|
+ public ShardRouting startShard(ESLogger logger, ShardRouting initializingShard) {
|
|
|
+ ensureMutable();
|
|
|
+ ShardRouting startedShard = started(initializingShard);
|
|
|
+ logger.trace("{} marked shard as started (routing: {})", initializingShard.shardId(), initializingShard);
|
|
|
+
|
|
|
+ if (initializingShard.relocatingNodeId() != null) {
|
|
|
+ // relocation target has been started, remove relocation source
|
|
|
+ RoutingNode relocationSourceNode = node(initializingShard.relocatingNodeId());
|
|
|
+ ShardRouting relocationSourceShard = relocationSourceNode.getByShardId(initializingShard.shardId());
|
|
|
+ assert relocationSourceShard.isRelocationSourceOf(initializingShard);
|
|
|
+ assert relocationSourceShard.getTargetRelocatingShard() == initializingShard : "relocation target mismatch, expected: "
|
|
|
+ + initializingShard + " but was: " + relocationSourceShard.getTargetRelocatingShard();
|
|
|
+ remove(relocationSourceShard);
|
|
|
+ }
|
|
|
+ return startedShard;
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Applies the relevant logic to handle a cancelled or failed shard.
|
|
|
+ *
|
|
|
+ * Moves the shard to unassigned or completely removes the shard (if relocation target).
|
|
|
+ *
|
|
|
+ * - If shard is a primary, this also fails initializing replicas.
|
|
|
+ * - If shard is an active primary, this also promotes an active replica to primary (if such a replica exists).
|
|
|
+ * - If shard is a relocating primary, this also removes the primary relocation target shard.
|
|
|
+ * - If shard is a relocating replica, this promotes the replica relocation target to a full initializing replica, removing the
|
|
|
+ * relocation source information. This is possible as peer recovery is always done from the primary.
|
|
|
+ * - If shard is a (primary or replica) relocation target, this also clears the relocation information on the source shard.
|
|
|
+ *
|
|
|
+ */
|
|
|
+ public void failShard(ESLogger logger, ShardRouting failedShard, UnassignedInfo unassignedInfo, IndexMetaData indexMetaData) {
|
|
|
ensureMutable();
|
|
|
+ assert failedShard.assignedToNode() : "only assigned shards can be failed";
|
|
|
+ assert indexMetaData.getIndex().equals(failedShard.index()) :
|
|
|
+ "shard failed for unknown index (shard entry: " + failedShard + ")";
|
|
|
+ assert getByAllocationId(failedShard.shardId(), failedShard.allocationId().getId()) == failedShard :
|
|
|
+ "shard routing to fail does not exist in routing table, expected: " + failedShard + " but was: " +
|
|
|
+ getByAllocationId(failedShard.shardId(), failedShard.allocationId().getId());
|
|
|
+
|
|
|
+ logger.debug("{} failing shard {} with unassigned info ({})", failedShard.shardId(), failedShard, unassignedInfo.shortSummary());
|
|
|
+
|
|
|
+ // if this is a primary, fail initializing replicas first (otherwise we move RoutingNodes into an inconsistent state)
|
|
|
+ if (failedShard.primary()) {
|
|
|
+ List<ShardRouting> assignedShards = assignedShards(failedShard.shardId());
|
|
|
+ if (assignedShards.isEmpty() == false) {
|
|
|
+ // copy list to prevent ConcurrentModificationException
|
|
|
+ for (ShardRouting routing : new ArrayList<>(assignedShards)) {
|
|
|
+ if (!routing.primary() && routing.initializing()) {
|
|
|
+ // re-resolve replica as earlier iteration could have changed source/target of replica relocation
|
|
|
+ ShardRouting replicaShard = getByAllocationId(routing.shardId(), routing.allocationId().getId());
|
|
|
+ assert replicaShard != null : "failed to re-resolve " + routing + " when failing replicas";
|
|
|
+ UnassignedInfo primaryFailedUnassignedInfo = new UnassignedInfo(UnassignedInfo.Reason.PRIMARY_FAILED,
|
|
|
+ "primary failed while replica initializing", null, 0, unassignedInfo.getUnassignedTimeInNanos(),
|
|
|
+ unassignedInfo.getUnassignedTimeInMillis(), false, AllocationStatus.NO_ATTEMPT);
|
|
|
+ failShard(logger, replicaShard, primaryFailedUnassignedInfo, indexMetaData);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ if (failedShard.relocating()) {
|
|
|
+ // find the shard that is initializing on the target node
|
|
|
+ ShardRouting targetShard = getByAllocationId(failedShard.shardId(), failedShard.allocationId().getRelocationId());
|
|
|
+ assert targetShard.isRelocationTargetOf(failedShard);
|
|
|
+ if (failedShard.primary()) {
|
|
|
+ logger.trace("{} is removed due to the failure/cancellation of the source shard", targetShard);
|
|
|
+ // cancel and remove target shard
|
|
|
+ remove(targetShard);
|
|
|
+ } else {
|
|
|
+ logger.trace("{}, relocation source failed / cancelled, mark as initializing without relocation source", targetShard);
|
|
|
+ // promote to initializing shard without relocation source and ensure that removed relocation source
|
|
|
+ // is not added back as unassigned shard
|
|
|
+ removeRelocationSource(targetShard);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ // fail actual shard
|
|
|
+ if (failedShard.initializing()) {
|
|
|
+ if (failedShard.relocatingNodeId() == null) {
|
|
|
+ // initializing shard that is not relocation target, just move to unassigned
|
|
|
+ moveToUnassigned(failedShard, unassignedInfo);
|
|
|
+ } else {
|
|
|
+ // The shard is a target of a relocating shard. In that case we only need to remove the target shard and cancel the source
|
|
|
+ // relocation. No shard is left unassigned
|
|
|
+ logger.trace("{} is a relocation target, resolving source to cancel relocation ({})", failedShard,
|
|
|
+ unassignedInfo.shortSummary());
|
|
|
+ ShardRouting sourceShard = getByAllocationId(failedShard.shardId(),
|
|
|
+ failedShard.allocationId().getRelocationId());
|
|
|
+ assert sourceShard.isRelocationSourceOf(failedShard);
|
|
|
+ logger.trace("{}, resolved source to [{}]. canceling relocation ... ({})", failedShard.shardId(), sourceShard,
|
|
|
+ unassignedInfo.shortSummary());
|
|
|
+ cancelRelocation(sourceShard);
|
|
|
+ remove(failedShard);
|
|
|
+ }
|
|
|
+ } else {
|
|
|
+ assert failedShard.active();
|
|
|
+ if (failedShard.primary()) {
|
|
|
+ // promote active replica to primary if active replica exists
|
|
|
+ ShardRouting candidate = activeReplica(failedShard.shardId());
|
|
|
+ if (candidate == null) {
|
|
|
+ moveToUnassigned(failedShard, unassignedInfo);
|
|
|
+ } else {
|
|
|
+ movePrimaryToUnassignedAndDemoteToReplica(failedShard, unassignedInfo);
|
|
|
+ ShardRouting primarySwappedCandidate = promoteAssignedReplicaShardToPrimary(candidate);
|
|
|
+ if (primarySwappedCandidate.relocatingNodeId() != null) {
|
|
|
+ // its also relocating, make sure to move the other routing to primary
|
|
|
+ RoutingNode node = node(primarySwappedCandidate.relocatingNodeId());
|
|
|
+ if (node != null) {
|
|
|
+ for (ShardRouting shardRouting : node) {
|
|
|
+ if (shardRouting.shardId().equals(primarySwappedCandidate.shardId()) && !shardRouting.primary()) {
|
|
|
+ promoteAssignedReplicaShardToPrimary(shardRouting);
|
|
|
+ break;
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+ if (IndexMetaData.isIndexUsingShadowReplicas(indexMetaData.getSettings())) {
|
|
|
+ reinitShadowPrimary(primarySwappedCandidate);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ } else {
|
|
|
+ assert failedShard.primary() == false;
|
|
|
+ if (failedShard.relocating()) {
|
|
|
+ remove(failedShard);
|
|
|
+ } else {
|
|
|
+ moveToUnassigned(failedShard, unassignedInfo);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+ assert node(failedShard.currentNodeId()).getByShardId(failedShard.shardId()) == null : "failedShard " + failedShard +
|
|
|
+ " was matched but wasn't removed";
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Mark a shard as started and adjusts internal statistics.
|
|
|
+ *
|
|
|
+ * @return the started shard
|
|
|
+ */
|
|
|
+ private ShardRouting started(ShardRouting shard) {
|
|
|
assert shard.initializing() : "expected an initializing shard " + shard;
|
|
|
if (shard.relocatingNodeId() == null) {
|
|
|
// if this is not a target shard for relocation, we need to update statistics
|
|
@@ -456,8 +608,7 @@ public class RoutingNodes implements Iterable<RoutingNode> {
|
|
|
*
|
|
|
* @return the shard after cancelling relocation
|
|
|
*/
|
|
|
- public ShardRouting cancelRelocation(ShardRouting shard) {
|
|
|
- ensureMutable();
|
|
|
+ private ShardRouting cancelRelocation(ShardRouting shard) {
|
|
|
relocatingShards--;
|
|
|
ShardRouting cancelledShard = shard.cancelRelocation();
|
|
|
updateAssigned(shard, cancelledShard);
|
|
@@ -470,8 +621,7 @@ public class RoutingNodes implements Iterable<RoutingNode> {
|
|
|
* @param replicaShard the replica shard to be promoted to primary
|
|
|
* @return the resulting primary shard
|
|
|
*/
|
|
|
- public ShardRouting promoteAssignedReplicaShardToPrimary(ShardRouting replicaShard) {
|
|
|
- ensureMutable();
|
|
|
+ private ShardRouting promoteAssignedReplicaShardToPrimary(ShardRouting replicaShard) {
|
|
|
assert replicaShard.unassigned() == false : "unassigned shard cannot be promoted to primary: " + replicaShard;
|
|
|
assert replicaShard.primary() == false : "primary shard cannot be promoted to primary: " + replicaShard;
|
|
|
ShardRouting primaryShard = replicaShard.moveToPrimary();
|
|
@@ -485,8 +635,7 @@ public class RoutingNodes implements Iterable<RoutingNode> {
|
|
|
* Cancels the give shard from the Routing nodes internal statistics and cancels
|
|
|
* the relocation if the shard is relocating.
|
|
|
*/
|
|
|
- public void remove(ShardRouting shard) {
|
|
|
- ensureMutable();
|
|
|
+ private void remove(ShardRouting shard) {
|
|
|
assert shard.unassigned() == false : "only assigned shards can be removed here (" + shard + ")";
|
|
|
node(shard.currentNodeId()).remove(shard);
|
|
|
if (shard.initializing() && shard.relocatingNodeId() == null) {
|
|
@@ -508,9 +657,8 @@ public class RoutingNodes implements Iterable<RoutingNode> {
|
|
|
* Removes relocation source of an initializing non-primary shard. This allows the replica shard to continue recovery from
|
|
|
* the primary even though its non-primary relocation source has failed.
|
|
|
*/
|
|
|
- public ShardRouting removeRelocationSource(ShardRouting shard) {
|
|
|
+ private ShardRouting removeRelocationSource(ShardRouting shard) {
|
|
|
assert shard.isRelocationTarget() : "only relocation target shards can have their relocation source removed (" + shard + ")";
|
|
|
- ensureMutable();
|
|
|
ShardRouting relocationMarkerRemoved = shard.removeRelocationSource();
|
|
|
updateAssigned(shard, relocationMarkerRemoved);
|
|
|
inactiveShardCount++; // relocation targets are not counted as inactive shards whereas initializing shards are
|
|
@@ -532,7 +680,6 @@ public class RoutingNodes implements Iterable<RoutingNode> {
|
|
|
}
|
|
|
|
|
|
private void assignedShardsRemove(ShardRouting shard) {
|
|
|
- ensureMutable();
|
|
|
final List<ShardRouting> replicaSet = assignedShards.get(shard.shardId());
|
|
|
if (replicaSet != null) {
|
|
|
final Iterator<ShardRouting> iterator = replicaSet.iterator();
|
|
@@ -547,8 +694,7 @@ public class RoutingNodes implements Iterable<RoutingNode> {
|
|
|
assert false : "No shard found to remove";
|
|
|
}
|
|
|
|
|
|
- public ShardRouting reinitShadowPrimary(ShardRouting candidate) {
|
|
|
- ensureMutable();
|
|
|
+ private ShardRouting reinitShadowPrimary(ShardRouting candidate) {
|
|
|
if (candidate.relocating()) {
|
|
|
cancelRelocation(candidate);
|
|
|
}
|
|
@@ -573,8 +719,7 @@ public class RoutingNodes implements Iterable<RoutingNode> {
|
|
|
shardsWithMatchingShardId.set(previousShardIndex, newShard);
|
|
|
}
|
|
|
|
|
|
- public ShardRouting moveToUnassigned(ShardRouting shard, UnassignedInfo unassignedInfo) {
|
|
|
- ensureMutable();
|
|
|
+ private ShardRouting moveToUnassigned(ShardRouting shard, UnassignedInfo unassignedInfo) {
|
|
|
assert shard.unassigned() == false : "only assigned shards can be moved to unassigned (" + shard + ")";
|
|
|
remove(shard);
|
|
|
ShardRouting unassigned = shard.moveToUnassigned(unassignedInfo);
|
|
@@ -582,6 +727,19 @@ public class RoutingNodes implements Iterable<RoutingNode> {
|
|
|
return unassigned;
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * Moves assigned primary to unassigned and demotes it to a replica.
|
|
|
+ * Used in conjunction with {@link #promoteAssignedReplicaShardToPrimary} when an active replica is promoted to primary.
|
|
|
+ */
|
|
|
+ private ShardRouting movePrimaryToUnassignedAndDemoteToReplica(ShardRouting shard, UnassignedInfo unassignedInfo) {
|
|
|
+ assert shard.unassigned() == false : "only assigned shards can be moved to unassigned (" + shard + ")";
|
|
|
+ assert shard.primary() : "only primary can be demoted to replica (" + shard + ")";
|
|
|
+ remove(shard);
|
|
|
+ ShardRouting unassigned = shard.moveToUnassigned(unassignedInfo).moveFromPrimary();
|
|
|
+ unassignedShards.add(unassigned);
|
|
|
+ return unassigned;
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* Returns the number of routing nodes
|
|
|
*/
|
|
@@ -612,6 +770,7 @@ public class RoutingNodes implements Iterable<RoutingNode> {
|
|
|
}
|
|
|
|
|
|
public void sort(Comparator<ShardRouting> comparator) {
|
|
|
+ nodes.ensureMutable();
|
|
|
CollectionUtil.timSort(unassigned, comparator);
|
|
|
}
|
|
|
|
|
@@ -661,6 +820,7 @@ public class RoutingNodes implements Iterable<RoutingNode> {
|
|
|
* @return true iff the decision caused a change to the unassigned info
|
|
|
*/
|
|
|
public boolean ignoreShard(ShardRouting shard, AllocationStatus allocationStatus) {
|
|
|
+ nodes.ensureMutable();
|
|
|
boolean changed = false;
|
|
|
if (shard.primary()) {
|
|
|
ignoredPrimaries++;
|
|
@@ -704,8 +864,9 @@ public class RoutingNodes implements Iterable<RoutingNode> {
|
|
|
* @param existingAllocationId allocation id to use. If null, a fresh allocation id is generated.
|
|
|
*/
|
|
|
public ShardRouting initialize(String nodeId, @Nullable String existingAllocationId, long expectedShardSize) {
|
|
|
+ nodes.ensureMutable();
|
|
|
innerRemove();
|
|
|
- return nodes.initialize(current, nodeId, existingAllocationId, expectedShardSize);
|
|
|
+ return nodes.initializeShard(current, nodeId, existingAllocationId, expectedShardSize);
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -718,6 +879,7 @@ public class RoutingNodes implements Iterable<RoutingNode> {
|
|
|
* @return true iff the decision caused an update to the unassigned info
|
|
|
*/
|
|
|
public boolean removeAndIgnore(AllocationStatus attempt) {
|
|
|
+ nodes.ensureMutable();
|
|
|
innerRemove();
|
|
|
return ignoreShard(current, attempt);
|
|
|
}
|
|
@@ -734,23 +896,12 @@ public class RoutingNodes implements Iterable<RoutingNode> {
|
|
|
* @return the shard with unassigned info updated
|
|
|
*/
|
|
|
public ShardRouting updateUnassignedInfo(UnassignedInfo unassignedInfo) {
|
|
|
+ nodes.ensureMutable();
|
|
|
ShardRouting updatedShardRouting = current.updateUnassignedInfo(unassignedInfo);
|
|
|
updateShardRouting(updatedShardRouting);
|
|
|
return updatedShardRouting;
|
|
|
}
|
|
|
|
|
|
- /**
|
|
|
- * marks the current primary shard as replica
|
|
|
- *
|
|
|
- * @return the shard with primary status swapped
|
|
|
- */
|
|
|
- public ShardRouting demotePrimaryToReplicaShard() {
|
|
|
- assert current.primary() : "non-primary shard " + current + " cannot be demoted";
|
|
|
- updateShardRouting(current.moveFromPrimary());
|
|
|
- primaries--;
|
|
|
- return current;
|
|
|
- }
|
|
|
-
|
|
|
/**
|
|
|
* Unsupported operation, just there for the interface. Use {@link #removeAndIgnore(AllocationStatus)} or
|
|
|
* {@link #initialize(String, String, long)}.
|
|
@@ -761,7 +912,6 @@ public class RoutingNodes implements Iterable<RoutingNode> {
|
|
|
}
|
|
|
|
|
|
private void innerRemove() {
|
|
|
- nodes.ensureMutable();
|
|
|
iterator.remove();
|
|
|
if (current.primary()) {
|
|
|
primaries--;
|
|
@@ -786,6 +936,7 @@ public class RoutingNodes implements Iterable<RoutingNode> {
|
|
|
}
|
|
|
|
|
|
public void shuffle() {
|
|
|
+ nodes.ensureMutable();
|
|
|
Randomness.shuffle(unassigned);
|
|
|
}
|
|
|
|
|
@@ -794,6 +945,7 @@ public class RoutingNodes implements Iterable<RoutingNode> {
|
|
|
* This method will not drain ignored shards.
|
|
|
*/
|
|
|
public ShardRouting[] drain() {
|
|
|
+ nodes.ensureMutable();
|
|
|
ShardRouting[] mutableShardRoutings = unassigned.toArray(new ShardRouting[unassigned.size()]);
|
|
|
unassigned.clear();
|
|
|
primaries = 0;
|