From d4c3e22ac0ee97c22ad69e7f45db8819edd41540 Mon Sep 17 00:00:00 2001
From: Yang Wang 
Date: Tue, 24 Jun 2025 17:05:34 +1000
Subject: [PATCH 01/12] Make SnapshotsService and related APIs support
 multi-project
---
 .../snapshots/SnapshotsServiceIT.java         |   5 +-
 .../TransportCleanupRepositoryAction.java     |  23 +-
 .../clone/TransportCloneSnapshotAction.java   |  10 +-
 .../create/TransportCreateSnapshotAction.java |  16 +-
 .../delete/TransportDeleteSnapshotAction.java |  10 +-
 .../get/TransportGetSnapshotsAction.java      |  46 ++-
 .../TransportSnapshotsStatusAction.java       | 111 ++++--
 .../cluster/SnapshotDeletionsInProgress.java  |   3 +-
 .../cluster/SnapshotsInProgress.java          |  27 +-
 .../repositories/FinalizeSnapshotContext.java |   3 +-
 .../blobstore/BlobStoreRepository.java        |  31 +-
 .../elasticsearch/snapshots/SnapshotInfo.java |   5 +
 .../snapshots/SnapshotsService.java           | 364 ++++++++++--------
 .../TransportSnapshotsStatusActionTests.java  |   5 +-
 .../snapshots/SnapshotResiliencyTests.java    |  45 ++-
 15 files changed, 429 insertions(+), 275 deletions(-)
diff --git a/server/src/internalClusterTest/java/org/elasticsearch/snapshots/SnapshotsServiceIT.java b/server/src/internalClusterTest/java/org/elasticsearch/snapshots/SnapshotsServiceIT.java
index 24539a270c03e..cf2e4c54f121b 100644
--- a/server/src/internalClusterTest/java/org/elasticsearch/snapshots/SnapshotsServiceIT.java
+++ b/server/src/internalClusterTest/java/org/elasticsearch/snapshots/SnapshotsServiceIT.java
@@ -16,6 +16,7 @@
 import org.elasticsearch.action.support.master.AcknowledgedResponse;
 import org.elasticsearch.cluster.SnapshotDeletionsInProgress;
 import org.elasticsearch.cluster.metadata.IndexMetadata;
+import org.elasticsearch.cluster.metadata.ProjectId;
 import org.elasticsearch.cluster.service.MasterService;
 import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.snapshots.mockstore.MockRepository;
@@ -176,10 +177,10 @@ private SubscribableListener createSnapshotDeletionListener(String reposit
                 return false;
             }
             if (deleteHasStarted.get() == false) {
-                deleteHasStarted.set(deletionsInProgress.hasExecutingDeletion(repositoryName));
+                deleteHasStarted.set(deletionsInProgress.hasExecutingDeletion(ProjectId.DEFAULT, repositoryName));
                 return false;
             } else {
-                return deletionsInProgress.hasExecutingDeletion(repositoryName) == false;
+                return deletionsInProgress.hasExecutingDeletion(ProjectId.DEFAULT, repositoryName) == false;
             }
         });
     }
diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/cleanup/TransportCleanupRepositoryAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/cleanup/TransportCleanupRepositoryAction.java
index cd749901eff92..9dda15a21ce30 100644
--- a/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/cleanup/TransportCleanupRepositoryAction.java
+++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/cleanup/TransportCleanupRepositoryAction.java
@@ -22,12 +22,13 @@
 import org.elasticsearch.cluster.block.ClusterBlockException;
 import org.elasticsearch.cluster.block.ClusterBlockLevel;
 import org.elasticsearch.cluster.metadata.ProjectId;
+import org.elasticsearch.cluster.metadata.ProjectMetadata;
 import org.elasticsearch.cluster.node.DiscoveryNode;
+import org.elasticsearch.cluster.project.ProjectResolver;
 import org.elasticsearch.cluster.service.ClusterService;
 import org.elasticsearch.common.blobstore.DeleteResult;
 import org.elasticsearch.common.util.concurrent.EsExecutors;
 import org.elasticsearch.common.util.concurrent.ListenableFuture;
-import org.elasticsearch.core.FixForMultiProject;
 import org.elasticsearch.core.Nullable;
 import org.elasticsearch.core.SuppressForbidden;
 import org.elasticsearch.injection.guice.Inject;
@@ -66,6 +67,7 @@ public final class TransportCleanupRepositoryAction extends TransportMasterNodeA
     private static final Logger logger = LogManager.getLogger(TransportCleanupRepositoryAction.class);
 
     private final RepositoriesService repositoriesService;
