@@ -509,7 +509,7 @@ private static boolean assertShardStateConsistent(
509509 int shardId ,
510510 ShardSnapshotStatus shardSnapshotStatus
511511 ) {
512- if (shardSnapshotStatus .isActiveOrQueuedWithGeneration ()) {
512+ if (shardSnapshotStatus .isActiveOrAssignedQueued ()) {
513513 Tuple <String , Integer > plainShardId = Tuple .tuple (indexName , shardId );
514514 assert assignedShards .add (plainShardId ) : plainShardId + " is assigned twice in " + entries ;
515515 assert queuedShards .contains (plainShardId ) == false : plainShardId + " is queued then assigned in " + entries ;
@@ -758,12 +758,13 @@ public static ShardSnapshotStatus success(String nodeId, ShardSnapshotResult sha
758758 }
759759
760760 @ SuppressForbidden (reason = "using a private constructor within the same file" )
761- public static ShardSnapshotStatus queuedWithGeneration ( ShardGeneration generation ) {
762- return new ShardSnapshotStatus (null , ShardState .QUEUED , generation , null , null );
761+ public static ShardSnapshotStatus assignedQueued ( String nodeId , ShardGeneration generation ) {
762+ return new ShardSnapshotStatus (nodeId , ShardState .QUEUED , generation , null , null );
763763 }
764764
765- public boolean isQueuedWithGeneration () {
766- return state == ShardState .QUEUED && generation != null && nodeId == null ;
765+ public boolean isAssignedQueued () {
766+ // generation can still be null if previous shard snapshots all failed
767+ return state == ShardState .QUEUED && nodeId != null ;
767768 }
768769
769770 public boolean isUnassignedQueued () {
@@ -790,7 +791,7 @@ private boolean assertConsistent() {
790791 assert state .failed () == false || reason != null ;
791792 assert (state != ShardState .INIT && state != ShardState .WAITING && state != ShardState .PAUSED_FOR_NODE_REMOVAL )
792793 || nodeId != null : "Null node id for state [" + state + "]" ;
793- assert state != ShardState .QUEUED || (isUnassignedQueued () || isQueuedWithGeneration ())
794+ assert state != ShardState .QUEUED || (isUnassignedQueued () || isAssignedQueued ())
794795 : "Found unexpected shard state=["
795796 + state
796797 + "], nodeId=["
@@ -853,8 +854,8 @@ public boolean isActive() {
853854 };
854855 }
855856
856- public boolean isActiveOrQueuedWithGeneration () {
857- return isActive () || isQueuedWithGeneration ();
857+ public boolean isActiveOrAssignedQueued () {
858+ return isActive () || isAssignedQueued ();
858859 }
859860
860861 @ Override
@@ -1230,30 +1231,38 @@ public Entry withClones(Map<RepositoryShardId, ShardSnapshotStatus> updatedClone
12301231 * @return aborted snapshot entry or {@code null} if entry can be removed from the cluster state directly
12311232 */
12321233 @ Nullable
1233- public Entry abort (BiConsumer <ShardId , ShardSnapshotStatus > abortedQueuedWithGenerationShardConsumer ) {
1234+ public Entry abort (BiConsumer <ShardId , ShardSnapshotStatus > abortedAssignedQueuedShardConsumer ) {
12341235 final Map <ShardId , ShardSnapshotStatus > shardsBuilder = new HashMap <>();
12351236 boolean completed = true ;
12361237 boolean allQueued = true ;
12371238 for (Map .Entry <ShardId , ShardSnapshotStatus > shardEntry : shards .entrySet ()) {
12381239 ShardSnapshotStatus status = shardEntry .getValue ();
1239- final var isQueuedWithGeneration = status .isQueuedWithGeneration ();
1240- allQueued &= (status .state () == ShardState .QUEUED && isQueuedWithGeneration == false );
1240+ final var isAssignedQueued = status .isAssignedQueued ();
1241+ allQueued &= (status .state () == ShardState .QUEUED && isAssignedQueued == false );
12411242 if (status .state ().completed () == false ) {
12421243 final String nodeId = status .nodeId ();
1243- status = new ShardSnapshotStatus (
1244- nodeId ,
1245- // QUEUED with generation transitioned to ABORTED (incomplete) and is completed by a separate cluster state update
1246- (nodeId == null && isQueuedWithGeneration == false ) ? ShardState .FAILED : ShardState .ABORTED ,
1247- status .generation (),
1248- "aborted by snapshot deletion"
1249- );
1250- if (isQueuedWithGeneration ) {
1251- // Accumulate the updates needed to complete the aborted QUEUED with generation shard snapshots
1244+ if (isAssignedQueued == false ) {
1245+ status = new ShardSnapshotStatus (
1246+ nodeId ,
1247+ nodeId == null ? ShardState .FAILED : ShardState .ABORTED ,
1248+ status .generation (),
1249+ "aborted by snapshot deletion"
1250+ );
1251+ } else {
12521252 assert isClone () == false
12531253 : "The state queued with generation should not be possible for a clone entry [" + this + "]" ;
1254- abortedQueuedWithGenerationShardConsumer .accept (
1254+ final String reason = "assigned-queued aborted by snapshot deletion" ;
1255+ status = new ShardSnapshotStatus (
1256+ nodeId ,
1257+ // Assigned QUEUED transitions to ABORTED (incomplete) and is completed by a separate cluster state update
1258+ ShardState .ABORTED ,
1259+ status .generation (),
1260+ reason
1261+ );
1262+ // Accumulate the updates needed to complete the aborted QUEUED with generation shard snapshots
1263+ abortedAssignedQueuedShardConsumer .accept (
12551264 shardEntry .getKey (),
1256- new ShardSnapshotStatus (null , ShardState .FAILED , status .generation , "aborted by snapshot deletion" )
1265+ new ShardSnapshotStatus (null , ShardState .FAILED , status .generation , reason )
12571266 );
12581267 }
12591268 }
0 commit comments