ソースを参照

Make SnapshotsInProgress.Entry#indices a Map (#72196)

We constantly need to look up from index name to `IndexId`, might as well
just have this field as a map to simplify some code here and there and save
a few cycles during cluster state updates for snapshots of large index counts.
Armin Braun 4 年 前
コミット
afb42fbf1b

+ 1 - 1
server/src/internalClusterTest/java/org/elasticsearch/cluster/ClusterStateDiffIT.java

@@ -707,7 +707,7 @@ public class ClusterStateDiffIT extends ESIntegTestCase {
                                 randomBoolean(),
                                 randomBoolean(),
                                 SnapshotsInProgressSerializationTests.randomState(ImmutableOpenMap.of()),
-                                Collections.emptyList(),
+                                Collections.emptyMap(),
                                 Collections.emptyList(),
                                 Collections.emptyList(),
                                 Math.abs(randomLong()), randomIntBetween(0, 1000),

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

@@ -143,7 +143,6 @@ public class TransportSnapshotsStatusAction extends TransportMasterNodeAction<Sn
             for (SnapshotsInProgress.Entry entry : currentSnapshotEntries) {
                 currentSnapshotNames.add(entry.snapshot().getSnapshotId().getName());
                 List<SnapshotIndexShardStatus> shardStatusBuilder = new ArrayList<>();
-                Map<String, IndexId> indexIdLookup = null;
                 for (ObjectObjectCursor<ShardId, SnapshotsInProgress.ShardSnapshotStatus> shardEntry : entry.shards()) {
                     SnapshotsInProgress.ShardSnapshotStatus status = shardEntry.value;
                     if (status.nodeId() != null) {
@@ -197,12 +196,9 @@ public class TransportSnapshotsStatusAction extends TransportMasterNodeAction<Sn
                     if (stage == SnapshotIndexShardStage.DONE) {
                         // Shard snapshot completed successfully so we should be able to load the exact statistics for this
                         // shard from the repository already.
-                        if (indexIdLookup == null) {
-                            indexIdLookup = entry.indices().stream().collect(Collectors.toMap(IndexId::getName, Function.identity()));
-                        }
                         final ShardId shardId = shardEntry.key;
                         shardStatus = new SnapshotIndexShardStatus(shardId, repositoriesService.repository(entry.repository())
-                            .getShardSnapshotStatus(entry.snapshot().getSnapshotId(), indexIdLookup.get(shardId.getIndexName()),
+                            .getShardSnapshotStatus(entry.snapshot().getSnapshotId(), entry.indices().get(shardId.getIndexName()),
                                 shardId).asCopy());
                     } else {
                         shardStatus = new SnapshotIndexShardStatus(shardEntry.key, stage);

+ 24 - 12
server/src/main/java/org/elasticsearch/cluster/SnapshotsInProgress.java

@@ -40,7 +40,6 @@ import java.util.List;
 import java.util.Map;
 import java.util.Objects;
 import java.util.Set;
-import java.util.stream.Collectors;
 
 /**
  * Meta data about snapshots that are currently executing
@@ -142,7 +141,7 @@ public class SnapshotsInProgress extends AbstractNamedDiffable<Custom> implement
      * Creates the initial {@link Entry} when starting a snapshot, if no shard-level snapshot work is to be done the resulting entry
      * will be in state {@link State#SUCCESS} right away otherwise it will be in state {@link State#STARTED}.
      */
-    public static Entry startedEntry(Snapshot snapshot, boolean includeGlobalState, boolean partial, List<IndexId> indices,
+    public static Entry startedEntry(Snapshot snapshot, boolean includeGlobalState, boolean partial, Map<String, IndexId> indices,
                                      List<String> dataStreams, long startTime, long repositoryStateId,
                                      ImmutableOpenMap<ShardId, ShardSnapshotStatus> shards,
                                      Map<String, Object> userMetadata, Version version, List<SnapshotFeatureInfo> featureStates) {
@@ -162,7 +161,7 @@ public class SnapshotsInProgress extends AbstractNamedDiffable<Custom> implement
      * @param version repository metadata version to write
      * @return snapshot clone entry
      */
-    public static Entry startClone(Snapshot snapshot, SnapshotId source, List<IndexId> indices, long startTime,
+    public static Entry startClone(Snapshot snapshot, SnapshotId source, Map<String, IndexId> indices, long startTime,
                                    long repositoryStateId, Version version) {
         return new SnapshotsInProgress.Entry(snapshot, true, false, State.STARTED, indices, Collections.emptyList(),
             Collections.emptyList(), startTime, repositoryStateId, ImmutableOpenMap.of(), null, Collections.emptyMap(), version,
@@ -497,7 +496,10 @@ public class SnapshotsInProgress extends AbstractNamedDiffable<Custom> implement
          * Map of {@link ShardId} to {@link ShardSnapshotStatus} tracking the state of each shard snapshot operation.
          */
         private final ImmutableOpenMap<ShardId, ShardSnapshotStatus> shards;
-        private final List<IndexId> indices;
+        /**
+         * Map of index name to {@link IndexId}.
+         */
+        private final Map<String, IndexId> indices;
         private final List<String> dataStreams;
         private final List<SnapshotFeatureInfo> featureStates;
         private final long startTime;
@@ -520,7 +522,7 @@ public class SnapshotsInProgress extends AbstractNamedDiffable<Custom> implement
         @Nullable private final String failure;
 
         // visible for testing, use #startedEntry and copy constructors in production code
-        public Entry(Snapshot snapshot, boolean includeGlobalState, boolean partial, State state, List<IndexId> indices,
+        public Entry(Snapshot snapshot, boolean includeGlobalState, boolean partial, State state, Map<String, IndexId> indices,
                      List<String> dataStreams, List<SnapshotFeatureInfo> featureStates, long startTime, long repositoryStateId,
                      ImmutableOpenMap<ShardId, ShardSnapshotStatus> shards, String failure, Map<String, Object> userMetadata,
                      Version version) {
@@ -528,7 +530,7 @@ public class SnapshotsInProgress extends AbstractNamedDiffable<Custom> implement
                     failure, userMetadata, version, null, ImmutableOpenMap.of());
         }
 
-        private Entry(Snapshot snapshot, boolean includeGlobalState, boolean partial, State state, List<IndexId> indices,
+        private Entry(Snapshot snapshot, boolean includeGlobalState, boolean partial, State state, Map<String, IndexId> indices,
                       List<String> dataStreams, List<SnapshotFeatureInfo> featureStates, long startTime, long repositoryStateId,
                       ImmutableOpenMap<ShardId, ShardSnapshotStatus> shards, String failure, Map<String, Object> userMetadata,
                       Version version, @Nullable SnapshotId source,
@@ -561,7 +563,17 @@ public class SnapshotsInProgress extends AbstractNamedDiffable<Custom> implement
             includeGlobalState = in.readBoolean();
             partial = in.readBoolean();
             state = State.fromValue(in.readByte());
-            indices = in.readList(IndexId::new);
+            final int indexCount = in.readVInt();
+            if (indexCount == 0) {
+                indices = Collections.emptyMap();
+            } else {
+                final Map<String, IndexId> idx = new HashMap<>(indexCount);
+                for (int i = 0; i < indexCount; i++) {
+                    final IndexId indexId = new IndexId(in);
+                    idx.put(indexId.getName(), indexId);
+                }
+                indices = Collections.unmodifiableMap(idx);
+            }
             startTime = in.readLong();
             shards = in.readImmutableMap(ShardId::new, ShardSnapshotStatus::readFrom);
             repositoryStateId = in.readLong();
@@ -574,13 +586,13 @@ public class SnapshotsInProgress extends AbstractNamedDiffable<Custom> implement
             featureStates = Collections.unmodifiableList(in.readList(SnapshotFeatureInfo::new));
         }
 
-        private static boolean assertShardsConsistent(SnapshotId source, State state, List<IndexId> indices,
+        private static boolean assertShardsConsistent(SnapshotId source, State state, Map<String, IndexId> indices,
                                                       ImmutableOpenMap<ShardId, ShardSnapshotStatus> shards,
                                                       ImmutableOpenMap<RepositoryShardId, ShardSnapshotStatus> clones) {
             if ((state == State.INIT || state == State.ABORTED) && shards.isEmpty()) {
                 return true;
             }
-            final Set<String> indexNames = indices.stream().map(IndexId::getName).collect(Collectors.toSet());
+            final Set<String> indexNames = indices.keySet();
             final Set<String> indexNamesInShards = new HashSet<>();
             shards.iterator().forEachRemaining(s -> {
                 indexNamesInShards.add(s.key.getIndexName());
@@ -702,7 +714,7 @@ public class SnapshotsInProgress extends AbstractNamedDiffable<Custom> implement
             return state;
         }
 
-        public List<IndexId> indices() {
+        public Map<String, IndexId> indices() {
             return indices;
         }
 
@@ -821,7 +833,7 @@ public class SnapshotsInProgress extends AbstractNamedDiffable<Custom> implement
             builder.field("state", state);
             builder.startArray("indices");
             {
-                for (IndexId index : indices) {
+                for (IndexId index : indices.values()) {
                     index.toXContent(builder, params);
                 }
             }
@@ -875,7 +887,7 @@ public class SnapshotsInProgress extends AbstractNamedDiffable<Custom> implement
             out.writeBoolean(includeGlobalState);
             out.writeBoolean(partial);
             out.writeByte(state.value());
-            out.writeList(indices);
+            out.writeCollection(indices.values());
             out.writeLong(startTime);
             out.writeMap(shards);
             out.writeLong(repositoryStateId);

+ 9 - 8
server/src/main/java/org/elasticsearch/repositories/RepositoryData.java

@@ -555,12 +555,13 @@ public final class RepositoryData {
     /**
      * Resolve the given index names to index ids.
      */
-    public List<IndexId> resolveIndices(final List<String> indices) {
-        List<IndexId> resolvedIndices = new ArrayList<>(indices.size());
+    public Map<String, IndexId> resolveIndices(final List<String> indices) {
+        Map<String, IndexId> resolvedIndices = new HashMap<>(indices.size());
         for (final String indexName : indices) {
-            resolvedIndices.add(resolveIndexId(indexName));
+            final IndexId indexId = resolveIndexId(indexName);
+            resolvedIndices.put(indexId.getName(), indexId);
         }
-        return resolvedIndices;
+        return Collections.unmodifiableMap(resolvedIndices);
     }
 
     /**
@@ -570,8 +571,8 @@ public final class RepositoryData {
      * @param indicesToResolve names of indices to resolve
      * @param inFlightIds      name to index mapping for currently in-flight snapshots not yet in the repository data to fall back to
      */
-    public List<IndexId> resolveNewIndices(List<String> indicesToResolve, Map<String, IndexId> inFlightIds) {
-        List<IndexId> snapshotIndices = new ArrayList<>();
+    public Map<String, IndexId> resolveNewIndices(List<String> indicesToResolve, Map<String, IndexId> inFlightIds) {
+        Map<String, IndexId> snapshotIndices = new HashMap<>(indicesToResolve.size());
         for (String index : indicesToResolve) {
             IndexId indexId = indices.get(index);
             if (indexId == null) {
@@ -580,9 +581,9 @@ public final class RepositoryData {
             if (indexId == null) {
                 indexId = new IndexId(index, UUIDs.randomBase64UUID());
             }
-            snapshotIndices.add(indexId);
+            snapshotIndices.put(indexId.getName(), indexId);
         }
-        return snapshotIndices;
+        return Collections.unmodifiableMap(snapshotIndices);
     }
 
     private static final String SHARD_GENERATIONS = "shard_generations";

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

@@ -296,8 +296,7 @@ public class RestoreService implements ClusterStateApplier {
                 ).distinct().collect(Collectors.toList());
 
                 final Set<String> explicitlyRequestedSystemIndices = new HashSet<>();
-                final List<IndexId> indexIdsInSnapshot = repositoryData.resolveIndices(requestedIndicesIncludingSystem);
-                for (IndexId indexId : indexIdsInSnapshot) {
+                for (IndexId indexId : repositoryData.resolveIndices(requestedIndicesIncludingSystem).values()) {
                     IndexMetadata snapshotIndexMetaData = repository.getSnapshotIndexMetaData(repositoryData, snapshotId, indexId);
                     if (snapshotIndexMetaData.isSystem()) {
                         if (requestedIndicesInSnapshot.contains(indexId.getName())) {

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

@@ -26,7 +26,6 @@ import org.elasticsearch.common.xcontent.ToXContentObject;
 import org.elasticsearch.common.xcontent.XContentBuilder;
 import org.elasticsearch.common.xcontent.XContentParser;
 import org.elasticsearch.common.xcontent.XContentParserUtils;
-import org.elasticsearch.repositories.IndexId;
 import org.elasticsearch.rest.RestStatus;
 
 import java.io.IOException;
@@ -39,7 +38,6 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Objects;
-import java.util.stream.Collectors;
 
 /**
  * Information about a snapshot
@@ -349,7 +347,7 @@ public final class SnapshotInfo implements Comparable<SnapshotInfo>, ToXContent,
     public SnapshotInfo(SnapshotsInProgress.Entry entry) {
         this(
                 entry.snapshot().getSnapshotId(),
-                entry.indices().stream().map(IndexId::getName).collect(Collectors.toList()),
+                List.copyOf(entry.indices().keySet()),
                 entry.dataStreams(),
                 entry.featureStates(),
                 null,

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

@@ -56,8 +56,6 @@ import java.io.IOException;
 import java.util.HashMap;
 import java.util.Iterator;
 import java.util.Map;
-import java.util.function.Function;
-import java.util.stream.Collectors;
 
 import static java.util.Collections.emptyMap;
 
@@ -238,12 +236,10 @@ public class SnapshotShardsService extends AbstractLifecycleComponent implements
     private void startNewShards(SnapshotsInProgress.Entry entry, Map<ShardId, IndexShardSnapshotStatus> startedShards) {
         threadPool.executor(ThreadPool.Names.SNAPSHOT).execute(() -> {
             final Snapshot snapshot = entry.snapshot();
-            final Map<String, IndexId> indicesMap =
-                entry.indices().stream().collect(Collectors.toMap(IndexId::getName, Function.identity()));
             for (final Map.Entry<ShardId, IndexShardSnapshotStatus> shardEntry : startedShards.entrySet()) {
                 final ShardId shardId = shardEntry.getKey();
                 final IndexShardSnapshotStatus snapshotStatus = shardEntry.getValue();
-                final IndexId indexId = indicesMap.get(shardId.getIndexName());
+                final IndexId indexId = entry.indices().get(shardId.getIndexName());
                 assert indexId != null;
                 assert SnapshotsService.useShardGenerations(entry.version()) ||
                         ShardGenerations.fixShardGeneration(snapshotStatus.generation()) == null :

+ 22 - 28
server/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java

@@ -313,11 +313,11 @@ public class SnapshotsService extends AbstractLifecycleComponent implements Clus
 
                 logger.trace("[{}][{}] creating snapshot for indices [{}]", repositoryName, snapshotName, indices);
 
-                final List<IndexId> indexIds = repositoryData.resolveNewIndices(
-                        indices, getInFlightIndexIds(runningSnapshots, repositoryName));
+                final Map<String, IndexId> indexIds =
+                        repositoryData.resolveNewIndices(indices, getInFlightIndexIds(runningSnapshots, repositoryName));
                 final Version version = minCompatibleVersion(currentState.nodes().getMinNodeVersion(), repositoryData, null);
                 ImmutableOpenMap<ShardId, ShardSnapshotStatus> shards = shards(snapshots, deletionsInProgress, currentState.metadata(),
-                    currentState.routingTable(), indexIds, useShardGenerations(version), repositoryData, repositoryName);
+                    currentState.routingTable(), indexIds.values(), useShardGenerations(version), repositoryData, repositoryName);
                 if (request.partial() == false) {
                     Set<String> missing = new HashSet<>();
                     for (ObjectObjectCursor<ShardId, SnapshotsInProgress.ShardSnapshotStatus> entry : shards) {
@@ -369,9 +369,13 @@ public class SnapshotsService extends AbstractLifecycleComponent implements Clus
     }
 
     private static Map<String, IndexId> getInFlightIndexIds(List<SnapshotsInProgress.Entry> runningSnapshots, String repositoryName) {
-        return runningSnapshots.stream().filter(entry -> entry.repository().equals(repositoryName))
-                .flatMap(entry -> entry.indices().stream()).distinct()
-                .collect(Collectors.toMap(IndexId::getName, Function.identity()));
+        final Map<String, IndexId> allIndices = new HashMap<>();
+        for (SnapshotsInProgress.Entry runningSnapshot : runningSnapshots) {
+            if (runningSnapshot.repository().equals(repositoryName)) {
+                allIndices.putAll(runningSnapshot.indices());
+            }
+        }
+        return Collections.unmodifiableMap(allIndices);
     }
 
     // TODO: It is worth revisiting the design choice of creating a placeholder entry in snapshots-in-progress here once we have a cache
@@ -477,7 +481,7 @@ public class SnapshotsService extends AbstractLifecycleComponent implements Clus
      * @param cloneEntry     clone operation in the cluster state
      */
     private void startCloning(Repository repository, SnapshotsInProgress.Entry cloneEntry) {
-        final List<IndexId> indices = cloneEntry.indices();
+        final Collection<IndexId> indices = cloneEntry.indices().values();
         final SnapshotId sourceSnapshot = cloneEntry.source();
         final Snapshot targetSnapshot = cloneEntry.snapshot();
 
@@ -689,11 +693,9 @@ public class SnapshotsService extends AbstractLifecycleComponent implements Clus
 
     private static ShardGenerations buildGenerations(SnapshotsInProgress.Entry snapshot, Metadata metadata) {
         ShardGenerations.Builder builder = ShardGenerations.builder();
-        final Map<String, IndexId> indexLookup = new HashMap<>();
-        snapshot.indices().forEach(idx -> indexLookup.put(idx.getName(), idx));
         if (snapshot.isClone()) {
             snapshot.clones().forEach(c -> {
-                final IndexId indexId = indexLookup.get(c.key.indexName());
+                final IndexId indexId = snapshot.indices().get(c.key.indexName());
                 builder.put(indexId, c.key.shardId(), c.value.generation());
             });
         } else {
@@ -703,7 +705,7 @@ public class SnapshotsService extends AbstractLifecycleComponent implements Clus
                             "Index [" + c.key.getIndex() + "] was deleted during a snapshot but snapshot was not partial.";
                     return;
                 }
-                final IndexId indexId = indexLookup.get(c.key.getIndexName());
+                final IndexId indexId = snapshot.indices().get(c.key.getIndexName());
                 if (indexId != null) {
                     builder.put(indexId, c.key.id(), c.value.generation());
                 }
@@ -717,7 +719,7 @@ public class SnapshotsService extends AbstractLifecycleComponent implements Clus
         if (snapshot.includeGlobalState() == false) {
             // Remove global state from the cluster state
             builder = Metadata.builder();
-            for (IndexId index : snapshot.indices()) {
+            for (IndexId index : snapshot.indices().values()) {
                 final IndexMetadata indexMetadata = metadata.index(index.getName());
                 if (indexMetadata == null) {
                     assert snapshot.partial() : "Index [" + index + "] was deleted during a snapshot but snapshot was not partial.";
@@ -731,7 +733,7 @@ public class SnapshotsService extends AbstractLifecycleComponent implements Clus
         // Only keep those data streams in the metadata that were actually requested by the initial snapshot create operation and that have
         // all their indices contained in the snapshot
         final Map<String, DataStream> dataStreams = new HashMap<>();
-        final Set<String> indicesInSnapshot = snapshot.indices().stream().map(IndexId::getName).collect(Collectors.toSet());
+        final Set<String> indicesInSnapshot = snapshot.indices().keySet();
         for (String dataStreamName : snapshot.dataStreams()) {
             DataStream dataStream = metadata.dataStreams().get(dataStreamName);
             if (dataStream == null) {
@@ -1237,7 +1239,7 @@ public class SnapshotsService extends AbstractLifecycleComponent implements Clus
                             final Metadata existing = repo.getSnapshotGlobalMetadata(entry.source());
                             final Metadata.Builder metaBuilder = Metadata.builder(existing);
                             final Set<Index> existingIndices = new HashSet<>();
-                            for (IndexId index : entry.indices()) {
+                            for (IndexId index : entry.indices().values()) {
                                 final IndexMetadata indexMetadata = repo.getSnapshotIndexMetaData(repositoryData, entry.source(), index);
                                 existingIndices.add(indexMetadata.getIndex());
                                 metaBuilder.put(indexMetadata, false);
@@ -2189,8 +2191,8 @@ public class SnapshotsService extends AbstractLifecycleComponent implements Clus
                             } else {
                                 if (shardAssignments == null) {
                                     shardAssignments = shards(snapshotsInProgress,
-                                            updatedDeletions, currentState.metadata(), currentState.routingTable(), entry.indices(),
-                                            entry.version().onOrAfter(SHARD_GEN_IN_REPO_DATA_VERSION), repositoryData, repoName);
+                                        updatedDeletions, currentState.metadata(), currentState.routingTable(), entry.indices().values(),
+                                        entry.version().onOrAfter(SHARD_GEN_IN_REPO_DATA_VERSION), repositoryData, repoName);
                                 }
                                 final ImmutableOpenMap.Builder<ShardId, ShardSnapshotStatus> updatedAssignmentsBuilder =
                                         ImmutableOpenMap.builder(entry.shards());
@@ -2291,7 +2293,7 @@ public class SnapshotsService extends AbstractLifecycleComponent implements Clus
      */
     private static ImmutableOpenMap<ShardId, SnapshotsInProgress.ShardSnapshotStatus> shards(
             SnapshotsInProgress snapshotsInProgress, SnapshotDeletionsInProgress deletionsInProgress,
-            Metadata metadata, RoutingTable routingTable, List<IndexId> indices, boolean useShardGenerations,
+            Metadata metadata, RoutingTable routingTable, Collection<IndexId> indices, boolean useShardGenerations,
             RepositoryData repositoryData, String repoName) {
         ImmutableOpenMap.Builder<ShardId, SnapshotsInProgress.ShardSnapshotStatus> builder = ImmutableOpenMap.builder();
         final ShardGenerations shardGenerations = repositoryData.shardGenerations();
@@ -2390,8 +2392,8 @@ public class SnapshotsService extends AbstractLifecycleComponent implements Clus
         final Set<Index> indices = new HashSet<>();
         for (final SnapshotsInProgress.Entry entry : snapshots.entries()) {
             if (entry.partial() == false) {
-                for (IndexId index : entry.indices()) {
-                    IndexMetadata indexMetadata = currentState.metadata().index(index.getName());
+                for (String indexName : entry.indices().keySet()) {
+                    IndexMetadata indexMetadata = currentState.metadata().index(indexName);
                     if (indexMetadata != null && indicesToCheck.contains(indexMetadata.getIndex())) {
                         indices.add(indexMetadata.getIndex());
                     }
@@ -2541,9 +2543,6 @@ public class SnapshotsService extends AbstractLifecycleComponent implements Clus
             // builder for updated shard clone status mappings if any could be computed
             private ImmutableOpenMap.Builder<RepositoryShardId, ShardSnapshotStatus> clonesBuilder = null;
 
-            // index lookup (index name -> IndexId) cache for #entry used to translate from ShardId to RepositoryShardId
-            private Map<String, IndexId> indicesLookup = null;
-
             EntryContext(SnapshotsInProgress.Entry entry, Iterator<ShardSnapshotUpdate> iterator) {
                 this.entry = entry;
                 this.iterator = iterator;
@@ -2690,13 +2689,8 @@ public class SnapshotsService extends AbstractLifecycleComponent implements Clus
 
             private void tryStartCloneAfterSnapshotFinish(ShardId shardId, ShardSnapshotStatus updatedState) {
                 // shard snapshot was completed, we check if we can start a clone operation for the same repo shard
-                if (indicesLookup == null) {
-                    // cache lookup of index name to IndexId for future ShardId -> RepositoryShardId translations
-                    indicesLookup = entry.indices().stream().collect(Collectors.toMap(IndexId::getName, Function.identity()));
-                }
-                final IndexId indexId = indicesLookup.get(shardId.getIndexName());
+                final IndexId indexId = entry.indices().get(shardId.getIndexName());
                 // If the lookup finds the index id then at least the entry is concerned with the index id just updated
-                // so we check on a shard level
                 if (indexId != null) {
                     final RepositoryShardId repoShardId = new RepositoryShardId(indexId, shardId.getId());
                     if (isQueued(entry.clones().get(repoShardId))) {

+ 2 - 2
server/src/test/java/org/elasticsearch/cluster/metadata/MetadataDeleteIndexServiceTests.java

@@ -37,7 +37,7 @@ import java.util.stream.Collectors;
 import java.util.stream.IntStream;
 
 import static java.util.Collections.singleton;
-import static java.util.Collections.singletonList;
+import static java.util.Collections.singletonMap;
 import static org.hamcrest.CoreMatchers.containsString;
 import static org.hamcrest.CoreMatchers.equalTo;
 import static org.mockito.Matchers.any;
@@ -71,7 +71,7 @@ public class MetadataDeleteIndexServiceTests extends ESTestCase {
         String index = randomAlphaOfLength(5);
         Snapshot snapshot = new Snapshot("doesn't matter", new SnapshotId("snapshot name", "snapshot uuid"));
         SnapshotsInProgress snaps = SnapshotsInProgress.of(List.of(new SnapshotsInProgress.Entry(snapshot, true, false,
-                SnapshotsInProgress.State.INIT, singletonList(new IndexId(index, "doesn't matter")),
+                SnapshotsInProgress.State.INIT, singletonMap(index, new IndexId(index, "doesn't matter")),
                 Collections.emptyList(), Collections.emptyList(), System.currentTimeMillis(), (long) randomIntBetween(0, 1000),
                 ImmutableOpenMap.of(), null, SnapshotInfoTestUtils.randomUserMetadata(), VersionUtils.randomVersion(random()))));
         ClusterState state = ClusterState.builder(clusterState(index))

+ 1 - 1
server/src/test/java/org/elasticsearch/cluster/metadata/MetadataIndexStateServiceTests.java

@@ -348,7 +348,7 @@ public class MetadataIndexStateServiceTests extends ESTestCase {
         final Snapshot snapshot = new Snapshot(randomAlphaOfLength(10), new SnapshotId(randomAlphaOfLength(5), randomAlphaOfLength(5)));
         final SnapshotsInProgress.Entry entry =
             new SnapshotsInProgress.Entry(snapshot, randomBoolean(), false, SnapshotsInProgress.State.INIT,
-                Collections.singletonList(new IndexId(index, index)), Collections.emptyList(), Collections.emptyList(),
+                Collections.singletonMap(index, new IndexId(index, index)), Collections.emptyList(), Collections.emptyList(),
                 randomNonNegativeLong(), randomLong(), shardsBuilder.build(), null, SnapshotInfoTestUtils.randomUserMetadata(),
                 VersionUtils.randomVersion(random())
             );

+ 8 - 6
server/src/test/java/org/elasticsearch/snapshots/SnapshotsInProgressSerializationTests.java

@@ -61,15 +61,16 @@ public class SnapshotsInProgressSerializationTests extends AbstractDiffableWireS
         boolean includeGlobalState = randomBoolean();
         boolean partial = randomBoolean();
         int numberOfIndices = randomIntBetween(0, 10);
-        List<IndexId> indices = new ArrayList<>();
+        Map<String, IndexId> indices = new HashMap<>();
         for (int i = 0; i < numberOfIndices; i++) {
-            indices.add(new IndexId(randomAlphaOfLength(10), randomAlphaOfLength(10)));
+            final String name = randomAlphaOfLength(10);
+            indices.put(name, new IndexId(name, randomAlphaOfLength(10)));
         }
         long startTime = randomLong();
         long repositoryStateId = randomLong();
         ImmutableOpenMap.Builder<ShardId, SnapshotsInProgress.ShardSnapshotStatus> builder = ImmutableOpenMap.builder();
         final List<Index> esIndices =
-            indices.stream().map(i -> new Index(i.getName(), randomAlphaOfLength(10))).collect(Collectors.toList());
+            indices.keySet().stream().map(i -> new Index(i, randomAlphaOfLength(10))).collect(Collectors.toList());
         List<String> dataStreams = Arrays.asList(generateRandomStringArray(10, 10, false));
         for (Index idx : esIndices) {
             int shardsCount = randomIntBetween(1, 10);
@@ -195,10 +196,10 @@ public class SnapshotsInProgressSerializationTests extends AbstractDiffableWireS
                     entry.dataStreams(), entry.featureStates(), entry.startTime(), entry.repositoryStateId(), entry.shards(), failure,
                     entry.userMetadata(), entry.version());
             case 6:
-                List<IndexId> indices = entry.indices();
+                Map<String, IndexId> indices = new HashMap<>(entry.indices());
                 ImmutableOpenMap<ShardId, SnapshotsInProgress.ShardSnapshotStatus> shards = entry.shards();
                 IndexId indexId = new IndexId(randomAlphaOfLength(10), randomAlphaOfLength(10));
-                indices.add(indexId);
+                indices.put(indexId. getName(), indexId);
                 ImmutableOpenMap.Builder<ShardId, SnapshotsInProgress.ShardSnapshotStatus> builder = ImmutableOpenMap.builder(shards);
                 Index index = new Index(indexId.getName(), randomAlphaOfLength(10));
                 int shardsCount = randomIntBetween(1, 10);
@@ -233,10 +234,11 @@ public class SnapshotsInProgressSerializationTests extends AbstractDiffableWireS
     }
 
     public void testXContent() throws IOException {
+        final IndexId indexId = new IndexId("index", "uuid");
         SnapshotsInProgress sip =
             SnapshotsInProgress.of(Collections.singletonList(new Entry(
                 new Snapshot("repo", new SnapshotId("name", "uuid")), true, true, State.SUCCESS,
-                Collections.singletonList(new IndexId("index", "uuid")), Collections.emptyList(), Collections.emptyList(), 1234567, 0,
+                Collections.singletonMap(indexId.getName(), indexId), Collections.emptyList(), Collections.emptyList(), 1234567, 0,
                 ImmutableOpenMap.<ShardId, SnapshotsInProgress.ShardSnapshotStatus>builder()
                     .fPut(new ShardId("index", "uuid", 0),
                         SnapshotsInProgress.ShardSnapshotStatus.success("nodeId",

+ 18 - 14
server/src/test/java/org/elasticsearch/snapshots/SnapshotsServiceTests.java

@@ -32,7 +32,8 @@ import org.elasticsearch.test.ESTestCase;
 
 import java.util.Arrays;
 import java.util.Collections;
-import java.util.List;
+import java.util.Map;
+import java.util.function.Function;
 import java.util.stream.Collectors;
 import java.util.stream.StreamSupport;
 
@@ -44,7 +45,7 @@ public class SnapshotsServiceTests extends ESTestCase {
     public void testNoopShardStateUpdates() throws Exception {
         final String repoName = "test-repo";
         final Snapshot snapshot = snapshot(repoName, "snapshot-1");
-        final SnapshotsInProgress.Entry snapshotNoShards = snapshotEntry(snapshot, Collections.emptyList(), ImmutableOpenMap.of());
+        final SnapshotsInProgress.Entry snapshotNoShards = snapshotEntry(snapshot, Collections.emptyMap(), ImmutableOpenMap.of());
 
         final String indexName1 = "index-1";
         final ShardId shardId1 = new ShardId(index(indexName1), 0);
@@ -55,9 +56,10 @@ public class SnapshotsServiceTests extends ESTestCase {
             assertIsNoop(state, shardCompletion);
         }
         {
+            final IndexId indexId = indexId(indexName1);
             final ClusterState state = stateWithSnapshots(
                     snapshotEntry(
-                            snapshot, Collections.singletonList(indexId(indexName1)), shardsMap(shardId1, initShardStatus(uuid()))));
+                            snapshot, Collections.singletonMap(indexId.getName(), indexId), shardsMap(shardId1, initShardStatus(uuid()))));
             final SnapshotsService.ShardSnapshotUpdate shardCompletion = new SnapshotsService.ShardSnapshotUpdate(
                     snapshot("other-repo", snapshot.getSnapshotId().getName()), shardId1, successfulShardStatus(uuid()));
             assertIsNoop(state, shardCompletion);
@@ -72,7 +74,7 @@ public class SnapshotsServiceTests extends ESTestCase {
         final IndexId indexId1 = indexId(indexName1);
         final ShardId shardId1 = new ShardId(index(indexName1), 0);
         final SnapshotsInProgress.Entry snapshotSingleShard =
-                snapshotEntry(sn1, Collections.singletonList(indexId1), shardsMap(shardId1, initShardStatus(dataNodeId)));
+            snapshotEntry(sn1, Collections.singletonMap(indexId1.getName(), indexId1), shardsMap(shardId1, initShardStatus(dataNodeId)));
 
         assertThat(snapshotSingleShard.state(), is(SnapshotsInProgress.State.STARTED));
 
@@ -94,7 +96,7 @@ public class SnapshotsServiceTests extends ESTestCase {
         final ShardId shardId1 = new ShardId(routingIndex1, 0);
         final ShardId shardId2 = new ShardId(routingIndex1, 1);
         final SnapshotsInProgress.ShardSnapshotStatus shardInitStatus = initShardStatus(dataNodeId);
-        final SnapshotsInProgress.Entry snapshotSingleShard = snapshotEntry(sn1, Collections.singletonList(indexId1),
+        final SnapshotsInProgress.Entry snapshotSingleShard = snapshotEntry(sn1, Collections.singletonMap(indexId1.getName(), indexId1),
                 ImmutableOpenMap.builder(shardsMap(shardId1, shardInitStatus)).fPut(shardId2, shardInitStatus).build());
 
         assertThat(snapshotSingleShard.state(), is(SnapshotsInProgress.State.STARTED));
@@ -166,7 +168,8 @@ public class SnapshotsServiceTests extends ESTestCase {
         final ClusterState stateWithIndex = stateWithUnassignedIndices(indexName1);
         final Snapshot plainSnapshot = snapshot(repoName, "test-snapshot");
         final ShardId routingShardId1 = new ShardId(stateWithIndex.metadata().index(indexName1).getIndex(), 0);
-        final SnapshotsInProgress.Entry snapshotSingleShard = snapshotEntry(plainSnapshot, Collections.singletonList(indexId1),
+        final SnapshotsInProgress.Entry snapshotSingleShard = snapshotEntry(plainSnapshot,
+                Collections.singletonMap(indexId1.getName(), indexId1),
                 shardsMap(routingShardId1, SnapshotsInProgress.ShardSnapshotStatus.UNASSIGNED_QUEUED));
 
         assertThat(cloneSingleShard.state(), is(SnapshotsInProgress.State.STARTED));
@@ -242,8 +245,8 @@ public class SnapshotsServiceTests extends ESTestCase {
         final ClusterState stateWithIndex = stateWithUnassignedIndices(indexName);
         final Snapshot plainSnapshot = snapshot(repoName, "test-snapshot");
         final ShardId routingShardId = new ShardId(stateWithIndex.metadata().index(indexName).getIndex(), 0);
-        final SnapshotsInProgress.Entry snapshotSingleShard = snapshotEntry(plainSnapshot, Collections.singletonList(indexId1),
-                shardsMap(routingShardId, initShardStatus(dataNodeId)));
+        final SnapshotsInProgress.Entry snapshotSingleShard = snapshotEntry(plainSnapshot,
+                Collections.singletonMap(indexId1.getName(), indexId1), shardsMap(routingShardId, initShardStatus(dataNodeId)));
 
         assertThat(cloneSingleShard.state(), is(SnapshotsInProgress.State.STARTED));
 
@@ -270,11 +273,12 @@ public class SnapshotsServiceTests extends ESTestCase {
         final ClusterState stateWithIndex = stateWithUnassignedIndices(indexName);
         final Snapshot plainSnapshot = snapshot(repoName, "test-snapshot-1");
         final ShardId routingShardId = new ShardId(stateWithIndex.metadata().index(indexName).getIndex(), 0);
-        final SnapshotsInProgress.Entry snapshotSingleShard = snapshotEntry(plainSnapshot, Collections.singletonList(indexId1),
-                shardsMap(routingShardId, initShardStatus(dataNodeId)));
+        final SnapshotsInProgress.Entry snapshotSingleShard = snapshotEntry(plainSnapshot,
+            Collections.singletonMap(indexId1.getName(), indexId1), shardsMap(routingShardId, initShardStatus(dataNodeId)));
 
         final Snapshot queuedSnapshot = snapshot(repoName, "test-snapshot-2");
-        final SnapshotsInProgress.Entry queuedSnapshotSingleShard = snapshotEntry(queuedSnapshot, Collections.singletonList(indexId1),
+        final SnapshotsInProgress.Entry queuedSnapshotSingleShard = snapshotEntry(queuedSnapshot,
+                Collections.singletonMap(indexId1.getName(), indexId1),
                 shardsMap(routingShardId, SnapshotsInProgress.ShardSnapshotStatus.UNASSIGNED_QUEUED));
 
         final SnapshotsService.ShardSnapshotUpdate completeShard = successUpdate(plainSnapshot, routingShardId, dataNodeId);
@@ -379,7 +383,7 @@ public class SnapshotsServiceTests extends ESTestCase {
         return SnapshotsService.SHARD_STATE_EXECUTOR.execute(state, Arrays.asList(updates)).resultingState;
     }
 
-    private static SnapshotsInProgress.Entry snapshotEntry(Snapshot snapshot, List<IndexId> indexIds,
+    private static SnapshotsInProgress.Entry snapshotEntry(Snapshot snapshot, Map<String, IndexId> indexIds,
                                                            ImmutableOpenMap<ShardId, SnapshotsInProgress.ShardSnapshotStatus> shards) {
         return SnapshotsInProgress.startedEntry(snapshot, randomBoolean(), randomBoolean(), indexIds, Collections.emptyList(), 1L,
             randomNonNegativeLong(), shards, Collections.emptyMap(), Version.CURRENT, Collections.emptyList());
@@ -387,8 +391,8 @@ public class SnapshotsServiceTests extends ESTestCase {
 
     private static SnapshotsInProgress.Entry cloneEntry(
             Snapshot snapshot, SnapshotId source, ImmutableOpenMap<RepositoryShardId, SnapshotsInProgress.ShardSnapshotStatus> clones) {
-        final List<IndexId> indexIds = StreamSupport.stream(clones.keys().spliterator(), false)
-                .map(k -> k.value.index()).distinct().collect(Collectors.toList());
+        final Map<String, IndexId> indexIds = StreamSupport.stream(clones.keys().spliterator(), false)
+                .map(k -> k.value.index()).distinct().collect(Collectors.toMap(IndexId::getName, Function.identity()));
         return SnapshotsInProgress.startClone(snapshot, source, indexIds, 1L, randomNonNegativeLong(), Version.CURRENT).withClones(clones);
     }
 

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

@@ -126,7 +126,7 @@ public abstract class AsyncRetryDuringSnapshotActionStep extends AsyncActionStep
                                 }
                                 for (SnapshotsInProgress.Entry snapshot :
                                         state.custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY).entries()) {
-                                    if (snapshot.indices().stream().anyMatch(name -> name.getName().equals(indexName))) {
+                                    if (snapshot.indices().containsKey(indexName)) {
                                         // There is a snapshot running with this index name
                                         return false;
                                     }

+ 1 - 1
x-pack/plugin/data-streams/src/test/java/org/elasticsearch/xpack/datastreams/action/DeleteDataStreamTransportActionTests.java

@@ -119,7 +119,7 @@ public class DeleteDataStreamTransportActionTests extends ESTestCase {
             false,
             partial,
             SnapshotsInProgress.State.SUCCESS,
-            Collections.emptyList(),
+            Collections.emptyMap(),
             List.of(dataStreamName),
             Collections.emptyList(),
             0,