+    private final ProjectResolver projectResolver;
 
     @Inject
     public TransportCleanupRepositoryAction(
@@ -73,7 +75,8 @@ public TransportCleanupRepositoryAction(
         ClusterService clusterService,
         RepositoriesService repositoriesService,
         ThreadPool threadPool,
-        ActionFilters actionFilters
+        ActionFilters actionFilters,
+        ProjectResolver projectResolver
     ) {
         super(
             TYPE.name(),
@@ -86,6 +89,7 @@ public TransportCleanupRepositoryAction(
             EsExecutors.DIRECT_EXECUTOR_SERVICE
         );
         this.repositoriesService = repositoriesService;
+        this.projectResolver = projectResolver;
         // We add a state applier that will remove any dangling repository cleanup actions on master failover.
         // This is safe to do since cleanups will increment the repository state id before executing any operations to prevent concurrent
         // operations from corrupting the repository. This is the same safety mechanism used by snapshot deletes.
@@ -134,7 +138,7 @@ protected void masterOperation(
         ClusterState state,
         ActionListener listener
     ) {
-        cleanupRepo(request.name(), listener.map(CleanupRepositoryResponse::new));
+        cleanupRepo(projectResolver.getProjectId(), request.name(), listener.map(CleanupRepositoryResponse::new));
     }
 
     @Override
@@ -145,11 +149,12 @@ protected ClusterBlockException checkBlock(CleanupRepositoryRequest request, Clu
 
     /**
      * Runs cleanup operations on the given repository.
+     * @param projectId Project for the repository
      * @param repositoryName Repository to clean up
      * @param listener Listener for cleanup result
      */
-    private void cleanupRepo(String repositoryName, ActionListener listener) {
-        final Repository repository = repositoriesService.repository(repositoryName);
+    private void cleanupRepo(ProjectId projectId, String repositoryName, ActionListener listener) {
+        final Repository repository = repositoriesService.repository(projectId, repositoryName);
         if (repository instanceof BlobStoreRepository == false) {
             listener.onFailure(new IllegalArgumentException("Repository [" + repositoryName + "] does not support repository cleanup"));
             return;
@@ -172,8 +177,10 @@ private void cleanupRepo(String repositoryName, ActionListener TYPE = new ActionType<>("cluster:admin/snapshot/clone");
     private final SnapshotsService snapshotsService;
+    private final ProjectResolver projectResolver;
 
     @Inject
     public TransportCloneSnapshotAction(
@@ -39,7 +41,8 @@ public TransportCloneSnapshotAction(
         ClusterService clusterService,
         ThreadPool threadPool,
         SnapshotsService snapshotsService,
-        ActionFilters actionFilters
+        ActionFilters actionFilters,
+        ProjectResolver projectResolver
     ) {
         super(
             TYPE.name(),
@@ -51,12 +54,13 @@ public TransportCloneSnapshotAction(
             EsExecutors.DIRECT_EXECUTOR_SERVICE
         );
         this.snapshotsService = snapshotsService;
+        this.projectResolver = projectResolver;
     }
 
     @Override
     protected ClusterBlockException checkBlock(CloneSnapshotRequest request, ClusterState state) {
         // Cluster is not affected but we look up repositories in metadata
-        return state.blocks().globalBlockedException(ClusterBlockLevel.METADATA_READ);
+        return state.blocks().globalBlockedException(projectResolver.getProjectId(), ClusterBlockLevel.METADATA_READ);
     }
 
     @Override
@@ -66,6 +70,6 @@ protected void masterOperation(
         ClusterState state,
         final ActionListener listener
     ) {
-        snapshotsService.cloneSnapshot(request, listener.map(v -> AcknowledgedResponse.TRUE));
+        snapshotsService.cloneSnapshot(projectResolver.getProjectId(), request, listener.map(v -> AcknowledgedResponse.TRUE));
     }
 }
diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/create/TransportCreateSnapshotAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/create/TransportCreateSnapshotAction.java
index f544e07ff3d51..05942d65b6e81 100644
--- a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/create/TransportCreateSnapshotAction.java
+++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/create/TransportCreateSnapshotAction.java
@@ -16,6 +16,7 @@
 import org.elasticsearch.cluster.ClusterState;
 import org.elasticsearch.cluster.block.ClusterBlockException;
 import org.elasticsearch.cluster.block.ClusterBlockLevel;
+import org.elasticsearch.cluster.project.ProjectResolver;
 import org.elasticsearch.cluster.service.ClusterService;
 import org.elasticsearch.common.util.concurrent.EsExecutors;
 import org.elasticsearch.injection.guice.Inject;
@@ -31,6 +32,7 @@
 public class TransportCreateSnapshotAction extends TransportMasterNodeAction {
     public static final ActionType TYPE = new ActionType<>("cluster:admin/snapshot/create");
     private final SnapshotsService snapshotsService;
+    private final ProjectResolver projectResolver;
 
     @Inject
     public TransportCreateSnapshotAction(
@@ -38,7 +40,8 @@ public TransportCreateSnapshotAction(
         ClusterService clusterService,
         ThreadPool threadPool,
         SnapshotsService snapshotsService,
-        ActionFilters actionFilters
+        ActionFilters actionFilters,
+        ProjectResolver projectResolver
     ) {
         super(
             TYPE.name(),
@@ -51,12 +54,13 @@ public TransportCreateSnapshotAction(
             EsExecutors.DIRECT_EXECUTOR_SERVICE
         );
         this.snapshotsService = snapshotsService;
+        this.projectResolver = projectResolver;
     }
 
     @Override
     protected ClusterBlockException checkBlock(CreateSnapshotRequest request, ClusterState state) {
         // We only check metadata block, as we want to snapshot closed indices (which have a read block)
-        return state.blocks().globalBlockedException(ClusterBlockLevel.METADATA_READ);
+        return state.blocks().globalBlockedException(projectResolver.getProjectId(), ClusterBlockLevel.METADATA_READ);
     }
 
     @Override
@@ -67,9 +71,13 @@ protected void masterOperation(
         final ActionListener listener
     ) {
         if (request.waitForCompletion()) {
-            snapshotsService.executeSnapshot(request, listener.map(CreateSnapshotResponse::new));
+            snapshotsService.executeSnapshot(projectResolver.getProjectId(), request, listener.map(CreateSnapshotResponse::new));
         } else {
-            snapshotsService.createSnapshot(request, listener.map(snapshot -> new CreateSnapshotResponse((SnapshotInfo) null)));
+            snapshotsService.createSnapshot(
+                projectResolver.getProjectId(),
+                request,
+                listener.map(snapshot -> new CreateSnapshotResponse((SnapshotInfo) null))
+            );
         }
     }
 }
diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/delete/TransportDeleteSnapshotAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/delete/TransportDeleteSnapshotAction.java
index 639452c833197..9fffdac13fae4 100644
--- a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/delete/TransportDeleteSnapshotAction.java
+++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/delete/TransportDeleteSnapshotAction.java
@@ -18,6 +18,7 @@
 import org.elasticsearch.cluster.ClusterState;
 import org.elasticsearch.cluster.block.ClusterBlockException;
 import org.elasticsearch.cluster.block.ClusterBlockLevel;
+import org.elasticsearch.cluster.project.ProjectResolver;
 import org.elasticsearch.cluster.service.ClusterService;
 import org.elasticsearch.common.util.concurrent.EsExecutors;
 import org.elasticsearch.injection.guice.Inject;
@@ -32,6 +33,7 @@
 public class TransportDeleteSnapshotAction extends AcknowledgedTransportMasterNodeAction {
     public static final ActionType TYPE = new ActionType<>("cluster:admin/snapshot/delete");
     private final SnapshotsService snapshotsService;
+    private final ProjectResolver projectResolver;
 
     @Inject
     public TransportDeleteSnapshotAction(
@@ -39,7 +41,8 @@ public TransportDeleteSnapshotAction(
         ClusterService clusterService,
         ThreadPool threadPool,
         SnapshotsService snapshotsService,
-        ActionFilters actionFilters
+        ActionFilters actionFilters,
+        ProjectResolver projectResolver
     ) {
         super(
             TYPE.name(),
@@ -51,12 +54,13 @@ public TransportDeleteSnapshotAction(
             EsExecutors.DIRECT_EXECUTOR_SERVICE
         );
         this.snapshotsService = snapshotsService;
+        this.projectResolver = projectResolver;
     }
 
     @Override
     protected ClusterBlockException checkBlock(DeleteSnapshotRequest request, ClusterState state) {
         // Cluster is not affected but we look up repositories in metadata
-        return state.blocks().globalBlockedException(ClusterBlockLevel.METADATA_READ);
+        return state.blocks().globalBlockedException(projectResolver.getProjectId(), ClusterBlockLevel.METADATA_READ);
     }
 
     @Override
@@ -74,6 +78,6 @@ protected void masterOperation(
         ClusterState state,
         final ActionListener listener
     ) {
-        snapshotsService.deleteSnapshots(request, listener.map(v -> AcknowledgedResponse.TRUE));
+        snapshotsService.deleteSnapshots(projectResolver.getProjectId(), request, listener.map(v -> AcknowledgedResponse.TRUE));
     }
 }
diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/get/TransportGetSnapshotsAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/get/TransportGetSnapshotsAction.java
index 7393ab23d8896..a98e364bb3d56 100644
--- a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/get/TransportGetSnapshotsAction.java
+++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/get/TransportGetSnapshotsAction.java
@@ -19,7 +19,9 @@
 import org.elasticsearch.cluster.SnapshotsInProgress;
 import org.elasticsearch.cluster.block.ClusterBlockException;
 import org.elasticsearch.cluster.block.ClusterBlockLevel;
+import org.elasticsearch.cluster.metadata.ProjectId;
 import org.elasticsearch.cluster.metadata.RepositoryMetadata;
+import org.elasticsearch.cluster.project.ProjectResolver;
 import org.elasticsearch.cluster.service.ClusterService;
 import org.elasticsearch.common.Strings;
 import org.elasticsearch.common.collect.Iterators;
@@ -78,6 +80,7 @@ public class TransportGetSnapshotsAction extends TransportMasterNodeAction repositories;
 
@@ -217,6 +225,7 @@ private class GetSnapshotsOperation {
 
         GetSnapshotsOperation(
             CancellableTask cancellableTask,
+            ProjectId projectId,
             List repositories,
             String[] snapshots,
             boolean ignoreUnavailable,
@@ -233,6 +242,7 @@ private class GetSnapshotsOperation {
             EnumSet states
         ) {
             this.cancellableTask = cancellableTask;
+            this.projectId = projectId;
             this.repositories = repositories;
             this.ignoreUnavailable = ignoreUnavailable;
             this.sortBy = sortBy;
@@ -310,7 +320,10 @@ private void populateResults(ActionListener listener) {
                                 assert ThreadPool.assertCurrentThreadPool(ThreadPool.Names.MANAGEMENT);
                                 cancellableTask.ensureNotCancelled();
                                 ensureRequiredNamesPresent(repositoryName, repositoryData);
-                                return getAsyncSnapshotInfoIterator(repositoriesService.repository(repositoryName), repositoryData);
+                                return getAsyncSnapshotInfoIterator(
+                                    repositoriesService.repository(projectId, repositoryName),
+                                    repositoryData
+                                );
                             })
                             .addListener(asyncRepositoryContentsListener)
                     ),
@@ -362,7 +375,7 @@ private void maybeGetRepositoryData(String repositoryName, ActionListener(snapshotNamePredicate.requiredNames());
-            for (final var snapshotInProgress : snapshotsInProgress.forRepo(repositoryName)) {
+            for (final var snapshotInProgress : snapshotsInProgress.forRepo(projectId, repositoryName)) {
                 unmatchedRequiredNames.remove(snapshotInProgress.snapshot().getSnapshotId().getName());
             }
             if (unmatchedRequiredNames.isEmpty()) {
@@ -464,7 +477,7 @@ public void getSnapshotInfo(ActionListener listener) {
                         ActionListener.completeWith(
                             listener,
                             () -> new SnapshotInfo(
-                                new Snapshot(repository.getMetadata().name(), snapshotId),
+                                new Snapshot(repository.getProjectId(), repository.getMetadata().name(), snapshotId),
                                 indicesLookup.getOrDefault(snapshotId, Collections.emptyList()),
                                 Collections.emptyList(),
                                 Collections.emptyList(),
@@ -491,15 +504,18 @@ private Iterator getAsyncSnapshotInfoIterator(Repository repo
             return Iterators.concat(
                 // matching in-progress snapshots first
                 Iterators.map(
-                    Iterators.filter(snapshotsInProgress.forRepo(repository.getMetadata().name()).iterator(), snapshotInProgress -> {
-                        final var snapshotId = snapshotInProgress.snapshot().getSnapshotId();
-                        if (snapshotNamePredicate.test(snapshotId.getName(), true)) {
-                            matchingInProgressSnapshots.add(snapshotId);
-                            return true;
-                        } else {
-                            return false;
+                    Iterators.filter(
+                        snapshotsInProgress.forRepo(repository.getProjectId(), repository.getMetadata().name()).iterator(),
+                        snapshotInProgress -> {
+                            final var snapshotId = snapshotInProgress.snapshot().getSnapshotId();
+                            if (snapshotNamePredicate.test(snapshotId.getName(), true)) {
+                                matchingInProgressSnapshots.add(snapshotId);
+                                return true;
+                            } else {
+                                return false;
+                            }
                         }
-                    }),
+                    ),
                     this::forSnapshotInProgress
                 ),
                 repositoryData == null
diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/status/TransportSnapshotsStatusAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/status/TransportSnapshotsStatusAction.java
index fbb3815eab079..a05a183d7f7e2 100644
--- a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/status/TransportSnapshotsStatusAction.java
+++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/status/TransportSnapshotsStatusAction.java
@@ -25,12 +25,12 @@
 import org.elasticsearch.cluster.block.ClusterBlockLevel;
 import org.elasticsearch.cluster.metadata.IndexMetadata;
 import org.elasticsearch.cluster.metadata.ProjectId;
+import org.elasticsearch.cluster.project.ProjectResolver;
 import org.elasticsearch.cluster.service.ClusterService;
 import org.elasticsearch.common.Strings;
 import org.elasticsearch.common.util.CollectionUtils;
 import org.elasticsearch.common.util.concurrent.ListenableFuture;
 import org.elasticsearch.common.util.set.Sets;
-import org.elasticsearch.core.FixForMultiProject;
 import org.elasticsearch.index.shard.ShardId;
 import org.elasticsearch.index.snapshots.IndexShardSnapshotStatus;
 import org.elasticsearch.injection.guice.Inject;
@@ -76,6 +76,7 @@ public class TransportSnapshotsStatusAction extends TransportMasterNodeAction currentSnapshots = SnapshotsService.currentSnapshots(
             snapshotsInProgress,
+            projectId,
             request.repository(),
             Arrays.asList(request.snapshots())
         );
         if (currentSnapshots.isEmpty()) {
-            buildResponse(snapshotsInProgress, request, currentSnapshots, null, state.getMinTransportVersion(), cancellableTask, listener);
+            buildResponse(
+                snapshotsInProgress,
+                projectId,
+                request,
+                currentSnapshots,
+                null,
+                state.getMinTransportVersion(),
+                cancellableTask,
+                listener
+            );
             return;
         }
 
@@ -153,6 +167,7 @@ protected void masterOperation(
                     listener.delegateFailureAndWrap(
                         (l, nodeSnapshotStatuses) -> buildResponse(
                             snapshotsInProgress,
+                            projectId,
                             request,
                             currentSnapshots,
                             nodeSnapshotStatuses,
@@ -165,7 +180,16 @@ protected void masterOperation(
             );
         } else {
             // We don't have any in-progress shards, just return current stats
-            buildResponse(snapshotsInProgress, request, currentSnapshots, null, state.getMinTransportVersion(), cancellableTask, listener);
+            buildResponse(
+                snapshotsInProgress,
+                projectId,
+                request,
+                currentSnapshots,
+                null,
+                state.getMinTransportVersion(),
+                cancellableTask,
+                listener
+            );
         }
 
     }
@@ -173,6 +197,7 @@ protected void masterOperation(
     // Package access for testing.
     void buildResponse(
         SnapshotsInProgress snapshotsInProgress,
+        ProjectId projectId,
         SnapshotsStatusRequest request,
         List currentSnapshotEntries,
         TransportNodesSnapshotsStatus.NodesSnapshotStatus nodeSnapshotStatuses,
@@ -282,7 +307,7 @@ void buildResponse(
         // Now add snapshots on disk that are not currently running
         final String repositoryName = request.repository();
         if (Strings.hasText(repositoryName) && CollectionUtils.isEmpty(request.snapshots()) == false) {
-            loadRepositoryData(snapshotsInProgress, request, builder, currentSnapshotNames, repositoryName, task, listener);
+            loadRepositoryData(snapshotsInProgress, request, builder, currentSnapshotNames, projectId, repositoryName, task, listener);
         } else {
             listener.onResponse(new SnapshotsStatusResponse(Collections.unmodifiableList(builder)));
         }
@@ -293,14 +318,13 @@ private void loadRepositoryData(
         SnapshotsStatusRequest request,
         List builder,
         Set currentSnapshotNames,
+        ProjectId projectId,
         String repositoryName,
         CancellableTask task,
         ActionListener listener
     ) {
         final Set requestedSnapshotNames = Sets.newHashSet(request.snapshots());
         final ListenableFuture repositoryDataListener = new ListenableFuture<>();
-        @FixForMultiProject(description = "resolve the actual projectId, ES-10166")
-        final var projectId = ProjectId.DEFAULT;
         repositoriesService.getRepositoryData(projectId, repositoryName, repositoryDataListener);
         final Collection snapshotIdsToLoad = new ArrayList<>();
         repositoryDataListener.addListener(listener.delegateFailureAndWrap((delegate, repositoryData) -> {
@@ -329,7 +353,7 @@ private void loadRepositoryData(
                         throw new SnapshotMissingException(repositoryName, snapshotName);
                     }
                 }
-                if (snapshotsInProgress.snapshot(new Snapshot(repositoryName, snapshotId)) == null) {
+                if (snapshotsInProgress.snapshot(new Snapshot(projectId, repositoryName, snapshotId)) == null) {
                     snapshotIdsToLoad.add(snapshotId);
                 }
             }
@@ -338,38 +362,39 @@ private void loadRepositoryData(
                 delegate.onResponse(new SnapshotsStatusResponse(Collections.unmodifiableList(builder)));
             } else {
                 final List threadSafeBuilder = Collections.synchronizedList(builder);
-                repositoriesService.repository(repositoryName).getSnapshotInfo(snapshotIdsToLoad, true, task::isCancelled, snapshotInfo -> {
-                    List shardStatusBuilder = new ArrayList<>();
-                    final Map shardStatuses;
-                    shardStatuses = snapshotShards(repositoryName, repositoryData, task, snapshotInfo);
-                    // an exception here stops further fetches of snapshotInfo since context is fail-fast
-                    for (final var shardStatus : shardStatuses.entrySet()) {
-                        IndexShardSnapshotStatus.Copy lastSnapshotStatus = shardStatus.getValue();
-                        shardStatusBuilder.add(new SnapshotIndexShardStatus(shardStatus.getKey(), lastSnapshotStatus));
-                    }
-                    final SnapshotsInProgress.State state = switch (snapshotInfo.state()) {
-                        case FAILED -> SnapshotsInProgress.State.FAILED;
-                        case SUCCESS, PARTIAL ->
-                            // Both of these means the snapshot has completed.
-                            SnapshotsInProgress.State.SUCCESS;
-                        default -> throw new IllegalArgumentException("Unexpected snapshot state " + snapshotInfo.state());
-                    };
-                    final long startTime = snapshotInfo.startTime();
-                    final long endTime = snapshotInfo.endTime();
-                    assert endTime >= startTime || (endTime == 0L && snapshotInfo.state().completed() == false)
-                        : "Inconsistent timestamps found in SnapshotInfo [" + snapshotInfo + "]";
-                    threadSafeBuilder.add(
-                        new SnapshotStatus(
-                            new Snapshot(repositoryName, snapshotInfo.snapshotId()),
-                            state,
-                            Collections.unmodifiableList(shardStatusBuilder),
-                            snapshotInfo.includeGlobalState(),
-                            startTime,
-                            // Use current time to calculate overall runtime for in-progress snapshots that have endTime == 0
-                            (endTime == 0 ? threadPool.absoluteTimeInMillis() : endTime) - startTime
-                        )
-                    );
-                }, delegate.map(v -> new SnapshotsStatusResponse(List.copyOf(threadSafeBuilder))));
+                repositoriesService.repository(projectId, repositoryName)
+                    .getSnapshotInfo(snapshotIdsToLoad, true, task::isCancelled, snapshotInfo -> {
+                        List shardStatusBuilder = new ArrayList<>();
+                        final Map shardStatuses;
+                        shardStatuses = snapshotShards(projectId, repositoryName, repositoryData, task, snapshotInfo);
+                        // an exception here stops further fetches of snapshotInfo since context is fail-fast
+                        for (final var shardStatus : shardStatuses.entrySet()) {
+                            IndexShardSnapshotStatus.Copy lastSnapshotStatus = shardStatus.getValue();
+                            shardStatusBuilder.add(new SnapshotIndexShardStatus(shardStatus.getKey(), lastSnapshotStatus));
+                        }
+                        final SnapshotsInProgress.State state = switch (snapshotInfo.state()) {
+                            case FAILED -> SnapshotsInProgress.State.FAILED;
+                            case SUCCESS, PARTIAL ->
+                                // Both of these means the snapshot has completed.
+                                SnapshotsInProgress.State.SUCCESS;
+                            default -> throw new IllegalArgumentException("Unexpected snapshot state " + snapshotInfo.state());
+                        };
+                        final long startTime = snapshotInfo.startTime();
+                        final long endTime = snapshotInfo.endTime();
+                        assert endTime >= startTime || (endTime == 0L && snapshotInfo.state().completed() == false)
+                            : "Inconsistent timestamps found in SnapshotInfo [" + snapshotInfo + "]";
+                        threadSafeBuilder.add(
+                            new SnapshotStatus(
+                                new Snapshot(projectId, repositoryName, snapshotInfo.snapshotId()),
+                                state,
+                                Collections.unmodifiableList(shardStatusBuilder),
+                                snapshotInfo.includeGlobalState(),
+                                startTime,
+                                // Use current time to calculate overall runtime for in-progress snapshots that have endTime == 0
+                                (endTime == 0 ? threadPool.absoluteTimeInMillis() : endTime) - startTime
+                            )
+                        );
+                    }, delegate.map(v -> new SnapshotsStatusResponse(List.copyOf(threadSafeBuilder))));
             }
         }));
     }
@@ -382,17 +407,19 @@ private void loadRepositoryData(
      * returns similar information but for already finished snapshots.
      * 
      *
+     * @param projectId       project for the repository
      * @param repositoryName  repository name
      * @param snapshotInfo    snapshot info
      * @return map of shard id to snapshot status
      */
     private Map snapshotShards(
+        final ProjectId projectId,
         final String repositoryName,
         final RepositoryData repositoryData,
         final CancellableTask task,
         final SnapshotInfo snapshotInfo
     ) throws IOException {
-        final Repository repository = repositoriesService.repository(repositoryName);
+        final Repository repository = repositoriesService.repository(projectId, repositoryName);
         final Map shardStatus = new HashMap<>();
         for (String index : snapshotInfo.indices()) {
             IndexId indexId = repositoryData.resolveIndexId(index);
diff --git a/server/src/main/java/org/elasticsearch/cluster/SnapshotDeletionsInProgress.java b/server/src/main/java/org/elasticsearch/cluster/SnapshotDeletionsInProgress.java
index 8136719612ce6..cbd4c2ed14ca4 100644
--- a/server/src/main/java/org/elasticsearch/cluster/SnapshotDeletionsInProgress.java
+++ b/server/src/main/java/org/elasticsearch/cluster/SnapshotDeletionsInProgress.java
@@ -114,9 +114,10 @@ public List getEntries() {
     /**
      * Checks if there is an actively executing delete operation for the given repository
      *
+     * @param projectId project to look for the repository
      * @param repository repository name
      */
-    public boolean hasExecutingDeletion(String repository) {
+    public boolean hasExecutingDeletion(ProjectId projectId, String repository) {
         for (Entry entry : entries) {
             if (entry.state() == State.STARTED && entry.repository().equals(repository)) {
                 return true;
diff --git a/server/src/main/java/org/elasticsearch/cluster/SnapshotsInProgress.java b/server/src/main/java/org/elasticsearch/cluster/SnapshotsInProgress.java
index 8390c00fc420a..146de4ee37f2c 100644
--- a/server/src/main/java/org/elasticsearch/cluster/SnapshotsInProgress.java
+++ b/server/src/main/java/org/elasticsearch/cluster/SnapshotsInProgress.java
@@ -174,11 +174,15 @@ public List forRepo(String repository) {
         return forRepo(Metadata.DEFAULT_PROJECT_ID, repository);
     }
 
+    public List forRepo(ProjectId projectId, String repository) {
+        return forRepo(new ProjectRepo(projectId, repository));
+    }
+
     /**
      * Returns the list of snapshots in the specified repository.
      */
-    public List forRepo(ProjectId projectId, String repository) {
-        return entries.getOrDefault(new ProjectRepo(projectId, repository), ByRepo.EMPTY).entries;
+    public List forRepo(ProjectRepo projectRepo) {
+        return entries.getOrDefault(projectRepo, ByRepo.EMPTY).entries;
     }
 
     public boolean isEmpty() {
@@ -221,25 +225,6 @@ private static Entry findSnapshotInList(Snapshot snapshotToFind, List for
         return null;
     }
 
-    /**
-     * Computes a map of repository shard id to set of shard generations, containing all shard generations that became obsolete and may be
-     * deleted from the repository as the cluster state moves from the given old value of {@link SnapshotsInProgress} to this instance.
-     * 
-     * An unique shard generation is created for every in-progress shard snapshot. The shard generation file contains information about all
-     * the files needed by pre-existing and any new shard snapshots that were in-progress. When a shard snapshot is finalized, its file list
-     * is promoted to the official shard snapshot list for the index shard. This final list will contain metadata about any other
-     * in-progress shard snapshots that were not yet finalized when it began. All these other in-progress shard snapshot lists are scheduled
-     * for deletion now.
-     */
-    @FixForMultiProject
-    @Deprecated(forRemoval = true)
-    public Map> obsoleteGenerations(
-        String repository,
-        SnapshotsInProgress oldClusterStateSnapshots
-    ) {
-        return obsoleteGenerations(Metadata.DEFAULT_PROJECT_ID, repository, oldClusterStateSnapshots);
-    }
-
     /**
      * Computes a map of repository shard id to set of shard generations, containing all shard generations that became obsolete and may be
      * deleted from the repository as the cluster state moves from the given old value of {@link SnapshotsInProgress} to this instance.
diff --git a/server/src/main/java/org/elasticsearch/repositories/FinalizeSnapshotContext.java b/server/src/main/java/org/elasticsearch/repositories/FinalizeSnapshotContext.java
index b508238406373..d643448b691f0 100644
--- a/server/src/main/java/org/elasticsearch/repositories/FinalizeSnapshotContext.java
+++ b/server/src/main/java/org/elasticsearch/repositories/FinalizeSnapshotContext.java
@@ -107,7 +107,8 @@ public ClusterState updatedClusterState(ClusterState state) {
         // Now that the updated cluster state may have changed in-progress shard snapshots' shard generations to the latest shard
         // generation, let's mark any now unreferenced shard generations as obsolete and ready to be deleted.
         obsoleteGenerations.set(
-            SnapshotsInProgress.get(updatedState).obsoleteGenerations(snapshotInfo.repository(), SnapshotsInProgress.get(state))
+            SnapshotsInProgress.get(updatedState)
+                .obsoleteGenerations(snapshotInfo.projectId(), snapshotInfo.repository(), SnapshotsInProgress.get(state))
         );
         return updatedState;
     }
diff --git a/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java b/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java
index 996c9cda4deab..cb3fa6f575c8c 100644
--- a/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java
+++ b/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java
@@ -710,7 +710,7 @@ public boolean canUpdateInPlace(Settings updatedSettings, Set ignoredSet
     @Override
     public void updateState(ClusterState state) {
         final Settings previousSettings = metadata.settings();
-        metadata = getRepoMetadata(state);
+        metadata = getRepoMetadata(state.metadata().getProject(getProjectId()));
         final Settings updatedSettings = metadata.settings();
         if (updatedSettings.equals(previousSettings) == false) {
             snapshotRateLimiter = getSnapshotRateLimiter();
@@ -725,7 +725,7 @@ public void updateState(ClusterState state) {
             return;
         }
         if (bestEffortConsistency) {
-            long bestGenerationFromCS = bestGeneration(SnapshotsInProgress.get(state).forRepo(this.metadata.name()));
+            long bestGenerationFromCS = bestGeneration(SnapshotsInProgress.get(state).forRepo(getProjectId(), 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
@@ -1823,7 +1823,7 @@ record RootBlobUpdateResult(RepositoryData oldRepositoryData, RepositoryData new
                     // Write the index metadata for each index in the snapshot
                     for (IndexId index : indices) {
                         executor.execute(ActionRunnable.run(allMetaListeners.acquire(), () -> {
-                            final IndexMetadata indexMetaData = clusterMetadata.getProject().index(index.getName());
+                            final IndexMetadata indexMetaData = clusterMetadata.getProject(getProjectId()).index(index.getName());
                             if (writeIndexGens) {
                                 final String identifiers = IndexMetaDataGenerations.buildUniqueIdentifier(indexMetaData);
                                 String metaUUID = existingRepositoryData.indexMetaDataGenerations().getIndexMetaBlobId(identifiers);
@@ -1836,7 +1836,7 @@ record RootBlobUpdateResult(RepositoryData oldRepositoryData, RepositoryData new
                                 metadataWriteResult.indexMetas().put(index, identifiers);
                             } else {
                                 INDEX_METADATA_FORMAT.write(
-                                    clusterMetadata.getProject().index(index.getName()),
+                                    clusterMetadata.getProject(getProjectId()).index(index.getName()),
                                     indexContainer(index),
                                     snapshotId.getUUID(),
                                     compress
@@ -2403,11 +2403,11 @@ public void clusterStateProcessed(ClusterState oldState, ClusterState newState)
     private ClusterState getClusterStateWithUpdatedRepositoryGeneration(ClusterState currentState, RepositoryData repoData) {
         // In theory we might have failed over to a different master which initialized the repo and then failed back to this node, so we
         // must check the repository generation in the cluster state is still unknown here.
-        final RepositoryMetadata repoMetadata = getRepoMetadata(currentState);
+        final var project = currentState.metadata().getProject(getProjectId());
+        final RepositoryMetadata repoMetadata = getRepoMetadata(project);
         if (repoMetadata.generation() != RepositoryData.UNKNOWN_REPO_GEN) {
             throw new RepositoryException(repoMetadata.name(), "Found unexpected initialized repo metadata [" + repoMetadata + "]");
         }
-        final var project = currentState.metadata().getProject(getProjectId());
         return ClusterState.builder(currentState)
             .putProjectMetadata(
                 ProjectMetadata.builder(project)
@@ -2594,7 +2594,7 @@ private void markRepoCorrupted(long corruptedGeneration, Exception originalExcep
             new ClusterStateUpdateTask() {
                 @Override
                 public ClusterState execute(ClusterState currentState) {
-                    final var project = currentState.metadata().getDefaultProject();
+                    final var project = currentState.metadata().getProject(projectId);
                     final RepositoriesMetadata state = RepositoriesMetadata.get(project);
                     final RepositoryMetadata repoState = state.repository(metadata.name());
                     if (repoState.generation() != corruptedGeneration) {
@@ -2748,7 +2748,8 @@ protected void writeIndexGen(
 
             @Override
             public ClusterState execute(ClusterState currentState) {
-                final RepositoryMetadata meta = getRepoMetadata(currentState);
+                final var project = currentState.metadata().getProject(projectId);
+                final RepositoryMetadata meta = getRepoMetadata(project);
                 final String repoName = metadata.name();
 
                 if (RepositoriesService.isReadOnly(meta.settings())) {
@@ -2788,7 +2789,6 @@ public ClusterState execute(ClusterState currentState) {
                         + "] must be larger than latest known generation ["
                         + latestKnownRepoGen.get()
                         + "]";
-                final var project = currentState.metadata().getDefaultProject();
                 return ClusterState.builder(currentState)
                     .putProjectMetadata(
                         ProjectMetadata.builder(project)
@@ -2914,7 +2914,8 @@ public void onFailure(Exception e) {
             submitUnbatchedTask(setSafeGenerationSource, new ClusterStateUpdateTask() {
                 @Override
                 public ClusterState execute(ClusterState currentState) {
-                    final RepositoryMetadata meta = getRepoMetadata(currentState);
+                    final var project = currentState.metadata().getProject(projectId);
+                    final RepositoryMetadata meta = getRepoMetadata(project);
                     if (meta.generation() != expectedGen) {
                         throw new IllegalStateException(
                             "Tried to update repo generation to [" + newGen + "] but saw unexpected generation in state [" + meta + "]"
@@ -2929,7 +2930,6 @@ public ClusterState execute(ClusterState currentState) {
                                 + "]"
                         );
                     }
-                    final var project = currentState.metadata().getDefaultProject();
                     final RepositoriesMetadata withGenerations = RepositoriesMetadata.get(project)
                         .withUpdatedGeneration(metadata.name(), newGen, newGen);
                     final RepositoriesMetadata withUuid = meta.uuid().equals(newRepositoryData.getUuid())
@@ -3089,7 +3089,7 @@ private ClusterState updateRepositoryGenerationsIfNecessary(ClusterState state,
         boolean changedSnapshots = false;
         final List snapshotEntries = new ArrayList<>();
         final SnapshotsInProgress snapshotsInProgress = SnapshotsInProgress.get(state);
-        for (SnapshotsInProgress.Entry entry : snapshotsInProgress.forRepo(repoName)) {
+        for (SnapshotsInProgress.Entry entry : snapshotsInProgress.forRepo(getProjectId(), repoName)) {
             if (entry.repositoryStateId() == oldGen) {
                 snapshotEntries.add(entry.withRepoGen(newGen));
                 changedSnapshots = true;
@@ -3098,7 +3098,7 @@ private ClusterState updateRepositoryGenerationsIfNecessary(ClusterState state,
             }
         }
         updatedSnapshotsInProgress = changedSnapshots
-            ? snapshotsInProgress.createCopyWithUpdatedEntriesForRepo(repoName, snapshotEntries)
+            ? snapshotsInProgress.createCopyWithUpdatedEntriesForRepo(getProjectId(), repoName, snapshotEntries)
             : null;
         final SnapshotDeletionsInProgress updatedDeletionsInProgress;
         boolean changedDeletions = false;
@@ -3115,9 +3115,8 @@ private ClusterState updateRepositoryGenerationsIfNecessary(ClusterState state,
         return SnapshotsService.updateWithSnapshots(state, updatedSnapshotsInProgress, updatedDeletionsInProgress);
     }
 
-    private RepositoryMetadata getRepoMetadata(ClusterState state) {
-        final RepositoryMetadata repositoryMetadata = RepositoriesMetadata.get(state.getMetadata().getProject(getProjectId()))
-            .repository(metadata.name());
+    private RepositoryMetadata getRepoMetadata(ProjectMetadata projectMetadata) {
+        final RepositoryMetadata repositoryMetadata = RepositoriesMetadata.get(projectMetadata).repository(metadata.name());
         assert repositoryMetadata != null || lifecycle.stoppedOrClosed()
             : "did not find metadata for repo [" + metadata.name() + "] in state [" + lifecycleState() + "]";
         return repositoryMetadata;
diff --git a/server/src/main/java/org/elasticsearch/snapshots/SnapshotInfo.java b/server/src/main/java/org/elasticsearch/snapshots/SnapshotInfo.java
index 2cedf4506c7c6..2bc75665da2a5 100644
--- a/server/src/main/java/org/elasticsearch/snapshots/SnapshotInfo.java
+++ b/server/src/main/java/org/elasticsearch/snapshots/SnapshotInfo.java
@@ -12,6 +12,7 @@
 import org.elasticsearch.action.admin.cluster.snapshots.get.GetSnapshotsRequest;
 import org.elasticsearch.cluster.SnapshotsInProgress;
 import org.elasticsearch.cluster.metadata.Metadata;
+import org.elasticsearch.cluster.metadata.ProjectId;
 import org.elasticsearch.common.io.stream.StreamInput;
 import org.elasticsearch.common.io.stream.StreamOutput;
 import org.elasticsearch.common.io.stream.Writeable;
@@ -355,6 +356,10 @@ public SnapshotId snapshotId() {
         return snapshot.getSnapshotId();
     }
 
+    public ProjectId projectId() {
+        return snapshot.getProjectId();
+    }
+
     public String repository() {
         return snapshot.getRepository();
     }
diff --git a/server/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java b/server/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java
index 8a1f1ad79a17f..396287af20208 100644
--- a/server/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java
+++ b/server/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java
@@ -97,6 +97,7 @@
 import org.elasticsearch.repositories.RepositoryData;
 import org.elasticsearch.repositories.RepositoryException;
 import org.elasticsearch.repositories.RepositoryMissingException;
+import org.elasticsearch.repositories.RepositoryOperation.ProjectRepo;
 import org.elasticsearch.repositories.RepositoryShardId;
 import org.elasticsearch.repositories.ShardGeneration;
 import org.elasticsearch.repositories.ShardGenerations;
@@ -181,7 +182,7 @@ public final class SnapshotsService extends AbstractLifecycleComponent implement
     private final Map>> snapshotDeletionListeners = new ConcurrentHashMap<>();
 
     // Set of repositories currently running either a snapshot finalization or a snapshot delete.
-    private final Set currentlyFinalizing = Collections.synchronizedSet(new HashSet<>());
+    private final Set currentlyFinalizing = Collections.synchronizedSet(new HashSet<>());
 
     // Set of snapshots that are currently being ended by this node
     private final Set endingSnapshots = Collections.synchronizedSet(new HashSet<>());
@@ -255,14 +256,19 @@ public SnapshotsService(
     }
 
     /**
-     * Same as {@link #createSnapshot(CreateSnapshotRequest, ActionListener)} but invokes its callback on completion of
+     * Same as {@link #createSnapshot(ProjectId, CreateSnapshotRequest, ActionListener)} but invokes its callback on completion of
      * the snapshot.
      *
+     * @param projectId the project for creating the snapshot
      * @param request snapshot request
      * @param listener snapshot completion listener
      */
-    public void executeSnapshot(final CreateSnapshotRequest request, final ActionListener listener) {
-        createSnapshot(request, listener.delegateFailureAndWrap((l, snapshot) -> addListener(snapshot, l)));
+    public void executeSnapshot(
+        final ProjectId projectId,
+        final CreateSnapshotRequest request,
+        final ActionListener listener
+    ) {
+        createSnapshot(projectId, request, listener.delegateFailureAndWrap((l, snapshot) -> addListener(snapshot, l)));
     }
 
     /**
@@ -271,20 +277,27 @@ public void executeSnapshot(final CreateSnapshotRequest request, final ActionLis
      * This method is used by clients to start snapshot. It makes sure that there is no snapshots are currently running and
      * creates a snapshot record in cluster state metadata.
      *
+     * @param projectId the project for creating the snapshot
      * @param request  snapshot request
      * @param listener snapshot creation listener
      */
-    public void createSnapshot(final CreateSnapshotRequest request, final ActionListener listener) {
+    public void createSnapshot(final ProjectId projectId, final CreateSnapshotRequest request, final ActionListener listener) {
         final String repositoryName = request.repository();
         final String snapshotName = IndexNameExpressionResolver.resolveDateMathExpression(request.snapshot());
         validate(repositoryName, snapshotName);
         final SnapshotId snapshotId = new SnapshotId(snapshotName, request.uuid());
-        Repository repository = repositoriesService.repository(request.repository());
+        Repository repository = repositoriesService.repository(projectId, request.repository());
         if (repository.isReadOnly()) {
             listener.onFailure(new RepositoryException(repository.getMetadata().name(), "cannot create snapshot in a readonly repository"));
             return;
         }
-        submitCreateSnapshotRequest(request, listener, repository, new Snapshot(repositoryName, snapshotId), repository.getMetadata());
+        submitCreateSnapshotRequest(
+            request,
+            listener,
+            repository,
+            new Snapshot(projectId, repositoryName, snapshotId),
+            repository.getMetadata()
+        );
     }
 
     /**
@@ -309,17 +322,24 @@ private void submitCreateSnapshotRequest(
         );
     }
 
-    private static void ensureSnapshotNameNotRunning(SnapshotsInProgress runningSnapshots, String repositoryName, String snapshotName) {
-        if (runningSnapshots.forRepo(repositoryName).stream().anyMatch(s -> s.snapshot().getSnapshotId().getName().equals(snapshotName))) {
+    private static void ensureSnapshotNameNotRunning(
+        SnapshotsInProgress runningSnapshots,
+        ProjectId projectId,
+        String repositoryName,
+        String snapshotName
+    ) {
+        if (runningSnapshots.forRepo(projectId, repositoryName)
+            .stream()
+            .anyMatch(s -> s.snapshot().getSnapshotId().getName().equals(snapshotName))) {
             throw new SnapshotNameAlreadyInUseException(repositoryName, snapshotName, "snapshot with the same name is already in-progress");
         }
     }
 
     // TODO: It is worth revisiting the design choice of creating a placeholder entry in snapshots-in-progress here once we have a cache
     // for repository metadata and loading it has predictable performance
-    public void cloneSnapshot(CloneSnapshotRequest request, ActionListener listener) {
+    public void cloneSnapshot(ProjectId projectId, CloneSnapshotRequest request, ActionListener listener) {
         final String repositoryName = request.repository();
-        Repository repository = repositoriesService.repository(repositoryName);
+        Repository repository = repositoriesService.repository(projectId, repositoryName);
         if (repository.isReadOnly()) {
             listener.onFailure(new RepositoryException(repositoryName, "cannot create snapshot in a readonly repository"));
             return;
@@ -329,7 +349,7 @@ public void cloneSnapshot(CloneSnapshotRequest request, ActionListener lis
         // TODO: create snapshot UUID in CloneSnapshotRequest and make this operation idempotent to cleanly deal with transport layer
         // retries
         final SnapshotId snapshotId = new SnapshotId(snapshotName, UUIDs.randomBase64UUID());
-        final Snapshot snapshot = new Snapshot(repositoryName, snapshotId);
+        final Snapshot snapshot = new Snapshot(projectId, repositoryName, snapshotId);
         initializingClones.add(snapshot);
         executeConsistentStateUpdate(repository, repositoryData -> new ClusterStateUpdateTask(request.masterNodeTimeout()) {
 
@@ -337,13 +357,14 @@ public void cloneSnapshot(CloneSnapshotRequest request, ActionListener lis
 
             @Override
             public ClusterState execute(ClusterState currentState) {
-                ensureRepositoryExists(repositoryName, currentState);
+                final var projectMetadata = currentState.metadata().getProject(projectId);
+                ensureRepositoryExists(repositoryName, projectMetadata);
                 ensureSnapshotNameAvailableInRepo(repositoryData, snapshotName, repository);
                 ensureNoCleanupInProgress(currentState, repositoryName, snapshotName, "clone snapshot");
-                ensureNotReadOnly(currentState, repositoryName);
+                ensureNotReadOnly(projectMetadata, repositoryName);
                 final SnapshotsInProgress snapshots = SnapshotsInProgress.get(currentState);
-                ensureSnapshotNameNotRunning(snapshots, repositoryName, snapshotName);
-                validate(repositoryName, snapshotName, currentState);
+                ensureSnapshotNameNotRunning(snapshots, projectId, repositoryName, snapshotName);
+                validate(repositoryName, snapshotName, projectMetadata);
 
                 final SnapshotId sourceSnapshotId = repositoryData.getSnapshotIds()
                     .stream()
@@ -372,7 +393,7 @@ public ClusterState execute(ClusterState currentState) {
                 );
                 if (matchingIndices.isEmpty()) {
                     throw new SnapshotException(
-                        new Snapshot(repositoryName, sourceSnapshotId),
+                        new Snapshot(projectId, repositoryName, sourceSnapshotId),
                         "No indices in the source snapshot ["
                             + sourceSnapshotId
                             + "] matched requested pattern ["
@@ -413,6 +434,7 @@ public void clusterStateProcessed(ClusterState oldState, final ClusterState newS
 
     /**
      * Checks the cluster state for any in-progress repository cleanup tasks ({@link RepositoryCleanupInProgress}).
+     * Repository cleanup is intentionally cluster wide exclusive
      */
     private static void ensureNoCleanupInProgress(
         final ClusterState currentState,
@@ -436,8 +458,8 @@ private static void ensureNoCleanupInProgress(
         }
     }
 
-    public static void ensureNotReadOnly(final ClusterState currentState, final String repositoryName) {
-        final var repositoryMetadata = RepositoriesMetadata.get(currentState).repository(repositoryName);
+    public static void ensureNotReadOnly(final ProjectMetadata projectMetadata, final String repositoryName) {
+        final var repositoryMetadata = RepositoriesMetadata.get(projectMetadata).repository(repositoryName);
         if (RepositoriesService.isReadOnly(repositoryMetadata.settings())) {
             throw new RepositoryException(repositoryMetadata.name(), "repository is readonly");
         }
@@ -516,8 +538,9 @@ private void startCloning(Repository repository, SnapshotsInProgress.Entry clone
                 @Override
                 public ClusterState execute(ClusterState currentState) {
                     final SnapshotsInProgress snapshotsInProgress = SnapshotsInProgress.get(currentState);
+                    final ProjectId projectId = cloneEntry.projectId();
                     final String repoName = cloneEntry.repository();
-                    final List existingEntries = snapshotsInProgress.forRepo(repoName);
+                    final List existingEntries = snapshotsInProgress.forRepo(projectId, repoName);
                     final List updatedEntries = new ArrayList<>(existingEntries.size());
                     final String localNodeId = currentState.nodes().getLocalNodeId();
                     final ShardGenerations shardGenerations = repoData.shardGenerations();
@@ -526,10 +549,12 @@ public ClusterState execute(ClusterState currentState) {
                             final ImmutableOpenMap.Builder clonesBuilder = ImmutableOpenMap
                                 .builder();
                             final boolean readyToExecute = SnapshotDeletionsInProgress.get(currentState)
-                                .hasExecutingDeletion(repoName) == false;
+                                .hasExecutingDeletion(projectId, repoName) == false;
                             final InFlightShardSnapshotStates inFlightShardStates;
                             if (readyToExecute) {
-                                inFlightShardStates = InFlightShardSnapshotStates.forEntries(snapshotsInProgress.forRepo(repoName));
+                                inFlightShardStates = InFlightShardSnapshotStates.forEntries(
+                                    snapshotsInProgress.forRepo(projectId, repoName)
+                                );
                             } else {
                                 // no need to compute these, we'll mark all shards as queued anyway because we wait for the delete
                                 inFlightShardStates = null;
@@ -563,7 +588,7 @@ public ClusterState execute(ClusterState currentState) {
                         updatedEntries.add(updatedEntry);
                         return updateWithSnapshots(
                             currentState,
-                            snapshotsInProgress.createCopyWithUpdatedEntriesForRepo(repoName, updatedEntries),
+                            snapshotsInProgress.createCopyWithUpdatedEntriesForRepo(projectId, repoName, updatedEntries),
                             null
                         );
                     }
@@ -607,6 +632,7 @@ public String toString() {
         );
     }
 
+    // RepositoryShardId is unique across projects because RepositoryShardId.index().getId() is an UUID
     private final Set currentlyCloning = Collections.synchronizedSet(new HashSet<>());
 
     private void runReadyClone(
@@ -677,6 +703,7 @@ private void runReadyClone(
         }
     }
 
+    // This is limit is cluster wide across all projects
     private void ensureBelowConcurrencyLimit(
         String repository,
         String name,
@@ -702,8 +729,8 @@ private void ensureBelowConcurrencyLimit(
     /**
      * 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 (RepositoriesMetadata.get(state).repository(repoName) == null) {
+    public static void ensureRepositoryExists(String repoName, ProjectMetadata projectMetadata) {
+        if (RepositoriesMetadata.get(projectMetadata).repository(repoName) == null) {
             throw new RepositoryMissingException(repoName);
         }
     }
@@ -713,10 +740,10 @@ public static void ensureRepositoryExists(String repoName, ClusterState state) {
      *
      * @param repositoryName repository name
      * @param snapshotName snapshot name
-     * @param state   current cluster state
+     * @param projectMetadata   current project metadata
      */
-    private static void validate(String repositoryName, String snapshotName, ClusterState state) {
-        if (RepositoriesMetadata.get(state).repository(repositoryName) == null) {
+    private static void validate(String repositoryName, String snapshotName, ProjectMetadata projectMetadata) {
+        if (RepositoriesMetadata.get(projectMetadata).repository(repositoryName) == null) {
             throw new RepositoryMissingException(repositoryName);
         }
         validate(repositoryName, snapshotName);
@@ -774,9 +801,8 @@ private static UpdatedShardGenerations buildGenerations(SnapshotsInProgress.Entr
         return new UpdatedShardGenerations(builder.build(), deletedBuilder == null ? ShardGenerations.EMPTY : deletedBuilder.build());
     }
 
-    private static Metadata metadataForSnapshot(SnapshotsInProgress.Entry snapshot, Metadata metadata) {
-        @FixForMultiProject
-        final ProjectMetadata snapshotProject = projectForSnapshot(snapshot, metadata.getProject());
+    private static Metadata metadataForSnapshot(SnapshotsInProgress.Entry snapshot, Metadata metadata, ProjectId projectId) {
+        final ProjectMetadata snapshotProject = projectForSnapshot(snapshot, metadata.getProject(projectId));
         final Metadata.Builder builder;
         if (snapshot.includeGlobalState() == false) {
             // Remove global state from the cluster state
@@ -830,12 +856,14 @@ private static ProjectMetadata projectForSnapshot(SnapshotsInProgress.Entry snap
      * 
      *
      * @param snapshotsInProgress snapshots in progress in the cluster state
+     * @param projectId           project to look for the repository
      * @param repository          repository id
      * @param snapshots           list of snapshots that will be used as a filter, empty list means no snapshots are filtered
      * @return list of metadata for currently running snapshots
      */
     public static List currentSnapshots(
         @Nullable SnapshotsInProgress snapshotsInProgress,
+        ProjectId projectId,
         String repository,
         List snapshots
     ) {
@@ -846,10 +874,10 @@ public static List currentSnapshots(
             return snapshotsInProgress.asStream().toList();
         }
         if (snapshots.isEmpty()) {
-            return snapshotsInProgress.forRepo(repository);
+            return snapshotsInProgress.forRepo(projectId, repository);
         }
         List builder = new ArrayList<>();
-        for (SnapshotsInProgress.Entry entry : snapshotsInProgress.forRepo(repository)) {
+        for (SnapshotsInProgress.Entry entry : snapshotsInProgress.forRepo(projectId, repository)) {
             for (String snapshot : snapshots) {
                 if (entry.snapshot().getSnapshotId().getName().equals(snapshot)) {
                     builder.add(entry);
@@ -1007,7 +1035,6 @@ private void processExternalChanges(boolean changedNodes, boolean changedShards)
 
             @Override
             public ClusterState execute(ClusterState currentState) {
-                RoutingTable routingTable = currentState.routingTable();
                 final SnapshotsInProgress snapshotsInProgress = SnapshotsInProgress.get(currentState);
                 final SnapshotDeletionsInProgress deletesInProgress = SnapshotDeletionsInProgress.get(currentState);
                 DiscoveryNodes nodes = currentState.nodes();
@@ -1035,6 +1062,7 @@ public ClusterState execute(ClusterState currentState) {
                     boolean changed = false;
                     final List updatedEntriesForRepo = new ArrayList<>();
                     final Map knownFailures = new HashMap<>();
+                    final ProjectId projectId = snapshotsInRepo.get(0).projectId();
                     final String repositoryName = snapshotsInRepo.get(0).repository();
                     for (SnapshotsInProgress.Entry snapshotEntry : snapshotsInRepo) {
                         if (statesToUpdate.contains(snapshotEntry.state())) {
@@ -1049,7 +1077,7 @@ public ClusterState execute(ClusterState currentState) {
                                     }
                                 } else {
                                     // see if any clones may have had a shard become available for execution because of failures
-                                    if (deletesInProgress.hasExecutingDeletion(repositoryName)) {
+                                    if (deletesInProgress.hasExecutingDeletion(projectId, repositoryName)) {
                                         // Currently executing a delete for this repo, no need to try and update any clone operations.
                                         // The logic for finishing the delete will update running clones with the latest changes.
                                         updatedEntriesForRepo.add(snapshotEntry);
@@ -1099,7 +1127,7 @@ public ClusterState execute(ClusterState currentState) {
 
                                 ImmutableOpenMap shards = processWaitingShardsAndRemovedNodes(
                                     snapshotEntry,
-                                    routingTable,
+                                    currentState.routingTable(projectId),
                                     nodes,
                                     snapshotsInProgress::isNodeIdForRemoval,
                                     knownFailures
@@ -1131,7 +1159,11 @@ public ClusterState execute(ClusterState currentState) {
                         }
                     }
                     if (changed) {
-                        updatedSnapshots = updatedSnapshots.createCopyWithUpdatedEntriesForRepo(repositoryName, updatedEntriesForRepo);
+                        updatedSnapshots = updatedSnapshots.createCopyWithUpdatedEntriesForRepo(
+                            projectId,
+                            repositoryName,
+                            updatedEntriesForRepo
+                        );
                     }
                 }
                 final ClusterState res = readyDeletions(
@@ -1176,7 +1208,7 @@ public void clusterStateProcessed(ClusterState oldState, ClusterState newState)
                 startExecutableClones(SnapshotsInProgress.get(newState), null);
                 // run newly ready deletes
                 for (SnapshotDeletionsInProgress.Entry entry : deletionsToExecute) {
-                    if (tryEnterRepoLoop(entry.repository())) {
+                    if (tryEnterRepoLoop(entry.projectId(), entry.repository())) {
                         deleteSnapshotsFromRepository(entry, newState.nodes().getMaxDataNodeCompatibleIndexVersion());
                     }
                 }
@@ -1306,6 +1338,7 @@ private static boolean waitingShardsStartedOrUnassigned(SnapshotsInProgress snap
         for (List entries : snapshotsInProgress.entriesByRepo()) {
             for (SnapshotsInProgress.Entry entry : entries) {
                 if (entry.state() == SnapshotsInProgress.State.STARTED && entry.isClone() == false) {
+                    final ProjectId projectId = entry.projectId();
                     for (Map.Entry shardStatus : entry.shardSnapshotStatusByRepoShardId()
                         .entrySet()) {
                         final ShardState state = shardStatus.getValue().state();
@@ -1315,7 +1348,7 @@ private static boolean waitingShardsStartedOrUnassigned(SnapshotsInProgress snap
                         final RepositoryShardId shardId = shardStatus.getKey();
                         final Index index = entry.indexByName(shardId.indexName());
                         if (event.indexRoutingTableChanged(index)) {
-                            IndexRoutingTable indexShardRoutingTable = event.state().getRoutingTable().index(index);
+                            IndexRoutingTable indexShardRoutingTable = event.state().routingTable(projectId).index(index);
                             if (indexShardRoutingTable == null) {
                                 // index got removed concurrently and we have to fail WAITING, QUEUED and PAUSED_FOR_REMOVAL state shards
                                 return true;
@@ -1374,10 +1407,11 @@ private void endSnapshot(SnapshotsInProgress.Entry entry, Metadata metadata, @Nu
             }
             return;
         }
+        final ProjectId projectId = snapshot.getProjectId();
         final String repoName = snapshot.getRepository();
-        if (tryEnterRepoLoop(repoName)) {
+        if (tryEnterRepoLoop(projectId, repoName)) {
             if (repositoryData == null) {
-                repositoriesService.repository(snapshot.getProjectId(), repoName)
+                repositoriesService.repository(projectId, repoName)
                     .getRepositoryData(
                         EsExecutors.DIRECT_EXECUTOR_SERVICE, // TODO contemplate threading here, do we need to fork, see #101445?
                         new ActionListener<>() {
@@ -1386,13 +1420,16 @@ public void onResponse(RepositoryData repositoryData) {
                                 if (newFinalization) {
                                     finalizeSnapshotEntry(snapshot, metadata, repositoryData);
                                 } else {
-                                    runNextQueuedOperation(repositoryData, repoName, false);
+                                    runNextQueuedOperation(repositoryData, projectId, repoName, false);
                                 }
                             }
 
                             @Override
                             public void onFailure(Exception e) {
-                                submitUnbatchedTask("fail repo tasks for [" + repoName + "]", new FailPendingRepoTasksTask(repoName, e));
+                                submitUnbatchedTask(
+                                    "fail repo tasks for [" + repoName + "]",
+                                    new FailPendingRepoTasksTask(projectId, repoName, e)
+                                );
                             }
                         }
                     );
@@ -1400,7 +1437,7 @@ public void onFailure(Exception e) {
                 if (newFinalization) {
                     finalizeSnapshotEntry(snapshot, metadata, repositoryData);
                 } else {
-                    runNextQueuedOperation(repositoryData, repoName, false);
+                    runNextQueuedOperation(repositoryData, projectId, repoName, false);
                 }
             }
         } else {
@@ -1413,21 +1450,21 @@ public void onFailure(Exception e) {
     /**
      * Try starting to run a snapshot finalization or snapshot delete for the given repository. If this method returns
      * {@code true} then snapshot finalizations and deletions for the repo may be executed. Once no more operations are
-     * ready for the repository {@link #leaveRepoLoop(String)} should be invoked so that a subsequent state change that
+     * ready for the repository {@link #leaveRepoLoop(ProjectId, String)} should be invoked so that a subsequent state change that
      * causes another operation to become ready can execute.
      *
      * @return true if a finalization or snapshot delete may be started at this point
      */
-    private boolean tryEnterRepoLoop(String repository) {
-        return currentlyFinalizing.add(repository);
+    private boolean tryEnterRepoLoop(ProjectId projectId, String repository) {
+        return currentlyFinalizing.add(new ProjectRepo(projectId, repository));
     }
 
     /**
      * Stop polling for ready snapshot finalizations or deletes in state {@link SnapshotDeletionsInProgress.State#STARTED} to execute
      * for the given repository.
      */
-    private void leaveRepoLoop(String repository) {
-        final boolean removed = currentlyFinalizing.remove(repository);
+    private void leaveRepoLoop(ProjectId projectId, String repository) {
+        final boolean removed = currentlyFinalizing.remove(new ProjectRepo(projectId, repository));
         assert removed;
     }
 
@@ -1445,6 +1482,7 @@ private void finalizeSnapshotEntry(Snapshot snapshot, Metadata metadata, Reposit
     private class SnapshotFinalization extends AbstractRunnable {
 
         private final Snapshot snapshot;
+        @FixForMultiProject(description = "replace with ProjectMetadata")
         private final Metadata metadata;
         private final RepositoryData repositoryData;
 
@@ -1457,7 +1495,7 @@ private class SnapshotFinalization extends AbstractRunnable {
         @Override
         protected void doRun() {
             assert ThreadPool.assertCurrentThreadPool(ThreadPool.Names.SNAPSHOT);
-            assert currentlyFinalizing.contains(snapshot.getRepository());
+            assert currentlyFinalizing.contains(new ProjectRepo(snapshot.getProjectId(), snapshot.getRepository()));
             assert repositoryOperations.assertNotQueued(snapshot);
 
             SnapshotsInProgress.Entry entry = SnapshotsInProgress.get(clusterService.state()).snapshot(snapshot);
@@ -1484,14 +1522,14 @@ protected void doRun() {
                     assert state == ShardState.SUCCESS;
                 }
             }
+            final ProjectId projectId = snapshot.getProjectId();
             final String repository = snapshot.getRepository();
             final ListenableFuture metadataListener = new ListenableFuture<>();
-            final Repository repo = repositoriesService.repository(snapshot.getProjectId(), snapshot.getRepository());
+            final Repository repo = repositoriesService.repository(projectId, repository);
             if (entry.isClone()) {
                 threadPool.executor(ThreadPool.Names.SNAPSHOT).execute(ActionRunnable.supply(metadataListener, () -> {
                     final Metadata existingMetadata = repo.getSnapshotGlobalMetadata(entry.source());
-                    @FixForMultiProject
-                    final ProjectMetadata existingProject = existingMetadata.getProject();
+                    final ProjectMetadata existingProject = existingMetadata.getProject(projectId);
                     final ProjectMetadata.Builder projBuilder = ProjectMetadata.builder(existingProject);
                     final Set existingIndices = new HashSet<>();
                     for (IndexId index : entry.indices().values()) {
@@ -1518,7 +1556,7 @@ protected void doRun() {
             }
             metadataListener.addListener(ActionListener.wrap(meta -> {
                 assert ThreadPool.assertCurrentThreadPool(ThreadPool.Names.SNAPSHOT);
-                final Metadata metaForSnapshot = metadataForSnapshot(entry, meta);
+                final Metadata metaForSnapshot = metadataForSnapshot(entry, meta, projectId);
 
                 final Map indexSnapshotDetails = Maps.newMapWithExpectedSize(
                     finalIndices.size()
@@ -1558,7 +1596,7 @@ protected void doRun() {
                 final SnapshotInfo snapshotInfo = new SnapshotInfo(
                     snapshot,
                     finalIndices,
-                    entry.dataStreams().stream().filter(metaForSnapshot.getProject().dataStreams()::containsKey).toList(),
+                    entry.dataStreams().stream().filter(metaForSnapshot.getProject(projectId).dataStreams()::containsKey).toList(),
                     entry.partial() ? onlySuccessfulFeatureStates(entry, finalIndices) : entry.featureStates(),
                     failure,
                     threadPool.absoluteTimeInMillis(),
@@ -1582,7 +1620,7 @@ protected void doRun() {
                             // by the snapshot info callback below and won't be failed needlessly if #runNextQueuedOperation runs into
                             // a fatal like e.g. this node stopped being the master node
                             snapshotListeners.onResponse(endAndGetListenersToResolve(snapshot));
-                            runNextQueuedOperation(updatedRepositoryData, repository, true);
+                            runNextQueuedOperation(updatedRepositoryData, projectId, repository, true);
                         },
                             e -> handleFinalizationFailure(
                                 e,
@@ -1711,18 +1749,19 @@ private void handleFinalizationFailure(
      * Run the next queued up repository operation for the given repository name.
      *
      * @param repositoryData current repository data
+     * @param projectId      project for the repository
      * @param repository     repository name
      * @param attemptDelete  whether to try and run delete operations that are ready in the cluster state if no
      *                       snapshot create operations remain to execute
      */
-    private void runNextQueuedOperation(RepositoryData repositoryData, String repository, boolean attemptDelete) {
-        assert currentlyFinalizing.contains(repository);
-        final Tuple nextFinalization = repositoryOperations.pollFinalization(repository);
+    private void runNextQueuedOperation(RepositoryData repositoryData, ProjectId projectId, String repository, boolean attemptDelete) {
+        assert currentlyFinalizing.contains(new ProjectRepo(projectId, repository));
+        final Tuple nextFinalization = repositoryOperations.pollFinalization(projectId, repository);
         if (nextFinalization == null) {
             if (attemptDelete) {
-                runReadyDeletions(repositoryData, repository);
+                runReadyDeletions(repositoryData, projectId, repository);
             } else {
-                leaveRepoLoop(repository);
+                leaveRepoLoop(projectId, repository);
             }
         } else {
             logger.trace("Moving on to finalizing next snapshot [{}]", nextFinalization);
@@ -1735,7 +1774,7 @@ private void runNextQueuedOperation(RepositoryData repositoryData, String reposi
      *
      * TODO: optimize this to execute in a single CS update together with finalizing the latest snapshot
      */
-    private void runReadyDeletions(RepositoryData repositoryData, String repository) {
+    private void runReadyDeletions(RepositoryData repositoryData, ProjectId projectId, String repository) {
         submitUnbatchedTask("Run ready deletions", new ClusterStateUpdateTask() {
 
             private SnapshotDeletionsInProgress.Entry deletionToRun;
@@ -1745,7 +1784,9 @@ 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 : SnapshotDeletionsInProgress.get(currentState).getEntries()) {
-                    if (entry.repository().equals(repository) && entry.state() == SnapshotDeletionsInProgress.State.STARTED) {
+                    if (entry.projectId().equals(projectId)
+                        && entry.repository().equals(repository)
+                        && entry.state() == SnapshotDeletionsInProgress.State.STARTED) {
                         deletionToRun = entry;
                         break;
                     }
@@ -1762,7 +1803,7 @@ public void onFailure(Exception e) {
             @Override
             public void clusterStateProcessed(ClusterState oldState, ClusterState newState) {
                 if (deletionToRun == null) {
-                    runNextQueuedOperation(repositoryData, repository, false);
+                    runNextQueuedOperation(repositoryData, projectId, repository, false);
                 } else {
                     deleteSnapshotsFromRepository(deletionToRun, repositoryData, newState.nodes().getMaxDataNodeCompatibleIndexVersion());
                 }
@@ -1787,15 +1828,15 @@ private static Tuple> read
         }
         final SnapshotsInProgress snapshotsInProgress = currentState.custom(SnapshotsInProgress.TYPE);
         assert snapshotsInProgress != null;
-        final Set repositoriesSeen = new HashSet<>();
+        final Set repositoriesSeen = new HashSet<>();
         boolean changed = false;
         final ArrayList readyDeletions = new ArrayList<>();
         final List newDeletes = new ArrayList<>();
         for (SnapshotDeletionsInProgress.Entry entry : deletions.getEntries()) {
-            final String repo = entry.repository();
-            if (repositoriesSeen.add(entry.repository())
+            final var projectRepo = new ProjectRepo(entry.projectId(), entry.repository());
+            if (repositoriesSeen.add(projectRepo)
                 && entry.state() == SnapshotDeletionsInProgress.State.WAITING
-                && snapshotsInProgress.forRepo(repo).stream().noneMatch(SnapshotsService::isWritingToRepository)) {
+                && snapshotsInProgress.forRepo(projectRepo).stream().noneMatch(SnapshotsService::isWritingToRepository)) {
                 changed = true;
                 final SnapshotDeletionsInProgress.Entry newEntry = entry.started();
                 readyDeletions.add(newEntry);
@@ -1836,7 +1877,7 @@ public static ClusterState stateWithoutSnapshot(
         ClusterState result = state;
         int indexOfEntry = -1;
         // Find the in-progress snapshot entry that matches {@code snapshot}.
-        final List entryList = inProgressSnapshots.forRepo(snapshot.getRepository());
+        final List entryList = inProgressSnapshots.forRepo(snapshot.getProjectId(), snapshot.getRepository());
         for (int i = 0; i < entryList.size(); i++) {
             SnapshotsInProgress.Entry entry = entryList.get(i);
             if (entry.snapshot().equals(snapshot)) {
@@ -1935,7 +1976,11 @@ public static ClusterState stateWithoutSnapshot(
             result = ClusterState.builder(state)
                 .putCustom(
                     SnapshotsInProgress.TYPE,
-                    inProgressSnapshots.createCopyWithUpdatedEntriesForRepo(snapshot.getRepository(), updatedEntries)
+                    inProgressSnapshots.createCopyWithUpdatedEntriesForRepo(
+                        snapshot.getProjectId(),
+                        snapshot.getRepository(),
+                        updatedEntries
+                    )
                 )
                 .build();
         }
@@ -2023,6 +2068,7 @@ public ClusterState execute(ClusterState currentState) {
                     deletionsWithoutSnapshots(
                         SnapshotDeletionsInProgress.get(updatedState),
                         Collections.singletonList(snapshot.getSnapshotId()),
+                        snapshot.getProjectId(),
                         snapshot.getRepository()
                     )
                 );
@@ -2050,7 +2096,7 @@ public void onFailure(Exception e) {
             public void clusterStateProcessed(ClusterState oldState, ClusterState newState) {
                 failSnapshotCompletionListeners(snapshot, failure, Runnable::run);
                 if (repositoryData != null) {
-                    runNextQueuedOperation(repositoryData, snapshot.getRepository(), true);
+                    runNextQueuedOperation(repositoryData, snapshot.getProjectId(), snapshot.getRepository(), true);
                 }
             }
         });
@@ -2064,6 +2110,7 @@ public void clusterStateProcessed(ClusterState oldState, ClusterState newState)
      *
      * @param deletions   snapshot deletions to update
      * @param snapshotIds snapshot ids to remove
+     * @param projectId   project for the repository
      * @param repository  repository that the snapshot ids belong to
      * @return            updated {@link SnapshotDeletionsInProgress} or {@code null} if unchanged
      */
@@ -2071,12 +2118,13 @@ public void clusterStateProcessed(ClusterState oldState, ClusterState newState)
     private static SnapshotDeletionsInProgress deletionsWithoutSnapshots(
         SnapshotDeletionsInProgress deletions,
         Collection snapshotIds,
+        ProjectId projectId,
         String repository
     ) {
         boolean changed = false;
         List updatedEntries = new ArrayList<>(deletions.getEntries().size());
         for (SnapshotDeletionsInProgress.Entry entry : deletions.getEntries()) {
-            if (entry.repository().equals(repository)) {
+            if (entry.projectId().equals(projectId) && entry.repository().equals(repository)) {
                 final List updatedSnapshotIds = new ArrayList<>(entry.snapshots());
                 if (updatedSnapshotIds.removeAll(snapshotIds)) {
                     changed = true;
@@ -2103,14 +2151,15 @@ private void failSnapshotCompletionListeners(Snapshot snapshot, Exception e, Con
      * When wait_for_completion is set to true, the passed action listener will only complete when all
      * matching snapshots are deleted, when it is false it will complete as soon as the deletes are scheduled
      *
+     * @param projectId       project to look for the snapshot
      * @param request         delete snapshot request
      * @param listener        listener a listener which will be resolved according to the wait_for_completion parameter
      */
-    public void deleteSnapshots(final DeleteSnapshotRequest request, final ActionListener listener) {
+    public void deleteSnapshots(final ProjectId projectId, final DeleteSnapshotRequest request, final ActionListener listener) {
         final String repositoryName = request.repository();
         final String[] snapshotNames = request.snapshots();
 
-        final Repository repository = repositoriesService.repository(repositoryName);
+        final Repository repository = repositoriesService.repository(projectId, repositoryName);
         executeConsistentStateUpdate(repository, repositoryData -> new ClusterStateUpdateTask(request.masterNodeTimeout()) {
 
             private SnapshotDeletionsInProgress.Entry newDelete = null;
@@ -2127,12 +2176,13 @@ public void deleteSnapshots(final DeleteSnapshotRequest request, final ActionLis
 
             @Override
             public ClusterState execute(ClusterState currentState) {
-                ensureRepositoryExists(repositoryName, currentState);
+                final var projectMetadata = currentState.metadata().getProject(projectId);
+                ensureRepositoryExists(repositoryName, projectMetadata);
                 final Set snapshotIds = new HashSet<>();
 
                 // find in-progress snapshots to delete in cluster state
                 final SnapshotsInProgress snapshotsInProgress = SnapshotsInProgress.get(currentState);
-                for (SnapshotsInProgress.Entry entry : snapshotsInProgress.forRepo(repositoryName)) {
+                for (SnapshotsInProgress.Entry entry : snapshotsInProgress.forRepo(projectId, repositoryName)) {
                     final SnapshotId snapshotId = entry.snapshot().getSnapshotId();
                     if (Regex.simpleMatch(snapshotNames, snapshotId.getName())) {
                         snapshotIds.add(snapshotId);
@@ -2170,12 +2220,13 @@ public ClusterState execute(ClusterState currentState) {
 
                 final Set activeCloneSources = snapshotsInProgress.asStream()
                     .filter(SnapshotsInProgress.Entry::isClone)
+                    .filter(entry -> entry.projectId().equals(projectId))
                     .map(SnapshotsInProgress.Entry::source)
                     .collect(Collectors.toSet());
                 for (SnapshotId snapshotId : snapshotIds) {
                     if (activeCloneSources.contains(snapshotId)) {
                         throw new ConcurrentSnapshotExecutionException(
-                            new Snapshot(repositoryName, snapshotId),
+                            new Snapshot(projectId, repositoryName, snapshotId),
                             "cannot delete snapshot while it is being cloned"
                         );
                     }
@@ -2188,7 +2239,7 @@ public ClusterState execute(ClusterState currentState) {
                     "delete snapshot"
                 );
 
-                ensureNotReadOnly(currentState, repositoryName);
+                ensureNotReadOnly(projectMetadata, repositoryName);
 
                 final SnapshotDeletionsInProgress deletionsInProgress = SnapshotDeletionsInProgress.get(currentState);
 
@@ -2200,7 +2251,7 @@ public ClusterState execute(ClusterState currentState) {
                 for (RestoreInProgress.Entry entry : restoreInProgress) {
                     if (repositoryName.equals(entry.snapshot().getRepository()) && snapshotIds.contains(entry.snapshot().getSnapshotId())) {
                         throw new ConcurrentSnapshotExecutionException(
-                            new Snapshot(repositoryName, snapshotIds.stream().findFirst().get()),
+                            new Snapshot(projectId, repositoryName, snapshotIds.stream().findFirst().get()),
                             "cannot delete snapshot during a restore in progress in [" + restoreInProgress + "]"
                         );
                     }
@@ -2208,8 +2259,9 @@ public ClusterState execute(ClusterState currentState) {
                 // Snapshot ids that will have to be physically deleted from the repository
                 final Set snapshotIdsRequiringCleanup = new HashSet<>(snapshotIds);
                 final SnapshotsInProgress updatedSnapshots = snapshotsInProgress.createCopyWithUpdatedEntriesForRepo(
+                    projectId,
                     repositoryName,
-                    snapshotsInProgress.forRepo(repositoryName).stream().map(existing -> {
+                    snapshotsInProgress.forRepo(projectId, repositoryName).stream().map(existing -> {
                         if (existing.state() == SnapshotsInProgress.State.STARTED
                             && snapshotIdsRequiringCleanup.contains(existing.snapshot().getSnapshotId())) {
                             // snapshot is started - mark every non completed shard as aborted
@@ -2238,7 +2290,7 @@ public ClusterState execute(ClusterState currentState) {
                 // add the snapshot deletion to the cluster state
                 final SnapshotDeletionsInProgress.Entry replacedEntry = deletionsInProgress.getEntries()
                     .stream()
-                    .filter(entry -> entry.repository().equals(repositoryName))
+                    .filter(entry -> entry.projectId().equals(projectId) && entry.repository().equals(repositoryName))
                     .filter(entry -> entry.state() == SnapshotDeletionsInProgress.State.WAITING)
                     .findFirst()
                     .orElse(null);
@@ -2246,7 +2298,8 @@ public ClusterState execute(ClusterState currentState) {
                     final Optional foundDuplicate = deletionsInProgress.getEntries()
                         .stream()
                         .filter(
-                            entry -> entry.repository().equals(repositoryName)
+                            entry -> entry.projectId().equals(projectId)
+                                && entry.repository().equals(repositoryName)
                                 && entry.state() == SnapshotDeletionsInProgress.State.STARTED
                                 && entry.snapshots().containsAll(snapshotIds)
                         )
@@ -2256,16 +2309,14 @@ public ClusterState execute(ClusterState currentState) {
                         reusedExistingDelete = true;
                         return currentState;
                     }
-                    @FixForMultiProject
-                    final var projectId = ProjectId.DEFAULT;
                     newDelete = new SnapshotDeletionsInProgress.Entry(
                         projectId,
                         repositoryName,
                         List.copyOf(snapshotIdsRequiringCleanup),
                         threadPool.absoluteTimeInMillis(),
                         repositoryData.getGenId(),
-                        updatedSnapshots.forRepo(repositoryName).stream().noneMatch(SnapshotsService::isWritingToRepository)
-                            && deletionsInProgress.hasExecutingDeletion(repositoryName) == false
+                        updatedSnapshots.forRepo(projectId, repositoryName).stream().noneMatch(SnapshotsService::isWritingToRepository)
+                            && deletionsInProgress.hasExecutingDeletion(projectId, repositoryName) == false
                                 ? SnapshotDeletionsInProgress.State.STARTED
                                 : SnapshotDeletionsInProgress.State.WAITING
                     );
@@ -2312,7 +2363,7 @@ public void clusterStateProcessed(ClusterState oldState, ClusterState newState)
                         return;
                     }
                     if (newDelete.state() == SnapshotDeletionsInProgress.State.STARTED) {
-                        if (tryEnterRepoLoop(repositoryName)) {
+                        if (tryEnterRepoLoop(projectId, repositoryName)) {
                             deleteSnapshotsFromRepository(
                                 newDelete,
                                 repositoryData,
@@ -2457,7 +2508,7 @@ public void onResponse(RepositoryData repositoryData) {
             public void onFailure(Exception e) {
                 submitUnbatchedTask(
                     "fail repo tasks for [" + deleteEntry.repository() + "]",
-                    new FailPendingRepoTasksTask(deleteEntry.repository(), e)
+                    new FailPendingRepoTasksTask(deleteEntry.projectId(), deleteEntry.repository(), e)
                 );
             }
         });
@@ -2505,8 +2556,9 @@ public ClusterState execute(ClusterState currentState) throws Exception {
                         // updated repository data and state on the retry. We don't want to wait for the write to finish though
                         // 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.
+                        final ProjectMetadata projectMetadata = currentState.metadata().getProject(repository.getProjectId());
                         if (repositoryMetadataStart.equals(
-                            RepositoriesMetadata.get(currentState).repository(repository.getMetadata().name())
+                            RepositoriesMetadata.get(projectMetadata).repository(repository.getMetadata().name())
                         )) {
                             executedTask = true;
                             return updateTask.execute(currentState);
@@ -2548,7 +2600,7 @@ private void deleteSnapshotsFromRepository(
         IndexVersion minNodeVersion
     ) {
         if (repositoryOperations.startDeletion(deleteEntry.uuid())) {
-            assert currentlyFinalizing.contains(deleteEntry.repository());
+            assert currentlyFinalizing.contains(new ProjectRepo(deleteEntry.projectId(), deleteEntry.repository()));
             final List snapshotIds = deleteEntry.snapshots();
             assert deleteEntry.state() == SnapshotDeletionsInProgress.State.STARTED : "incorrect state for entry [" + deleteEntry + "]";
             if (snapshotIds.isEmpty()) {
@@ -2645,6 +2697,7 @@ protected SnapshotDeletionsInProgress filterDeletions(SnapshotDeletionsInProgres
                 final SnapshotDeletionsInProgress updatedDeletions = deletionsWithoutSnapshots(
                     deletions,
                     deleteEntry.snapshots(),
+                    deleteEntry.projectId(),
                     deleteEntry.repository()
                 );
                 return updatedDeletions == null ? deletions : updatedDeletions;
@@ -2761,7 +2814,7 @@ public final void clusterStateProcessed(ClusterState oldState, ClusterState newS
             readyToResolveListeners.forEach(Runnable::run);
             if (newFinalizations.isEmpty()) {
                 if (readyDeletions.isEmpty()) {
-                    leaveRepoLoop(deleteEntry.repository());
+                    leaveRepoLoop(deleteEntry.projectId(), deleteEntry.repository());
                 } else {
                     for (SnapshotDeletionsInProgress.Entry readyDeletion : readyDeletions) {
                         deleteSnapshotsFromRepository(
@@ -2772,7 +2825,7 @@ public final void clusterStateProcessed(ClusterState oldState, ClusterState newS
                     }
                 }
             } else {
-                leaveRepoLoop(deleteEntry.repository());
+                leaveRepoLoop(deleteEntry.projectId(), deleteEntry.repository());
                 assert readyDeletions.stream().noneMatch(entry -> entry.repository().equals(deleteEntry.repository()))
                     : "New finalizations " + newFinalizations + " added even though deletes " + readyDeletions + " are ready";
                 for (SnapshotsInProgress.Entry entry : newFinalizations) {
@@ -2821,12 +2874,13 @@ private SnapshotsInProgress updatedSnapshotsInProgress(ClusterState currentState
             boolean changed = false;
 
             final String localNodeId = currentState.nodes().getLocalNodeId();
+            final ProjectId projectId = deleteEntry.projectId();
             final String repoName = deleteEntry.repository();
             InFlightShardSnapshotStates inFlightShardStates = null;
             // Keep track of IndexId values that may have gone unreferenced due to the delete entry just executed.
             // See org.elasticsearch.cluster.SnapshotsInProgress.Entry#withUpdatedIndexIds for details.
             final Set newIndexIdsToRefresh = new HashSet<>();
-            for (SnapshotsInProgress.Entry entry : snapshotsInProgress.forRepo(repoName)) {
+            for (SnapshotsInProgress.Entry entry : snapshotsInProgress.forRepo(projectId, repoName)) {
                 if (entry.state().completed() == false) {
                     // TODO: dry up redundant computation and code between clone and non-clone case, in particular reuse
                     // `inFlightShardStates` across both clone and standard snapshot code
@@ -2842,12 +2896,14 @@ private SnapshotsInProgress updatedSnapshotsInProgress(ClusterState currentState
                         }
                         // TODO: the below logic is very similar to that in #startCloning and both could be dried up against each other
                         // also the code for standard snapshots could make use of this breakout as well
-                        if (canBeUpdated.isEmpty() || updatedDeletions.hasExecutingDeletion(repoName)) {
+                        if (canBeUpdated.isEmpty() || updatedDeletions.hasExecutingDeletion(projectId, repoName)) {
                             // No shards can be updated in this snapshot so we just add it as is again
                             snapshotEntries.add(entry);
                         } else {
                             if (inFlightShardStates == null) {
-                                inFlightShardStates = InFlightShardSnapshotStates.forEntries(snapshotsInProgress.forRepo(repoName));
+                                inFlightShardStates = InFlightShardSnapshotStates.forEntries(
+                                    snapshotsInProgress.forRepo(projectId, repoName)
+                                );
                             }
                             final ImmutableOpenMap.Builder updatedAssignmentsBuilder =
                                 ImmutableOpenMap.builder(entry.shardSnapshotStatusByRepoShardId());
@@ -2891,7 +2947,7 @@ private SnapshotsInProgress updatedSnapshotsInProgress(ClusterState currentState
                             final ImmutableOpenMap shardAssignments = shards(
                                 snapshotsInProgress,
                                 updatedDeletions,
-                                currentState,
+                                currentState.projectState(projectId),
                                 entry.indices().values(),
                                 entry.version().onOrAfter(SHARD_GEN_IN_REPO_DATA_VERSION),
                                 repositoryData,
@@ -2904,7 +2960,7 @@ private SnapshotsInProgress updatedSnapshotsInProgress(ClusterState currentState
                                 final ShardSnapshotStatus updated = shardAssignments.get(sid);
                                 if (updated == null) {
                                     // We don't have a new assignment for this shard because its index was concurrently deleted
-                                    assert currentState.routingTable().hasIndex(sid.getIndex()) == false
+                                    assert currentState.routingTable(projectId).hasIndex(sid.getIndex()) == false
                                         : "Missing assignment for [" + sid + "]";
                                     updatedAssignmentsBuilder.put(sid, ShardSnapshotStatus.MISSING);
                                 } else {
@@ -2936,7 +2992,7 @@ private SnapshotsInProgress updatedSnapshotsInProgress(ClusterState currentState
                 }
                 snapshotEntries.replaceAll(entry -> entry.withUpdatedIndexIds(updatedIndexIds));
             }
-            return changed ? snapshotsInProgress.createCopyWithUpdatedEntriesForRepo(repoName, snapshotEntries) : null;
+            return changed ? snapshotsInProgress.createCopyWithUpdatedEntriesForRepo(projectId, repoName, snapshotEntries) : null;
         }
 
         private static void markShardReassigned(RepositoryShardId shardId, Set reassignments) {
@@ -3010,7 +3066,7 @@ private static  void completeListenersIgnoringException(@Nullable List shards(
         SnapshotsInProgress snapshotsInProgress,
         SnapshotDeletionsInProgress deletionsInProgress,
-        ClusterState currentState,
+        ProjectState currentState,
         Collection indices,
         boolean useShardGenerations,
         RepositoryData repositoryData,
@@ -3019,13 +3075,13 @@ private static ImmutableOpenMap builder = ImmutableOpenMap.builder();
         final ShardGenerations shardGenerations = repositoryData.shardGenerations();
         final InFlightShardSnapshotStates inFlightShardStates = InFlightShardSnapshotStates.forEntries(
-            snapshotsInProgress.forRepo(repoName)
+            snapshotsInProgress.forRepo(currentState.projectId(), repoName)
         );
-        final boolean readyToExecute = deletionsInProgress.hasExecutingDeletion(repoName) == false;
+        final boolean readyToExecute = deletionsInProgress.hasExecutingDeletion(currentState.projectId(), repoName) == false;
         for (IndexId index : indices) {
             final String indexName = index.getName();
             final boolean isNewIndex = repositoryData.getIndices().containsKey(indexName) == false;
-            IndexMetadata indexMetadata = currentState.metadata().getProject().index(indexName);
+            IndexMetadata indexMetadata = currentState.metadata().index(indexName);
             if (indexMetadata == null) {
                 // The index was deleted before we managed to start the snapshot - mark it as missing.
                 builder.put(new ShardId(indexName, IndexMetadata.INDEX_UUID_NA_VALUE, 0), ShardSnapshotStatus.MISSING);
@@ -3247,7 +3303,7 @@ static final class SnapshotShardsUpdateContext {
         private final Predicate nodeIdRemovalPredicate;
 
         /** Updates outstanding to be applied to existing snapshot entries. Maps repository name to shard snapshot updates. */
-        private final Map> updatesByRepo;
+        private final Map> updatesByRepo;
 
         /** Updates that were used to update an existing in-progress shard snapshot */
         private final Set executedUpdates = new HashSet<>();
@@ -3270,11 +3326,14 @@ static final class SnapshotShardsUpdateContext {
             this.nodeIdRemovalPredicate = SnapshotsInProgress.get(initialState)::isNodeIdForRemoval;
             this.completionHandler = completionHandler;
 
-            // SnapshotsInProgress is organized by repository name, so organize the shard snapshot updates similarly.
+            // SnapshotsInProgress is organized by ProjectRepo, so organize the shard snapshot updates similarly.
             this.updatesByRepo = new HashMap<>();
             for (final var taskContext : batchExecutionContext.taskContexts()) {
                 if (taskContext.getTask() instanceof ShardSnapshotUpdate task) {
-                    updatesByRepo.computeIfAbsent(task.snapshot.getRepository(), r -> new ArrayList<>()).add(task);
+                    updatesByRepo.computeIfAbsent(
+                        new ProjectRepo(task.snapshot.getProjectId(), task.snapshot.getRepository()),
+                        r -> new ArrayList<>()
+                    ).add(task);
                 }
             }
         }
@@ -3287,9 +3346,9 @@ static final class SnapshotShardsUpdateContext {
         SnapshotsInProgress computeUpdatedState() {
             final SnapshotsInProgress existing = SnapshotsInProgress.get(initialState);
             SnapshotsInProgress updated = existing;
-            for (Map.Entry> updates : updatesByRepo.entrySet()) {
-                final String repoName = updates.getKey();
-                final List oldEntries = existing.forRepo(repoName);
+            for (Map.Entry> updates : updatesByRepo.entrySet()) {
+                final var projectRepo = updates.getKey();
+                final List oldEntries = existing.forRepo(projectRepo);
                 if (oldEntries.isEmpty()) {
                     continue;
                 }
@@ -3303,7 +3362,7 @@ SnapshotsInProgress computeUpdatedState() {
                         newlyCompletedEntries.add(newEntry);
                     }
                 }
-                updated = updated.createCopyWithUpdatedEntriesForRepo(repoName, newEntries);
+                updated = updated.createCopyWithUpdatedEntriesForRepo(projectRepo.projectId(), projectRepo.name(), newEntries);
             }
 
             if (changedCount > 0) {
@@ -3604,7 +3663,7 @@ private void startShardSnapshot(RepositoryShardId repoShardId, ShardGeneration g
                         + "] because it's a normal snapshot but did not";
                 // work out the node to run the snapshot task on as it might have changed from the previous operation if it was a clone
                 // or there was a primary failover
-                final IndexRoutingTable indexRouting = initialState.routingTable().index(index);
+                final IndexRoutingTable indexRouting = initialState.routingTable(entry.projectId()).index(index);
                 final ShardRouting shardRouting;
                 if (indexRouting == null) {
                     shardRouting = null;
@@ -3649,14 +3708,14 @@ interface ShardSnapshotUpdateCompletionHandler {
         void handleCompletion(
             ShardSnapshotUpdateResult shardSnapshotUpdateResult,
             List newlyCompletedEntries,
-            Set updatedRepositories
+            Set updatedRepositories
         );
     }
 
     private void handleShardSnapshotUpdateCompletion(
         ShardSnapshotUpdateResult shardSnapshotUpdateResult,
         List newlyCompletedEntries,
-        Set updatedRepositories
+        Set updatedRepositories
     ) {
         // Maybe this state update completed one or more snapshots. If we are not already ending them because of some earlier update, end
         // them now.
@@ -3767,13 +3826,13 @@ private void innerUpdateSnapshotState(
         masterServiceTaskQueue.submitTask("update snapshot state", update, null);
     }
 
-    private void startExecutableClones(SnapshotsInProgress snapshotsInProgress, @Nullable String repoName) {
-        if (repoName == null) {
+    private void startExecutableClones(SnapshotsInProgress snapshotsInProgress, @Nullable ProjectRepo projectRepo) {
+        if (projectRepo == null) {
             for (List entries : snapshotsInProgress.entriesByRepo()) {
                 startExecutableClones(entries);
             }
         } else {
-            startExecutableClones(snapshotsInProgress.forRepo(repoName));
+            startExecutableClones(snapshotsInProgress.forRepo(projectRepo));
         }
     }
 
@@ -3856,9 +3915,11 @@ private final class FailPendingRepoTasksTask extends ClusterStateUpdateTask {
         // Failure that caused the decision to fail all snapshots and deletes for a repo
         private final Exception failure;
 
+        private final ProjectId projectId;
         private final String repository;
 
-        FailPendingRepoTasksTask(String repository, Exception failure) {
+        FailPendingRepoTasksTask(ProjectId projectId, String repository, Exception failure) {
+            this.projectId = projectId;
             this.repository = repository;
             this.failure = failure;
         }
@@ -3870,7 +3931,7 @@ public ClusterState execute(ClusterState currentState) {
             final List remainingEntries = deletionsInProgress.getEntries();
             List updatedEntries = new ArrayList<>(remainingEntries.size());
             for (SnapshotDeletionsInProgress.Entry entry : remainingEntries) {
-                if (entry.repository().equals(repository)) {
+                if (entry.projectId().equals(projectId) && entry.repository().equals(repository)) {
                     changed = true;
                     deletionsToFail.add(entry.uuid());
                 } else {
@@ -3880,14 +3941,14 @@ public ClusterState execute(ClusterState currentState) {
             final SnapshotDeletionsInProgress updatedDeletions = changed ? SnapshotDeletionsInProgress.of(updatedEntries) : null;
             final SnapshotsInProgress snapshotsInProgress = SnapshotsInProgress.get(currentState);
             boolean changedSnapshots = false;
-            for (SnapshotsInProgress.Entry entry : snapshotsInProgress.forRepo(repository)) {
+            for (SnapshotsInProgress.Entry entry : snapshotsInProgress.forRepo(projectId, repository)) {
                 // We failed to read repository data for this delete, it is not the job of SnapshotsService to
                 // retry these kinds of issues so we fail all the pending snapshots
                 snapshotsToFail.add(entry.snapshot());
                 changedSnapshots = true;
             }
             final SnapshotsInProgress updatedSnapshotsInProgress = changedSnapshots
-                ? snapshotsInProgress.createCopyWithUpdatedEntriesForRepo(repository, List.of())
+                ? snapshotsInProgress.createCopyWithUpdatedEntriesForRepo(projectId, repository, List.of())
                 : null;
             return updateWithSnapshots(currentState, updatedSnapshotsInProgress, updatedDeletions);
         }
@@ -3907,11 +3968,11 @@ public void clusterStateProcessed(ClusterState oldState, ClusterState newState)
             final List readyToResolveListeners = new ArrayList<>();
             synchronized (currentlyFinalizing) {
                 Tuple finalization;
-                while ((finalization = repositoryOperations.pollFinalization(repository)) != null) {
+                while ((finalization = repositoryOperations.pollFinalization(projectId, repository)) != null) {
                     assert snapshotsToFail.contains(finalization.v1())
                         : "[" + finalization.v1() + "] not found in snapshots to fail " + snapshotsToFail;
                 }
-                leaveRepoLoop(repository);
+                leaveRepoLoop(projectId, repository);
                 for (Snapshot snapshot : snapshotsToFail) {
                     failSnapshotCompletionListeners(snapshot, failure, readyToResolveListeners::add);
                 }
@@ -3932,7 +3993,7 @@ private static final class OngoingRepositoryOperations {
          * Map of repository name to a deque of {@link Snapshot} that need to be finalized for the repository and the
          * {@link Metadata to use when finalizing}.
          */
-        private final Map> snapshotsToFinalize = new HashMap<>();
+        private final Map> snapshotsToFinalize = new HashMap<>();
 
         /**
          * Set of delete operations currently being executed against the repository. The values in this set are the delete UUIDs returned
@@ -3944,10 +4005,11 @@ private static final class OngoingRepositoryOperations {
         private Metadata latestKnownMetaData;
 
         @Nullable
-        synchronized Tuple pollFinalization(String repository) {
+        synchronized Tuple pollFinalization(ProjectId projectId, String repository) {
             assertConsistent();
+            final var projectRepo = new ProjectRepo(projectId, repository);
             final Snapshot nextEntry;
-            final Deque queued = snapshotsToFinalize.get(repository);
+            final Deque queued = snapshotsToFinalize.get(projectRepo);
             if (queued == null) {
                 return null;
             }
@@ -3955,7 +4017,7 @@ synchronized Tuple pollFinalization(String repository) {
             assert nextEntry != null;
             final Tuple res = Tuple.tuple(nextEntry, latestKnownMetaData);
             if (queued.isEmpty()) {
-                snapshotsToFinalize.remove(repository);
+                snapshotsToFinalize.remove(projectRepo);
             }
             if (snapshotsToFinalize.isEmpty()) {
                 latestKnownMetaData = null;
@@ -3973,7 +4035,8 @@ void finishDeletion(String deleteUUID) {
         }
 
         synchronized void addFinalization(Snapshot snapshot, Metadata metadata) {
-            snapshotsToFinalize.computeIfAbsent(snapshot.getRepository(), k -> new LinkedList<>()).add(snapshot);
+            snapshotsToFinalize.computeIfAbsent(new ProjectRepo(snapshot.getProjectId(), snapshot.getRepository()), k -> new LinkedList<>())
+                .add(snapshot);
             this.latestKnownMetaData = metadata;
             assertConsistent();
         }
@@ -3993,7 +4056,7 @@ synchronized boolean isEmpty() {
         }
 
         synchronized boolean assertNotQueued(Snapshot snapshot) {
-            if (snapshotsToFinalize.getOrDefault(snapshot.getRepository(), new LinkedList<>())
+            if (snapshotsToFinalize.getOrDefault(new ProjectRepo(snapshot.getProjectId(), snapshot.getRepository()), new LinkedList<>())
                 .stream()
                 .anyMatch(entry -> entry.equals(snapshot))) {
 
@@ -4094,25 +4157,22 @@ public ClusterState execute(BatchExecutionContext batchExecutionCo
             // Handle the tasks to apply the shard snapshot updates (ShardSnapshotUpdate tasks).
             SnapshotsInProgress snapshotsInProgress = shardsUpdateContext.computeUpdatedState();
 
-            final var project = state.metadata().getProject();
-            final RegisteredPolicySnapshots.Builder registeredPolicySnapshots = project.custom(
-                RegisteredPolicySnapshots.TYPE,
-                RegisteredPolicySnapshots.EMPTY
-            ).builder();
+            final Map registeredPolicySnapshotsBuilders = new HashMap<>();
             // Handle the tasks to create new snapshots (CreateSnapshotTask tasks).
             for (final var taskContext : batchExecutionContext.taskContexts()) {
                 if (taskContext.getTask() instanceof CreateSnapshotTask task) {
                     try {
-                        final var repoMeta = RepositoriesMetadata.get(state).repository(task.snapshot.getRepository());
+                        final var projectMetadata = state.metadata().getProject(task.snapshot.getProjectId());
+                        final var repoMeta = RepositoriesMetadata.get(projectMetadata).repository(task.snapshot.getRepository());
                         if (RepositoriesService.isReadOnly(repoMeta.settings())) {
                             taskContext.onFailure(new RepositoryException(repoMeta.name(), "repository is readonly"));
                             continue;
                         }
 
-                        registeredPolicySnapshots.addIfSnapshotIsSLMInitiated(
-                            task.createSnapshotRequest.userMetadata(),
-                            task.snapshot.getSnapshotId()
-                        );
+                        registeredPolicySnapshotsBuilders.computeIfAbsent(
+                            projectMetadata.id(),
+                            ignored -> projectMetadata.custom(RegisteredPolicySnapshots.TYPE, RegisteredPolicySnapshots.EMPTY).builder()
+                        ).addIfSnapshotIsSLMInitiated(task.createSnapshotRequest.userMetadata(), task.snapshot.getSnapshotId());
                         if (Objects.equals(task.initialRepositoryMetadata, repoMeta)) {
                             snapshotsInProgress = createSnapshot(task, taskContext, state, snapshotsInProgress);
                         } else {
@@ -4138,12 +4198,11 @@ public ClusterState execute(BatchExecutionContext batchExecutionCo
                 return state;
             }
 
-            return ClusterState.builder(state)
-                .putCustom(SnapshotsInProgress.TYPE, snapshotsInProgress)
-                .putProjectMetadata(
-                    ProjectMetadata.builder(project).putCustom(RegisteredPolicySnapshots.TYPE, registeredPolicySnapshots.build())
-                )
-                .build();
+            final var metadataBuilder = Metadata.builder(state.metadata());
+            registeredPolicySnapshotsBuilders.forEach(
+                (projectId, builder) -> metadataBuilder.getProject(projectId).putCustom(RegisteredPolicySnapshots.TYPE, builder.build())
+            );
+            return ClusterState.builder(state).putCustom(SnapshotsInProgress.TYPE, snapshotsInProgress).metadata(metadataBuilder).build();
         }
 
         private SnapshotsInProgress createSnapshot(
@@ -4156,11 +4215,12 @@ private SnapshotsInProgress createSnapshot(
             final Snapshot snapshot = createSnapshotTask.snapshot;
             final String repositoryName = snapshot.getRepository();
             final String snapshotName = snapshot.getSnapshotId().getName();
-            ensureRepositoryExists(repositoryName, currentState);
+            final var projectState = currentState.projectState(snapshot.getProjectId());
+            ensureRepositoryExists(repositoryName, projectState.metadata());
             final Repository repository = createSnapshotTask.repository;
             ensureSnapshotNameAvailableInRepo(repositoryData, snapshotName, repository);
-            ensureSnapshotNameNotRunning(snapshotsInProgress, repositoryName, snapshotName);
-            validate(repositoryName, snapshotName, currentState);
+            ensureSnapshotNameNotRunning(snapshotsInProgress, snapshot.getProjectId(), repositoryName, snapshotName);
+            validate(repositoryName, snapshotName, projectState.metadata());
             final SnapshotDeletionsInProgress deletionsInProgress = SnapshotDeletionsInProgress.get(currentState);
             ensureNoCleanupInProgress(currentState, repositoryName, snapshotName, "create snapshot");
             ensureBelowConcurrencyLimit(repositoryName, snapshotName, snapshotsInProgress, deletionsInProgress);
@@ -4168,7 +4228,7 @@ private SnapshotsInProgress createSnapshot(
             final CreateSnapshotRequest request = createSnapshotTask.createSnapshotRequest;
             // Store newSnapshot here to be processed in clusterStateProcessed
             Map> requestedIndices = Arrays.stream(
-                indexNameExpressionResolver.concreteIndexNames(currentState, request)
+                indexNameExpressionResolver.concreteIndexNames(projectState.metadata(), request)
             ).collect(Collectors.partitioningBy(systemIndices::isSystemIndex));
 
             List requestedSystemIndices = requestedIndices.get(true);
@@ -4228,17 +4288,17 @@ private SnapshotsInProgress createSnapshot(
 
                 Set featureSystemIndices = feature.getIndexDescriptors()
                     .stream()
-                    .flatMap(descriptor -> descriptor.getMatchingIndices(currentState.metadata().getProject()).stream())
+                    .flatMap(descriptor -> descriptor.getMatchingIndices(projectState.metadata()).stream())
                     .collect(Collectors.toSet());
                 Set featureAssociatedIndices = feature.getAssociatedIndexDescriptors()
                     .stream()
-                    .flatMap(descriptor -> descriptor.getMatchingIndices(currentState.metadata().getProject()).stream())
+                    .flatMap(descriptor -> descriptor.getMatchingIndices(projectState.metadata()).stream())
                     .collect(Collectors.toSet());
 
                 Set featureSystemDataStreams = new HashSet<>();
                 Set featureDataStreamBackingIndices = new HashSet<>();
                 for (SystemDataStreamDescriptor sdd : feature.getDataStreamDescriptors()) {
-                    List backingIndexNames = sdd.getBackingIndexNames(currentState.metadata());
+                    List backingIndexNames = sdd.getBackingIndexNames(projectState.metadata());
                     if (backingIndexNames.size() > 0) {
                         featureDataStreamBackingIndices.addAll(backingIndexNames);
                         featureSystemDataStreams.add(sdd.getDataStreamName());
@@ -4267,7 +4327,7 @@ private SnapshotsInProgress createSnapshot(
             logger.trace("[{}][{}] creating snapshot for indices [{}]", repositoryName, snapshotName, indices);
 
             final Map allIndices = new HashMap<>();
-            for (SnapshotsInProgress.Entry runningSnapshot : snapshotsInProgress.forRepo(repositoryName)) {
+            for (SnapshotsInProgress.Entry runningSnapshot : snapshotsInProgress.forRepo(projectState.projectId(), repositoryName)) {
                 allIndices.putAll(runningSnapshot.indices());
             }
             final Map indexIds = repositoryData.resolveNewIndices(indices, allIndices);
@@ -4283,7 +4343,7 @@ private SnapshotsInProgress createSnapshot(
             ImmutableOpenMap shards = shards(
                 snapshotsInProgress,
                 deletionsInProgress,
-                currentState,
+                projectState,
                 indexIds.values(),
                 useShardGenerations(version),
                 repositoryData,
diff --git a/server/src/test/java/org/elasticsearch/action/admin/cluster/snapshots/status/TransportSnapshotsStatusActionTests.java b/server/src/test/java/org/elasticsearch/action/admin/cluster/snapshots/status/TransportSnapshotsStatusActionTests.java
index adaad4eede0d6..09bf072c7225d 100644
--- a/server/src/test/java/org/elasticsearch/action/admin/cluster/snapshots/status/TransportSnapshotsStatusActionTests.java
+++ b/server/src/test/java/org/elasticsearch/action/admin/cluster/snapshots/status/TransportSnapshotsStatusActionTests.java
@@ -82,7 +82,8 @@ public void initializeComponents() throws Exception {
             threadPool,
             repositoriesService,
             nodeClient,
-            new ActionFilters(Set.of())
+            new ActionFilters(Set.of()),
+            TestProjectResolvers.DEFAULT_PROJECT_ONLY
         );
     }
 
@@ -198,6 +199,7 @@ public void onFailure(Exception e) {
 
         action.buildResponse(
             SnapshotsInProgress.EMPTY,
+            ProjectId.DEFAULT,
             new SnapshotsStatusRequest(TEST_REQUEST_TIMEOUT),
             currentSnapshotEntries,
             nodeSnapshotStatuses,
@@ -357,6 +359,7 @@ public void onFailure(Exception e) {
 
         action.buildResponse(
             SnapshotsInProgress.EMPTY,
+            ProjectId.DEFAULT,
             new SnapshotsStatusRequest(TEST_REQUEST_TIMEOUT),
             currentSnapshotEntries,
             nodeSnapshotStatuses,
diff --git a/server/src/test/java/org/elasticsearch/snapshots/SnapshotResiliencyTests.java b/server/src/test/java/org/elasticsearch/snapshots/SnapshotResiliencyTests.java
index c005691b212b2..d3f0c1aa2bb42 100644
--- a/server/src/test/java/org/elasticsearch/snapshots/SnapshotResiliencyTests.java
+++ b/server/src/test/java/org/elasticsearch/snapshots/SnapshotResiliencyTests.java
@@ -2781,19 +2781,47 @@ public boolean clusterHasFeature(ClusterState state, NodeFeature feature) {
                 );
                 actions.put(
                     TransportCleanupRepositoryAction.TYPE,
-                    new TransportCleanupRepositoryAction(transportService, clusterService, repositoriesService, threadPool, actionFilters)
+                    new TransportCleanupRepositoryAction(
+                        transportService,
+                        clusterService,
+                        repositoriesService,
+                        threadPool,
+                        actionFilters,
+                        TestProjectResolvers.DEFAULT_PROJECT_ONLY
+                    )
                 );
                 actions.put(
                     TransportCreateSnapshotAction.TYPE,
-                    new TransportCreateSnapshotAction(transportService, clusterService, threadPool, snapshotsService, actionFilters)
+                    new TransportCreateSnapshotAction(
+                        transportService,
+                        clusterService,
+                        threadPool,
+                        snapshotsService,
+                        actionFilters,
+                        TestProjectResolvers.DEFAULT_PROJECT_ONLY
+                    )
                 );
                 actions.put(
                     TransportCloneSnapshotAction.TYPE,
-                    new TransportCloneSnapshotAction(transportService, clusterService, threadPool, snapshotsService, actionFilters)
+                    new TransportCloneSnapshotAction(
+                        transportService,
+                        clusterService,
+                        threadPool,
+                        snapshotsService,
+                        actionFilters,
+                        TestProjectResolvers.DEFAULT_PROJECT_ONLY
+                    )
                 );
                 actions.put(
                     TransportGetSnapshotsAction.TYPE,
-                    new TransportGetSnapshotsAction(transportService, clusterService, threadPool, repositoriesService, actionFilters)
+                    new TransportGetSnapshotsAction(
+                        transportService,
+                        clusterService,
+                        threadPool,
+                        repositoriesService,
+                        actionFilters,
+                        TestProjectResolvers.DEFAULT_PROJECT_ONLY
+                    )
                 );
                 actions.put(
                     TransportClusterRerouteAction.TYPE,
@@ -2843,7 +2871,14 @@ public boolean clusterHasFeature(ClusterState state, NodeFeature feature) {
                 );
                 actions.put(
                     TransportDeleteSnapshotAction.TYPE,
-                    new TransportDeleteSnapshotAction(transportService, clusterService, threadPool, snapshotsService, actionFilters)
+                    new TransportDeleteSnapshotAction(
+                        transportService,
+                        clusterService,
+                        threadPool,
+                        snapshotsService,
+                        actionFilters,
+                        TestProjectResolvers.DEFAULT_PROJECT_ONLY
+                    )
                 );
                 client.initialize(
                     actions,
From f3c531ffa2e21b3f633f1f5ad2b40e4c1dce9bcb Mon Sep 17 00:00:00 2001
From: Yang Wang 
Date: Wed, 25 Jun 2025 17:51:51 +1000
Subject: [PATCH 02/12] Fix more issues and write the correct metadata
---
 .../repositories/s3/S3Repository.java         |   1 +
 .../snapshots/CloneSnapshotIT.java            |   2 +-
 ...etadataLoadingDuringSnapshotRestoreIT.java |   4 +-
 .../cluster/SnapshotDeletionsInProgress.java  |  13 +-
 .../cluster/SnapshotsInProgress.java          |  24 +++-
 .../cluster/metadata/ProjectMetadata.java     |   3 +
 .../SnapshotInProgressAllocationDecider.java  |   4 +-
 .../elasticsearch/node/NodeConstruction.java  |   3 +-
 .../repositories/FilterRepository.java        |   4 +-
 .../repositories/FinalizeSnapshotContext.java |   8 ++
 .../repositories/InvalidRepository.java       |   2 +-
 .../repositories/ProjectRepo.java             |  66 +++++++++
 .../repositories/RepositoriesService.java     |   2 +-
 .../repositories/Repository.java              |  12 +-
 .../repositories/RepositoryOperation.java     |  48 -------
 .../repositories/UnknownTypeRepository.java   |   2 +-
 .../blobstore/BlobStoreRepository.java        |  55 +++++---
 .../blobstore/ChecksumBlobStoreFormat.java    |  23 +--
 .../InFlightShardSnapshotStates.java          |   2 +-
 .../snapshots/RestoreService.java             |   4 +-
 .../elasticsearch/snapshots/SnapshotInfo.java |   5 +-
 .../snapshots/SnapshotsService.java           | 132 ++++++++++++------
 .../repositories/InvalidRepositoryTests.java  |   2 +-
 .../RepositoriesServiceTests.java             |   2 +-
 .../UnknownTypeRepositoryTests.java           |   2 +-
 .../BlobStoreRepositoryRestoreTests.java      |   1 +
 .../snapshots/BlobStoreFormatTests.java       |  12 +-
 .../SnapshotInfoBlobSerializationTests.java   |   3 +-
 .../snapshots/SnapshotResiliencyTests.java    |   3 +-
 ...SnapshotsInProgressSerializationTests.java |   2 +-
 .../index/shard/RestoreOnlyRepository.java    |   2 +-
 .../AbstractSnapshotIntegTestCase.java        |   5 +-
 .../xpack/ccr/repository/CcrRepository.java   |   2 +-
 .../SourceOnlySnapshotRepository.java         |   1 +
 .../AsyncRetryDuringSnapshotActionStep.java   |   2 +-
 .../SourceOnlySnapshotShardTests.java         |   1 +
 .../xpack/slm/SLMStatDisruptionIT.java        |   1 +
 .../xpack/slm/SnapshotLifecycleTask.java      |   5 +-
 .../TransportGetSnapshotLifecycleAction.java  |   2 +-
 .../RepositoryVerifyIntegrityIT.java          |  10 +-
 .../RepositoryIntegrityVerifier.java          |   2 +-
 41 files changed, 318 insertions(+), 161 deletions(-)
 create mode 100644 server/src/main/java/org/elasticsearch/repositories/ProjectRepo.java
diff --git a/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3Repository.java b/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3Repository.java
index 92b0dcc2f1cdc..6befdeb2c068a 100644
--- a/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3Repository.java
+++ b/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3Repository.java
@@ -391,6 +391,7 @@ public void finalizeSnapshot(final FinalizeSnapshotContext finalizeSnapshotConte
         if (SnapshotsService.useShardGenerations(finalizeSnapshotContext.repositoryMetaVersion()) == false) {
             final ListenableFuture metadataDone = new ListenableFuture<>();
             wrappedFinalizeContext = new FinalizeSnapshotContext(
+                finalizeSnapshotContext.serializeProjectMetadata(),
                 finalizeSnapshotContext.updatedShardGenerations(),
                 finalizeSnapshotContext.repositoryStateId(),
                 finalizeSnapshotContext.clusterMetadata(),
diff --git a/server/src/internalClusterTest/java/org/elasticsearch/snapshots/CloneSnapshotIT.java b/server/src/internalClusterTest/java/org/elasticsearch/snapshots/CloneSnapshotIT.java
index caf7f852cc15e..c93c66e440403 100644
--- a/server/src/internalClusterTest/java/org/elasticsearch/snapshots/CloneSnapshotIT.java
+++ b/server/src/internalClusterTest/java/org/elasticsearch/snapshots/CloneSnapshotIT.java
@@ -890,7 +890,7 @@ private static BlobStoreIndexShardSnapshots readShardGeneration(
         ShardGeneration generation
     ) throws IOException {
         return BlobStoreRepository.INDEX_SHARD_SNAPSHOTS_FORMAT.read(
-            repository.getMetadata().name(),
+            repository.getProjectRepo(),
             repository.shardContainer(repositoryShardId.index(), repositoryShardId.shardId()),
             generation.getGenerationUUID(),
             NamedXContentRegistry.EMPTY
diff --git a/server/src/internalClusterTest/java/org/elasticsearch/snapshots/MetadataLoadingDuringSnapshotRestoreIT.java b/server/src/internalClusterTest/java/org/elasticsearch/snapshots/MetadataLoadingDuringSnapshotRestoreIT.java
index 89a8af49efdef..480a079eb8b5c 100644
--- a/server/src/internalClusterTest/java/org/elasticsearch/snapshots/MetadataLoadingDuringSnapshotRestoreIT.java
+++ b/server/src/internalClusterTest/java/org/elasticsearch/snapshots/MetadataLoadingDuringSnapshotRestoreIT.java
@@ -189,9 +189,9 @@ public CountingMockRepository(
         }
 
         @Override
-        public Metadata getSnapshotGlobalMetadata(SnapshotId snapshotId) {
+        public Metadata getSnapshotGlobalMetadata(SnapshotId snapshotId, boolean fromProjectMetadata) {
             globalMetadata.computeIfAbsent(snapshotId.getName(), (s) -> new AtomicInteger(0)).incrementAndGet();
-            return super.getSnapshotGlobalMetadata(snapshotId);
+            return super.getSnapshotGlobalMetadata(snapshotId, fromProjectMetadata);
         }
 
         @Override
diff --git a/server/src/main/java/org/elasticsearch/cluster/SnapshotDeletionsInProgress.java b/server/src/main/java/org/elasticsearch/cluster/SnapshotDeletionsInProgress.java
index cbd4c2ed14ca4..f16e03a7e9594 100644
--- a/server/src/main/java/org/elasticsearch/cluster/SnapshotDeletionsInProgress.java
+++ b/server/src/main/java/org/elasticsearch/cluster/SnapshotDeletionsInProgress.java
@@ -20,6 +20,7 @@
 import org.elasticsearch.common.io.stream.Writeable;
 import org.elasticsearch.common.util.CollectionUtils;
 import org.elasticsearch.core.SuppressForbidden;
+import org.elasticsearch.repositories.ProjectRepo;
 import org.elasticsearch.repositories.RepositoryOperation;
 import org.elasticsearch.snapshots.SnapshotId;
 import org.elasticsearch.xcontent.ToXContent;
@@ -65,10 +66,10 @@ public SnapshotDeletionsInProgress(StreamInput in) throws IOException {
     }
 
     private static boolean assertNoConcurrentDeletionsForSameRepository(List entries) {
-        final Set activeRepositories = new HashSet<>();
+        final Set activeRepositories = new HashSet<>();
         for (Entry entry : entries) {
             if (entry.state() == State.STARTED) {
-                final boolean added = activeRepositories.add(entry.repository());
+                final boolean added = activeRepositories.add(new ProjectRepo(entry.projectId(), entry.repository()));
                 assert added : "Found multiple running deletes for a single repository in " + entries;
             }
         }
@@ -114,12 +115,12 @@ public List getEntries() {
     /**
      * Checks if there is an actively executing delete operation for the given repository
      *
-     * @param projectId project to look for the repository
+     * @param projectId project for the repository
      * @param repository repository name
      */
     public boolean hasExecutingDeletion(ProjectId projectId, String repository) {
         for (Entry entry : entries) {
-            if (entry.state() == State.STARTED && entry.repository().equals(repository)) {
+            if (entry.state() == State.STARTED && entry.projectId.equals(projectId) && entry.repository().equals(repository)) {
                 return true;
             }
         }
@@ -134,6 +135,10 @@ public boolean hasDeletionsInProgress() {
         return entries.isEmpty() == false;
     }
 
+    public boolean hasDeletionsInProgress(ProjectId projectId) {
+        return entries.stream().filter(entry -> entry.projectId().equals(projectId)).findFirst().isPresent();
+    }
+
     @Override
     public String getWriteableName() {
         return TYPE;
diff --git a/server/src/main/java/org/elasticsearch/cluster/SnapshotsInProgress.java b/server/src/main/java/org/elasticsearch/cluster/SnapshotsInProgress.java
index 146de4ee37f2c..155de1c889320 100644
--- a/server/src/main/java/org/elasticsearch/cluster/SnapshotsInProgress.java
+++ b/server/src/main/java/org/elasticsearch/cluster/SnapshotsInProgress.java
@@ -34,8 +34,8 @@
 import org.elasticsearch.logging.LogManager;
 import org.elasticsearch.logging.Logger;
 import org.elasticsearch.repositories.IndexId;
+import org.elasticsearch.repositories.ProjectRepo;
 import org.elasticsearch.repositories.RepositoryOperation;
-import org.elasticsearch.repositories.RepositoryOperation.ProjectRepo;
 import org.elasticsearch.repositories.RepositoryShardId;
 import org.elasticsearch.repositories.ShardGeneration;
 import org.elasticsearch.repositories.ShardSnapshotResult;
@@ -60,7 +60,7 @@
 import java.util.Set;
 import java.util.stream.Stream;
 
-import static org.elasticsearch.repositories.RepositoryOperation.PROJECT_REPO_SERIALIZER;
+import static org.elasticsearch.repositories.ProjectRepo.PROJECT_REPO_SERIALIZER;
 
 /**
  * Meta data about snapshots that are currently executing
@@ -201,10 +201,30 @@ public Iterable> entriesByRepo() {
         return () -> Iterators.map(entries.values().iterator(), byRepo -> byRepo.entries);
     }
 
+    /**
+     * Similar to {@link #entriesByRepo()} but only returns entries for the specified project.
+     */
+    public Iterable> entriesByRepo(ProjectId projectId) {
+        return () -> Iterators.map(
+            Iterators.filter(entries.entrySet().iterator(), entry -> entry.getKey().projectId().equals(projectId)),
+            entry -> entry.getValue().entries
+        );
+    }
+
     public Stream asStream() {
         return entries.values().stream().flatMap(t -> t.entries.stream());
     }
 
+    /**
+     * Similar to {@link #asStream()} but only returns entries for the specified project.
+     */
+    public Stream asStream(ProjectId projectId) {
+        return entries.entrySet()
+            .stream()
+            .filter(entry -> entry.getKey().projectId().equals(projectId))
+            .flatMap(entry -> entry.getValue().entries.stream());
+    }
+
     @Nullable
     public Entry snapshot(final Snapshot snapshot) {
         return findSnapshotInList(snapshot, forRepo(snapshot.getProjectId(), snapshot.getRepository()));
diff --git a/server/src/main/java/org/elasticsearch/cluster/metadata/ProjectMetadata.java b/server/src/main/java/org/elasticsearch/cluster/metadata/ProjectMetadata.java
index 4a10964b1be57..4b574b8313c28 100644
--- a/server/src/main/java/org/elasticsearch/cluster/metadata/ProjectMetadata.java
+++ b/server/src/main/java/org/elasticsearch/cluster/metadata/ProjectMetadata.java
@@ -2089,6 +2089,9 @@ static boolean assertDataStreams(Map indices, DataStreamM
 
         public static ProjectMetadata fromXContent(XContentParser parser) throws IOException {
             XContentParser.Token token = parser.currentToken();
+            if (token == null) {
+                token = parser.nextToken();
+            }
             String currentFieldName = null;
 
             XContentParserUtils.ensureExpectedToken(XContentParser.Token.START_OBJECT, token, parser);
diff --git a/server/src/main/java/org/elasticsearch/cluster/routing/allocation/decider/SnapshotInProgressAllocationDecider.java b/server/src/main/java/org/elasticsearch/cluster/routing/allocation/decider/SnapshotInProgressAllocationDecider.java
index 2a76d36466c80..d204dfe8a3841 100644
--- a/server/src/main/java/org/elasticsearch/cluster/routing/allocation/decider/SnapshotInProgressAllocationDecider.java
+++ b/server/src/main/java/org/elasticsearch/cluster/routing/allocation/decider/SnapshotInProgressAllocationDecider.java
@@ -10,6 +10,7 @@
 package org.elasticsearch.cluster.routing.allocation.decider;
 
 import org.elasticsearch.cluster.SnapshotsInProgress;
+import org.elasticsearch.cluster.metadata.ProjectId;
 import org.elasticsearch.cluster.metadata.SingleNodeShutdownMetadata;
 import org.elasticsearch.cluster.routing.RoutingNode;
 import org.elasticsearch.cluster.routing.ShardRouting;
@@ -73,7 +74,8 @@ private static Decision canMove(ShardRouting shardRouting, RoutingAllocation all
             return YES_NOT_SNAPSHOTTED;
         }
 
-        for (final var entriesByRepo : snapshotsInProgress.entriesByRepo()) {
+        final ProjectId projectId = allocation.metadata().projectFor(shardRouting.index()).id();
+        for (final var entriesByRepo : snapshotsInProgress.entriesByRepo(projectId)) {
             for (final var entry : entriesByRepo) {
                 if (entry.isClone()) {
                     // clones do not run on data nodes
diff --git a/server/src/main/java/org/elasticsearch/node/NodeConstruction.java b/server/src/main/java/org/elasticsearch/node/NodeConstruction.java
index 73a2efabb9e81..9e1facab0ea56 100644
--- a/server/src/main/java/org/elasticsearch/node/NodeConstruction.java
+++ b/server/src/main/java/org/elasticsearch/node/NodeConstruction.java
@@ -1115,7 +1115,8 @@ public Map queryFields() {
             repositoriesService,
             transportService,
             actionModule.getActionFilters(),
-            systemIndices
+            systemIndices,
+            projectResolver.supportsMultipleProjects()
         );
         SnapshotShardsService snapshotShardsService = new SnapshotShardsService(
             settings,
diff --git a/server/src/main/java/org/elasticsearch/repositories/FilterRepository.java b/server/src/main/java/org/elasticsearch/repositories/FilterRepository.java
index 34803a12d66cd..b0a55d1f66411 100644
--- a/server/src/main/java/org/elasticsearch/repositories/FilterRepository.java
+++ b/server/src/main/java/org/elasticsearch/repositories/FilterRepository.java
@@ -67,8 +67,8 @@ public void getSnapshotInfo(
     }
 
     @Override
-    public Metadata getSnapshotGlobalMetadata(SnapshotId snapshotId) {
-        return in.getSnapshotGlobalMetadata(snapshotId);
+    public Metadata getSnapshotGlobalMetadata(SnapshotId snapshotId, boolean fromProjectMetadata) {
+        return in.getSnapshotGlobalMetadata(snapshotId, fromProjectMetadata);
     }
 
     @Override
diff --git a/server/src/main/java/org/elasticsearch/repositories/FinalizeSnapshotContext.java b/server/src/main/java/org/elasticsearch/repositories/FinalizeSnapshotContext.java
index d643448b691f0..c9d8a703bbfdd 100644
--- a/server/src/main/java/org/elasticsearch/repositories/FinalizeSnapshotContext.java
+++ b/server/src/main/java/org/elasticsearch/repositories/FinalizeSnapshotContext.java
@@ -27,6 +27,7 @@
  */
 public final class FinalizeSnapshotContext extends DelegatingActionListener {
 
+    private final boolean serializeProjectMetadata;
     private final UpdatedShardGenerations updatedShardGenerations;
 
     /**
@@ -46,6 +47,7 @@ public final class FinalizeSnapshotContext extends DelegatingActionListener PROJECT_REPO_SERIALIZER = new DiffableUtils.KeySerializer<>() {
+        @Override
+        public void writeKey(ProjectRepo key, StreamOutput out) throws IOException {
+            key.writeTo(out);
+        }
+
+        @Override
+        public ProjectRepo readKey(StreamInput in) throws IOException {
+            return new ProjectRepo(in);
+        }
+    };
+
+    public ProjectRepo(StreamInput in) throws IOException {
+        this(ProjectId.readFrom(in), in.readString());
+    }
+
+    @Override
+    public void writeTo(StreamOutput out) throws IOException {
+        projectId.writeTo(out);
+        out.writeString(name);
+    }
+
+    @Override
+    public String toString() {
+        return projectRepoString(projectId, name);
+    }
+
+    public static String projectRepoString(ProjectId projectId, String repositoryName) {
+        return "[" + projectId + "][" + repositoryName + "]";
+    }
+}
diff --git a/server/src/main/java/org/elasticsearch/repositories/RepositoriesService.java b/server/src/main/java/org/elasticsearch/repositories/RepositoriesService.java
index 6248e74bee109..f5d8cad85a290 100644
--- a/server/src/main/java/org/elasticsearch/repositories/RepositoriesService.java
+++ b/server/src/main/java/org/elasticsearch/repositories/RepositoriesService.java
@@ -77,7 +77,7 @@
 
 import static java.util.Collections.unmodifiableMap;
 import static org.elasticsearch.core.Strings.format;
-import static org.elasticsearch.repositories.RepositoryOperation.projectRepoString;
+import static org.elasticsearch.repositories.ProjectRepo.projectRepoString;
 import static org.elasticsearch.snapshots.SearchableSnapshotsSettings.SEARCHABLE_SNAPSHOTS_REPOSITORY_NAME_SETTING_KEY;
 import static org.elasticsearch.snapshots.SearchableSnapshotsSettings.SEARCHABLE_SNAPSHOTS_REPOSITORY_UUID_SETTING_KEY;
 
diff --git a/server/src/main/java/org/elasticsearch/repositories/Repository.java b/server/src/main/java/org/elasticsearch/repositories/Repository.java
index 74c5f3e0e9dc0..e98c3ffb16532 100644
--- a/server/src/main/java/org/elasticsearch/repositories/Repository.java
+++ b/server/src/main/java/org/elasticsearch/repositories/Repository.java
@@ -87,6 +87,13 @@ default Repository create(
     @Nullable
     ProjectId getProjectId();
 
+    /**
+     * Get the project qualified repository
+     */
+    default ProjectRepo getProjectRepo() {
+        return new ProjectRepo(getProjectId(), getMetadata().name());
+    }
+
     /**
      * Returns metadata about this repository.
      */
@@ -138,10 +145,11 @@ public void onFailure(Exception e) {
     /**
      * Returns global metadata associated with the snapshot.
      *
-     * @param snapshotId the snapshot id to load the global metadata from
+     * @param snapshotId                 the snapshot id to load the global metadata from
+     * @param fromProjectMetadata        The metadata may need to be constructed by first reading the project metadata
      * @return the global metadata about the snapshot
      */
-    Metadata getSnapshotGlobalMetadata(SnapshotId snapshotId);
+    Metadata getSnapshotGlobalMetadata(SnapshotId snapshotId, boolean fromProjectMetadata);
 
     /**
      * Returns the index metadata associated with the snapshot.
diff --git a/server/src/main/java/org/elasticsearch/repositories/RepositoryOperation.java b/server/src/main/java/org/elasticsearch/repositories/RepositoryOperation.java
index e6bdf5bc5efc8..a0660758d2b80 100644
--- a/server/src/main/java/org/elasticsearch/repositories/RepositoryOperation.java
+++ b/server/src/main/java/org/elasticsearch/repositories/RepositoryOperation.java
@@ -8,13 +8,7 @@
  */
 package org.elasticsearch.repositories;
 
-import org.elasticsearch.cluster.DiffableUtils;
 import org.elasticsearch.cluster.metadata.ProjectId;
-import org.elasticsearch.common.io.stream.StreamInput;
-import org.elasticsearch.common.io.stream.StreamOutput;
-import org.elasticsearch.common.io.stream.Writeable;
-
-import java.io.IOException;
 
 /**
  * Coordinates of an operation that modifies a repository, assuming that repository at a specific generation.
@@ -36,46 +30,4 @@ public interface RepositoryOperation {
      */
     long repositoryStateId();
 
-    /**
-     * A project qualified repository
-     * @param projectId The project that the repository belongs to
-     * @param name Name of the repository
-     */
-    record ProjectRepo(ProjectId projectId, String name) implements Writeable {
-
-        public ProjectRepo(StreamInput in) throws IOException {
-            this(ProjectId.readFrom(in), in.readString());
-        }
-
-        @Override
-        public void writeTo(StreamOutput out) throws IOException {
-            projectId.writeTo(out);
-            out.writeString(name);
-        }
-
-        @Override
-        public String toString() {
-            return projectRepoString(projectId, name);
-        }
-    }
-
-    static ProjectRepo projectRepo(ProjectId projectId, String repositoryName) {
-        return new ProjectRepo(projectId, repositoryName);
-    }
-
-    static String projectRepoString(ProjectId projectId, String repositoryName) {
-        return "[" + projectId + "][" + repositoryName + "]";
-    }
-
-    DiffableUtils.KeySerializer PROJECT_REPO_SERIALIZER = new DiffableUtils.KeySerializer<>() {
-        @Override
-        public void writeKey(ProjectRepo key, StreamOutput out) throws IOException {
-            key.writeTo(out);
-        }
-
-        @Override
-        public ProjectRepo readKey(StreamInput in) throws IOException {
-            return new ProjectRepo(in);
-        }
-    };
 }
diff --git a/server/src/main/java/org/elasticsearch/repositories/UnknownTypeRepository.java b/server/src/main/java/org/elasticsearch/repositories/UnknownTypeRepository.java
index 13d81d2d0d484..9750666c8c8a9 100644
--- a/server/src/main/java/org/elasticsearch/repositories/UnknownTypeRepository.java
+++ b/server/src/main/java/org/elasticsearch/repositories/UnknownTypeRepository.java
@@ -74,7 +74,7 @@ public void getSnapshotInfo(
     }
 
     @Override
-    public Metadata getSnapshotGlobalMetadata(SnapshotId snapshotId) {
+    public Metadata getSnapshotGlobalMetadata(SnapshotId snapshotId, boolean fromProjectMetadata) {
         throw createUnknownTypeException();
     }
 
diff --git a/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java b/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java
index cb3fa6f575c8c..44314e924bd6a 100644
--- a/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java
+++ b/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java
@@ -371,6 +371,15 @@ public static String getRepositoryDataBlobName(long repositoryGeneration) {
         ChunkedToXContent::wrapAsToXContent
     );
 
+    public static final ChecksumBlobStoreFormat PROJECT_METADATA_FORMAT = new ChecksumBlobStoreFormat<>(
+        "project-metadata",
+        METADATA_NAME_FORMAT,
+        (repoName, parser) -> ProjectMetadata.Builder.fromXContent(parser),
+        projectMetadata -> ChunkedToXContent.wrapAsToXContent(
+            params -> Iterators.concat(Iterators.single((builder, ignored) -> builder.field("id", projectMetadata.id())))
+        )
+    );
+
     public static final ChecksumBlobStoreFormat INDEX_METADATA_FORMAT = new ChecksumBlobStoreFormat<>(
         "index-metadata",
         METADATA_NAME_FORMAT,
@@ -1282,7 +1291,7 @@ private void determineShardCount(ActionListener listener) {
             private void getOneShardCount(String indexMetaGeneration) {
                 try {
                     updateShardCount(
-                        INDEX_METADATA_FORMAT.read(metadata.name(), indexContainer, indexMetaGeneration, namedXContentRegistry)
+                        INDEX_METADATA_FORMAT.read(getProjectRepo(), indexContainer, indexMetaGeneration, namedXContentRegistry)
                             .getNumberOfShards()
                     );
                 } catch (Exception ex) {
@@ -1749,6 +1758,8 @@ int sizeInBytes() {
     @Override
     public void finalizeSnapshot(final FinalizeSnapshotContext finalizeSnapshotContext) {
         assert ThreadPool.assertCurrentThreadPool(ThreadPool.Names.SNAPSHOT);
+        assert finalizeSnapshotContext.snapshotInfo().projectId().equals(getProjectId())
+            : "project-id mismatch: " + finalizeSnapshotContext.snapshotInfo().projectId() + " != " + getProjectId();
         final long repositoryStateId = finalizeSnapshotContext.repositoryStateId();
         final SnapshotInfo snapshotInfo = finalizeSnapshotContext.snapshotInfo();
         assert repositoryStateId > RepositoryData.UNKNOWN_REPO_GEN
@@ -1813,17 +1824,19 @@ record RootBlobUpdateResult(RepositoryData oldRepositoryData, RepositoryData new
 
                     // Write global metadata
                     final Metadata clusterMetadata = finalizeSnapshotContext.clusterMetadata();
-                    executor.execute(
-                        ActionRunnable.run(
-                            allMetaListeners.acquire(),
-                            () -> GLOBAL_METADATA_FORMAT.write(clusterMetadata, blobContainer(), snapshotId.getUUID(), compress)
-                        )
-                    );
+                    final var projectMetadata = clusterMetadata.getProject(getProjectId());
+                    executor.execute(ActionRunnable.run(allMetaListeners.acquire(), () -> {
+                        if (finalizeSnapshotContext.serializeProjectMetadata()) {
+                            PROJECT_METADATA_FORMAT.write(projectMetadata, blobContainer(), snapshotId.getUUID(), compress);
+                        } else {
+                            GLOBAL_METADATA_FORMAT.write(clusterMetadata, blobContainer(), snapshotId.getUUID(), compress);
+                        }
+                    }));
 
                     // Write the index metadata for each index in the snapshot
                     for (IndexId index : indices) {
                         executor.execute(ActionRunnable.run(allMetaListeners.acquire(), () -> {
-                            final IndexMetadata indexMetaData = clusterMetadata.getProject(getProjectId()).index(index.getName());
+                            final IndexMetadata indexMetaData = projectMetadata.index(index.getName());
                             if (writeIndexGens) {
                                 final String identifiers = IndexMetaDataGenerations.buildUniqueIdentifier(indexMetaData);
                                 String metaUUID = existingRepositoryData.indexMetaDataGenerations().getIndexMetaBlobId(identifiers);
@@ -2014,7 +2027,7 @@ private void getOneSnapshotInfo(BlockingQueue queue, GetSnapshotInfo
             Exception failure = null;
             SnapshotInfo snapshotInfo = null;
             try {
-                snapshotInfo = SNAPSHOT_FORMAT.read(metadata.name(), blobContainer(), snapshotId.getUUID(), namedXContentRegistry);
+                snapshotInfo = SNAPSHOT_FORMAT.read(getProjectRepo(), blobContainer(), snapshotId.getUUID(), namedXContentRegistry);
             } catch (NoSuchFileException ex) {
                 failure = new SnapshotMissingException(metadata.name(), snapshotId, ex);
             } catch (IOException | NotXContentException ex) {
@@ -2038,9 +2051,19 @@ private void getOneSnapshotInfo(BlockingQueue queue, GetSnapshotInfo
     }
 
     @Override
-    public Metadata getSnapshotGlobalMetadata(final SnapshotId snapshotId) {
+    public Metadata getSnapshotGlobalMetadata(final SnapshotId snapshotId, boolean fromProjectMetadata) {
         try {
-            return GLOBAL_METADATA_FORMAT.read(metadata.name(), blobContainer(), snapshotId.getUUID(), namedXContentRegistry);
+            if (fromProjectMetadata) {
+                final var projectMetadata = PROJECT_METADATA_FORMAT.read(
+                    getProjectRepo(),
+                    blobContainer(),
+                    snapshotId.getUUID(),
+                    namedXContentRegistry
+                );
+                return Metadata.builder().put(projectMetadata).build();
+            } else {
+                return GLOBAL_METADATA_FORMAT.read(getProjectRepo(), blobContainer(), snapshotId.getUUID(), namedXContentRegistry);
+            }
         } catch (NoSuchFileException ex) {
             throw new SnapshotMissingException(metadata.name(), snapshotId, ex);
         } catch (IOException ex) {
@@ -2052,7 +2075,7 @@ public Metadata getSnapshotGlobalMetadata(final SnapshotId snapshotId) {
     public IndexMetadata getSnapshotIndexMetaData(RepositoryData repositoryData, SnapshotId snapshotId, IndexId index) throws IOException {
         try {
             return INDEX_METADATA_FORMAT.read(
-                metadata.name(),
+                getProjectRepo(),
                 indexContainer(index),
                 repositoryData.indexMetaDataGenerations().indexMetaBlobId(snapshotId, index),
                 namedXContentRegistry
@@ -3897,7 +3920,7 @@ private void writeShardIndexBlobAtomic(
      */
     public BlobStoreIndexShardSnapshot loadShardSnapshot(BlobContainer shardContainer, SnapshotId snapshotId) {
         try {
-            return INDEX_SHARD_SNAPSHOT_FORMAT.read(metadata.name(), shardContainer, snapshotId.getUUID(), namedXContentRegistry);
+            return INDEX_SHARD_SNAPSHOT_FORMAT.read(getProjectRepo(), shardContainer, snapshotId.getUUID(), namedXContentRegistry);
         } catch (NoSuchFileException ex) {
             throw new SnapshotMissingException(metadata.name(), snapshotId, ex);
         } catch (IOException ex) {
@@ -3952,7 +3975,7 @@ private Tuple buildBlobStoreIndex
             try {
                 return new Tuple<>(
                     INDEX_SHARD_SNAPSHOTS_FORMAT.read(
-                        metadata.name(),
+                        getProjectRepo(),
                         shardContainer,
                         generation.getGenerationUUID(),
                         namedXContentRegistry
@@ -4008,7 +4031,7 @@ private Tuple buildBlobStoreIndex
                             && shardSnapshotBlobName.endsWith(METADATA_BLOB_NAME_SUFFIX)
                             && shardSnapshotBlobName.length() == shardSnapshotBlobNameLength) {
                             final var shardSnapshot = INDEX_SHARD_SNAPSHOT_FORMAT.read(
-                                metadata.name(),
+                                getProjectRepo(),
                                 shardContainer,
                                 shardSnapshotBlobName.substring(SNAPSHOT_PREFIX.length(), shardSnapshotBlobNameLengthBeforeExt),
                                 namedXContentRegistry
@@ -4066,7 +4089,7 @@ private Tuple buildBlobStoreIndexShardSnapsh
         long latest = latestGeneration(blobs);
         if (latest >= 0) {
             final BlobStoreIndexShardSnapshots shardSnapshots = INDEX_SHARD_SNAPSHOTS_FORMAT.read(
-                metadata.name(),
+                getProjectRepo(),
                 shardContainer,
                 Long.toString(latest),
                 namedXContentRegistry
diff --git a/server/src/main/java/org/elasticsearch/repositories/blobstore/ChecksumBlobStoreFormat.java b/server/src/main/java/org/elasticsearch/repositories/blobstore/ChecksumBlobStoreFormat.java
index bae5ae0fdadf0..60a5f6c09d3e4 100644
--- a/server/src/main/java/org/elasticsearch/repositories/blobstore/ChecksumBlobStoreFormat.java
+++ b/server/src/main/java/org/elasticsearch/repositories/blobstore/ChecksumBlobStoreFormat.java
@@ -30,6 +30,7 @@
 import org.elasticsearch.common.xcontent.XContentParserUtils;
 import org.elasticsearch.core.Nullable;
 import org.elasticsearch.gateway.CorruptStateException;
+import org.elasticsearch.repositories.ProjectRepo;
 import org.elasticsearch.xcontent.NamedXContentRegistry;
 import org.elasticsearch.xcontent.ToXContent;
 import org.elasticsearch.xcontent.XContentBuilder;
@@ -69,9 +70,9 @@ public final class ChecksumBlobStoreFormat {
 
     private final String blobNameFormat;
 
-    private final CheckedBiFunction reader;
+    private final CheckedBiFunction reader;
 
-    private final CheckedBiFunction fallbackReader;
+    private final CheckedBiFunction fallbackReader;
 
     private final Function writer;
 
@@ -85,8 +86,8 @@ public final class ChecksumBlobStoreFormat {
     public ChecksumBlobStoreFormat(
         String codec,
         String blobNameFormat,
-        CheckedBiFunction reader,
-        @Nullable CheckedBiFunction fallbackReader,
+        CheckedBiFunction reader,
+        @Nullable CheckedBiFunction fallbackReader,
         Function writer
     ) {
         this.reader = reader;
@@ -105,7 +106,7 @@ public ChecksumBlobStoreFormat(
     public ChecksumBlobStoreFormat(
         String codec,
         String blobNameFormat,
-        CheckedBiFunction reader,
+        CheckedBiFunction reader,
         Function writer
     ) {
         this(codec, blobNameFormat, reader, null, writer);
@@ -118,11 +119,11 @@ public ChecksumBlobStoreFormat(
      * @param name          name to be translated into
      * @return parsed blob object
      */
-    public T read(String repoName, BlobContainer blobContainer, String name, NamedXContentRegistry namedXContentRegistry)
+    public T read(ProjectRepo projectRepo, BlobContainer blobContainer, String name, NamedXContentRegistry namedXContentRegistry)
         throws IOException {
         String blobName = blobName(name);
         try (InputStream in = blobContainer.readBlob(OperationPurpose.SNAPSHOT_METADATA, blobName)) {
-            return deserialize(repoName, namedXContentRegistry, in);
+            return deserialize(projectRepo, namedXContentRegistry, in);
         }
     }
 
@@ -130,7 +131,7 @@ public String blobName(String name) {
         return String.format(Locale.ROOT, blobNameFormat, name);
     }
 
-    public T deserialize(String repoName, NamedXContentRegistry namedXContentRegistry, InputStream input) throws IOException {
+    public T deserialize(ProjectRepo projectRepo, NamedXContentRegistry namedXContentRegistry, InputStream input) throws IOException {
         final DeserializeMetaBlobInputStream deserializeMetaBlobInputStream = new DeserializeMetaBlobInputStream(input);
         try {
             CodecUtil.checkHeader(new InputStreamDataInput(deserializeMetaBlobInputStream), codec, VERSION, VERSION);
@@ -154,7 +155,7 @@ public T deserialize(String repoName, NamedXContentRegistry namedXContentRegistr
                         XContentType.SMILE
                     )
                 ) {
-                    result = reader.apply(repoName, parser);
+                    result = reader.apply(projectRepo, parser);
                     XContentParserUtils.ensureExpectedToken(null, parser.nextToken(), parser);
                 } catch (Exception e) {
                     try (
@@ -165,7 +166,7 @@ public T deserialize(String repoName, NamedXContentRegistry namedXContentRegistr
                             XContentType.SMILE
                         )
                     ) {
-                        result = fallbackReader.apply(repoName, parser);
+                        result = fallbackReader.apply(projectRepo, parser);
                         XContentParserUtils.ensureExpectedToken(null, parser.nextToken(), parser);
                     }
                 }
@@ -174,7 +175,7 @@ public T deserialize(String repoName, NamedXContentRegistry namedXContentRegistr
                     XContentParser parser = XContentType.SMILE.xContent()
                         .createParser(namedXContentRegistry, LoggingDeprecationHandler.INSTANCE, wrappedStream)
                 ) {
-                    result = reader.apply(repoName, parser);
+                    result = reader.apply(projectRepo, parser);
                     XContentParserUtils.ensureExpectedToken(null, parser.nextToken(), parser);
                 }
                 deserializeMetaBlobInputStream.verifyFooter();
diff --git a/server/src/main/java/org/elasticsearch/snapshots/InFlightShardSnapshotStates.java b/server/src/main/java/org/elasticsearch/snapshots/InFlightShardSnapshotStates.java
index 89ebf13b58c95..94fc6dfad06d8 100644
--- a/server/src/main/java/org/elasticsearch/snapshots/InFlightShardSnapshotStates.java
+++ b/server/src/main/java/org/elasticsearch/snapshots/InFlightShardSnapshotStates.java
@@ -12,7 +12,7 @@
 import org.elasticsearch.cluster.SnapshotsInProgress;
 import org.elasticsearch.core.Nullable;
 import org.elasticsearch.repositories.IndexId;
-import org.elasticsearch.repositories.RepositoryOperation.ProjectRepo;
+import org.elasticsearch.repositories.ProjectRepo;
 import org.elasticsearch.repositories.RepositoryShardId;
 import org.elasticsearch.repositories.ShardGeneration;
 import org.elasticsearch.repositories.ShardGenerations;
diff --git a/server/src/main/java/org/elasticsearch/snapshots/RestoreService.java b/server/src/main/java/org/elasticsearch/snapshots/RestoreService.java
index f57db79d4625d..1680bfa3a59a0 100644
--- a/server/src/main/java/org/elasticsearch/snapshots/RestoreService.java
+++ b/server/src/main/java/org/elasticsearch/snapshots/RestoreService.java
@@ -359,7 +359,7 @@ private void startRestore(
         Metadata globalMetadata = null;
         final Metadata.Builder metadataBuilder;
         if (request.includeGlobalState()) {
-            globalMetadata = repository.getSnapshotGlobalMetadata(snapshotId);
+            globalMetadata = repository.getSnapshotGlobalMetadata(snapshotId, false);
             metadataBuilder = Metadata.builder(globalMetadata);
         } else {
             metadataBuilder = Metadata.builder();
@@ -652,7 +652,7 @@ private static Tuple