Pārlūkot izejas kodu

Permit rename of repository of searchable snapshot (#67968)

Today the name of the repository containing a searchable snapshot is
recorded in the metadata of the corresponding searchable snapshot index,
under the assumption that the repository name is a stable identifier.
This is not a safe assumption: we may be restoring a snapshot containing
some searchable snapshot indices into a completely separate cluster.

This commit adds the repository UUID to the index metadata if it is
available at mount time, and uses this to find the correct repository
instead of the name.

Relates #66431
David Turner 4 gadi atpakaļ
vecāks
revīzija
edfa5f3e1f

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

@@ -125,7 +125,7 @@ public class MultiClusterRepoAccessIT extends AbstractSnapshotIntegTestCase {
         secondCluster.startDataOnlyNode();
         assertAcked(secondCluster.client().admin().cluster().preparePutRepository(repoName)
                 .setType("fs")
-                .setSettings(Settings.builder().put("location", repoPath).put("read_only", true)));
+                .setSettings(Settings.builder().put("location", repoPath).put("readonly", true)));
         assertThat(secondCluster.client().admin().cluster().prepareGetRepositories(repoName).get().repositories()
                 .stream().filter(r -> r.name().equals(repoName)).findFirst().orElseThrow().uuid(), equalTo(repoUuid));
 

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

@@ -66,6 +66,8 @@ import java.util.Set;
 import java.util.stream.Collectors;
 import java.util.stream.Stream;
 
+import static java.util.Collections.unmodifiableMap;
+
 /**
  * Service responsible for maintaining and providing access to snapshot repositories on nodes.
  */
@@ -454,7 +456,7 @@ public class RepositoriesService extends AbstractLifecycleComponent implements C
             for (Repository repo : builder.values()) {
                 repo.updateState(state);
             }
-            repositories = Collections.unmodifiableMap(builder);
+            repositories = unmodifiableMap(builder);
         } catch (Exception ex) {
             assert false : new AssertionError(ex);
             logger.warn("failure updating cluster state ", ex);
@@ -479,8 +481,6 @@ public class RepositoriesService extends AbstractLifecycleComponent implements C
 
     /**
      * Returns registered repository
-     * <p>
-     * This method is called only on the master node
      *
      * @param repositoryName repository name
      * @return registered repository
@@ -498,6 +498,13 @@ public class RepositoriesService extends AbstractLifecycleComponent implements C
         throw new RepositoryMissingException(repositoryName);
     }
 
+    /**
+     * @return the current collection of registered repositories, keyed by name.
+     */
+    public Map<String, Repository> getRepositories() {
+        return unmodifiableMap(repositories);
+    }
+
     public List<RepositoryStatsSnapshot> repositoriesStats() {
         List<RepositoryStatsSnapshot> archivedRepoStats = repositoriesStatsArchive.getArchivedStats();
         List<RepositoryStatsSnapshot> activeRepoStats = getRepositoryStatsForActiveRepositories();

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

@@ -23,6 +23,7 @@ import org.elasticsearch.action.index.IndexRequestBuilder;
 import org.elasticsearch.cluster.ClusterState;
 import org.elasticsearch.cluster.metadata.DataStream;
 import org.elasticsearch.cluster.metadata.IndexMetadata;
+import org.elasticsearch.cluster.metadata.RepositoryMetadata;
 import org.elasticsearch.cluster.node.DiscoveryNode;
 import org.elasticsearch.cluster.routing.ShardRouting;
 import org.elasticsearch.common.Priority;
@@ -43,9 +44,11 @@ import org.elasticsearch.index.shard.ShardId;
 import org.elasticsearch.index.shard.ShardPath;
 import org.elasticsearch.indices.IndicesService;
 import org.elasticsearch.indices.recovery.RecoveryState;
+import org.elasticsearch.repositories.RepositoryData;
 import org.elasticsearch.rest.RestStatus;
 import org.elasticsearch.snapshots.SnapshotId;
 import org.elasticsearch.snapshots.SnapshotInfo;
+import org.elasticsearch.snapshots.SnapshotsService;
 import org.elasticsearch.xpack.cluster.routing.allocation.DataTierAllocationDecider;
 import org.elasticsearch.xpack.core.DataTier;
 import org.elasticsearch.xpack.core.searchablesnapshots.MountSearchableSnapshotAction;
@@ -217,13 +220,23 @@ public class SearchableSnapshotsIntegTests extends BaseSearchableSnapshotsIntegT
         final RestoreSnapshotResponse restoreSnapshotResponse = client().execute(MountSearchableSnapshotAction.INSTANCE, req).get();
         assertThat(restoreSnapshotResponse.getRestoreInfo().failedShards(), equalTo(0));
 
+        final RepositoryMetadata repositoryMetadata = client().admin()
+            .cluster()
+            .prepareGetRepositories(fsRepoName)
+            .get()
+            .repositories()
+            .get(0);
+        assertThat(repositoryMetadata.name(), equalTo(fsRepoName));
+        assertThat(repositoryMetadata.uuid(), not(equalTo(RepositoryData.MISSING_UUID)));
+
         final Settings settings = client().admin()
             .indices()
             .prepareGetSettings(restoredIndexName)
             .get()
             .getIndexToSettings()
             .get(restoredIndexName);
-        assertThat(SearchableSnapshots.SNAPSHOT_REPOSITORY_SETTING.get(settings), equalTo(fsRepoName));
+        assertThat(SearchableSnapshots.SNAPSHOT_REPOSITORY_UUID_SETTING.get(settings), equalTo(repositoryMetadata.uuid()));
+        assertThat(SearchableSnapshots.SNAPSHOT_REPOSITORY_NAME_SETTING.get(settings), equalTo(fsRepoName));
         assertThat(SearchableSnapshots.SNAPSHOT_SNAPSHOT_NAME_SETTING.get(settings), equalTo(snapshotName));
         assertThat(IndexModule.INDEX_STORE_TYPE_SETTING.get(settings), equalTo(SNAPSHOT_DIRECTORY_FACTORY_KEY));
         assertThat(IndexModule.INDEX_RECOVERY_TYPE_SETTING.get(settings), equalTo(SNAPSHOT_RECOVERY_STATE_FACTORY_KEY));
@@ -351,7 +364,7 @@ public class SearchableSnapshotsIntegTests extends BaseSearchableSnapshotsIntegT
             .getIndexToSettings()
             .get(clonedIndexName);
         assertFalse(clonedIndexSettings.hasValue(IndexModule.INDEX_STORE_TYPE_SETTING.getKey()));
-        assertFalse(clonedIndexSettings.hasValue(SearchableSnapshots.SNAPSHOT_REPOSITORY_SETTING.getKey()));
+        assertFalse(clonedIndexSettings.hasValue(SearchableSnapshots.SNAPSHOT_REPOSITORY_NAME_SETTING.getKey()));
         assertFalse(clonedIndexSettings.hasValue(SearchableSnapshots.SNAPSHOT_SNAPSHOT_NAME_SETTING.getKey()));
         assertFalse(clonedIndexSettings.hasValue(SearchableSnapshots.SNAPSHOT_SNAPSHOT_ID_SETTING.getKey()));
         assertFalse(clonedIndexSettings.hasValue(SearchableSnapshots.SNAPSHOT_INDEX_ID_SETTING.getKey()));
@@ -816,8 +829,38 @@ public class SearchableSnapshotsIntegTests extends BaseSearchableSnapshotsIntegT
             .getTotalHits();
         logger.info("--> [{}] in total, of which [{}] match the query", originalAllHits, originalBarHits);
 
+        // The repository that contains the actual data
         final String repositoryName = randomAlphaOfLength(10).toLowerCase(Locale.ROOT);
-        createRepository(repositoryName, "fs");
+        final boolean hasRepositoryUuid = randomBoolean();
+        if (hasRepositoryUuid) {
+            createRepository(repositoryName, "fs");
+        } else {
+            // Prepare the repo with an older version first, to suppress the repository UUID and fall back to matching by repo name
+            final String tmpRepositoryName = randomAlphaOfLength(10).toLowerCase(Locale.ROOT);
+            createRepositoryNoVerify(tmpRepositoryName, "fs");
+            final Path repoPath = internalCluster().getCurrentMasterNodeInstance(Environment.class)
+                .resolveRepoFile(
+                    client().admin()
+                        .cluster()
+                        .prepareGetRepositories(tmpRepositoryName)
+                        .get()
+                        .repositories()
+                        .get(0)
+                        .settings()
+                        .get("location")
+                );
+            initWithSnapshotVersion(
+                tmpRepositoryName,
+                repoPath,
+                randomFrom(
+                    SnapshotsService.OLD_SNAPSHOT_FORMAT,
+                    SnapshotsService.SHARD_GEN_IN_REPO_DATA_VERSION,
+                    SnapshotsService.INDEX_GEN_IN_REPO_DATA_VERSION
+                )
+            );
+            assertAcked(client().admin().cluster().prepareDeleteRepository(tmpRepositoryName));
+            createRepository(repositoryName, "fs", repoPath);
+        }
 
         final SnapshotId snapshotOne = createSnapshot(repositoryName, "snapshot-1", List.of(indexName)).snapshotId();
         for (final SnapshotStatus snapshotStatus : client().admin()
@@ -835,6 +878,11 @@ public class SearchableSnapshotsIntegTests extends BaseSearchableSnapshotsIntegT
         }
         assertAcked(client().admin().indices().prepareDelete(indexName));
 
+        assertThat(
+            client().admin().cluster().prepareGetRepositories(repositoryName).get().repositories().get(0).uuid(),
+            hasRepositoryUuid ? not(equalTo(RepositoryData.MISSING_UUID)) : equalTo(RepositoryData.MISSING_UUID)
+        );
+
         final String restoredIndexName = randomValueOtherThan(indexName, () -> randomAlphaOfLength(10).toLowerCase(Locale.ROOT));
         mountSnapshot(repositoryName, snapshotOne.getName(), indexName, restoredIndexName, Settings.EMPTY);
         ensureGreen(restoredIndexName);
@@ -844,12 +892,21 @@ public class SearchableSnapshotsIntegTests extends BaseSearchableSnapshotsIntegT
             assertAcked(client().admin().indices().prepareClose(restoredIndexName));
         }
 
+        // The repository that contains the cluster snapshot (may be different from the one containing the data)
+        final String backupRepositoryName;
+        if (randomBoolean()) {
+            backupRepositoryName = randomAlphaOfLength(10).toLowerCase(Locale.ROOT);
+            createRepository(backupRepositoryName, "fs");
+        } else {
+            backupRepositoryName = repositoryName;
+        }
+
         logger.info("--> starting to take snapshot-2");
-        final SnapshotId snapshotTwo = createSnapshot(repositoryName, "snapshot-2", List.of(restoredIndexName)).snapshotId();
+        final SnapshotId snapshotTwo = createSnapshot(backupRepositoryName, "snapshot-2", List.of(restoredIndexName)).snapshotId();
         logger.info("--> finished taking snapshot-2");
         for (final SnapshotStatus snapshotStatus : client().admin()
             .cluster()
-            .prepareSnapshotStatus(repositoryName)
+            .prepareSnapshotStatus(backupRepositoryName)
             .setSnapshots(snapshotTwo.getName())
             .get()
             .getSnapshots()) {
@@ -864,11 +921,41 @@ public class SearchableSnapshotsIntegTests extends BaseSearchableSnapshotsIntegT
         }
         assertAcked(client().admin().indices().prepareDelete(restoredIndexName));
 
+        // The repository that contains the cluster snapshot -- may be different from backupRepositoryName if we're only using one repo and
+        // we rename it.
+        final String restoreRepositoryName;
+        if (hasRepositoryUuid && randomBoolean()) {
+            // Re-mount the repository containing the actual data under a different name
+            final RepositoryMetadata repositoryMetadata = client().admin()
+                .cluster()
+                .prepareGetRepositories(repositoryName)
+                .get()
+                .repositories()
+                .get(0);
+
+            // Rename the repository containing the actual data.
+            final String newRepositoryName = randomAlphaOfLength(10).toLowerCase(Locale.ROOT);
+            assertAcked(client().admin().cluster().prepareDeleteRepository(repositoryName));
+            final Settings.Builder settings = Settings.builder().put(repositoryMetadata.settings());
+            if (randomBoolean()) {
+                settings.put("readonly", "true");
+            }
+            assertAcked(
+                clusterAdmin().preparePutRepository(newRepositoryName)
+                    .setVerify(true) // TODO if we're missing repo UUIDs then load all the repository metadata
+                    .setType("fs")
+                    .setSettings(settings)
+            );
+            restoreRepositoryName = backupRepositoryName.equals(repositoryName) ? newRepositoryName : backupRepositoryName;
+        } else {
+            restoreRepositoryName = backupRepositoryName;
+        }
+
         logger.info("--> starting to restore snapshot-2");
         assertThat(
             client().admin()
                 .cluster()
-                .prepareRestoreSnapshot(repositoryName, snapshotTwo.getName())
+                .prepareRestoreSnapshot(restoreRepositoryName, snapshotTwo.getName())
                 .setIndices(restoredIndexName)
                 .get()
                 .status(),

+ 29 - 4
x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/SearchableSnapshotDirectory.java

@@ -51,6 +51,7 @@ import org.elasticsearch.indices.recovery.SearchableSnapshotRecoveryState;
 import org.elasticsearch.repositories.IndexId;
 import org.elasticsearch.repositories.RepositoriesService;
 import org.elasticsearch.repositories.Repository;
+import org.elasticsearch.repositories.RepositoryMissingException;
 import org.elasticsearch.repositories.blobstore.BlobStoreRepository;
 import org.elasticsearch.snapshots.SnapshotId;
 import org.elasticsearch.snapshots.SourceOnlySnapshotRepository;
@@ -86,7 +87,8 @@ import static org.elasticsearch.xpack.searchablesnapshots.SearchableSnapshots.SN
 import static org.elasticsearch.xpack.searchablesnapshots.SearchableSnapshots.SNAPSHOT_CACHE_PREWARM_ENABLED_SETTING;
 import static org.elasticsearch.xpack.searchablesnapshots.SearchableSnapshots.SNAPSHOT_INDEX_ID_SETTING;
 import static org.elasticsearch.xpack.searchablesnapshots.SearchableSnapshots.SNAPSHOT_INDEX_NAME_SETTING;
-import static org.elasticsearch.xpack.searchablesnapshots.SearchableSnapshots.SNAPSHOT_REPOSITORY_SETTING;
+import static org.elasticsearch.xpack.searchablesnapshots.SearchableSnapshots.SNAPSHOT_REPOSITORY_NAME_SETTING;
+import static org.elasticsearch.xpack.searchablesnapshots.SearchableSnapshots.SNAPSHOT_REPOSITORY_UUID_SETTING;
 import static org.elasticsearch.xpack.searchablesnapshots.SearchableSnapshots.SNAPSHOT_SNAPSHOT_ID_SETTING;
 import static org.elasticsearch.xpack.searchablesnapshots.SearchableSnapshots.SNAPSHOT_SNAPSHOT_NAME_SETTING;
 import static org.elasticsearch.xpack.searchablesnapshots.SearchableSnapshots.SNAPSHOT_UNCACHED_CHUNK_SIZE_SETTING;
@@ -537,7 +539,7 @@ public class SearchableSnapshotDirectory extends BaseDirectory {
         BlobStoreCacheService blobStoreCacheService
     ) throws IOException {
 
-        if (SNAPSHOT_REPOSITORY_SETTING.exists(indexSettings.getSettings()) == false
+        if (SNAPSHOT_REPOSITORY_NAME_SETTING.exists(indexSettings.getSettings()) == false
             || SNAPSHOT_INDEX_NAME_SETTING.exists(indexSettings.getSettings()) == false
             || SNAPSHOT_INDEX_ID_SETTING.exists(indexSettings.getSettings()) == false
             || SNAPSHOT_SNAPSHOT_NAME_SETTING.exists(indexSettings.getSettings()) == false
@@ -563,8 +565,22 @@ public class SearchableSnapshotDirectory extends BaseDirectory {
             );
         }
 
-        final String repositoryName = SNAPSHOT_REPOSITORY_SETTING.get(indexSettings.getSettings());
-        Repository repository = repositories.repository(repositoryName);
+        Repository repository;
+        final String repositoryName;
+        if (SNAPSHOT_REPOSITORY_UUID_SETTING.exists(indexSettings.getSettings())) {
+            repository = repositoryByUuid(
+                repositories.getRepositories(),
+                SNAPSHOT_REPOSITORY_UUID_SETTING.get(indexSettings.getSettings()),
+                SNAPSHOT_REPOSITORY_NAME_SETTING.get(indexSettings.getSettings())
+            );
+            repositoryName = repository.getMetadata().name();
+        } else {
+            // repository containing pre-7.12 snapshots has no UUID so we assume it matches by name
+            repositoryName = SNAPSHOT_REPOSITORY_NAME_SETTING.get(indexSettings.getSettings());
+            repository = repositories.repository(repositoryName);
+            assert repository.getMetadata().name().equals(repositoryName) : repository.getMetadata().name() + " vs " + repositoryName;
+        }
+
         if (repository instanceof SourceOnlySnapshotRepository) {
             repository = ((SourceOnlySnapshotRepository) repository).getDelegate();
         }
@@ -640,6 +656,15 @@ public class SearchableSnapshotDirectory extends BaseDirectory {
         blobStoreCacheService.putAsync(repository, name, blobStoreCachePath, offset, content, listener);
     }
 
+    private static Repository repositoryByUuid(Map<String, Repository> repositories, String repositoryUuid, String originalName) {
+        for (Repository repository : repositories.values()) {
+            if (repository.getMetadata().uuid().equals(repositoryUuid)) {
+                return repository;
+            }
+        }
+        throw new RepositoryMissingException("uuid [" + repositoryUuid + "], original name [" + originalName + "]");
+    }
+
     /**
      * A {@link FilterBlobContainer} that uses {@link BlobStoreRepository#maybeRateLimitRestores(InputStream)} to limit the rate at which
      * blobs are read from the repository.

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

@@ -55,7 +55,7 @@ import java.util.concurrent.ConcurrentMap;
 
 import static org.elasticsearch.xpack.searchablesnapshots.SearchableSnapshots.SNAPSHOT_INDEX_ID_SETTING;
 import static org.elasticsearch.xpack.searchablesnapshots.SearchableSnapshots.SNAPSHOT_INDEX_NAME_SETTING;
-import static org.elasticsearch.xpack.searchablesnapshots.SearchableSnapshots.SNAPSHOT_REPOSITORY_SETTING;
+import static org.elasticsearch.xpack.searchablesnapshots.SearchableSnapshots.SNAPSHOT_REPOSITORY_NAME_SETTING;
 import static org.elasticsearch.xpack.searchablesnapshots.SearchableSnapshots.SNAPSHOT_SNAPSHOT_ID_SETTING;
 import static org.elasticsearch.xpack.searchablesnapshots.SearchableSnapshots.SNAPSHOT_SNAPSHOT_NAME_SETTING;
 
@@ -114,7 +114,7 @@ public class SearchableSnapshotAllocator implements ExistingShardsAllocator {
                 SNAPSHOT_SNAPSHOT_NAME_SETTING.get(indexSettings),
                 SNAPSHOT_SNAPSHOT_ID_SETTING.get(indexSettings)
             );
-            final String repository = SNAPSHOT_REPOSITORY_SETTING.get(indexSettings);
+            final String repository = SNAPSHOT_REPOSITORY_NAME_SETTING.get(indexSettings);
             final Snapshot snapshot = new Snapshot(repository, snapshotId);
 
             shardRouting = unassignedAllocationHandler.updateUnassigned(

+ 9 - 2
x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshots.java

@@ -106,12 +106,18 @@ import static org.elasticsearch.xpack.searchablesnapshots.SearchableSnapshotsUti
  */
 public class SearchableSnapshots extends Plugin implements IndexStorePlugin, EnginePlugin, ActionPlugin, ClusterPlugin, SystemIndexPlugin {
 
-    public static final Setting<String> SNAPSHOT_REPOSITORY_SETTING = Setting.simpleString(
+    public static final Setting<String> SNAPSHOT_REPOSITORY_NAME_SETTING = Setting.simpleString(
         "index.store.snapshot.repository_name",
         Setting.Property.IndexScope,
         Setting.Property.PrivateIndex,
         Setting.Property.NotCopyableOnResize
     );
+    public static final Setting<String> SNAPSHOT_REPOSITORY_UUID_SETTING = Setting.simpleString(
+        "index.store.snapshot.repository_uuid",
+        Setting.Property.IndexScope,
+        Setting.Property.PrivateIndex,
+        Setting.Property.NotCopyableOnResize
+    );
     public static final Setting<String> SNAPSHOT_SNAPSHOT_NAME_SETTING = Setting.simpleString(
         "index.store.snapshot.snapshot_name",
         Setting.Property.IndexScope,
@@ -191,7 +197,8 @@ public class SearchableSnapshots extends Plugin implements IndexStorePlugin, Eng
     @Override
     public List<Setting<?>> getSettings() {
         return List.of(
-            SNAPSHOT_REPOSITORY_SETTING,
+            SNAPSHOT_REPOSITORY_UUID_SETTING,
+            SNAPSHOT_REPOSITORY_NAME_SETTING,
             SNAPSHOT_SNAPSHOT_NAME_SETTING,
             SNAPSHOT_SNAPSHOT_ID_SETTING,
             SNAPSHOT_INDEX_NAME_SETTING,

+ 11 - 8
x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/action/TransportMountSearchableSnapshotAction.java

@@ -51,11 +51,9 @@ import static org.elasticsearch.xpack.searchablesnapshots.SearchableSnapshots.DA
 
 /**
  * Action that mounts a snapshot as a searchable snapshot, by converting the mount request into a restore request with specific settings
- * using {@link TransportMountSearchableSnapshotAction#buildIndexSettings(String, SnapshotId, IndexId)}.
+ * using {@link #buildIndexSettings}.
  *
- * This action doesn't technically need to run on the master node, but it needs to get metadata from the repository and we only expect the
- * repository to be accessible from data and master-eligible nodes so we can't run it everywhere.  Given that we already have a way to run
- * actions on the master and that we have to do the restore via the master, it's simplest to use {@link TransportMasterNodeAction}.
+ * This action needs to run on the master node because it retrieves the {@link RepositoryData}.
  */
 public class TransportMountSearchableSnapshotAction extends TransportMasterNodeAction<
     MountSearchableSnapshotRequest,
@@ -106,9 +104,14 @@ public class TransportMountSearchableSnapshotAction extends TransportMasterNodeA
     /**
      * Return the index settings required to make a snapshot searchable
      */
-    private static Settings buildIndexSettings(String repoName, SnapshotId snapshotId, IndexId indexId) {
-        return Settings.builder()
-            .put(SearchableSnapshots.SNAPSHOT_REPOSITORY_SETTING.getKey(), repoName)
+    private static Settings buildIndexSettings(String repoUuid, String repoName, SnapshotId snapshotId, IndexId indexId) {
+        final Settings.Builder settings = Settings.builder();
+
+        if (repoUuid.equals(RepositoryData.MISSING_UUID) == false) {
+            settings.put(SearchableSnapshots.SNAPSHOT_REPOSITORY_UUID_SETTING.getKey(), repoUuid);
+        }
+
+        return settings.put(SearchableSnapshots.SNAPSHOT_REPOSITORY_NAME_SETTING.getKey(), repoName)
             .put(SearchableSnapshots.SNAPSHOT_SNAPSHOT_NAME_SETTING.getKey(), snapshotId.getName())
             .put(SearchableSnapshots.SNAPSHOT_SNAPSHOT_ID_SETTING.getKey(), snapshotId.getUUID())
             .put(SearchableSnapshots.SNAPSHOT_INDEX_NAME_SETTING.getKey(), indexId.getName())
@@ -178,7 +181,7 @@ public class TransportMountSearchableSnapshotAction extends TransportMasterNodeA
                                 .put(IndexMetadata.SETTING_AUTO_EXPAND_REPLICAS, false) // can be overridden
                                 .put(DataTierAllocationDecider.INDEX_ROUTING_PREFER, DATA_TIERS_PREFERENCE)
                                 .put(request.indexSettings())
-                                .put(buildIndexSettings(request.repositoryName(), snapshotId, indexId))
+                                .put(buildIndexSettings(repoData.getUuid(), request.repositoryName(), snapshotId, indexId))
                                 .build()
                         )
                         // Pass through ignored index settings

+ 2 - 2
x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/index/store/SearchableSnapshotDirectoryTests.java

@@ -119,7 +119,7 @@ import static org.elasticsearch.xpack.searchablesnapshots.SearchableSnapshots.SN
 import static org.elasticsearch.xpack.searchablesnapshots.SearchableSnapshots.SNAPSHOT_CACHE_PREWARM_ENABLED_SETTING;
 import static org.elasticsearch.xpack.searchablesnapshots.SearchableSnapshots.SNAPSHOT_INDEX_ID_SETTING;
 import static org.elasticsearch.xpack.searchablesnapshots.SearchableSnapshots.SNAPSHOT_INDEX_NAME_SETTING;
-import static org.elasticsearch.xpack.searchablesnapshots.SearchableSnapshots.SNAPSHOT_REPOSITORY_SETTING;
+import static org.elasticsearch.xpack.searchablesnapshots.SearchableSnapshots.SNAPSHOT_REPOSITORY_NAME_SETTING;
 import static org.elasticsearch.xpack.searchablesnapshots.SearchableSnapshots.SNAPSHOT_SNAPSHOT_ID_SETTING;
 import static org.elasticsearch.xpack.searchablesnapshots.SearchableSnapshots.SNAPSHOT_SNAPSHOT_NAME_SETTING;
 import static org.elasticsearch.xpack.searchablesnapshots.SearchableSnapshotsUtils.toIntBytes;
@@ -740,7 +740,7 @@ public class SearchableSnapshotDirectoryTests extends AbstractSearchableSnapshot
 
     public void testRequiresAdditionalSettings() {
         final List<Setting<String>> requiredSettings = List.of(
-            SNAPSHOT_REPOSITORY_SETTING,
+            SNAPSHOT_REPOSITORY_NAME_SETTING,
             SNAPSHOT_INDEX_NAME_SETTING,
             SNAPSHOT_INDEX_ID_SETTING,
             SNAPSHOT_SNAPSHOT_NAME_SETTING,