Selaa lähdekoodia

Convert o.e.cluster to use Map in public methods (#87168)

Many classes that use ImmutableOpenMap expose the maps directly, when
the consumers actually only use methods from the base Map interface.
This commit converts public methods in subpackages of o.e.cluster to
return Map instead of ImmutableOpenMap. Note that o.e.cluster classes
are not yet converted because they require more involved changes.

relates #86239
Ryan Ernst 3 vuotta sitten
vanhempi
commit
341cc1e812
21 muutettua tiedostoa jossa 60 lisäystä ja 107 poistoa
  1. 6 43
      server/src/internalClusterTest/java/org/elasticsearch/action/admin/indices/create/ShrinkIndexIT.java
  2. 3 2
      server/src/internalClusterTest/java/org/elasticsearch/cluster/ClusterInfoServiceIT.java
  3. 1 2
      server/src/internalClusterTest/java/org/elasticsearch/search/routing/SearchReplicaSelectionIT.java
  4. 1 2
      server/src/internalClusterTest/java/org/elasticsearch/snapshots/SnapshotStressTestsIT.java
  5. 2 2
      server/src/main/java/org/elasticsearch/action/support/tasks/TransportTasksAction.java
  6. 6 6
      server/src/main/java/org/elasticsearch/cluster/ClusterInfo.java
  7. 6 6
      server/src/main/java/org/elasticsearch/cluster/SnapshotsInProgress.java
  8. 2 2
      server/src/main/java/org/elasticsearch/cluster/block/ClusterBlocks.java
  9. 2 2
      server/src/main/java/org/elasticsearch/cluster/metadata/IndexTemplateMetadata.java
  10. 6 6
      server/src/main/java/org/elasticsearch/cluster/node/DiscoveryNodes.java
  11. 2 2
      server/src/main/java/org/elasticsearch/cluster/routing/RoutingTable.java
  12. 2 3
      server/src/main/java/org/elasticsearch/cluster/routing/allocation/DiskThresholdMonitor.java
  13. 7 7
      server/src/main/java/org/elasticsearch/cluster/routing/allocation/decider/DiskThresholdDecider.java
  14. 1 2
      server/src/main/java/org/elasticsearch/snapshots/SnapshotShardsService.java
  15. 2 2
      server/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java
  16. 1 2
      server/src/test/java/org/elasticsearch/cluster/metadata/MetadataIndexTemplateServiceTests.java
  17. 2 2
      server/src/test/java/org/elasticsearch/snapshots/SnapshotsInProgressSerializationTests.java
  18. 1 2
      x-pack/plugin/autoscaling/src/main/java/org/elasticsearch/xpack/autoscaling/capacity/AutoscalingCalculateCapacityService.java
  19. 2 2
      x-pack/plugin/autoscaling/src/test/java/org/elasticsearch/xpack/autoscaling/storage/ReactiveStorageDeciderDecisionTests.java
  20. 2 2
      x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/action/cache/TransportSearchableSnapshotsNodeCachesStatsAction.java
  21. 3 8
      x-pack/plugin/security/src/internalClusterTest/java/org/elasticsearch/integration/ShrinkIndexWithSecurityTests.java

+ 6 - 43
server/src/internalClusterTest/java/org/elasticsearch/action/admin/indices/create/ShrinkIndexIT.java

@@ -37,7 +37,6 @@ import org.elasticsearch.cluster.routing.ShardRouting;
 import org.elasticsearch.cluster.routing.UnassignedInfo;
 import org.elasticsearch.cluster.routing.allocation.decider.EnableAllocationDecider;
 import org.elasticsearch.common.Priority;
-import org.elasticsearch.common.collect.ImmutableOpenMap;
 import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.core.TimeValue;
 import org.elasticsearch.index.Index;
@@ -85,13 +84,7 @@ public class ShrinkIndexIT extends ESIntegTestCase {
                 .setSource("{\"foo\" : \"bar\", \"i\" : " + i + "}", XContentType.JSON)
                 .get();
         }
-        ImmutableOpenMap<String, DiscoveryNode> dataNodes = client().admin()
-            .cluster()
-            .prepareState()
-            .get()
-            .getState()
-            .nodes()
-            .getDataNodes();
+        Map<String, DiscoveryNode> dataNodes = client().admin().cluster().prepareState().get().getState().nodes().getDataNodes();
         assertTrue("at least 2 nodes but was: " + dataNodes.size(), dataNodes.size() >= 2);
         DiscoveryNode[] discoveryNodes = dataNodes.values().toArray(DiscoveryNode[]::new);
         String mergeNode = discoveryNodes[0].getName();
@@ -184,13 +177,7 @@ public class ShrinkIndexIT extends ESIntegTestCase {
         internalCluster().ensureAtLeastNumDataNodes(2);
         prepareCreate("source").setSettings(Settings.builder().put(indexSettings()).put("number_of_shards", numberOfShards)).get();
 
-        final ImmutableOpenMap<String, DiscoveryNode> dataNodes = client().admin()
-            .cluster()
-            .prepareState()
-            .get()
-            .getState()
-            .nodes()
-            .getDataNodes();
+        final Map<String, DiscoveryNode> dataNodes = client().admin().cluster().prepareState().get().getState().nodes().getDataNodes();
         assertThat(dataNodes.size(), greaterThanOrEqualTo(2));
         final DiscoveryNode[] discoveryNodes = dataNodes.values().toArray(DiscoveryNode[]::new);
         final String mergeNode = discoveryNodes[0].getName();
@@ -270,13 +257,7 @@ public class ShrinkIndexIT extends ESIntegTestCase {
         for (int i = 0; i < docs; i++) {
             client().prepareIndex("source").setSource("{\"foo\" : \"bar\", \"i\" : " + i + "}", XContentType.JSON).get();
         }
-        ImmutableOpenMap<String, DiscoveryNode> dataNodes = client().admin()
-            .cluster()
-            .prepareState()
-            .get()
-            .getState()
-            .nodes()
-            .getDataNodes();
+        Map<String, DiscoveryNode> dataNodes = client().admin().cluster().prepareState().get().getState().nodes().getDataNodes();
         assertTrue("at least 2 nodes but was: " + dataNodes.size(), dataNodes.size() >= 2);
         DiscoveryNode[] discoveryNodes = dataNodes.values().toArray(DiscoveryNode[]::new);
         // ensure all shards are allocated otherwise the ensure green below might not succeed since we require the merge node
@@ -400,13 +381,7 @@ public class ShrinkIndexIT extends ESIntegTestCase {
         for (int i = 0; i < 20; i++) {
             client().prepareIndex("source").setSource("{\"foo\" : \"bar\", \"i\" : " + i + "}", XContentType.JSON).get();
         }
-        ImmutableOpenMap<String, DiscoveryNode> dataNodes = client().admin()
-            .cluster()
-            .prepareState()
-            .get()
-            .getState()
-            .nodes()
-            .getDataNodes();
+        Map<String, DiscoveryNode> dataNodes = client().admin().cluster().prepareState().get().getState().nodes().getDataNodes();
         assertTrue("at least 2 nodes but was: " + dataNodes.size(), dataNodes.size() >= 2);
         DiscoveryNode[] discoveryNodes = dataNodes.values().toArray(DiscoveryNode[]::new);
         String spareNode = discoveryNodes[0].getName();
@@ -498,13 +473,7 @@ public class ShrinkIndexIT extends ESIntegTestCase {
                 .setSource("{\"foo\" : \"bar\", \"id\" : " + i + "}", XContentType.JSON)
                 .get();
         }
-        ImmutableOpenMap<String, DiscoveryNode> dataNodes = client().admin()
-            .cluster()
-            .prepareState()
-            .get()
-            .getState()
-            .nodes()
-            .getDataNodes();
+        Map<String, DiscoveryNode> dataNodes = client().admin().cluster().prepareState().get().getState().nodes().getDataNodes();
         assertTrue("at least 2 nodes but was: " + dataNodes.size(), dataNodes.size() >= 2);
         DiscoveryNode[] discoveryNodes = dataNodes.values().toArray(DiscoveryNode[]::new);
         String mergeNode = discoveryNodes[0].getName();
@@ -578,13 +547,7 @@ public class ShrinkIndexIT extends ESIntegTestCase {
             client().prepareIndex("source").setSource("{\"foo\" : \"bar\", \"i\" : " + i + "}", XContentType.JSON).get();
         }
         client().admin().indices().prepareFlush("source").get();
-        ImmutableOpenMap<String, DiscoveryNode> dataNodes = client().admin()
-            .cluster()
-            .prepareState()
-            .get()
-            .getState()
-            .nodes()
-            .getDataNodes();
+        Map<String, DiscoveryNode> dataNodes = client().admin().cluster().prepareState().get().getState().nodes().getDataNodes();
         DiscoveryNode[] discoveryNodes = dataNodes.values().toArray(DiscoveryNode[]::new);
         // ensure all shards are allocated otherwise the ensure green below might not succeed since we require the merge node
         // if we change the setting too quickly we will end up with one replica unassigned which can't be assigned anymore due

+ 3 - 2
server/src/internalClusterTest/java/org/elasticsearch/cluster/ClusterInfoServiceIT.java

@@ -46,6 +46,7 @@ import java.util.Arrays;
 import java.util.Collection;
 import java.util.List;
 import java.util.Locale;
+import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.atomic.AtomicBoolean;
 
@@ -161,8 +162,8 @@ public class ClusterInfoServiceIT extends ESIntegTestCase {
         infoService.setUpdateFrequency(TimeValue.timeValueMillis(200));
         ClusterInfo info = ClusterInfoServiceUtils.refresh(infoService);
         assertNotNull("info should not be null", info);
-        ImmutableOpenMap<String, DiskUsage> leastUsages = info.getNodeLeastAvailableDiskUsages();
-        ImmutableOpenMap<String, DiskUsage> mostUsages = info.getNodeMostAvailableDiskUsages();
+        Map<String, DiskUsage> leastUsages = info.getNodeLeastAvailableDiskUsages();
+        Map<String, DiskUsage> mostUsages = info.getNodeMostAvailableDiskUsages();
         ImmutableOpenMap<String, Long> shardSizes = info.shardSizes;
         ImmutableOpenMap<ShardId, Long> shardDataSetSizes = info.shardDataSetSizes;
         assertNotNull(leastUsages);

+ 1 - 2
server/src/internalClusterTest/java/org/elasticsearch/search/routing/SearchReplicaSelectionIT.java

@@ -15,7 +15,6 @@ import org.elasticsearch.action.search.SearchResponse;
 import org.elasticsearch.client.internal.Client;
 import org.elasticsearch.cluster.node.DiscoveryNode;
 import org.elasticsearch.cluster.routing.OperationRouting;
-import org.elasticsearch.common.collect.ImmutableOpenMap;
 import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.test.ESIntegTestCase;
 
@@ -76,7 +75,7 @@ public class SearchReplicaSelectionIT extends ESIntegTestCase {
         }
 
         ClusterStateResponse clusterStateResponse = client.admin().cluster().prepareState().get();
-        ImmutableOpenMap<String, DiscoveryNode> coordinatingNodes = clusterStateResponse.getState().nodes().getCoordinatingOnlyNodes();
+        Map<String, DiscoveryNode> coordinatingNodes = clusterStateResponse.getState().nodes().getCoordinatingOnlyNodes();
         assertEquals(1, coordinatingNodes.size());
 
         String coordinatingNodeId = coordinatingNodes.values().iterator().next().getId();

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

@@ -35,7 +35,6 @@ import org.elasticsearch.cluster.service.ClusterService;
 import org.elasticsearch.common.Priority;
 import org.elasticsearch.common.Randomness;
 import org.elasticsearch.common.Strings;
-import org.elasticsearch.common.collect.ImmutableOpenMap;
 import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.common.util.concurrent.AbstractRunnable;
 import org.elasticsearch.common.util.concurrent.ConcurrentCollections;
@@ -86,7 +85,7 @@ public class SnapshotStressTestsIT extends AbstractSnapshotIntegTestCase {
         disableRepoConsistencyCheck("have not necessarily written to all repositories");
     }
 
-    private static Set<String> nodeNames(ImmutableOpenMap<String, DiscoveryNode> nodesMap) {
+    private static Set<String> nodeNames(Map<String, DiscoveryNode> nodesMap) {
         return nodesMap.values().stream().map(DiscoveryNode::getName).collect(Collectors.toSet());
     }
 

+ 2 - 2
server/src/main/java/org/elasticsearch/action/support/tasks/TransportTasksAction.java

@@ -19,7 +19,6 @@ import org.elasticsearch.cluster.ClusterState;
 import org.elasticsearch.cluster.node.DiscoveryNode;
 import org.elasticsearch.cluster.node.DiscoveryNodes;
 import org.elasticsearch.cluster.service.ClusterService;
-import org.elasticsearch.common.collect.ImmutableOpenMap;
 import org.elasticsearch.common.io.stream.StreamInput;
 import org.elasticsearch.common.io.stream.StreamOutput;
 import org.elasticsearch.common.io.stream.Writeable;
@@ -38,6 +37,7 @@ import org.elasticsearch.transport.TransportService;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
+import java.util.Map;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicReferenceArray;
 import java.util.function.Consumer;
@@ -226,7 +226,7 @@ public abstract class TransportTasksAction<
             ClusterState clusterState = clusterService.state();
             String[] nodesIds = resolveNodes(request, clusterState);
             this.nodesIds = filterNodeIds(clusterState.nodes(), nodesIds);
-            ImmutableOpenMap<String, DiscoveryNode> nodes = clusterState.nodes().getNodes();
+            Map<String, DiscoveryNode> nodes = clusterState.nodes().getNodes();
             this.nodes = new DiscoveryNode[nodesIds.length];
             for (int i = 0; i < this.nodesIds.length; i++) {
                 this.nodes[i] = nodes.get(this.nodesIds[i]);

+ 6 - 6
server/src/main/java/org/elasticsearch/cluster/ClusterInfo.java

@@ -36,8 +36,8 @@ public class ClusterInfo implements ToXContentFragment, Writeable {
 
     public static final Version DATA_SET_SIZE_SIZE_VERSION = Version.V_7_13_0;
 
-    private final ImmutableOpenMap<String, DiskUsage> leastAvailableSpaceUsage;
-    private final ImmutableOpenMap<String, DiskUsage> mostAvailableSpaceUsage;
+    private final Map<String, DiskUsage> leastAvailableSpaceUsage;
+    private final Map<String, DiskUsage> mostAvailableSpaceUsage;
     final ImmutableOpenMap<String, Long> shardSizes;
     final ImmutableOpenMap<ShardId, Long> shardDataSetSizes;
     public static final ClusterInfo EMPTY = new ClusterInfo();
@@ -67,8 +67,8 @@ public class ClusterInfo implements ToXContentFragment, Writeable {
      * @see #shardIdentifierFromRouting
      */
     public ClusterInfo(
-        ImmutableOpenMap<String, DiskUsage> leastAvailableSpaceUsage,
-        ImmutableOpenMap<String, DiskUsage> mostAvailableSpaceUsage,
+        Map<String, DiskUsage> leastAvailableSpaceUsage,
+        Map<String, DiskUsage> mostAvailableSpaceUsage,
         ImmutableOpenMap<String, Long> shardSizes,
         ImmutableOpenMap<ShardId, Long> shardDataSetSizes,
         ImmutableOpenMap<ShardRouting, String> routingToDataPath,
@@ -194,7 +194,7 @@ public class ClusterInfo implements ToXContentFragment, Writeable {
      * Returns a node id to disk usage mapping for the path that has the least available space on the node.
      * Note that this does not take account of reserved space: there may be another path with less available _and unreserved_ space.
      */
-    public ImmutableOpenMap<String, DiskUsage> getNodeLeastAvailableDiskUsages() {
+    public Map<String, DiskUsage> getNodeLeastAvailableDiskUsages() {
         return this.leastAvailableSpaceUsage;
     }
 
@@ -202,7 +202,7 @@ public class ClusterInfo implements ToXContentFragment, Writeable {
      * Returns a node id to disk usage mapping for the path that has the most available space on the node.
      * Note that this does not take account of reserved space: there may be another path with more available _and unreserved_ space.
      */
-    public ImmutableOpenMap<String, DiskUsage> getNodeMostAvailableDiskUsages() {
+    public Map<String, DiskUsage> getNodeMostAvailableDiskUsages() {
         return this.mostAvailableSpaceUsage;
     }
 

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

@@ -662,7 +662,7 @@ 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 Map<ShardId, ShardSnapshotStatus> shards;
         /**
          * Map of index name to {@link IndexId}.
          */
@@ -703,7 +703,7 @@ public class SnapshotsInProgress extends AbstractNamedDiffable<Custom> implement
             List<SnapshotFeatureInfo> featureStates,
             long startTime,
             long repositoryStateId,
-            ImmutableOpenMap<ShardId, ShardSnapshotStatus> shards,
+            Map<ShardId, ShardSnapshotStatus> shards,
             String failure,
             Map<String, Object> userMetadata,
             Version version
@@ -737,7 +737,7 @@ public class SnapshotsInProgress extends AbstractNamedDiffable<Custom> implement
             List<SnapshotFeatureInfo> featureStates,
             long startTime,
             long repositoryStateId,
-            ImmutableOpenMap<ShardId, ShardSnapshotStatus> shards,
+            Map<ShardId, ShardSnapshotStatus> shards,
             String failure,
             Map<String, Object> userMetadata,
             Version version,
@@ -839,7 +839,7 @@ public class SnapshotsInProgress extends AbstractNamedDiffable<Custom> implement
             SnapshotId source,
             State state,
             Map<String, IndexId> indices,
-            ImmutableOpenMap<ShardId, ShardSnapshotStatus> shards,
+            Map<ShardId, ShardSnapshotStatus> shards,
             ImmutableOpenMap<RepositoryShardId, ShardSnapshotStatus> statusByRepoShardId
         ) {
             if ((state == State.INIT || state == State.ABORTED) && shards.isEmpty()) {
@@ -1088,7 +1088,7 @@ public class SnapshotsInProgress extends AbstractNamedDiffable<Custom> implement
             return this.snapshot;
         }
 
-        public ImmutableOpenMap<RepositoryShardId, ShardSnapshotStatus> shardsByRepoShardId() {
+        public Map<RepositoryShardId, ShardSnapshotStatus> shardsByRepoShardId() {
             return shardStatusByRepoShardId;
         }
 
@@ -1097,7 +1097,7 @@ public class SnapshotsInProgress extends AbstractNamedDiffable<Custom> implement
             return snapshotIndices.get(name);
         }
 
-        public ImmutableOpenMap<ShardId, ShardSnapshotStatus> shards() {
+        public Map<ShardId, ShardSnapshotStatus> shards() {
             assert isClone() == false : "tried to get routing shards for clone entry [" + this + "]";
             return this.shards;
         }

+ 2 - 2
server/src/main/java/org/elasticsearch/cluster/block/ClusterBlocks.java

@@ -54,7 +54,7 @@ public class ClusterBlocks implements SimpleDiffable<ClusterBlocks> {
         return global;
     }
 
-    public ImmutableOpenMap<String, Set<ClusterBlock>> indices() {
+    public Map<String, Set<ClusterBlock>> indices() {
         return indicesBlocks;
     }
 
@@ -62,7 +62,7 @@ public class ClusterBlocks implements SimpleDiffable<ClusterBlocks> {
         return levelHolders.get(level).global();
     }
 
-    public ImmutableOpenMap<String, Set<ClusterBlock>> indices(ClusterBlockLevel level) {
+    public Map<String, Set<ClusterBlock>> indices(ClusterBlockLevel level) {
         return levelHolders.get(level).indices();
     }
 

+ 2 - 2
server/src/main/java/org/elasticsearch/cluster/metadata/IndexTemplateMetadata.java

@@ -139,11 +139,11 @@ public class IndexTemplateMetadata implements SimpleDiffable<IndexTemplateMetada
         return this.mappings();
     }
 
-    public ImmutableOpenMap<String, AliasMetadata> aliases() {
+    public Map<String, AliasMetadata> aliases() {
         return this.aliases;
     }
 
-    public ImmutableOpenMap<String, AliasMetadata> getAliases() {
+    public Map<String, AliasMetadata> getAliases() {
         return this.aliases;
     }
 

+ 6 - 6
server/src/main/java/org/elasticsearch/cluster/node/DiscoveryNodes.java

@@ -122,7 +122,7 @@ public class DiscoveryNodes extends AbstractCollection<DiscoveryNode> implements
      *
      * @return {@link Map} of the discovered nodes arranged by their ids
      */
-    public ImmutableOpenMap<String, DiscoveryNode> getNodes() {
+    public Map<String, DiscoveryNode> getNodes() {
         return this.nodes;
     }
 
@@ -131,7 +131,7 @@ public class DiscoveryNodes extends AbstractCollection<DiscoveryNode> implements
      *
      * @return {@link Map} of the discovered data nodes arranged by their ids
      */
-    public ImmutableOpenMap<String, DiscoveryNode> getDataNodes() {
+    public Map<String, DiscoveryNode> getDataNodes() {
         return this.dataNodes;
     }
 
@@ -140,14 +140,14 @@ public class DiscoveryNodes extends AbstractCollection<DiscoveryNode> implements
      *
      * @return {@link Map} of the discovered master nodes arranged by their ids
      */
-    public ImmutableOpenMap<String, DiscoveryNode> getMasterNodes() {
+    public Map<String, DiscoveryNode> getMasterNodes() {
         return this.masterNodes;
     }
 
     /**
      * @return All the ingest nodes arranged by their ids
      */
-    public ImmutableOpenMap<String, DiscoveryNode> getIngestNodes() {
+    public Map<String, DiscoveryNode> getIngestNodes() {
         return ingestNodes;
     }
 
@@ -156,7 +156,7 @@ public class DiscoveryNodes extends AbstractCollection<DiscoveryNode> implements
      *
      * @return {@link Map} of the discovered master and data nodes arranged by their ids
      */
-    public ImmutableOpenMap<String, DiscoveryNode> getMasterAndDataNodes() {
+    public Map<String, DiscoveryNode> getMasterAndDataNodes() {
         ImmutableOpenMap.Builder<String, DiscoveryNode> nodes = ImmutableOpenMap.builder(dataNodes);
         nodes.putAllFromMap(masterNodes);
         return nodes.build();
@@ -167,7 +167,7 @@ public class DiscoveryNodes extends AbstractCollection<DiscoveryNode> implements
      *
      * @return {@link Map} of the coordinating only nodes arranged by their ids
      */
-    public ImmutableOpenMap<String, DiscoveryNode> getCoordinatingOnlyNodes() {
+    public Map<String, DiscoveryNode> getCoordinatingOnlyNodes() {
         ImmutableOpenMap.Builder<String, DiscoveryNode> nodes = ImmutableOpenMap.builder(this.nodes);
         nodes.removeAllFromCollection(masterNodes.keySet());
         nodes.removeAllFromCollection(dataNodes.keySet());

+ 2 - 2
server/src/main/java/org/elasticsearch/cluster/routing/RoutingTable.java

@@ -107,11 +107,11 @@ public class RoutingTable implements Iterable<IndexRoutingTable>, Diffable<Routi
         return indexRouting != null && indexRouting.getIndex().equals(index) ? indexRouting : null;
     }
 
-    public ImmutableOpenMap<String, IndexRoutingTable> indicesRouting() {
+    public Map<String, IndexRoutingTable> indicesRouting() {
         return indicesRouting;
     }
 
-    public ImmutableOpenMap<String, IndexRoutingTable> getIndicesRouting() {
+    public Map<String, IndexRoutingTable> getIndicesRouting() {
         return indicesRouting();
     }
 

+ 2 - 3
server/src/main/java/org/elasticsearch/cluster/routing/allocation/DiskThresholdMonitor.java

@@ -27,7 +27,6 @@ import org.elasticsearch.cluster.routing.ShardRouting;
 import org.elasticsearch.cluster.routing.allocation.decider.DiskThresholdDecider;
 import org.elasticsearch.common.Priority;
 import org.elasticsearch.common.Strings;
-import org.elasticsearch.common.collect.ImmutableOpenMap;
 import org.elasticsearch.common.settings.ClusterSettings;
 import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.common.unit.ByteSizeValue;
@@ -127,7 +126,7 @@ public class DiskThresholdMonitor {
             return;
         }
 
-        final ImmutableOpenMap<String, DiskUsage> usages = info.getNodeLeastAvailableDiskUsages();
+        final Map<String, DiskUsage> usages = info.getNodeLeastAvailableDiskUsages();
         if (usages == null) {
             logger.trace("skipping monitor as no disk usage information is available");
             lastNodes = Collections.emptySet();
@@ -435,7 +434,7 @@ public class DiskThresholdMonitor {
 
     private static void markNodesMissingUsageIneligibleForRelease(
         RoutingNodes routingNodes,
-        ImmutableOpenMap<String, DiskUsage> usages,
+        Map<String, DiskUsage> usages,
         Set<String> indicesToMarkIneligibleForAutoRelease
     ) {
         for (RoutingNode routingNode : routingNodes) {

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

@@ -25,7 +25,6 @@ import org.elasticsearch.cluster.routing.ShardRoutingState;
 import org.elasticsearch.cluster.routing.allocation.DiskThresholdSettings;
 import org.elasticsearch.cluster.routing.allocation.RoutingAllocation;
 import org.elasticsearch.common.Strings;
-import org.elasticsearch.common.collect.ImmutableOpenMap;
 import org.elasticsearch.common.settings.ClusterSettings;
 import org.elasticsearch.common.settings.Setting;
 import org.elasticsearch.common.settings.Settings;
@@ -35,6 +34,7 @@ import org.elasticsearch.index.Index;
 import org.elasticsearch.index.shard.ShardId;
 import org.elasticsearch.snapshots.SnapshotShardSizeInfo;
 
+import java.util.Map;
 import java.util.Set;
 
 import static org.elasticsearch.cluster.routing.allocation.DiskThresholdSettings.CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK_SETTING;
@@ -174,7 +174,7 @@ public class DiskThresholdDecider extends AllocationDecider {
 
     @Override
     public Decision canAllocate(ShardRouting shardRouting, RoutingNode node, RoutingAllocation allocation) {
-        ImmutableOpenMap<String, DiskUsage> usages = allocation.clusterInfo().getNodeMostAvailableDiskUsages();
+        Map<String, DiskUsage> usages = allocation.clusterInfo().getNodeMostAvailableDiskUsages();
         final Decision decision = earlyTerminate(allocation, usages);
         if (decision != null) {
             return decision;
@@ -420,7 +420,7 @@ public class DiskThresholdDecider extends AllocationDecider {
 
     @Override
     public Decision canForceAllocateDuringReplace(ShardRouting shardRouting, RoutingNode node, RoutingAllocation allocation) {
-        ImmutableOpenMap<String, DiskUsage> usages = allocation.clusterInfo().getNodeMostAvailableDiskUsages();
+        Map<String, DiskUsage> usages = allocation.clusterInfo().getNodeMostAvailableDiskUsages();
         final Decision decision = earlyTerminate(allocation, usages);
         if (decision != null) {
             return decision;
@@ -467,7 +467,7 @@ public class DiskThresholdDecider extends AllocationDecider {
             throw new IllegalArgumentException("Shard [" + shardRouting + "] is not allocated on node: [" + node.nodeId() + "]");
         }
         final ClusterInfo clusterInfo = allocation.clusterInfo();
-        final ImmutableOpenMap<String, DiskUsage> usages = clusterInfo.getNodeLeastAvailableDiskUsages();
+        final Map<String, DiskUsage> usages = clusterInfo.getNodeLeastAvailableDiskUsages();
         final Decision decision = earlyTerminate(allocation, usages);
         if (decision != null) {
             return decision;
@@ -567,7 +567,7 @@ public class DiskThresholdDecider extends AllocationDecider {
     private static DiskUsageWithRelocations getDiskUsage(
         RoutingNode node,
         RoutingAllocation allocation,
-        ImmutableOpenMap<String, DiskUsage> usages,
+        Map<String, DiskUsage> usages,
         boolean subtractLeavingShards
     ) {
         DiskUsage usage = usages.get(node.nodeId());
@@ -606,7 +606,7 @@ public class DiskThresholdDecider extends AllocationDecider {
      * @param usages Map of nodeId to DiskUsage for all known nodes
      * @return DiskUsage representing given node using the average disk usage
      */
-    static DiskUsage averageUsage(RoutingNode node, ImmutableOpenMap<String, DiskUsage> usages) {
+    static DiskUsage averageUsage(RoutingNode node, Map<String, DiskUsage> usages) {
         if (usages.size() == 0) {
             return new DiskUsage(node.nodeId(), node.node().getName(), "_na_", 0, 0);
         }
@@ -642,7 +642,7 @@ public class DiskThresholdDecider extends AllocationDecider {
 
     private static final Decision YES_USAGES_UNAVAILABLE = Decision.single(Decision.Type.YES, NAME, "disk usages are unavailable");
 
-    private Decision earlyTerminate(RoutingAllocation allocation, ImmutableOpenMap<String, DiskUsage> usages) {
+    private Decision earlyTerminate(RoutingAllocation allocation, Map<String, DiskUsage> usages) {
         // Always allow allocation if the decider is disabled
         if (diskThresholdSettings.isEnabled() == false) {
             return YES_DISABLED;

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

@@ -23,7 +23,6 @@ import org.elasticsearch.cluster.SnapshotsInProgress.ShardState;
 import org.elasticsearch.cluster.SnapshotsInProgress.State;
 import org.elasticsearch.cluster.node.DiscoveryNode;
 import org.elasticsearch.cluster.service.ClusterService;
-import org.elasticsearch.common.collect.ImmutableOpenMap;
 import org.elasticsearch.common.component.AbstractLifecycleComponent;
 import org.elasticsearch.common.io.stream.StreamInput;
 import org.elasticsearch.common.settings.Settings;
@@ -416,7 +415,7 @@ public class SnapshotShardsService extends AbstractLifecycleComponent implements
             if (snapshot.state() == State.STARTED || snapshot.state() == State.ABORTED) {
                 Map<ShardId, IndexShardSnapshotStatus> localShards = currentSnapshotShards(snapshot.snapshot());
                 if (localShards != null) {
-                    ImmutableOpenMap<ShardId, ShardSnapshotStatus> masterShards = snapshot.shards();
+                    Map<ShardId, ShardSnapshotStatus> masterShards = snapshot.shards();
                     for (Map.Entry<ShardId, IndexShardSnapshotStatus> localShard : localShards.entrySet()) {
                         ShardId shardId = localShard.getKey();
                         ShardSnapshotStatus masterShard = masterShards.get(shardId);

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

@@ -1928,7 +1928,7 @@ public class SnapshotsService extends AbstractLifecycleComponent implements Clus
         @Nullable ImmutableOpenMap.Builder<T, ShardSnapshotStatus> updatedShardAssignments,
         ShardSnapshotStatus finishedShardState,
         T shardId,
-        ImmutableOpenMap<T, ShardSnapshotStatus> statesToUpdate
+        Map<T, ShardSnapshotStatus> statesToUpdate
     ) {
         final ShardGeneration newGeneration = finishedShardState.generation();
         final ShardSnapshotStatus stateToUpdate = statesToUpdate.get(shardId);
@@ -3193,7 +3193,7 @@ public class SnapshotsService extends AbstractLifecycleComponent implements Clus
             }
 
             private <T> void executeShardSnapshotUpdate(
-                ImmutableOpenMap<T, ShardSnapshotStatus> existingStates,
+                Map<T, ShardSnapshotStatus> existingStates,
                 Supplier<ImmutableOpenMap.Builder<T, ShardSnapshotStatus>> newStates,
                 ShardSnapshotUpdate updateSnapshotState,
                 T updatedShard

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

@@ -16,7 +16,6 @@ import org.elasticsearch.action.support.master.AcknowledgedResponse;
 import org.elasticsearch.cluster.ClusterState;
 import org.elasticsearch.cluster.metadata.MetadataIndexTemplateService.PutRequest;
 import org.elasticsearch.cluster.service.ClusterService;
-import org.elasticsearch.common.collect.ImmutableOpenMap;
 import org.elasticsearch.common.compress.CompressedXContent;
 import org.elasticsearch.common.settings.IndexScopedSettings;
 import org.elasticsearch.common.settings.Settings;
@@ -241,7 +240,7 @@ public class MetadataIndexTemplateServiceTests extends ESSingleNodeTestCase {
 
         final Metadata metadata = client().admin().cluster().prepareState().get().getState().metadata();
         IndexTemplateMetadata template = metadata.templates().get(templateName);
-        ImmutableOpenMap<String, AliasMetadata> aliasMap = template.getAliases();
+        Map<String, AliasMetadata> aliasMap = template.getAliases();
         assertThat(aliasMap.size(), equalTo(1));
         AliasMetadata metaAlias = aliasMap.get(aliasName);
         String filterString = metaAlias.filter() == null ? null : metaAlias.filter().string();

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

@@ -300,7 +300,7 @@ public class SnapshotsInProgressSerializationTests extends SimpleDiffableWireSer
             }
             case 6 -> {
                 Map<String, IndexId> indices = new HashMap<>(entry.indices());
-                ImmutableOpenMap<ShardId, SnapshotsInProgress.ShardSnapshotStatus> shards = entry.shards();
+                Map<ShardId, SnapshotsInProgress.ShardSnapshotStatus> shards = entry.shards();
                 IndexId indexId = new IndexId(randomAlphaOfLength(10), randomAlphaOfLength(10));
                 indices.put(indexId.getName(), indexId);
                 ImmutableOpenMap.Builder<ShardId, SnapshotsInProgress.ShardSnapshotStatus> builder = ImmutableOpenMap.builder(shards);
@@ -524,7 +524,7 @@ public class SnapshotsInProgressSerializationTests extends SimpleDiffableWireSer
         }
     }
 
-    public static State randomState(ImmutableOpenMap<ShardId, SnapshotsInProgress.ShardSnapshotStatus> shards) {
+    public static State randomState(Map<ShardId, SnapshotsInProgress.ShardSnapshotStatus> shards) {
         return SnapshotsInProgress.completed(shards.values())
             ? randomFrom(State.SUCCESS, State.FAILED)
             : randomFrom(State.STARTED, State.INIT, State.ABORTED);

+ 1 - 2
x-pack/plugin/autoscaling/src/main/java/org/elasticsearch/xpack/autoscaling/capacity/AutoscalingCalculateCapacityService.java

@@ -14,7 +14,6 @@ import org.elasticsearch.cluster.DiskUsage;
 import org.elasticsearch.cluster.node.DiscoveryNode;
 import org.elasticsearch.cluster.node.DiscoveryNodeRole;
 import org.elasticsearch.cluster.routing.allocation.decider.AllocationDeciders;
-import org.elasticsearch.common.collect.ImmutableOpenMap;
 import org.elasticsearch.common.settings.Setting;
 import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.common.unit.ByteSizeValue;
@@ -327,7 +326,7 @@ public class AutoscalingCalculateCapacityService implements PolicyValidator {
             );
         }
 
-        private long totalStorage(ImmutableOpenMap<String, DiskUsage> diskUsages, DiscoveryNode node) {
+        private long totalStorage(Map<String, DiskUsage> diskUsages, DiscoveryNode node) {
             DiskUsage diskUsage = diskUsages.get(node.getId());
             return diskUsage != null ? diskUsage.getTotalBytes() : -1;
         }

+ 2 - 2
x-pack/plugin/autoscaling/src/test/java/org/elasticsearch/xpack/autoscaling/storage/ReactiveStorageDeciderDecisionTests.java

@@ -571,12 +571,12 @@ public class ReactiveStorageDeciderDecisionTests extends AutoscalingTestCase {
 
         return new ClusterInfo() {
             @Override
-            public ImmutableOpenMap<String, DiskUsage> getNodeLeastAvailableDiskUsages() {
+            public Map<String, DiskUsage> getNodeLeastAvailableDiskUsages() {
                 return immutableDiskUsages;
             }
 
             @Override
-            public ImmutableOpenMap<String, DiskUsage> getNodeMostAvailableDiskUsages() {
+            public Map<String, DiskUsage> getNodeMostAvailableDiskUsages() {
                 return immutableDiskUsages;
             }
 

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

@@ -17,7 +17,6 @@ import org.elasticsearch.cluster.ClusterName;
 import org.elasticsearch.cluster.ClusterState;
 import org.elasticsearch.cluster.node.DiscoveryNode;
 import org.elasticsearch.cluster.service.ClusterService;
-import org.elasticsearch.common.collect.ImmutableOpenMap;
 import org.elasticsearch.common.inject.Inject;
 import org.elasticsearch.common.io.stream.StreamInput;
 import org.elasticsearch.common.io.stream.StreamOutput;
@@ -36,6 +35,7 @@ import org.elasticsearch.xpack.searchablesnapshots.cache.shared.FrozenCacheServi
 import java.io.IOException;
 import java.util.Arrays;
 import java.util.List;
+import java.util.Map;
 import java.util.function.Supplier;
 import java.util.stream.Collectors;
 
@@ -101,7 +101,7 @@ public class TransportSearchableSnapshotsNodeCachesStatsAction extends Transport
 
     @Override
     protected void resolveRequest(NodesRequest request, ClusterState clusterState) {
-        final ImmutableOpenMap<String, DiscoveryNode> dataNodes = clusterState.getNodes().getDataNodes();
+        final Map<String, DiscoveryNode> dataNodes = clusterState.getNodes().getDataNodes();
 
         final DiscoveryNode[] resolvedNodes;
         if (request.nodesIds() == null || request.nodesIds().length == 0) {

+ 3 - 8
x-pack/plugin/security/src/internalClusterTest/java/org/elasticsearch/integration/ShrinkIndexWithSecurityTests.java

@@ -7,12 +7,13 @@
 package org.elasticsearch.integration;
 
 import org.elasticsearch.cluster.node.DiscoveryNode;
-import org.elasticsearch.common.collect.ImmutableOpenMap;
 import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.index.query.TermsQueryBuilder;
 import org.elasticsearch.test.ESIntegTestCase.ClusterScope;
 import org.elasticsearch.test.SecurityIntegTestCase;
 
+import java.util.Map;
+
 import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
 import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount;
 
@@ -38,13 +39,7 @@ public class ShrinkIndexWithSecurityTests extends SecurityIntegTestCase {
             client().prepareIndex("bigindex").setSource("foo", "bar").get();
         }
 
-        ImmutableOpenMap<String, DiscoveryNode> dataNodes = client().admin()
-            .cluster()
-            .prepareState()
-            .get()
-            .getState()
-            .nodes()
-            .getDataNodes();
+        Map<String, DiscoveryNode> dataNodes = client().admin().cluster().prepareState().get().getState().nodes().getDataNodes();
         DiscoveryNode[] discoveryNodes = dataNodes.values().toArray(DiscoveryNode[]::new);
         final String mergeNode = discoveryNodes[0].getName();
         ensureGreen();