Просмотр исходного кода

Add shortcuts for getting snapshot customs from cluster state (#98906)

Drying up the logic for looking up the customs and falling back to the empty default. Hopefully, this improves readability somewhat by taking away that complexity and some null checks.
I like it at least, saves a lot of clutter.

Re-revert of #98896, only fixing a single null check in tests.
Armin Braun 2 лет назад
Родитель
Сommit
4d29cb20b8
50 измененных файлов с 209 добавлено и 357 удалено
  1. 3 4
      qa/smoke-test-http/src/javaRestTest/java/org/elasticsearch/http/snapshots/RestGetSnapshotsIT.java
  2. 1 2
      server/src/internalClusterTest/java/org/elasticsearch/discovery/SnapshotDisruptionIT.java
  3. 3 9
      server/src/internalClusterTest/java/org/elasticsearch/repositories/blobstore/BlobStoreRepositoryCleanupIT.java
  4. 2 6
      server/src/internalClusterTest/java/org/elasticsearch/snapshots/AbortedSnapshotIT.java
  5. 3 9
      server/src/internalClusterTest/java/org/elasticsearch/snapshots/CloneSnapshotIT.java
  6. 10 13
      server/src/internalClusterTest/java/org/elasticsearch/snapshots/ConcurrentSnapshotsIT.java
  7. 1 2
      server/src/internalClusterTest/java/org/elasticsearch/snapshots/CorruptedBlobStoreRepositoryIT.java
  8. 1 1
      server/src/internalClusterTest/java/org/elasticsearch/snapshots/DedicatedClusterSnapshotRestoreIT.java
  9. 1 1
      server/src/internalClusterTest/java/org/elasticsearch/snapshots/GetSnapshotsIT.java
  10. 1 1
      server/src/internalClusterTest/java/org/elasticsearch/snapshots/SharedClusterSnapshotRestoreIT.java
  11. 2 4
      server/src/internalClusterTest/java/org/elasticsearch/snapshots/SnapshotStatusApisIT.java
  12. 5 12
      server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/cleanup/TransportCleanupRepositoryAction.java
  13. 1 1
      server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/get/TransportGetRepositoriesAction.java
  14. 1 1
      server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/get/TransportGetSnapshotsAction.java
  15. 1 4
      server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/get/shard/TransportGetShardSnapshotAction.java
  16. 1 1
      server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/status/TransportSnapshotsStatusAction.java
  17. 4 4
      server/src/main/java/org/elasticsearch/cluster/ClusterSnapshotStats.java
  18. 4 0
      server/src/main/java/org/elasticsearch/cluster/RepositoryCleanupInProgress.java
  19. 4 0
      server/src/main/java/org/elasticsearch/cluster/RestoreInProgress.java
  20. 4 0
      server/src/main/java/org/elasticsearch/cluster/SnapshotDeletionsInProgress.java
  21. 4 0
      server/src/main/java/org/elasticsearch/cluster/SnapshotsInProgress.java
  22. 1 1
      server/src/main/java/org/elasticsearch/cluster/metadata/MetadataDeleteIndexService.java
  23. 5 0
      server/src/main/java/org/elasticsearch/cluster/metadata/RepositoriesMetadata.java
  24. 8 10
      server/src/main/java/org/elasticsearch/cluster/routing/allocation/AllocationService.java
  25. 7 11
      server/src/main/java/org/elasticsearch/cluster/routing/allocation/decider/RestoreInProgressAllocationDecider.java
  26. 2 2
      server/src/main/java/org/elasticsearch/cluster/routing/allocation/decider/SnapshotInProgressAllocationDecider.java
  27. 2 5
      server/src/main/java/org/elasticsearch/indices/recovery/plan/ShardSnapshotsService.java
  28. 1 2
      server/src/main/java/org/elasticsearch/repositories/FinalizeSnapshotContext.java
  29. 10 25
      server/src/main/java/org/elasticsearch/repositories/RepositoriesService.java
  30. 12 23
      server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java
  31. 1 1
      server/src/main/java/org/elasticsearch/snapshots/RepositoryIntegrityHealthIndicatorService.java
  32. 6 9
      server/src/main/java/org/elasticsearch/snapshots/RestoreService.java
  33. 2 3
      server/src/main/java/org/elasticsearch/snapshots/SnapshotShardsService.java
  34. 46 97
      server/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java
  35. 1 3
      server/src/test/java/org/elasticsearch/snapshots/InternalSnapshotsInfoServiceTests.java
  36. 8 18
      server/src/test/java/org/elasticsearch/snapshots/SnapshotResiliencyTests.java
  37. 1 2
      server/src/test/java/org/elasticsearch/snapshots/SnapshotsServiceTests.java
  38. 6 12
      test/framework/src/main/java/org/elasticsearch/snapshots/AbstractSnapshotIntegTestCase.java
  39. 2 6
      x-pack/plugin/ccr/src/internalClusterTest/java/org/elasticsearch/xpack/ccr/CcrRepositoryIT.java
  40. 1 2
      x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ilm/AsyncRetryDuringSnapshotActionStep.java
  41. 1 1
      x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ilm/GenerateSnapshotNameStep.java
  42. 1 1
      x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/ilm/action/TransportPutLifecycleAction.java
  43. 3 6
      x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsIntegTests.java
  44. 4 12
      x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/allocation/SearchableSnapshotAllocator.java
  45. 2 2
      x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/allocation/decider/SearchableSnapshotRepositoryExistsAllocationDecider.java
  46. 1 2
      x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/cache/blob/BlobStoreCacheMaintenanceService.java
  47. 3 4
      x-pack/plugin/security/src/internalClusterTest/java/org/elasticsearch/integration/ClusterPrivilegeIntegrationTests.java
  48. 3 4
      x-pack/plugin/security/src/internalClusterTest/java/org/elasticsearch/integration/SecurityFeatureStateIntegTests.java
  49. 1 1
      x-pack/plugin/slm/src/main/java/org/elasticsearch/xpack/slm/SnapshotLifecycleService.java
  50. 11 17
      x-pack/plugin/slm/src/main/java/org/elasticsearch/xpack/slm/action/TransportGetSnapshotLifecycleAction.java

+ 3 - 4
qa/smoke-test-http/src/javaRestTest/java/org/elasticsearch/http/snapshots/RestGetSnapshotsIT.java

@@ -201,15 +201,14 @@ public class RestGetSnapshotsIT extends AbstractSnapshotRestTestCase {
         }
         AbstractSnapshotIntegTestCase.awaitNumberOfSnapshotsInProgress(logger, inProgressCount);
         AbstractSnapshotIntegTestCase.awaitClusterState(logger, state -> {
-            boolean firstIndexSuccessfullySnapshot = state.custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY)
-                .asStream()
+            final var snapshotsInProgress = SnapshotsInProgress.get(state);
+            boolean firstIndexSuccessfullySnapshot = snapshotsInProgress.asStream()
                 .flatMap(s -> s.shards().entrySet().stream())
                 .allMatch(
                     e -> e.getKey().getIndexName().equals("test-index-1") == false
                         || e.getValue().state() == SnapshotsInProgress.ShardState.SUCCESS
                 );
-            boolean secondIndexIsBlocked = state.custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY)
-                .asStream()
+            boolean secondIndexIsBlocked = snapshotsInProgress.asStream()
                 .flatMap(s -> s.shards().entrySet().stream())
                 .filter(e -> e.getKey().getIndexName().equals("test-index-2"))
                 .map(e -> e.getValue().state())

+ 1 - 2
server/src/internalClusterTest/java/org/elasticsearch/discovery/SnapshotDisruptionIT.java

@@ -87,8 +87,7 @@ public class SnapshotDisruptionIT extends AbstractSnapshotIntegTestCase {
                 if (snapshots != null && snapshots.isEmpty() == false) {
                     final SnapshotsInProgress.Entry snapshotEntry = snapshots.forRepo(repoName).get(0);
                     if (snapshotEntry.state() == SnapshotsInProgress.State.SUCCESS) {
-                        final RepositoriesMetadata repoMeta = event.state().metadata().custom(RepositoriesMetadata.TYPE);
-                        final RepositoryMetadata metadata = repoMeta.repository(repoName);
+                        final RepositoryMetadata metadata = RepositoriesMetadata.get(event.state()).repository(repoName);
                         if (metadata.pendingGeneration() > snapshotEntry.repositoryStateId()) {
                             logger.info("--> starting disruption");
                             networkDisruption.startDisrupting();

+ 3 - 9
server/src/internalClusterTest/java/org/elasticsearch/repositories/blobstore/BlobStoreRepositoryCleanupIT.java

@@ -41,9 +41,7 @@ public class BlobStoreRepositoryCleanupIT extends AbstractSnapshotIntegTestCase
         ensureStableCluster(nodeCount - 1);
 
         logger.info("-->  wait for cleanup to finish and disappear from cluster state");
-        awaitClusterState(
-            state -> state.custom(RepositoryCleanupInProgress.TYPE, RepositoryCleanupInProgress.EMPTY).hasCleanupInProgress() == false
-        );
+        awaitClusterState(state -> RepositoryCleanupInProgress.get(state).hasCleanupInProgress() == false);
 
         try {
             cleanupFuture.get();
@@ -70,9 +68,7 @@ public class BlobStoreRepositoryCleanupIT extends AbstractSnapshotIntegTestCase
         unblockNode("test-repo", internalCluster().getMasterName());
 
         logger.info("-->  wait for cleanup to finish and disappear from cluster state");
-        awaitClusterState(
-            state -> state.custom(RepositoryCleanupInProgress.TYPE, RepositoryCleanupInProgress.EMPTY).hasCleanupInProgress() == false
-        );
+        awaitClusterState(state -> RepositoryCleanupInProgress.get(state).hasCleanupInProgress() == false);
 
         final ExecutionException e = expectThrows(ExecutionException.class, cleanupFuture::get);
         final Throwable ioe = ExceptionsHelper.unwrap(e, IOException.class);
@@ -119,9 +115,7 @@ public class BlobStoreRepositoryCleanupIT extends AbstractSnapshotIntegTestCase
 
         final String masterNode = internalCluster().getMasterName();
         waitForBlock(masterNode, repoName);
-        awaitClusterState(
-            state -> state.custom(RepositoryCleanupInProgress.TYPE, RepositoryCleanupInProgress.EMPTY).hasCleanupInProgress()
-        );
+        awaitClusterState(state -> RepositoryCleanupInProgress.get(state).hasCleanupInProgress());
         return future;
     }
 

+ 2 - 6
server/src/internalClusterTest/java/org/elasticsearch/snapshots/AbortedSnapshotIT.java

@@ -63,11 +63,7 @@ public class AbortedSnapshotIT extends AbstractSnapshotIntegTestCase {
         clusterAdmin().prepareCreateSnapshot(repoName, "snapshot-1").setWaitForCompletion(false).setPartial(true).get();
         // resulting cluster state has been applied on all nodes, which means the first task for the SNAPSHOT pool is queued up
 
-        final var snapshot = clusterService.state()
-            .custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY)
-            .forRepo(repoName)
-            .get(0)
-            .snapshot();
+        final var snapshot = SnapshotsInProgress.get(clusterService.state()).forRepo(repoName).get(0).snapshot();
         final var snapshotShardsService = internalCluster().getInstance(SnapshotShardsService.class, dataNode);
 
         // Run up to 3 snapshot tasks, which are (in order):
@@ -105,7 +101,7 @@ public class AbortedSnapshotIT extends AbstractSnapshotIntegTestCase {
         stopBlocking.set(true);
         safeAwait(barrier); // release snapshot thread
 
-        assertBusy(() -> assertTrue(clusterService.state().custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY).isEmpty()));
+        assertBusy(() -> assertTrue(SnapshotsInProgress.get(clusterService.state()).isEmpty()));
     }
 
 }

+ 3 - 9
server/src/internalClusterTest/java/org/elasticsearch/snapshots/CloneSnapshotIT.java

@@ -642,8 +642,7 @@ public class CloneSnapshotIT extends AbstractSnapshotIntegTestCase {
         logger.info("--> wait for clone to start fully with shards assigned in the cluster state");
         try {
             awaitClusterState(clusterState -> {
-                final List<SnapshotsInProgress.Entry> entries = clusterState.custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY)
-                    .forRepo(repoName);
+                final List<SnapshotsInProgress.Entry> entries = SnapshotsInProgress.get(clusterState).forRepo(repoName);
                 return entries.size() == 2 && entries.get(1).shardsByRepoShardId().isEmpty() == false;
             });
             assertFalse(blockedSnapshot.isDone());
@@ -680,7 +679,7 @@ public class CloneSnapshotIT extends AbstractSnapshotIntegTestCase {
         final ActionFuture<AcknowledgedResponse> cloneFuture = startClone(repoName, sourceSnapshot, "target-snapshot", indexName);
         logger.info("--> waiting for snapshot clone to be fully initialized");
         awaitClusterState(state -> {
-            for (SnapshotsInProgress.Entry entry : state.custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY).forRepo(repoName)) {
+            for (SnapshotsInProgress.Entry entry : SnapshotsInProgress.get(state).forRepo(repoName)) {
                 if (entry.shardsByRepoShardId().isEmpty() == false) {
                     assertEquals(sourceSnapshot, entry.source().getName());
                     for (SnapshotsInProgress.ShardSnapshotStatus value : entry.shardsByRepoShardId().values()) {
@@ -721,12 +720,7 @@ public class CloneSnapshotIT extends AbstractSnapshotIntegTestCase {
         final ActionFuture<AcknowledgedResponse> clone2 = startClone(repoName, sourceSnapshot, "target-snapshot-2", testIndex);
 
         awaitNumberOfSnapshotsInProgress(2);
-        awaitClusterState(
-            state -> state.custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY)
-                .forRepo(repoName)
-                .stream()
-                .anyMatch(entry -> entry.state().completed())
-        );
+        awaitClusterState(state -> SnapshotsInProgress.get(state).forRepo(repoName).stream().anyMatch(entry -> entry.state().completed()));
         repo.unblock();
 
         assertAcked(clone1.get());

+ 10 - 13
server/src/internalClusterTest/java/org/elasticsearch/snapshots/ConcurrentSnapshotsIT.java

@@ -366,7 +366,7 @@ public class ConcurrentSnapshotsIT extends AbstractSnapshotIntegTestCase {
 
         logger.info("--> wait for snapshot on second data node to finish");
         awaitClusterState(state -> {
-            final SnapshotsInProgress snapshotsInProgress = state.custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY);
+            final SnapshotsInProgress snapshotsInProgress = SnapshotsInProgress.get(state);
             return snapshotsInProgress.count() == 2 && snapshotHasCompletedShard(repoName, secondSnapshot, snapshotsInProgress);
         });
 
@@ -422,7 +422,7 @@ public class ConcurrentSnapshotsIT extends AbstractSnapshotIntegTestCase {
 
         logger.info("--> wait for snapshot on second data node to finish");
         awaitClusterState(state -> {
-            final SnapshotsInProgress snapshotsInProgress = state.custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY);
+            final SnapshotsInProgress snapshotsInProgress = SnapshotsInProgress.get(state);
             return snapshotsInProgress.count() == 2 && snapshotHasCompletedShard(repoName, secondSnapshot, snapshotsInProgress);
         });
 
@@ -442,8 +442,10 @@ public class ConcurrentSnapshotsIT extends AbstractSnapshotIntegTestCase {
         logger.info("--> waiting for second and third snapshot to finish");
         assertBusy(() -> {
             assertThat(currentSnapshots(repoName), hasSize(1));
-            final SnapshotsInProgress snapshotsInProgress = clusterService().state().custom(SnapshotsInProgress.TYPE);
-            assertThat(snapshotsInProgress.forRepo(repoName).get(0).state(), is(SnapshotsInProgress.State.ABORTED));
+            assertThat(
+                SnapshotsInProgress.get(clusterService().state()).forRepo(repoName).get(0).state(),
+                is(SnapshotsInProgress.State.ABORTED)
+            );
         }, 30L, TimeUnit.SECONDS);
 
         unblockNode(repoName, dataNode);
@@ -485,7 +487,7 @@ public class ConcurrentSnapshotsIT extends AbstractSnapshotIntegTestCase {
 
         logger.info("--> wait for snapshot on second data node to finish");
         awaitClusterState(state -> {
-            final SnapshotsInProgress snapshotsInProgress = state.custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY);
+            final SnapshotsInProgress snapshotsInProgress = SnapshotsInProgress.get(state);
             return snapshotsInProgress.count() == 2 && snapshotHasCompletedShard(repoName, secondSnapshot, snapshotsInProgress);
         });
 
@@ -512,9 +514,7 @@ public class ConcurrentSnapshotsIT extends AbstractSnapshotIntegTestCase {
         logger.info("--> waiting for second snapshot to finish and the other two snapshots to become aborted");
         assertBusy(() -> {
             assertThat(currentSnapshots(repoName), hasSize(2));
-            for (SnapshotsInProgress.Entry entry : clusterService().state()
-                .custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY)
-                .forRepo(repoName)) {
+            for (SnapshotsInProgress.Entry entry : SnapshotsInProgress.get(clusterService().state()).forRepo(repoName)) {
                 assertThat(entry.state(), is(SnapshotsInProgress.State.ABORTED));
                 assertThat(entry.snapshot().getSnapshotId().getName(), not(secondSnapshot));
             }
@@ -1583,15 +1583,12 @@ public class ConcurrentSnapshotsIT extends AbstractSnapshotIntegTestCase {
             .execute();
 
         awaitClusterState(state -> {
-            final List<SnapshotsInProgress.Entry> snapshotsInProgress = state.custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY)
-                .forRepo(repository);
+            final List<SnapshotsInProgress.Entry> snapshotsInProgress = SnapshotsInProgress.get(state).forRepo(repository);
             return snapshotsInProgress.size() == 2 && snapshotsInProgress.get(1).state().completed();
         });
 
         unblockAllDataNodes(repository);
-        awaitClusterState(
-            state -> state.custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY).forRepo(repository).get(0).state().completed()
-        );
+        awaitClusterState(state -> SnapshotsInProgress.get(state).forRepo(repository).get(0).state().completed());
 
         unblockNode(repository, master);
         assertSuccessful(snapshot2);

+ 1 - 2
server/src/internalClusterTest/java/org/elasticsearch/snapshots/CorruptedBlobStoreRepositoryIT.java

@@ -219,8 +219,7 @@ public class CorruptedBlobStoreRepositoryIT extends AbstractSnapshotIntegTestCas
                     Metadata.builder(currentState.getMetadata())
                         .putCustom(
                             RepositoriesMetadata.TYPE,
-                            currentState.metadata()
-                                .<RepositoriesMetadata>custom(RepositoriesMetadata.TYPE)
+                            RepositoriesMetadata.get(currentState)
                                 .withUpdatedGeneration(repository.getMetadata().name(), beforeMoveGen, beforeMoveGen + 1)
                         )
                         .build()

+ 1 - 1
server/src/internalClusterTest/java/org/elasticsearch/snapshots/DedicatedClusterSnapshotRestoreIT.java

@@ -1056,7 +1056,7 @@ public class DedicatedClusterSnapshotRestoreIT extends AbstractSnapshotIntegTest
         awaitClusterState(
             logger,
             otherDataNode,
-            state -> state.custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY)
+            state -> SnapshotsInProgress.get(state)
                 .forRepo(repoName)
                 .stream()
                 .anyMatch(entry -> entry.state() == SnapshotsInProgress.State.ABORTED)

+ 1 - 1
server/src/internalClusterTest/java/org/elasticsearch/snapshots/GetSnapshotsIT.java

@@ -182,7 +182,7 @@ public class GetSnapshotsIT extends AbstractSnapshotIntegTestCase {
         }
         awaitNumberOfSnapshotsInProgress(inProgressCount);
         awaitClusterState(
-            state -> state.custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY)
+            state -> SnapshotsInProgress.get(state)
                 .asStream()
                 .flatMap(s -> s.shards().entrySet().stream())
                 .allMatch(

+ 1 - 1
server/src/internalClusterTest/java/org/elasticsearch/snapshots/SharedClusterSnapshotRestoreIT.java

@@ -1102,7 +1102,7 @@ public class SharedClusterSnapshotRestoreIT extends AbstractSnapshotIntegTestCas
         waitForBlock(blockedNode, "test-repo");
 
         awaitClusterState(state -> {
-            SnapshotsInProgress snapshotsInProgress = state.custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY);
+            SnapshotsInProgress snapshotsInProgress = SnapshotsInProgress.get(state);
             Set<Snapshot> snapshots = snapshotsInProgress.asStream().map(SnapshotsInProgress.Entry::snapshot).collect(Collectors.toSet());
             if (snapshots.size() != 1) {
                 return false;

+ 2 - 4
server/src/internalClusterTest/java/org/elasticsearch/snapshots/SnapshotStatusApisIT.java

@@ -422,7 +422,7 @@ public class SnapshotStatusApisIT extends AbstractSnapshotIntegTestCase {
 
         logger.info("--> wait for snapshots to get to a consistent state");
         awaitClusterState(state -> {
-            SnapshotsInProgress snapshotsInProgress = state.custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY);
+            SnapshotsInProgress snapshotsInProgress = SnapshotsInProgress.get(state);
             Set<Snapshot> snapshots = snapshotsInProgress.asStream().map(SnapshotsInProgress.Entry::snapshot).collect(Collectors.toSet());
             if (snapshots.size() != 1) {
                 return false;
@@ -670,9 +670,7 @@ public class SnapshotStatusApisIT extends AbstractSnapshotIntegTestCase {
         // ... and then some more status requests until all snapshots are done
         var masterClusterService = internalCluster().getCurrentMasterNodeInstance(ClusterService.class);
         assertBusy(() -> {
-            final var stillRunning = masterClusterService.state()
-                .custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY)
-                .isEmpty() == false;
+            final var stillRunning = SnapshotsInProgress.get(masterClusterService.state()).isEmpty() == false;
             statuses.add(dataNodeClient.admin().cluster().prepareSnapshotStatus(repoName).setSnapshots(snapshotNames).execute());
             gets.add(dataNodeClient.admin().cluster().prepareGetSnapshots(repoName).setSnapshots(snapshotNames).execute());
             assertFalse(stillRunning);

+ 5 - 12
server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/cleanup/TransportCleanupRepositoryAction.java

@@ -94,8 +94,7 @@ public final class TransportCleanupRepositoryAction extends TransportMasterNodeA
     private static void addClusterStateApplier(ClusterService clusterService) {
         clusterService.addStateApplier(event -> {
             if (event.localNodeMaster() && event.previousState().nodes().isLocalNodeElectedMaster() == false) {
-                final RepositoryCleanupInProgress repositoryCleanupInProgress = event.state()
-                    .custom(RepositoryCleanupInProgress.TYPE, RepositoryCleanupInProgress.EMPTY);
+                final RepositoryCleanupInProgress repositoryCleanupInProgress = RepositoryCleanupInProgress.get(event.state());
                 if (repositoryCleanupInProgress.hasCleanupInProgress() == false) {
                     return;
                 }
@@ -120,7 +119,7 @@ public final class TransportCleanupRepositoryAction extends TransportMasterNodeA
     }
 
     private static ClusterState removeInProgressCleanup(final ClusterState currentState) {
-        return currentState.custom(RepositoryCleanupInProgress.TYPE, RepositoryCleanupInProgress.EMPTY).hasCleanupInProgress()
+        return RepositoryCleanupInProgress.get(currentState).hasCleanupInProgress()
             ? ClusterState.builder(currentState).putCustom(RepositoryCleanupInProgress.TYPE, RepositoryCleanupInProgress.EMPTY).build()
             : currentState;
     }
@@ -168,10 +167,7 @@ public final class TransportCleanupRepositoryAction extends TransportMasterNodeA
                     @Override
                     public ClusterState execute(ClusterState currentState) {
                         SnapshotsService.ensureRepositoryExists(repositoryName, currentState);
-                        final RepositoryCleanupInProgress repositoryCleanupInProgress = currentState.custom(
-                            RepositoryCleanupInProgress.TYPE,
-                            RepositoryCleanupInProgress.EMPTY
-                        );
+                        final RepositoryCleanupInProgress repositoryCleanupInProgress = RepositoryCleanupInProgress.get(currentState);
                         if (repositoryCleanupInProgress.hasCleanupInProgress()) {
                             throw new IllegalStateException(
                                 "Cannot cleanup ["
@@ -181,10 +177,7 @@ public final class TransportCleanupRepositoryAction extends TransportMasterNodeA
                                     + "]"
                             );
                         }
-                        final SnapshotDeletionsInProgress deletionsInProgress = currentState.custom(
-                            SnapshotDeletionsInProgress.TYPE,
-                            SnapshotDeletionsInProgress.EMPTY
-                        );
+                        final SnapshotDeletionsInProgress deletionsInProgress = SnapshotDeletionsInProgress.get(currentState);
                         if (deletionsInProgress.hasDeletionsInProgress()) {
                             throw new IllegalStateException(
                                 "Cannot cleanup ["
@@ -194,7 +187,7 @@ public final class TransportCleanupRepositoryAction extends TransportMasterNodeA
                                     + "]"
                             );
                         }
-                        SnapshotsInProgress snapshots = currentState.custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY);
+                        SnapshotsInProgress snapshots = SnapshotsInProgress.get(currentState);
                         if (snapshots.isEmpty() == false) {
                             throw new IllegalStateException(
                                 "Cannot cleanup [" + repositoryName + "] - a snapshot is currently running in [" + snapshots + "]"

+ 1 - 1
server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/get/TransportGetRepositoriesAction.java

@@ -92,7 +92,7 @@ public class TransportGetRepositoriesAction extends TransportMasterNodeReadActio
      * @return a result with the repository metadata that were found in the cluster state and the missing repositories
      */
     public static RepositoriesResult getRepositories(ClusterState state, String[] repoNames) {
-        RepositoriesMetadata repositories = state.metadata().custom(RepositoriesMetadata.TYPE, RepositoriesMetadata.EMPTY);
+        RepositoriesMetadata repositories = RepositoriesMetadata.get(state);
         if (isMatchAll(repoNames)) {
             return new RepositoriesResult(repositories.repositories());
         }

+ 1 - 1
server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/get/TransportGetSnapshotsAction.java

@@ -110,7 +110,7 @@ public class TransportGetSnapshotsAction extends TransportMasterNodeAction<GetSn
 
         getMultipleReposSnapshotInfo(
             request.isSingleRepositoryRequest() == false,
-            state.custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY),
+            SnapshotsInProgress.get(state),
             TransportGetRepositoriesAction.getRepositories(state, request.repositories()),
             request.snapshots(),
             request.ignoreUnavailable(),

+ 1 - 4
server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/get/shard/TransportGetShardSnapshotAction.java

@@ -136,10 +136,7 @@ public class TransportGetShardSnapshotAction extends TransportMasterNodeAction<G
 
     private static Iterator<String> getRequestedRepositories(GetShardSnapshotRequest request, ClusterState state) {
         if (request.getFromAllRepositories()) {
-            return Iterators.map(
-                state.metadata().custom(RepositoriesMetadata.TYPE, RepositoriesMetadata.EMPTY).repositories().iterator(),
-                RepositoryMetadata::name
-            );
+            return Iterators.map(RepositoriesMetadata.get(state).repositories().iterator(), RepositoryMetadata::name);
         } else {
             return request.getRepositories().iterator();
         }

+ 1 - 1
server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/status/TransportSnapshotsStatusAction.java

@@ -112,7 +112,7 @@ public class TransportSnapshotsStatusAction extends TransportMasterNodeAction<Sn
         assert task instanceof CancellableTask : task + " not cancellable";
         final CancellableTask cancellableTask = (CancellableTask) task;
 
-        final SnapshotsInProgress snapshotsInProgress = state.custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY);
+        final SnapshotsInProgress snapshotsInProgress = SnapshotsInProgress.get(state);
         List<SnapshotsInProgress.Entry> currentSnapshots = SnapshotsService.currentSnapshots(
             snapshotsInProgress,
             request.repository(),

+ 4 - 4
server/src/main/java/org/elasticsearch/cluster/ClusterSnapshotStats.java

@@ -39,10 +39,10 @@ public record ClusterSnapshotStats(
 
     public static ClusterSnapshotStats of(ClusterState clusterState, long currentTimeMillis) {
         return of(
-            clusterState.metadata().custom(RepositoriesMetadata.TYPE, RepositoriesMetadata.EMPTY),
-            clusterState.custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY),
-            clusterState.custom(SnapshotDeletionsInProgress.TYPE, SnapshotDeletionsInProgress.EMPTY),
-            clusterState.custom(RepositoryCleanupInProgress.TYPE, RepositoryCleanupInProgress.EMPTY),
+            RepositoriesMetadata.get(clusterState),
+            SnapshotsInProgress.get(clusterState),
+            SnapshotDeletionsInProgress.get(clusterState),
+            RepositoryCleanupInProgress.get(clusterState),
             currentTimeMillis
         );
     }

+ 4 - 0
server/src/main/java/org/elasticsearch/cluster/RepositoryCleanupInProgress.java

@@ -28,6 +28,10 @@ public final class RepositoryCleanupInProgress extends AbstractNamedDiffable<Clu
 
     private final List<Entry> entries;
 
+    public static RepositoryCleanupInProgress get(ClusterState state) {
+        return state.custom(TYPE, EMPTY);
+    }
+
     public RepositoryCleanupInProgress(List<Entry> entries) {
         this.entries = entries;
     }

+ 4 - 0
server/src/main/java/org/elasticsearch/cluster/RestoreInProgress.java

@@ -39,6 +39,10 @@ public class RestoreInProgress extends AbstractNamedDiffable<Custom> implements
 
     private final Map<String, Entry> entries;
 
+    public static RestoreInProgress get(ClusterState state) {
+        return state.custom(TYPE, EMPTY);
+    }
+
     /**
      * Constructs new restore metadata
      *

+ 4 - 0
server/src/main/java/org/elasticsearch/cluster/SnapshotDeletionsInProgress.java

@@ -70,6 +70,10 @@ public class SnapshotDeletionsInProgress extends AbstractNamedDiffable<Custom> i
         return true;
     }
 
+    public static SnapshotDeletionsInProgress get(ClusterState state) {
+        return state.custom(TYPE, EMPTY);
+    }
+
     /**
      * Returns a new instance of {@link SnapshotDeletionsInProgress} which adds
      * the given {@link Entry} to the invoking instance.

+ 4 - 0
server/src/main/java/org/elasticsearch/cluster/SnapshotsInProgress.java

@@ -61,6 +61,10 @@ public class SnapshotsInProgress extends AbstractNamedDiffable<Custom> implement
     // keyed by repository name
     private final Map<String, ByRepo> entries;
 
+    public static SnapshotsInProgress get(ClusterState state) {
+        return state.custom(TYPE, EMPTY);
+    }
+
     public SnapshotsInProgress(StreamInput in) throws IOException {
         this(collectByRepo(in));
     }

+ 1 - 1
server/src/main/java/org/elasticsearch/cluster/metadata/MetadataDeleteIndexService.java

@@ -155,7 +155,7 @@ public class MetadataDeleteIndexService {
 
         // update snapshot restore entries
         Map<String, ClusterState.Custom> customs = currentState.getCustoms();
-        final RestoreInProgress restoreInProgress = currentState.custom(RestoreInProgress.TYPE, RestoreInProgress.EMPTY);
+        final RestoreInProgress restoreInProgress = RestoreInProgress.get(currentState);
         RestoreInProgress updatedRestoreInProgress = RestoreService.updateRestoreStateWithDeletedIndices(restoreInProgress, indices);
         if (updatedRestoreInProgress != restoreInProgress) {
             ImmutableOpenMap.Builder<String, ClusterState.Custom> builder = ImmutableOpenMap.builder(customs);

+ 5 - 0
server/src/main/java/org/elasticsearch/cluster/metadata/RepositoriesMetadata.java

@@ -11,6 +11,7 @@ package org.elasticsearch.cluster.metadata;
 import org.elasticsearch.ElasticsearchParseException;
 import org.elasticsearch.TransportVersion;
 import org.elasticsearch.cluster.AbstractNamedDiffable;
+import org.elasticsearch.cluster.ClusterState;
 import org.elasticsearch.cluster.NamedDiff;
 import org.elasticsearch.cluster.metadata.Metadata.Custom;
 import org.elasticsearch.common.Strings;
@@ -49,6 +50,10 @@ public class RepositoriesMetadata extends AbstractNamedDiffable<Custom> implemen
 
     private final List<RepositoryMetadata> repositories;
 
+    public static RepositoriesMetadata get(ClusterState state) {
+        return state.metadata().custom(TYPE, EMPTY);
+    }
+
     /**
      * Constructs new repository metadata
      *

+ 8 - 10
server/src/main/java/org/elasticsearch/cluster/routing/allocation/AllocationService.java

@@ -160,16 +160,14 @@ public class AllocationService {
         assert newRoutingTable.validate(newMetadata); // validates the routing table is coherent with the cluster state metadata
 
         final ClusterState.Builder newStateBuilder = ClusterState.builder(oldState).routingTable(newRoutingTable).metadata(newMetadata);
-        final RestoreInProgress restoreInProgress = allocation.getClusterState().custom(RestoreInProgress.TYPE);
-        if (restoreInProgress != null) {
-            RestoreInProgress updatedRestoreInProgress = allocation.updateRestoreInfoWithRoutingChanges(restoreInProgress);
-            if (updatedRestoreInProgress != restoreInProgress) {
-                ImmutableOpenMap.Builder<String, ClusterState.Custom> customsBuilder = ImmutableOpenMap.builder(
-                    allocation.getClusterState().getCustoms()
-                );
-                customsBuilder.put(RestoreInProgress.TYPE, updatedRestoreInProgress);
-                newStateBuilder.customs(customsBuilder.build());
-            }
+        final RestoreInProgress restoreInProgress = RestoreInProgress.get(allocation.getClusterState());
+        RestoreInProgress updatedRestoreInProgress = allocation.updateRestoreInfoWithRoutingChanges(restoreInProgress);
+        if (updatedRestoreInProgress != restoreInProgress) {
+            ImmutableOpenMap.Builder<String, ClusterState.Custom> customsBuilder = ImmutableOpenMap.builder(
+                allocation.getClusterState().getCustoms()
+            );
+            customsBuilder.put(RestoreInProgress.TYPE, updatedRestoreInProgress);
+            newStateBuilder.customs(customsBuilder.build());
         }
         final ClusterState newState = newStateBuilder.build();
 

+ 7 - 11
server/src/main/java/org/elasticsearch/cluster/routing/allocation/decider/RestoreInProgressAllocationDecider.java

@@ -39,17 +39,13 @@ public class RestoreInProgressAllocationDecider extends AllocationDecider {
             return allocation.decision(Decision.YES, NAME, "not an API-level restore");
         }
 
-        final RestoreInProgress restoresInProgress = allocation.getClusterState().custom(RestoreInProgress.TYPE);
-
-        if (restoresInProgress != null) {
-            RestoreInProgress.Entry restoreInProgress = restoresInProgress.get(source.restoreUUID());
-            if (restoreInProgress != null) {
-                RestoreInProgress.ShardRestoreStatus shardRestoreStatus = restoreInProgress.shards().get(shardRouting.shardId());
-                if (shardRestoreStatus != null && shardRestoreStatus.state().completed() == false) {
-                    assert shardRestoreStatus.state() != RestoreInProgress.State.SUCCESS
-                        : "expected shard [" + shardRouting + "] to be in initializing state but got [" + shardRestoreStatus.state() + "]";
-                    return allocation.decision(Decision.YES, NAME, "shard is currently being restored");
-                }
+        RestoreInProgress.Entry restoreInProgress = RestoreInProgress.get(allocation.getClusterState()).get(source.restoreUUID());
+        if (restoreInProgress != null) {
+            RestoreInProgress.ShardRestoreStatus shardRestoreStatus = restoreInProgress.shards().get(shardRouting.shardId());
+            if (shardRestoreStatus != null && shardRestoreStatus.state().completed() == false) {
+                assert shardRestoreStatus.state() != RestoreInProgress.State.SUCCESS
+                    : "expected shard [" + shardRouting + "] to be in initializing state but got [" + shardRestoreStatus.state() + "]";
+                return allocation.decision(Decision.YES, NAME, "shard is currently being restored");
             }
         }
         return allocation.decision(

+ 2 - 2
server/src/main/java/org/elasticsearch/cluster/routing/allocation/decider/SnapshotInProgressAllocationDecider.java

@@ -59,8 +59,8 @@ public class SnapshotInProgressAllocationDecider extends AllocationDecider {
             return YES_NOT_SNAPSHOTTED;
         }
 
-        SnapshotsInProgress snapshotsInProgress = allocation.getClusterState().custom(SnapshotsInProgress.TYPE);
-        if (snapshotsInProgress == null || snapshotsInProgress.isEmpty()) {
+        SnapshotsInProgress snapshotsInProgress = SnapshotsInProgress.get(allocation.getClusterState());
+        if (snapshotsInProgress.isEmpty()) {
             // Snapshots are not running
             return YES_NOT_RUNNING;
         }

+ 2 - 5
server/src/main/java/org/elasticsearch/indices/recovery/plan/ShardSnapshotsService.java

@@ -77,11 +77,8 @@ public class ShardSnapshotsService {
     public void fetchLatestSnapshotsForShard(ShardId shardId, ActionListener<Optional<ShardSnapshot>> listener) {
         assert shardId != null : "ShardId was null but a value was expected";
 
-        final RepositoriesMetadata currentReposMetadata = clusterService.state()
-            .metadata()
-            .custom(RepositoriesMetadata.TYPE, RepositoriesMetadata.EMPTY);
-
-        List<String> repositories = currentReposMetadata.repositories()
+        List<String> repositories = RepositoriesMetadata.get(clusterService.state())
+            .repositories()
             .stream()
             .filter(repositoryMetadata -> BlobStoreRepository.USE_FOR_PEER_RECOVERY_SETTING.get(repositoryMetadata.settings()))
             .map(RepositoryMetadata::name)

+ 1 - 2
server/src/main/java/org/elasticsearch/repositories/FinalizeSnapshotContext.java

@@ -102,8 +102,7 @@ public final class FinalizeSnapshotContext extends DelegatingActionListener<Repo
     public ClusterState updatedClusterState(ClusterState state) {
         final ClusterState updatedState = SnapshotsService.stateWithoutSnapshot(state, snapshotInfo.snapshot());
         obsoleteGenerations.set(
-            updatedState.custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY)
-                .obsoleteGenerations(snapshotInfo.repository(), state.custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY))
+            SnapshotsInProgress.get(updatedState).obsoleteGenerations(snapshotInfo.repository(), SnapshotsInProgress.get(state))
         );
         return updatedState;
     }

+ 10 - 25
server/src/main/java/org/elasticsearch/repositories/RepositoriesService.java

@@ -266,9 +266,8 @@ public class RepositoriesService extends AbstractLifecycleComponent implements C
         @Override
         public ClusterState execute(ClusterState currentState) {
             RepositoryMetadata newRepositoryMetadata = new RepositoryMetadata(request.name(), request.type(), request.settings());
-            Metadata metadata = currentState.metadata();
             Metadata.Builder mdBuilder = Metadata.builder(currentState.metadata());
-            RepositoriesMetadata repositories = metadata.custom(RepositoriesMetadata.TYPE, RepositoriesMetadata.EMPTY);
+            RepositoriesMetadata repositories = RepositoriesMetadata.get(currentState);
             List<RepositoryMetadata> repositoriesMetadata = new ArrayList<>(repositories.repositories().size() + 1);
             for (RepositoryMetadata repositoryMetadata : repositories.repositories()) {
                 if (repositoryMetadata.name().equals(newRepositoryMetadata.name())) {
@@ -368,10 +367,7 @@ public class RepositoriesService extends AbstractLifecycleComponent implements C
             return;
         }
 
-        final RepositoriesMetadata currentReposMetadata = clusterService.state()
-            .metadata()
-            .custom(RepositoriesMetadata.TYPE, RepositoriesMetadata.EMPTY);
-        final RepositoryMetadata repositoryMetadata = currentReposMetadata.repository(repositoryName);
+        final RepositoryMetadata repositoryMetadata = RepositoriesMetadata.get(clusterService.state()).repository(repositoryName);
         if (repositoryMetadata == null || repositoryMetadata.uuid().equals(repositoryUuid)) {
             listener.onResponse(null);
             return;
@@ -383,8 +379,7 @@ public class RepositoriesService extends AbstractLifecycleComponent implements C
             new ClusterStateUpdateTask() {
                 @Override
                 public ClusterState execute(ClusterState currentState) {
-                    final RepositoriesMetadata currentReposMetadata = currentState.metadata()
-                        .custom(RepositoriesMetadata.TYPE, RepositoriesMetadata.EMPTY);
+                    final RepositoriesMetadata currentReposMetadata = RepositoriesMetadata.get(currentState);
 
                     final RepositoryMetadata repositoryMetadata = currentReposMetadata.repository(repositoryName);
                     if (repositoryMetadata == null || repositoryMetadata.uuid().equals(repositoryUuid)) {
@@ -458,9 +453,8 @@ public class RepositoriesService extends AbstractLifecycleComponent implements C
 
         @Override
         public ClusterState execute(ClusterState currentState) {
-            Metadata metadata = currentState.metadata();
             Metadata.Builder mdBuilder = Metadata.builder(currentState.metadata());
-            RepositoriesMetadata repositories = metadata.custom(RepositoriesMetadata.TYPE, RepositoriesMetadata.EMPTY);
+            RepositoriesMetadata repositories = RepositoriesMetadata.get(currentState);
             if (repositories.repositories().size() > 0) {
                 List<RepositoryMetadata> repositoriesMetadata = new ArrayList<>(repositories.repositories().size());
                 boolean changed = false;
@@ -545,10 +539,8 @@ public class RepositoriesService extends AbstractLifecycleComponent implements C
     public void applyClusterState(ClusterChangedEvent event) {
         try {
             final ClusterState state = event.state();
-            final RepositoriesMetadata oldMetadata = event.previousState()
-                .getMetadata()
-                .custom(RepositoriesMetadata.TYPE, RepositoriesMetadata.EMPTY);
-            final RepositoriesMetadata newMetadata = state.getMetadata().custom(RepositoriesMetadata.TYPE, RepositoriesMetadata.EMPTY);
+            final RepositoriesMetadata oldMetadata = RepositoriesMetadata.get(event.previousState());
+            final RepositoriesMetadata newMetadata = RepositoriesMetadata.get(state);
 
             // Check if repositories got changed
             if (oldMetadata.equalsIgnoreGenerations(newMetadata)) {
@@ -829,27 +821,20 @@ public class RepositoriesService extends AbstractLifecycleComponent implements C
     }
 
     private static void ensureRepositoryNotInUse(ClusterState clusterState, String repository) {
-        final SnapshotsInProgress snapshots = clusterState.custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY);
-        if (snapshots.forRepo(repository).isEmpty() == false) {
+        if (SnapshotsInProgress.get(clusterState).forRepo(repository).isEmpty() == false) {
             throw newRepositoryConflictException(repository, "snapshot is in progress");
         }
-        for (SnapshotDeletionsInProgress.Entry entry : clusterState.custom(
-            SnapshotDeletionsInProgress.TYPE,
-            SnapshotDeletionsInProgress.EMPTY
-        ).getEntries()) {
+        for (SnapshotDeletionsInProgress.Entry entry : SnapshotDeletionsInProgress.get(clusterState).getEntries()) {
             if (entry.repository().equals(repository)) {
                 throw newRepositoryConflictException(repository, "snapshot deletion is in progress");
             }
         }
-        for (RepositoryCleanupInProgress.Entry entry : clusterState.custom(
-            RepositoryCleanupInProgress.TYPE,
-            RepositoryCleanupInProgress.EMPTY
-        ).entries()) {
+        for (RepositoryCleanupInProgress.Entry entry : RepositoryCleanupInProgress.get(clusterState).entries()) {
             if (entry.repository().equals(repository)) {
                 throw newRepositoryConflictException(repository, "repository clean up is in progress");
             }
         }
-        for (RestoreInProgress.Entry entry : clusterState.custom(RestoreInProgress.TYPE, RestoreInProgress.EMPTY)) {
+        for (RestoreInProgress.Entry entry : RestoreInProgress.get(clusterState)) {
             if (repository.equals(entry.snapshot().getRepository())) {
                 throw newRepositoryConflictException(repository, "snapshot restore is in progress");
             }

+ 12 - 23
server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java

@@ -623,20 +623,15 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp
             return;
         }
         if (bestEffortConsistency) {
-            final SnapshotsInProgress snapshotsInProgress = state.custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY);
-            long bestGenerationFromCS = bestGeneration(snapshotsInProgress.forRepo(this.metadata.name()));
+            long bestGenerationFromCS = bestGeneration(SnapshotsInProgress.get(state).forRepo(this.metadata.name()));
             // Don't use generation from the delete task if we already found a generation for an in progress snapshot.
             // In this case, the generation points at the generation the repo will be in after the snapshot finishes so it may not yet
             // exist
             if (bestGenerationFromCS == RepositoryData.EMPTY_REPO_GEN) {
-                bestGenerationFromCS = bestGeneration(
-                    state.custom(SnapshotDeletionsInProgress.TYPE, SnapshotDeletionsInProgress.EMPTY).getEntries()
-                );
+                bestGenerationFromCS = bestGeneration(SnapshotDeletionsInProgress.get(state).getEntries());
             }
             if (bestGenerationFromCS == RepositoryData.EMPTY_REPO_GEN) {
-                bestGenerationFromCS = bestGeneration(
-                    state.custom(RepositoryCleanupInProgress.TYPE, RepositoryCleanupInProgress.EMPTY).entries()
-                );
+                bestGenerationFromCS = bestGeneration(RepositoryCleanupInProgress.get(state).entries());
             }
             final long finalBestGen = Math.max(bestGenerationFromCS, metadata.generation());
             latestKnownRepoGen.accumulateAndGet(finalBestGen, Math::max);
@@ -1841,8 +1836,7 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp
                                             Metadata.builder(currentState.getMetadata())
                                                 .putCustom(
                                                     RepositoriesMetadata.TYPE,
-                                                    currentState.metadata()
-                                                        .<RepositoriesMetadata>custom(RepositoriesMetadata.TYPE)
+                                                    RepositoriesMetadata.get(currentState)
                                                         .withUpdatedGeneration(metadata.name(), repoData.getGenId(), repoData.getGenId())
                                                 )
                                         )
@@ -2064,7 +2058,7 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp
             new ClusterStateUpdateTask() {
                 @Override
                 public ClusterState execute(ClusterState currentState) {
-                    final RepositoriesMetadata state = currentState.metadata().custom(RepositoriesMetadata.TYPE);
+                    final RepositoriesMetadata state = RepositoriesMetadata.get(currentState);
                     final RepositoryMetadata repoState = state.repository(metadata.name());
                     if (repoState.generation() != corruptedGeneration) {
                         throw new IllegalStateException(
@@ -2233,9 +2227,7 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp
                         Metadata.builder(currentState.getMetadata())
                             .putCustom(
                                 RepositoriesMetadata.TYPE,
-                                currentState.metadata()
-                                    .<RepositoriesMetadata>custom(RepositoriesMetadata.TYPE)
-                                    .withUpdatedGeneration(repoName, safeGeneration, newGen)
+                                RepositoriesMetadata.get(currentState).withUpdatedGeneration(repoName, safeGeneration, newGen)
                             )
                             .build()
                     )
@@ -2357,8 +2349,8 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp
                                 + "]"
                         );
                     }
-                    final RepositoriesMetadata currentMetadata = currentState.metadata().custom(RepositoriesMetadata.TYPE);
-                    final RepositoriesMetadata withGenerations = currentMetadata.withUpdatedGeneration(metadata.name(), newGen, newGen);
+                    final RepositoriesMetadata withGenerations = RepositoriesMetadata.get(currentState)
+                        .withUpdatedGeneration(metadata.name(), newGen, newGen);
                     final RepositoriesMetadata withUuid = meta.uuid().equals(newRepositoryData.getUuid())
                         ? withGenerations
                         : withGenerations.withUuid(metadata.name(), newRepositoryData.getUuid());
@@ -2516,8 +2508,8 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp
         final SnapshotsInProgress updatedSnapshotsInProgress;
         boolean changedSnapshots = false;
         final List<SnapshotsInProgress.Entry> snapshotEntries = new ArrayList<>();
-        final SnapshotsInProgress snapshotsInProgress = state.custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY);
-        for (SnapshotsInProgress.Entry entry : state.custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY).forRepo(repoName)) {
+        final SnapshotsInProgress snapshotsInProgress = SnapshotsInProgress.get(state);
+        for (SnapshotsInProgress.Entry entry : snapshotsInProgress.forRepo(repoName)) {
             if (entry.repositoryStateId() == oldGen) {
                 snapshotEntries.add(entry.withRepoGen(newGen));
                 changedSnapshots = true;
@@ -2529,8 +2521,7 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp
         final SnapshotDeletionsInProgress updatedDeletionsInProgress;
         boolean changedDeletions = false;
         final List<SnapshotDeletionsInProgress.Entry> deletionEntries = new ArrayList<>();
-        for (SnapshotDeletionsInProgress.Entry entry : state.custom(SnapshotDeletionsInProgress.TYPE, SnapshotDeletionsInProgress.EMPTY)
-            .getEntries()) {
+        for (SnapshotDeletionsInProgress.Entry entry : SnapshotDeletionsInProgress.get(state).getEntries()) {
             if (entry.repository().equals(repoName) && entry.repositoryStateId() == oldGen) {
                 deletionEntries.add(entry.withRepoGen(newGen));
                 changedDeletions = true;
@@ -2543,9 +2534,7 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp
     }
 
     private RepositoryMetadata getRepoMetadata(ClusterState state) {
-        final RepositoryMetadata repositoryMetadata = state.getMetadata()
-            .<RepositoriesMetadata>custom(RepositoriesMetadata.TYPE)
-            .repository(metadata.name());
+        final RepositoryMetadata repositoryMetadata = RepositoriesMetadata.get(state).repository(metadata.name());
         assert repositoryMetadata != null || lifecycle.stoppedOrClosed()
             : "did not find metadata for repo [" + metadata.name() + "] in state [" + lifecycleState() + "]";
         return repositoryMetadata;

+ 1 - 1
server/src/main/java/org/elasticsearch/snapshots/RepositoryIntegrityHealthIndicatorService.java

@@ -73,7 +73,7 @@ public class RepositoryIntegrityHealthIndicatorService implements HealthIndicato
 
     @Override
     public HealthIndicatorResult calculate(boolean verbose, int maxAffectedResourcesCount, HealthInfo healthInfo) {
-        var snapshotMetadata = clusterService.state().metadata().custom(RepositoriesMetadata.TYPE, RepositoriesMetadata.EMPTY);
+        var snapshotMetadata = RepositoriesMetadata.get(clusterService.state());
 
         if (snapshotMetadata.repositories().isEmpty()) {
             return createIndicator(

+ 6 - 9
server/src/main/java/org/elasticsearch/snapshots/RestoreService.java

@@ -1001,7 +1001,7 @@ public class RestoreService implements ClusterStateApplier {
      */
     public static Set<Index> restoringIndices(final ClusterState currentState, final Set<Index> indicesToCheck) {
         final Set<Index> indices = new HashSet<>();
-        for (RestoreInProgress.Entry entry : currentState.custom(RestoreInProgress.TYPE, RestoreInProgress.EMPTY)) {
+        for (RestoreInProgress.Entry entry : RestoreInProgress.get(currentState)) {
             for (Map.Entry<ShardId, RestoreInProgress.ShardRestoreStatus> shard : entry.shards().entrySet()) {
                 Index index = shard.getKey().getIndex();
                 if (indicesToCheck.contains(index)
@@ -1015,7 +1015,7 @@ public class RestoreService implements ClusterStateApplier {
     }
 
     public static RestoreInProgress.Entry restoreInProgress(ClusterState state, String restoreUUID) {
-        return state.custom(RestoreInProgress.TYPE, RestoreInProgress.EMPTY).get(restoreUUID);
+        return RestoreInProgress.get(state).get(restoreUUID);
     }
 
     /**
@@ -1031,7 +1031,7 @@ public class RestoreService implements ClusterStateApplier {
             public ClusterState execute(ClusterState currentState) {
                 RestoreInProgress.Builder restoreInProgressBuilder = new RestoreInProgress.Builder();
                 boolean changed = false;
-                for (RestoreInProgress.Entry entry : currentState.custom(RestoreInProgress.TYPE, RestoreInProgress.EMPTY)) {
+                for (RestoreInProgress.Entry entry : RestoreInProgress.get(currentState)) {
                     if (entry.state().completed()) {
                         logger.log(
                             entry.quiet() ? Level.DEBUG : Level.INFO,
@@ -1070,7 +1070,7 @@ public class RestoreService implements ClusterStateApplier {
     public void applyClusterState(ClusterChangedEvent event) {
         try {
             if (event.localNodeMaster() && cleanupInProgress == false) {
-                for (RestoreInProgress.Entry entry : event.state().custom(RestoreInProgress.TYPE, RestoreInProgress.EMPTY)) {
+                for (RestoreInProgress.Entry entry : RestoreInProgress.get(event.state())) {
                     if (entry.state().completed()) {
                         assert completed(entry.shards()) : "state says completed but restore entries are not";
                         removeCompletedRestoresFromClusterState();
@@ -1359,7 +1359,7 @@ public class RestoreService implements ClusterStateApplier {
             if (shards.isEmpty() == false) {
                 builder.putCustom(
                     RestoreInProgress.TYPE,
-                    new RestoreInProgress.Builder(currentState.custom(RestoreInProgress.TYPE, RestoreInProgress.EMPTY)).add(
+                    new RestoreInProgress.Builder(RestoreInProgress.get(currentState)).add(
                         new RestoreInProgress.Entry(
                             restoreUUID,
                             snapshot,
@@ -1417,10 +1417,7 @@ public class RestoreService implements ClusterStateApplier {
         }
 
         private void ensureSnapshotNotDeleted(ClusterState currentState) {
-            SnapshotDeletionsInProgress deletionsInProgress = currentState.custom(
-                SnapshotDeletionsInProgress.TYPE,
-                SnapshotDeletionsInProgress.EMPTY
-            );
+            SnapshotDeletionsInProgress deletionsInProgress = SnapshotDeletionsInProgress.get(currentState);
             if (deletionsInProgress.getEntries().stream().anyMatch(entry -> entry.getSnapshots().contains(snapshot.getSnapshotId()))) {
                 throw new ConcurrentSnapshotExecutionException(
                     snapshot,

+ 2 - 3
server/src/main/java/org/elasticsearch/snapshots/SnapshotShardsService.java

@@ -128,9 +128,8 @@ public class SnapshotShardsService extends AbstractLifecycleComponent implements
     @Override
     public void clusterChanged(ClusterChangedEvent event) {
         try {
-            SnapshotsInProgress previousSnapshots = event.previousState().custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY);
-            SnapshotsInProgress currentSnapshots = event.state().custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY);
-            if (previousSnapshots.equals(currentSnapshots) == false) {
+            SnapshotsInProgress currentSnapshots = SnapshotsInProgress.get(event.state());
+            if (SnapshotsInProgress.get(event.previousState()).equals(currentSnapshots) == false) {
                 synchronized (shardSnapshots) {
                     cancelRemoved(currentSnapshots);
                     for (List<SnapshotsInProgress.Entry> snapshots : currentSnapshots.entriesByRepo()) {

+ 46 - 97
server/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java

@@ -316,7 +316,7 @@ public class SnapshotsService extends AbstractLifecycleComponent implements Clus
                 ensureRepositoryExists(repositoryName, currentState);
                 ensureSnapshotNameAvailableInRepo(repositoryData, snapshotName, repository);
                 ensureNoCleanupInProgress(currentState, repositoryName, snapshotName, "clone snapshot");
-                final SnapshotsInProgress snapshots = currentState.custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY);
+                final SnapshotsInProgress snapshots = SnapshotsInProgress.get(currentState);
                 ensureSnapshotNameNotRunning(snapshots, repositoryName, snapshotName);
                 validate(repositoryName, snapshotName, currentState);
 
@@ -325,10 +325,7 @@ public class SnapshotsService extends AbstractLifecycleComponent implements Clus
                     .filter(src -> src.getName().equals(request.source()))
                     .findAny()
                     .orElseThrow(() -> new SnapshotMissingException(repositoryName, request.source()));
-                final SnapshotDeletionsInProgress deletionsInProgress = currentState.custom(
-                    SnapshotDeletionsInProgress.TYPE,
-                    SnapshotDeletionsInProgress.EMPTY
-                );
+                final SnapshotDeletionsInProgress deletionsInProgress = SnapshotDeletionsInProgress.get(currentState);
                 if (deletionsInProgress.getEntries().stream().anyMatch(entry -> entry.getSnapshots().contains(sourceSnapshotId))) {
                     throw new ConcurrentSnapshotExecutionException(
                         repositoryName,
@@ -391,10 +388,7 @@ public class SnapshotsService extends AbstractLifecycleComponent implements Clus
         final String snapshotName,
         final String reason
     ) {
-        final RepositoryCleanupInProgress repositoryCleanupInProgress = currentState.custom(
-            RepositoryCleanupInProgress.TYPE,
-            RepositoryCleanupInProgress.EMPTY
-        );
+        final RepositoryCleanupInProgress repositoryCleanupInProgress = RepositoryCleanupInProgress.get(currentState);
         if (repositoryCleanupInProgress.hasCleanupInProgress()) {
             throw new ConcurrentSnapshotExecutionException(
                 repositoryName,
@@ -479,10 +473,7 @@ public class SnapshotsService extends AbstractLifecycleComponent implements Clus
 
                 @Override
                 public ClusterState execute(ClusterState currentState) {
-                    final SnapshotsInProgress snapshotsInProgress = currentState.custom(
-                        SnapshotsInProgress.TYPE,
-                        SnapshotsInProgress.EMPTY
-                    );
+                    final SnapshotsInProgress snapshotsInProgress = SnapshotsInProgress.get(currentState);
                     final String repoName = cloneEntry.repository();
                     final List<SnapshotsInProgress.Entry> existingEntries = snapshotsInProgress.forRepo(repoName);
                     final List<SnapshotsInProgress.Entry> updatedEntries = new ArrayList<>(existingEntries.size());
@@ -492,10 +483,8 @@ public class SnapshotsService extends AbstractLifecycleComponent implements Clus
                         if (cloneEntry.snapshot().getSnapshotId().equals(existing.snapshot().getSnapshotId())) {
                             final ImmutableOpenMap.Builder<RepositoryShardId, ShardSnapshotStatus> clonesBuilder = ImmutableOpenMap
                                 .builder();
-                            final boolean readyToExecute = currentState.custom(
-                                SnapshotDeletionsInProgress.TYPE,
-                                SnapshotDeletionsInProgress.EMPTY
-                            ).hasExecutingDeletion(repoName) == false;
+                            final boolean readyToExecute = SnapshotDeletionsInProgress.get(currentState)
+                                .hasExecutingDeletion(repoName) == false;
                             final InFlightShardSnapshotStates inFlightShardStates;
                             if (readyToExecute) {
                                 inFlightShardStates = InFlightShardSnapshotStates.forEntries(snapshotsInProgress.forRepo(repoName));
@@ -666,7 +655,7 @@ public class SnapshotsService extends AbstractLifecycleComponent implements Clus
      * Throws {@link RepositoryMissingException} if no repository by the given name is found in the given cluster state.
      */
     public static void ensureRepositoryExists(String repoName, ClusterState state) {
-        if (state.metadata().custom(RepositoriesMetadata.TYPE, RepositoriesMetadata.EMPTY).repository(repoName) == null) {
+        if (RepositoriesMetadata.get(state).repository(repoName) == null) {
             throw new RepositoryMissingException(repoName);
         }
     }
@@ -679,8 +668,7 @@ public class SnapshotsService extends AbstractLifecycleComponent implements Clus
      * @param state   current cluster state
      */
     private static void validate(String repositoryName, String snapshotName, ClusterState state) {
-        RepositoriesMetadata repositoriesMetadata = state.getMetadata().custom(RepositoriesMetadata.TYPE, RepositoriesMetadata.EMPTY);
-        if (repositoriesMetadata.repository(repositoryName) == null) {
+        if (RepositoriesMetadata.get(state).repository(repositoryName) == null) {
             throw new RepositoryMissingException(repositoryName);
         }
         validate(repositoryName, snapshotName);
@@ -717,15 +705,15 @@ public class SnapshotsService extends AbstractLifecycleComponent implements Clus
     private static ShardGenerations buildGenerations(SnapshotsInProgress.Entry snapshot, Metadata metadata) {
         ShardGenerations.Builder builder = ShardGenerations.builder();
         if (snapshot.isClone()) {
-            snapshot.shardsByRepoShardId().entrySet().forEach(c -> builder.put(c.getKey().index(), c.getKey().shardId(), c.getValue()));
+            snapshot.shardsByRepoShardId().forEach((key, value) -> builder.put(key.index(), key.shardId(), value));
         } else {
-            snapshot.shardsByRepoShardId().entrySet().forEach(c -> {
-                final Index index = snapshot.indexByName(c.getKey().indexName());
+            snapshot.shardsByRepoShardId().forEach((key, value) -> {
+                final Index index = snapshot.indexByName(key.indexName());
                 if (metadata.index(index) == null) {
                     assert snapshot.partial() : "Index [" + index + "] was deleted during a snapshot but snapshot was not partial.";
                     return;
                 }
-                builder.put(c.getKey().index(), c.getKey().shardId(), c.getValue());
+                builder.put(key.index(), key.shardId(), value);
             });
         }
         return builder.build();
@@ -816,7 +804,7 @@ public class SnapshotsService extends AbstractLifecycleComponent implements Clus
         try {
             if (event.localNodeMaster()) {
                 // We don't remove old master when master flips anymore. So, we need to check for change in master
-                SnapshotsInProgress snapshotsInProgress = event.state().custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY);
+                SnapshotsInProgress snapshotsInProgress = SnapshotsInProgress.get(event.state());
                 final boolean newMaster = event.previousState().nodes().isLocalNodeElectedMaster() == false;
                 processExternalChanges(
                     newMaster || removedNodesCleanupNeeded(snapshotsInProgress, event.nodesDelta().removedNodes()),
@@ -850,7 +838,7 @@ public class SnapshotsService extends AbstractLifecycleComponent implements Clus
     }
 
     private boolean assertConsistentWithClusterState(ClusterState state) {
-        final SnapshotsInProgress snapshotsInProgress = state.custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY);
+        final SnapshotsInProgress snapshotsInProgress = SnapshotsInProgress.get(state);
         if (snapshotsInProgress.isEmpty() == false) {
             synchronized (endingSnapshots) {
                 final Set<Snapshot> runningSnapshots = Stream.concat(
@@ -865,10 +853,7 @@ public class SnapshotsService extends AbstractLifecycleComponent implements Clus
                         + runningSnapshots;
             }
         }
-        final SnapshotDeletionsInProgress snapshotDeletionsInProgress = state.custom(
-            SnapshotDeletionsInProgress.TYPE,
-            SnapshotDeletionsInProgress.EMPTY
-        );
+        final SnapshotDeletionsInProgress snapshotDeletionsInProgress = SnapshotDeletionsInProgress.get(state);
         if (snapshotDeletionsInProgress.hasDeletionsInProgress()) {
             synchronized (repositoryOperations.runningDeletions) {
                 final Set<String> runningDeletes = Stream.concat(
@@ -886,11 +871,8 @@ public class SnapshotsService extends AbstractLifecycleComponent implements Clus
     // Assert that there are no snapshots that have a shard that is waiting to be assigned even though the cluster state would allow for it
     // to be assigned
     private static boolean assertNoDanglingSnapshots(ClusterState state) {
-        final SnapshotsInProgress snapshotsInProgress = state.custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY);
-        final SnapshotDeletionsInProgress snapshotDeletionsInProgress = state.custom(
-            SnapshotDeletionsInProgress.TYPE,
-            SnapshotDeletionsInProgress.EMPTY
-        );
+        final SnapshotsInProgress snapshotsInProgress = SnapshotsInProgress.get(state);
+        final SnapshotDeletionsInProgress snapshotDeletionsInProgress = SnapshotDeletionsInProgress.get(state);
         final Set<String> reposWithRunningDelete = snapshotDeletionsInProgress.getEntries()
             .stream()
             .filter(entry -> entry.state() == SnapshotDeletionsInProgress.State.STARTED)
@@ -943,11 +925,8 @@ public class SnapshotsService extends AbstractLifecycleComponent implements Clus
             @Override
             public ClusterState execute(ClusterState currentState) {
                 RoutingTable routingTable = currentState.routingTable();
-                final SnapshotsInProgress snapshots = currentState.custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY);
-                final SnapshotDeletionsInProgress deletes = currentState.custom(
-                    SnapshotDeletionsInProgress.TYPE,
-                    SnapshotDeletionsInProgress.EMPTY
-                );
+                final SnapshotsInProgress snapshots = SnapshotsInProgress.get(currentState);
+                final SnapshotDeletionsInProgress deletes = SnapshotDeletionsInProgress.get(currentState);
                 DiscoveryNodes nodes = currentState.nodes();
                 final EnumSet<State> statesToUpdate;
                 // If we are reacting to a change in the cluster node configuration we have to update the shard states of both started
@@ -1070,10 +1049,7 @@ public class SnapshotsService extends AbstractLifecycleComponent implements Clus
                         ? ClusterState.builder(currentState).putCustom(SnapshotsInProgress.TYPE, updated).build()
                         : currentState
                 ).v1();
-                for (SnapshotDeletionsInProgress.Entry delete : res.custom(
-                    SnapshotDeletionsInProgress.TYPE,
-                    SnapshotDeletionsInProgress.EMPTY
-                ).getEntries()) {
+                for (SnapshotDeletionsInProgress.Entry delete : SnapshotDeletionsInProgress.get(res).getEntries()) {
                     if (delete.state() == SnapshotDeletionsInProgress.State.STARTED) {
                         deletionsToExecute.add(delete);
                     }
@@ -1088,10 +1064,7 @@ public class SnapshotsService extends AbstractLifecycleComponent implements Clus
 
             @Override
             public void clusterStateProcessed(ClusterState oldState, ClusterState newState) {
-                final SnapshotDeletionsInProgress snapshotDeletionsInProgress = newState.custom(
-                    SnapshotDeletionsInProgress.TYPE,
-                    SnapshotDeletionsInProgress.EMPTY
-                );
+                final SnapshotDeletionsInProgress snapshotDeletionsInProgress = SnapshotDeletionsInProgress.get(newState);
                 if (finishedSnapshots.isEmpty() == false) {
                     // If we found snapshots that should be finalized as a result of the CS update we try to initiate finalization for
                     // them
@@ -1110,7 +1083,7 @@ public class SnapshotsService extends AbstractLifecycleComponent implements Clus
                         }
                     }
                 }
-                startExecutableClones(newState.custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY), null);
+                startExecutableClones(SnapshotsInProgress.get(newState), null);
                 // run newly ready deletes
                 for (SnapshotDeletionsInProgress.Entry entry : deletionsToExecute) {
                     if (tryEnterRepoLoop(entry.repository())) {
@@ -1328,9 +1301,7 @@ public class SnapshotsService extends AbstractLifecycleComponent implements Clus
     private void finalizeSnapshotEntry(Snapshot snapshot, Metadata metadata, RepositoryData repositoryData) {
         assert currentlyFinalizing.contains(snapshot.getRepository());
         try {
-            SnapshotsInProgress.Entry entry = clusterService.state()
-                .custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY)
-                .snapshot(snapshot);
+            SnapshotsInProgress.Entry entry = SnapshotsInProgress.get(clusterService.state()).snapshot(snapshot);
             final String failure = entry.failure();
             logger.trace("[{}] finalizing snapshot in repository, state: [{}], failure[{}]", snapshot, entry.state(), failure);
             final ShardGenerations shardGenerations = buildGenerations(entry, metadata);
@@ -1483,10 +1454,10 @@ public class SnapshotsService extends AbstractLifecycleComponent implements Clus
 
         // Figure out which indices have unsuccessful shards
         Set<String> indicesWithUnsuccessfulShards = new HashSet<>();
-        entry.shardsByRepoShardId().entrySet().forEach(shard -> {
-            final ShardState shardState = shard.getValue().state();
+        entry.shardsByRepoShardId().forEach((key, value) -> {
+            final ShardState shardState = value.state();
             if (shardState.failed() || shardState.completed() == false) {
-                indicesWithUnsuccessfulShards.add(shard.getKey().indexName());
+                indicesWithUnsuccessfulShards.add(key.indexName());
             }
         });
 
@@ -1573,10 +1544,7 @@ public class SnapshotsService extends AbstractLifecycleComponent implements Clus
             public ClusterState execute(ClusterState currentState) {
                 assert readyDeletions(currentState).v1() == currentState
                     : "Deletes should have been set to ready by finished snapshot deletes and finalizations";
-                for (SnapshotDeletionsInProgress.Entry entry : currentState.custom(
-                    SnapshotDeletionsInProgress.TYPE,
-                    SnapshotDeletionsInProgress.EMPTY
-                ).getEntries()) {
+                for (SnapshotDeletionsInProgress.Entry entry : SnapshotDeletionsInProgress.get(currentState).getEntries()) {
                     if (entry.repository().equals(repository) && entry.state() == SnapshotDeletionsInProgress.State.STARTED) {
                         deletionToRun = entry;
                         break;
@@ -1613,10 +1581,7 @@ public class SnapshotsService extends AbstractLifecycleComponent implements Clus
      * @return tuple of an updated cluster state and currently executable snapshot delete operations
      */
     private static Tuple<ClusterState, List<SnapshotDeletionsInProgress.Entry>> readyDeletions(ClusterState currentState) {
-        final SnapshotDeletionsInProgress deletions = currentState.custom(
-            SnapshotDeletionsInProgress.TYPE,
-            SnapshotDeletionsInProgress.EMPTY
-        );
+        final SnapshotDeletionsInProgress deletions = SnapshotDeletionsInProgress.get(currentState);
         if (deletions.hasDeletionsInProgress() == false) {
             return Tuple.tuple(currentState, List.of());
         }
@@ -1659,7 +1624,7 @@ public class SnapshotsService extends AbstractLifecycleComponent implements Clus
      * @return updated cluster state
      */
     public static ClusterState stateWithoutSnapshot(ClusterState state, Snapshot snapshot) {
-        final SnapshotsInProgress snapshots = state.custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY);
+        final SnapshotsInProgress snapshots = SnapshotsInProgress.get(state);
         ClusterState result = state;
         int indexOfEntry = -1;
         final List<SnapshotsInProgress.Entry> entryList = snapshots.forRepo(snapshot.getRepository());
@@ -1833,7 +1798,7 @@ public class SnapshotsService extends AbstractLifecycleComponent implements Clus
                     updatedState,
                     null,
                     deletionsWithoutSnapshots(
-                        updatedState.custom(SnapshotDeletionsInProgress.TYPE, SnapshotDeletionsInProgress.EMPTY),
+                        SnapshotDeletionsInProgress.get(updatedState),
                         Collections.singletonList(snapshot.getSnapshotId()),
                         snapshot.getRepository()
                     )
@@ -1941,7 +1906,7 @@ public class SnapshotsService extends AbstractLifecycleComponent implements Clus
                 final Set<SnapshotId> snapshotIds = new HashSet<>();
 
                 // find in-progress snapshots to delete in cluster state
-                final SnapshotsInProgress snapshotsInProgress = currentState.custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY);
+                final SnapshotsInProgress snapshotsInProgress = SnapshotsInProgress.get(currentState);
                 for (SnapshotsInProgress.Entry entry : snapshotsInProgress.forRepo(repositoryName)) {
                     final SnapshotId snapshotId = entry.snapshot().getSnapshotId();
                     if (Regex.simpleMatch(snapshotNames, snapshotId.getName())) {
@@ -1996,12 +1961,9 @@ public class SnapshotsService extends AbstractLifecycleComponent implements Clus
                     "delete snapshot"
                 );
 
-                final SnapshotDeletionsInProgress deletionsInProgress = currentState.custom(
-                    SnapshotDeletionsInProgress.TYPE,
-                    SnapshotDeletionsInProgress.EMPTY
-                );
+                final SnapshotDeletionsInProgress deletionsInProgress = SnapshotDeletionsInProgress.get(currentState);
 
-                final RestoreInProgress restoreInProgress = currentState.custom(RestoreInProgress.TYPE, RestoreInProgress.EMPTY);
+                final RestoreInProgress restoreInProgress = RestoreInProgress.get(currentState);
                 // don't allow snapshot deletions while a restore is taking place,
                 // otherwise we could end up deleting a snapshot that is being restored
                 // and the files the restore depends on would all be gone
@@ -2293,9 +2255,7 @@ public class SnapshotsService extends AbstractLifecycleComponent implements Clus
                     // because it could fail for any number of reasons so we just retry instead of waiting on the cluster state
                     // to change in any form.
                     if (repositoryMetadataStart.equals(
-                        currentState.getMetadata()
-                            .<RepositoriesMetadata>custom(RepositoriesMetadata.TYPE)
-                            .repository(repository.getMetadata().name())
+                        RepositoriesMetadata.get(currentState).repository(repository.getMetadata().name())
                     )) {
                         executedTask = true;
                         return updateTask.execute(currentState);
@@ -2552,7 +2512,7 @@ public class SnapshotsService extends AbstractLifecycleComponent implements Clus
             }
             // TODO: be more efficient here, we could collect newly ready shard clones as we compute them and then directly start them
             // instead of looping over all possible clones to execute
-            startExecutableClones(newState.custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY), null);
+            startExecutableClones(SnapshotsInProgress.get(newState), null);
         }
 
         /**
@@ -2582,7 +2542,7 @@ public class SnapshotsService extends AbstractLifecycleComponent implements Clus
          */
         @Nullable
         private SnapshotsInProgress updatedSnapshotsInProgress(ClusterState currentState, SnapshotDeletionsInProgress updatedDeletions) {
-            final SnapshotsInProgress snapshotsInProgress = currentState.custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY);
+            final SnapshotsInProgress snapshotsInProgress = SnapshotsInProgress.get(currentState);
             final List<SnapshotsInProgress.Entry> snapshotEntries = new ArrayList<>();
 
             // Keep track of shardIds that we started snapshots for as a result of removing this delete so we don't assign
@@ -2703,9 +2663,7 @@ public class SnapshotsService extends AbstractLifecycleComponent implements Clus
                 for (IndexId indexIdToRefresh : newIndexIdsToRefresh) {
                     updatedIndexIds.put(indexIdToRefresh, new IndexId(indexIdToRefresh.getName(), UUIDs.randomBase64UUID()));
                 }
-                for (int i = 0; i < snapshotEntries.size(); i++) {
-                    snapshotEntries.set(i, snapshotEntries.get(i).withUpdatedIndexIds(updatedIndexIds));
-                }
+                snapshotEntries.replaceAll(entry -> entry.withUpdatedIndexIds(updatedIndexIds));
             }
             return changed ? snapshotsInProgress.withUpdatedEntriesForRepo(repoName, snapshotEntries) : null;
         }
@@ -2771,7 +2729,7 @@ public class SnapshotsService extends AbstractLifecycleComponent implements Clus
      *
      * @param indices             Indices to snapshot
      * @param useShardGenerations whether to write {@link ShardGenerations} during the snapshot
-     * @return list of shard to be included into current snapshot
+     * @return map of shard-id to snapshot-status of all shards included into current snapshot
      */
     private static ImmutableOpenMap<ShardId, SnapshotsInProgress.ShardSnapshotStatus> shards(
         SnapshotsInProgress snapshotsInProgress,
@@ -2863,7 +2821,7 @@ public class SnapshotsService extends AbstractLifecycleComponent implements Clus
      */
     public static Set<String> snapshottingDataStreams(final ClusterState currentState, final Set<String> dataStreamsToCheck) {
         Map<String, DataStream> dataStreams = currentState.metadata().dataStreams();
-        return currentState.custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY)
+        return SnapshotsInProgress.get(currentState)
             .asStream()
             .filter(e -> e.partial() == false)
             .flatMap(e -> e.dataStreams().stream())
@@ -2876,8 +2834,7 @@ public class SnapshotsService extends AbstractLifecycleComponent implements Clus
      */
     public static Set<Index> snapshottingIndices(final ClusterState currentState, final Set<Index> indicesToCheck) {
         final Set<Index> indices = new HashSet<>();
-        for (List<SnapshotsInProgress.Entry> snapshotsInRepo : currentState.custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY)
-            .entriesByRepo()) {
+        for (List<SnapshotsInProgress.Entry> snapshotsInRepo : SnapshotsInProgress.get(currentState).entriesByRepo()) {
             for (final SnapshotsInProgress.Entry entry : snapshotsInRepo) {
                 if (entry.partial() == false && entry.isClone() == false) {
                     for (String indexName : entry.indices().keySet()) {
@@ -3014,7 +2971,7 @@ public class SnapshotsService extends AbstractLifecycleComponent implements Clus
         }
 
         SnapshotsInProgress computeUpdatedState() {
-            final SnapshotsInProgress existing = initialState.custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY);
+            final SnapshotsInProgress existing = SnapshotsInProgress.get(initialState);
             SnapshotsInProgress updated = existing;
             for (Map.Entry<String, List<ShardSnapshotUpdate>> updates : updatesByRepo.entrySet()) {
                 final String repoName = updates.getKey();
@@ -3484,10 +3441,7 @@ public class SnapshotsService extends AbstractLifecycleComponent implements Clus
 
         @Override
         public ClusterState execute(ClusterState currentState) {
-            final SnapshotDeletionsInProgress deletionsInProgress = currentState.custom(
-                SnapshotDeletionsInProgress.TYPE,
-                SnapshotDeletionsInProgress.EMPTY
-            );
+            final SnapshotDeletionsInProgress deletionsInProgress = SnapshotDeletionsInProgress.get(currentState);
             boolean changed = false;
             final List<SnapshotDeletionsInProgress.Entry> remainingEntries = deletionsInProgress.getEntries();
             List<SnapshotDeletionsInProgress.Entry> updatedEntries = new ArrayList<>(remainingEntries.size());
@@ -3500,7 +3454,7 @@ public class SnapshotsService extends AbstractLifecycleComponent implements Clus
                 }
             }
             final SnapshotDeletionsInProgress updatedDeletions = changed ? SnapshotDeletionsInProgress.of(updatedEntries) : null;
-            final SnapshotsInProgress snapshotsInProgress = currentState.custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY);
+            final SnapshotsInProgress snapshotsInProgress = SnapshotsInProgress.get(currentState);
             boolean changedSnapshots = false;
             for (SnapshotsInProgress.Entry entry : snapshotsInProgress.forRepo(repository)) {
                 // We failed to read repository data for this delete, it is not the job of SnapshotsService to
@@ -3652,14 +3606,12 @@ public class SnapshotsService extends AbstractLifecycleComponent implements Clus
         public ClusterState execute(BatchExecutionContext<SnapshotTask> batchExecutionContext) throws Exception {
             final ClusterState state = batchExecutionContext.initialState();
             final SnapshotShardsUpdateContext shardsUpdateContext = new SnapshotShardsUpdateContext(batchExecutionContext);
-            final SnapshotsInProgress initialSnapshots = state.custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY);
+            final SnapshotsInProgress initialSnapshots = SnapshotsInProgress.get(state);
             SnapshotsInProgress snapshotsInProgress = shardsUpdateContext.computeUpdatedState();
             for (final var taskContext : batchExecutionContext.taskContexts()) {
                 if (taskContext.getTask() instanceof CreateSnapshotTask task) {
                     try {
-                        final var repoMeta = state.metadata()
-                            .custom(RepositoriesMetadata.TYPE, RepositoriesMetadata.EMPTY)
-                            .repository(task.snapshot.getRepository());
+                        final var repoMeta = RepositoriesMetadata.get(state).repository(task.snapshot.getRepository());
                         if (Objects.equals(task.initialRepositoryMetadata, repoMeta)) {
                             snapshotsInProgress = createSnapshot(task, taskContext, state, snapshotsInProgress);
                         } else {
@@ -3701,10 +3653,7 @@ public class SnapshotsService extends AbstractLifecycleComponent implements Clus
             ensureSnapshotNameAvailableInRepo(repositoryData, snapshotName, repository);
             ensureSnapshotNameNotRunning(snapshotsInProgress, repositoryName, snapshotName);
             validate(repositoryName, snapshotName, currentState);
-            final SnapshotDeletionsInProgress deletionsInProgress = currentState.custom(
-                SnapshotDeletionsInProgress.TYPE,
-                SnapshotDeletionsInProgress.EMPTY
-            );
+            final SnapshotDeletionsInProgress deletionsInProgress = SnapshotDeletionsInProgress.get(currentState);
             ensureNoCleanupInProgress(currentState, repositoryName, snapshotName, "create snapshot");
             ensureBelowConcurrencyLimit(repositoryName, snapshotName, snapshotsInProgress, deletionsInProgress);
 

+ 1 - 3
server/src/test/java/org/elasticsearch/snapshots/InternalSnapshotsInfoServiceTests.java

@@ -438,9 +438,7 @@ public class InternalSnapshotsInfoServiceTests extends ESTestCase {
                 .build()
         );
 
-        final RestoreInProgress.Builder restores = new RestoreInProgress.Builder(
-            currentState.custom(RestoreInProgress.TYPE, RestoreInProgress.EMPTY)
-        );
+        final RestoreInProgress.Builder restores = new RestoreInProgress.Builder(RestoreInProgress.get(currentState));
         final Map<ShardId, RestoreInProgress.ShardRestoreStatus> shards = new HashMap<>();
         for (int i = 0; i < indexMetadata.getNumberOfShards(); i++) {
             shards.put(new ShardId(index, i), new RestoreInProgress.ShardRestoreStatus(clusterService.state().nodes().getLocalNodeId()));

+ 8 - 18
server/src/test/java/org/elasticsearch/snapshots/SnapshotResiliencyTests.java

@@ -353,7 +353,7 @@ public class SnapshotResiliencyTests extends ESTestCase {
         assertNotNull(createSnapshotResponseListener.result());
         assertNotNull(restoreSnapshotResponseListener.result());
         assertTrue(documentCountVerified.get());
-        assertTrue(masterNode.clusterService.state().custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY).isEmpty());
+        assertTrue(SnapshotsInProgress.get(masterNode.clusterService.state()).isEmpty());
         final Repository repository = masterNode.repositoriesService.repository(repoName);
         Collection<SnapshotId> snapshotIds = getRepositoryData(repository).getSnapshotIds();
         assertThat(snapshotIds, hasSize(1));
@@ -438,8 +438,7 @@ public class SnapshotResiliencyTests extends ESTestCase {
 
         final TestClusterNodes.TestClusterNode randomMaster = testClusterNodes.randomMasterNode()
             .orElseThrow(() -> new AssertionError("expected to find at least one active master node"));
-        SnapshotsInProgress finalSnapshotsInProgress = randomMaster.clusterService.state()
-            .custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY);
+        SnapshotsInProgress finalSnapshotsInProgress = SnapshotsInProgress.get(randomMaster.clusterService.state());
         assertTrue(finalSnapshotsInProgress.isEmpty());
         final Repository repository = randomMaster.repositoriesService.repository(repoName);
         Collection<SnapshotId> snapshotIds = getRepositoryData(repository).getSnapshotIds();
@@ -484,10 +483,7 @@ public class SnapshotResiliencyTests extends ESTestCase {
             () -> testClusterNodes.randomMasterNode()
                 .map(
                     master -> snapshotDeleteResponded.get()
-                        && master.clusterService.state()
-                            .custom(SnapshotDeletionsInProgress.TYPE, SnapshotDeletionsInProgress.EMPTY)
-                            .getEntries()
-                            .isEmpty()
+                        && SnapshotDeletionsInProgress.get(master.clusterService.state()).getEntries().isEmpty()
                 )
                 .orElse(false),
             TimeUnit.MINUTES.toMillis(1L)
@@ -497,8 +493,7 @@ public class SnapshotResiliencyTests extends ESTestCase {
 
         final TestClusterNodes.TestClusterNode randomMaster = testClusterNodes.randomMasterNode()
             .orElseThrow(() -> new AssertionError("expected to find at least one active master node"));
-        SnapshotsInProgress finalSnapshotsInProgress = randomMaster.clusterService.state().custom(SnapshotsInProgress.TYPE);
-        assertTrue(finalSnapshotsInProgress.isEmpty());
+        assertTrue(SnapshotsInProgress.get(randomMaster.clusterService.state()).isEmpty());
         final Repository repository = randomMaster.repositoriesService.repository(repoName);
         Collection<SnapshotId> snapshotIds = getRepositoryData(repository).getSnapshotIds();
         assertThat(snapshotIds, hasSize(0));
@@ -531,7 +526,7 @@ public class SnapshotResiliencyTests extends ESTestCase {
         masterNode.clusterService.addListener(new ClusterStateListener() {
             @Override
             public void clusterChanged(ClusterChangedEvent event) {
-                if (event.state().custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY).isEmpty() == false) {
+                if (SnapshotsInProgress.get(event.state()).isEmpty() == false) {
                     client().admin().cluster().prepareDeleteSnapshot(repoName, snapshotName).execute(deleteSnapshotStepListener);
                     masterNode.clusterService.removeListener(this);
                 }
@@ -957,8 +952,7 @@ public class SnapshotResiliencyTests extends ESTestCase {
 
         deterministicTaskQueue.runAllRunnableTasks();
 
-        SnapshotDeletionsInProgress deletionsInProgress = masterNode.clusterService.state().custom(SnapshotDeletionsInProgress.TYPE);
-        assertFalse(deletionsInProgress.hasDeletionsInProgress());
+        assertFalse(SnapshotDeletionsInProgress.get(masterNode.clusterService.state()).hasDeletionsInProgress());
         final Repository repository = masterNode.repositoriesService.repository(repoName);
         final RepositoryData repositoryData = getRepositoryData(repository);
         Collection<SnapshotId> snapshotIds = repositoryData.getSnapshotIds();
@@ -1051,17 +1045,13 @@ public class SnapshotResiliencyTests extends ESTestCase {
             if (createdSnapshot.get() == false) {
                 return false;
             }
-            return master.clusterService.state().custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY).isEmpty();
+            return SnapshotsInProgress.get(master.clusterService.state()).isEmpty();
         }).orElse(false), TimeUnit.MINUTES.toMillis(1L));
 
         clearDisruptionsAndAwaitSync();
 
         assertTrue(createdSnapshot.get());
-        assertTrue(
-            testClusterNodes.randomDataNodeSafe().clusterService.state()
-                .custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY)
-                .isEmpty()
-        );
+        assertTrue(SnapshotsInProgress.get(testClusterNodes.randomDataNodeSafe().clusterService.state()).isEmpty());
         final Repository repository = testClusterNodes.randomMasterNodeSafe().repositoriesService.repository(repoName);
         Collection<SnapshotId> snapshotIds = getRepositoryData(repository).getSnapshotIds();
         assertThat(snapshotIds, either(hasSize(1)).or(hasSize(0)));

+ 1 - 2
server/src/test/java/org/elasticsearch/snapshots/SnapshotsServiceTests.java

@@ -503,8 +503,7 @@ public class SnapshotsServiceTests extends ESTestCase {
 
     private static ClusterState applyUpdates(ClusterState state, SnapshotsService.SnapshotTask... updates) throws Exception {
         return ClusterStateTaskExecutorUtils.executeAndAssertSuccessful(state, batchExecutionContext -> {
-            final SnapshotsInProgress existing = batchExecutionContext.initialState()
-                .custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY);
+            final SnapshotsInProgress existing = SnapshotsInProgress.get(batchExecutionContext.initialState());
             final var context = new SnapshotsService.SnapshotShardsUpdateContext(batchExecutionContext);
             final SnapshotsInProgress updated = context.computeUpdatedState();
             context.completeWithUpdatedState(updated);

+ 6 - 12
test/framework/src/main/java/org/elasticsearch/snapshots/AbstractSnapshotIntegTestCase.java

@@ -556,9 +556,7 @@ public abstract class AbstractSnapshotIntegTestCase extends ESIntegTestCase {
 
     protected void awaitNDeletionsInProgress(int count) throws Exception {
         logger.info("--> wait for [{}] deletions to show up in the cluster state", count);
-        awaitClusterState(
-            state -> state.custom(SnapshotDeletionsInProgress.TYPE, SnapshotDeletionsInProgress.EMPTY).getEntries().size() == count
-        );
+        awaitClusterState(state -> SnapshotDeletionsInProgress.get(state).getEntries().size() == count);
     }
 
     protected void awaitNoMoreRunningOperations() throws Exception {
@@ -570,8 +568,7 @@ public abstract class AbstractSnapshotIntegTestCase extends ESIntegTestCase {
         awaitClusterState(
             logger,
             viaNode,
-            state -> state.custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY).isEmpty()
-                && state.custom(SnapshotDeletionsInProgress.TYPE, SnapshotDeletionsInProgress.EMPTY).hasDeletionsInProgress() == false
+            state -> SnapshotsInProgress.get(state).isEmpty() && SnapshotDeletionsInProgress.get(state).hasDeletionsInProgress() == false
         );
     }
 
@@ -614,13 +611,12 @@ public abstract class AbstractSnapshotIntegTestCase extends ESIntegTestCase {
     }
 
     protected void awaitNumberOfSnapshotsInProgress(int count) throws Exception {
-        logger.info("--> wait for [{}] snapshots to show up in the cluster state", count);
-        awaitClusterState(state -> state.custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY).count() == count);
+        awaitNumberOfSnapshotsInProgress(logger, count);
     }
 
     public static void awaitNumberOfSnapshotsInProgress(Logger logger, int count) throws Exception {
         logger.info("--> wait for [{}] snapshots to show up in the cluster state", count);
-        awaitClusterState(logger, state -> state.custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY).count() == count);
+        awaitClusterState(logger, state -> SnapshotsInProgress.get(state).count() == count);
     }
 
     protected SnapshotInfo assertSuccessful(ActionFuture<CreateSnapshotResponse> future) throws Exception {
@@ -810,10 +806,8 @@ public abstract class AbstractSnapshotIntegTestCase extends ESIntegTestCase {
     }
 
     public RepositoryMetadata getRepositoryMetadata(String repo) {
-        RepositoriesMetadata repositories = clusterService().state()
-            .metadata()
-            .custom(RepositoriesMetadata.TYPE, RepositoriesMetadata.EMPTY);
-        Optional<RepositoryMetadata> repositoryMetadata = repositories.repositories()
+        Optional<RepositoryMetadata> repositoryMetadata = RepositoriesMetadata.get(clusterService().state())
+            .repositories()
             .stream()
             .filter(x -> x.name().equals(repo))
             .findFirst();

+ 2 - 6
x-pack/plugin/ccr/src/internalClusterTest/java/org/elasticsearch/xpack/ccr/CcrRepositoryIT.java

@@ -550,8 +550,7 @@ public class CcrRepositoryIT extends CcrIntegTestCase {
 
         final PlainActionFuture<Void> waitForRestoreInProgress = PlainActionFuture.newFuture();
         final ClusterStateListener listener = event -> {
-            RestoreInProgress restoreInProgress = event.state().custom(RestoreInProgress.TYPE, RestoreInProgress.EMPTY);
-            if (restoreInProgress != null && restoreInProgress.isEmpty() == false && event.state().routingTable().hasIndex(followerIndex)) {
+            if (RestoreInProgress.get(event.state()).isEmpty() == false && event.state().routingTable().hasIndex(followerIndex)) {
                 final IndexRoutingTable indexRoutingTable = event.state().routingTable().index(followerIndex);
                 for (ShardRouting shardRouting : indexRoutingTable.shardsWithState(ShardRoutingState.UNASSIGNED)) {
                     if (shardRouting.unassignedInfo().getLastAllocationStatus() == AllocationStatus.FETCHING_SHARD_DATA) {
@@ -637,10 +636,7 @@ public class CcrRepositoryIT extends CcrIntegTestCase {
 
             final PlainActionFuture<Void> waitForAllShardSnapshotSizesFailures = PlainActionFuture.newFuture();
             final ClusterStateListener listener = event -> {
-                RestoreInProgress restoreInProgress = event.state().custom(RestoreInProgress.TYPE, RestoreInProgress.EMPTY);
-                if (restoreInProgress != null
-                    && restoreInProgress.isEmpty() == false
-                    && event.state().routingTable().hasIndex(followerIndex)) {
+                if (RestoreInProgress.get(event.state()).isEmpty() == false && event.state().routingTable().hasIndex(followerIndex)) {
                     try {
                         final IndexRoutingTable indexRoutingTable = event.state().routingTable().index(followerIndex);
                         // this assertBusy completes because the listener is added after the InternalSnapshotsInfoService

+ 1 - 2
x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ilm/AsyncRetryDuringSnapshotActionStep.java

@@ -137,8 +137,7 @@ public abstract class AsyncRetryDuringSnapshotActionStep extends AsyncActionStep
                             // The index has since been deleted, mission accomplished!
                             return true;
                         }
-                        for (List<SnapshotsInProgress.Entry> snapshots : state.custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY)
-                            .entriesByRepo()) {
+                        for (List<SnapshotsInProgress.Entry> snapshots : SnapshotsInProgress.get(state).entriesByRepo()) {
                             for (SnapshotsInProgress.Entry snapshot : snapshots) {
                                 if (snapshot.indices().containsKey(indexName)) {
                                     // There is a snapshot running with this index name

+ 1 - 1
x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ilm/GenerateSnapshotNameStep.java

@@ -60,7 +60,7 @@ public class GenerateSnapshotNameStep extends ClusterStateActionStep {
         // validate that the snapshot repository exists -- because policies are refreshed on later retries, and because
         // this fails prior to the snapshot repository being recorded in the ilm metadata, the policy can just be corrected
         // and everything will pass on the subsequent retry
-        if (clusterState.metadata().custom(RepositoriesMetadata.TYPE, RepositoriesMetadata.EMPTY).repository(snapshotRepository) == null) {
+        if (RepositoriesMetadata.get(clusterState).repository(snapshotRepository) == null) {
             throw new IllegalStateException(
                 "repository ["
                     + snapshotRepository

+ 1 - 1
x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/ilm/action/TransportPutLifecycleAction.java

@@ -263,7 +263,7 @@ public class TransportPutLifecycleAction extends TransportMasterNodeAction<Reque
         for (Phase phase : phasesWithSearchableSnapshotActions) {
             SearchableSnapshotAction action = (SearchableSnapshotAction) phase.getActions().get(SearchableSnapshotAction.NAME);
             String repository = action.getSnapshotRepository();
-            if (state.metadata().custom(RepositoriesMetadata.TYPE, RepositoriesMetadata.EMPTY).repository(repository) == null) {
+            if (RepositoriesMetadata.get(state).repository(repository) == null) {
                 throw new IllegalArgumentException(
                     "no such repository ["
                         + repository

+ 3 - 6
x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsIntegTests.java

@@ -1017,12 +1017,9 @@ public class SearchableSnapshotsIntegTests extends BaseSearchableSnapshotsIntegT
         });
 
         assertBusy(() -> {
-            final RestoreInProgress restoreInProgress = clusterAdmin().prepareState()
-                .clear()
-                .setCustoms(true)
-                .get()
-                .getState()
-                .custom(RestoreInProgress.TYPE, RestoreInProgress.EMPTY);
+            final RestoreInProgress restoreInProgress = RestoreInProgress.get(
+                clusterAdmin().prepareState().clear().setCustoms(true).get().getState()
+            );
             assertTrue(Strings.toString(restoreInProgress, true, true), restoreInProgress.isEmpty());
         });
 

+ 4 - 12
x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/allocation/SearchableSnapshotAllocator.java

@@ -60,7 +60,6 @@ import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.ConcurrentMap;
 
-import static java.util.Collections.emptyList;
 import static java.util.stream.Collectors.toSet;
 import static org.elasticsearch.gateway.ReplicaShardAllocator.augmentExplanationsWithStoreInfo;
 import static org.elasticsearch.snapshots.SearchableSnapshotsSettings.SNAPSHOT_PARTIAL_SETTING;
@@ -155,9 +154,9 @@ public class SearchableSnapshotAllocator implements ExistingShardsAllocator {
                 if (Strings.hasLength(repositoryUuid) == false) {
                     repositoryName = SNAPSHOT_REPOSITORY_NAME_SETTING.get(indexSettings);
                 } else {
-                    final RepositoriesMetadata repoMetadata = allocation.metadata().custom(RepositoriesMetadata.TYPE);
-                    final List<RepositoryMetadata> repositories = repoMetadata == null ? emptyList() : repoMetadata.repositories();
-                    repositoryName = repositories.stream()
+                    repositoryName = RepositoriesMetadata.get(allocation.getClusterState())
+                        .repositories()
+                        .stream()
                         .filter(r -> repositoryUuid.equals(r.uuid()))
                         .map(RepositoryMetadata::name)
                         .findFirst()
@@ -234,14 +233,7 @@ public class SearchableSnapshotAllocator implements ExistingShardsAllocator {
                 // else we're recovering from a real snapshot, in which case we can only fix up the recovery source once the "real"
                 // recovery attempt has completed. It might succeed, but if it doesn't then we replace it with a dummy restore to bypass
                 // the RestoreInProgressAllocationDecider
-
-                final RestoreInProgress restoreInProgress = allocation.getClusterState().custom(RestoreInProgress.TYPE);
-                if (restoreInProgress == null) {
-                    // no ongoing restores, so this shard definitely completed
-                    return RecoverySource.SnapshotRecoverySource.NO_API_RESTORE_UUID;
-                }
-
-                final RestoreInProgress.Entry entry = restoreInProgress.get(recoverySource.restoreUUID());
+                final RestoreInProgress.Entry entry = RestoreInProgress.get(allocation.getClusterState()).get(recoverySource.restoreUUID());
                 if (entry == null) {
                     // this specific restore is not ongoing, so this shard definitely completed
                     return RecoverySource.SnapshotRecoverySource.NO_API_RESTORE_UUID;

+ 2 - 2
x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/allocation/decider/SearchableSnapshotRepositoryExistsAllocationDecider.java

@@ -58,8 +58,8 @@ public class SearchableSnapshotRepositoryExistsAllocationDecider extends Allocat
         if (indexMetadata.isSearchableSnapshot()) {
             final Settings settings = indexMetadata.getSettings();
 
-            final RepositoriesMetadata repositoriesMetadata = allocation.metadata().custom(RepositoriesMetadata.TYPE);
-            if (repositoriesMetadata == null || repositoriesMetadata.repositories().isEmpty()) {
+            final var repositoriesMetadata = RepositoriesMetadata.get(allocation.getClusterState());
+            if (repositoriesMetadata.repositories().isEmpty()) {
                 return allocation.decision(Decision.NO, NAME, "there are no repositories registered in this cluster");
             }
 

+ 1 - 2
x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/cache/blob/BlobStoreCacheMaintenanceService.java

@@ -506,8 +506,7 @@ public class BlobStoreCacheMaintenanceService implements ClusterStateListener {
                         final ClusterState state = clusterService.state();
                         // compute the list of existing searchable snapshots and repositories once
                         existingSnapshots = listSearchableSnapshots(state);
-                        existingRepositories = state.metadata()
-                            .custom(RepositoriesMetadata.TYPE, RepositoriesMetadata.EMPTY)
+                        existingRepositories = RepositoriesMetadata.get(state)
                             .repositories()
                             .stream()
                             .map(RepositoryMetadata::name)

+ 3 - 4
x-pack/plugin/security/src/internalClusterTest/java/org/elasticsearch/integration/ClusterPrivilegeIntegrationTests.java

@@ -257,10 +257,9 @@ public class ClusterPrivilegeIntegrationTests extends AbstractPrivilegeTestCase
             assertThat(response.getSnapshots().get(0).getState(), is(SnapshotsInProgress.State.SUCCESS));
             // The status of the snapshot in the repository can become SUCCESS before it is fully finalized in the cluster state so wait for
             // it to disappear from the cluster state as well
-            SnapshotsInProgress snapshotsInProgress = clusterAdmin().state(new ClusterStateRequest())
-                .get()
-                .getState()
-                .custom(SnapshotsInProgress.TYPE);
+            SnapshotsInProgress snapshotsInProgress = SnapshotsInProgress.get(
+                clusterAdmin().state(new ClusterStateRequest()).get().getState()
+            );
             assertTrue(snapshotsInProgress.isEmpty());
         });
     }

+ 3 - 4
x-pack/plugin/security/src/internalClusterTest/java/org/elasticsearch/integration/SecurityFeatureStateIntegTests.java

@@ -172,10 +172,9 @@ public class SecurityFeatureStateIntegTests extends AbstractPrivilegeTestCase {
             assertThat(response.getSnapshots().get(0).getState(), is(SnapshotsInProgress.State.SUCCESS));
             // The status of the snapshot in the repository can become SUCCESS before it is fully finalized in the cluster state so wait for
             // it to disappear from the cluster state as well
-            SnapshotsInProgress snapshotsInProgress = clusterAdmin().state(new ClusterStateRequest())
-                .get()
-                .getState()
-                .custom(SnapshotsInProgress.TYPE);
+            SnapshotsInProgress snapshotsInProgress = SnapshotsInProgress.get(
+                clusterAdmin().state(new ClusterStateRequest()).get().getState()
+            );
             assertTrue(snapshotsInProgress.isEmpty());
         });
     }

+ 1 - 1
x-pack/plugin/slm/src/main/java/org/elasticsearch/xpack/slm/SnapshotLifecycleService.java

@@ -237,7 +237,7 @@ public class SnapshotLifecycleService implements Closeable, ClusterStateListener
      * @throws IllegalArgumentException if the repository does not exist
      */
     public static void validateRepositoryExists(final String repository, final ClusterState state) {
-        if (state.metadata().custom(RepositoriesMetadata.TYPE, RepositoriesMetadata.EMPTY).repository(repository) == null) {
+        if (RepositoriesMetadata.get(state).repository(repository) == null) {
             throw new IllegalArgumentException("no such repository [" + repository + "]");
         }
     }

+ 11 - 17
x-pack/plugin/slm/src/main/java/org/elasticsearch/xpack/slm/action/TransportGetSnapshotLifecycleAction.java

@@ -80,24 +80,18 @@ public class TransportGetSnapshotLifecycleAction extends TransportMasterNodeActi
                 );
             }
         } else {
-            final Map<String, SnapshotLifecyclePolicyItem.SnapshotInProgress> inProgress;
-            SnapshotsInProgress sip = state.custom(SnapshotsInProgress.TYPE);
-            if (sip == null) {
-                inProgress = Collections.emptyMap();
-            } else {
-                inProgress = new HashMap<>();
-                for (List<SnapshotsInProgress.Entry> entriesForRepo : sip.entriesByRepo()) {
-                    for (SnapshotsInProgress.Entry entry : entriesForRepo) {
-                        Map<String, Object> meta = entry.userMetadata();
-                        if (meta == null
-                            || meta.get(SnapshotsService.POLICY_ID_METADATA_FIELD) == null
-                            || (meta.get(SnapshotsService.POLICY_ID_METADATA_FIELD) instanceof String == false)) {
-                            continue;
-                        }
-
-                        String policyId = (String) meta.get(SnapshotsService.POLICY_ID_METADATA_FIELD);
-                        inProgress.put(policyId, SnapshotLifecyclePolicyItem.SnapshotInProgress.fromEntry(entry));
+            final Map<String, SnapshotLifecyclePolicyItem.SnapshotInProgress> inProgress = new HashMap<>();
+            for (List<SnapshotsInProgress.Entry> entriesForRepo : SnapshotsInProgress.get(state).entriesByRepo()) {
+                for (SnapshotsInProgress.Entry entry : entriesForRepo) {
+                    Map<String, Object> meta = entry.userMetadata();
+                    if (meta == null
+                        || meta.get(SnapshotsService.POLICY_ID_METADATA_FIELD) == null
+                        || (meta.get(SnapshotsService.POLICY_ID_METADATA_FIELD) instanceof String == false)) {
+                        continue;
                     }
+
+                    String policyId = (String) meta.get(SnapshotsService.POLICY_ID_METADATA_FIELD);
+                    inProgress.put(policyId, SnapshotLifecyclePolicyItem.SnapshotInProgress.fromEntry(entry));
                 }
             }