From f2d66668af7efea70432d81473724726138834ae Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Tue, 20 Jul 2021 16:19:34 -0600 Subject: [PATCH 01/47] First cut of an integration test (fails, obviously) --- .../NodeShutdownDelayedAllocationIT.java | 92 +++++++++++++++++++ 1 file changed, 92 insertions(+) create mode 100644 x-pack/plugin/shutdown/src/internalClusterTest/java/org/elasticsearch/xpack/shutdown/NodeShutdownDelayedAllocationIT.java diff --git a/x-pack/plugin/shutdown/src/internalClusterTest/java/org/elasticsearch/xpack/shutdown/NodeShutdownDelayedAllocationIT.java b/x-pack/plugin/shutdown/src/internalClusterTest/java/org/elasticsearch/xpack/shutdown/NodeShutdownDelayedAllocationIT.java new file mode 100644 index 0000000000000..8d57af55ff84a --- /dev/null +++ b/x-pack/plugin/shutdown/src/internalClusterTest/java/org/elasticsearch/xpack/shutdown/NodeShutdownDelayedAllocationIT.java @@ -0,0 +1,92 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.xpack.shutdown; + +import org.elasticsearch.action.index.IndexRequestBuilder; +import org.elasticsearch.action.support.master.AcknowledgedResponse; +import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.metadata.IndexMetadata; +import org.elasticsearch.cluster.metadata.SingleNodeShutdownMetadata; +import org.elasticsearch.cluster.routing.ShardRouting; +import org.elasticsearch.cluster.routing.ShardRoutingState; +import org.elasticsearch.cluster.routing.UnassignedInfo; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.plugins.Plugin; +import org.elasticsearch.test.ESIntegTestCase; +import org.elasticsearch.test.InternalTestCluster; + +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.List; + +import static org.hamcrest.Matchers.equalTo; + +@ESIntegTestCase.ClusterScope(scope = ESIntegTestCase.Scope.TEST, numDataNodes = 0) +public class NodeShutdownDelayedAllocationIT extends ESIntegTestCase { + + @Override + protected Collection> nodePlugins() { + return Arrays.asList(ShutdownPlugin.class); + } + + public void testShardAllocationIsDelayedForRestartingNode() throws Exception { + internalCluster().startNodes(3); + prepareCreate("test").setSettings(Settings.builder() + .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1) + .put(UnassignedInfo.INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING.getKey(), 0)).get(); // Disable "normal" delayed allocation + ensureGreen("test"); + indexRandomData(); + + final String nodeToRestart = findNodeWithShard(); + Settings nodeToRestartDataPathSettings = internalCluster().dataPathSettings(nodeToRestart); + + // Mark the node for shutdown + PutShutdownNodeAction.Request putShutdownRequest = new PutShutdownNodeAction.Request( + nodeToRestart, + SingleNodeShutdownMetadata.Type.RESTART, + this.getTestName() + ); + AcknowledgedResponse putShutdownResponse = client().execute(PutShutdownNodeAction.INSTANCE, putShutdownRequest).get(); + assertTrue(putShutdownResponse.isAcknowledged()); + + // Actually stop the node + internalCluster().stopRandomNode(InternalTestCluster.nameFilter(nodeToRestart)); + + // Verify that the shard's allocation is delayed + assertThat(client().admin().cluster().prepareHealth().get().getDelayedUnassignedShards(), equalTo(1)); + + // Bring the node back + internalCluster().startNode(nodeToRestartDataPathSettings); // this will use the same data location as the stopped node + + // And the index should turn green again + ensureGreen("test"); + } + + // GWB-> Next tests to write: + // - Timeout + // - Change the timeout setting + // - Delete the shutdown record + + private void indexRandomData() throws Exception { + int numDocs = scaledRandomIntBetween(100, 1000); + IndexRequestBuilder[] builders = new IndexRequestBuilder[numDocs]; + for (int i = 0; i < builders.length; i++) { + builders[i] = client().prepareIndex("test").setSource("field", "value"); + } + indexRandom(true, builders); + } + + private String findNodeWithShard() { + ClusterState state = client().admin().cluster().prepareState().get().getState(); + List startedShards = state.routingTable().shardsWithState(ShardRoutingState.STARTED); + Collections.shuffle(startedShards,random()); + return state.nodes().get(startedShards.get(0).currentNodeId()).getName(); + } +} From 29641f449109fda50b37667a8b9030735c89bc39 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Wed, 21 Jul 2021 10:39:34 -0600 Subject: [PATCH 02/47] WIP - test passes! --- .../cluster/metadata/Metadata.java | 7 +++ .../cluster/routing/RoutingNodes.java | 31 ++++++--- .../cluster/routing/UnassignedInfo.java | 38 +++++++++-- .../routing/allocation/AllocationService.java | 63 ++++++++++++++----- .../allocator/BalancedShardsAllocator.java | 2 +- ...AllocateEmptyPrimaryAllocationCommand.java | 2 +- .../gateway/ReplicaShardAllocator.java | 4 +- .../cluster/routing/UnassignedInfoTests.java | 55 +++++++++------- ...storeInProgressAllocationDeciderTests.java | 15 ++++- .../gateway/ReplicaShardAllocatorTests.java | 8 +-- .../NodeShutdownDelayedAllocationIT.java | 22 ++++--- 11 files changed, 182 insertions(+), 65 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/cluster/metadata/Metadata.java b/server/src/main/java/org/elasticsearch/cluster/metadata/Metadata.java index cedf6375532b7..676d798926e07 100644 --- a/server/src/main/java/org/elasticsearch/cluster/metadata/Metadata.java +++ b/server/src/main/java/org/elasticsearch/cluster/metadata/Metadata.java @@ -709,6 +709,13 @@ public Map dataStreamAliases() { .orElse(Collections.emptyMap()); } + @SuppressWarnings("unchecked") + public Map nodeShutdowns() { + return Optional.ofNullable((NodesShutdownMetadata) this.custom(NodesShutdownMetadata.TYPE)) + .map(NodesShutdownMetadata::getAllNodeMetadataMap) + .orElse(Collections.EMPTY_MAP); + } + public ImmutableOpenMap customs() { return this.customs; } diff --git a/server/src/main/java/org/elasticsearch/cluster/routing/RoutingNodes.java b/server/src/main/java/org/elasticsearch/cluster/routing/RoutingNodes.java index 78bade7493875..21b84b7bf0d93 100644 --- a/server/src/main/java/org/elasticsearch/cluster/routing/RoutingNodes.java +++ b/server/src/main/java/org/elasticsearch/cluster/routing/RoutingNodes.java @@ -533,9 +533,18 @@ assert getByAllocationId(failedShard.shardId(), failedShard.allocationId().getId // 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, Collections.emptySet()); + UnassignedInfo primaryFailedUnassignedInfo = new UnassignedInfo( + UnassignedInfo.Reason.PRIMARY_FAILED, + "primary failed while replica initializing", + null, + 0, + unassignedInfo.getUnassignedTimeInNanos(), + unassignedInfo.getUnassignedTimeInMillis(), + false, + AllocationStatus.NO_ATTEMPT, + Collections.emptySet(), + routing.currentNodeId() + ); failShard(logger, replicaShard, primaryFailedUnassignedInfo, indexMetadata, routingChangesObserver); } } @@ -858,10 +867,18 @@ public void ignoreShard(ShardRouting shard, AllocationStatus allocationStatus, R UnassignedInfo currInfo = shard.unassignedInfo(); assert currInfo != null; if (allocationStatus.equals(currInfo.getLastAllocationStatus()) == false) { - UnassignedInfo newInfo = new UnassignedInfo(currInfo.getReason(), currInfo.getMessage(), currInfo.getFailure(), - currInfo.getNumFailedAllocations(), currInfo.getUnassignedTimeInNanos(), - currInfo.getUnassignedTimeInMillis(), currInfo.isDelayed(), - allocationStatus, currInfo.getFailedNodeIds()); + UnassignedInfo newInfo = new UnassignedInfo( + currInfo.getReason(), + currInfo.getMessage(), + currInfo.getFailure(), + currInfo.getNumFailedAllocations(), + currInfo.getUnassignedTimeInNanos(), + currInfo.getUnassignedTimeInMillis(), + currInfo.isDelayed(), + allocationStatus, + currInfo.getFailedNodeIds(), + currInfo.getLastAllocatedNodeId() + ); ShardRouting updatedShard = shard.updateUnassigned(newInfo, shard.recoverySource()); changes.unassignedInfoUpdated(shard, newInfo); shard = updatedShard; diff --git a/server/src/main/java/org/elasticsearch/cluster/routing/UnassignedInfo.java b/server/src/main/java/org/elasticsearch/cluster/routing/UnassignedInfo.java index f8e201e1f2361..b563345c7f68a 100644 --- a/server/src/main/java/org/elasticsearch/cluster/routing/UnassignedInfo.java +++ b/server/src/main/java/org/elasticsearch/cluster/routing/UnassignedInfo.java @@ -9,8 +9,10 @@ package org.elasticsearch.cluster.routing; import org.elasticsearch.ExceptionsHelper; +import org.elasticsearch.Version; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.metadata.Metadata; +import org.elasticsearch.cluster.metadata.SingleNodeShutdownMetadata; import org.elasticsearch.cluster.routing.allocation.RoutingAllocation; import org.elasticsearch.cluster.routing.allocation.decider.Decision; import org.elasticsearch.core.Nullable; @@ -32,13 +34,17 @@ import java.util.List; import java.util.Locale; import java.util.Objects; +import java.util.Optional; import java.util.Set; +import java.util.concurrent.TimeUnit; /** * Holds additional information as to why the shard is in unassigned state. */ public final class UnassignedInfo implements ToXContentFragment, Writeable { + private static final Version LAST_ALLOCATED_NODE_VERSION = Version.V_8_0_0; + public static final DateFormatter DATE_TIME_FORMATTER = DateFormatter.forPattern("date_optional_time").withZone(ZoneOffset.UTC); public static final Setting INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING = @@ -208,6 +214,7 @@ public String value() { private final int failedAllocations; private final Set failedNodeIds; private final AllocationStatus lastAllocationStatus; // result of the last allocation attempt for this shard + private final String lastAllocatedNodeId; /** * creates an UnassignedInfo object based on **current** time @@ -217,7 +224,7 @@ public String value() { **/ public UnassignedInfo(Reason reason, String message) { this(reason, message, null, reason == Reason.ALLOCATION_FAILED ? 1 : 0, System.nanoTime(), System.currentTimeMillis(), false, - AllocationStatus.NO_ATTEMPT, Collections.emptySet()); + AllocationStatus.NO_ATTEMPT, Collections.emptySet(), null); } /** @@ -232,7 +239,7 @@ public UnassignedInfo(Reason reason, String message) { */ public UnassignedInfo(Reason reason, @Nullable String message, @Nullable Exception failure, int failedAllocations, long unassignedTimeNanos, long unassignedTimeMillis, boolean delayed, AllocationStatus lastAllocationStatus, - Set failedNodeIds) { + Set failedNodeIds, @Nullable String lastAllocatedNodeId) { this.reason = Objects.requireNonNull(reason); this.unassignedTimeMillis = unassignedTimeMillis; this.unassignedTimeNanos = unassignedTimeNanos; @@ -242,6 +249,7 @@ public UnassignedInfo(Reason reason, @Nullable String message, @Nullable Excepti this.failedAllocations = failedAllocations; this.lastAllocationStatus = Objects.requireNonNull(lastAllocationStatus); this.failedNodeIds = Collections.unmodifiableSet(failedNodeIds); + this.lastAllocatedNodeId = lastAllocatedNodeId; assert (failedAllocations > 0) == (reason == Reason.ALLOCATION_FAILED) : "failedAllocations: " + failedAllocations + " for reason " + reason; assert (message == null && failure != null) == false : "provide a message if a failure exception is provided"; @@ -260,6 +268,11 @@ public UnassignedInfo(StreamInput in) throws IOException { this.failedAllocations = in.readVInt(); this.lastAllocationStatus = AllocationStatus.readFrom(in); this.failedNodeIds = Collections.unmodifiableSet(in.readSet(StreamInput::readString)); + if (in.getVersion().onOrAfter(LAST_ALLOCATED_NODE_VERSION)) { + this.lastAllocatedNodeId = in.readOptionalString(); + } else { + this.lastAllocatedNodeId = null; + } } public void writeTo(StreamOutput out) throws IOException { @@ -272,6 +285,9 @@ public void writeTo(StreamOutput out) throws IOException { out.writeVInt(failedAllocations); lastAllocationStatus.writeTo(out); out.writeCollection(failedNodeIds, StreamOutput::writeString); + if (out.getVersion().onOrAfter(LAST_ALLOCATED_NODE_VERSION)) { + out.writeOptionalString(lastAllocatedNodeId); + } } /** @@ -339,6 +355,14 @@ public String getDetails() { return message + (failure == null ? "" : ", failure " + ExceptionsHelper.stackTrace(failure)); } + /** + * Gets the ID of the node this shard was last allocated to, or null if unavailable. + */ + @Nullable + public String getLastAllocatedNodeId() { + return lastAllocatedNodeId; + } + /** * Get the status for the last allocation attempt for this shard. */ @@ -366,8 +390,12 @@ public Set getFailedNodeIds() { * * @return calculated delay in nanoseconds */ - public long getRemainingDelay(final long nanoTimeNow, final Settings indexSettings) { - long delayTimeoutNanos = INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING.get(indexSettings).nanos(); + // GWB-> NOCOMMIT don't need the full ShardRouting here + public long getRemainingDelay(final long nanoTimeNow, final ShardRouting routing, final Metadata metadata) { + long delayTimeoutNanos = Optional.ofNullable(metadata.nodeShutdowns().get(lastAllocatedNodeId)) + .filter(shutdownMetadata -> SingleNodeShutdownMetadata.Type.RESTART.equals(shutdownMetadata.getType())) + .map(shutdownMetadata -> new TimeValue(10, TimeUnit.MINUTES).getNanos()) // GWB-> NOCOMMIT plumb in a setting here + .orElse(INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING.get(metadata.getIndexSafe(routing.index()).getSettings()).nanos()); assert nanoTimeNow >= unassignedTimeNanos; return Math.max(0L, delayTimeoutNanos - (nanoTimeNow - unassignedTimeNanos)); } @@ -399,7 +427,7 @@ public static long findNextDelayedAllocation(long currentNanoTime, ClusterState if (unassignedInfo.isDelayed()) { Settings indexSettings = metadata.index(shard.index()).getSettings(); // calculate next time to schedule - final long newComputedLeftDelayNanos = unassignedInfo.getRemainingDelay(currentNanoTime, indexSettings); + final long newComputedLeftDelayNanos = unassignedInfo.getRemainingDelay(currentNanoTime, shard, metadata); if (newComputedLeftDelayNanos < nextDelayNanos) { nextDelayNanos = newComputedLeftDelayNanos; } diff --git a/server/src/main/java/org/elasticsearch/cluster/routing/allocation/AllocationService.java b/server/src/main/java/org/elasticsearch/cluster/routing/allocation/AllocationService.java index 5e7d008f1e6f2..684f15c29b820 100644 --- a/server/src/main/java/org/elasticsearch/cluster/routing/allocation/AllocationService.java +++ b/server/src/main/java/org/elasticsearch/cluster/routing/allocation/AllocationService.java @@ -19,6 +19,8 @@ import org.elasticsearch.cluster.metadata.AutoExpandReplicas; import org.elasticsearch.cluster.metadata.IndexMetadata; import org.elasticsearch.cluster.metadata.Metadata; +import org.elasticsearch.cluster.metadata.NodesShutdownMetadata; +import org.elasticsearch.cluster.metadata.SingleNodeShutdownMetadata; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.routing.RerouteService; import org.elasticsearch.cluster.routing.RoutingNode; @@ -45,6 +47,7 @@ import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Set; import java.util.function.Function; import java.util.stream.Collectors; @@ -209,7 +212,7 @@ public ClusterState applyFailedShards(final ClusterState clusterState, final Lis String message = "failed shard on node [" + shardToFail.currentNodeId() + "]: " + failedShardEntry.getMessage(); UnassignedInfo unassignedInfo = new UnassignedInfo(UnassignedInfo.Reason.ALLOCATION_FAILED, message, failedShardEntry.getFailure(), failedAllocations + 1, currentNanoTime, System.currentTimeMillis(), false, - AllocationStatus.NO_ATTEMPT, failedNodeIds); + AllocationStatus.NO_ATTEMPT, failedNodeIds, shardToFail.currentNodeId()); if (failedShardEntry.markAsStale()) { allocation.removeAllocationId(failedShard); } @@ -300,13 +303,28 @@ private void removeDelayMarkers(RoutingAllocation allocation) { ShardRouting shardRouting = unassignedIterator.next(); UnassignedInfo unassignedInfo = shardRouting.unassignedInfo(); if (unassignedInfo.isDelayed()) { - final long newComputedLeftDelayNanos = unassignedInfo.getRemainingDelay(allocation.getCurrentNanoTime(), - metadata.getIndexSafe(shardRouting.index()).getSettings()); + final long newComputedLeftDelayNanos = unassignedInfo.getRemainingDelay( + allocation.getCurrentNanoTime(), + shardRouting, + metadata + ); if (newComputedLeftDelayNanos == 0) { - unassignedIterator.updateUnassigned(new UnassignedInfo(unassignedInfo.getReason(), unassignedInfo.getMessage(), - unassignedInfo.getFailure(), unassignedInfo.getNumFailedAllocations(), unassignedInfo.getUnassignedTimeInNanos(), - unassignedInfo.getUnassignedTimeInMillis(), false, unassignedInfo.getLastAllocationStatus(), - unassignedInfo.getFailedNodeIds()), shardRouting.recoverySource(), allocation.changes()); + unassignedIterator.updateUnassigned( + new UnassignedInfo( + unassignedInfo.getReason(), + unassignedInfo.getMessage(), + unassignedInfo.getFailure(), + unassignedInfo.getNumFailedAllocations(), + unassignedInfo.getUnassignedTimeInNanos(), + unassignedInfo.getUnassignedTimeInMillis(), + false, + unassignedInfo.getLastAllocationStatus(), + unassignedInfo.getFailedNodeIds(), + unassignedInfo.getLastAllocatedNodeId() + ), + shardRouting.recoverySource(), + allocation.changes() + ); } } } @@ -320,11 +338,22 @@ private void resetFailedAllocationCounter(RoutingAllocation allocation) { while (unassignedIterator.hasNext()) { ShardRouting shardRouting = unassignedIterator.next(); UnassignedInfo unassignedInfo = shardRouting.unassignedInfo(); - unassignedIterator.updateUnassigned(new UnassignedInfo(unassignedInfo.getNumFailedAllocations() > 0 ? - UnassignedInfo.Reason.MANUAL_ALLOCATION : unassignedInfo.getReason(), unassignedInfo.getMessage(), - unassignedInfo.getFailure(), 0, unassignedInfo.getUnassignedTimeInNanos(), - unassignedInfo.getUnassignedTimeInMillis(), unassignedInfo.isDelayed(), - unassignedInfo.getLastAllocationStatus(), Collections.emptySet()), shardRouting.recoverySource(), allocation.changes()); + unassignedIterator.updateUnassigned( + new UnassignedInfo( + unassignedInfo.getNumFailedAllocations() > 0 ? UnassignedInfo.Reason.MANUAL_ALLOCATION : unassignedInfo.getReason(), + unassignedInfo.getMessage(), + unassignedInfo.getFailure(), + 0, + unassignedInfo.getUnassignedTimeInNanos(), + unassignedInfo.getUnassignedTimeInMillis(), + unassignedInfo.isDelayed(), + unassignedInfo.getLastAllocationStatus(), + Collections.emptySet(), + unassignedInfo.getLastAllocatedNodeId() + ), + shardRouting.recoverySource(), + allocation.changes() + ); } } @@ -460,6 +489,8 @@ private void allocateExistingUnassignedShards(RoutingAllocation allocation) { } private void disassociateDeadNodes(RoutingAllocation allocation) { + Map nodesShutdownMetadata = allocation.metadata().nodeShutdowns(); + for (Iterator it = allocation.routingNodes().mutableIterator(); it.hasNext(); ) { RoutingNode node = it.next(); if (allocation.nodes().getDataNodes().containsKey(node.nodeId())) { @@ -469,10 +500,14 @@ private void disassociateDeadNodes(RoutingAllocation allocation) { // now, go over all the shards routing on the node, and fail them for (ShardRouting shardRouting : node.copyShards()) { final IndexMetadata indexMetadata = allocation.metadata().getIndexSafe(shardRouting.index()); - boolean delayed = INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING.get(indexMetadata.getSettings()).nanos() > 0; + boolean delayed = Optional.ofNullable(nodesShutdownMetadata.get(node.nodeId())) + // If we know this node is restarting, then the allocation should be delayed + .map(shutdownMetadata -> SingleNodeShutdownMetadata.Type.RESTART.equals(shutdownMetadata.getType())) + // Otherwise, use the "normal" allocation delay logic + .orElse(INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING.get(indexMetadata.getSettings()).nanos() > 0); UnassignedInfo unassignedInfo = new UnassignedInfo(UnassignedInfo.Reason.NODE_LEFT, "node_left [" + node.nodeId() + "]", null, 0, allocation.getCurrentNanoTime(), System.currentTimeMillis(), delayed, AllocationStatus.NO_ATTEMPT, - Collections.emptySet()); + Collections.emptySet(), shardRouting.currentNodeId()); allocation.routingNodes().failShard(logger, shardRouting, unassignedInfo, indexMetadata, allocation.changes()); } // its a dead node, remove it, note, its important to remove it *after* we apply failed shard diff --git a/server/src/main/java/org/elasticsearch/cluster/routing/allocation/allocator/BalancedShardsAllocator.java b/server/src/main/java/org/elasticsearch/cluster/routing/allocation/allocator/BalancedShardsAllocator.java index 59a332d6ce5bf..d9ae70ecdf958 100644 --- a/server/src/main/java/org/elasticsearch/cluster/routing/allocation/allocator/BalancedShardsAllocator.java +++ b/server/src/main/java/org/elasticsearch/cluster/routing/allocation/allocator/BalancedShardsAllocator.java @@ -141,7 +141,7 @@ private void failAllocationOfNewPrimaries(RoutingAllocation allocation) { unassignedIterator.updateUnassigned(new UnassignedInfo(unassignedInfo.getReason(), unassignedInfo.getMessage(), unassignedInfo.getFailure(), unassignedInfo.getNumFailedAllocations(), unassignedInfo.getUnassignedTimeInNanos(), unassignedInfo.getUnassignedTimeInMillis(), unassignedInfo.isDelayed(), AllocationStatus.DECIDERS_NO, - unassignedInfo.getFailedNodeIds()), + unassignedInfo.getFailedNodeIds(), unassignedInfo.getLastAllocatedNodeId()), shardRouting.recoverySource(), allocation.changes()); } } diff --git a/server/src/main/java/org/elasticsearch/cluster/routing/allocation/command/AllocateEmptyPrimaryAllocationCommand.java b/server/src/main/java/org/elasticsearch/cluster/routing/allocation/command/AllocateEmptyPrimaryAllocationCommand.java index eeb76f336bd59..980d88d3c42a8 100644 --- a/server/src/main/java/org/elasticsearch/cluster/routing/allocation/command/AllocateEmptyPrimaryAllocationCommand.java +++ b/server/src/main/java/org/elasticsearch/cluster/routing/allocation/command/AllocateEmptyPrimaryAllocationCommand.java @@ -129,7 +129,7 @@ public RerouteExplanation execute(RoutingAllocation allocation, boolean explain) ", " + shardRouting.unassignedInfo().getMessage(); unassignedInfoToUpdate = new UnassignedInfo(UnassignedInfo.Reason.FORCED_EMPTY_PRIMARY, unassignedInfoMessage, shardRouting.unassignedInfo().getFailure(), 0, System.nanoTime(), System.currentTimeMillis(), false, - shardRouting.unassignedInfo().getLastAllocationStatus(), Collections.emptySet()); + shardRouting.unassignedInfo().getLastAllocationStatus(), Collections.emptySet(), null); } initializeUnassignedShard(allocation, routingNodes, routingNode, shardRouting, unassignedInfoToUpdate, diff --git a/server/src/main/java/org/elasticsearch/gateway/ReplicaShardAllocator.java b/server/src/main/java/org/elasticsearch/gateway/ReplicaShardAllocator.java index 2108585a54c5b..9a81b07dcf2a9 100644 --- a/server/src/main/java/org/elasticsearch/gateway/ReplicaShardAllocator.java +++ b/server/src/main/java/org/elasticsearch/gateway/ReplicaShardAllocator.java @@ -103,7 +103,7 @@ && canPerformOperationBasedRecovery(primaryStore, shardStores, currentNode) == f "existing allocation of replica to [" + currentNode + "] cancelled, can perform a noop recovery on ["+ nodeWithHighestMatch + "]", null, 0, allocation.getCurrentNanoTime(), System.currentTimeMillis(), false, - UnassignedInfo.AllocationStatus.NO_ATTEMPT, failedNodeIds); + UnassignedInfo.AllocationStatus.NO_ATTEMPT, failedNodeIds, null); // don't cancel shard in the loop as it will cause a ConcurrentModificationException shardCancellationActions.add(() -> routingNodes.failShard(logger, shard, unassignedInfo, metadata.getIndexSafe(shard.index()), allocation.changes())); @@ -213,7 +213,7 @@ public AllocateUnassignedDecision makeAllocationDecision(final ShardRouting unas Metadata metadata = allocation.metadata(); IndexMetadata indexMetadata = metadata.index(unassignedShard.index()); totalDelayMillis = INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING.get(indexMetadata.getSettings()).getMillis(); - long remainingDelayNanos = unassignedInfo.getRemainingDelay(System.nanoTime(), indexMetadata.getSettings()); + long remainingDelayNanos = unassignedInfo.getRemainingDelay(System.nanoTime(), unassignedShard, metadata); remainingDelayMillis = TimeValue.timeValueNanos(remainingDelayNanos).millis(); } return AllocateUnassignedDecision.delayed(remainingDelayMillis, totalDelayMillis, nodeDecisions); diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/UnassignedInfoTests.java b/server/src/test/java/org/elasticsearch/cluster/routing/UnassignedInfoTests.java index 092f7c651b075..10b7c4ea95d68 100644 --- a/server/src/test/java/org/elasticsearch/cluster/routing/UnassignedInfoTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/routing/UnassignedInfoTests.java @@ -76,10 +76,21 @@ public void testSerialization() throws Exception { int failedAllocations = randomIntBetween(1, 100); Set failedNodes = IntStream.range(0, between(0, failedAllocations)) .mapToObj(n -> "failed-node-" + n).collect(Collectors.toSet()); - UnassignedInfo meta = reason == UnassignedInfo.Reason.ALLOCATION_FAILED ? - new UnassignedInfo(reason, randomBoolean() ? randomAlphaOfLength(4) : null, null, - failedAllocations, System.nanoTime(), System.currentTimeMillis(), false, AllocationStatus.NO_ATTEMPT, failedNodes): - new UnassignedInfo(reason, randomBoolean() ? randomAlphaOfLength(4) : null); + String lastAssignedNodeId = randomBoolean() ? randomAlphaOfLength(10) : null; + UnassignedInfo meta = reason == UnassignedInfo.Reason.ALLOCATION_FAILED + ? new UnassignedInfo( + reason, + randomBoolean() ? randomAlphaOfLength(4) : null, + null, + failedAllocations, + System.nanoTime(), + System.currentTimeMillis(), + false, + AllocationStatus.NO_ATTEMPT, + failedNodes, + lastAssignedNodeId + ) + : new UnassignedInfo(reason, randomBoolean() ? randomAlphaOfLength(4) : null); BytesStreamOutput out = new BytesStreamOutput(); meta.writeTo(out); out.close(); @@ -91,6 +102,7 @@ public void testSerialization() throws Exception { assertThat(read.getDetails(), equalTo(meta.getDetails())); assertThat(read.getNumFailedAllocations(), equalTo(meta.getNumFailedAllocations())); assertThat(read.getFailedNodeIds(), equalTo(meta.getFailedNodeIds())); + assertThat(read.getLastAllocatedNodeId(), equalTo(meta.getLastAllocatedNodeId())); } public void testIndexCreated() { @@ -292,23 +304,24 @@ public void testFailedShard() { /** * Verifies that delayed allocation calculation are correct. */ - public void testRemainingDelayCalculation() throws Exception { - final long baseTime = System.nanoTime(); - UnassignedInfo unassignedInfo = new UnassignedInfo(UnassignedInfo.Reason.NODE_LEFT, "test", null, 0, baseTime, - System.currentTimeMillis(), randomBoolean(), AllocationStatus.NO_ATTEMPT, Collections.emptySet()); - final long totalDelayNanos = TimeValue.timeValueMillis(10).nanos(); - final Settings indexSettings = Settings.builder() - .put(UnassignedInfo.INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING.getKey(), TimeValue.timeValueNanos(totalDelayNanos)).build(); - long delay = unassignedInfo.getRemainingDelay(baseTime, indexSettings); - assertThat(delay, equalTo(totalDelayNanos)); - long delta1 = randomIntBetween(1, (int) (totalDelayNanos - 1)); - delay = unassignedInfo.getRemainingDelay(baseTime + delta1, indexSettings); - assertThat(delay, equalTo(totalDelayNanos - delta1)); - delay = unassignedInfo.getRemainingDelay(baseTime + totalDelayNanos, indexSettings); - assertThat(delay, equalTo(0L)); - delay = unassignedInfo.getRemainingDelay(baseTime + totalDelayNanos + randomIntBetween(1, 20), indexSettings); - assertThat(delay, equalTo(0L)); - } +// public void testRemainingDelayCalculation() throws Exception { +// final long baseTime = System.nanoTime(); +// UnassignedInfo unassignedInfo = new UnassignedInfo(UnassignedInfo.Reason.NODE_LEFT, "test", null, 0, baseTime, +// System.currentTimeMillis(), randomBoolean(), AllocationStatus.NO_ATTEMPT, Collections.emptySet()); +// final long totalDelayNanos = TimeValue.timeValueMillis(10).nanos(); +// final Settings indexSettings = Settings.builder() +// .put(UnassignedInfo.INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING.getKey(), TimeValue.timeValueNanos(totalDelayNanos)).build(); +// long delay = unassignedInfo.getRemainingDelay(baseTime, indexSettings); +// assertThat(delay, equalTo(totalDelayNanos)); +// long delta1 = randomIntBetween(1, (int) (totalDelayNanos - 1)); +// delay = unassignedInfo.getRemainingDelay(baseTime + delta1, indexSettings); +// assertThat(delay, equalTo(totalDelayNanos - delta1)); +// delay = unassignedInfo.getRemainingDelay(baseTime + totalDelayNanos, indexSettings); +// assertThat(delay, equalTo(0L)); +// delay = unassignedInfo.getRemainingDelay(baseTime + totalDelayNanos + randomIntBetween(1, 20), indexSettings); +// assertThat(delay, equalTo(0L)); +// } + // NOCOMMIT FIX THIS TEST INSTEAD OF JUST COMMENTING IT public void testNumberOfDelayedUnassigned() throws Exception { diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/RestoreInProgressAllocationDeciderTests.java b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/RestoreInProgressAllocationDeciderTests.java index c25ca14028481..10e2ed88c8f02 100644 --- a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/RestoreInProgressAllocationDeciderTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/RestoreInProgressAllocationDeciderTests.java @@ -108,9 +108,18 @@ public void testCanAllocatePrimaryExistingInRestoreInProgress() { shardState = RestoreInProgress.State.FAILURE; UnassignedInfo currentInfo = primary.unassignedInfo(); - UnassignedInfo newInfo = new UnassignedInfo(currentInfo.getReason(), currentInfo.getMessage(), new IOException("i/o failure"), - currentInfo.getNumFailedAllocations(), currentInfo.getUnassignedTimeInNanos(), currentInfo.getUnassignedTimeInMillis(), - currentInfo.isDelayed(), currentInfo.getLastAllocationStatus(), currentInfo.getFailedNodeIds()); + UnassignedInfo newInfo = new UnassignedInfo( + currentInfo.getReason(), + currentInfo.getMessage(), + new IOException("i/o failure"), + currentInfo.getNumFailedAllocations(), + currentInfo.getUnassignedTimeInNanos(), + currentInfo.getUnassignedTimeInMillis(), + currentInfo.isDelayed(), + currentInfo.getLastAllocationStatus(), + currentInfo.getFailedNodeIds(), + currentInfo.getLastAllocatedNodeId() + ); primary = primary.updateUnassigned(newInfo, primary.recoverySource()); IndexRoutingTable indexRoutingTable = routingTable.index("test"); diff --git a/server/src/test/java/org/elasticsearch/gateway/ReplicaShardAllocatorTests.java b/server/src/test/java/org/elasticsearch/gateway/ReplicaShardAllocatorTests.java index 0b5a289a32f5e..81ab455d5f70f 100644 --- a/server/src/test/java/org/elasticsearch/gateway/ReplicaShardAllocatorTests.java +++ b/server/src/test/java/org/elasticsearch/gateway/ReplicaShardAllocatorTests.java @@ -191,7 +191,7 @@ public void testCancelRecoveryIfFoundCopyWithNoopRetentionLease() { } else { failedNodeIds = new HashSet<>(randomSubsetOf(Set.of("node-4", "node-5", "node-6", "node-7"))); unassignedInfo = new UnassignedInfo(UnassignedInfo.Reason.ALLOCATION_FAILED, null, null, randomIntBetween(1, 10), - System.nanoTime(), System.currentTimeMillis(), false, UnassignedInfo.AllocationStatus.NO_ATTEMPT, failedNodeIds); + System.nanoTime(), System.currentTimeMillis(), false, UnassignedInfo.AllocationStatus.NO_ATTEMPT, failedNodeIds, null); } RoutingAllocation allocation = onePrimaryOnNode1And1ReplicaRecovering(yesAllocationDeciders(), unassignedInfo); long retainingSeqNo = randomLongBetween(1, Long.MAX_VALUE); @@ -376,7 +376,7 @@ public void testNotCancellingRecoveryIfSyncedOnExistingRecovery() { unassignedInfo = new UnassignedInfo(UnassignedInfo.Reason.CLUSTER_RECOVERED, null); } else { unassignedInfo = new UnassignedInfo(UnassignedInfo.Reason.ALLOCATION_FAILED, null, null, randomIntBetween(1, 10), - System.nanoTime(), System.currentTimeMillis(), false, UnassignedInfo.AllocationStatus.NO_ATTEMPT, Set.of("node-4")); + System.nanoTime(), System.currentTimeMillis(), false, UnassignedInfo.AllocationStatus.NO_ATTEMPT, Set.of("node-4"), null); } RoutingAllocation allocation = onePrimaryOnNode1And1ReplicaRecovering(yesAllocationDeciders(), unassignedInfo); List retentionLeases = new ArrayList<>(); @@ -417,7 +417,7 @@ public void testDoNotCancelForBrokenNode() { } UnassignedInfo unassignedInfo = new UnassignedInfo(UnassignedInfo.Reason.ALLOCATION_FAILED, null, null, randomIntBetween(failedNodes.size(), 10), System.nanoTime(), System.currentTimeMillis(), false, - UnassignedInfo.AllocationStatus.NO_ATTEMPT, failedNodes); + UnassignedInfo.AllocationStatus.NO_ATTEMPT, failedNodes, null); RoutingAllocation allocation = onePrimaryOnNode1And1ReplicaRecovering(yesAllocationDeciders(), unassignedInfo); long retainingSeqNoOnPrimary = randomLongBetween(0, Long.MAX_VALUE); List retentionLeases = Arrays.asList( @@ -454,7 +454,7 @@ private RoutingAllocation onePrimaryOnNode1And1Replica(AllocationDeciders decide RecoverySource.PeerRecoverySource.INSTANCE, new UnassignedInfo(reason, null, null, failedAllocations, System.nanoTime(), System.currentTimeMillis(), delayed, UnassignedInfo.AllocationStatus.NO_ATTEMPT, - Collections.emptySet()))) + Collections.emptySet(), null))) .build()) ) .build(); diff --git a/x-pack/plugin/shutdown/src/internalClusterTest/java/org/elasticsearch/xpack/shutdown/NodeShutdownDelayedAllocationIT.java b/x-pack/plugin/shutdown/src/internalClusterTest/java/org/elasticsearch/xpack/shutdown/NodeShutdownDelayedAllocationIT.java index 8d57af55ff84a..85fc9a972bdc6 100644 --- a/x-pack/plugin/shutdown/src/internalClusterTest/java/org/elasticsearch/xpack/shutdown/NodeShutdownDelayedAllocationIT.java +++ b/x-pack/plugin/shutdown/src/internalClusterTest/java/org/elasticsearch/xpack/shutdown/NodeShutdownDelayedAllocationIT.java @@ -44,12 +44,13 @@ public void testShardAllocationIsDelayedForRestartingNode() throws Exception { ensureGreen("test"); indexRandomData(); - final String nodeToRestart = findNodeWithShard(); - Settings nodeToRestartDataPathSettings = internalCluster().dataPathSettings(nodeToRestart); + final String nodeToRestartId = findIdOfNodeWithShard(); + final String nodeToRestartName = findNodeNameFromId(nodeToRestartId); + Settings nodeToRestartDataPathSettings = internalCluster().dataPathSettings(nodeToRestartName); // Mark the node for shutdown PutShutdownNodeAction.Request putShutdownRequest = new PutShutdownNodeAction.Request( - nodeToRestart, + nodeToRestartId, SingleNodeShutdownMetadata.Type.RESTART, this.getTestName() ); @@ -57,10 +58,12 @@ public void testShardAllocationIsDelayedForRestartingNode() throws Exception { assertTrue(putShutdownResponse.isAcknowledged()); // Actually stop the node - internalCluster().stopRandomNode(InternalTestCluster.nameFilter(nodeToRestart)); + internalCluster().stopRandomNode(InternalTestCluster.nameFilter(nodeToRestartName)); // Verify that the shard's allocation is delayed - assertThat(client().admin().cluster().prepareHealth().get().getDelayedUnassignedShards(), equalTo(1)); + assertBusy(() -> { + assertThat(client().admin().cluster().prepareHealth().get().getDelayedUnassignedShards(), equalTo(1)); + }); // Bring the node back internalCluster().startNode(nodeToRestartDataPathSettings); // this will use the same data location as the stopped node @@ -83,10 +86,15 @@ private void indexRandomData() throws Exception { indexRandom(true, builders); } - private String findNodeWithShard() { + private String findIdOfNodeWithShard() { ClusterState state = client().admin().cluster().prepareState().get().getState(); List startedShards = state.routingTable().shardsWithState(ShardRoutingState.STARTED); Collections.shuffle(startedShards,random()); - return state.nodes().get(startedShards.get(0).currentNodeId()).getName(); + return startedShards.get(0).currentNodeId(); + } + + private String findNodeNameFromId(String id) { + ClusterState state = client().admin().cluster().prepareState().get().getState(); + return state.nodes().get(id).getName(); } } From b87dfc4387fc6a77149d910f4b437317beee5f01 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Wed, 21 Jul 2021 14:30:22 -0600 Subject: [PATCH 03/47] Clean up getRemainingDelay params + reinstate unit test --- .../cluster/routing/UnassignedInfo.java | 20 +++++++--- .../routing/allocation/AllocationService.java | 4 +- .../gateway/ReplicaShardAllocator.java | 6 ++- .../cluster/routing/UnassignedInfoTests.java | 38 ++++++++++--------- 4 files changed, 42 insertions(+), 26 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/cluster/routing/UnassignedInfo.java b/server/src/main/java/org/elasticsearch/cluster/routing/UnassignedInfo.java index b563345c7f68a..2f81c1e51a771 100644 --- a/server/src/main/java/org/elasticsearch/cluster/routing/UnassignedInfo.java +++ b/server/src/main/java/org/elasticsearch/cluster/routing/UnassignedInfo.java @@ -26,6 +26,7 @@ import org.elasticsearch.core.TimeValue; import org.elasticsearch.common.xcontent.ToXContentFragment; import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.index.Index; import java.io.IOException; import java.time.Instant; @@ -33,6 +34,7 @@ import java.util.Collections; import java.util.List; import java.util.Locale; +import java.util.Map; import java.util.Objects; import java.util.Optional; import java.util.Set; @@ -390,12 +392,16 @@ public Set getFailedNodeIds() { * * @return calculated delay in nanoseconds */ - // GWB-> NOCOMMIT don't need the full ShardRouting here - public long getRemainingDelay(final long nanoTimeNow, final ShardRouting routing, final Metadata metadata) { - long delayTimeoutNanos = Optional.ofNullable(metadata.nodeShutdowns().get(lastAllocatedNodeId)) + public long getRemainingDelay( + final long nanoTimeNow, + final Settings indexSettings, + final Map nodeShutdowns + ) { + long delayTimeoutNanos = Optional.ofNullable(lastAllocatedNodeId) + .map(nodeShutdowns::get) .filter(shutdownMetadata -> SingleNodeShutdownMetadata.Type.RESTART.equals(shutdownMetadata.getType())) .map(shutdownMetadata -> new TimeValue(10, TimeUnit.MINUTES).getNanos()) // GWB-> NOCOMMIT plumb in a setting here - .orElse(INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING.get(metadata.getIndexSafe(routing.index()).getSettings()).nanos()); + .orElse(INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING.get(indexSettings).nanos()); assert nanoTimeNow >= unassignedTimeNanos; return Math.max(0L, delayTimeoutNanos - (nanoTimeNow - unassignedTimeNanos)); } @@ -427,7 +433,11 @@ public static long findNextDelayedAllocation(long currentNanoTime, ClusterState if (unassignedInfo.isDelayed()) { Settings indexSettings = metadata.index(shard.index()).getSettings(); // calculate next time to schedule - final long newComputedLeftDelayNanos = unassignedInfo.getRemainingDelay(currentNanoTime, shard, metadata); + final long newComputedLeftDelayNanos = unassignedInfo.getRemainingDelay( + currentNanoTime, + indexSettings, + metadata.nodeShutdowns() + ); if (newComputedLeftDelayNanos < nextDelayNanos) { nextDelayNanos = newComputedLeftDelayNanos; } diff --git a/server/src/main/java/org/elasticsearch/cluster/routing/allocation/AllocationService.java b/server/src/main/java/org/elasticsearch/cluster/routing/allocation/AllocationService.java index 684f15c29b820..6050d9dacc8e2 100644 --- a/server/src/main/java/org/elasticsearch/cluster/routing/allocation/AllocationService.java +++ b/server/src/main/java/org/elasticsearch/cluster/routing/allocation/AllocationService.java @@ -305,8 +305,8 @@ private void removeDelayMarkers(RoutingAllocation allocation) { if (unassignedInfo.isDelayed()) { final long newComputedLeftDelayNanos = unassignedInfo.getRemainingDelay( allocation.getCurrentNanoTime(), - shardRouting, - metadata + metadata.getIndexSafe(shardRouting.index()).getSettings(), + metadata.nodeShutdowns() ); if (newComputedLeftDelayNanos == 0) { unassignedIterator.updateUnassigned( diff --git a/server/src/main/java/org/elasticsearch/gateway/ReplicaShardAllocator.java b/server/src/main/java/org/elasticsearch/gateway/ReplicaShardAllocator.java index 9a81b07dcf2a9..91508cf5ea46d 100644 --- a/server/src/main/java/org/elasticsearch/gateway/ReplicaShardAllocator.java +++ b/server/src/main/java/org/elasticsearch/gateway/ReplicaShardAllocator.java @@ -213,7 +213,11 @@ public AllocateUnassignedDecision makeAllocationDecision(final ShardRouting unas Metadata metadata = allocation.metadata(); IndexMetadata indexMetadata = metadata.index(unassignedShard.index()); totalDelayMillis = INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING.get(indexMetadata.getSettings()).getMillis(); - long remainingDelayNanos = unassignedInfo.getRemainingDelay(System.nanoTime(), unassignedShard, metadata); + long remainingDelayNanos = unassignedInfo.getRemainingDelay( + System.nanoTime(), + indexMetadata.getSettings(), + metadata.nodeShutdowns() + ); remainingDelayMillis = TimeValue.timeValueNanos(remainingDelayNanos).millis(); } return AllocateUnassignedDecision.delayed(remainingDelayMillis, totalDelayMillis, nodeDecisions); diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/UnassignedInfoTests.java b/server/src/test/java/org/elasticsearch/cluster/routing/UnassignedInfoTests.java index 10b7c4ea95d68..3a23886efa0dc 100644 --- a/server/src/test/java/org/elasticsearch/cluster/routing/UnassignedInfoTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/routing/UnassignedInfoTests.java @@ -16,6 +16,7 @@ import org.elasticsearch.cluster.ESAllocationTestCase; import org.elasticsearch.cluster.metadata.IndexMetadata; import org.elasticsearch.cluster.metadata.Metadata; +import org.elasticsearch.cluster.metadata.SingleNodeShutdownMetadata; import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.cluster.routing.RecoverySource.SnapshotRecoverySource; import org.elasticsearch.cluster.routing.UnassignedInfo.AllocationStatus; @@ -34,6 +35,7 @@ import java.io.IOException; import java.nio.ByteBuffer; import java.util.Collections; +import java.util.Map; import java.util.Set; import java.util.stream.Collectors; import java.util.stream.IntStream; @@ -304,24 +306,24 @@ public void testFailedShard() { /** * Verifies that delayed allocation calculation are correct. */ -// public void testRemainingDelayCalculation() throws Exception { -// final long baseTime = System.nanoTime(); -// UnassignedInfo unassignedInfo = new UnassignedInfo(UnassignedInfo.Reason.NODE_LEFT, "test", null, 0, baseTime, -// System.currentTimeMillis(), randomBoolean(), AllocationStatus.NO_ATTEMPT, Collections.emptySet()); -// final long totalDelayNanos = TimeValue.timeValueMillis(10).nanos(); -// final Settings indexSettings = Settings.builder() -// .put(UnassignedInfo.INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING.getKey(), TimeValue.timeValueNanos(totalDelayNanos)).build(); -// long delay = unassignedInfo.getRemainingDelay(baseTime, indexSettings); -// assertThat(delay, equalTo(totalDelayNanos)); -// long delta1 = randomIntBetween(1, (int) (totalDelayNanos - 1)); -// delay = unassignedInfo.getRemainingDelay(baseTime + delta1, indexSettings); -// assertThat(delay, equalTo(totalDelayNanos - delta1)); -// delay = unassignedInfo.getRemainingDelay(baseTime + totalDelayNanos, indexSettings); -// assertThat(delay, equalTo(0L)); -// delay = unassignedInfo.getRemainingDelay(baseTime + totalDelayNanos + randomIntBetween(1, 20), indexSettings); -// assertThat(delay, equalTo(0L)); -// } - // NOCOMMIT FIX THIS TEST INSTEAD OF JUST COMMENTING IT + public void testRemainingDelayCalculation() throws Exception { + final long baseTime = System.nanoTime(); + UnassignedInfo unassignedInfo = new UnassignedInfo(UnassignedInfo.Reason.NODE_LEFT, "test", null, 0, baseTime, + System.currentTimeMillis(), randomBoolean(), AllocationStatus.NO_ATTEMPT, Collections.emptySet(), null); + final long totalDelayNanos = TimeValue.timeValueMillis(10).nanos(); + final Settings indexSettings = Settings.builder() + .put(UnassignedInfo.INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING.getKey(), TimeValue.timeValueNanos(totalDelayNanos)).build(); + Map nodeShutdowns = Collections.emptyMap(); + long delay = unassignedInfo.getRemainingDelay(baseTime, indexSettings, nodeShutdowns); + assertThat(delay, equalTo(totalDelayNanos)); + long delta1 = randomIntBetween(1, (int) (totalDelayNanos - 1)); + delay = unassignedInfo.getRemainingDelay(baseTime + delta1, indexSettings, nodeShutdowns); + assertThat(delay, equalTo(totalDelayNanos - delta1)); + delay = unassignedInfo.getRemainingDelay(baseTime + totalDelayNanos, indexSettings, nodeShutdowns); + assertThat(delay, equalTo(0L)); + delay = unassignedInfo.getRemainingDelay(baseTime + totalDelayNanos + randomIntBetween(1, 20), indexSettings, nodeShutdowns); + assertThat(delay, equalTo(0L)); + } public void testNumberOfDelayedUnassigned() throws Exception { From 4f8f0bda7651a015d8455e682aee402ab0e7425a Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Wed, 21 Jul 2021 14:52:25 -0600 Subject: [PATCH 04/47] Back to taking a `Metadata` since we need a setting... --- .../cluster/routing/UnassignedInfo.java | 7 +++-- .../routing/allocation/AllocationService.java | 2 +- .../gateway/ReplicaShardAllocator.java | 2 +- .../cluster/routing/UnassignedInfoTests.java | 28 ++++++++++++++----- 4 files changed, 28 insertions(+), 11 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/cluster/routing/UnassignedInfo.java b/server/src/main/java/org/elasticsearch/cluster/routing/UnassignedInfo.java index 2f81c1e51a771..b5877df0d0e60 100644 --- a/server/src/main/java/org/elasticsearch/cluster/routing/UnassignedInfo.java +++ b/server/src/main/java/org/elasticsearch/cluster/routing/UnassignedInfo.java @@ -395,8 +395,11 @@ public Set getFailedNodeIds() { public long getRemainingDelay( final long nanoTimeNow, final Settings indexSettings, - final Map nodeShutdowns + final Metadata metadata ) { + Map nodeShutdowns = metadata.nodeShutdowns() != null + ? metadata.nodeShutdowns() + : Collections.emptyMap(); long delayTimeoutNanos = Optional.ofNullable(lastAllocatedNodeId) .map(nodeShutdowns::get) .filter(shutdownMetadata -> SingleNodeShutdownMetadata.Type.RESTART.equals(shutdownMetadata.getType())) @@ -436,7 +439,7 @@ public static long findNextDelayedAllocation(long currentNanoTime, ClusterState final long newComputedLeftDelayNanos = unassignedInfo.getRemainingDelay( currentNanoTime, indexSettings, - metadata.nodeShutdowns() + metadata ); if (newComputedLeftDelayNanos < nextDelayNanos) { nextDelayNanos = newComputedLeftDelayNanos; diff --git a/server/src/main/java/org/elasticsearch/cluster/routing/allocation/AllocationService.java b/server/src/main/java/org/elasticsearch/cluster/routing/allocation/AllocationService.java index 6050d9dacc8e2..a5a0b47b1d592 100644 --- a/server/src/main/java/org/elasticsearch/cluster/routing/allocation/AllocationService.java +++ b/server/src/main/java/org/elasticsearch/cluster/routing/allocation/AllocationService.java @@ -306,7 +306,7 @@ private void removeDelayMarkers(RoutingAllocation allocation) { final long newComputedLeftDelayNanos = unassignedInfo.getRemainingDelay( allocation.getCurrentNanoTime(), metadata.getIndexSafe(shardRouting.index()).getSettings(), - metadata.nodeShutdowns() + metadata ); if (newComputedLeftDelayNanos == 0) { unassignedIterator.updateUnassigned( diff --git a/server/src/main/java/org/elasticsearch/gateway/ReplicaShardAllocator.java b/server/src/main/java/org/elasticsearch/gateway/ReplicaShardAllocator.java index 91508cf5ea46d..60d398316cd5a 100644 --- a/server/src/main/java/org/elasticsearch/gateway/ReplicaShardAllocator.java +++ b/server/src/main/java/org/elasticsearch/gateway/ReplicaShardAllocator.java @@ -216,7 +216,7 @@ public AllocateUnassignedDecision makeAllocationDecision(final ShardRouting unas long remainingDelayNanos = unassignedInfo.getRemainingDelay( System.nanoTime(), indexMetadata.getSettings(), - metadata.nodeShutdowns() + metadata ); remainingDelayMillis = TimeValue.timeValueNanos(remainingDelayNanos).millis(); } diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/UnassignedInfoTests.java b/server/src/test/java/org/elasticsearch/cluster/routing/UnassignedInfoTests.java index 3a23886efa0dc..47937124c83c7 100644 --- a/server/src/test/java/org/elasticsearch/cluster/routing/UnassignedInfoTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/routing/UnassignedInfoTests.java @@ -16,6 +16,7 @@ import org.elasticsearch.cluster.ESAllocationTestCase; import org.elasticsearch.cluster.metadata.IndexMetadata; import org.elasticsearch.cluster.metadata.Metadata; +import org.elasticsearch.cluster.metadata.NodesShutdownMetadata; import org.elasticsearch.cluster.metadata.SingleNodeShutdownMetadata; import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.cluster.routing.RecoverySource.SnapshotRecoverySource; @@ -308,20 +309,33 @@ public void testFailedShard() { */ public void testRemainingDelayCalculation() throws Exception { final long baseTime = System.nanoTime(); - UnassignedInfo unassignedInfo = new UnassignedInfo(UnassignedInfo.Reason.NODE_LEFT, "test", null, 0, baseTime, - System.currentTimeMillis(), randomBoolean(), AllocationStatus.NO_ATTEMPT, Collections.emptySet(), null); + UnassignedInfo unassignedInfo = new UnassignedInfo( + UnassignedInfo.Reason.NODE_LEFT, + "test", + null, + 0, + baseTime, + System.currentTimeMillis(), + randomBoolean(), + AllocationStatus.NO_ATTEMPT, + Collections.emptySet(), + randomFrom("bogusNodeId", null) + ); final long totalDelayNanos = TimeValue.timeValueMillis(10).nanos(); final Settings indexSettings = Settings.builder() .put(UnassignedInfo.INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING.getKey(), TimeValue.timeValueNanos(totalDelayNanos)).build(); - Map nodeShutdowns = Collections.emptyMap(); - long delay = unassignedInfo.getRemainingDelay(baseTime, indexSettings, nodeShutdowns); + Metadata metadata = Metadata.builder() + .persistentSettings(Settings.EMPTY) + .putCustom(NodesShutdownMetadata.TYPE, new NodesShutdownMetadata(Collections.emptyMap())) + .build(); + long delay = unassignedInfo.getRemainingDelay(baseTime, indexSettings, metadata); assertThat(delay, equalTo(totalDelayNanos)); long delta1 = randomIntBetween(1, (int) (totalDelayNanos - 1)); - delay = unassignedInfo.getRemainingDelay(baseTime + delta1, indexSettings, nodeShutdowns); + delay = unassignedInfo.getRemainingDelay(baseTime + delta1, indexSettings, metadata); assertThat(delay, equalTo(totalDelayNanos - delta1)); - delay = unassignedInfo.getRemainingDelay(baseTime + totalDelayNanos, indexSettings, nodeShutdowns); + delay = unassignedInfo.getRemainingDelay(baseTime + totalDelayNanos, indexSettings, metadata); assertThat(delay, equalTo(0L)); - delay = unassignedInfo.getRemainingDelay(baseTime + totalDelayNanos + randomIntBetween(1, 20), indexSettings, nodeShutdowns); + delay = unassignedInfo.getRemainingDelay(baseTime + totalDelayNanos + randomIntBetween(1, 20), indexSettings, metadata); assertThat(delay, equalTo(0L)); } From a8fde872d3d9355c0dbedc21766f74e7d51e32f4 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Wed, 21 Jul 2021 16:07:45 -0600 Subject: [PATCH 05/47] Add a bunch of unit tests --- .../cluster/routing/UnassignedInfo.java | 5 +- .../cluster/routing/UnassignedInfoTests.java | 89 +++++++++++++++++-- 2 files changed, 83 insertions(+), 11 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/cluster/routing/UnassignedInfo.java b/server/src/main/java/org/elasticsearch/cluster/routing/UnassignedInfo.java index b5877df0d0e60..99b627ba3b8d8 100644 --- a/server/src/main/java/org/elasticsearch/cluster/routing/UnassignedInfo.java +++ b/server/src/main/java/org/elasticsearch/cluster/routing/UnassignedInfo.java @@ -15,7 +15,6 @@ import org.elasticsearch.cluster.metadata.SingleNodeShutdownMetadata; import org.elasticsearch.cluster.routing.allocation.RoutingAllocation; import org.elasticsearch.cluster.routing.allocation.decider.Decision; -import org.elasticsearch.core.Nullable; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.Writeable; @@ -23,10 +22,10 @@ import org.elasticsearch.common.settings.Setting.Property; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.time.DateFormatter; -import org.elasticsearch.core.TimeValue; import org.elasticsearch.common.xcontent.ToXContentFragment; import org.elasticsearch.common.xcontent.XContentBuilder; -import org.elasticsearch.index.Index; +import org.elasticsearch.core.Nullable; +import org.elasticsearch.core.TimeValue; import java.io.IOException; import java.time.Instant; diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/UnassignedInfoTests.java b/server/src/test/java/org/elasticsearch/cluster/routing/UnassignedInfoTests.java index 47937124c83c7..2acca6d5a5fe9 100644 --- a/server/src/test/java/org/elasticsearch/cluster/routing/UnassignedInfoTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/routing/UnassignedInfoTests.java @@ -36,8 +36,11 @@ import java.io.IOException; import java.nio.ByteBuffer; import java.util.Collections; +import java.util.EnumSet; +import java.util.HashMap; import java.util.Map; import java.util.Set; +import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; import java.util.stream.IntStream; @@ -305,9 +308,72 @@ public void testFailedShard() { } /** - * Verifies that delayed allocation calculation are correct. + * Verifies that delayed allocation calculation are correct when there are no registered node shutdowns. */ - public void testRemainingDelayCalculation() throws Exception { + public void testRemainingDelayCalculationWithNoShutdowns() throws Exception { + checkRemainingDelayCalculation("bogusNodeId", TimeValue.timeValueNanos(10), Settings.EMPTY, null, TimeValue.timeValueNanos(10)); + } + + /** + * Verifies that delayed allocation calculation are correct when there have never been any registered node shutdowns. This ensures we + * don't accidentally introduce an NPE in trying to read the custom metadata. + */ + public void testRemainingDelayCalculationWithNoShutdownsEverRegistered() throws Exception { + checkRemainingDelayCalculation("bogusNodeId", TimeValue.timeValueNanos(10), Settings.EMPTY, null, TimeValue.timeValueNanos(10)); + } + + /** + * Verifies that delayed allocation calculations are correct when there are registered node shutdowns for nodes which are not relevant + * to the shard currently being evaluated. + */ + public void testRemainingDelayCalculationsWithUnrelatedShutdowns() throws Exception { + String lastNodeId = "bogusNodeId"; + Map shutdowns = new HashMap<>(); + int numberOfShutdowns = randomIntBetween(1,15); + for (int i = 0; i <= numberOfShutdowns; i++) { + SingleNodeShutdownMetadata shutdown = SingleNodeShutdownMetadata.builder() + .setNodeId(randomValueOtherThan(lastNodeId, () -> randomAlphaOfLengthBetween(5,10))) + .setReason(this.getTestName()) + .setStartedAtMillis(randomNonNegativeLong()) + .setType(randomFrom(EnumSet.allOf(SingleNodeShutdownMetadata.Type.class))) + .build(); + shutdowns.put(shutdown.getNodeId(), shutdown); + } + checkRemainingDelayCalculation(lastNodeId, TimeValue.timeValueNanos(10), Settings.EMPTY, shutdowns, TimeValue.timeValueNanos(10)); + } + + /** + * Verifies that delay calculation is not impacted when the node the shard was last assigned to was registered for removal. + */ + public void testRemainingDelayCalculationWhenNodeIsShuttingDownForRemoval() throws Exception { + String lastNodeId = "bogusNodeId"; + Map shutdowns = new HashMap<>(); + SingleNodeShutdownMetadata shutdown = SingleNodeShutdownMetadata.builder() + .setNodeId(lastNodeId) + .setReason(this.getTestName()) + .setStartedAtMillis(randomNonNegativeLong()) + .setType(SingleNodeShutdownMetadata.Type.REMOVE) + .build(); + shutdowns.put(shutdown.getNodeId(), shutdown); + + checkRemainingDelayCalculation(lastNodeId, TimeValue.timeValueNanos(10), Settings.EMPTY, shutdowns, TimeValue.timeValueNanos(10)); + } + + /** + * Verifies that the delay calculation uses the delay value for nodes known to be restarting, because they are registered for a + * `RESTART`-type shutdown, rather than the default delay. + */ + public void testRemainingDelayCalculationWhenNodeIsKnownToBeRestarting() throws Exception { + // GWB-> To be written! + } + + private void checkRemainingDelayCalculation( + String lastNodeId, + TimeValue indexLevelTimeoutSetting, + Settings clusterSettings, + Map nodeShutdowns, + TimeValue expectedTotalDelay + ) throws Exception { final long baseTime = System.nanoTime(); UnassignedInfo unassignedInfo = new UnassignedInfo( UnassignedInfo.Reason.NODE_LEFT, @@ -319,15 +385,22 @@ public void testRemainingDelayCalculation() throws Exception { randomBoolean(), AllocationStatus.NO_ATTEMPT, Collections.emptySet(), - randomFrom("bogusNodeId", null) + lastNodeId ); - final long totalDelayNanos = TimeValue.timeValueMillis(10).nanos(); + final long totalDelayNanos = expectedTotalDelay.nanos(); final Settings indexSettings = Settings.builder() - .put(UnassignedInfo.INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING.getKey(), TimeValue.timeValueNanos(totalDelayNanos)).build(); - Metadata metadata = Metadata.builder() - .persistentSettings(Settings.EMPTY) - .putCustom(NodesShutdownMetadata.TYPE, new NodesShutdownMetadata(Collections.emptyMap())) + .put(UnassignedInfo.INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING.getKey(), indexLevelTimeoutSetting) .build(); + Metadata.Builder metadataBuilder = Metadata.builder(); + if (randomBoolean()) { + metadataBuilder.persistentSettings(clusterSettings); + } else { + metadataBuilder.transientSettings(clusterSettings); + } + if (nodeShutdowns != null) { + metadataBuilder.putCustom(NodesShutdownMetadata.TYPE, new NodesShutdownMetadata(nodeShutdowns)); + } + Metadata metadata = metadataBuilder.build(); long delay = unassignedInfo.getRemainingDelay(baseTime, indexSettings, metadata); assertThat(delay, equalTo(totalDelayNanos)); long delta1 = randomIntBetween(1, (int) (totalDelayNanos - 1)); From 8eadf322c2246144eefe1cf82354169f411cc455 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Wed, 21 Jul 2021 16:42:55 -0600 Subject: [PATCH 06/47] Add shard allocation delay to `SingleNodeShutdownMetadata` --- .../metadata/SingleNodeShutdownMetadata.java | 55 +++++++++++++++++-- .../metadata/NodesShutdownMetadataTests.java | 16 ++++-- 2 files changed, 62 insertions(+), 9 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/cluster/metadata/SingleNodeShutdownMetadata.java b/server/src/main/java/org/elasticsearch/cluster/metadata/SingleNodeShutdownMetadata.java index bfe73c4d409d9..6511054785dd5 100644 --- a/server/src/main/java/org/elasticsearch/cluster/metadata/SingleNodeShutdownMetadata.java +++ b/server/src/main/java/org/elasticsearch/cluster/metadata/SingleNodeShutdownMetadata.java @@ -10,6 +10,7 @@ import org.elasticsearch.cluster.AbstractDiffable; import org.elasticsearch.cluster.Diffable; +import org.elasticsearch.common.xcontent.ObjectParser; import org.elasticsearch.common.xcontent.ParseField; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; @@ -17,6 +18,8 @@ import org.elasticsearch.common.xcontent.ToXContentObject; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentParser; +import org.elasticsearch.core.Nullable; +import org.elasticsearch.core.TimeValue; import java.io.IOException; import java.util.Locale; @@ -35,6 +38,7 @@ public class SingleNodeShutdownMetadata extends AbstractDiffable PARSER = new ConstructingObjectParser<>( "node_shutdown_info", @@ -42,7 +46,8 @@ public class SingleNodeShutdownMetadata extends AbstractDiffable TimeValue.parseTimeValue(p.textOrNull(), SHARD_REALLOCATION_DELAY.getPreferredName()), SHARD_REALLOCATION_DELAY, + ObjectParser.ValueType.STRING_OR_NULL + ); } public static SingleNodeShutdownMetadata parse(XContentParser parser) { return PARSER.apply(parser, null); } + public static final TimeValue DEFAULT_RESTART_SHARD_ALLOCATION_DELAY = TimeValue.timeValueMinutes(10); + private final String nodeId; private final Type type; private final String reason; private final long startedAtMillis; + @Nullable private final TimeValue shardReallocationDelay; /** * @param nodeId The node ID that this shutdown metadata refers to. @@ -72,12 +85,17 @@ private SingleNodeShutdownMetadata( String nodeId, Type type, String reason, - long startedAtMillis + long startedAtMillis, + @Nullable TimeValue shardReallocationDelay ) { this.nodeId = Objects.requireNonNull(nodeId, "node ID must not be null"); this.type = Objects.requireNonNull(type, "shutdown type must not be null"); this.reason = Objects.requireNonNull(reason, "shutdown reason must not be null"); this.startedAtMillis = startedAtMillis; + if (shardReallocationDelay != null && Type.RESTART.equals(type) == false) { + throw new IllegalArgumentException("shard allocation delay is only valid for RESTART-type shutdowns"); + } + this.shardReallocationDelay = shardReallocationDelay; } public SingleNodeShutdownMetadata(StreamInput in) throws IOException { @@ -85,6 +103,7 @@ public SingleNodeShutdownMetadata(StreamInput in) throws IOException { this.type = in.readEnum(Type.class); this.reason = in.readString(); this.startedAtMillis = in.readVLong(); + this.shardReallocationDelay = in.readOptionalTimeValue(); } /** @@ -115,12 +134,22 @@ public long getStartedAtMillis() { return startedAtMillis; } + /** + * @return The amount of time shard reallocation should be delayed for shards on this node, so that they will not be automatically + * reassigned while the node is restarting. Will be {@code null} for non-restart shutdowns. + */ + @Nullable + public TimeValue getShardReallocationDelay() { + return shardReallocationDelay; + } + @Override public void writeTo(StreamOutput out) throws IOException { out.writeString(nodeId); out.writeEnum(type); out.writeString(reason); out.writeVLong(startedAtMillis); + out.writeOptionalTimeValue(shardReallocationDelay); } @Override @@ -131,6 +160,9 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws builder.field(TYPE_FIELD.getPreferredName(), type); builder.field(REASON_FIELD.getPreferredName(), reason); builder.timeField(STARTED_AT_MILLIS_FIELD.getPreferredName(), STARTED_AT_READABLE_FIELD, startedAtMillis); + if (shardReallocationDelay != null) { + builder.field(SHARD_REALLOCATION_DELAY.getPreferredName(), shardReallocationDelay.getStringRep()); + } } builder.endObject(); @@ -145,7 +177,8 @@ public boolean equals(Object o) { return getStartedAtMillis() == that.getStartedAtMillis() && getNodeId().equals(that.getNodeId()) && getType() == that.getType() - && getReason().equals(that.getReason()); + && getReason().equals(that.getReason()) + && Objects.equals(getShardReallocationDelay(), that.getShardReallocationDelay()); } @Override @@ -154,7 +187,8 @@ public int hashCode() { getNodeId(), getType(), getReason(), - getStartedAtMillis() + getStartedAtMillis(), + getShardReallocationDelay() ); } @@ -178,6 +212,7 @@ public static class Builder { private Type type; private String reason; private long startedAtMillis = -1; + private TimeValue shardReallocationDelay; private Builder() {} @@ -217,15 +252,25 @@ public Builder setStartedAtMillis(long startedAtMillis) { return this; } + public Builder setShardReallocationDelay(TimeValue shardReallocationDelay) { + this.shardReallocationDelay = shardReallocationDelay; + return this; + } + public SingleNodeShutdownMetadata build() { if (startedAtMillis == -1) { throw new IllegalArgumentException("start timestamp must be set"); } + TimeValue delayOrDefault = shardReallocationDelay; + if (Type.RESTART.equals(type) && delayOrDefault == null) { + delayOrDefault = DEFAULT_RESTART_SHARD_ALLOCATION_DELAY; + } return new SingleNodeShutdownMetadata( nodeId, type, reason, - startedAtMillis + startedAtMillis, + delayOrDefault ); } } diff --git a/server/src/test/java/org/elasticsearch/cluster/metadata/NodesShutdownMetadataTests.java b/server/src/test/java/org/elasticsearch/cluster/metadata/NodesShutdownMetadataTests.java index 363659fbacec0..899fd904f1336 100644 --- a/server/src/test/java/org/elasticsearch/cluster/metadata/NodesShutdownMetadataTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/metadata/NodesShutdownMetadataTests.java @@ -11,6 +11,7 @@ import org.elasticsearch.cluster.Diff; import org.elasticsearch.common.io.stream.Writeable; import org.elasticsearch.common.xcontent.XContentParser; +import org.elasticsearch.core.TimeValue; import org.elasticsearch.test.AbstractDiffableSerializationTestCase; import java.io.IOException; @@ -78,11 +79,18 @@ protected NodesShutdownMetadata createTestInstance() { } private SingleNodeShutdownMetadata randomNodeShutdownInfo() { - return SingleNodeShutdownMetadata.builder().setNodeId(randomAlphaOfLength(5)) - .setType(randomBoolean() ? SingleNodeShutdownMetadata.Type.REMOVE : SingleNodeShutdownMetadata.Type.RESTART) + final SingleNodeShutdownMetadata.Type type = randomBoolean() + ? SingleNodeShutdownMetadata.Type.REMOVE + : SingleNodeShutdownMetadata.Type.RESTART; + final SingleNodeShutdownMetadata.Builder builder = SingleNodeShutdownMetadata.builder() + .setNodeId(randomAlphaOfLength(5)) + .setType(type) .setReason(randomAlphaOfLength(5)) - .setStartedAtMillis(randomNonNegativeLong()) - .build(); + .setStartedAtMillis(randomNonNegativeLong()); + if (type.equals(SingleNodeShutdownMetadata.Type.RESTART) && randomBoolean()) { + builder.setShardReallocationDelay(TimeValue.parseTimeValue(randomTimeValue(), this.getTestName())); + } + return builder.build(); } @Override From 917bde5e8eea32445abcad5548ac61b5e3f549e0 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Wed, 21 Jul 2021 16:57:30 -0600 Subject: [PATCH 07/47] Value plumbed into getRemainingDelay + more tests --- .../cluster/routing/UnassignedInfo.java | 11 ++- .../routing/allocation/AllocationService.java | 2 +- .../gateway/ReplicaShardAllocator.java | 2 +- .../cluster/routing/UnassignedInfoTests.java | 76 +++++++++++++------ 4 files changed, 58 insertions(+), 33 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/cluster/routing/UnassignedInfo.java b/server/src/main/java/org/elasticsearch/cluster/routing/UnassignedInfo.java index 99b627ba3b8d8..d00c3534ff051 100644 --- a/server/src/main/java/org/elasticsearch/cluster/routing/UnassignedInfo.java +++ b/server/src/main/java/org/elasticsearch/cluster/routing/UnassignedInfo.java @@ -394,15 +394,14 @@ public Set getFailedNodeIds() { public long getRemainingDelay( final long nanoTimeNow, final Settings indexSettings, - final Metadata metadata + final Map nodesShutdownMap ) { - Map nodeShutdowns = metadata.nodeShutdowns() != null - ? metadata.nodeShutdowns() - : Collections.emptyMap(); + Map nodeShutdowns = nodesShutdownMap != null ? nodesShutdownMap : Collections.emptyMap(); long delayTimeoutNanos = Optional.ofNullable(lastAllocatedNodeId) .map(nodeShutdowns::get) .filter(shutdownMetadata -> SingleNodeShutdownMetadata.Type.RESTART.equals(shutdownMetadata.getType())) - .map(shutdownMetadata -> new TimeValue(10, TimeUnit.MINUTES).getNanos()) // GWB-> NOCOMMIT plumb in a setting here + .map(SingleNodeShutdownMetadata::getShardReallocationDelay) + .map(TimeValue::nanos) .orElse(INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING.get(indexSettings).nanos()); assert nanoTimeNow >= unassignedTimeNanos; return Math.max(0L, delayTimeoutNanos - (nanoTimeNow - unassignedTimeNanos)); @@ -438,7 +437,7 @@ public static long findNextDelayedAllocation(long currentNanoTime, ClusterState final long newComputedLeftDelayNanos = unassignedInfo.getRemainingDelay( currentNanoTime, indexSettings, - metadata + metadata.nodeShutdowns() ); if (newComputedLeftDelayNanos < nextDelayNanos) { nextDelayNanos = newComputedLeftDelayNanos; diff --git a/server/src/main/java/org/elasticsearch/cluster/routing/allocation/AllocationService.java b/server/src/main/java/org/elasticsearch/cluster/routing/allocation/AllocationService.java index a5a0b47b1d592..6050d9dacc8e2 100644 --- a/server/src/main/java/org/elasticsearch/cluster/routing/allocation/AllocationService.java +++ b/server/src/main/java/org/elasticsearch/cluster/routing/allocation/AllocationService.java @@ -306,7 +306,7 @@ private void removeDelayMarkers(RoutingAllocation allocation) { final long newComputedLeftDelayNanos = unassignedInfo.getRemainingDelay( allocation.getCurrentNanoTime(), metadata.getIndexSafe(shardRouting.index()).getSettings(), - metadata + metadata.nodeShutdowns() ); if (newComputedLeftDelayNanos == 0) { unassignedIterator.updateUnassigned( diff --git a/server/src/main/java/org/elasticsearch/gateway/ReplicaShardAllocator.java b/server/src/main/java/org/elasticsearch/gateway/ReplicaShardAllocator.java index 60d398316cd5a..91508cf5ea46d 100644 --- a/server/src/main/java/org/elasticsearch/gateway/ReplicaShardAllocator.java +++ b/server/src/main/java/org/elasticsearch/gateway/ReplicaShardAllocator.java @@ -216,7 +216,7 @@ public AllocateUnassignedDecision makeAllocationDecision(final ShardRouting unas long remainingDelayNanos = unassignedInfo.getRemainingDelay( System.nanoTime(), indexMetadata.getSettings(), - metadata + metadata.nodeShutdowns() ); remainingDelayMillis = TimeValue.timeValueNanos(remainingDelayNanos).millis(); } diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/UnassignedInfoTests.java b/server/src/test/java/org/elasticsearch/cluster/routing/UnassignedInfoTests.java index 2acca6d5a5fe9..d0ec241d3445f 100644 --- a/server/src/test/java/org/elasticsearch/cluster/routing/UnassignedInfoTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/routing/UnassignedInfoTests.java @@ -40,7 +40,6 @@ import java.util.HashMap; import java.util.Map; import java.util.Set; -import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; import java.util.stream.IntStream; @@ -311,7 +310,7 @@ public void testFailedShard() { * Verifies that delayed allocation calculation are correct when there are no registered node shutdowns. */ public void testRemainingDelayCalculationWithNoShutdowns() throws Exception { - checkRemainingDelayCalculation("bogusNodeId", TimeValue.timeValueNanos(10), Settings.EMPTY, null, TimeValue.timeValueNanos(10)); + checkRemainingDelayCalculation("bogusNodeId", TimeValue.timeValueNanos(10), null, TimeValue.timeValueNanos(10)); } /** @@ -319,7 +318,7 @@ public void testRemainingDelayCalculationWithNoShutdowns() throws Exception { * don't accidentally introduce an NPE in trying to read the custom metadata. */ public void testRemainingDelayCalculationWithNoShutdownsEverRegistered() throws Exception { - checkRemainingDelayCalculation("bogusNodeId", TimeValue.timeValueNanos(10), Settings.EMPTY, null, TimeValue.timeValueNanos(10)); + checkRemainingDelayCalculation("bogusNodeId", TimeValue.timeValueNanos(10), Collections.emptyMap(), TimeValue.timeValueNanos(10)); } /** @@ -339,7 +338,7 @@ public void testRemainingDelayCalculationsWithUnrelatedShutdowns() throws Except .build(); shutdowns.put(shutdown.getNodeId(), shutdown); } - checkRemainingDelayCalculation(lastNodeId, TimeValue.timeValueNanos(10), Settings.EMPTY, shutdowns, TimeValue.timeValueNanos(10)); + checkRemainingDelayCalculation(lastNodeId, TimeValue.timeValueNanos(10), shutdowns, TimeValue.timeValueNanos(10)); } /** @@ -356,21 +355,58 @@ public void testRemainingDelayCalculationWhenNodeIsShuttingDownForRemoval() thro .build(); shutdowns.put(shutdown.getNodeId(), shutdown); - checkRemainingDelayCalculation(lastNodeId, TimeValue.timeValueNanos(10), Settings.EMPTY, shutdowns, TimeValue.timeValueNanos(10)); + checkRemainingDelayCalculation(lastNodeId, TimeValue.timeValueNanos(10), shutdowns, TimeValue.timeValueNanos(10)); } /** - * Verifies that the delay calculation uses the delay value for nodes known to be restarting, because they are registered for a - * `RESTART`-type shutdown, rather than the default delay. + * Verifies that the delay calculation uses the configured delay value for nodes known to be restarting, because they are registered for + * a `RESTART`-type shutdown, rather than the default global delay. */ - public void testRemainingDelayCalculationWhenNodeIsKnownToBeRestarting() throws Exception { - // GWB-> To be written! + public void testRemainingDelayCalculationWhenNodeIsKnownToBeRestartingWithCustomDelay() throws Exception { + String lastNodeId = "bogusNodeId"; + Map shutdowns = new HashMap<>(); + SingleNodeShutdownMetadata shutdown = SingleNodeShutdownMetadata.builder() + .setNodeId(lastNodeId) + .setReason(this.getTestName()) + .setStartedAtMillis(randomNonNegativeLong()) + .setType(SingleNodeShutdownMetadata.Type.RESTART) + .setShardReallocationDelay(TimeValue.timeValueMinutes(1)) + .build(); + shutdowns.put(shutdown.getNodeId(), shutdown); + + // Use a different index-level delay so this test will fail if that one gets used instead of the one from the shutdown metadata + checkRemainingDelayCalculation(lastNodeId, TimeValue.timeValueNanos(10), shutdowns, TimeValue.timeValueMinutes(1)); + } + + /** + * Verifies that the delay calculation uses the default delay value for nodes known to be restarting, because they are registered for + * a `RESTART`-type shutdown, rather than the default global delay. + */ + public void testRemainingDelayCalculationWhenNodeIsKnownToBeRestartingWithDefaultDelay() throws Exception { + String lastNodeId = "bogusNodeId"; + Map shutdowns = new HashMap<>(); + + // Note that we do not explicitly configure the reallocation delay here. + SingleNodeShutdownMetadata shutdown = SingleNodeShutdownMetadata.builder() + .setNodeId(lastNodeId) + .setReason(this.getTestName()) + .setStartedAtMillis(randomNonNegativeLong()) + .setType(SingleNodeShutdownMetadata.Type.RESTART) + .build(); + shutdowns.put(shutdown.getNodeId(), shutdown); + + // Use a different index-level delay so this test will fail if that one gets used instead of the one from the shutdown metadata + checkRemainingDelayCalculation( + lastNodeId, + TimeValue.timeValueNanos(10), + shutdowns, + SingleNodeShutdownMetadata.DEFAULT_RESTART_SHARD_ALLOCATION_DELAY + ); } private void checkRemainingDelayCalculation( String lastNodeId, TimeValue indexLevelTimeoutSetting, - Settings clusterSettings, Map nodeShutdowns, TimeValue expectedTotalDelay ) throws Exception { @@ -391,24 +427,14 @@ private void checkRemainingDelayCalculation( final Settings indexSettings = Settings.builder() .put(UnassignedInfo.INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING.getKey(), indexLevelTimeoutSetting) .build(); - Metadata.Builder metadataBuilder = Metadata.builder(); - if (randomBoolean()) { - metadataBuilder.persistentSettings(clusterSettings); - } else { - metadataBuilder.transientSettings(clusterSettings); - } - if (nodeShutdowns != null) { - metadataBuilder.putCustom(NodesShutdownMetadata.TYPE, new NodesShutdownMetadata(nodeShutdowns)); - } - Metadata metadata = metadataBuilder.build(); - long delay = unassignedInfo.getRemainingDelay(baseTime, indexSettings, metadata); + long delay = unassignedInfo.getRemainingDelay(baseTime, indexSettings, nodeShutdowns); assertThat(delay, equalTo(totalDelayNanos)); - long delta1 = randomIntBetween(1, (int) (totalDelayNanos - 1)); - delay = unassignedInfo.getRemainingDelay(baseTime + delta1, indexSettings, metadata); + long delta1 = randomLongBetween(1, (totalDelayNanos - 1)); + delay = unassignedInfo.getRemainingDelay(baseTime + delta1, indexSettings, nodeShutdowns); assertThat(delay, equalTo(totalDelayNanos - delta1)); - delay = unassignedInfo.getRemainingDelay(baseTime + totalDelayNanos, indexSettings, metadata); + delay = unassignedInfo.getRemainingDelay(baseTime + totalDelayNanos, indexSettings, nodeShutdowns); assertThat(delay, equalTo(0L)); - delay = unassignedInfo.getRemainingDelay(baseTime + totalDelayNanos + randomIntBetween(1, 20), indexSettings, metadata); + delay = unassignedInfo.getRemainingDelay(baseTime + totalDelayNanos + randomIntBetween(1, 20), indexSettings, nodeShutdowns); assertThat(delay, equalTo(0L)); } From 4e9c299adcf3572dda7271bfc2b1752b3ef3a7cf Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Wed, 21 Jul 2021 17:09:59 -0600 Subject: [PATCH 08/47] spotless --- .../NodeShutdownDelayedAllocationIT.java | 22 +++++++++---------- 1 file changed, 11 insertions(+), 11 deletions(-) diff --git a/x-pack/plugin/shutdown/src/internalClusterTest/java/org/elasticsearch/xpack/shutdown/NodeShutdownDelayedAllocationIT.java b/x-pack/plugin/shutdown/src/internalClusterTest/java/org/elasticsearch/xpack/shutdown/NodeShutdownDelayedAllocationIT.java index 85fc9a972bdc6..a8b20cb2b7f15 100644 --- a/x-pack/plugin/shutdown/src/internalClusterTest/java/org/elasticsearch/xpack/shutdown/NodeShutdownDelayedAllocationIT.java +++ b/x-pack/plugin/shutdown/src/internalClusterTest/java/org/elasticsearch/xpack/shutdown/NodeShutdownDelayedAllocationIT.java @@ -37,10 +37,12 @@ protected Collection> nodePlugins() { public void testShardAllocationIsDelayedForRestartingNode() throws Exception { internalCluster().startNodes(3); - prepareCreate("test").setSettings(Settings.builder() - .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) - .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1) - .put(UnassignedInfo.INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING.getKey(), 0)).get(); // Disable "normal" delayed allocation + prepareCreate("test").setSettings( + Settings.builder() + .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1) + .put(UnassignedInfo.INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING.getKey(), 0) + ).get(); // Disable "normal" delayed allocation ensureGreen("test"); indexRandomData(); @@ -61,9 +63,7 @@ public void testShardAllocationIsDelayedForRestartingNode() throws Exception { internalCluster().stopRandomNode(InternalTestCluster.nameFilter(nodeToRestartName)); // Verify that the shard's allocation is delayed - assertBusy(() -> { - assertThat(client().admin().cluster().prepareHealth().get().getDelayedUnassignedShards(), equalTo(1)); - }); + assertBusy(() -> { assertThat(client().admin().cluster().prepareHealth().get().getDelayedUnassignedShards(), equalTo(1)); }); // Bring the node back internalCluster().startNode(nodeToRestartDataPathSettings); // this will use the same data location as the stopped node @@ -73,9 +73,9 @@ public void testShardAllocationIsDelayedForRestartingNode() throws Exception { } // GWB-> Next tests to write: - // - Timeout - // - Change the timeout setting - // - Delete the shutdown record + // - Timeout + // - Change the timeout setting + // - Delete the shutdown record private void indexRandomData() throws Exception { int numDocs = scaledRandomIntBetween(100, 1000); @@ -89,7 +89,7 @@ private void indexRandomData() throws Exception { private String findIdOfNodeWithShard() { ClusterState state = client().admin().cluster().prepareState().get().getState(); List startedShards = state.routingTable().shardsWithState(ShardRoutingState.STARTED); - Collections.shuffle(startedShards,random()); + Collections.shuffle(startedShards, random()); return startedShards.get(0).currentNodeId(); } From cd2a67a6e92a31dd550bcebe44677f52c7485892 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Wed, 21 Jul 2021 17:13:47 -0600 Subject: [PATCH 09/47] Imports --- .../elasticsearch/cluster/metadata/NodesShutdownMetadata.java | 2 +- .../java/org/elasticsearch/cluster/routing/UnassignedInfo.java | 1 - .../cluster/routing/allocation/AllocationService.java | 1 - .../org/elasticsearch/cluster/routing/UnassignedInfoTests.java | 2 +- 4 files changed, 2 insertions(+), 4 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/cluster/metadata/NodesShutdownMetadata.java b/server/src/main/java/org/elasticsearch/cluster/metadata/NodesShutdownMetadata.java index a52d27f56a325..59d2743a189be 100644 --- a/server/src/main/java/org/elasticsearch/cluster/metadata/NodesShutdownMetadata.java +++ b/server/src/main/java/org/elasticsearch/cluster/metadata/NodesShutdownMetadata.java @@ -14,10 +14,10 @@ import org.elasticsearch.cluster.Diff; import org.elasticsearch.cluster.DiffableUtils; import org.elasticsearch.cluster.NamedDiff; -import org.elasticsearch.common.xcontent.ParseField; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.xcontent.ConstructingObjectParser; +import org.elasticsearch.common.xcontent.ParseField; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentParser; diff --git a/server/src/main/java/org/elasticsearch/cluster/routing/UnassignedInfo.java b/server/src/main/java/org/elasticsearch/cluster/routing/UnassignedInfo.java index d00c3534ff051..19732f48b41a2 100644 --- a/server/src/main/java/org/elasticsearch/cluster/routing/UnassignedInfo.java +++ b/server/src/main/java/org/elasticsearch/cluster/routing/UnassignedInfo.java @@ -37,7 +37,6 @@ import java.util.Objects; import java.util.Optional; import java.util.Set; -import java.util.concurrent.TimeUnit; /** * Holds additional information as to why the shard is in unassigned state. diff --git a/server/src/main/java/org/elasticsearch/cluster/routing/allocation/AllocationService.java b/server/src/main/java/org/elasticsearch/cluster/routing/allocation/AllocationService.java index 6050d9dacc8e2..45b4e869a996a 100644 --- a/server/src/main/java/org/elasticsearch/cluster/routing/allocation/AllocationService.java +++ b/server/src/main/java/org/elasticsearch/cluster/routing/allocation/AllocationService.java @@ -19,7 +19,6 @@ import org.elasticsearch.cluster.metadata.AutoExpandReplicas; import org.elasticsearch.cluster.metadata.IndexMetadata; import org.elasticsearch.cluster.metadata.Metadata; -import org.elasticsearch.cluster.metadata.NodesShutdownMetadata; import org.elasticsearch.cluster.metadata.SingleNodeShutdownMetadata; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.routing.RerouteService; diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/UnassignedInfoTests.java b/server/src/test/java/org/elasticsearch/cluster/routing/UnassignedInfoTests.java index d0ec241d3445f..fed9c4916b24a 100644 --- a/server/src/test/java/org/elasticsearch/cluster/routing/UnassignedInfoTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/routing/UnassignedInfoTests.java @@ -10,13 +10,13 @@ import com.carrotsearch.hppc.IntHashSet; import com.carrotsearch.randomizedtesting.generators.RandomPicks; + import org.elasticsearch.Version; import org.elasticsearch.cluster.ClusterName; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ESAllocationTestCase; import org.elasticsearch.cluster.metadata.IndexMetadata; import org.elasticsearch.cluster.metadata.Metadata; -import org.elasticsearch.cluster.metadata.NodesShutdownMetadata; import org.elasticsearch.cluster.metadata.SingleNodeShutdownMetadata; import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.cluster.routing.RecoverySource.SnapshotRecoverySource; From 1b2df0af475d050feaf03e0e336de866f8e7e0bd Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Wed, 21 Jul 2021 17:18:49 -0600 Subject: [PATCH 10/47] EMPTY_MAP -> emptyMap() --- .../main/java/org/elasticsearch/cluster/metadata/Metadata.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/cluster/metadata/Metadata.java b/server/src/main/java/org/elasticsearch/cluster/metadata/Metadata.java index 676d798926e07..e2ac284028a55 100644 --- a/server/src/main/java/org/elasticsearch/cluster/metadata/Metadata.java +++ b/server/src/main/java/org/elasticsearch/cluster/metadata/Metadata.java @@ -709,11 +709,10 @@ public Map dataStreamAliases() { .orElse(Collections.emptyMap()); } - @SuppressWarnings("unchecked") public Map nodeShutdowns() { return Optional.ofNullable((NodesShutdownMetadata) this.custom(NodesShutdownMetadata.TYPE)) .map(NodesShutdownMetadata::getAllNodeMetadataMap) - .orElse(Collections.EMPTY_MAP); + .orElse(Collections.emptyMap()); } public ImmutableOpenMap customs() { From acb871ac1252444c9f8876ac89ce05ae321917ff Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Thu, 22 Jul 2021 15:59:15 -0600 Subject: [PATCH 11/47] More integration tests --- .../NodeShutdownDelayedAllocationIT.java | 155 +++++++++++++++++- .../xpack/shutdown/NodeShutdownPluginsIT.java | 2 +- .../xpack/shutdown/NodeShutdownTasksIT.java | 2 +- .../xpack/shutdown/PutShutdownNodeAction.java | 22 ++- .../TransportPutShutdownNodeAction.java | 1 + 5 files changed, 169 insertions(+), 13 deletions(-) diff --git a/x-pack/plugin/shutdown/src/internalClusterTest/java/org/elasticsearch/xpack/shutdown/NodeShutdownDelayedAllocationIT.java b/x-pack/plugin/shutdown/src/internalClusterTest/java/org/elasticsearch/xpack/shutdown/NodeShutdownDelayedAllocationIT.java index a8b20cb2b7f15..256a8e73aa41f 100644 --- a/x-pack/plugin/shutdown/src/internalClusterTest/java/org/elasticsearch/xpack/shutdown/NodeShutdownDelayedAllocationIT.java +++ b/x-pack/plugin/shutdown/src/internalClusterTest/java/org/elasticsearch/xpack/shutdown/NodeShutdownDelayedAllocationIT.java @@ -16,6 +16,7 @@ import org.elasticsearch.cluster.routing.ShardRoutingState; import org.elasticsearch.cluster.routing.UnassignedInfo; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.core.TimeValue; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.test.ESIntegTestCase; import org.elasticsearch.test.InternalTestCluster; @@ -24,6 +25,7 @@ import java.util.Collection; import java.util.Collections; import java.util.List; +import java.util.concurrent.TimeUnit; import static org.hamcrest.Matchers.equalTo; @@ -41,8 +43,8 @@ public void testShardAllocationIsDelayedForRestartingNode() throws Exception { Settings.builder() .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1) - .put(UnassignedInfo.INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING.getKey(), 0) - ).get(); // Disable "normal" delayed allocation + .put(UnassignedInfo.INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING.getKey(), 0) // Disable "normal" delayed allocation + ).get(); ensureGreen("test"); indexRandomData(); @@ -54,7 +56,8 @@ public void testShardAllocationIsDelayedForRestartingNode() throws Exception { PutShutdownNodeAction.Request putShutdownRequest = new PutShutdownNodeAction.Request( nodeToRestartId, SingleNodeShutdownMetadata.Type.RESTART, - this.getTestName() + this.getTestName(), + null // Make sure it works with the default - we'll check this override in other tests ); AcknowledgedResponse putShutdownResponse = client().execute(PutShutdownNodeAction.INSTANCE, putShutdownRequest).get(); assertTrue(putShutdownResponse.isAcknowledged()); @@ -72,10 +75,148 @@ public void testShardAllocationIsDelayedForRestartingNode() throws Exception { ensureGreen("test"); } - // GWB-> Next tests to write: - // - Timeout - // - Change the timeout setting - // - Delete the shutdown record + public void testShardAllocationWillProceedAfterTimeout() throws Exception { + internalCluster().startNodes(3); + prepareCreate("test").setSettings( + Settings.builder() + .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1) + .put(UnassignedInfo.INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING.getKey(), 0) // Disable "normal" delayed allocation + ).get(); + ensureGreen("test"); + indexRandomData(); + + final String nodeToRestartId = findIdOfNodeWithShard(); + final String nodeToRestartName = findNodeNameFromId(nodeToRestartId); + + // Mark the node for shutdown + PutShutdownNodeAction.Request putShutdownRequest = new PutShutdownNodeAction.Request( + nodeToRestartId, + SingleNodeShutdownMetadata.Type.RESTART, + this.getTestName(), + TimeValue.timeValueSeconds(3) + ); + AcknowledgedResponse putShutdownResponse = client().execute(PutShutdownNodeAction.INSTANCE, putShutdownRequest).get(); + assertTrue(putShutdownResponse.isAcknowledged()); + + // Actually stop the node + internalCluster().stopRandomNode(InternalTestCluster.nameFilter(nodeToRestartName)); + + // Verify that the shard's allocation is delayed - but with a shorter wait than the reallocation timeout + assertBusy( + () -> { assertThat(client().admin().cluster().prepareHealth().get().getDelayedUnassignedShards(), equalTo(1)); }, + 2, + TimeUnit.SECONDS + ); + + // And the index should turn green again well within the 30-second timeout + ensureGreen(TimeValue.timeValueSeconds(30), "test"); + } + + public void testShardAllocationTimeoutCanBeChanged() throws Exception { + internalCluster().startNodes(3); + prepareCreate("test").setSettings( + Settings.builder() + .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1) + .put(UnassignedInfo.INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING.getKey(), 0) // Disable "normal" delayed allocation + ).get(); + ensureGreen("test"); + indexRandomData(); + + final String nodeToRestartId = findIdOfNodeWithShard(); + final String nodeToRestartName = findNodeNameFromId(nodeToRestartId); + + { + // Mark the node for shutdown with a delay that we'll never reach in the test + PutShutdownNodeAction.Request putShutdownRequest = new PutShutdownNodeAction.Request( + nodeToRestartId, + SingleNodeShutdownMetadata.Type.RESTART, + this.getTestName(), + TimeValue.timeValueHours(3) + ); + AcknowledgedResponse putShutdownResponse = client().execute(PutShutdownNodeAction.INSTANCE, putShutdownRequest).get(); + assertTrue(putShutdownResponse.isAcknowledged()); + } + + // Actually stop the node + internalCluster().stopRandomNode(InternalTestCluster.nameFilter(nodeToRestartName)); + + // Verify that the shard's allocation is delayed - but with a shorter wait than the reallocation timeout + assertBusy( + () -> { assertThat(client().admin().cluster().prepareHealth().get().getDelayedUnassignedShards(), equalTo(1)); }, + 2, + TimeUnit.SECONDS + ); + + { + // Update the timeout on the shutdown request to something shorter, but that still shouldn't be quite up yet + PutShutdownNodeAction.Request putShutdownRequest = new PutShutdownNodeAction.Request( + nodeToRestartId, + SingleNodeShutdownMetadata.Type.RESTART, + this.getTestName(), + TimeValue.timeValueSeconds(10) + ); + AcknowledgedResponse putShutdownResponse = client().execute(PutShutdownNodeAction.INSTANCE, putShutdownRequest).get(); + assertTrue(putShutdownResponse.isAcknowledged()); + } + + // Verify that the shard's allocation is still delayed + assertBusy( + () -> { assertThat(client().admin().cluster().prepareHealth().get().getDelayedUnassignedShards(), equalTo(1)); }, + 2, + TimeUnit.SECONDS + ); + + // And the index should turn green again well within the 30-second timeout + ensureGreen(TimeValue.timeValueSeconds(30), "test"); + } + + public void testShardAllocationStartsImmediatelyIfShutdownDeleted() throws Exception { + internalCluster().startNodes(3); + prepareCreate("test").setSettings( + Settings.builder() + .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1) + .put(UnassignedInfo.INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING.getKey(), 0) // Disable "normal" delayed allocation + ).get(); + ensureGreen("test"); + indexRandomData(); + + final String nodeToRestartId = findIdOfNodeWithShard(); + final String nodeToRestartName = findNodeNameFromId(nodeToRestartId); + + { + // Mark the node for shutdown with a delay that we'll never reach in the test + PutShutdownNodeAction.Request putShutdownRequest = new PutShutdownNodeAction.Request( + nodeToRestartId, + SingleNodeShutdownMetadata.Type.RESTART, + this.getTestName(), + TimeValue.timeValueHours(3) + ); + AcknowledgedResponse putShutdownResponse = client().execute(PutShutdownNodeAction.INSTANCE, putShutdownRequest).get(); + assertTrue(putShutdownResponse.isAcknowledged()); + } + + // Actually stop the node + internalCluster().stopRandomNode(InternalTestCluster.nameFilter(nodeToRestartName)); + + // Verify that the shard's allocation is delayed - but with a shorter wait than the reallocation timeout + assertBusy( + () -> { assertThat(client().admin().cluster().prepareHealth().get().getDelayedUnassignedShards(), equalTo(1)); }, + 2, + TimeUnit.SECONDS + ); + + { + DeleteShutdownNodeAction.Request deleteShutdownRequest = new DeleteShutdownNodeAction.Request(nodeToRestartId); + AcknowledgedResponse putShutdownResponse = client().execute(DeleteShutdownNodeAction.INSTANCE, deleteShutdownRequest).get(); + assertTrue(putShutdownResponse.isAcknowledged()); + } + + // And the index should turn green again well within the 30-second timeout + ensureGreen(TimeValue.timeValueSeconds(30), "test"); + } private void indexRandomData() throws Exception { int numDocs = scaledRandomIntBetween(100, 1000); diff --git a/x-pack/plugin/shutdown/src/internalClusterTest/java/org/elasticsearch/xpack/shutdown/NodeShutdownPluginsIT.java b/x-pack/plugin/shutdown/src/internalClusterTest/java/org/elasticsearch/xpack/shutdown/NodeShutdownPluginsIT.java index 29325c3626277..f216dcd7562ac 100644 --- a/x-pack/plugin/shutdown/src/internalClusterTest/java/org/elasticsearch/xpack/shutdown/NodeShutdownPluginsIT.java +++ b/x-pack/plugin/shutdown/src/internalClusterTest/java/org/elasticsearch/xpack/shutdown/NodeShutdownPluginsIT.java @@ -80,7 +80,7 @@ public void testShutdownAwarePlugin() throws Exception { // Mark the node as shutting down client().execute( PutShutdownNodeAction.INSTANCE, - new PutShutdownNodeAction.Request(shutdownNode, SingleNodeShutdownMetadata.Type.REMOVE, "removal for testing") + new PutShutdownNodeAction.Request(shutdownNode, SingleNodeShutdownMetadata.Type.REMOVE, "removal for testing", null) ).get(); GetShutdownStatusAction.Response getResp = client().execute( diff --git a/x-pack/plugin/shutdown/src/internalClusterTest/java/org/elasticsearch/xpack/shutdown/NodeShutdownTasksIT.java b/x-pack/plugin/shutdown/src/internalClusterTest/java/org/elasticsearch/xpack/shutdown/NodeShutdownTasksIT.java index a55c3dab3ad38..dd50014d6a84d 100644 --- a/x-pack/plugin/shutdown/src/internalClusterTest/java/org/elasticsearch/xpack/shutdown/NodeShutdownTasksIT.java +++ b/x-pack/plugin/shutdown/src/internalClusterTest/java/org/elasticsearch/xpack/shutdown/NodeShutdownTasksIT.java @@ -113,7 +113,7 @@ public void testTasksAreNotAssignedToShuttingDownNode() throws Exception { // Mark the node as shutting down client().execute( PutShutdownNodeAction.INSTANCE, - new PutShutdownNodeAction.Request(shutdownNode, SingleNodeShutdownMetadata.Type.REMOVE, "removal for testing") + new PutShutdownNodeAction.Request(shutdownNode, SingleNodeShutdownMetadata.Type.REMOVE, "removal for testing", null) ).get(); // Tell the persistent task executor it can start allocating the task diff --git a/x-pack/plugin/shutdown/src/main/java/org/elasticsearch/xpack/shutdown/PutShutdownNodeAction.java b/x-pack/plugin/shutdown/src/main/java/org/elasticsearch/xpack/shutdown/PutShutdownNodeAction.java index c3f7081185060..680f02fccfc0b 100644 --- a/x-pack/plugin/shutdown/src/main/java/org/elasticsearch/xpack/shutdown/PutShutdownNodeAction.java +++ b/x-pack/plugin/shutdown/src/main/java/org/elasticsearch/xpack/shutdown/PutShutdownNodeAction.java @@ -18,11 +18,14 @@ import org.elasticsearch.common.xcontent.ConstructingObjectParser; import org.elasticsearch.common.xcontent.ParseField; import org.elasticsearch.common.xcontent.XContentParser; +import org.elasticsearch.core.Nullable; +import org.elasticsearch.core.TimeValue; import java.io.IOException; public class PutShutdownNodeAction extends ActionType { + public static final PutShutdownNodeAction INSTANCE = new PutShutdownNodeAction(); public static final String NAME = "cluster:admin/shutdown/create"; @@ -35,42 +38,49 @@ public static class Request extends AcknowledgedRequest { private final String nodeId; private final SingleNodeShutdownMetadata.Type type; private final String reason; + @Nullable + private final TimeValue shardReallocationDelay; private static final ParseField TYPE_FIELD = new ParseField("type"); - public static final ParseField REASON_FIELD = new ParseField("reason"); + private static final ParseField REASON_FIELD = new ParseField("reason"); + private static final ParseField REALLOCATION_DELAY_FIELD = new ParseField("shard_reallocation_delay"); private static final ConstructingObjectParser PARSER = new ConstructingObjectParser<>( "put_node_shutdown_request", false, - (a, nodeId) -> new Request(nodeId, SingleNodeShutdownMetadata.Type.parse((String) a[0]), (String) a[1]) + (a, nodeId) -> new Request(nodeId, SingleNodeShutdownMetadata.Type.parse((String) a[0]), (String) a[1], (TimeValue) a[2]) ); static { PARSER.declareString(ConstructingObjectParser.constructorArg(), TYPE_FIELD); PARSER.declareString(ConstructingObjectParser.constructorArg(), REASON_FIELD); + PARSER.declareString(ConstructingObjectParser.optionalConstructorArg(), REALLOCATION_DELAY_FIELD); } public static Request parseRequest(String nodeId, XContentParser parser) { return PARSER.apply(parser, nodeId); } - public Request(String nodeId, SingleNodeShutdownMetadata.Type type, String reason) { + public Request(String nodeId, SingleNodeShutdownMetadata.Type type, String reason, @Nullable TimeValue shardReallocationDelay) { this.nodeId = nodeId; this.type = type; this.reason = reason; + this.shardReallocationDelay = shardReallocationDelay; } public Request(StreamInput in) throws IOException { this.nodeId = in.readString(); this.type = in.readEnum(SingleNodeShutdownMetadata.Type.class); this.reason = in.readString(); + this.shardReallocationDelay = in.readOptionalTimeValue(); } @Override public void writeTo(StreamOutput out) throws IOException { - out.writeString(this.nodeId); + out.writeString(nodeId); out.writeEnum(type); out.writeString(reason); + out.writeOptionalTimeValue(shardReallocationDelay); } public String getNodeId() { @@ -85,6 +95,10 @@ public String getReason() { return reason; } + public TimeValue getShardReallocationDelay() { + return shardReallocationDelay; + } + @Override public ActionRequestValidationException validate() { ActionRequestValidationException arve = new ActionRequestValidationException(); diff --git a/x-pack/plugin/shutdown/src/main/java/org/elasticsearch/xpack/shutdown/TransportPutShutdownNodeAction.java b/x-pack/plugin/shutdown/src/main/java/org/elasticsearch/xpack/shutdown/TransportPutShutdownNodeAction.java index 19c06341b9003..5b55399f8dc14 100644 --- a/x-pack/plugin/shutdown/src/main/java/org/elasticsearch/xpack/shutdown/TransportPutShutdownNodeAction.java +++ b/x-pack/plugin/shutdown/src/main/java/org/elasticsearch/xpack/shutdown/TransportPutShutdownNodeAction.java @@ -87,6 +87,7 @@ public ClusterState execute(ClusterState currentState) { .setType(request.getType()) .setReason(request.getReason()) .setStartedAtMillis(System.currentTimeMillis()) + .setShardReallocationDelay(request.getShardReallocationDelay()) .build(); return ClusterState.builder(currentState) From edf9afa8656d85c03d1ba999c16e48a9adc6df89 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Thu, 22 Jul 2021 16:28:51 -0600 Subject: [PATCH 12/47] spotless --- .../org/elasticsearch/xpack/shutdown/PutShutdownNodeAction.java | 1 - 1 file changed, 1 deletion(-) diff --git a/x-pack/plugin/shutdown/src/main/java/org/elasticsearch/xpack/shutdown/PutShutdownNodeAction.java b/x-pack/plugin/shutdown/src/main/java/org/elasticsearch/xpack/shutdown/PutShutdownNodeAction.java index 680f02fccfc0b..b61bbad36ab09 100644 --- a/x-pack/plugin/shutdown/src/main/java/org/elasticsearch/xpack/shutdown/PutShutdownNodeAction.java +++ b/x-pack/plugin/shutdown/src/main/java/org/elasticsearch/xpack/shutdown/PutShutdownNodeAction.java @@ -25,7 +25,6 @@ public class PutShutdownNodeAction extends ActionType { - public static final PutShutdownNodeAction INSTANCE = new PutShutdownNodeAction(); public static final String NAME = "cluster:admin/shutdown/create"; From 28fea4417edab1f3bd5370079f0e94e989a14300 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Thu, 22 Jul 2021 16:37:33 -0600 Subject: [PATCH 13/47] Fix compilation in ML tests --- .../ml/integration/MlNodeShutdownIT.java | 23 +++++++++++++++---- 1 file changed, 18 insertions(+), 5 deletions(-) diff --git a/x-pack/plugin/ml/src/internalClusterTest/java/org/elasticsearch/xpack/ml/integration/MlNodeShutdownIT.java b/x-pack/plugin/ml/src/internalClusterTest/java/org/elasticsearch/xpack/ml/integration/MlNodeShutdownIT.java index ab0c7cee70230..e6d46254f2ec0 100644 --- a/x-pack/plugin/ml/src/internalClusterTest/java/org/elasticsearch/xpack/ml/integration/MlNodeShutdownIT.java +++ b/x-pack/plugin/ml/src/internalClusterTest/java/org/elasticsearch/xpack/ml/integration/MlNodeShutdownIT.java @@ -78,9 +78,15 @@ public void testJobsVacateShuttingDownNode() throws Exception { }); // Call the shutdown API for the chosen node. - client().execute(PutShutdownNodeAction.INSTANCE, - new PutShutdownNodeAction.Request(nodeIdToShutdown.get(), randomFrom(SingleNodeShutdownMetadata.Type.values()), "just testing")) - .actionGet(); + client().execute( + PutShutdownNodeAction.INSTANCE, + new PutShutdownNodeAction.Request( + nodeIdToShutdown.get(), + randomFrom(SingleNodeShutdownMetadata.Type.values()), + "just testing", + null + ) + ).actionGet(); // Wait for the desired end state of all 6 jobs running on nodes that are not shutting down. assertBusy(() -> { @@ -145,8 +151,15 @@ public void testCloseJobVacatingShuttingDownNode() throws Exception { }); // Call the shutdown API for the chosen node. - client().execute(PutShutdownNodeAction.INSTANCE, - new PutShutdownNodeAction.Request(nodeIdToShutdown.get(), randomFrom(SingleNodeShutdownMetadata.Type.values()), "just testing")) + client().execute( + PutShutdownNodeAction.INSTANCE, + new PutShutdownNodeAction.Request( + nodeIdToShutdown.get(), + randomFrom(SingleNodeShutdownMetadata.Type.values()), + "just testing", + null + ) + ) .actionGet(); if (randomBoolean()) { From 34e983e83a7b3bd435651406fe15987435a4c792 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Mon, 26 Jul 2021 14:28:05 -0600 Subject: [PATCH 14/47] Change version constant name & add comment per review --- .../elasticsearch/cluster/routing/UnassignedInfo.java | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/cluster/routing/UnassignedInfo.java b/server/src/main/java/org/elasticsearch/cluster/routing/UnassignedInfo.java index 19732f48b41a2..6dfe04525a64d 100644 --- a/server/src/main/java/org/elasticsearch/cluster/routing/UnassignedInfo.java +++ b/server/src/main/java/org/elasticsearch/cluster/routing/UnassignedInfo.java @@ -43,7 +43,11 @@ */ public final class UnassignedInfo implements ToXContentFragment, Writeable { - private static final Version LAST_ALLOCATED_NODE_VERSION = Version.V_8_0_0; + /** + * The version that the {@code lastAllocatedNode} field was added in. Used to adapt streaming of this class as appropriate for the + * version of the node sending/receiving it. Should be removed once wire compatibility with this version is no longer necessary. + */ + private static final Version VERSION_LAST_ALLOCATED_NODE_ADDED = Version.V_8_0_0; public static final DateFormatter DATE_TIME_FORMATTER = DateFormatter.forPattern("date_optional_time").withZone(ZoneOffset.UTC); @@ -268,7 +272,7 @@ public UnassignedInfo(StreamInput in) throws IOException { this.failedAllocations = in.readVInt(); this.lastAllocationStatus = AllocationStatus.readFrom(in); this.failedNodeIds = Collections.unmodifiableSet(in.readSet(StreamInput::readString)); - if (in.getVersion().onOrAfter(LAST_ALLOCATED_NODE_VERSION)) { + if (in.getVersion().onOrAfter(VERSION_LAST_ALLOCATED_NODE_ADDED)) { this.lastAllocatedNodeId = in.readOptionalString(); } else { this.lastAllocatedNodeId = null; @@ -285,7 +289,7 @@ public void writeTo(StreamOutput out) throws IOException { out.writeVInt(failedAllocations); lastAllocationStatus.writeTo(out); out.writeCollection(failedNodeIds, StreamOutput::writeString); - if (out.getVersion().onOrAfter(LAST_ALLOCATED_NODE_VERSION)) { + if (out.getVersion().onOrAfter(VERSION_LAST_ALLOCATED_NODE_ADDED)) { out.writeOptionalString(lastAllocatedNodeId); } } From a933857882d6d9451e05e9c7d54aee5308c434f6 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Mon, 26 Jul 2021 14:51:58 -0600 Subject: [PATCH 15/47] Remove unnecessary null check per review --- .../java/org/elasticsearch/cluster/routing/UnassignedInfo.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/cluster/routing/UnassignedInfo.java b/server/src/main/java/org/elasticsearch/cluster/routing/UnassignedInfo.java index 6dfe04525a64d..96b98e0d7f356 100644 --- a/server/src/main/java/org/elasticsearch/cluster/routing/UnassignedInfo.java +++ b/server/src/main/java/org/elasticsearch/cluster/routing/UnassignedInfo.java @@ -399,9 +399,8 @@ public long getRemainingDelay( final Settings indexSettings, final Map nodesShutdownMap ) { - Map nodeShutdowns = nodesShutdownMap != null ? nodesShutdownMap : Collections.emptyMap(); long delayTimeoutNanos = Optional.ofNullable(lastAllocatedNodeId) - .map(nodeShutdowns::get) + .map(nodesShutdownMap::get) .filter(shutdownMetadata -> SingleNodeShutdownMetadata.Type.RESTART.equals(shutdownMetadata.getType())) .map(SingleNodeShutdownMetadata::getShardReallocationDelay) .map(TimeValue::nanos) From 219045a6c3169a333c91ee7c94a9c20c3153382d Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Mon, 26 Jul 2021 14:52:36 -0600 Subject: [PATCH 16/47] Remove potentially-flaky assert per review. --- .../xpack/shutdown/NodeShutdownDelayedAllocationIT.java | 7 ------- 1 file changed, 7 deletions(-) diff --git a/x-pack/plugin/shutdown/src/internalClusterTest/java/org/elasticsearch/xpack/shutdown/NodeShutdownDelayedAllocationIT.java b/x-pack/plugin/shutdown/src/internalClusterTest/java/org/elasticsearch/xpack/shutdown/NodeShutdownDelayedAllocationIT.java index 256a8e73aa41f..ab06573e3c33a 100644 --- a/x-pack/plugin/shutdown/src/internalClusterTest/java/org/elasticsearch/xpack/shutdown/NodeShutdownDelayedAllocationIT.java +++ b/x-pack/plugin/shutdown/src/internalClusterTest/java/org/elasticsearch/xpack/shutdown/NodeShutdownDelayedAllocationIT.java @@ -161,13 +161,6 @@ public void testShardAllocationTimeoutCanBeChanged() throws Exception { assertTrue(putShutdownResponse.isAcknowledged()); } - // Verify that the shard's allocation is still delayed - assertBusy( - () -> { assertThat(client().admin().cluster().prepareHealth().get().getDelayedUnassignedShards(), equalTo(1)); }, - 2, - TimeUnit.SECONDS - ); - // And the index should turn green again well within the 30-second timeout ensureGreen(TimeValue.timeValueSeconds(30), "test"); } From 1159b8633fa038be59847f6fac99c6d59a0126a9 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Mon, 26 Jul 2021 14:53:14 -0600 Subject: [PATCH 17/47] Remove timeout override on assert per review --- .../xpack/shutdown/NodeShutdownDelayedAllocationIT.java | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/x-pack/plugin/shutdown/src/internalClusterTest/java/org/elasticsearch/xpack/shutdown/NodeShutdownDelayedAllocationIT.java b/x-pack/plugin/shutdown/src/internalClusterTest/java/org/elasticsearch/xpack/shutdown/NodeShutdownDelayedAllocationIT.java index ab06573e3c33a..02d78d564f9f8 100644 --- a/x-pack/plugin/shutdown/src/internalClusterTest/java/org/elasticsearch/xpack/shutdown/NodeShutdownDelayedAllocationIT.java +++ b/x-pack/plugin/shutdown/src/internalClusterTest/java/org/elasticsearch/xpack/shutdown/NodeShutdownDelayedAllocationIT.java @@ -195,11 +195,7 @@ public void testShardAllocationStartsImmediatelyIfShutdownDeleted() throws Excep internalCluster().stopRandomNode(InternalTestCluster.nameFilter(nodeToRestartName)); // Verify that the shard's allocation is delayed - but with a shorter wait than the reallocation timeout - assertBusy( - () -> { assertThat(client().admin().cluster().prepareHealth().get().getDelayedUnassignedShards(), equalTo(1)); }, - 2, - TimeUnit.SECONDS - ); + assertBusy(() -> { assertThat(client().admin().cluster().prepareHealth().get().getDelayedUnassignedShards(), equalTo(1)); }); { DeleteShutdownNodeAction.Request deleteShutdownRequest = new DeleteShutdownNodeAction.Request(nodeToRestartId); From bcab443a8968b02493f3c933d10da380e00adf63 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Mon, 26 Jul 2021 15:24:17 -0600 Subject: [PATCH 18/47] Move default reallocation delay to getting per review --- .../metadata/SingleNodeShutdownMetadata.java | 18 ++++++++++-------- 1 file changed, 10 insertions(+), 8 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/cluster/metadata/SingleNodeShutdownMetadata.java b/server/src/main/java/org/elasticsearch/cluster/metadata/SingleNodeShutdownMetadata.java index 6511054785dd5..97625d50c3fb4 100644 --- a/server/src/main/java/org/elasticsearch/cluster/metadata/SingleNodeShutdownMetadata.java +++ b/server/src/main/java/org/elasticsearch/cluster/metadata/SingleNodeShutdownMetadata.java @@ -10,11 +10,11 @@ import org.elasticsearch.cluster.AbstractDiffable; import org.elasticsearch.cluster.Diffable; -import org.elasticsearch.common.xcontent.ObjectParser; -import org.elasticsearch.common.xcontent.ParseField; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.xcontent.ConstructingObjectParser; +import org.elasticsearch.common.xcontent.ObjectParser; +import org.elasticsearch.common.xcontent.ParseField; import org.elasticsearch.common.xcontent.ToXContentObject; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentParser; @@ -140,7 +140,12 @@ public long getStartedAtMillis() { */ @Nullable public TimeValue getShardReallocationDelay() { - return shardReallocationDelay; + if (shardReallocationDelay != null) { + return shardReallocationDelay; + } else if (Type.RESTART.equals(type)) { + return DEFAULT_RESTART_SHARD_ALLOCATION_DELAY; + } + return null; } @Override @@ -261,16 +266,13 @@ public SingleNodeShutdownMetadata build() { if (startedAtMillis == -1) { throw new IllegalArgumentException("start timestamp must be set"); } - TimeValue delayOrDefault = shardReallocationDelay; - if (Type.RESTART.equals(type) && delayOrDefault == null) { - delayOrDefault = DEFAULT_RESTART_SHARD_ALLOCATION_DELAY; - } + return new SingleNodeShutdownMetadata( nodeId, type, reason, startedAtMillis, - delayOrDefault + shardReallocationDelay ); } } From 47324db56508bcfc28a068602188f01a449cd7d8 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Mon, 26 Jul 2021 16:03:43 -0600 Subject: [PATCH 19/47] Remove unnecessary unit test --- .../elasticsearch/cluster/routing/UnassignedInfoTests.java | 7 ------- 1 file changed, 7 deletions(-) diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/UnassignedInfoTests.java b/server/src/test/java/org/elasticsearch/cluster/routing/UnassignedInfoTests.java index fed9c4916b24a..7f519a420a19e 100644 --- a/server/src/test/java/org/elasticsearch/cluster/routing/UnassignedInfoTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/routing/UnassignedInfoTests.java @@ -306,13 +306,6 @@ public void testFailedShard() { greaterThan(0L)); } - /** - * Verifies that delayed allocation calculation are correct when there are no registered node shutdowns. - */ - public void testRemainingDelayCalculationWithNoShutdowns() throws Exception { - checkRemainingDelayCalculation("bogusNodeId", TimeValue.timeValueNanos(10), null, TimeValue.timeValueNanos(10)); - } - /** * Verifies that delayed allocation calculation are correct when there have never been any registered node shutdowns. This ensures we * don't accidentally introduce an NPE in trying to read the custom metadata. From 523977eafa034a980a64f7dad8808abbba25d200 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Mon, 26 Jul 2021 16:04:37 -0600 Subject: [PATCH 20/47] Rename + change incorrect comment on unit test --- .../elasticsearch/cluster/routing/UnassignedInfoTests.java | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/UnassignedInfoTests.java b/server/src/test/java/org/elasticsearch/cluster/routing/UnassignedInfoTests.java index 7f519a420a19e..bdbcac7a74191 100644 --- a/server/src/test/java/org/elasticsearch/cluster/routing/UnassignedInfoTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/routing/UnassignedInfoTests.java @@ -307,10 +307,9 @@ public void testFailedShard() { } /** - * Verifies that delayed allocation calculation are correct when there have never been any registered node shutdowns. This ensures we - * don't accidentally introduce an NPE in trying to read the custom metadata. + * Verifies that delayed allocation calculation are correct when there are no registered node shutdowns. */ - public void testRemainingDelayCalculationWithNoShutdownsEverRegistered() throws Exception { + public void testRemainingDelayCalculationWithNoShutdowns() throws Exception { checkRemainingDelayCalculation("bogusNodeId", TimeValue.timeValueNanos(10), Collections.emptyMap(), TimeValue.timeValueNanos(10)); } From 0630dd6b88bde438a6ba7eadc4e14885c9cce8a4 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Mon, 2 Aug 2021 15:16:54 -0600 Subject: [PATCH 21/47] Use raw allocation delay value in equals/hashcode --- .../cluster/metadata/SingleNodeShutdownMetadata.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/cluster/metadata/SingleNodeShutdownMetadata.java b/server/src/main/java/org/elasticsearch/cluster/metadata/SingleNodeShutdownMetadata.java index 97625d50c3fb4..bb2e8843a7a19 100644 --- a/server/src/main/java/org/elasticsearch/cluster/metadata/SingleNodeShutdownMetadata.java +++ b/server/src/main/java/org/elasticsearch/cluster/metadata/SingleNodeShutdownMetadata.java @@ -183,7 +183,7 @@ public boolean equals(Object o) { && getNodeId().equals(that.getNodeId()) && getType() == that.getType() && getReason().equals(that.getReason()) - && Objects.equals(getShardReallocationDelay(), that.getShardReallocationDelay()); + && Objects.equals(shardReallocationDelay, that.shardReallocationDelay); } @Override @@ -193,7 +193,7 @@ public int hashCode() { getType(), getReason(), getStartedAtMillis(), - getShardReallocationDelay() + shardReallocationDelay ); } From 4906783690e9993d4ace578cfdec38e2c884a93a Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Mon, 2 Aug 2021 15:35:27 -0600 Subject: [PATCH 22/47] Adjust `delayed` calculation per review to... 1. Not ignore the default timeout for non-RESTART shutdowns 2. Ensure delayed=false if both timeouts are 0 --- .../cluster/routing/allocation/AllocationService.java | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/cluster/routing/allocation/AllocationService.java b/server/src/main/java/org/elasticsearch/cluster/routing/allocation/AllocationService.java index 45b4e869a996a..e2db4f85829d7 100644 --- a/server/src/main/java/org/elasticsearch/cluster/routing/allocation/AllocationService.java +++ b/server/src/main/java/org/elasticsearch/cluster/routing/allocation/AllocationService.java @@ -20,6 +20,7 @@ import org.elasticsearch.cluster.metadata.IndexMetadata; import org.elasticsearch.cluster.metadata.Metadata; import org.elasticsearch.cluster.metadata.SingleNodeShutdownMetadata; +import org.elasticsearch.cluster.metadata.SingleNodeShutdownMetadata.Type; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.routing.RerouteService; import org.elasticsearch.cluster.routing.RoutingNode; @@ -499,11 +500,11 @@ private void disassociateDeadNodes(RoutingAllocation allocation) { // now, go over all the shards routing on the node, and fail them for (ShardRouting shardRouting : node.copyShards()) { final IndexMetadata indexMetadata = allocation.metadata().getIndexSafe(shardRouting.index()); - boolean delayed = Optional.ofNullable(nodesShutdownMetadata.get(node.nodeId())) - // If we know this node is restarting, then the allocation should be delayed - .map(shutdownMetadata -> SingleNodeShutdownMetadata.Type.RESTART.equals(shutdownMetadata.getType())) - // Otherwise, use the "normal" allocation delay logic - .orElse(INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING.get(indexMetadata.getSettings()).nanos() > 0); + boolean delayedDueToKnownRestart = Optional.ofNullable(nodesShutdownMetadata.get(node.nodeId())) + .map(shutdown -> Type.RESTART.equals(shutdown.getType()) && shutdown.getShardReallocationDelay().nanos() > 0) + .orElse(false); + boolean delayed = delayedDueToKnownRestart + || INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING.get(indexMetadata.getSettings()).nanos() > 0; UnassignedInfo unassignedInfo = new UnassignedInfo(UnassignedInfo.Reason.NODE_LEFT, "node_left [" + node.nodeId() + "]", null, 0, allocation.getCurrentNanoTime(), System.currentTimeMillis(), delayed, AllocationStatus.NO_ATTEMPT, Collections.emptySet(), shardRouting.currentNodeId()); From 97696798f70dd6b2c251647e9c633983ab1fce8f Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Mon, 2 Aug 2021 15:39:04 -0600 Subject: [PATCH 23/47] Use randomFrom(Type.value()) to generate test shutdown metadata per review --- .../cluster/metadata/NodesShutdownMetadataTests.java | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/server/src/test/java/org/elasticsearch/cluster/metadata/NodesShutdownMetadataTests.java b/server/src/test/java/org/elasticsearch/cluster/metadata/NodesShutdownMetadataTests.java index 899fd904f1336..32567bffc9662 100644 --- a/server/src/test/java/org/elasticsearch/cluster/metadata/NodesShutdownMetadataTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/metadata/NodesShutdownMetadataTests.java @@ -79,9 +79,7 @@ protected NodesShutdownMetadata createTestInstance() { } private SingleNodeShutdownMetadata randomNodeShutdownInfo() { - final SingleNodeShutdownMetadata.Type type = randomBoolean() - ? SingleNodeShutdownMetadata.Type.REMOVE - : SingleNodeShutdownMetadata.Type.RESTART; + final SingleNodeShutdownMetadata.Type type = randomFrom(SingleNodeShutdownMetadata.Type.values()); final SingleNodeShutdownMetadata.Builder builder = SingleNodeShutdownMetadata.builder() .setNodeId(randomAlphaOfLength(5)) .setType(type) From 2d27b21aba48da9e22fbf36b82828c0b31b50839 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Mon, 2 Aug 2021 15:44:18 -0600 Subject: [PATCH 24/47] Use `restartNode()` instead of explicitly stopping/starting per review --- .../NodeShutdownDelayedAllocationIT.java | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/x-pack/plugin/shutdown/src/internalClusterTest/java/org/elasticsearch/xpack/shutdown/NodeShutdownDelayedAllocationIT.java b/x-pack/plugin/shutdown/src/internalClusterTest/java/org/elasticsearch/xpack/shutdown/NodeShutdownDelayedAllocationIT.java index 02d78d564f9f8..29ec7d44315bc 100644 --- a/x-pack/plugin/shutdown/src/internalClusterTest/java/org/elasticsearch/xpack/shutdown/NodeShutdownDelayedAllocationIT.java +++ b/x-pack/plugin/shutdown/src/internalClusterTest/java/org/elasticsearch/xpack/shutdown/NodeShutdownDelayedAllocationIT.java @@ -50,7 +50,6 @@ public void testShardAllocationIsDelayedForRestartingNode() throws Exception { final String nodeToRestartId = findIdOfNodeWithShard(); final String nodeToRestartName = findNodeNameFromId(nodeToRestartId); - Settings nodeToRestartDataPathSettings = internalCluster().dataPathSettings(nodeToRestartName); // Mark the node for shutdown PutShutdownNodeAction.Request putShutdownRequest = new PutShutdownNodeAction.Request( @@ -62,14 +61,15 @@ public void testShardAllocationIsDelayedForRestartingNode() throws Exception { AcknowledgedResponse putShutdownResponse = client().execute(PutShutdownNodeAction.INSTANCE, putShutdownRequest).get(); assertTrue(putShutdownResponse.isAcknowledged()); - // Actually stop the node - internalCluster().stopRandomNode(InternalTestCluster.nameFilter(nodeToRestartName)); - - // Verify that the shard's allocation is delayed - assertBusy(() -> { assertThat(client().admin().cluster().prepareHealth().get().getDelayedUnassignedShards(), equalTo(1)); }); - - // Bring the node back - internalCluster().startNode(nodeToRestartDataPathSettings); // this will use the same data location as the stopped node + internalCluster().restartNode(nodeToRestartName, new InternalTestCluster.RestartCallback() { + @Override + public Settings onNodeStopped(String nodeName) throws Exception { + assertBusy( + () -> { assertThat(client().admin().cluster().prepareHealth().get().getDelayedUnassignedShards(), equalTo(1)); } + ); + return super.onNodeStopped(nodeName); + } + }); // And the index should turn green again ensureGreen("test"); From b0437d9045840677a07dd66e3e62034016e98516 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Mon, 2 Aug 2021 15:54:52 -0600 Subject: [PATCH 25/47] Clean up timeouts per review --- .../NodeShutdownDelayedAllocationIT.java | 34 +++++++------------ 1 file changed, 12 insertions(+), 22 deletions(-) diff --git a/x-pack/plugin/shutdown/src/internalClusterTest/java/org/elasticsearch/xpack/shutdown/NodeShutdownDelayedAllocationIT.java b/x-pack/plugin/shutdown/src/internalClusterTest/java/org/elasticsearch/xpack/shutdown/NodeShutdownDelayedAllocationIT.java index 29ec7d44315bc..fde5f6a1dfb91 100644 --- a/x-pack/plugin/shutdown/src/internalClusterTest/java/org/elasticsearch/xpack/shutdown/NodeShutdownDelayedAllocationIT.java +++ b/x-pack/plugin/shutdown/src/internalClusterTest/java/org/elasticsearch/xpack/shutdown/NodeShutdownDelayedAllocationIT.java @@ -25,7 +25,6 @@ import java.util.Collection; import java.util.Collections; import java.util.List; -import java.util.concurrent.TimeUnit; import static org.hamcrest.Matchers.equalTo; @@ -94,7 +93,7 @@ public void testShardAllocationWillProceedAfterTimeout() throws Exception { nodeToRestartId, SingleNodeShutdownMetadata.Type.RESTART, this.getTestName(), - TimeValue.timeValueSeconds(3) + TimeValue.timeValueMillis(randomIntBetween(10,1000)) ); AcknowledgedResponse putShutdownResponse = client().execute(PutShutdownNodeAction.INSTANCE, putShutdownRequest).get(); assertTrue(putShutdownResponse.isAcknowledged()); @@ -102,15 +101,8 @@ public void testShardAllocationWillProceedAfterTimeout() throws Exception { // Actually stop the node internalCluster().stopRandomNode(InternalTestCluster.nameFilter(nodeToRestartName)); - // Verify that the shard's allocation is delayed - but with a shorter wait than the reallocation timeout - assertBusy( - () -> { assertThat(client().admin().cluster().prepareHealth().get().getDelayedUnassignedShards(), equalTo(1)); }, - 2, - TimeUnit.SECONDS - ); - // And the index should turn green again well within the 30-second timeout - ensureGreen(TimeValue.timeValueSeconds(30), "test"); + ensureGreen("test"); } public void testShardAllocationTimeoutCanBeChanged() throws Exception { @@ -144,25 +136,23 @@ public void testShardAllocationTimeoutCanBeChanged() throws Exception { // Verify that the shard's allocation is delayed - but with a shorter wait than the reallocation timeout assertBusy( - () -> { assertThat(client().admin().cluster().prepareHealth().get().getDelayedUnassignedShards(), equalTo(1)); }, - 2, - TimeUnit.SECONDS + () -> { assertThat(client().admin().cluster().prepareHealth().get().getDelayedUnassignedShards(), equalTo(1)); } ); { - // Update the timeout on the shutdown request to something shorter, but that still shouldn't be quite up yet + // Update the timeout on the shutdown request to something shorter PutShutdownNodeAction.Request putShutdownRequest = new PutShutdownNodeAction.Request( nodeToRestartId, SingleNodeShutdownMetadata.Type.RESTART, this.getTestName(), - TimeValue.timeValueSeconds(10) + TimeValue.timeValueSeconds(1) ); AcknowledgedResponse putShutdownResponse = client().execute(PutShutdownNodeAction.INSTANCE, putShutdownRequest).get(); assertTrue(putShutdownResponse.isAcknowledged()); } - // And the index should turn green again well within the 30-second timeout - ensureGreen(TimeValue.timeValueSeconds(30), "test"); + // And the index should turn green again + ensureGreen("test"); } public void testShardAllocationStartsImmediatelyIfShutdownDeleted() throws Exception { @@ -194,17 +184,17 @@ public void testShardAllocationStartsImmediatelyIfShutdownDeleted() throws Excep // Actually stop the node internalCluster().stopRandomNode(InternalTestCluster.nameFilter(nodeToRestartName)); - // Verify that the shard's allocation is delayed - but with a shorter wait than the reallocation timeout + // Verify that the shard's allocation is delayed assertBusy(() -> { assertThat(client().admin().cluster().prepareHealth().get().getDelayedUnassignedShards(), equalTo(1)); }); { DeleteShutdownNodeAction.Request deleteShutdownRequest = new DeleteShutdownNodeAction.Request(nodeToRestartId); - AcknowledgedResponse putShutdownResponse = client().execute(DeleteShutdownNodeAction.INSTANCE, deleteShutdownRequest).get(); - assertTrue(putShutdownResponse.isAcknowledged()); + AcknowledgedResponse deleteShutdownResponse = client().execute(DeleteShutdownNodeAction.INSTANCE, deleteShutdownRequest).get(); + assertTrue(deleteShutdownResponse.isAcknowledged()); } - // And the index should turn green again well within the 30-second timeout - ensureGreen(TimeValue.timeValueSeconds(30), "test"); + // And the index should turn green again + ensureGreen("test"); } private void indexRandomData() throws Exception { From 0c157f08eac8b5889d913eda957a459b266dcf8b Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Mon, 2 Aug 2021 16:03:59 -0600 Subject: [PATCH 26/47] Factor out shared test logic per review --- .../NodeShutdownDelayedAllocationIT.java | 75 +++++++------------ 1 file changed, 26 insertions(+), 49 deletions(-) diff --git a/x-pack/plugin/shutdown/src/internalClusterTest/java/org/elasticsearch/xpack/shutdown/NodeShutdownDelayedAllocationIT.java b/x-pack/plugin/shutdown/src/internalClusterTest/java/org/elasticsearch/xpack/shutdown/NodeShutdownDelayedAllocationIT.java index fde5f6a1dfb91..431a336a1c940 100644 --- a/x-pack/plugin/shutdown/src/internalClusterTest/java/org/elasticsearch/xpack/shutdown/NodeShutdownDelayedAllocationIT.java +++ b/x-pack/plugin/shutdown/src/internalClusterTest/java/org/elasticsearch/xpack/shutdown/NodeShutdownDelayedAllocationIT.java @@ -106,56 +106,40 @@ public void testShardAllocationWillProceedAfterTimeout() throws Exception { } public void testShardAllocationTimeoutCanBeChanged() throws Exception { - internalCluster().startNodes(3); - prepareCreate("test").setSettings( - Settings.builder() - .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) - .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1) - .put(UnassignedInfo.INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING.getKey(), 0) // Disable "normal" delayed allocation - ).get(); - ensureGreen("test"); - indexRandomData(); - - final String nodeToRestartId = findIdOfNodeWithShard(); - final String nodeToRestartName = findNodeNameFromId(nodeToRestartId); + String nodeToRestartId = setupLongTimeoutTestCase(); - { - // Mark the node for shutdown with a delay that we'll never reach in the test - PutShutdownNodeAction.Request putShutdownRequest = new PutShutdownNodeAction.Request( - nodeToRestartId, - SingleNodeShutdownMetadata.Type.RESTART, - this.getTestName(), - TimeValue.timeValueHours(3) - ); - AcknowledgedResponse putShutdownResponse = client().execute(PutShutdownNodeAction.INSTANCE, putShutdownRequest).get(); - assertTrue(putShutdownResponse.isAcknowledged()); - } + // Update the timeout on the shutdown request to something shorter + PutShutdownNodeAction.Request putShutdownRequest = new PutShutdownNodeAction.Request( + nodeToRestartId, + SingleNodeShutdownMetadata.Type.RESTART, + this.getTestName(), + TimeValue.timeValueSeconds(1) + ); + AcknowledgedResponse putShutdownResponse = client().execute(PutShutdownNodeAction.INSTANCE, putShutdownRequest).get(); + assertTrue(putShutdownResponse.isAcknowledged()); - // Actually stop the node - internalCluster().stopRandomNode(InternalTestCluster.nameFilter(nodeToRestartName)); + // And the index should turn green again + ensureGreen("test"); + } - // Verify that the shard's allocation is delayed - but with a shorter wait than the reallocation timeout - assertBusy( - () -> { assertThat(client().admin().cluster().prepareHealth().get().getDelayedUnassignedShards(), equalTo(1)); } - ); + public void testShardAllocationStartsImmediatelyIfShutdownDeleted() throws Exception { + String nodeToRestartId = setupLongTimeoutTestCase(); - { - // Update the timeout on the shutdown request to something shorter - PutShutdownNodeAction.Request putShutdownRequest = new PutShutdownNodeAction.Request( - nodeToRestartId, - SingleNodeShutdownMetadata.Type.RESTART, - this.getTestName(), - TimeValue.timeValueSeconds(1) - ); - AcknowledgedResponse putShutdownResponse = client().execute(PutShutdownNodeAction.INSTANCE, putShutdownRequest).get(); - assertTrue(putShutdownResponse.isAcknowledged()); - } + DeleteShutdownNodeAction.Request deleteShutdownRequest = new DeleteShutdownNodeAction.Request(nodeToRestartId); + AcknowledgedResponse deleteShutdownResponse = client().execute(DeleteShutdownNodeAction.INSTANCE, deleteShutdownRequest).get(); + assertTrue(deleteShutdownResponse.isAcknowledged()); // And the index should turn green again ensureGreen("test"); } - public void testShardAllocationStartsImmediatelyIfShutdownDeleted() throws Exception { + /** + * Sets up a cluster and an index, picks a random node that has a shard, marks it for shutdown with a long timeout, and then stops the + * node. + * + * @return The ID of the node that was randomly chosen to be marked for shutdown and stopped. + */ + private String setupLongTimeoutTestCase() throws Exception { internalCluster().startNodes(3); prepareCreate("test").setSettings( Settings.builder() @@ -187,14 +171,7 @@ public void testShardAllocationStartsImmediatelyIfShutdownDeleted() throws Excep // Verify that the shard's allocation is delayed assertBusy(() -> { assertThat(client().admin().cluster().prepareHealth().get().getDelayedUnassignedShards(), equalTo(1)); }); - { - DeleteShutdownNodeAction.Request deleteShutdownRequest = new DeleteShutdownNodeAction.Request(nodeToRestartId); - AcknowledgedResponse deleteShutdownResponse = client().execute(DeleteShutdownNodeAction.INSTANCE, deleteShutdownRequest).get(); - assertTrue(deleteShutdownResponse.isAcknowledged()); - } - - // And the index should turn green again - ensureGreen("test"); + return nodeToRestartId; } private void indexRandomData() throws Exception { From b57f7cdee8c402997e99657387ee8c42cf0d10d0 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Mon, 2 Aug 2021 16:07:41 -0600 Subject: [PATCH 27/47] Add type & allocation delay validation to PutShutdownNodeAction.Request per review --- .../elasticsearch/xpack/shutdown/PutShutdownNodeAction.java | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/x-pack/plugin/shutdown/src/main/java/org/elasticsearch/xpack/shutdown/PutShutdownNodeAction.java b/x-pack/plugin/shutdown/src/main/java/org/elasticsearch/xpack/shutdown/PutShutdownNodeAction.java index b61bbad36ab09..50491f4b33be0 100644 --- a/x-pack/plugin/shutdown/src/main/java/org/elasticsearch/xpack/shutdown/PutShutdownNodeAction.java +++ b/x-pack/plugin/shutdown/src/main/java/org/elasticsearch/xpack/shutdown/PutShutdownNodeAction.java @@ -114,6 +114,10 @@ public ActionRequestValidationException validate() { arve.addValidationError("the reason for shutdown is required"); } + if (shardReallocationDelay != null && SingleNodeShutdownMetadata.Type.RESTART.equals(type) == false) { + arve.addValidationError(REALLOCATION_DELAY_FIELD + "is only allowed for RESTART-type shutdown requests"); + } + if (arve.validationErrors().isEmpty() == false) { return arve; } else { From a2e4b07348af16535aaf8fbe6aa2b1d2e035e39b Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Mon, 2 Aug 2021 16:10:45 -0600 Subject: [PATCH 28/47] Change delay field to `allocation_delay` --- .../cluster/metadata/SingleNodeShutdownMetadata.java | 6 +++--- .../elasticsearch/xpack/shutdown/PutShutdownNodeAction.java | 6 +++--- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/cluster/metadata/SingleNodeShutdownMetadata.java b/server/src/main/java/org/elasticsearch/cluster/metadata/SingleNodeShutdownMetadata.java index bb2e8843a7a19..bc4420a50ef63 100644 --- a/server/src/main/java/org/elasticsearch/cluster/metadata/SingleNodeShutdownMetadata.java +++ b/server/src/main/java/org/elasticsearch/cluster/metadata/SingleNodeShutdownMetadata.java @@ -38,7 +38,7 @@ public class SingleNodeShutdownMetadata extends AbstractDiffable PARSER = new ConstructingObjectParser<>( "node_shutdown_info", @@ -58,7 +58,7 @@ public class SingleNodeShutdownMetadata extends AbstractDiffable TimeValue.parseTimeValue(p.textOrNull(), SHARD_REALLOCATION_DELAY.getPreferredName()), SHARD_REALLOCATION_DELAY, + (p, c) -> TimeValue.parseTimeValue(p.textOrNull(), ALLOCATION_DELAY_FIELD.getPreferredName()), ALLOCATION_DELAY_FIELD, ObjectParser.ValueType.STRING_OR_NULL ); } @@ -166,7 +166,7 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws builder.field(REASON_FIELD.getPreferredName(), reason); builder.timeField(STARTED_AT_MILLIS_FIELD.getPreferredName(), STARTED_AT_READABLE_FIELD, startedAtMillis); if (shardReallocationDelay != null) { - builder.field(SHARD_REALLOCATION_DELAY.getPreferredName(), shardReallocationDelay.getStringRep()); + builder.field(ALLOCATION_DELAY_FIELD.getPreferredName(), shardReallocationDelay.getStringRep()); } } builder.endObject(); diff --git a/x-pack/plugin/shutdown/src/main/java/org/elasticsearch/xpack/shutdown/PutShutdownNodeAction.java b/x-pack/plugin/shutdown/src/main/java/org/elasticsearch/xpack/shutdown/PutShutdownNodeAction.java index 50491f4b33be0..404fa96c35432 100644 --- a/x-pack/plugin/shutdown/src/main/java/org/elasticsearch/xpack/shutdown/PutShutdownNodeAction.java +++ b/x-pack/plugin/shutdown/src/main/java/org/elasticsearch/xpack/shutdown/PutShutdownNodeAction.java @@ -42,7 +42,7 @@ public static class Request extends AcknowledgedRequest { private static final ParseField TYPE_FIELD = new ParseField("type"); private static final ParseField REASON_FIELD = new ParseField("reason"); - private static final ParseField REALLOCATION_DELAY_FIELD = new ParseField("shard_reallocation_delay"); + private static final ParseField ALLOCATION_DELAY_FIELD = new ParseField("allocation_delay"); private static final ConstructingObjectParser PARSER = new ConstructingObjectParser<>( "put_node_shutdown_request", @@ -53,7 +53,7 @@ public static class Request extends AcknowledgedRequest { static { PARSER.declareString(ConstructingObjectParser.constructorArg(), TYPE_FIELD); PARSER.declareString(ConstructingObjectParser.constructorArg(), REASON_FIELD); - PARSER.declareString(ConstructingObjectParser.optionalConstructorArg(), REALLOCATION_DELAY_FIELD); + PARSER.declareString(ConstructingObjectParser.optionalConstructorArg(), ALLOCATION_DELAY_FIELD); } public static Request parseRequest(String nodeId, XContentParser parser) { @@ -115,7 +115,7 @@ public ActionRequestValidationException validate() { } if (shardReallocationDelay != null && SingleNodeShutdownMetadata.Type.RESTART.equals(type) == false) { - arve.addValidationError(REALLOCATION_DELAY_FIELD + "is only allowed for RESTART-type shutdown requests"); + arve.addValidationError(ALLOCATION_DELAY_FIELD + "is only allowed for RESTART-type shutdown requests"); } if (arve.validationErrors().isEmpty() == false) { From 1de72bce2b76eed78a153bcb70c6c9c8165475ab Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Mon, 2 Aug 2021 16:11:02 -0600 Subject: [PATCH 29/47] spotless --- .../xpack/shutdown/NodeShutdownDelayedAllocationIT.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/x-pack/plugin/shutdown/src/internalClusterTest/java/org/elasticsearch/xpack/shutdown/NodeShutdownDelayedAllocationIT.java b/x-pack/plugin/shutdown/src/internalClusterTest/java/org/elasticsearch/xpack/shutdown/NodeShutdownDelayedAllocationIT.java index 431a336a1c940..e51a424f466bc 100644 --- a/x-pack/plugin/shutdown/src/internalClusterTest/java/org/elasticsearch/xpack/shutdown/NodeShutdownDelayedAllocationIT.java +++ b/x-pack/plugin/shutdown/src/internalClusterTest/java/org/elasticsearch/xpack/shutdown/NodeShutdownDelayedAllocationIT.java @@ -93,7 +93,7 @@ public void testShardAllocationWillProceedAfterTimeout() throws Exception { nodeToRestartId, SingleNodeShutdownMetadata.Type.RESTART, this.getTestName(), - TimeValue.timeValueMillis(randomIntBetween(10,1000)) + TimeValue.timeValueMillis(randomIntBetween(10, 1000)) ); AcknowledgedResponse putShutdownResponse = client().execute(PutShutdownNodeAction.INSTANCE, putShutdownRequest).get(); assertTrue(putShutdownResponse.isAcknowledged()); From 30c1c3f1bb8c0bcfed15213a272dedc38f0399ee Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Wed, 4 Aug 2021 12:36:01 -0600 Subject: [PATCH 30/47] Change default delay when restarting to 5m --- .../cluster/metadata/SingleNodeShutdownMetadata.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/main/java/org/elasticsearch/cluster/metadata/SingleNodeShutdownMetadata.java b/server/src/main/java/org/elasticsearch/cluster/metadata/SingleNodeShutdownMetadata.java index bc4420a50ef63..a3eb0dfd992ca 100644 --- a/server/src/main/java/org/elasticsearch/cluster/metadata/SingleNodeShutdownMetadata.java +++ b/server/src/main/java/org/elasticsearch/cluster/metadata/SingleNodeShutdownMetadata.java @@ -67,7 +67,7 @@ public static SingleNodeShutdownMetadata parse(XContentParser parser) { return PARSER.apply(parser, null); } - public static final TimeValue DEFAULT_RESTART_SHARD_ALLOCATION_DELAY = TimeValue.timeValueMinutes(10); + public static final TimeValue DEFAULT_RESTART_SHARD_ALLOCATION_DELAY = TimeValue.timeValueMinutes(5); private final String nodeId; private final Type type; From 9a309ac921fbc28535a69799d42c1824eaf80f4f Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Wed, 4 Aug 2021 12:50:55 -0600 Subject: [PATCH 31/47] Use maximum of `index-level delay` and `restart delay`. --- .../cluster/routing/UnassignedInfo.java | 5 ++- .../NodeShutdownDelayedAllocationIT.java | 38 +++++++++++++++++++ 2 files changed, 42 insertions(+), 1 deletion(-) diff --git a/server/src/main/java/org/elasticsearch/cluster/routing/UnassignedInfo.java b/server/src/main/java/org/elasticsearch/cluster/routing/UnassignedInfo.java index 96b98e0d7f356..3b5b9f9bc79c7 100644 --- a/server/src/main/java/org/elasticsearch/cluster/routing/UnassignedInfo.java +++ b/server/src/main/java/org/elasticsearch/cluster/routing/UnassignedInfo.java @@ -240,6 +240,7 @@ public UnassignedInfo(Reason reason, String message) { * @param delayed if allocation of this shard is delayed due to INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING. * @param lastAllocationStatus the result of the last allocation attempt for this shard * @param failedNodeIds a set of nodeIds that failed to complete allocations for this shard + * @param lastAllocatedNodeId the ID of the node this shard was last allocated to */ public UnassignedInfo(Reason reason, @Nullable String message, @Nullable Exception failure, int failedAllocations, long unassignedTimeNanos, long unassignedTimeMillis, boolean delayed, AllocationStatus lastAllocationStatus, @@ -399,12 +400,14 @@ public long getRemainingDelay( final Settings indexSettings, final Map nodesShutdownMap ) { + final long indexLevelDelay = INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING.get(indexSettings).nanos(); long delayTimeoutNanos = Optional.ofNullable(lastAllocatedNodeId) .map(nodesShutdownMap::get) .filter(shutdownMetadata -> SingleNodeShutdownMetadata.Type.RESTART.equals(shutdownMetadata.getType())) .map(SingleNodeShutdownMetadata::getShardReallocationDelay) .map(TimeValue::nanos) - .orElse(INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING.get(indexSettings).nanos()); + .map(knownRestartDelay -> Math.max(indexLevelDelay, knownRestartDelay)) + .orElse(indexLevelDelay); assert nanoTimeNow >= unassignedTimeNanos; return Math.max(0L, delayTimeoutNanos - (nanoTimeNow - unassignedTimeNanos)); } diff --git a/x-pack/plugin/shutdown/src/internalClusterTest/java/org/elasticsearch/xpack/shutdown/NodeShutdownDelayedAllocationIT.java b/x-pack/plugin/shutdown/src/internalClusterTest/java/org/elasticsearch/xpack/shutdown/NodeShutdownDelayedAllocationIT.java index e51a424f466bc..5caa3d856360a 100644 --- a/x-pack/plugin/shutdown/src/internalClusterTest/java/org/elasticsearch/xpack/shutdown/NodeShutdownDelayedAllocationIT.java +++ b/x-pack/plugin/shutdown/src/internalClusterTest/java/org/elasticsearch/xpack/shutdown/NodeShutdownDelayedAllocationIT.java @@ -105,6 +105,44 @@ public void testShardAllocationWillProceedAfterTimeout() throws Exception { ensureGreen("test"); } + public void testIndexLevelAllocationDelayWillBeUsedIfLongerThanShutdownDelay() throws Exception { + internalCluster().startNodes(3); + prepareCreate("test").setSettings( + Settings.builder() + .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1) + .put(UnassignedInfo.INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING.getKey(), "3h") // Use a long timeout we definitely won't hit + ).get(); + ensureGreen("test"); + indexRandomData(); + + final String nodeToRestartId = findIdOfNodeWithShard(); + final String nodeToRestartName = findNodeNameFromId(nodeToRestartId); + + // Mark the node for shutdown + PutShutdownNodeAction.Request putShutdownRequest = new PutShutdownNodeAction.Request( + nodeToRestartId, + SingleNodeShutdownMetadata.Type.RESTART, + this.getTestName(), + TimeValue.timeValueMillis(0) // No delay for reallocating these shards, IF this timeout is used. + ); + AcknowledgedResponse putShutdownResponse = client().execute(PutShutdownNodeAction.INSTANCE, putShutdownRequest).get(); + assertTrue(putShutdownResponse.isAcknowledged()); + + internalCluster().restartNode(nodeToRestartName, new InternalTestCluster.RestartCallback() { + @Override + public Settings onNodeStopped(String nodeName) throws Exception { + assertBusy( + () -> { assertThat(client().admin().cluster().prepareHealth().get().getDelayedUnassignedShards(), equalTo(1)); } + ); + return super.onNodeStopped(nodeName); + } + }); + + // And the index should turn green again + ensureGreen("test"); + } + public void testShardAllocationTimeoutCanBeChanged() throws Exception { String nodeToRestartId = setupLongTimeoutTestCase(); From b34b2b075bdae83deef116efc57884d02a8e30a2 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Wed, 4 Aug 2021 16:37:54 -0600 Subject: [PATCH 32/47] Determine delay based on shutdown state at time shard became unassigned --- .../cluster/routing/RoutingNodes.java | 16 +++-- .../cluster/routing/UnassignedInfo.java | 67 ++++++++++++++----- .../routing/allocation/AllocationService.java | 18 +++-- .../allocator/BalancedShardsAllocator.java | 22 ++++-- ...AllocateEmptyPrimaryAllocationCommand.java | 2 +- .../gateway/ReplicaShardAllocator.java | 7 +- .../cluster/routing/UnassignedInfoTests.java | 61 ++++++++++++++--- ...storeInProgressAllocationDeciderTests.java | 4 +- .../gateway/ReplicaShardAllocatorTests.java | 34 ++++++++-- 9 files changed, 182 insertions(+), 49 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/cluster/routing/RoutingNodes.java b/server/src/main/java/org/elasticsearch/cluster/routing/RoutingNodes.java index 21b84b7bf0d93..06c5302981ee4 100644 --- a/server/src/main/java/org/elasticsearch/cluster/routing/RoutingNodes.java +++ b/server/src/main/java/org/elasticsearch/cluster/routing/RoutingNodes.java @@ -15,6 +15,7 @@ import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.metadata.IndexMetadata; import org.elasticsearch.cluster.metadata.Metadata; +import org.elasticsearch.cluster.metadata.SingleNodeShutdownMetadata; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.routing.UnassignedInfo.AllocationStatus; import org.elasticsearch.cluster.routing.allocation.ExistingShardsAllocator; @@ -38,6 +39,7 @@ import java.util.Map; import java.util.NoSuchElementException; import java.util.Objects; +import java.util.Optional; import java.util.Queue; import java.util.Set; import java.util.function.Predicate; @@ -65,6 +67,8 @@ public class RoutingNodes implements Iterable { private final Map> assignedShards = new HashMap<>(); + private final Map nodeShutdowns; + private final boolean readOnly; private int inactivePrimaryCount = 0; @@ -83,6 +87,7 @@ public RoutingNodes(ClusterState clusterState) { public RoutingNodes(ClusterState clusterState, boolean readOnly) { this.readOnly = readOnly; final RoutingTable routingTable = clusterState.routingTable(); + nodeShutdowns = clusterState.metadata().nodeShutdowns(); Map> nodesToShards = new HashMap<>(); // fill in the nodeToShards with the "live" nodes @@ -533,6 +538,9 @@ assert getByAllocationId(failedShard.shardId(), failedShard.allocationId().getId // 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"; + boolean nodeIsRestarting = Optional.ofNullable(nodeShutdowns.get(replicaShard.currentNodeId())) + .map(shutdownInfo -> shutdownInfo.getType().equals(SingleNodeShutdownMetadata.Type.RESTART)) + .orElse(false); UnassignedInfo primaryFailedUnassignedInfo = new UnassignedInfo( UnassignedInfo.Reason.PRIMARY_FAILED, "primary failed while replica initializing", @@ -543,8 +551,8 @@ assert getByAllocationId(failedShard.shardId(), failedShard.allocationId().getId false, AllocationStatus.NO_ATTEMPT, Collections.emptySet(), - routing.currentNodeId() - ); + routing.currentNodeId(), + nodeIsRestarting); failShard(logger, replicaShard, primaryFailedUnassignedInfo, indexMetadata, routingChangesObserver); } } @@ -877,8 +885,8 @@ public void ignoreShard(ShardRouting shard, AllocationStatus allocationStatus, R currInfo.isDelayed(), allocationStatus, currInfo.getFailedNodeIds(), - currInfo.getLastAllocatedNodeId() - ); + currInfo.getLastAllocatedNodeId(), + currInfo.isLastAllocatedNodeIsRestarting()); ShardRouting updatedShard = shard.updateUnassigned(newInfo, shard.recoverySource()); changes.unassignedInfoUpdated(shard, newInfo); shard = updatedShard; diff --git a/server/src/main/java/org/elasticsearch/cluster/routing/UnassignedInfo.java b/server/src/main/java/org/elasticsearch/cluster/routing/UnassignedInfo.java index 3b5b9f9bc79c7..3b7b1a93fb4ea 100644 --- a/server/src/main/java/org/elasticsearch/cluster/routing/UnassignedInfo.java +++ b/server/src/main/java/org/elasticsearch/cluster/routing/UnassignedInfo.java @@ -219,6 +219,7 @@ public String value() { private final Set failedNodeIds; private final AllocationStatus lastAllocationStatus; // result of the last allocation attempt for this shard private final String lastAllocatedNodeId; + private final boolean lastAllocatedNodeIsRestarting; /** * creates an UnassignedInfo object based on **current** time @@ -228,23 +229,34 @@ public String value() { **/ public UnassignedInfo(Reason reason, String message) { this(reason, message, null, reason == Reason.ALLOCATION_FAILED ? 1 : 0, System.nanoTime(), System.currentTimeMillis(), false, - AllocationStatus.NO_ATTEMPT, Collections.emptySet(), null); + AllocationStatus.NO_ATTEMPT, Collections.emptySet(), null, false); } /** - * @param reason the cause for making this shard unassigned. See {@link Reason} for more information. - * @param message more information about cause. - * @param failure the shard level failure that caused this shard to be unassigned, if exists. - * @param unassignedTimeNanos the time to use as the base for any delayed re-assignment calculation - * @param unassignedTimeMillis the time of unassignment used to display to in our reporting. - * @param delayed if allocation of this shard is delayed due to INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING. - * @param lastAllocationStatus the result of the last allocation attempt for this shard - * @param failedNodeIds a set of nodeIds that failed to complete allocations for this shard - * @param lastAllocatedNodeId the ID of the node this shard was last allocated to + * @param reason the cause for making this shard unassigned. See {@link Reason} for more information. + * @param message more information about cause. + * @param failure the shard level failure that caused this shard to be unassigned, if exists. + * @param unassignedTimeNanos the time to use as the base for any delayed re-assignment calculation + * @param unassignedTimeMillis the time of unassignment used to display to in our reporting. + * @param delayed if allocation of this shard is delayed due to INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING. + * @param lastAllocationStatus the result of the last allocation attempt for this shard + * @param failedNodeIds a set of nodeIds that failed to complete allocations for this shard + * @param lastAllocatedNodeId the ID of the node this shard was last allocated to + * @param lastAllocatedNodeWasRestarting whether the last allocated node was known to be restarting when this shard became unassigned */ - public UnassignedInfo(Reason reason, @Nullable String message, @Nullable Exception failure, int failedAllocations, - long unassignedTimeNanos, long unassignedTimeMillis, boolean delayed, AllocationStatus lastAllocationStatus, - Set failedNodeIds, @Nullable String lastAllocatedNodeId) { + public UnassignedInfo( + Reason reason, + @Nullable String message, + @Nullable Exception failure, + int failedAllocations, + long unassignedTimeNanos, + long unassignedTimeMillis, + boolean delayed, + AllocationStatus lastAllocationStatus, + Set failedNodeIds, + @Nullable String lastAllocatedNodeId, + boolean lastAllocatedNodeWasRestarting + ) { this.reason = Objects.requireNonNull(reason); this.unassignedTimeMillis = unassignedTimeMillis; this.unassignedTimeNanos = unassignedTimeNanos; @@ -255,8 +267,11 @@ public UnassignedInfo(Reason reason, @Nullable String message, @Nullable Excepti this.lastAllocationStatus = Objects.requireNonNull(lastAllocationStatus); this.failedNodeIds = Collections.unmodifiableSet(failedNodeIds); this.lastAllocatedNodeId = lastAllocatedNodeId; - assert (failedAllocations > 0) == (reason == Reason.ALLOCATION_FAILED) : - "failedAllocations: " + failedAllocations + " for reason " + reason; + this.lastAllocatedNodeIsRestarting = lastAllocatedNodeWasRestarting; + assert (failedAllocations > 0) == (reason == Reason.ALLOCATION_FAILED) : "failedAllocations: " + + failedAllocations + + " for reason " + + reason; assert (message == null && failure != null) == false : "provide a message if a failure exception is provided"; assert (delayed && reason != Reason.NODE_LEFT) == false : "shard can only be delayed if it is unassigned due to a node leaving"; } @@ -275,8 +290,10 @@ public UnassignedInfo(StreamInput in) throws IOException { this.failedNodeIds = Collections.unmodifiableSet(in.readSet(StreamInput::readString)); if (in.getVersion().onOrAfter(VERSION_LAST_ALLOCATED_NODE_ADDED)) { this.lastAllocatedNodeId = in.readOptionalString(); + this.lastAllocatedNodeIsRestarting = in.readBoolean(); } else { this.lastAllocatedNodeId = null; + this.lastAllocatedNodeIsRestarting = false; } } @@ -292,6 +309,7 @@ public void writeTo(StreamOutput out) throws IOException { out.writeCollection(failedNodeIds, StreamOutput::writeString); if (out.getVersion().onOrAfter(VERSION_LAST_ALLOCATED_NODE_ADDED)) { out.writeOptionalString(lastAllocatedNodeId); + out.writeBoolean(lastAllocatedNodeIsRestarting); } } @@ -368,6 +386,13 @@ public String getLastAllocatedNodeId() { return lastAllocatedNodeId; } + /** + * Whether the node this shard was last assigned to was restarting when it became unassigned. + */ + public boolean isLastAllocatedNodeIsRestarting() { + return lastAllocatedNodeIsRestarting; + } + /** * Get the status for the last allocation attempt for this shard. */ @@ -402,6 +427,7 @@ public long getRemainingDelay( ) { final long indexLevelDelay = INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING.get(indexSettings).nanos(); long delayTimeoutNanos = Optional.ofNullable(lastAllocatedNodeId) + .filter(nodeId -> lastAllocatedNodeIsRestarting) // If the node wasn't restarting when this became unassigned, use default delay .map(nodesShutdownMap::get) .filter(shutdownMetadata -> SingleNodeShutdownMetadata.Type.RESTART.equals(shutdownMetadata.getType())) .map(SingleNodeShutdownMetadata::getShardReallocationDelay) @@ -530,6 +556,15 @@ public boolean equals(Object o) { if (Objects.equals(failure, that.failure) == false) { return false; } + + if (Objects.equals(lastAllocatedNodeId, that.lastAllocatedNodeId) == false) { + return false; + } + + if (lastAllocatedNodeIsRestarting != that.lastAllocatedNodeIsRestarting) { + return false; + } + return failedNodeIds.equals(that.failedNodeIds); } @@ -543,6 +578,8 @@ public int hashCode() { result = 31 * result + (failure != null ? failure.hashCode() : 0); result = 31 * result + lastAllocationStatus.hashCode(); result = 31 * result + failedNodeIds.hashCode(); + result = 31 * result + (lastAllocatedNodeId != null ? lastAllocatedNodeId.hashCode() : 0); + result = 31 * result + Boolean.hashCode(lastAllocatedNodeIsRestarting); return result; } diff --git a/server/src/main/java/org/elasticsearch/cluster/routing/allocation/AllocationService.java b/server/src/main/java/org/elasticsearch/cluster/routing/allocation/AllocationService.java index e2db4f85829d7..3443be62a33b3 100644 --- a/server/src/main/java/org/elasticsearch/cluster/routing/allocation/AllocationService.java +++ b/server/src/main/java/org/elasticsearch/cluster/routing/allocation/AllocationService.java @@ -210,9 +210,12 @@ public ClusterState applyFailedShards(final ClusterState clusterState, final Lis failedNodeIds = Collections.emptySet(); } String message = "failed shard on node [" + shardToFail.currentNodeId() + "]: " + failedShardEntry.getMessage(); + boolean nodeIsRestarting = Optional.ofNullable(clusterState.metadata().nodeShutdowns().get(shardToFail.currentNodeId())) + .map(shutdownInfo -> shutdownInfo.getType().equals(Type.RESTART)) + .orElse(false); UnassignedInfo unassignedInfo = new UnassignedInfo(UnassignedInfo.Reason.ALLOCATION_FAILED, message, failedShardEntry.getFailure(), failedAllocations + 1, currentNanoTime, System.currentTimeMillis(), false, - AllocationStatus.NO_ATTEMPT, failedNodeIds, shardToFail.currentNodeId()); + AllocationStatus.NO_ATTEMPT, failedNodeIds, shardToFail.currentNodeId(), nodeIsRestarting); if (failedShardEntry.markAsStale()) { allocation.removeAllocationId(failedShard); } @@ -320,8 +323,8 @@ private void removeDelayMarkers(RoutingAllocation allocation) { false, unassignedInfo.getLastAllocationStatus(), unassignedInfo.getFailedNodeIds(), - unassignedInfo.getLastAllocatedNodeId() - ), + unassignedInfo.getLastAllocatedNodeId(), + unassignedInfo.isLastAllocatedNodeIsRestarting()), shardRouting.recoverySource(), allocation.changes() ); @@ -349,8 +352,8 @@ private void resetFailedAllocationCounter(RoutingAllocation allocation) { unassignedInfo.isDelayed(), unassignedInfo.getLastAllocationStatus(), Collections.emptySet(), - unassignedInfo.getLastAllocatedNodeId() - ), + unassignedInfo.getLastAllocatedNodeId(), + unassignedInfo.isLastAllocatedNodeIsRestarting()), shardRouting.recoverySource(), allocation.changes() ); @@ -505,9 +508,12 @@ private void disassociateDeadNodes(RoutingAllocation allocation) { .orElse(false); boolean delayed = delayedDueToKnownRestart || INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING.get(indexMetadata.getSettings()).nanos() > 0; + boolean nodeIsRestarting = Optional.ofNullable(nodesShutdownMetadata.get(shardRouting.currentNodeId())) + .map(shutdownInfo -> shutdownInfo.getType().equals(Type.RESTART)) + .orElse(false); UnassignedInfo unassignedInfo = new UnassignedInfo(UnassignedInfo.Reason.NODE_LEFT, "node_left [" + node.nodeId() + "]", null, 0, allocation.getCurrentNanoTime(), System.currentTimeMillis(), delayed, AllocationStatus.NO_ATTEMPT, - Collections.emptySet(), shardRouting.currentNodeId()); + Collections.emptySet(), shardRouting.currentNodeId(), nodeIsRestarting); allocation.routingNodes().failShard(logger, shardRouting, unassignedInfo, indexMetadata, allocation.changes()); } // its a dead node, remove it, note, its important to remove it *after* we apply failed shard diff --git a/server/src/main/java/org/elasticsearch/cluster/routing/allocation/allocator/BalancedShardsAllocator.java b/server/src/main/java/org/elasticsearch/cluster/routing/allocation/allocator/BalancedShardsAllocator.java index d9ae70ecdf958..dce5ce025d642 100644 --- a/server/src/main/java/org/elasticsearch/cluster/routing/allocation/allocator/BalancedShardsAllocator.java +++ b/server/src/main/java/org/elasticsearch/cluster/routing/allocation/allocator/BalancedShardsAllocator.java @@ -138,11 +138,23 @@ private void failAllocationOfNewPrimaries(RoutingAllocation allocation) { final ShardRouting shardRouting = unassignedIterator.next(); final UnassignedInfo unassignedInfo = shardRouting.unassignedInfo(); if (shardRouting.primary() && unassignedInfo.getLastAllocationStatus() == AllocationStatus.NO_ATTEMPT) { - unassignedIterator.updateUnassigned(new UnassignedInfo(unassignedInfo.getReason(), unassignedInfo.getMessage(), - unassignedInfo.getFailure(), unassignedInfo.getNumFailedAllocations(), unassignedInfo.getUnassignedTimeInNanos(), - unassignedInfo.getUnassignedTimeInMillis(), unassignedInfo.isDelayed(), AllocationStatus.DECIDERS_NO, - unassignedInfo.getFailedNodeIds(), unassignedInfo.getLastAllocatedNodeId()), - shardRouting.recoverySource(), allocation.changes()); + unassignedIterator.updateUnassigned( + new UnassignedInfo( + unassignedInfo.getReason(), + unassignedInfo.getMessage(), + unassignedInfo.getFailure(), + unassignedInfo.getNumFailedAllocations(), + unassignedInfo.getUnassignedTimeInNanos(), + unassignedInfo.getUnassignedTimeInMillis(), + unassignedInfo.isDelayed(), + AllocationStatus.DECIDERS_NO, + unassignedInfo.getFailedNodeIds(), + unassignedInfo.getLastAllocatedNodeId(), + unassignedInfo.isLastAllocatedNodeIsRestarting() + ), + shardRouting.recoverySource(), + allocation.changes() + ); } } } diff --git a/server/src/main/java/org/elasticsearch/cluster/routing/allocation/command/AllocateEmptyPrimaryAllocationCommand.java b/server/src/main/java/org/elasticsearch/cluster/routing/allocation/command/AllocateEmptyPrimaryAllocationCommand.java index 980d88d3c42a8..3dc45ae651c65 100644 --- a/server/src/main/java/org/elasticsearch/cluster/routing/allocation/command/AllocateEmptyPrimaryAllocationCommand.java +++ b/server/src/main/java/org/elasticsearch/cluster/routing/allocation/command/AllocateEmptyPrimaryAllocationCommand.java @@ -129,7 +129,7 @@ public RerouteExplanation execute(RoutingAllocation allocation, boolean explain) ", " + shardRouting.unassignedInfo().getMessage(); unassignedInfoToUpdate = new UnassignedInfo(UnassignedInfo.Reason.FORCED_EMPTY_PRIMARY, unassignedInfoMessage, shardRouting.unassignedInfo().getFailure(), 0, System.nanoTime(), System.currentTimeMillis(), false, - shardRouting.unassignedInfo().getLastAllocationStatus(), Collections.emptySet(), null); + shardRouting.unassignedInfo().getLastAllocationStatus(), Collections.emptySet(), null, false); } initializeUnassignedShard(allocation, routingNodes, routingNode, shardRouting, unassignedInfoToUpdate, diff --git a/server/src/main/java/org/elasticsearch/gateway/ReplicaShardAllocator.java b/server/src/main/java/org/elasticsearch/gateway/ReplicaShardAllocator.java index 91508cf5ea46d..54f66332ffd9b 100644 --- a/server/src/main/java/org/elasticsearch/gateway/ReplicaShardAllocator.java +++ b/server/src/main/java/org/elasticsearch/gateway/ReplicaShardAllocator.java @@ -12,6 +12,7 @@ import org.apache.logging.log4j.Logger; import org.elasticsearch.cluster.metadata.IndexMetadata; import org.elasticsearch.cluster.metadata.Metadata; +import org.elasticsearch.cluster.metadata.SingleNodeShutdownMetadata; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.routing.RoutingNode; import org.elasticsearch.cluster.routing.RoutingNodes; @@ -37,6 +38,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Set; import static org.elasticsearch.cluster.routing.UnassignedInfo.INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING; @@ -99,11 +101,14 @@ && canPerformOperationBasedRecovery(primaryStore, shardStores, currentNode) == f currentNode, nodeWithHighestMatch); final Set failedNodeIds = shard.unassignedInfo() == null ? Collections.emptySet() : shard.unassignedInfo().getFailedNodeIds(); + boolean nodeIsRestarting = Optional.ofNullable(metadata.nodeShutdowns().get(shard.currentNodeId())) + .map(shutdownInfo -> shutdownInfo.getType().equals(SingleNodeShutdownMetadata.Type.RESTART)) + .orElse(false); UnassignedInfo unassignedInfo = new UnassignedInfo(UnassignedInfo.Reason.REALLOCATED_REPLICA, "existing allocation of replica to [" + currentNode + "] cancelled, can perform a noop recovery on ["+ nodeWithHighestMatch + "]", null, 0, allocation.getCurrentNanoTime(), System.currentTimeMillis(), false, - UnassignedInfo.AllocationStatus.NO_ATTEMPT, failedNodeIds, null); + UnassignedInfo.AllocationStatus.NO_ATTEMPT, failedNodeIds, null, nodeIsRestarting); // don't cancel shard in the loop as it will cause a ConcurrentModificationException shardCancellationActions.add(() -> routingNodes.failShard(logger, shard, unassignedInfo, metadata.getIndexSafe(shard.index()), allocation.changes())); diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/UnassignedInfoTests.java b/server/src/test/java/org/elasticsearch/cluster/routing/UnassignedInfoTests.java index bdbcac7a74191..f50d1c5a90807 100644 --- a/server/src/test/java/org/elasticsearch/cluster/routing/UnassignedInfoTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/routing/UnassignedInfoTests.java @@ -93,8 +93,8 @@ public void testSerialization() throws Exception { false, AllocationStatus.NO_ATTEMPT, failedNodes, - lastAssignedNodeId - ) + lastAssignedNodeId, + randomBoolean()) : new UnassignedInfo(reason, randomBoolean() ? randomAlphaOfLength(4) : null); BytesStreamOutput out = new BytesStreamOutput(); meta.writeTo(out); @@ -310,7 +310,13 @@ public void testFailedShard() { * Verifies that delayed allocation calculation are correct when there are no registered node shutdowns. */ public void testRemainingDelayCalculationWithNoShutdowns() throws Exception { - checkRemainingDelayCalculation("bogusNodeId", TimeValue.timeValueNanos(10), Collections.emptyMap(), TimeValue.timeValueNanos(10)); + checkRemainingDelayCalculation( + "bogusNodeId", + TimeValue.timeValueNanos(10), + Collections.emptyMap(), + TimeValue.timeValueNanos(10), + false + ); } /** @@ -330,7 +336,7 @@ public void testRemainingDelayCalculationsWithUnrelatedShutdowns() throws Except .build(); shutdowns.put(shutdown.getNodeId(), shutdown); } - checkRemainingDelayCalculation(lastNodeId, TimeValue.timeValueNanos(10), shutdowns, TimeValue.timeValueNanos(10)); + checkRemainingDelayCalculation(lastNodeId, TimeValue.timeValueNanos(10), shutdowns, TimeValue.timeValueNanos(10), false); } /** @@ -347,7 +353,7 @@ public void testRemainingDelayCalculationWhenNodeIsShuttingDownForRemoval() thro .build(); shutdowns.put(shutdown.getNodeId(), shutdown); - checkRemainingDelayCalculation(lastNodeId, TimeValue.timeValueNanos(10), shutdowns, TimeValue.timeValueNanos(10)); + checkRemainingDelayCalculation(lastNodeId, TimeValue.timeValueNanos(10), shutdowns, TimeValue.timeValueNanos(10), false); } /** @@ -367,7 +373,7 @@ public void testRemainingDelayCalculationWhenNodeIsKnownToBeRestartingWithCustom shutdowns.put(shutdown.getNodeId(), shutdown); // Use a different index-level delay so this test will fail if that one gets used instead of the one from the shutdown metadata - checkRemainingDelayCalculation(lastNodeId, TimeValue.timeValueNanos(10), shutdowns, TimeValue.timeValueMinutes(1)); + checkRemainingDelayCalculation(lastNodeId, TimeValue.timeValueNanos(10), shutdowns, TimeValue.timeValueMinutes(1), true); } /** @@ -392,7 +398,42 @@ public void testRemainingDelayCalculationWhenNodeIsKnownToBeRestartingWithDefaul lastNodeId, TimeValue.timeValueNanos(10), shutdowns, - SingleNodeShutdownMetadata.DEFAULT_RESTART_SHARD_ALLOCATION_DELAY + SingleNodeShutdownMetadata.DEFAULT_RESTART_SHARD_ALLOCATION_DELAY, + true + ); + } + + public void testRemainingDelayUsesIndexLevelDelayIfNodeWasNotRestartingWhenShardBecameUnassigned() throws Exception { + String lastNodeId = "bogusNodeId"; + Map shutdowns = new HashMap<>(); + + // Generate a random time value - but don't use nanos as extremely small values of nanos can break assertion calculations + final TimeValue shutdownDelay = TimeValue.parseTimeValue( + randomTimeValue(2, 1000, "d", "h", "ms", "s", "m", "micros"), + this.getTestName() + ); + SingleNodeShutdownMetadata shutdown = SingleNodeShutdownMetadata.builder() + .setNodeId(lastNodeId) + .setReason(this.getTestName()) + .setStartedAtMillis(randomNonNegativeLong()) + .setType(SingleNodeShutdownMetadata.Type.RESTART) + .setShardReallocationDelay(shutdownDelay) + .build(); + shutdowns.put(shutdown.getNodeId(), shutdown); + + // We want an index level delay that's less than the shutdown delay to avoid picking the index-level delay because it's larger + final TimeValue indexLevelDelay = randomValueOtherThanMany( + tv -> shutdownDelay.compareTo(tv) < 0, + () -> TimeValue.parseTimeValue(randomTimeValue(1, 1000, "d", "h", "ms", "s", "m", "micros"), this.getTestName()) + ); + + logger.info("index level delay: {}, shutdown delay: {}", indexLevelDelay, shutdownDelay); + checkRemainingDelayCalculation( + lastNodeId, + indexLevelDelay, + shutdowns, + indexLevelDelay, + false ); } @@ -400,7 +441,8 @@ private void checkRemainingDelayCalculation( String lastNodeId, TimeValue indexLevelTimeoutSetting, Map nodeShutdowns, - TimeValue expectedTotalDelay + TimeValue expectedTotalDelay, + boolean nodeRestarting ) throws Exception { final long baseTime = System.nanoTime(); UnassignedInfo unassignedInfo = new UnassignedInfo( @@ -413,7 +455,8 @@ private void checkRemainingDelayCalculation( randomBoolean(), AllocationStatus.NO_ATTEMPT, Collections.emptySet(), - lastNodeId + lastNodeId, + nodeRestarting ); final long totalDelayNanos = expectedTotalDelay.nanos(); final Settings indexSettings = Settings.builder() diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/RestoreInProgressAllocationDeciderTests.java b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/RestoreInProgressAllocationDeciderTests.java index 10e2ed88c8f02..d7d6da1030c2c 100644 --- a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/RestoreInProgressAllocationDeciderTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/RestoreInProgressAllocationDeciderTests.java @@ -118,8 +118,8 @@ public void testCanAllocatePrimaryExistingInRestoreInProgress() { currentInfo.isDelayed(), currentInfo.getLastAllocationStatus(), currentInfo.getFailedNodeIds(), - currentInfo.getLastAllocatedNodeId() - ); + currentInfo.getLastAllocatedNodeId(), + currentInfo.isLastAllocatedNodeIsRestarting()); primary = primary.updateUnassigned(newInfo, primary.recoverySource()); IndexRoutingTable indexRoutingTable = routingTable.index("test"); diff --git a/server/src/test/java/org/elasticsearch/gateway/ReplicaShardAllocatorTests.java b/server/src/test/java/org/elasticsearch/gateway/ReplicaShardAllocatorTests.java index 81ab455d5f70f..5347ae57052dd 100644 --- a/server/src/test/java/org/elasticsearch/gateway/ReplicaShardAllocatorTests.java +++ b/server/src/test/java/org/elasticsearch/gateway/ReplicaShardAllocatorTests.java @@ -190,8 +190,19 @@ public void testCancelRecoveryIfFoundCopyWithNoopRetentionLease() { unassignedInfo = new UnassignedInfo(UnassignedInfo.Reason.CLUSTER_RECOVERED, null); } else { failedNodeIds = new HashSet<>(randomSubsetOf(Set.of("node-4", "node-5", "node-6", "node-7"))); - unassignedInfo = new UnassignedInfo(UnassignedInfo.Reason.ALLOCATION_FAILED, null, null, randomIntBetween(1, 10), - System.nanoTime(), System.currentTimeMillis(), false, UnassignedInfo.AllocationStatus.NO_ATTEMPT, failedNodeIds, null); + unassignedInfo = new UnassignedInfo( + UnassignedInfo.Reason.ALLOCATION_FAILED, + null, + null, + randomIntBetween(1, 10), + System.nanoTime(), + System.currentTimeMillis(), + false, + UnassignedInfo.AllocationStatus.NO_ATTEMPT, + failedNodeIds, + null, + false + ); } RoutingAllocation allocation = onePrimaryOnNode1And1ReplicaRecovering(yesAllocationDeciders(), unassignedInfo); long retainingSeqNo = randomLongBetween(1, Long.MAX_VALUE); @@ -375,8 +386,19 @@ public void testNotCancellingRecoveryIfSyncedOnExistingRecovery() { if (randomBoolean()) { unassignedInfo = new UnassignedInfo(UnassignedInfo.Reason.CLUSTER_RECOVERED, null); } else { - unassignedInfo = new UnassignedInfo(UnassignedInfo.Reason.ALLOCATION_FAILED, null, null, randomIntBetween(1, 10), - System.nanoTime(), System.currentTimeMillis(), false, UnassignedInfo.AllocationStatus.NO_ATTEMPT, Set.of("node-4"), null); + unassignedInfo = new UnassignedInfo( + UnassignedInfo.Reason.ALLOCATION_FAILED, + null, + null, + randomIntBetween(1, 10), + System.nanoTime(), + System.currentTimeMillis(), + false, + UnassignedInfo.AllocationStatus.NO_ATTEMPT, + Set.of("node-4"), + null, + false + ); } RoutingAllocation allocation = onePrimaryOnNode1And1ReplicaRecovering(yesAllocationDeciders(), unassignedInfo); List retentionLeases = new ArrayList<>(); @@ -417,7 +439,7 @@ public void testDoNotCancelForBrokenNode() { } UnassignedInfo unassignedInfo = new UnassignedInfo(UnassignedInfo.Reason.ALLOCATION_FAILED, null, null, randomIntBetween(failedNodes.size(), 10), System.nanoTime(), System.currentTimeMillis(), false, - UnassignedInfo.AllocationStatus.NO_ATTEMPT, failedNodes, null); + UnassignedInfo.AllocationStatus.NO_ATTEMPT, failedNodes, null, false); RoutingAllocation allocation = onePrimaryOnNode1And1ReplicaRecovering(yesAllocationDeciders(), unassignedInfo); long retainingSeqNoOnPrimary = randomLongBetween(0, Long.MAX_VALUE); List retentionLeases = Arrays.asList( @@ -454,7 +476,7 @@ private RoutingAllocation onePrimaryOnNode1And1Replica(AllocationDeciders decide RecoverySource.PeerRecoverySource.INSTANCE, new UnassignedInfo(reason, null, null, failedAllocations, System.nanoTime(), System.currentTimeMillis(), delayed, UnassignedInfo.AllocationStatus.NO_ATTEMPT, - Collections.emptySet(), null))) + Collections.emptySet(), null, false))) .build()) ) .build(); From 0e23e09fad42cbd553e29702dd2a5841cd1e09f2 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Thu, 5 Aug 2021 14:07:43 -0600 Subject: [PATCH 33/47] Switch to NODE_RESTARTING enum value instead of separate boolean --- .../cluster/routing/RoutingNodes.java | 13 ++---- .../cluster/routing/UnassignedInfo.java | 43 ++++++++----------- .../routing/allocation/AllocationService.java | 31 +++++++------ .../allocator/BalancedShardsAllocator.java | 3 +- ...AllocateEmptyPrimaryAllocationCommand.java | 2 +- .../gateway/ReplicaShardAllocator.java | 2 +- .../cluster/routing/UnassignedInfoTests.java | 11 +++-- ...storeInProgressAllocationDeciderTests.java | 3 +- .../gateway/ReplicaShardAllocatorTests.java | 10 ++--- 9 files changed, 52 insertions(+), 66 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/cluster/routing/RoutingNodes.java b/server/src/main/java/org/elasticsearch/cluster/routing/RoutingNodes.java index 06c5302981ee4..b207776d49260 100644 --- a/server/src/main/java/org/elasticsearch/cluster/routing/RoutingNodes.java +++ b/server/src/main/java/org/elasticsearch/cluster/routing/RoutingNodes.java @@ -9,6 +9,7 @@ package org.elasticsearch.cluster.routing; import com.carrotsearch.hppc.cursors.ObjectCursor; + import org.apache.logging.log4j.Logger; import org.apache.lucene.util.CollectionUtil; import org.elasticsearch.Assertions; @@ -20,8 +21,8 @@ import org.elasticsearch.cluster.routing.UnassignedInfo.AllocationStatus; import org.elasticsearch.cluster.routing.allocation.ExistingShardsAllocator; import org.elasticsearch.cluster.service.MasterService; -import org.elasticsearch.core.Nullable; import org.elasticsearch.common.Randomness; +import org.elasticsearch.core.Nullable; import org.elasticsearch.core.Tuple; import org.elasticsearch.index.Index; import org.elasticsearch.index.shard.ShardId; @@ -39,7 +40,6 @@ import java.util.Map; import java.util.NoSuchElementException; import java.util.Objects; -import java.util.Optional; import java.util.Queue; import java.util.Set; import java.util.function.Predicate; @@ -538,9 +538,6 @@ assert getByAllocationId(failedShard.shardId(), failedShard.allocationId().getId // 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"; - boolean nodeIsRestarting = Optional.ofNullable(nodeShutdowns.get(replicaShard.currentNodeId())) - .map(shutdownInfo -> shutdownInfo.getType().equals(SingleNodeShutdownMetadata.Type.RESTART)) - .orElse(false); UnassignedInfo primaryFailedUnassignedInfo = new UnassignedInfo( UnassignedInfo.Reason.PRIMARY_FAILED, "primary failed while replica initializing", @@ -551,8 +548,7 @@ assert getByAllocationId(failedShard.shardId(), failedShard.allocationId().getId false, AllocationStatus.NO_ATTEMPT, Collections.emptySet(), - routing.currentNodeId(), - nodeIsRestarting); + routing.currentNodeId()); failShard(logger, replicaShard, primaryFailedUnassignedInfo, indexMetadata, routingChangesObserver); } } @@ -885,8 +881,7 @@ public void ignoreShard(ShardRouting shard, AllocationStatus allocationStatus, R currInfo.isDelayed(), allocationStatus, currInfo.getFailedNodeIds(), - currInfo.getLastAllocatedNodeId(), - currInfo.isLastAllocatedNodeIsRestarting()); + currInfo.getLastAllocatedNodeId()); ShardRouting updatedShard = shard.updateUnassigned(newInfo, shard.recoverySource()); changes.unassignedInfoUpdated(shard, newInfo); shard = updatedShard; diff --git a/server/src/main/java/org/elasticsearch/cluster/routing/UnassignedInfo.java b/server/src/main/java/org/elasticsearch/cluster/routing/UnassignedInfo.java index 3b7b1a93fb4ea..5dc006f143bda 100644 --- a/server/src/main/java/org/elasticsearch/cluster/routing/UnassignedInfo.java +++ b/server/src/main/java/org/elasticsearch/cluster/routing/UnassignedInfo.java @@ -124,7 +124,11 @@ public enum Reason { /** * Unassigned as a result of closing an index. */ - INDEX_CLOSED + INDEX_CLOSED, + /** + * Similar to NODE_LEFT, but at the time the node left, it had been registered for a restart via the Node Shutdown API. + */ + NODE_RESTARTING } /** @@ -219,7 +223,6 @@ public String value() { private final Set failedNodeIds; private final AllocationStatus lastAllocationStatus; // result of the last allocation attempt for this shard private final String lastAllocatedNodeId; - private final boolean lastAllocatedNodeIsRestarting; /** * creates an UnassignedInfo object based on **current** time @@ -229,7 +232,7 @@ public String value() { **/ public UnassignedInfo(Reason reason, String message) { this(reason, message, null, reason == Reason.ALLOCATION_FAILED ? 1 : 0, System.nanoTime(), System.currentTimeMillis(), false, - AllocationStatus.NO_ATTEMPT, Collections.emptySet(), null, false); + AllocationStatus.NO_ATTEMPT, Collections.emptySet(), null); } /** @@ -242,7 +245,6 @@ public UnassignedInfo(Reason reason, String message) { * @param lastAllocationStatus the result of the last allocation attempt for this shard * @param failedNodeIds a set of nodeIds that failed to complete allocations for this shard * @param lastAllocatedNodeId the ID of the node this shard was last allocated to - * @param lastAllocatedNodeWasRestarting whether the last allocated node was known to be restarting when this shard became unassigned */ public UnassignedInfo( Reason reason, @@ -254,8 +256,7 @@ public UnassignedInfo( boolean delayed, AllocationStatus lastAllocationStatus, Set failedNodeIds, - @Nullable String lastAllocatedNodeId, - boolean lastAllocatedNodeWasRestarting + @Nullable String lastAllocatedNodeId ) { this.reason = Objects.requireNonNull(reason); this.unassignedTimeMillis = unassignedTimeMillis; @@ -267,16 +268,18 @@ public UnassignedInfo( this.lastAllocationStatus = Objects.requireNonNull(lastAllocationStatus); this.failedNodeIds = Collections.unmodifiableSet(failedNodeIds); this.lastAllocatedNodeId = lastAllocatedNodeId; - this.lastAllocatedNodeIsRestarting = lastAllocatedNodeWasRestarting; assert (failedAllocations > 0) == (reason == Reason.ALLOCATION_FAILED) : "failedAllocations: " + failedAllocations + " for reason " + reason; assert (message == null && failure != null) == false : "provide a message if a failure exception is provided"; - assert (delayed && reason != Reason.NODE_LEFT) == false : "shard can only be delayed if it is unassigned due to a node leaving"; + assert (delayed + && (reason != Reason.NODE_LEFT + && reason != Reason.NODE_RESTARTING)) == false : "shard can only be delayed if it is unassigned due to a node leaving"; } public UnassignedInfo(StreamInput in) throws IOException { + // Because Reason.NODE_RESTARTING is new and can't be sent by older versions, there's no need to vary the deserialization behavior this.reason = Reason.values()[(int) in.readByte()]; this.unassignedTimeMillis = in.readLong(); // As System.nanoTime() cannot be compared across different JVMs, reset it to now. @@ -290,15 +293,17 @@ public UnassignedInfo(StreamInput in) throws IOException { this.failedNodeIds = Collections.unmodifiableSet(in.readSet(StreamInput::readString)); if (in.getVersion().onOrAfter(VERSION_LAST_ALLOCATED_NODE_ADDED)) { this.lastAllocatedNodeId = in.readOptionalString(); - this.lastAllocatedNodeIsRestarting = in.readBoolean(); } else { this.lastAllocatedNodeId = null; - this.lastAllocatedNodeIsRestarting = false; } } public void writeTo(StreamOutput out) throws IOException { - out.writeByte((byte) reason.ordinal()); + if (reason.equals(Reason.NODE_RESTARTING) && out.getVersion().before(VERSION_LAST_ALLOCATED_NODE_ADDED)) { + out.writeByte((byte) Reason.NODE_LEFT.ordinal()); + } else { + out.writeByte((byte) reason.ordinal()); + } out.writeLong(unassignedTimeMillis); // Do not serialize unassignedTimeNanos as System.nanoTime() cannot be compared across different JVMs out.writeBoolean(delayed); @@ -309,7 +314,6 @@ public void writeTo(StreamOutput out) throws IOException { out.writeCollection(failedNodeIds, StreamOutput::writeString); if (out.getVersion().onOrAfter(VERSION_LAST_ALLOCATED_NODE_ADDED)) { out.writeOptionalString(lastAllocatedNodeId); - out.writeBoolean(lastAllocatedNodeIsRestarting); } } @@ -386,13 +390,6 @@ public String getLastAllocatedNodeId() { return lastAllocatedNodeId; } - /** - * Whether the node this shard was last assigned to was restarting when it became unassigned. - */ - public boolean isLastAllocatedNodeIsRestarting() { - return lastAllocatedNodeIsRestarting; - } - /** * Get the status for the last allocation attempt for this shard. */ @@ -427,7 +424,8 @@ public long getRemainingDelay( ) { final long indexLevelDelay = INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING.get(indexSettings).nanos(); long delayTimeoutNanos = Optional.ofNullable(lastAllocatedNodeId) - .filter(nodeId -> lastAllocatedNodeIsRestarting) // If the node wasn't restarting when this became unassigned, use default delay + // If the node wasn't restarting when this became unassigned, use default delay + .filter(nodeId -> reason.equals(Reason.NODE_RESTARTING)) .map(nodesShutdownMap::get) .filter(shutdownMetadata -> SingleNodeShutdownMetadata.Type.RESTART.equals(shutdownMetadata.getType())) .map(SingleNodeShutdownMetadata::getShardReallocationDelay) @@ -561,10 +559,6 @@ public boolean equals(Object o) { return false; } - if (lastAllocatedNodeIsRestarting != that.lastAllocatedNodeIsRestarting) { - return false; - } - return failedNodeIds.equals(that.failedNodeIds); } @@ -579,7 +573,6 @@ public int hashCode() { result = 31 * result + lastAllocationStatus.hashCode(); result = 31 * result + failedNodeIds.hashCode(); result = 31 * result + (lastAllocatedNodeId != null ? lastAllocatedNodeId.hashCode() : 0); - result = 31 * result + Boolean.hashCode(lastAllocatedNodeIsRestarting); return result; } diff --git a/server/src/main/java/org/elasticsearch/cluster/routing/allocation/AllocationService.java b/server/src/main/java/org/elasticsearch/cluster/routing/allocation/AllocationService.java index 3443be62a33b3..0997f8ffdf479 100644 --- a/server/src/main/java/org/elasticsearch/cluster/routing/allocation/AllocationService.java +++ b/server/src/main/java/org/elasticsearch/cluster/routing/allocation/AllocationService.java @@ -210,12 +210,9 @@ public ClusterState applyFailedShards(final ClusterState clusterState, final Lis failedNodeIds = Collections.emptySet(); } String message = "failed shard on node [" + shardToFail.currentNodeId() + "]: " + failedShardEntry.getMessage(); - boolean nodeIsRestarting = Optional.ofNullable(clusterState.metadata().nodeShutdowns().get(shardToFail.currentNodeId())) - .map(shutdownInfo -> shutdownInfo.getType().equals(Type.RESTART)) - .orElse(false); UnassignedInfo unassignedInfo = new UnassignedInfo(UnassignedInfo.Reason.ALLOCATION_FAILED, message, failedShardEntry.getFailure(), failedAllocations + 1, currentNanoTime, System.currentTimeMillis(), false, - AllocationStatus.NO_ATTEMPT, failedNodeIds, shardToFail.currentNodeId(), nodeIsRestarting); + AllocationStatus.NO_ATTEMPT, failedNodeIds, shardToFail.currentNodeId()); if (failedShardEntry.markAsStale()) { allocation.removeAllocationId(failedShard); } @@ -323,8 +320,7 @@ private void removeDelayMarkers(RoutingAllocation allocation) { false, unassignedInfo.getLastAllocationStatus(), unassignedInfo.getFailedNodeIds(), - unassignedInfo.getLastAllocatedNodeId(), - unassignedInfo.isLastAllocatedNodeIsRestarting()), + unassignedInfo.getLastAllocatedNodeId()), shardRouting.recoverySource(), allocation.changes() ); @@ -352,8 +348,7 @@ private void resetFailedAllocationCounter(RoutingAllocation allocation) { unassignedInfo.isDelayed(), unassignedInfo.getLastAllocationStatus(), Collections.emptySet(), - unassignedInfo.getLastAllocatedNodeId(), - unassignedInfo.isLastAllocatedNodeIsRestarting()), + unassignedInfo.getLastAllocatedNodeId()), shardRouting.recoverySource(), allocation.changes() ); @@ -508,12 +503,20 @@ private void disassociateDeadNodes(RoutingAllocation allocation) { .orElse(false); boolean delayed = delayedDueToKnownRestart || INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING.get(indexMetadata.getSettings()).nanos() > 0; - boolean nodeIsRestarting = Optional.ofNullable(nodesShutdownMetadata.get(shardRouting.currentNodeId())) - .map(shutdownInfo -> shutdownInfo.getType().equals(Type.RESTART)) - .orElse(false); - UnassignedInfo unassignedInfo = new UnassignedInfo(UnassignedInfo.Reason.NODE_LEFT, "node_left [" + node.nodeId() + "]", - null, 0, allocation.getCurrentNanoTime(), System.currentTimeMillis(), delayed, AllocationStatus.NO_ATTEMPT, - Collections.emptySet(), shardRouting.currentNodeId(), nodeIsRestarting); + UnassignedInfo unassignedInfo = new UnassignedInfo( + nodesShutdownMetadata.containsKey(node.nodeId()) + ? UnassignedInfo.Reason.NODE_RESTARTING + : UnassignedInfo.Reason.NODE_LEFT, + "node_left [" + node.nodeId() + "]", + null, + 0, + allocation.getCurrentNanoTime(), + System.currentTimeMillis(), + delayed, + AllocationStatus.NO_ATTEMPT, + Collections.emptySet(), + shardRouting.currentNodeId() + ); allocation.routingNodes().failShard(logger, shardRouting, unassignedInfo, indexMetadata, allocation.changes()); } // its a dead node, remove it, note, its important to remove it *after* we apply failed shard diff --git a/server/src/main/java/org/elasticsearch/cluster/routing/allocation/allocator/BalancedShardsAllocator.java b/server/src/main/java/org/elasticsearch/cluster/routing/allocation/allocator/BalancedShardsAllocator.java index dce5ce025d642..0b223d0e69c3d 100644 --- a/server/src/main/java/org/elasticsearch/cluster/routing/allocation/allocator/BalancedShardsAllocator.java +++ b/server/src/main/java/org/elasticsearch/cluster/routing/allocation/allocator/BalancedShardsAllocator.java @@ -149,8 +149,7 @@ private void failAllocationOfNewPrimaries(RoutingAllocation allocation) { unassignedInfo.isDelayed(), AllocationStatus.DECIDERS_NO, unassignedInfo.getFailedNodeIds(), - unassignedInfo.getLastAllocatedNodeId(), - unassignedInfo.isLastAllocatedNodeIsRestarting() + unassignedInfo.getLastAllocatedNodeId() ), shardRouting.recoverySource(), allocation.changes() diff --git a/server/src/main/java/org/elasticsearch/cluster/routing/allocation/command/AllocateEmptyPrimaryAllocationCommand.java b/server/src/main/java/org/elasticsearch/cluster/routing/allocation/command/AllocateEmptyPrimaryAllocationCommand.java index 3dc45ae651c65..980d88d3c42a8 100644 --- a/server/src/main/java/org/elasticsearch/cluster/routing/allocation/command/AllocateEmptyPrimaryAllocationCommand.java +++ b/server/src/main/java/org/elasticsearch/cluster/routing/allocation/command/AllocateEmptyPrimaryAllocationCommand.java @@ -129,7 +129,7 @@ public RerouteExplanation execute(RoutingAllocation allocation, boolean explain) ", " + shardRouting.unassignedInfo().getMessage(); unassignedInfoToUpdate = new UnassignedInfo(UnassignedInfo.Reason.FORCED_EMPTY_PRIMARY, unassignedInfoMessage, shardRouting.unassignedInfo().getFailure(), 0, System.nanoTime(), System.currentTimeMillis(), false, - shardRouting.unassignedInfo().getLastAllocationStatus(), Collections.emptySet(), null, false); + shardRouting.unassignedInfo().getLastAllocationStatus(), Collections.emptySet(), null); } initializeUnassignedShard(allocation, routingNodes, routingNode, shardRouting, unassignedInfoToUpdate, diff --git a/server/src/main/java/org/elasticsearch/gateway/ReplicaShardAllocator.java b/server/src/main/java/org/elasticsearch/gateway/ReplicaShardAllocator.java index 54f66332ffd9b..7e9ed9ff10b4e 100644 --- a/server/src/main/java/org/elasticsearch/gateway/ReplicaShardAllocator.java +++ b/server/src/main/java/org/elasticsearch/gateway/ReplicaShardAllocator.java @@ -108,7 +108,7 @@ && canPerformOperationBasedRecovery(primaryStore, shardStores, currentNode) == f "existing allocation of replica to [" + currentNode + "] cancelled, can perform a noop recovery on ["+ nodeWithHighestMatch + "]", null, 0, allocation.getCurrentNanoTime(), System.currentTimeMillis(), false, - UnassignedInfo.AllocationStatus.NO_ATTEMPT, failedNodeIds, null, nodeIsRestarting); + UnassignedInfo.AllocationStatus.NO_ATTEMPT, failedNodeIds, null); // don't cancel shard in the loop as it will cause a ConcurrentModificationException shardCancellationActions.add(() -> routingNodes.failShard(logger, shard, unassignedInfo, metadata.getIndexSafe(shard.index()), allocation.changes())); diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/UnassignedInfoTests.java b/server/src/test/java/org/elasticsearch/cluster/routing/UnassignedInfoTests.java index f50d1c5a90807..f66013d3b58e4 100644 --- a/server/src/test/java/org/elasticsearch/cluster/routing/UnassignedInfoTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/routing/UnassignedInfoTests.java @@ -69,7 +69,8 @@ public void testReasonOrdinalOrder() { UnassignedInfo.Reason.PRIMARY_FAILED, UnassignedInfo.Reason.FORCED_EMPTY_PRIMARY, UnassignedInfo.Reason.MANUAL_ALLOCATION, - UnassignedInfo.Reason.INDEX_CLOSED,}; + UnassignedInfo.Reason.INDEX_CLOSED, + UnassignedInfo.Reason.NODE_RESTARTING}; for (int i = 0; i < order.length; i++) { assertThat(order[i].ordinal(), equalTo(i)); } @@ -93,8 +94,7 @@ public void testSerialization() throws Exception { false, AllocationStatus.NO_ATTEMPT, failedNodes, - lastAssignedNodeId, - randomBoolean()) + lastAssignedNodeId) : new UnassignedInfo(reason, randomBoolean() ? randomAlphaOfLength(4) : null); BytesStreamOutput out = new BytesStreamOutput(); meta.writeTo(out); @@ -446,7 +446,7 @@ private void checkRemainingDelayCalculation( ) throws Exception { final long baseTime = System.nanoTime(); UnassignedInfo unassignedInfo = new UnassignedInfo( - UnassignedInfo.Reason.NODE_LEFT, + nodeRestarting ? UnassignedInfo.Reason.NODE_RESTARTING : UnassignedInfo.Reason.NODE_LEFT, "test", null, 0, @@ -455,8 +455,7 @@ private void checkRemainingDelayCalculation( randomBoolean(), AllocationStatus.NO_ATTEMPT, Collections.emptySet(), - lastNodeId, - nodeRestarting + lastNodeId ); final long totalDelayNanos = expectedTotalDelay.nanos(); final Settings indexSettings = Settings.builder() diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/RestoreInProgressAllocationDeciderTests.java b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/RestoreInProgressAllocationDeciderTests.java index d7d6da1030c2c..fc3b66bd34a70 100644 --- a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/RestoreInProgressAllocationDeciderTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/RestoreInProgressAllocationDeciderTests.java @@ -118,8 +118,7 @@ public void testCanAllocatePrimaryExistingInRestoreInProgress() { currentInfo.isDelayed(), currentInfo.getLastAllocationStatus(), currentInfo.getFailedNodeIds(), - currentInfo.getLastAllocatedNodeId(), - currentInfo.isLastAllocatedNodeIsRestarting()); + currentInfo.getLastAllocatedNodeId()); primary = primary.updateUnassigned(newInfo, primary.recoverySource()); IndexRoutingTable indexRoutingTable = routingTable.index("test"); diff --git a/server/src/test/java/org/elasticsearch/gateway/ReplicaShardAllocatorTests.java b/server/src/test/java/org/elasticsearch/gateway/ReplicaShardAllocatorTests.java index 5347ae57052dd..249fccbe69372 100644 --- a/server/src/test/java/org/elasticsearch/gateway/ReplicaShardAllocatorTests.java +++ b/server/src/test/java/org/elasticsearch/gateway/ReplicaShardAllocatorTests.java @@ -200,8 +200,7 @@ public void testCancelRecoveryIfFoundCopyWithNoopRetentionLease() { false, UnassignedInfo.AllocationStatus.NO_ATTEMPT, failedNodeIds, - null, - false + null ); } RoutingAllocation allocation = onePrimaryOnNode1And1ReplicaRecovering(yesAllocationDeciders(), unassignedInfo); @@ -396,8 +395,7 @@ public void testNotCancellingRecoveryIfSyncedOnExistingRecovery() { false, UnassignedInfo.AllocationStatus.NO_ATTEMPT, Set.of("node-4"), - null, - false + null ); } RoutingAllocation allocation = onePrimaryOnNode1And1ReplicaRecovering(yesAllocationDeciders(), unassignedInfo); @@ -439,7 +437,7 @@ public void testDoNotCancelForBrokenNode() { } UnassignedInfo unassignedInfo = new UnassignedInfo(UnassignedInfo.Reason.ALLOCATION_FAILED, null, null, randomIntBetween(failedNodes.size(), 10), System.nanoTime(), System.currentTimeMillis(), false, - UnassignedInfo.AllocationStatus.NO_ATTEMPT, failedNodes, null, false); + UnassignedInfo.AllocationStatus.NO_ATTEMPT, failedNodes, null); RoutingAllocation allocation = onePrimaryOnNode1And1ReplicaRecovering(yesAllocationDeciders(), unassignedInfo); long retainingSeqNoOnPrimary = randomLongBetween(0, Long.MAX_VALUE); List retentionLeases = Arrays.asList( @@ -476,7 +474,7 @@ private RoutingAllocation onePrimaryOnNode1And1Replica(AllocationDeciders decide RecoverySource.PeerRecoverySource.INSTANCE, new UnassignedInfo(reason, null, null, failedAllocations, System.nanoTime(), System.currentTimeMillis(), delayed, UnassignedInfo.AllocationStatus.NO_ATTEMPT, - Collections.emptySet(), null, false))) + Collections.emptySet(), null))) .build()) ) .build(); From 9df8ac22c9ea9549c6577144fb9d380437c7299a Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Thu, 5 Aug 2021 16:49:02 -0600 Subject: [PATCH 34/47] Rename to `allocationDelay` --- .../metadata/SingleNodeShutdownMetadata.java | 39 ++++++++++--------- .../cluster/routing/UnassignedInfo.java | 2 +- .../routing/allocation/AllocationService.java | 2 +- .../metadata/NodesShutdownMetadataTests.java | 2 +- .../cluster/routing/UnassignedInfoTests.java | 4 +- .../xpack/shutdown/PutShutdownNodeAction.java | 16 ++++---- .../TransportPutShutdownNodeAction.java | 2 +- 7 files changed, 35 insertions(+), 32 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/cluster/metadata/SingleNodeShutdownMetadata.java b/server/src/main/java/org/elasticsearch/cluster/metadata/SingleNodeShutdownMetadata.java index a3eb0dfd992ca..200878577b411 100644 --- a/server/src/main/java/org/elasticsearch/cluster/metadata/SingleNodeShutdownMetadata.java +++ b/server/src/main/java/org/elasticsearch/cluster/metadata/SingleNodeShutdownMetadata.java @@ -73,7 +73,7 @@ public static SingleNodeShutdownMetadata parse(XContentParser parser) { private final Type type; private final String reason; private final long startedAtMillis; - @Nullable private final TimeValue shardReallocationDelay; + @Nullable private final TimeValue allocationDelay; /** * @param nodeId The node ID that this shutdown metadata refers to. @@ -86,16 +86,16 @@ private SingleNodeShutdownMetadata( Type type, String reason, long startedAtMillis, - @Nullable TimeValue shardReallocationDelay + @Nullable TimeValue allocationDelay ) { this.nodeId = Objects.requireNonNull(nodeId, "node ID must not be null"); this.type = Objects.requireNonNull(type, "shutdown type must not be null"); this.reason = Objects.requireNonNull(reason, "shutdown reason must not be null"); this.startedAtMillis = startedAtMillis; - if (shardReallocationDelay != null && Type.RESTART.equals(type) == false) { + if (allocationDelay != null && Type.RESTART.equals(type) == false) { throw new IllegalArgumentException("shard allocation delay is only valid for RESTART-type shutdowns"); } - this.shardReallocationDelay = shardReallocationDelay; + this.allocationDelay = allocationDelay; } public SingleNodeShutdownMetadata(StreamInput in) throws IOException { @@ -103,7 +103,7 @@ public SingleNodeShutdownMetadata(StreamInput in) throws IOException { this.type = in.readEnum(Type.class); this.reason = in.readString(); this.startedAtMillis = in.readVLong(); - this.shardReallocationDelay = in.readOptionalTimeValue(); + this.allocationDelay = in.readOptionalTimeValue(); } /** @@ -139,9 +139,9 @@ public long getStartedAtMillis() { * reassigned while the node is restarting. Will be {@code null} for non-restart shutdowns. */ @Nullable - public TimeValue getShardReallocationDelay() { - if (shardReallocationDelay != null) { - return shardReallocationDelay; + public TimeValue getAllocationDelay() { + if (allocationDelay != null) { + return allocationDelay; } else if (Type.RESTART.equals(type)) { return DEFAULT_RESTART_SHARD_ALLOCATION_DELAY; } @@ -154,7 +154,7 @@ public void writeTo(StreamOutput out) throws IOException { out.writeEnum(type); out.writeString(reason); out.writeVLong(startedAtMillis); - out.writeOptionalTimeValue(shardReallocationDelay); + out.writeOptionalTimeValue(allocationDelay); } @Override @@ -165,8 +165,8 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws builder.field(TYPE_FIELD.getPreferredName(), type); builder.field(REASON_FIELD.getPreferredName(), reason); builder.timeField(STARTED_AT_MILLIS_FIELD.getPreferredName(), STARTED_AT_READABLE_FIELD, startedAtMillis); - if (shardReallocationDelay != null) { - builder.field(ALLOCATION_DELAY_FIELD.getPreferredName(), shardReallocationDelay.getStringRep()); + if (allocationDelay != null) { + builder.field(ALLOCATION_DELAY_FIELD.getPreferredName(), allocationDelay.getStringRep()); } } builder.endObject(); @@ -183,7 +183,7 @@ public boolean equals(Object o) { && getNodeId().equals(that.getNodeId()) && getType() == that.getType() && getReason().equals(that.getReason()) - && Objects.equals(shardReallocationDelay, that.shardReallocationDelay); + && Objects.equals(allocationDelay, that.allocationDelay); } @Override @@ -193,7 +193,7 @@ public int hashCode() { getType(), getReason(), getStartedAtMillis(), - shardReallocationDelay + allocationDelay ); } @@ -217,7 +217,7 @@ public static class Builder { private Type type; private String reason; private long startedAtMillis = -1; - private TimeValue shardReallocationDelay; + private TimeValue allocationDelay; private Builder() {} @@ -257,8 +257,12 @@ public Builder setStartedAtMillis(long startedAtMillis) { return this; } - public Builder setShardReallocationDelay(TimeValue shardReallocationDelay) { - this.shardReallocationDelay = shardReallocationDelay; + /** + * @param allocationDelay The amount of time shard reallocation should be delayed while this node is offline. + * @return This builder. + */ + public Builder setAllocationDelay(TimeValue allocationDelay) { + this.allocationDelay = allocationDelay; return this; } @@ -271,8 +275,7 @@ public SingleNodeShutdownMetadata build() { nodeId, type, reason, - startedAtMillis, - shardReallocationDelay + startedAtMillis, allocationDelay ); } } diff --git a/server/src/main/java/org/elasticsearch/cluster/routing/UnassignedInfo.java b/server/src/main/java/org/elasticsearch/cluster/routing/UnassignedInfo.java index 5dc006f143bda..a82f46b26ecb9 100644 --- a/server/src/main/java/org/elasticsearch/cluster/routing/UnassignedInfo.java +++ b/server/src/main/java/org/elasticsearch/cluster/routing/UnassignedInfo.java @@ -428,7 +428,7 @@ public long getRemainingDelay( .filter(nodeId -> reason.equals(Reason.NODE_RESTARTING)) .map(nodesShutdownMap::get) .filter(shutdownMetadata -> SingleNodeShutdownMetadata.Type.RESTART.equals(shutdownMetadata.getType())) - .map(SingleNodeShutdownMetadata::getShardReallocationDelay) + .map(SingleNodeShutdownMetadata::getAllocationDelay) .map(TimeValue::nanos) .map(knownRestartDelay -> Math.max(indexLevelDelay, knownRestartDelay)) .orElse(indexLevelDelay); diff --git a/server/src/main/java/org/elasticsearch/cluster/routing/allocation/AllocationService.java b/server/src/main/java/org/elasticsearch/cluster/routing/allocation/AllocationService.java index 0997f8ffdf479..f73da4a7fc776 100644 --- a/server/src/main/java/org/elasticsearch/cluster/routing/allocation/AllocationService.java +++ b/server/src/main/java/org/elasticsearch/cluster/routing/allocation/AllocationService.java @@ -499,7 +499,7 @@ private void disassociateDeadNodes(RoutingAllocation allocation) { for (ShardRouting shardRouting : node.copyShards()) { final IndexMetadata indexMetadata = allocation.metadata().getIndexSafe(shardRouting.index()); boolean delayedDueToKnownRestart = Optional.ofNullable(nodesShutdownMetadata.get(node.nodeId())) - .map(shutdown -> Type.RESTART.equals(shutdown.getType()) && shutdown.getShardReallocationDelay().nanos() > 0) + .map(shutdown -> Type.RESTART.equals(shutdown.getType()) && shutdown.getAllocationDelay().nanos() > 0) .orElse(false); boolean delayed = delayedDueToKnownRestart || INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING.get(indexMetadata.getSettings()).nanos() > 0; diff --git a/server/src/test/java/org/elasticsearch/cluster/metadata/NodesShutdownMetadataTests.java b/server/src/test/java/org/elasticsearch/cluster/metadata/NodesShutdownMetadataTests.java index 32567bffc9662..cfa3909c4db5b 100644 --- a/server/src/test/java/org/elasticsearch/cluster/metadata/NodesShutdownMetadataTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/metadata/NodesShutdownMetadataTests.java @@ -86,7 +86,7 @@ private SingleNodeShutdownMetadata randomNodeShutdownInfo() { .setReason(randomAlphaOfLength(5)) .setStartedAtMillis(randomNonNegativeLong()); if (type.equals(SingleNodeShutdownMetadata.Type.RESTART) && randomBoolean()) { - builder.setShardReallocationDelay(TimeValue.parseTimeValue(randomTimeValue(), this.getTestName())); + builder.setAllocationDelay(TimeValue.parseTimeValue(randomTimeValue(), this.getTestName())); } return builder.build(); } diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/UnassignedInfoTests.java b/server/src/test/java/org/elasticsearch/cluster/routing/UnassignedInfoTests.java index f66013d3b58e4..5e2c6e96a6577 100644 --- a/server/src/test/java/org/elasticsearch/cluster/routing/UnassignedInfoTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/routing/UnassignedInfoTests.java @@ -368,7 +368,7 @@ public void testRemainingDelayCalculationWhenNodeIsKnownToBeRestartingWithCustom .setReason(this.getTestName()) .setStartedAtMillis(randomNonNegativeLong()) .setType(SingleNodeShutdownMetadata.Type.RESTART) - .setShardReallocationDelay(TimeValue.timeValueMinutes(1)) + .setAllocationDelay(TimeValue.timeValueMinutes(1)) .build(); shutdowns.put(shutdown.getNodeId(), shutdown); @@ -417,7 +417,7 @@ public void testRemainingDelayUsesIndexLevelDelayIfNodeWasNotRestartingWhenShard .setReason(this.getTestName()) .setStartedAtMillis(randomNonNegativeLong()) .setType(SingleNodeShutdownMetadata.Type.RESTART) - .setShardReallocationDelay(shutdownDelay) + .setAllocationDelay(shutdownDelay) .build(); shutdowns.put(shutdown.getNodeId(), shutdown); diff --git a/x-pack/plugin/shutdown/src/main/java/org/elasticsearch/xpack/shutdown/PutShutdownNodeAction.java b/x-pack/plugin/shutdown/src/main/java/org/elasticsearch/xpack/shutdown/PutShutdownNodeAction.java index 404fa96c35432..e58497413261d 100644 --- a/x-pack/plugin/shutdown/src/main/java/org/elasticsearch/xpack/shutdown/PutShutdownNodeAction.java +++ b/x-pack/plugin/shutdown/src/main/java/org/elasticsearch/xpack/shutdown/PutShutdownNodeAction.java @@ -38,7 +38,7 @@ public static class Request extends AcknowledgedRequest { private final SingleNodeShutdownMetadata.Type type; private final String reason; @Nullable - private final TimeValue shardReallocationDelay; + private final TimeValue allocationDelay; private static final ParseField TYPE_FIELD = new ParseField("type"); private static final ParseField REASON_FIELD = new ParseField("reason"); @@ -60,18 +60,18 @@ public static Request parseRequest(String nodeId, XContentParser parser) { return PARSER.apply(parser, nodeId); } - public Request(String nodeId, SingleNodeShutdownMetadata.Type type, String reason, @Nullable TimeValue shardReallocationDelay) { + public Request(String nodeId, SingleNodeShutdownMetadata.Type type, String reason, @Nullable TimeValue allocationDelay) { this.nodeId = nodeId; this.type = type; this.reason = reason; - this.shardReallocationDelay = shardReallocationDelay; + this.allocationDelay = allocationDelay; } public Request(StreamInput in) throws IOException { this.nodeId = in.readString(); this.type = in.readEnum(SingleNodeShutdownMetadata.Type.class); this.reason = in.readString(); - this.shardReallocationDelay = in.readOptionalTimeValue(); + this.allocationDelay = in.readOptionalTimeValue(); } @Override @@ -79,7 +79,7 @@ public void writeTo(StreamOutput out) throws IOException { out.writeString(nodeId); out.writeEnum(type); out.writeString(reason); - out.writeOptionalTimeValue(shardReallocationDelay); + out.writeOptionalTimeValue(allocationDelay); } public String getNodeId() { @@ -94,8 +94,8 @@ public String getReason() { return reason; } - public TimeValue getShardReallocationDelay() { - return shardReallocationDelay; + public TimeValue getAllocationDelay() { + return allocationDelay; } @Override @@ -114,7 +114,7 @@ public ActionRequestValidationException validate() { arve.addValidationError("the reason for shutdown is required"); } - if (shardReallocationDelay != null && SingleNodeShutdownMetadata.Type.RESTART.equals(type) == false) { + if (allocationDelay != null && SingleNodeShutdownMetadata.Type.RESTART.equals(type) == false) { arve.addValidationError(ALLOCATION_DELAY_FIELD + "is only allowed for RESTART-type shutdown requests"); } diff --git a/x-pack/plugin/shutdown/src/main/java/org/elasticsearch/xpack/shutdown/TransportPutShutdownNodeAction.java b/x-pack/plugin/shutdown/src/main/java/org/elasticsearch/xpack/shutdown/TransportPutShutdownNodeAction.java index 5b55399f8dc14..034c9d0c4e154 100644 --- a/x-pack/plugin/shutdown/src/main/java/org/elasticsearch/xpack/shutdown/TransportPutShutdownNodeAction.java +++ b/x-pack/plugin/shutdown/src/main/java/org/elasticsearch/xpack/shutdown/TransportPutShutdownNodeAction.java @@ -87,7 +87,7 @@ public ClusterState execute(ClusterState currentState) { .setType(request.getType()) .setReason(request.getReason()) .setStartedAtMillis(System.currentTimeMillis()) - .setShardReallocationDelay(request.getShardReallocationDelay()) + .setAllocationDelay(request.getAllocationDelay()) .build(); return ClusterState.builder(currentState) From c2bb7298865c2d07dd98f629000b19ea895a89cb Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Thu, 5 Aug 2021 16:56:57 -0600 Subject: [PATCH 35/47] Add assert per review --- .../org/elasticsearch/cluster/routing/UnassignedInfo.java | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/cluster/routing/UnassignedInfo.java b/server/src/main/java/org/elasticsearch/cluster/routing/UnassignedInfo.java index a82f46b26ecb9..85f56c9849b80 100644 --- a/server/src/main/java/org/elasticsearch/cluster/routing/UnassignedInfo.java +++ b/server/src/main/java/org/elasticsearch/cluster/routing/UnassignedInfo.java @@ -274,8 +274,12 @@ public UnassignedInfo( + reason; assert (message == null && failure != null) == false : "provide a message if a failure exception is provided"; assert (delayed - && (reason != Reason.NODE_LEFT - && reason != Reason.NODE_RESTARTING)) == false : "shard can only be delayed if it is unassigned due to a node leaving"; + && reason != Reason.NODE_LEFT + && reason != Reason.NODE_RESTARTING) == false : "shard can only be delayed if it is unassigned due to a node leaving"; + assert (lastAllocatedNodeId != null + && reason != Reason.NODE_LEFT + && reason != Reason.NODE_RESTARTING) == false + : "last allocated node ID should only be set if the shard is unassigned due to a node leaving"; } public UnassignedInfo(StreamInput in) throws IOException { From 010c5f62765b04b1352453f9fc5b41e5ec4f5e4f Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Thu, 5 Aug 2021 19:37:04 -0600 Subject: [PATCH 36/47] Add space to error message --- .../org/elasticsearch/xpack/shutdown/PutShutdownNodeAction.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/x-pack/plugin/shutdown/src/main/java/org/elasticsearch/xpack/shutdown/PutShutdownNodeAction.java b/x-pack/plugin/shutdown/src/main/java/org/elasticsearch/xpack/shutdown/PutShutdownNodeAction.java index e58497413261d..388227e5c6328 100644 --- a/x-pack/plugin/shutdown/src/main/java/org/elasticsearch/xpack/shutdown/PutShutdownNodeAction.java +++ b/x-pack/plugin/shutdown/src/main/java/org/elasticsearch/xpack/shutdown/PutShutdownNodeAction.java @@ -115,7 +115,7 @@ public ActionRequestValidationException validate() { } if (allocationDelay != null && SingleNodeShutdownMetadata.Type.RESTART.equals(type) == false) { - arve.addValidationError(ALLOCATION_DELAY_FIELD + "is only allowed for RESTART-type shutdown requests"); + arve.addValidationError(ALLOCATION_DELAY_FIELD + " is only allowed for RESTART-type shutdown requests"); } if (arve.validationErrors().isEmpty() == false) { From 8bc7442b96dc909532d79ae81e5d268a41aa1776 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Wed, 11 Aug 2021 14:28:03 -0600 Subject: [PATCH 37/47] Ensure we don't try to set lastNodeId if the reason isn't NODE_LEFT or NODE_RESTARTING --- .../cluster/routing/RoutingNodes.java | 2 +- .../routing/allocation/AllocationService.java | 14 +++++++++++--- 2 files changed, 12 insertions(+), 4 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/cluster/routing/RoutingNodes.java b/server/src/main/java/org/elasticsearch/cluster/routing/RoutingNodes.java index b207776d49260..7ebaa0d8b373a 100644 --- a/server/src/main/java/org/elasticsearch/cluster/routing/RoutingNodes.java +++ b/server/src/main/java/org/elasticsearch/cluster/routing/RoutingNodes.java @@ -548,7 +548,7 @@ assert getByAllocationId(failedShard.shardId(), failedShard.allocationId().getId false, AllocationStatus.NO_ATTEMPT, Collections.emptySet(), - routing.currentNodeId()); + null); failShard(logger, replicaShard, primaryFailedUnassignedInfo, indexMetadata, routingChangesObserver); } } diff --git a/server/src/main/java/org/elasticsearch/cluster/routing/allocation/AllocationService.java b/server/src/main/java/org/elasticsearch/cluster/routing/allocation/AllocationService.java index f73da4a7fc776..dcabca0b0d130 100644 --- a/server/src/main/java/org/elasticsearch/cluster/routing/allocation/AllocationService.java +++ b/server/src/main/java/org/elasticsearch/cluster/routing/allocation/AllocationService.java @@ -212,7 +212,7 @@ public ClusterState applyFailedShards(final ClusterState clusterState, final Lis String message = "failed shard on node [" + shardToFail.currentNodeId() + "]: " + failedShardEntry.getMessage(); UnassignedInfo unassignedInfo = new UnassignedInfo(UnassignedInfo.Reason.ALLOCATION_FAILED, message, failedShardEntry.getFailure(), failedAllocations + 1, currentNanoTime, System.currentTimeMillis(), false, - AllocationStatus.NO_ATTEMPT, failedNodeIds, shardToFail.currentNodeId()); + AllocationStatus.NO_ATTEMPT, failedNodeIds, null); if (failedShardEntry.markAsStale()) { allocation.removeAllocationId(failedShard); } @@ -337,9 +337,17 @@ private void resetFailedAllocationCounter(RoutingAllocation allocation) { while (unassignedIterator.hasNext()) { ShardRouting shardRouting = unassignedIterator.next(); UnassignedInfo unassignedInfo = shardRouting.unassignedInfo(); + UnassignedInfo.Reason reason = unassignedInfo.getReason(); + String lastAllocatedNodeId = unassignedInfo.getLastAllocatedNodeId(); + if (unassignedInfo.getNumFailedAllocations() > 0) { + reason = UnassignedInfo.Reason.MANUAL_ALLOCATION; + // If we're changing the reason, we need to make sure the last allocated node ID is unset to avoid tripping an assertion, + // because the last allocated node ID is only valid for certain reasons (of which MANUAL_ALLOCATION is not one) + lastAllocatedNodeId = null; + } unassignedIterator.updateUnassigned( new UnassignedInfo( - unassignedInfo.getNumFailedAllocations() > 0 ? UnassignedInfo.Reason.MANUAL_ALLOCATION : unassignedInfo.getReason(), + reason, unassignedInfo.getMessage(), unassignedInfo.getFailure(), 0, @@ -348,7 +356,7 @@ private void resetFailedAllocationCounter(RoutingAllocation allocation) { unassignedInfo.isDelayed(), unassignedInfo.getLastAllocationStatus(), Collections.emptySet(), - unassignedInfo.getLastAllocatedNodeId()), + lastAllocatedNodeId), shardRouting.recoverySource(), allocation.changes() ); From ca514676db7a9cef1b12aad26578f70457a117c7 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Fri, 13 Aug 2021 14:25:56 -0600 Subject: [PATCH 38/47] Expand comment per review --- .../java/org/elasticsearch/cluster/routing/UnassignedInfo.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/server/src/main/java/org/elasticsearch/cluster/routing/UnassignedInfo.java b/server/src/main/java/org/elasticsearch/cluster/routing/UnassignedInfo.java index 85f56c9849b80..8b1c8dfea6dd4 100644 --- a/server/src/main/java/org/elasticsearch/cluster/routing/UnassignedInfo.java +++ b/server/src/main/java/org/elasticsearch/cluster/routing/UnassignedInfo.java @@ -126,7 +126,8 @@ public enum Reason { */ INDEX_CLOSED, /** - * Similar to NODE_LEFT, but at the time the node left, it had been registered for a restart via the Node Shutdown API. + * Similar to NODE_LEFT, but at the time the node left, it had been registered for a restart via the Node Shutdown API. Note that + * there is no verification that it was ready to be restarted, so this may be an intentional restart or a node crash. */ NODE_RESTARTING } From 04914935b60285fb90d02d38f5d65b0641ae066b Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Fri, 13 Aug 2021 14:32:06 -0600 Subject: [PATCH 39/47] Modify assert to require lastAllocatedNodeId for NODE_RESTARTING Rather than only allowing NODE_LEFT/NODE_RESTARTING to have a lastAllocatedNodeId --- .../org/elasticsearch/cluster/routing/UnassignedInfo.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/cluster/routing/UnassignedInfo.java b/server/src/main/java/org/elasticsearch/cluster/routing/UnassignedInfo.java index 8b1c8dfea6dd4..f582edc13c50d 100644 --- a/server/src/main/java/org/elasticsearch/cluster/routing/UnassignedInfo.java +++ b/server/src/main/java/org/elasticsearch/cluster/routing/UnassignedInfo.java @@ -277,10 +277,10 @@ public UnassignedInfo( assert (delayed && reason != Reason.NODE_LEFT && reason != Reason.NODE_RESTARTING) == false : "shard can only be delayed if it is unassigned due to a node leaving"; - assert (lastAllocatedNodeId != null - && reason != Reason.NODE_LEFT - && reason != Reason.NODE_RESTARTING) == false - : "last allocated node ID should only be set if the shard is unassigned due to a node leaving"; + // The below check should be expanded to require `lastAllocatedNodeId` for `NODE_LEFT` as well, once we no longer have to consider + // BWC with versions prior to `VERSION_LAST_ALLOCATED_NODE_ADDED`. + assert (reason == Reason.NODE_RESTARTING && lastAllocatedNodeId == null) == false + : "last allocated node ID must be set if the shard is unassigned due to a node restarting"; } public UnassignedInfo(StreamInput in) throws IOException { From dc5d85128694611a8c17019b25d4bfa61c66acfc Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Fri, 13 Aug 2021 14:32:21 -0600 Subject: [PATCH 40/47] Revert "Ensure we don't try to set lastNodeId if the reason isn't NODE_LEFT or NODE_RESTARTING" This reverts commit 8bc7442b96dc909532d79ae81e5d268a41aa1776. --- .../cluster/routing/RoutingNodes.java | 2 +- .../routing/allocation/AllocationService.java | 14 +++----------- 2 files changed, 4 insertions(+), 12 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/cluster/routing/RoutingNodes.java b/server/src/main/java/org/elasticsearch/cluster/routing/RoutingNodes.java index 7ebaa0d8b373a..b207776d49260 100644 --- a/server/src/main/java/org/elasticsearch/cluster/routing/RoutingNodes.java +++ b/server/src/main/java/org/elasticsearch/cluster/routing/RoutingNodes.java @@ -548,7 +548,7 @@ assert getByAllocationId(failedShard.shardId(), failedShard.allocationId().getId false, AllocationStatus.NO_ATTEMPT, Collections.emptySet(), - null); + routing.currentNodeId()); failShard(logger, replicaShard, primaryFailedUnassignedInfo, indexMetadata, routingChangesObserver); } } diff --git a/server/src/main/java/org/elasticsearch/cluster/routing/allocation/AllocationService.java b/server/src/main/java/org/elasticsearch/cluster/routing/allocation/AllocationService.java index dcabca0b0d130..f73da4a7fc776 100644 --- a/server/src/main/java/org/elasticsearch/cluster/routing/allocation/AllocationService.java +++ b/server/src/main/java/org/elasticsearch/cluster/routing/allocation/AllocationService.java @@ -212,7 +212,7 @@ public ClusterState applyFailedShards(final ClusterState clusterState, final Lis String message = "failed shard on node [" + shardToFail.currentNodeId() + "]: " + failedShardEntry.getMessage(); UnassignedInfo unassignedInfo = new UnassignedInfo(UnassignedInfo.Reason.ALLOCATION_FAILED, message, failedShardEntry.getFailure(), failedAllocations + 1, currentNanoTime, System.currentTimeMillis(), false, - AllocationStatus.NO_ATTEMPT, failedNodeIds, null); + AllocationStatus.NO_ATTEMPT, failedNodeIds, shardToFail.currentNodeId()); if (failedShardEntry.markAsStale()) { allocation.removeAllocationId(failedShard); } @@ -337,17 +337,9 @@ private void resetFailedAllocationCounter(RoutingAllocation allocation) { while (unassignedIterator.hasNext()) { ShardRouting shardRouting = unassignedIterator.next(); UnassignedInfo unassignedInfo = shardRouting.unassignedInfo(); - UnassignedInfo.Reason reason = unassignedInfo.getReason(); - String lastAllocatedNodeId = unassignedInfo.getLastAllocatedNodeId(); - if (unassignedInfo.getNumFailedAllocations() > 0) { - reason = UnassignedInfo.Reason.MANUAL_ALLOCATION; - // If we're changing the reason, we need to make sure the last allocated node ID is unset to avoid tripping an assertion, - // because the last allocated node ID is only valid for certain reasons (of which MANUAL_ALLOCATION is not one) - lastAllocatedNodeId = null; - } unassignedIterator.updateUnassigned( new UnassignedInfo( - reason, + unassignedInfo.getNumFailedAllocations() > 0 ? UnassignedInfo.Reason.MANUAL_ALLOCATION : unassignedInfo.getReason(), unassignedInfo.getMessage(), unassignedInfo.getFailure(), 0, @@ -356,7 +348,7 @@ private void resetFailedAllocationCounter(RoutingAllocation allocation) { unassignedInfo.isDelayed(), unassignedInfo.getLastAllocationStatus(), Collections.emptySet(), - lastAllocatedNodeId), + unassignedInfo.getLastAllocatedNodeId()), shardRouting.recoverySource(), allocation.changes() ); From 3269afd002cfef45969c47313c7397153441a47b Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Fri, 13 Aug 2021 14:41:10 -0600 Subject: [PATCH 41/47] Compute reason once per node when disassociating dead nodes, instead of per shard --- .../cluster/routing/allocation/AllocationService.java | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/cluster/routing/allocation/AllocationService.java b/server/src/main/java/org/elasticsearch/cluster/routing/allocation/AllocationService.java index f73da4a7fc776..948ec14bea434 100644 --- a/server/src/main/java/org/elasticsearch/cluster/routing/allocation/AllocationService.java +++ b/server/src/main/java/org/elasticsearch/cluster/routing/allocation/AllocationService.java @@ -495,6 +495,11 @@ private void disassociateDeadNodes(RoutingAllocation allocation) { // its a live node, continue continue; } + final UnassignedInfo.Reason + unassignedReason = + nodesShutdownMetadata.containsKey(node.nodeId()) ? + UnassignedInfo.Reason.NODE_RESTARTING : + UnassignedInfo.Reason.NODE_LEFT; // now, go over all the shards routing on the node, and fail them for (ShardRouting shardRouting : node.copyShards()) { final IndexMetadata indexMetadata = allocation.metadata().getIndexSafe(shardRouting.index()); @@ -503,10 +508,9 @@ private void disassociateDeadNodes(RoutingAllocation allocation) { .orElse(false); boolean delayed = delayedDueToKnownRestart || INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING.get(indexMetadata.getSettings()).nanos() > 0; + UnassignedInfo unassignedInfo = new UnassignedInfo( - nodesShutdownMetadata.containsKey(node.nodeId()) - ? UnassignedInfo.Reason.NODE_RESTARTING - : UnassignedInfo.Reason.NODE_LEFT, + unassignedReason, "node_left [" + node.nodeId() + "]", null, 0, From 6b2f2d16b4d33b79b22480ecc288272948ba2c85 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Fri, 13 Aug 2021 14:42:08 -0600 Subject: [PATCH 42/47] Remove unused code block per review --- .../java/org/elasticsearch/gateway/ReplicaShardAllocator.java | 3 --- 1 file changed, 3 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/gateway/ReplicaShardAllocator.java b/server/src/main/java/org/elasticsearch/gateway/ReplicaShardAllocator.java index 7e9ed9ff10b4e..db9b2b111493e 100644 --- a/server/src/main/java/org/elasticsearch/gateway/ReplicaShardAllocator.java +++ b/server/src/main/java/org/elasticsearch/gateway/ReplicaShardAllocator.java @@ -101,9 +101,6 @@ && canPerformOperationBasedRecovery(primaryStore, shardStores, currentNode) == f currentNode, nodeWithHighestMatch); final Set failedNodeIds = shard.unassignedInfo() == null ? Collections.emptySet() : shard.unassignedInfo().getFailedNodeIds(); - boolean nodeIsRestarting = Optional.ofNullable(metadata.nodeShutdowns().get(shard.currentNodeId())) - .map(shutdownInfo -> shutdownInfo.getType().equals(SingleNodeShutdownMetadata.Type.RESTART)) - .orElse(false); UnassignedInfo unassignedInfo = new UnassignedInfo(UnassignedInfo.Reason.REALLOCATED_REPLICA, "existing allocation of replica to [" + currentNode + "] cancelled, can perform a noop recovery on ["+ nodeWithHighestMatch + "]", From 1a45d0ff52c9554f7c52bf50747e119e806e329f Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Fri, 13 Aug 2021 15:04:05 -0600 Subject: [PATCH 43/47] Adjust UnassignedInfo test instance creation per review --- .../cluster/routing/UnassignedInfoTests.java | 31 ++++++++++++++++--- 1 file changed, 26 insertions(+), 5 deletions(-) diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/UnassignedInfoTests.java b/server/src/test/java/org/elasticsearch/cluster/routing/UnassignedInfoTests.java index 5e2c6e96a6577..529e73af4db99 100644 --- a/server/src/test/java/org/elasticsearch/cluster/routing/UnassignedInfoTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/routing/UnassignedInfoTests.java @@ -82,9 +82,10 @@ public void testSerialization() throws Exception { int failedAllocations = randomIntBetween(1, 100); Set failedNodes = IntStream.range(0, between(0, failedAllocations)) .mapToObj(n -> "failed-node-" + n).collect(Collectors.toSet()); - String lastAssignedNodeId = randomBoolean() ? randomAlphaOfLength(10) : null; - UnassignedInfo meta = reason == UnassignedInfo.Reason.ALLOCATION_FAILED - ? new UnassignedInfo( + + UnassignedInfo meta; + if (reason == UnassignedInfo.Reason.ALLOCATION_FAILED) { + meta = new UnassignedInfo( reason, randomBoolean() ? randomAlphaOfLength(4) : null, null, @@ -94,8 +95,28 @@ public void testSerialization() throws Exception { false, AllocationStatus.NO_ATTEMPT, failedNodes, - lastAssignedNodeId) - : new UnassignedInfo(reason, randomBoolean() ? randomAlphaOfLength(4) : null); + null); + } else if (reason == UnassignedInfo.Reason.NODE_LEFT || reason == UnassignedInfo.Reason.NODE_RESTARTING) { + String lastAssignedNodeId = randomAlphaOfLength(10); + if (reason == UnassignedInfo.Reason.NODE_LEFT && randomBoolean()) { + // If the reason is `NODE_LEFT`, sometimes we'll have an empty lastAllocatedNodeId due to BWC + lastAssignedNodeId = null; + } + meta = new UnassignedInfo( + reason, + randomBoolean() ? randomAlphaOfLength(4) : null, + null, + 0, + System.nanoTime(), + System.currentTimeMillis(), + false, + AllocationStatus.NO_ATTEMPT, + Collections.emptySet(), + lastAssignedNodeId + ); + } else { + meta = new UnassignedInfo(reason, randomBoolean() ? randomAlphaOfLength(4) : null); + } BytesStreamOutput out = new BytesStreamOutput(); meta.writeTo(out); out.close(); From 806610452f3568bf21021fe9820f66792327a3d4 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Fri, 13 Aug 2021 15:05:01 -0600 Subject: [PATCH 44/47] Adjust test delays per review --- .../org/elasticsearch/cluster/routing/UnassignedInfoTests.java | 2 +- .../xpack/shutdown/NodeShutdownDelayedAllocationIT.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/UnassignedInfoTests.java b/server/src/test/java/org/elasticsearch/cluster/routing/UnassignedInfoTests.java index 529e73af4db99..9cb66965eeb60 100644 --- a/server/src/test/java/org/elasticsearch/cluster/routing/UnassignedInfoTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/routing/UnassignedInfoTests.java @@ -430,7 +430,7 @@ public void testRemainingDelayUsesIndexLevelDelayIfNodeWasNotRestartingWhenShard // Generate a random time value - but don't use nanos as extremely small values of nanos can break assertion calculations final TimeValue shutdownDelay = TimeValue.parseTimeValue( - randomTimeValue(2, 1000, "d", "h", "ms", "s", "m", "micros"), + randomTimeValue(100, 1000, "d", "h", "ms", "s", "m", "micros"), this.getTestName() ); SingleNodeShutdownMetadata shutdown = SingleNodeShutdownMetadata.builder() diff --git a/x-pack/plugin/shutdown/src/internalClusterTest/java/org/elasticsearch/xpack/shutdown/NodeShutdownDelayedAllocationIT.java b/x-pack/plugin/shutdown/src/internalClusterTest/java/org/elasticsearch/xpack/shutdown/NodeShutdownDelayedAllocationIT.java index 5caa3d856360a..c89c95d1cf332 100644 --- a/x-pack/plugin/shutdown/src/internalClusterTest/java/org/elasticsearch/xpack/shutdown/NodeShutdownDelayedAllocationIT.java +++ b/x-pack/plugin/shutdown/src/internalClusterTest/java/org/elasticsearch/xpack/shutdown/NodeShutdownDelayedAllocationIT.java @@ -151,7 +151,7 @@ public void testShardAllocationTimeoutCanBeChanged() throws Exception { nodeToRestartId, SingleNodeShutdownMetadata.Type.RESTART, this.getTestName(), - TimeValue.timeValueSeconds(1) + TimeValue.timeValueMillis(1) ); AcknowledgedResponse putShutdownResponse = client().execute(PutShutdownNodeAction.INSTANCE, putShutdownRequest).get(); assertTrue(putShutdownResponse.isAcknowledged()); From cdb3f6e08835cc8c90d5abade3c1d90691f5f164 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Fri, 13 Aug 2021 15:17:30 -0600 Subject: [PATCH 45/47] imports --- .../org/elasticsearch/gateway/ReplicaShardAllocator.java | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/gateway/ReplicaShardAllocator.java b/server/src/main/java/org/elasticsearch/gateway/ReplicaShardAllocator.java index db9b2b111493e..43adcf9ef70e1 100644 --- a/server/src/main/java/org/elasticsearch/gateway/ReplicaShardAllocator.java +++ b/server/src/main/java/org/elasticsearch/gateway/ReplicaShardAllocator.java @@ -9,10 +9,10 @@ package org.elasticsearch.gateway; import com.carrotsearch.hppc.cursors.ObjectCursor; + import org.apache.logging.log4j.Logger; import org.elasticsearch.cluster.metadata.IndexMetadata; import org.elasticsearch.cluster.metadata.Metadata; -import org.elasticsearch.cluster.metadata.SingleNodeShutdownMetadata; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.routing.RoutingNode; import org.elasticsearch.cluster.routing.RoutingNodes; @@ -24,10 +24,10 @@ import org.elasticsearch.cluster.routing.allocation.NodeAllocationResult.ShardStoreInfo; import org.elasticsearch.cluster.routing.allocation.RoutingAllocation; import org.elasticsearch.cluster.routing.allocation.decider.Decision; -import org.elasticsearch.core.Nullable; -import org.elasticsearch.core.Tuple; import org.elasticsearch.common.unit.ByteSizeValue; +import org.elasticsearch.core.Nullable; import org.elasticsearch.core.TimeValue; +import org.elasticsearch.core.Tuple; import org.elasticsearch.index.store.StoreFileMetadata; import org.elasticsearch.indices.store.TransportNodesListShardStoreMetadata; import org.elasticsearch.indices.store.TransportNodesListShardStoreMetadata.NodeStoreFilesMetadata; @@ -38,7 +38,6 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.Optional; import java.util.Set; import static org.elasticsearch.cluster.routing.UnassignedInfo.INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING; From b9f98f091a49f0f063927534eef8ffd8d34f8e09 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Fri, 13 Aug 2021 16:42:23 -0600 Subject: [PATCH 46/47] Assert that NODE_RESTARTING requires lastAllocatedNodeId + fix tests A number of tests created `UnassignedInfo`s without regard for the `reason`, which started failing after adjusting the assertion. These tests now generate their `UnassignedInfo`s via methods which handle the requirements. --- .../cluster/ClusterStateDiffIT.java | 5 +-- .../cluster/routing/AllocationIdTests.java | 16 ++++++++- .../routing/RandomShardRoutingMutator.java | 31 ++-------------- .../cluster/routing/UnassignedInfoTests.java | 36 +++++++++++++++++++ .../gateway/PriorityComparatorTests.java | 17 +++++---- .../cluster/routing/TestShardRouting.java | 36 ++++++++++++++++--- .../core/ilm/CheckShrinkReadyStepTests.java | 27 +++++++++++++- .../ilm/DataTierMigrationRoutedStepTests.java | 3 +- .../indices/IndexStatsMonitoringDocTests.java | 4 +-- 9 files changed, 127 insertions(+), 48 deletions(-) diff --git a/server/src/internalClusterTest/java/org/elasticsearch/cluster/ClusterStateDiffIT.java b/server/src/internalClusterTest/java/org/elasticsearch/cluster/ClusterStateDiffIT.java index cf4de829d559c..600770ea5f163 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/cluster/ClusterStateDiffIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/cluster/ClusterStateDiffIT.java @@ -9,6 +9,7 @@ package org.elasticsearch.cluster; import com.carrotsearch.hppc.cursors.ObjectCursor; + import org.elasticsearch.cluster.block.ClusterBlock; import org.elasticsearch.cluster.block.ClusterBlocks; import org.elasticsearch.cluster.coordination.CoordinationMetadata; @@ -58,7 +59,7 @@ import static java.util.Collections.emptySet; import static org.elasticsearch.cluster.metadata.AliasMetadata.newAliasMetadataBuilder; import static org.elasticsearch.cluster.routing.RandomShardRoutingMutator.randomChange; -import static org.elasticsearch.cluster.routing.RandomShardRoutingMutator.randomReason; +import static org.elasticsearch.cluster.routing.UnassignedInfoTests.randomUnassignedInfo; import static org.elasticsearch.test.VersionUtils.randomVersion; import static org.elasticsearch.test.XContentTestUtils.convertToMap; import static org.elasticsearch.test.XContentTestUtils.differenceBetweenMapsIgnoringArrayOrder; @@ -270,7 +271,7 @@ private IndexRoutingTable randomIndexRoutingTable(String index, String[] nodeIds for (int j = 0; j < replicaCount; j++) { UnassignedInfo unassignedInfo = null; if (randomInt(5) == 1) { - unassignedInfo = new UnassignedInfo(randomReason(), randomAlphaOfLength(10)); + unassignedInfo = randomUnassignedInfo(randomAlphaOfLength(10)); } if (availableNodeIds.isEmpty()) { break; diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/AllocationIdTests.java b/server/src/test/java/org/elasticsearch/cluster/routing/AllocationIdTests.java index 0b5e08c269ca0..ab7b27dfa6bf4 100644 --- a/server/src/test/java/org/elasticsearch/cluster/routing/AllocationIdTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/routing/AllocationIdTests.java @@ -17,6 +17,7 @@ import org.elasticsearch.test.ESTestCase; import java.io.IOException; +import java.util.Collections; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.not; @@ -98,7 +99,20 @@ public void testMoveToUnassigned() { shard = shard.moveToStarted(); logger.info("-- move to unassigned"); - shard = shard.moveToUnassigned(new UnassignedInfo(UnassignedInfo.Reason.NODE_LEFT, null)); + shard = shard.moveToUnassigned( + new UnassignedInfo( + UnassignedInfo.Reason.NODE_LEFT, + this.getTestName(), + null, + 0, + System.nanoTime(), + System.currentTimeMillis(), + false, + UnassignedInfo.AllocationStatus.NO_ATTEMPT, + Collections.emptySet(), + randomAlphaOfLength(10) + ) + ); assertThat(shard.allocationId(), nullValue()); } diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/RandomShardRoutingMutator.java b/server/src/test/java/org/elasticsearch/cluster/routing/RandomShardRoutingMutator.java index 14ae849dee906..de6de339e6ada 100644 --- a/server/src/test/java/org/elasticsearch/cluster/routing/RandomShardRoutingMutator.java +++ b/server/src/test/java/org/elasticsearch/cluster/routing/RandomShardRoutingMutator.java @@ -10,6 +10,7 @@ import java.util.Set; +import static org.elasticsearch.cluster.routing.UnassignedInfoTests.randomUnassignedInfo; import static org.elasticsearch.test.ESTestCase.randomAlphaOfLength; import static org.elasticsearch.test.ESTestCase.randomFrom; import static org.elasticsearch.test.ESTestCase.randomInt; @@ -26,9 +27,9 @@ public static ShardRouting randomChange(ShardRouting shardRouting, Set n switch (randomInt(2)) { case 0: if (shardRouting.unassigned() == false && shardRouting.primary() == false) { - shardRouting = shardRouting.moveToUnassigned(new UnassignedInfo(randomReason(), randomAlphaOfLength(10))); + shardRouting = shardRouting.moveToUnassigned(randomUnassignedInfo(randomAlphaOfLength(10), false)); } else if (shardRouting.unassignedInfo() != null) { - shardRouting = shardRouting.updateUnassigned(new UnassignedInfo(randomReason(), randomAlphaOfLength(10)), + shardRouting = shardRouting.updateUnassigned(randomUnassignedInfo(randomAlphaOfLength(10), false), shardRouting.recoverySource()); } break; @@ -45,30 +46,4 @@ public static ShardRouting randomChange(ShardRouting shardRouting, Set n } return shardRouting; } - - - public static UnassignedInfo.Reason randomReason() { - switch (randomInt(9)) { - case 0: - return UnassignedInfo.Reason.INDEX_CREATED; - case 1: - return UnassignedInfo.Reason.CLUSTER_RECOVERED; - case 2: - return UnassignedInfo.Reason.INDEX_REOPENED; - case 3: - return UnassignedInfo.Reason.DANGLING_INDEX_IMPORTED; - case 4: - return UnassignedInfo.Reason.NEW_INDEX_RESTORED; - case 5: - return UnassignedInfo.Reason.EXISTING_INDEX_RESTORED; - case 6: - return UnassignedInfo.Reason.REPLICA_ADDED; - case 7: - return UnassignedInfo.Reason.ALLOCATION_FAILED; - case 8: - return UnassignedInfo.Reason.NODE_LEFT; - default: - return UnassignedInfo.Reason.REROUTE_CANCELLED; - } - } } diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/UnassignedInfoTests.java b/server/src/test/java/org/elasticsearch/cluster/routing/UnassignedInfoTests.java index 9cb66965eeb60..21c1e8b3484fd 100644 --- a/server/src/test/java/org/elasticsearch/cluster/routing/UnassignedInfoTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/routing/UnassignedInfoTests.java @@ -27,6 +27,7 @@ import org.elasticsearch.common.io.stream.ByteBufferStreamInput; import org.elasticsearch.common.io.stream.BytesStreamOutput; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.core.Nullable; import org.elasticsearch.core.TimeValue; import org.elasticsearch.index.Index; import org.elasticsearch.repositories.IndexId; @@ -569,4 +570,39 @@ public void testAllocationStatusSerialization() throws IOException { assertThat(readStatus, equalTo(allocationStatus)); } } + + public static UnassignedInfo randomUnassignedInfo(String message) { + return randomUnassignedInfo(message, null); + } + + /** + * Randomly generates an UnassignedInfo. + * @param message The message to be used. + * @param delayed Used for the `delayed` flag if provided. + * @return A randomly-generated UnassignedInfo with the given message and delayed value (if any) + */ + public static UnassignedInfo randomUnassignedInfo(String message, @Nullable Boolean delayed) { + UnassignedInfo.Reason reason = randomFrom(UnassignedInfo.Reason.values()); + String lastAllocatedNodeId = null; + boolean delayedFlag = delayed == null ? false : delayed; + if (reason == UnassignedInfo.Reason.NODE_LEFT || reason == UnassignedInfo.Reason.NODE_RESTARTING) { + if (randomBoolean() && delayed == null) { + delayedFlag = true; + } + lastAllocatedNodeId = randomAlphaOfLength(10); + } + int failedAllocations = reason == UnassignedInfo.Reason.ALLOCATION_FAILED ? 1 : 0; + return new UnassignedInfo( + reason, + message, + null, + failedAllocations, + System.nanoTime(), + System.currentTimeMillis(), + delayedFlag, + UnassignedInfo.AllocationStatus.NO_ATTEMPT, + Collections.emptySet(), + lastAllocatedNodeId + ); + } } diff --git a/server/src/test/java/org/elasticsearch/gateway/PriorityComparatorTests.java b/server/src/test/java/org/elasticsearch/gateway/PriorityComparatorTests.java index 7477849c4288c..0b41e4d626032 100644 --- a/server/src/test/java/org/elasticsearch/gateway/PriorityComparatorTests.java +++ b/server/src/test/java/org/elasticsearch/gateway/PriorityComparatorTests.java @@ -13,7 +13,6 @@ import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.cluster.routing.ShardRoutingState; import org.elasticsearch.cluster.routing.TestShardRouting; -import org.elasticsearch.cluster.routing.UnassignedInfo; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.index.Index; import org.elasticsearch.test.ESTestCase; @@ -25,6 +24,7 @@ import java.util.Locale; import java.util.Map; +import static org.elasticsearch.cluster.routing.UnassignedInfoTests.randomUnassignedInfo; import static org.hamcrest.Matchers.greaterThan; import static org.mockito.Mockito.mock; @@ -34,9 +34,9 @@ public void testPreferNewIndices() { RoutingNodes.UnassignedShards shards = new RoutingNodes.UnassignedShards(mock(RoutingNodes.class)); List shardRoutings = Arrays.asList( TestShardRouting.newShardRouting("oldest", 0, null, null, - randomBoolean(), ShardRoutingState.UNASSIGNED, new UnassignedInfo(randomFrom(UnassignedInfo.Reason.values()), "foobar")), + randomBoolean(), ShardRoutingState.UNASSIGNED, randomUnassignedInfo("foobar")), TestShardRouting.newShardRouting("newest", 0, null, null, - randomBoolean(), ShardRoutingState.UNASSIGNED, new UnassignedInfo(randomFrom(UnassignedInfo.Reason.values()), "foobar"))); + randomBoolean(), ShardRoutingState.UNASSIGNED, randomUnassignedInfo("foobar"))); Collections.shuffle(shardRoutings, random()); for (ShardRouting routing : shardRoutings) { shards.add(routing); @@ -68,9 +68,9 @@ public void testPreferPriorityIndices() { RoutingNodes.UnassignedShards shards = new RoutingNodes.UnassignedShards(mock(RoutingNodes.class)); List shardRoutings = Arrays.asList( TestShardRouting.newShardRouting("oldest", 0, null, null, - randomBoolean(), ShardRoutingState.UNASSIGNED, new UnassignedInfo(randomFrom(UnassignedInfo.Reason.values()), "foobar")), + randomBoolean(), ShardRoutingState.UNASSIGNED, randomUnassignedInfo("foobar")), TestShardRouting.newShardRouting("newest", 0, null, null, - randomBoolean(), ShardRoutingState.UNASSIGNED, new UnassignedInfo(randomFrom(UnassignedInfo.Reason.values()), "foobar"))); + randomBoolean(), ShardRoutingState.UNASSIGNED, randomUnassignedInfo("foobar"))); Collections.shuffle(shardRoutings, random()); for (ShardRouting routing : shardRoutings) { shards.add(routing); @@ -102,9 +102,9 @@ public void testPreferSystemIndices() { RoutingNodes.UnassignedShards shards = new RoutingNodes.UnassignedShards(mock(RoutingNodes.class)); List shardRoutings = Arrays.asList( TestShardRouting.newShardRouting("oldest", 0, null, null, - randomBoolean(), ShardRoutingState.UNASSIGNED, new UnassignedInfo(randomFrom(UnassignedInfo.Reason.values()), "foobar")), + randomBoolean(), ShardRoutingState.UNASSIGNED, randomUnassignedInfo("foobar")), TestShardRouting.newShardRouting("newest", 0, null, null, - randomBoolean(), ShardRoutingState.UNASSIGNED, new UnassignedInfo(randomFrom(UnassignedInfo.Reason.values()), "foobar"))); + randomBoolean(), ShardRoutingState.UNASSIGNED, randomUnassignedInfo("foobar"))); Collections.shuffle(shardRoutings, random()); for (ShardRouting routing : shardRoutings) { shards.add(routing); @@ -165,8 +165,7 @@ public void testPriorityComparatorSort() { for (int i = 0; i < numShards; i++) { IndexMetadata indexMeta = randomFrom(indices); shards.add(TestShardRouting.newShardRouting(indexMeta.getIndex().getName(), randomIntBetween(1, 5), null, null, - randomBoolean(), ShardRoutingState.UNASSIGNED, new UnassignedInfo(randomFrom(UnassignedInfo.Reason.values()), - "foobar"))); + randomBoolean(), ShardRoutingState.UNASSIGNED, randomUnassignedInfo("foobar"))); } shards.sort(new PriorityComparator() { @Override diff --git a/test/framework/src/main/java/org/elasticsearch/cluster/routing/TestShardRouting.java b/test/framework/src/main/java/org/elasticsearch/cluster/routing/TestShardRouting.java index 14e1ac5fb86d3..79e4cb4f76f8a 100644 --- a/test/framework/src/main/java/org/elasticsearch/cluster/routing/TestShardRouting.java +++ b/test/framework/src/main/java/org/elasticsearch/cluster/routing/TestShardRouting.java @@ -15,10 +15,13 @@ import org.elasticsearch.repositories.IndexId; import org.elasticsearch.snapshots.Snapshot; import org.elasticsearch.snapshots.SnapshotId; -import org.elasticsearch.test.ESTestCase; + +import java.util.Collections; import static org.apache.lucene.util.LuceneTestCase.random; import static org.elasticsearch.test.ESTestCase.randomAlphaOfLength; +import static org.elasticsearch.test.ESTestCase.randomBoolean; +import static org.elasticsearch.test.ESTestCase.randomFrom; /** * A helper that allows to create shard routing instances within tests, while not requiring to expose @@ -88,7 +91,7 @@ private static RecoverySource buildRecoveryTarget(boolean primary, ShardRoutingS case UNASSIGNED: case INITIALIZING: if (primary) { - return ESTestCase.randomFrom(RecoverySource.EmptyStoreRecoverySource.INSTANCE, + return randomFrom(RecoverySource.EmptyStoreRecoverySource.INSTANCE, RecoverySource.ExistingStoreRecoverySource.INSTANCE); } else { return RecoverySource.PeerRecoverySource.INSTANCE; @@ -120,7 +123,7 @@ private static UnassignedInfo buildUnassignedInfo(ShardRoutingState state) { switch (state) { case UNASSIGNED: case INITIALIZING: - return new UnassignedInfo(ESTestCase.randomFrom(UnassignedInfo.Reason.values()), "auto generated for test"); + return randomUnassignedInfo("auto generated for test"); case STARTED: case RELOCATING: return null; @@ -129,8 +132,33 @@ private static UnassignedInfo buildUnassignedInfo(ShardRoutingState state) { } } + public static UnassignedInfo randomUnassignedInfo(String message) { + UnassignedInfo.Reason reason = randomFrom(UnassignedInfo.Reason.values()); + String lastAllocatedNodeId = null; + boolean delayed = false; + if (reason == UnassignedInfo.Reason.NODE_LEFT || reason == UnassignedInfo.Reason.NODE_RESTARTING) { + if (randomBoolean()) { + delayed = true; + } + lastAllocatedNodeId = randomAlphaOfLength(10); + } + int failedAllocations = reason == UnassignedInfo.Reason.ALLOCATION_FAILED ? 1 : 0; + return new UnassignedInfo( + reason, + message, + null, + failedAllocations, + System.nanoTime(), + System.currentTimeMillis(), + delayed, + UnassignedInfo.AllocationStatus.NO_ATTEMPT, + Collections.emptySet(), + lastAllocatedNodeId + ); + } + public static RecoverySource randomRecoverySource() { - return ESTestCase.randomFrom(RecoverySource.EmptyStoreRecoverySource.INSTANCE, + return randomFrom(RecoverySource.EmptyStoreRecoverySource.INSTANCE, RecoverySource.ExistingStoreRecoverySource.INSTANCE, RecoverySource.PeerRecoverySource.INSTANCE, RecoverySource.LocalShardsRecoverySource.INSTANCE, diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ilm/CheckShrinkReadyStepTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ilm/CheckShrinkReadyStepTests.java index 1c27a4f9ea0dd..00fb39ac1b682 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ilm/CheckShrinkReadyStepTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ilm/CheckShrinkReadyStepTests.java @@ -244,7 +244,7 @@ public void testExecuteAllocateReplicaUnassigned() { IndexRoutingTable.Builder indexRoutingTable = IndexRoutingTable.builder(index) .addShard(TestShardRouting.newShardRouting(new ShardId(index, 0), "node1", true, ShardRoutingState.STARTED)) .addShard(TestShardRouting.newShardRouting(new ShardId(index, 0), null, null, false, ShardRoutingState.UNASSIGNED, - new UnassignedInfo(randomFrom(UnassignedInfo.Reason.values()), "the shard is intentionally unassigned"))); + randomUnassignedInfo("the shard is intentionally unassigned"))); CheckShrinkReadyStep step = createRandomInstance(); assertAllocateStatus(index, 1, 1, step, existingSettings, node1Settings, node2Settings, indexRoutingTable, @@ -458,4 +458,29 @@ private void assertAllocateStatus(Index index, int shards, int replicas, CheckSh assertEquals(expectedResult.isComplete(), actualResult.isComplete()); assertEquals(expectedResult.getInfomationContext(), actualResult.getInfomationContext()); } + + public static UnassignedInfo randomUnassignedInfo(String message) { + UnassignedInfo.Reason reason = randomFrom(UnassignedInfo.Reason.values()); + String lastAllocatedNodeId = null; + boolean delayed = false; + if (reason == UnassignedInfo.Reason.NODE_LEFT || reason == UnassignedInfo.Reason.NODE_RESTARTING) { + if (randomBoolean()) { + delayed = true; + } + lastAllocatedNodeId = randomAlphaOfLength(10); + } + int failedAllocations = reason == UnassignedInfo.Reason.ALLOCATION_FAILED ? 1 : 0; + return new UnassignedInfo( + reason, + message, + null, + failedAllocations, + System.nanoTime(), + System.currentTimeMillis(), + delayed, + UnassignedInfo.AllocationStatus.NO_ATTEMPT, + Collections.emptySet(), + lastAllocatedNodeId + ); + } } diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ilm/DataTierMigrationRoutedStepTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ilm/DataTierMigrationRoutedStepTests.java index 6c321a990c219..6f8e4fe86c64f 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ilm/DataTierMigrationRoutedStepTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ilm/DataTierMigrationRoutedStepTests.java @@ -32,6 +32,7 @@ import static java.util.Collections.emptyMap; import static org.elasticsearch.xpack.cluster.routing.allocation.DataTierAllocationDecider.INDEX_ROUTING_PREFER; +import static org.elasticsearch.xpack.core.ilm.CheckShrinkReadyStepTests.randomUnassignedInfo; import static org.elasticsearch.xpack.core.ilm.step.info.AllocationInfo.waitingForActiveShardsAllocationInfo; import static org.hamcrest.Matchers.is; import static org.hamcrest.Matchers.nullValue; @@ -77,7 +78,7 @@ public void testExecuteWithUnassignedShard() { IndexRoutingTable.Builder indexRoutingTable = IndexRoutingTable.builder(index) .addShard(TestShardRouting.newShardRouting(new ShardId(index, 0), "node1", true, ShardRoutingState.STARTED)) .addShard(TestShardRouting.newShardRouting(new ShardId(index, 1), null, null, true, ShardRoutingState.UNASSIGNED, - new UnassignedInfo(randomFrom(Reason.values()), "the shard is intentionally unassigned"))); + randomUnassignedInfo("the shard is intentionally unassigned"))); ClusterState clusterState = ClusterState.builder(ClusterState.EMPTY_STATE).metadata(Metadata.builder().put(indexMetadata, true).build()) diff --git a/x-pack/plugin/monitoring/src/test/java/org/elasticsearch/xpack/monitoring/collector/indices/IndexStatsMonitoringDocTests.java b/x-pack/plugin/monitoring/src/test/java/org/elasticsearch/xpack/monitoring/collector/indices/IndexStatsMonitoringDocTests.java index 199fb3f3adc96..2320a154c2a95 100644 --- a/x-pack/plugin/monitoring/src/test/java/org/elasticsearch/xpack/monitoring/collector/indices/IndexStatsMonitoringDocTests.java +++ b/x-pack/plugin/monitoring/src/test/java/org/elasticsearch/xpack/monitoring/collector/indices/IndexStatsMonitoringDocTests.java @@ -47,6 +47,7 @@ import static org.elasticsearch.common.xcontent.XContentHelper.convertToJson; import static org.elasticsearch.common.xcontent.XContentHelper.stripWhitespace; +import static org.elasticsearch.xpack.core.ilm.CheckShrinkReadyStepTests.randomUnassignedInfo; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.greaterThanOrEqualTo; import static org.hamcrest.Matchers.is; @@ -424,8 +425,7 @@ private static IndexRoutingTable mockIndexRoutingTable(final Index index, --unassignedTotal; --unassignedPrimaries; - final UnassignedInfo unassignedInfo = - new UnassignedInfo(randomFrom(UnassignedInfo.Reason.values()), randomAlphaOfLength(3)); + final UnassignedInfo unassignedInfo = randomUnassignedInfo(randomAlphaOfLength(3)); final String nodeId; final ShardRoutingState state; From b090715262e0072d2ccb3fdc11cc5a2414fdc6be Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Mon, 16 Aug 2021 14:39:05 -0600 Subject: [PATCH 47/47] Alter test to generated `lastAllocatedNodeId` if necessary --- .../elasticsearch/gateway/ReplicaShardAllocatorTests.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/server/src/test/java/org/elasticsearch/gateway/ReplicaShardAllocatorTests.java b/server/src/test/java/org/elasticsearch/gateway/ReplicaShardAllocatorTests.java index 249fccbe69372..75369c0c89fd2 100644 --- a/server/src/test/java/org/elasticsearch/gateway/ReplicaShardAllocatorTests.java +++ b/server/src/test/java/org/elasticsearch/gateway/ReplicaShardAllocatorTests.java @@ -466,6 +466,9 @@ private RoutingAllocation onePrimaryOnNode1And1Replica(AllocationDeciders decide boolean delayed = reason == UnassignedInfo.Reason.NODE_LEFT && UnassignedInfo.INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING.get(settings).nanos() > 0; int failedAllocations = reason == UnassignedInfo.Reason.ALLOCATION_FAILED ? 1 : 0; + final String lastAllocatedNodeId = reason == UnassignedInfo.Reason.NODE_RESTARTING || randomBoolean() + ? randomAlphaOfLength(10) + : null; RoutingTable routingTable = RoutingTable.builder() .add(IndexRoutingTable.builder(shardId.getIndex()) .addIndexShard(new IndexShardRoutingTable.Builder(shardId) @@ -474,7 +477,7 @@ private RoutingAllocation onePrimaryOnNode1And1Replica(AllocationDeciders decide RecoverySource.PeerRecoverySource.INSTANCE, new UnassignedInfo(reason, null, null, failedAllocations, System.nanoTime(), System.currentTimeMillis(), delayed, UnassignedInfo.AllocationStatus.NO_ATTEMPT, - Collections.emptySet(), null))) + Collections.emptySet(), lastAllocatedNodeId))) .build()) ) .build();