From dd05eddfaac4b39d378fcd94be276bb99287b229 Mon Sep 17 00:00:00 2001 From: Nick Knize Date: Tue, 25 Apr 2023 15:22:57 +0000 Subject: [PATCH] [Refactor] more ImmutableOpenMap to jdk Map in cluster package (#7301) This continues the refactor of ImmutableOpenMap to j.u.Map in the o.o.cluster package in preparation for removal of HPPC dependencies. Signed-off-by: Nicholas Walter Knize --- .../client/indices/GetIndexResponseTests.java | 6 +- .../ValidateIndicesAliasesRequestIT.java | 4 +- .../action/admin/indices/get/GetIndexIT.java | 7 +- .../opensearch/aliases/IndexAliasesIT.java | 7 +- .../cluster/ClusterStateDiffIT.java | 17 ++- .../indices/IndicesLifecycleListenerIT.java | 4 +- .../SegmentReplicationAllocationIT.java | 14 +- .../snapshots/ConcurrentSnapshotsIT.java | 6 +- .../restore/RestoreClusterStateListener.java | 4 +- .../TransportSnapshotsStatusAction.java | 26 ++-- .../state/TransportClusterStateAction.java | 7 +- .../alias/TransportIndicesAliasesAction.java | 9 +- .../indices/alias/get/GetAliasesResponse.java | 13 +- .../alias/get/TransportGetAliasesAction.java | 17 +-- .../admin/indices/get/GetIndexResponse.java | 22 +-- .../indices/get/TransportGetIndexAction.java | 4 +- .../org/opensearch/cluster/ClusterModule.java | 2 +- .../org/opensearch/cluster/ClusterState.java | 64 ++++----- .../opensearch/cluster/RestoreInProgress.java | 58 ++++---- .../cluster/SnapshotsInProgress.java | 99 +++++++------- .../cluster/block/ClusterBlocks.java | 55 ++++---- .../opensearch/cluster/metadata/Metadata.java | 16 +-- .../metadata/MetadataDeleteIndexService.java | 8 +- .../cluster/routing/RoutingNodes.java | 4 +- .../cluster/routing/RoutingTable.java | 33 +++-- .../routing/allocation/AllocationService.java | 6 +- .../allocation/DiskThresholdMonitor.java | 8 +- .../routing/allocation/RoutingAllocation.java | 5 +- .../allocator/LocalShardsBalancer.java | 2 +- .../admin/indices/RestGetAliasesAction.java | 19 ++- .../rest/action/cat/RestAliasAction.java | 8 +- .../InFlightShardSnapshotStates.java | 14 +- .../opensearch/snapshots/RestoreService.java | 57 ++++---- .../snapshots/SnapshotShardsService.java | 18 ++- .../snapshots/SnapshotsService.java | 127 +++++++++--------- .../alias/get/GetAliasesResponseTests.java | 27 ++-- .../get/TransportGetAliasesActionTests.java | 56 +++----- .../DeleteDataStreamRequestTests.java | 4 +- .../indices/get/GetIndexResponseTests.java | 4 +- .../opensearch/cluster/ClusterStateTests.java | 6 +- .../cluster/block/ClusterBlockTests.java | 4 +- .../MetadataDeleteIndexServiceTests.java | 4 +- .../MetadataIndexStateServiceTests.java | 9 +- .../cluster/metadata/MetadataTests.java | 12 +- .../allocation/AddIncrementallyTests.java | 10 +- .../allocation/BalanceConfigurationTests.java | 19 ++- .../allocation/ThrottlingAllocationTests.java | 10 +- .../decider/DiskThresholdDeciderTests.java | 4 +- ...storeInProgressAllocationDeciderTests.java | 7 +- .../ClusterSerializationTests.java | 4 +- .../indices/RestGetAliasesActionTests.java | 41 +++--- .../InternalSnapshotsInfoServiceTests.java | 6 +- ...SnapshotsInProgressSerializationTests.java | 10 +- .../snapshots/SnapshotsServiceTests.java | 24 ++-- .../opensearch/test/InternalTestCluster.java | 9 +- 55 files changed, 490 insertions(+), 550 deletions(-) diff --git a/client/rest-high-level/src/test/java/org/opensearch/client/indices/GetIndexResponseTests.java b/client/rest-high-level/src/test/java/org/opensearch/client/indices/GetIndexResponseTests.java index db2d327e50a65..39bfd52b4dcfc 100644 --- a/client/rest-high-level/src/test/java/org/opensearch/client/indices/GetIndexResponseTests.java +++ b/client/rest-high-level/src/test/java/org/opensearch/client/indices/GetIndexResponseTests.java @@ -63,7 +63,7 @@ public class GetIndexResponseTests extends AbstractResponseTestCase< protected org.opensearch.action.admin.indices.get.GetIndexResponse createServerTestInstance(XContentType xContentType) { String[] indices = generateRandomStringArray(5, 5, false, false); final Map mappings = new HashMap<>(); - ImmutableOpenMap.Builder> aliases = ImmutableOpenMap.builder(); + final Map> aliases = new HashMap<>(); ImmutableOpenMap.Builder settings = ImmutableOpenMap.builder(); ImmutableOpenMap.Builder defaultSettings = ImmutableOpenMap.builder(); ImmutableOpenMap.Builder dataStreams = ImmutableOpenMap.builder(); @@ -95,7 +95,7 @@ protected org.opensearch.action.admin.indices.get.GetIndexResponse createServerT return new org.opensearch.action.admin.indices.get.GetIndexResponse( indices, mappings, - aliases.build(), + aliases, settings.build(), defaultSettings.build(), dataStreams.build() @@ -116,7 +116,7 @@ protected void assertInstances( assertEquals(serverTestInstance.getMappings(), clientInstance.getMappings()); assertMapEquals(serverTestInstance.getSettings(), clientInstance.getSettings()); assertMapEquals(serverTestInstance.defaultSettings(), clientInstance.getDefaultSettings()); - assertMapEquals(serverTestInstance.getAliases(), clientInstance.getAliases()); + assertEquals(serverTestInstance.getAliases(), clientInstance.getAliases()); } private static MappingMetadata createMappingsForIndex() { diff --git a/server/src/internalClusterTest/java/org/opensearch/action/admin/indices/alias/ValidateIndicesAliasesRequestIT.java b/server/src/internalClusterTest/java/org/opensearch/action/admin/indices/alias/ValidateIndicesAliasesRequestIT.java index bcbc93b5500bc..63141b590a67f 100644 --- a/server/src/internalClusterTest/java/org/opensearch/action/admin/indices/alias/ValidateIndicesAliasesRequestIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/action/admin/indices/alias/ValidateIndicesAliasesRequestIT.java @@ -107,8 +107,8 @@ public void testAllowed() { request.addAliasAction(IndicesAliasesRequest.AliasActions.add().index("index").alias("alias")); assertAcked(client().admin().indices().aliases(request).actionGet()); final GetAliasesResponse response = client().admin().indices().getAliases(new GetAliasesRequest("alias")).actionGet(); - assertThat(response.getAliases().keys().size(), equalTo(1)); - assertThat(response.getAliases().keys().iterator().next().value, equalTo("index")); + assertThat(response.getAliases().keySet().size(), equalTo(1)); + assertThat(response.getAliases().keySet().iterator().next(), equalTo("index")); final List aliasMetadata = response.getAliases().get("index"); assertThat(aliasMetadata, hasSize(1)); assertThat(aliasMetadata.get(0).alias(), equalTo("alias")); diff --git a/server/src/internalClusterTest/java/org/opensearch/action/admin/indices/get/GetIndexIT.java b/server/src/internalClusterTest/java/org/opensearch/action/admin/indices/get/GetIndexIT.java index 0d4cf6bd19188..8237f7e785395 100644 --- a/server/src/internalClusterTest/java/org/opensearch/action/admin/indices/get/GetIndexIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/action/admin/indices/get/GetIndexIT.java @@ -32,7 +32,6 @@ package org.opensearch.action.admin.indices.get; -import com.carrotsearch.hppc.cursors.ObjectObjectCursor; import org.opensearch.action.admin.indices.alias.Alias; import org.opensearch.action.admin.indices.get.GetIndexRequest.Feature; import org.opensearch.action.support.IndicesOptions; @@ -288,7 +287,7 @@ private void assertEmptyOrOnlyDefaultMappings(GetIndexResponse response, String } private void assertAliases(GetIndexResponse response, String indexName) { - ImmutableOpenMap> aliases = response.aliases(); + final Map> aliases = response.aliases(); assertThat(aliases, notNullValue()); assertThat(aliases.size(), equalTo(1)); List indexAliases = aliases.get(indexName); @@ -311,8 +310,8 @@ private void assertEmptyMappings(GetIndexResponse response) { private void assertEmptyAliases(GetIndexResponse response) { assertThat(response.aliases(), notNullValue()); - for (final ObjectObjectCursor> entry : response.getAliases()) { - assertTrue(entry.value.isEmpty()); + for (final List entry : response.getAliases().values()) { + assertTrue(entry.isEmpty()); } } } diff --git a/server/src/internalClusterTest/java/org/opensearch/aliases/IndexAliasesIT.java b/server/src/internalClusterTest/java/org/opensearch/aliases/IndexAliasesIT.java index 574046509de75..0b050fd60f920 100644 --- a/server/src/internalClusterTest/java/org/opensearch/aliases/IndexAliasesIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/aliases/IndexAliasesIT.java @@ -32,7 +32,6 @@ package org.opensearch.aliases; -import com.carrotsearch.hppc.cursors.ObjectObjectCursor; import org.opensearch.action.admin.indices.alias.Alias; import org.opensearch.action.admin.indices.alias.IndicesAliasesRequest.AliasActions; import org.opensearch.action.admin.indices.alias.get.GetAliasesResponse; @@ -87,7 +86,6 @@ import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_READ_ONLY; import static org.opensearch.index.query.QueryBuilders.rangeQuery; import static org.opensearch.index.query.QueryBuilders.termQuery; -import static org.opensearch.test.hamcrest.CollectionAssertions.hasKey; import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertAcked; import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertBlocked; import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertHitCount; @@ -96,6 +94,7 @@ import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.emptyArray; import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.hasKey; import static org.hamcrest.Matchers.lessThan; import static org.hamcrest.Matchers.notNullValue; import static org.hamcrest.Matchers.nullValue; @@ -1044,8 +1043,8 @@ public void testIndicesGetAliases() throws Exception { assertAcked(admin().indices().prepareAliases().removeAlias("foobar", "foo")); getResponse = admin().indices().prepareGetAliases("foo").addIndices("foobar").get(); - for (final ObjectObjectCursor> entry : getResponse.getAliases()) { - assertTrue(entry.value.isEmpty()); + for (final List entry : getResponse.getAliases().values()) { + assertTrue(entry.isEmpty()); } assertTrue(admin().indices().prepareGetAliases("foo").addIndices("foobar").get().getAliases().isEmpty()); } diff --git a/server/src/internalClusterTest/java/org/opensearch/cluster/ClusterStateDiffIT.java b/server/src/internalClusterTest/java/org/opensearch/cluster/ClusterStateDiffIT.java index f0c83680c131b..029686f228d7d 100644 --- a/server/src/internalClusterTest/java/org/opensearch/cluster/ClusterStateDiffIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/cluster/ClusterStateDiffIT.java @@ -56,7 +56,6 @@ import org.opensearch.cluster.routing.UnassignedInfo; import org.opensearch.common.UUIDs; import org.opensearch.common.bytes.BytesReference; -import org.opensearch.common.collect.ImmutableOpenMap; import org.opensearch.common.io.stream.BytesStreamOutput; import org.opensearch.common.io.stream.NamedWriteableAwareStreamInput; import org.opensearch.common.io.stream.NamedWriteableRegistry; @@ -283,7 +282,7 @@ private ClusterState.Builder randomRoutingTable(ClusterState clusterState) { if (numberOfIndices > 0) { List randomIndices = randomSubsetOf( randomInt(numberOfIndices - 1), - clusterState.routingTable().indicesRouting().keys().toArray(String.class) + clusterState.routingTable().indicesRouting().keySet().toArray(new String[0]) ); for (String index : randomIndices) { if (randomBoolean()) { @@ -413,7 +412,7 @@ private interface RandomClusterPart { /** * Returns list of parts from metadata */ - ImmutableOpenMap parts(ClusterState clusterState); + Map parts(ClusterState clusterState); /** * Puts the part back into metadata @@ -443,12 +442,12 @@ private interface RandomClusterPart { */ private ClusterState randomClusterStateParts(ClusterState clusterState, String prefix, RandomClusterPart randomPart) { ClusterState.Builder builder = ClusterState.builder(clusterState); - ImmutableOpenMap parts = randomPart.parts(clusterState); + final Map parts = randomPart.parts(clusterState); int partCount = parts.size(); if (partCount > 0) { List randomParts = randomSubsetOf( randomInt(partCount - 1), - randomPart.parts(clusterState).keys().toArray(String.class) + randomPart.parts(clusterState).keySet().toArray(new String[0]) ); for (String part : randomParts) { if (randomBoolean()) { @@ -745,7 +744,7 @@ private ClusterState.Builder randomClusterStateCustoms(final ClusterState cluste return ClusterState.builder(randomClusterStateParts(clusterState, "custom", new RandomClusterPart() { @Override - public ImmutableOpenMap parts(ClusterState clusterState) { + public Map parts(ClusterState clusterState) { return clusterState.customs(); } @@ -769,12 +768,12 @@ public ClusterState.Custom randomCreate(String name) { new Snapshot(randomName("repo"), new SnapshotId(randomName("snap"), UUIDs.randomBase64UUID())), randomBoolean(), randomBoolean(), - SnapshotsInProgressSerializationTests.randomState(ImmutableOpenMap.of()), + SnapshotsInProgressSerializationTests.randomState(Map.of()), Collections.emptyList(), Collections.emptyList(), Math.abs(randomLong()), randomIntBetween(0, 1000), - ImmutableOpenMap.of(), + Map.of(), null, SnapshotInfoTests.randomUserMetadata(), randomVersion(random()) @@ -788,7 +787,7 @@ public ClusterState.Custom randomCreate(String name) { new Snapshot(randomName("repo"), new SnapshotId(randomName("snap"), UUIDs.randomBase64UUID())), RestoreInProgress.State.fromValue((byte) randomIntBetween(0, 3)), emptyList(), - ImmutableOpenMap.of() + Map.of() ) ).build(); default: diff --git a/server/src/internalClusterTest/java/org/opensearch/indices/IndicesLifecycleListenerIT.java b/server/src/internalClusterTest/java/org/opensearch/indices/IndicesLifecycleListenerIT.java index 17a7d4c84b6fe..d0b9f7c4134fb 100644 --- a/server/src/internalClusterTest/java/org/opensearch/indices/IndicesLifecycleListenerIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/indices/IndicesLifecycleListenerIT.java @@ -129,7 +129,7 @@ public void beforeIndexCreated(Index index, Settings indexSettings) { } catch (Exception e) { assertTrue(e.getMessage().contains("failing on purpose")); ClusterStateResponse resp = client().admin().cluster().prepareState().get(); - assertFalse(resp.getState().routingTable().indicesRouting().keys().contains("failed")); + assertFalse(resp.getState().routingTable().indicesRouting().keySet().contains("failed")); } } @@ -179,7 +179,7 @@ public void testIndexStateShardChanged() throws Throwable { } catch (OpenSearchException e) { assertTrue(e.getMessage().contains("failing on purpose")); ClusterStateResponse resp = client().admin().cluster().prepareState().get(); - assertFalse(resp.getState().routingTable().indicesRouting().keys().contains("failed")); + assertFalse(resp.getState().routingTable().indicesRouting().keySet().contains("failed")); } // create an index diff --git a/server/src/internalClusterTest/java/org/opensearch/indices/replication/SegmentReplicationAllocationIT.java b/server/src/internalClusterTest/java/org/opensearch/indices/replication/SegmentReplicationAllocationIT.java index f2c760638b54b..5056adea32f83 100644 --- a/server/src/internalClusterTest/java/org/opensearch/indices/replication/SegmentReplicationAllocationIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/indices/replication/SegmentReplicationAllocationIT.java @@ -8,7 +8,6 @@ package org.opensearch.indices.replication; -import com.carrotsearch.hppc.cursors.ObjectObjectCursor; import org.opensearch.cluster.ClusterState; import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.cluster.routing.IndexRoutingTable; @@ -25,6 +24,7 @@ import java.util.ArrayList; import java.util.List; +import java.util.Map; import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; @@ -231,11 +231,11 @@ private void verifyPerIndexPrimaryBalance() throws Exception { assertBusy(() -> { final ClusterState currentState = client().admin().cluster().prepareState().execute().actionGet().getState(); RoutingNodes nodes = currentState.getRoutingNodes(); - for (ObjectObjectCursor index : currentState.getRoutingTable().indicesRouting()) { - final int totalPrimaryShards = index.value.primaryShardsActive(); + for (final Map.Entry index : currentState.getRoutingTable().indicesRouting().entrySet()) { + final int totalPrimaryShards = index.getValue().primaryShardsActive(); final int avgPrimaryShardsPerNode = (int) Math.ceil(totalPrimaryShards * 1f / currentState.getRoutingNodes().size()); for (RoutingNode node : nodes) { - final int primaryCount = node.shardsWithState(index.key, STARTED) + final int primaryCount = node.shardsWithState(index.getKey(), STARTED) .stream() .filter(ShardRouting::primary) .collect(Collectors.toList()) @@ -243,7 +243,7 @@ private void verifyPerIndexPrimaryBalance() throws Exception { if (primaryCount > avgPrimaryShardsPerNode) { logger.info( "--> Primary shard balance assertion failure for index {} on node {} {} <= {}", - index.key, + index.getKey(), node.node().getName(), primaryCount, avgPrimaryShardsPerNode @@ -260,8 +260,8 @@ private void verifyPrimaryBalance() throws Exception { final ClusterState currentState = client().admin().cluster().prepareState().execute().actionGet().getState(); RoutingNodes nodes = currentState.getRoutingNodes(); int totalPrimaryShards = 0; - for (ObjectObjectCursor index : currentState.getRoutingTable().indicesRouting()) { - totalPrimaryShards += index.value.primaryShardsActive(); + for (final IndexRoutingTable index : currentState.getRoutingTable().indicesRouting().values()) { + totalPrimaryShards += index.primaryShardsActive(); } final int avgPrimaryShardsPerNode = (int) Math.ceil(totalPrimaryShards * 1f / currentState.getRoutingNodes().size()); for (RoutingNode node : nodes) { diff --git a/server/src/internalClusterTest/java/org/opensearch/snapshots/ConcurrentSnapshotsIT.java b/server/src/internalClusterTest/java/org/opensearch/snapshots/ConcurrentSnapshotsIT.java index f483ed7fe6c5d..0a1de615d4f3c 100644 --- a/server/src/internalClusterTest/java/org/opensearch/snapshots/ConcurrentSnapshotsIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/snapshots/ConcurrentSnapshotsIT.java @@ -31,8 +31,6 @@ package org.opensearch.snapshots; -import com.carrotsearch.hppc.cursors.ObjectCursor; - import org.opensearch.OpenSearchException; import org.opensearch.action.ActionFuture; import org.opensearch.action.ActionListener; @@ -1428,8 +1426,8 @@ private void createIndexWithContent(String indexName, String nodeInclude, String private static boolean snapshotHasCompletedShard(String snapshot, SnapshotsInProgress snapshotsInProgress) { for (SnapshotsInProgress.Entry entry : snapshotsInProgress.entries()) { if (entry.snapshot().getSnapshotId().getName().equals(snapshot)) { - for (ObjectCursor shard : entry.shards().values()) { - if (shard.value.state().completed()) { + for (final SnapshotsInProgress.ShardSnapshotStatus shard : entry.shards().values()) { + if (shard.state().completed()) { return true; } } diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/snapshots/restore/RestoreClusterStateListener.java b/server/src/main/java/org/opensearch/action/admin/cluster/snapshots/restore/RestoreClusterStateListener.java index d0f78e85e26a5..64538226ccf9e 100644 --- a/server/src/main/java/org/opensearch/action/admin/cluster/snapshots/restore/RestoreClusterStateListener.java +++ b/server/src/main/java/org/opensearch/action/admin/cluster/snapshots/restore/RestoreClusterStateListener.java @@ -40,11 +40,11 @@ import org.opensearch.cluster.ClusterStateListener; import org.opensearch.cluster.RestoreInProgress; import org.opensearch.cluster.service.ClusterService; -import org.opensearch.common.collect.ImmutableOpenMap; import org.opensearch.index.shard.ShardId; import org.opensearch.snapshots.RestoreInfo; import org.opensearch.snapshots.RestoreService; +import java.util.Map; import java.util.function.Function; import static org.opensearch.snapshots.RestoreService.restoreInProgress; @@ -89,7 +89,7 @@ public void clusterChanged(ClusterChangedEvent changedEvent) { listener.onResponse(actionResponseFactory.apply(null)); } else if (newEntry == null) { clusterService.removeListener(this); - ImmutableOpenMap shards = prevEntry.shards(); + final Map shards = prevEntry.shards(); assert prevEntry.state().completed() : "expected completed snapshot/remote store restore state but was " + prevEntry.state(); assert RestoreService.completed(shards) : "expected all restore entries to be completed"; RestoreInfo ri = new RestoreInfo( diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/snapshots/status/TransportSnapshotsStatusAction.java b/server/src/main/java/org/opensearch/action/admin/cluster/snapshots/status/TransportSnapshotsStatusAction.java index bd7391a7939a1..0400c3afbaf99 100644 --- a/server/src/main/java/org/opensearch/action/admin/cluster/snapshots/status/TransportSnapshotsStatusAction.java +++ b/server/src/main/java/org/opensearch/action/admin/cluster/snapshots/status/TransportSnapshotsStatusAction.java @@ -32,8 +32,6 @@ package org.opensearch.action.admin.cluster.snapshots.status; -import com.carrotsearch.hppc.cursors.ObjectCursor; -import com.carrotsearch.hppc.cursors.ObjectObjectCursor; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.opensearch.action.ActionListener; @@ -154,9 +152,9 @@ protected void clusterManagerOperation( Set nodesIds = new HashSet<>(); for (SnapshotsInProgress.Entry entry : currentSnapshots) { - for (ObjectCursor status : entry.shards().values()) { - if (status.value.nodeId() != null) { - nodesIds.add(status.value.nodeId()); + for (final SnapshotsInProgress.ShardSnapshotStatus status : entry.shards().values()) { + if (status.nodeId() != null) { + nodesIds.add(status.nodeId()); } } } @@ -210,26 +208,26 @@ private void buildResponse( currentSnapshotNames.add(entry.snapshot().getSnapshotId().getName()); List shardStatusBuilder = new ArrayList<>(); Map indexIdLookup = null; - for (ObjectObjectCursor shardEntry : entry.shards()) { - SnapshotsInProgress.ShardSnapshotStatus status = shardEntry.value; + for (final Map.Entry shardEntry : entry.shards().entrySet()) { + SnapshotsInProgress.ShardSnapshotStatus status = shardEntry.getValue(); if (status.nodeId() != null) { // We should have information about this shard from the shard: TransportNodesSnapshotsStatus.NodeSnapshotStatus nodeStatus = nodeSnapshotStatusMap.get(status.nodeId()); if (nodeStatus != null) { Map shardStatues = nodeStatus.status().get(entry.snapshot()); if (shardStatues != null) { - SnapshotIndexShardStatus shardStatus = shardStatues.get(shardEntry.key); + SnapshotIndexShardStatus shardStatus = shardStatues.get(shardEntry.getKey()); if (shardStatus != null) { // We have full information about this shard if (shardStatus.getStage() == SnapshotIndexShardStage.DONE - && shardEntry.value.state() != SnapshotsInProgress.ShardState.SUCCESS) { + && shardEntry.getValue().state() != SnapshotsInProgress.ShardState.SUCCESS) { // Unlikely edge case: // Data node has finished snapshotting the shard but the cluster state has not yet been updated // to reflect this. We adjust the status to show up as snapshot metadata being written because // technically if the data node failed before successfully reporting DONE state to cluster-manager, // then this shards state would jump to a failed state. shardStatus = new SnapshotIndexShardStatus( - shardEntry.key, + shardEntry.getKey(), SnapshotIndexShardStage.FINALIZE, shardStatus.getStats(), shardStatus.getNodeId(), @@ -247,7 +245,7 @@ private void buildResponse( // We rebuild the information they would have provided from their in memory state from the cluster // state and the repository contents in the below logic final SnapshotIndexShardStage stage; - switch (shardEntry.value.state()) { + switch (shardEntry.getValue().state()) { case FAILED: case ABORTED: case MISSING: @@ -262,7 +260,7 @@ private void buildResponse( stage = SnapshotIndexShardStage.DONE; break; default: - throw new IllegalArgumentException("Unknown snapshot state " + shardEntry.value.state()); + throw new IllegalArgumentException("Unknown snapshot state " + shardEntry.getValue().state()); } final SnapshotIndexShardStatus shardStatus; if (stage == SnapshotIndexShardStage.DONE) { @@ -271,7 +269,7 @@ private void buildResponse( if (indexIdLookup == null) { indexIdLookup = entry.indices().stream().collect(Collectors.toMap(IndexId::getName, Function.identity())); } - final ShardId shardId = shardEntry.key; + final ShardId shardId = shardEntry.getKey(); shardStatus = new SnapshotIndexShardStatus( shardId, repositoriesService.repository(entry.repository()) @@ -283,7 +281,7 @@ private void buildResponse( .asCopy() ); } else { - shardStatus = new SnapshotIndexShardStatus(shardEntry.key, stage); + shardStatus = new SnapshotIndexShardStatus(shardEntry.getKey(), stage); } shardStatusBuilder.add(shardStatus); } diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/state/TransportClusterStateAction.java b/server/src/main/java/org/opensearch/action/admin/cluster/state/TransportClusterStateAction.java index 88f94cacf3a81..15c2cea31f530 100644 --- a/server/src/main/java/org/opensearch/action/admin/cluster/state/TransportClusterStateAction.java +++ b/server/src/main/java/org/opensearch/action/admin/cluster/state/TransportClusterStateAction.java @@ -32,7 +32,6 @@ package org.opensearch.action.admin.cluster.state; -import com.carrotsearch.hppc.cursors.ObjectObjectCursor; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.opensearch.action.ActionListener; @@ -222,9 +221,9 @@ private ClusterStateResponse buildResponse(final ClusterStateRequest request, fi builder.metadata(mdBuilder); if (request.customs()) { - for (ObjectObjectCursor custom : currentState.customs()) { - if (custom.value.isPrivate() == false) { - builder.putCustom(custom.key, custom.value); + for (final Map.Entry custom : currentState.customs().entrySet()) { + if (custom.getValue().isPrivate() == false) { + builder.putCustom(custom.getKey(), custom.getValue()); } } } diff --git a/server/src/main/java/org/opensearch/action/admin/indices/alias/TransportIndicesAliasesAction.java b/server/src/main/java/org/opensearch/action/admin/indices/alias/TransportIndicesAliasesAction.java index 0ab9faa94090a..e8f4f9764dd47 100644 --- a/server/src/main/java/org/opensearch/action/admin/indices/alias/TransportIndicesAliasesAction.java +++ b/server/src/main/java/org/opensearch/action/admin/indices/alias/TransportIndicesAliasesAction.java @@ -32,7 +32,6 @@ package org.opensearch.action.admin.indices.alias; -import com.carrotsearch.hppc.cursors.ObjectCursor; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.opensearch.action.ActionListener; @@ -51,7 +50,6 @@ import org.opensearch.cluster.metadata.Metadata; import org.opensearch.cluster.metadata.MetadataIndexAliasesService; import org.opensearch.cluster.service.ClusterService; -import org.opensearch.common.collect.ImmutableOpenMap; import org.opensearch.common.inject.Inject; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.index.Index; @@ -64,6 +62,7 @@ import java.util.Collections; import java.util.HashSet; import java.util.List; +import java.util.Map; import java.util.Objects; import java.util.Optional; import java.util.Set; @@ -220,10 +219,10 @@ private static String[] concreteAliases(IndicesAliasesRequest.AliasActions actio if (action.expandAliasesWildcards()) { // for DELETE we expand the aliases String[] indexAsArray = { concreteIndex }; - ImmutableOpenMap> aliasMetadata = metadata.findAliases(action, indexAsArray); + final Map> aliasMetadata = metadata.findAliases(action, indexAsArray); List finalAliases = new ArrayList<>(); - for (ObjectCursor> curAliases : aliasMetadata.values()) { - for (AliasMetadata aliasMeta : curAliases.value) { + for (final List curAliases : aliasMetadata.values()) { + for (AliasMetadata aliasMeta : curAliases) { finalAliases.add(aliasMeta.alias()); } } diff --git a/server/src/main/java/org/opensearch/action/admin/indices/alias/get/GetAliasesResponse.java b/server/src/main/java/org/opensearch/action/admin/indices/alias/get/GetAliasesResponse.java index 60c0a403566d5..02e999c797e3d 100644 --- a/server/src/main/java/org/opensearch/action/admin/indices/alias/get/GetAliasesResponse.java +++ b/server/src/main/java/org/opensearch/action/admin/indices/alias/get/GetAliasesResponse.java @@ -34,12 +34,13 @@ import org.opensearch.action.ActionResponse; import org.opensearch.cluster.metadata.AliasMetadata; -import org.opensearch.common.collect.ImmutableOpenMap; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.io.stream.StreamOutput; import java.io.IOException; +import java.util.Collections; import java.util.List; +import java.util.Map; import java.util.Objects; /** @@ -49,18 +50,18 @@ */ public class GetAliasesResponse extends ActionResponse { - private final ImmutableOpenMap> aliases; + private final Map> aliases; - public GetAliasesResponse(ImmutableOpenMap> aliases) { - this.aliases = aliases; + public GetAliasesResponse(final Map> aliases) { + this.aliases = Collections.unmodifiableMap(aliases); } public GetAliasesResponse(StreamInput in) throws IOException { super(in); - aliases = in.readImmutableMap(StreamInput::readString, i -> i.readList(AliasMetadata::new)); + aliases = in.readMap(StreamInput::readString, i -> i.readList(AliasMetadata::new)); } - public ImmutableOpenMap> getAliases() { + public Map> getAliases() { return aliases; } diff --git a/server/src/main/java/org/opensearch/action/admin/indices/alias/get/TransportGetAliasesAction.java b/server/src/main/java/org/opensearch/action/admin/indices/alias/get/TransportGetAliasesAction.java index fe9c2dbccdf7b..a30074588cc42 100644 --- a/server/src/main/java/org/opensearch/action/admin/indices/alias/get/TransportGetAliasesAction.java +++ b/server/src/main/java/org/opensearch/action/admin/indices/alias/get/TransportGetAliasesAction.java @@ -41,7 +41,6 @@ import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.cluster.metadata.IndexNameExpressionResolver; import org.opensearch.cluster.service.ClusterService; -import org.opensearch.common.collect.ImmutableOpenMap; import org.opensearch.common.inject.Inject; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.logging.DeprecationLogger; @@ -53,9 +52,11 @@ import java.io.IOException; import java.util.Arrays; import java.util.Collections; +import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; import java.util.List; +import java.util.Map; import java.util.Set; import java.util.stream.Collectors; @@ -120,7 +121,7 @@ protected void clusterManagerOperation(GetAliasesRequest request, ClusterState s concreteIndices = indexNameExpressionResolver.concreteIndexNames(state, request); } final boolean systemIndexAccessAllowed = indexNameExpressionResolver.isSystemIndexAccessAllowed(); - ImmutableOpenMap> aliases = state.metadata().findAliases(request, concreteIndices); + final Map> aliases = state.metadata().findAliases(request, concreteIndices); listener.onResponse( new GetAliasesResponse(postProcess(request, concreteIndices, aliases, state, systemIndexAccessAllowed, systemIndices)) ); @@ -129,23 +130,23 @@ protected void clusterManagerOperation(GetAliasesRequest request, ClusterState s /** * Fills alias result with empty entries for requested indices when no specific aliases were requested. */ - static ImmutableOpenMap> postProcess( + static Map> postProcess( GetAliasesRequest request, String[] concreteIndices, - ImmutableOpenMap> aliases, + final Map> aliases, ClusterState state, boolean systemIndexAccessAllowed, SystemIndices systemIndices ) { boolean noAliasesSpecified = request.getOriginalAliases() == null || request.getOriginalAliases().length == 0; - ImmutableOpenMap.Builder> mapBuilder = ImmutableOpenMap.builder(aliases); + final Map> mapBuilder = new HashMap<>(aliases); for (String index : concreteIndices) { if (aliases.get(index) == null && noAliasesSpecified) { List previous = mapBuilder.put(index, Collections.emptyList()); assert previous == null; } } - final ImmutableOpenMap> finalResponse = mapBuilder.build(); + final Map> finalResponse = Collections.unmodifiableMap(mapBuilder); if (systemIndexAccessAllowed == false) { checkSystemIndexAccess(request, systemIndices, state, finalResponse); } @@ -156,10 +157,10 @@ private static void checkSystemIndexAccess( GetAliasesRequest request, SystemIndices systemIndices, ClusterState state, - ImmutableOpenMap> aliasesMap + final Map> aliasesMap ) { Set systemIndicesNames = new HashSet<>(); - for (Iterator it = aliasesMap.keysIt(); it.hasNext();) { + for (Iterator it = aliasesMap.keySet().iterator(); it.hasNext();) { String indexName = it.next(); IndexMetadata index = state.metadata().index(indexName); if (index != null && index.isSystem()) { diff --git a/server/src/main/java/org/opensearch/action/admin/indices/get/GetIndexResponse.java b/server/src/main/java/org/opensearch/action/admin/indices/get/GetIndexResponse.java index d15573ad3276c..a745af84f4bfc 100644 --- a/server/src/main/java/org/opensearch/action/admin/indices/get/GetIndexResponse.java +++ b/server/src/main/java/org/opensearch/action/admin/indices/get/GetIndexResponse.java @@ -64,7 +64,7 @@ public class GetIndexResponse extends ActionResponse implements ToXContentObject { private Map mappings = Map.of(); - private ImmutableOpenMap> aliases = ImmutableOpenMap.of(); + private Map> aliases = Map.of(); private ImmutableOpenMap settings = ImmutableOpenMap.of(); private ImmutableOpenMap defaultSettings = ImmutableOpenMap.of(); private ImmutableOpenMap dataStreams = ImmutableOpenMap.of(); @@ -73,7 +73,7 @@ public class GetIndexResponse extends ActionResponse implements ToXContentObject public GetIndexResponse( String[] indices, Map mappings, - ImmutableOpenMap> aliases, + final Map> aliases, ImmutableOpenMap settings, ImmutableOpenMap defaultSettings, ImmutableOpenMap dataStreams @@ -85,7 +85,7 @@ public GetIndexResponse( this.mappings = mappings; } if (aliases != null) { - this.aliases = aliases; + this.aliases = Collections.unmodifiableMap(aliases); } if (settings != null) { this.settings = settings; @@ -127,7 +127,7 @@ public GetIndexResponse( mappings = Collections.unmodifiableMap(mappingsMapBuilder); int aliasesSize = in.readVInt(); - ImmutableOpenMap.Builder> aliasesMapBuilder = ImmutableOpenMap.builder(); + final Map> aliasesMapBuilder = new HashMap<>(); for (int i = 0; i < aliasesSize; i++) { String key = in.readString(); int valueSize = in.readVInt(); @@ -137,7 +137,7 @@ public GetIndexResponse( } aliasesMapBuilder.put(key, Collections.unmodifiableList(aliasEntryBuilder)); } - aliases = aliasesMapBuilder.build(); + aliases = Collections.unmodifiableMap(aliasesMapBuilder); int settingsSize = in.readVInt(); ImmutableOpenMap.Builder settingsMapBuilder = ImmutableOpenMap.builder(); @@ -178,11 +178,11 @@ public Map getMappings() { return mappings(); } - public ImmutableOpenMap> aliases() { + public Map> aliases() { return aliases; } - public ImmutableOpenMap> getAliases() { + public Map> getAliases() { return aliases(); } @@ -255,10 +255,10 @@ public void writeTo(StreamOutput out) throws IOException { } } out.writeVInt(aliases.size()); - for (ObjectObjectCursor> indexEntry : aliases) { - out.writeString(indexEntry.key); - out.writeVInt(indexEntry.value.size()); - for (AliasMetadata aliasEntry : indexEntry.value) { + for (final Map.Entry> indexEntry : aliases.entrySet()) { + out.writeString(indexEntry.getKey()); + out.writeVInt(indexEntry.getValue().size()); + for (AliasMetadata aliasEntry : indexEntry.getValue()) { aliasEntry.writeTo(out); } } diff --git a/server/src/main/java/org/opensearch/action/admin/indices/get/TransportGetIndexAction.java b/server/src/main/java/org/opensearch/action/admin/indices/get/TransportGetIndexAction.java index dbd9afd58a1fa..48812783c136c 100644 --- a/server/src/main/java/org/opensearch/action/admin/indices/get/TransportGetIndexAction.java +++ b/server/src/main/java/org/opensearch/action/admin/indices/get/TransportGetIndexAction.java @@ -106,7 +106,7 @@ protected void doClusterManagerOperation( final ActionListener listener ) { Map mappingsResult = Map.of(); - ImmutableOpenMap> aliasesResult = ImmutableOpenMap.of(); + Map> aliasesResult = Map.of(); ImmutableOpenMap settings = ImmutableOpenMap.of(); ImmutableOpenMap defaultSettings = ImmutableOpenMap.of(); ImmutableOpenMap dataStreams = ImmutableOpenMap.builder() @@ -133,7 +133,7 @@ protected void doClusterManagerOperation( } break; case ALIASES: - if (!doneAliases) { + if (doneAliases == false) { aliasesResult = state.metadata().findAllAliases(concreteIndices); doneAliases = true; } diff --git a/server/src/main/java/org/opensearch/cluster/ClusterModule.java b/server/src/main/java/org/opensearch/cluster/ClusterModule.java index 011a3d988c7f9..cf00f44bea4b4 100644 --- a/server/src/main/java/org/opensearch/cluster/ClusterModule.java +++ b/server/src/main/java/org/opensearch/cluster/ClusterModule.java @@ -224,7 +224,7 @@ public static List getNamedWriteables() { */ public static ClusterState filterCustomsForPre63Clients(ClusterState clusterState) { final ClusterState.Builder builder = ClusterState.builder(clusterState); - clusterState.customs().keysIt().forEachRemaining(name -> { + clusterState.customs().keySet().iterator().forEachRemaining(name -> { if (PRE_6_3_CLUSTER_CUSTOMS_WHITE_LIST.contains(name) == false) { builder.removeCustom(name); } diff --git a/server/src/main/java/org/opensearch/cluster/ClusterState.java b/server/src/main/java/org/opensearch/cluster/ClusterState.java index 5d32d2edb8494..fda9b9a0ea588 100644 --- a/server/src/main/java/org/opensearch/cluster/ClusterState.java +++ b/server/src/main/java/org/opensearch/cluster/ClusterState.java @@ -32,8 +32,6 @@ package org.opensearch.cluster; -import com.carrotsearch.hppc.cursors.ObjectCursor; -import com.carrotsearch.hppc.cursors.ObjectObjectCursor; import org.opensearch.cluster.block.ClusterBlock; import org.opensearch.cluster.block.ClusterBlocks; import org.opensearch.cluster.coordination.CoordinationMetadata; @@ -53,7 +51,6 @@ import org.opensearch.common.Strings; import org.opensearch.common.UUIDs; import org.opensearch.common.bytes.BytesReference; -import org.opensearch.common.collect.ImmutableOpenMap; import org.opensearch.common.io.stream.BytesStreamOutput; import org.opensearch.common.io.stream.NamedWriteableAwareStreamInput; import org.opensearch.common.io.stream.NamedWriteableRegistry; @@ -66,12 +63,14 @@ import org.opensearch.discovery.Discovery; import java.io.IOException; +import java.util.Collections; import java.util.EnumSet; import java.util.HashMap; import java.util.Map; import java.util.Objects; import java.util.Optional; import java.util.Set; +import java.util.Spliterators; import java.util.stream.StreamSupport; import static org.opensearch.cluster.coordination.Coordinator.ZEN1_BWC_TERM; @@ -172,7 +171,7 @@ default boolean isPrivate() { private final ClusterBlocks blocks; - private final ImmutableOpenMap customs; + private final Map customs; private final ClusterName clusterName; @@ -206,7 +205,7 @@ public ClusterState( RoutingTable routingTable, DiscoveryNodes nodes, ClusterBlocks blocks, - ImmutableOpenMap customs, + final Map customs, int minimumClusterManagerNodesOnPublishingClusterManager, boolean wasReadFromDiff ) { @@ -217,7 +216,7 @@ public ClusterState( this.routingTable = routingTable; this.nodes = nodes; this.blocks = blocks; - this.customs = customs; + this.customs = Collections.unmodifiableMap(customs); this.minimumClusterManagerNodesOnPublishingClusterManager = minimumClusterManagerNodesOnPublishingClusterManager; this.wasReadFromDiff = wasReadFromDiff; } @@ -285,11 +284,11 @@ public ClusterBlocks getBlocks() { return blocks; } - public ImmutableOpenMap customs() { + public Map customs() { return this.customs; } - public ImmutableOpenMap getCustoms() { + public Map getCustoms() { return this.customs; } @@ -388,9 +387,9 @@ public String toString() { sb.append(getRoutingNodes()); if (customs.isEmpty() == false) { sb.append("customs:\n"); - for (ObjectObjectCursor cursor : customs) { - final String type = cursor.key; - final Custom custom = cursor.value; + for (final Map.Entry cursor : customs.entrySet()) { + final String type = cursor.getKey(); + final Custom custom = cursor.getValue(); sb.append(TAB).append(type).append(": ").append(custom); } } @@ -496,7 +495,7 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws if (metrics.contains(Metric.BLOCKS)) { builder.startObject("blocks"); - if (!blocks().global().isEmpty()) { + if (blocks().global().isEmpty() == false) { builder.startObject("global"); for (ClusterBlock block : blocks().global()) { block.toXContent(builder, params); @@ -504,11 +503,11 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws builder.endObject(); } - if (!blocks().indices().isEmpty()) { + if (blocks().indices().isEmpty() == false) { builder.startObject("indices"); - for (ObjectObjectCursor> entry : blocks().indices()) { - builder.startObject(entry.key); - for (ClusterBlock block : entry.value) { + for (final Map.Entry> entry : blocks().indices().entrySet()) { + builder.startObject(entry.getKey()); + for (ClusterBlock block : entry.getValue()) { block.toXContent(builder, params); } builder.endObject(); @@ -576,9 +575,9 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws builder.endObject(); } if (metrics.contains(Metric.CUSTOMS)) { - for (ObjectObjectCursor cursor : customs) { - builder.startObject(cursor.key); - cursor.value.toXContent(builder, params); + for (final Map.Entry cursor : customs.entrySet()) { + builder.startObject(cursor.getKey()); + cursor.getValue().toXContent(builder, params); builder.endObject(); } } @@ -608,7 +607,7 @@ public static class Builder { private RoutingTable routingTable = RoutingTable.EMPTY_ROUTING_TABLE; private DiscoveryNodes nodes = DiscoveryNodes.EMPTY_NODES; private ClusterBlocks blocks = ClusterBlocks.EMPTY_CLUSTER_BLOCK; - private final ImmutableOpenMap.Builder customs; + private final Map customs; private boolean fromDiff; private int minimumClusterManagerNodesOnPublishingClusterManager = -1; @@ -620,13 +619,13 @@ public Builder(ClusterState state) { this.routingTable = state.routingTable(); this.metadata = state.metadata(); this.blocks = state.blocks(); - this.customs = ImmutableOpenMap.builder(state.customs()); + this.customs = new HashMap<>(state.customs()); this.minimumClusterManagerNodesOnPublishingClusterManager = state.minimumClusterManagerNodesOnPublishingClusterManager; this.fromDiff = false; } public Builder(ClusterName clusterName) { - customs = ImmutableOpenMap.builder(); + customs = new HashMap<>(); this.clusterName = clusterName; } @@ -697,8 +696,9 @@ public Builder removeCustom(String type) { return this; } - public Builder customs(ImmutableOpenMap customs) { - StreamSupport.stream(customs.spliterator(), false).forEach(cursor -> Objects.requireNonNull(cursor.value, cursor.key)); + public Builder customs(final Map customs) { + StreamSupport.stream(Spliterators.spliterator(customs.entrySet(), 0), false) + .forEach(cursor -> Objects.requireNonNull(cursor.getValue(), cursor.getKey())); this.customs.putAll(customs); return this; } @@ -720,7 +720,7 @@ public ClusterState build() { routingTable, nodes, blocks, - customs.build(), + customs, minimumClusterManagerNodesOnPublishingClusterManager, fromDiff ); @@ -781,15 +781,15 @@ public void writeTo(StreamOutput out) throws IOException { blocks.writeTo(out); // filter out custom states not supported by the other node int numberOfCustoms = 0; - for (final ObjectCursor cursor : customs.values()) { - if (FeatureAware.shouldSerialize(out, cursor.value)) { + for (final Custom custom : customs.values()) { + if (FeatureAware.shouldSerialize(out, custom)) { numberOfCustoms++; } } out.writeVInt(numberOfCustoms); - for (final ObjectCursor cursor : customs.values()) { - if (FeatureAware.shouldSerialize(out, cursor.value)) { - out.writeNamedWriteable(cursor.value); + for (final Custom custom : customs.values()) { + if (FeatureAware.shouldSerialize(out, custom)) { + out.writeNamedWriteable(custom); } } out.writeVInt(minimumClusterManagerNodesOnPublishingClusterManager); @@ -818,7 +818,7 @@ private static class ClusterStateDiff implements Diff { private final Diff blocks; - private final Diff> customs; + private final Diff> customs; private final int minimumClusterManagerNodesOnPublishingClusterManager; @@ -844,7 +844,7 @@ private static class ClusterStateDiff implements Diff { nodes = DiscoveryNodes.readDiffFrom(in, localNode); metadata = Metadata.readDiffFrom(in); blocks = ClusterBlocks.readDiffFrom(in); - customs = DiffableUtils.readImmutableOpenMapDiff(in, DiffableUtils.getStringKeySerializer(), CUSTOM_VALUE_SERIALIZER); + customs = DiffableUtils.readJdkMapDiff(in, DiffableUtils.getStringKeySerializer(), CUSTOM_VALUE_SERIALIZER); minimumClusterManagerNodesOnPublishingClusterManager = in.readVInt(); } diff --git a/server/src/main/java/org/opensearch/cluster/RestoreInProgress.java b/server/src/main/java/org/opensearch/cluster/RestoreInProgress.java index 75153e265481e..8fa56fd073ac0 100644 --- a/server/src/main/java/org/opensearch/cluster/RestoreInProgress.java +++ b/server/src/main/java/org/opensearch/cluster/RestoreInProgress.java @@ -32,11 +32,8 @@ package org.opensearch.cluster; -import com.carrotsearch.hppc.cursors.ObjectCursor; -import com.carrotsearch.hppc.cursors.ObjectObjectCursor; import org.opensearch.Version; import org.opensearch.cluster.ClusterState.Custom; -import org.opensearch.common.collect.ImmutableOpenMap; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.io.stream.StreamOutput; import org.opensearch.common.io.stream.Writeable; @@ -47,8 +44,10 @@ import java.io.IOException; import java.util.Collections; +import java.util.HashMap; import java.util.Iterator; import java.util.List; +import java.util.Map; import java.util.Objects; import java.util.UUID; @@ -66,17 +65,17 @@ public class RestoreInProgress extends AbstractNamedDiffable implements public static final String TYPE = "restore"; - public static final RestoreInProgress EMPTY = new RestoreInProgress(ImmutableOpenMap.of()); + public static final RestoreInProgress EMPTY = new RestoreInProgress(Map.of()); - private final ImmutableOpenMap entries; + private final Map entries; /** * Constructs new restore metadata * * @param entries map of currently running restore processes keyed by their restore uuid */ - private RestoreInProgress(ImmutableOpenMap entries) { - this.entries = entries; + private RestoreInProgress(final Map entries) { + this.entries = Collections.unmodifiableMap(entries); } @Override @@ -94,7 +93,7 @@ public int hashCode() { @Override public String toString() { StringBuilder builder = new StringBuilder("RestoreInProgress["); - entries.forEach(entry -> builder.append("{").append(entry.key).append("}{").append(entry.value.snapshot).append("},")); + entries.forEach((s, entry) -> builder.append("{").append(s).append("}{").append(entry.snapshot).append("},")); builder.setCharAt(builder.length() - 1, ']'); return builder.toString(); } @@ -109,7 +108,7 @@ public boolean isEmpty() { @Override public Iterator iterator() { - return entries.valuesIt(); + return entries.values().iterator(); } /** @@ -119,7 +118,7 @@ public Iterator iterator() { */ public static final class Builder { - private final ImmutableOpenMap.Builder entries = ImmutableOpenMap.builder(); + private final Map entries = new HashMap<>(); public Builder() {} @@ -133,7 +132,7 @@ public Builder add(Entry entry) { } public RestoreInProgress build() { - return entries.isEmpty() ? EMPTY : new RestoreInProgress(entries.build()); + return entries.isEmpty() ? EMPTY : new RestoreInProgress(entries); } } @@ -146,7 +145,7 @@ public static class Entry { private final String uuid; private final State state; private final Snapshot snapshot; - private final ImmutableOpenMap shards; + private final Map shards; private final List indices; /** @@ -158,20 +157,14 @@ public static class Entry { * @param indices list of indices being restored * @param shards map of shards being restored to their current restore status */ - public Entry( - String uuid, - Snapshot snapshot, - State state, - List indices, - ImmutableOpenMap shards - ) { + public Entry(String uuid, Snapshot snapshot, State state, List indices, final Map shards) { this.snapshot = Objects.requireNonNull(snapshot); this.state = Objects.requireNonNull(state); this.indices = Objects.requireNonNull(indices); if (shards == null) { - this.shards = ImmutableOpenMap.of(); + this.shards = Map.of(); } else { - this.shards = shards; + this.shards = Collections.unmodifiableMap(shards); } this.uuid = Objects.requireNonNull(uuid); } @@ -198,7 +191,7 @@ public Snapshot snapshot() { * * @return list of shards */ - public ImmutableOpenMap shards() { + public Map shards() { return this.shards; } @@ -460,7 +453,7 @@ public static NamedDiff readDiffFrom(StreamInput in) throws IOException public RestoreInProgress(StreamInput in) throws IOException { int count = in.readVInt(); - final ImmutableOpenMap.Builder entriesBuilder = ImmutableOpenMap.builder(count); + final Map entriesBuilder = new HashMap<>(count); for (int i = 0; i < count; i++) { final String uuid; uuid = in.readString(); @@ -474,31 +467,30 @@ public RestoreInProgress(StreamInput in) throws IOException { snapshot, state, Collections.unmodifiableList(indexBuilder), - in.readImmutableMap(ShardId::new, ShardRestoreStatus::readShardRestoreStatus) + in.readMap(ShardId::new, ShardRestoreStatus::readShardRestoreStatus) ) ); } - this.entries = entriesBuilder.build(); + this.entries = Collections.unmodifiableMap(entriesBuilder); } @Override public void writeTo(StreamOutput out) throws IOException { out.writeVInt(entries.size()); - for (ObjectCursor v : entries.values()) { - Entry entry = v.value; + for (final Entry entry : entries.values()) { out.writeString(entry.uuid); entry.snapshot().writeTo(out); out.writeByte(entry.state().value()); out.writeStringCollection(entry.indices); - out.writeMap(entry.shards); + out.writeMap(entry.shards, (o, shardId) -> shardId.writeTo(o), (o, status) -> status.writeTo(o)); } } @Override public XContentBuilder toXContent(XContentBuilder builder, ToXContent.Params params) throws IOException { builder.startArray("snapshots"); - for (ObjectCursor entry : entries.values()) { - toXContent(entry.value, builder); + for (final Entry entry : entries.values()) { + toXContent(entry, builder); } builder.endArray(); return builder; @@ -524,9 +516,9 @@ public void toXContent(Entry entry, XContentBuilder builder) throws IOException builder.endArray(); builder.startArray("shards"); { - for (ObjectObjectCursor shardEntry : entry.shards) { - ShardId shardId = shardEntry.key; - ShardRestoreStatus status = shardEntry.value; + for (final Map.Entry shardEntry : entry.shards.entrySet()) { + ShardId shardId = shardEntry.getKey(); + ShardRestoreStatus status = shardEntry.getValue(); builder.startObject(); { builder.field("index", shardId.getIndex()); diff --git a/server/src/main/java/org/opensearch/cluster/SnapshotsInProgress.java b/server/src/main/java/org/opensearch/cluster/SnapshotsInProgress.java index d0d0da301ffaa..8514af1f640e2 100644 --- a/server/src/main/java/org/opensearch/cluster/SnapshotsInProgress.java +++ b/server/src/main/java/org/opensearch/cluster/SnapshotsInProgress.java @@ -32,14 +32,10 @@ package org.opensearch.cluster; -import com.carrotsearch.hppc.ObjectContainer; -import com.carrotsearch.hppc.cursors.ObjectCursor; -import com.carrotsearch.hppc.cursors.ObjectObjectCursor; import org.opensearch.Version; import org.opensearch.cluster.ClusterState.Custom; import org.opensearch.common.Nullable; import org.opensearch.common.Strings; -import org.opensearch.common.collect.ImmutableOpenMap; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.io.stream.StreamOutput; import org.opensearch.common.io.stream.Writeable; @@ -56,6 +52,7 @@ import org.opensearch.snapshots.SnapshotId; import java.io.IOException; +import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; @@ -114,7 +111,7 @@ public static Entry startedEntry( List dataStreams, long startTime, long repositoryStateId, - ImmutableOpenMap shards, + final Map shards, Map userMetadata, Version version ) { @@ -162,12 +159,12 @@ public static Entry startClone( Collections.emptyList(), startTime, repositoryStateId, - ImmutableOpenMap.of(), + Map.of(), null, Collections.emptyMap(), version, source, - ImmutableOpenMap.of() + Map.of() ); } @@ -184,7 +181,7 @@ public static class Entry implements Writeable, ToXContent, RepositoryOperation /** * Map of {@link ShardId} to {@link ShardSnapshotStatus} tracking the state of each shard snapshot operation. */ - private final ImmutableOpenMap shards; + private final Map shards; private final List indices; private final List dataStreams; private final long startTime; @@ -202,7 +199,7 @@ public static class Entry implements Writeable, ToXContent, RepositoryOperation * Map of {@link RepositoryShardId} to {@link ShardSnapshotStatus} tracking the state of each shard clone operation in this entry * the same way {@link #shards} tracks the status of each shard snapshot operation in non-clone entries. */ - private final ImmutableOpenMap clones; + private final Map clones; @Nullable private final Map userMetadata; @@ -219,7 +216,7 @@ public Entry( List dataStreams, long startTime, long repositoryStateId, - ImmutableOpenMap shards, + final Map shards, String failure, Map userMetadata, Version version @@ -238,7 +235,7 @@ public Entry( userMetadata, version, null, - ImmutableOpenMap.of() + Map.of() ); } @@ -251,12 +248,12 @@ private Entry( List dataStreams, long startTime, long repositoryStateId, - ImmutableOpenMap shards, + final Map shards, String failure, - Map userMetadata, + final Map userMetadata, Version version, @Nullable SnapshotId source, - @Nullable ImmutableOpenMap clones + @Nullable final Map clones ) { this.state = state; this.snapshot = snapshot; @@ -265,7 +262,7 @@ private Entry( this.indices = indices; this.dataStreams = dataStreams; this.startTime = startTime; - this.shards = shards; + this.shards = Collections.unmodifiableMap(shards); this.repositoryStateId = repositoryStateId; this.failure = failure; this.userMetadata = userMetadata; @@ -273,9 +270,9 @@ private Entry( this.source = source; if (source == null) { assert clones == null || clones.isEmpty() : "Provided [" + clones + "] but no source"; - this.clones = ImmutableOpenMap.of(); + this.clones = Map.of(); } else { - this.clones = clones; + this.clones = Collections.unmodifiableMap(clones); } assert assertShardsConsistent(this.source, this.state, this.indices, this.shards, this.clones); } @@ -287,31 +284,31 @@ private Entry(StreamInput in) throws IOException { state = State.fromValue(in.readByte()); indices = in.readList(IndexId::new); startTime = in.readLong(); - shards = in.readImmutableMap(ShardId::new, ShardSnapshotStatus::readFrom); + shards = in.readMap(ShardId::new, ShardSnapshotStatus::readFrom); repositoryStateId = in.readLong(); failure = in.readOptionalString(); userMetadata = in.readMap(); version = Version.readVersion(in); dataStreams = in.readStringList(); source = in.readOptionalWriteable(SnapshotId::new); - clones = in.readImmutableMap(RepositoryShardId::new, ShardSnapshotStatus::readFrom); + clones = in.readMap(RepositoryShardId::new, ShardSnapshotStatus::readFrom); } private static boolean assertShardsConsistent( SnapshotId source, State state, List indices, - ImmutableOpenMap shards, - ImmutableOpenMap clones + final Map shards, + final Map clones ) { if ((state == State.INIT || state == State.ABORTED) && shards.isEmpty()) { return true; } final Set indexNames = indices.stream().map(IndexId::getName).collect(Collectors.toSet()); final Set indexNamesInShards = new HashSet<>(); - shards.iterator().forEachRemaining(s -> { - indexNamesInShards.add(s.key.getIndexName()); - assert source == null || s.value.nodeId == null + shards.entrySet().forEach(s -> { + indexNamesInShards.add(s.getKey().getIndexName()); + assert source == null || s.getValue().nodeId == null : "Shard snapshot must not be assigned to data node when copying from snapshot [" + source + "]"; }); assert source == null || indexNames.isEmpty() == false : "No empty snapshot clones allowed"; @@ -347,7 +344,7 @@ public Entry( List dataStreams, long startTime, long repositoryStateId, - ImmutableOpenMap shards, + final Map shards, Map userMetadata, Version version ) { @@ -372,7 +369,7 @@ public Entry( State state, List indices, long repositoryStateId, - ImmutableOpenMap shards, + final Map shards, Version version, String failure ) { @@ -416,7 +413,7 @@ public Entry withRepoGen(long newRepoGen) { ); } - public Entry withClones(ImmutableOpenMap updatedClones) { + public Entry withClones(final Map updatedClones) { if (updatedClones.equals(clones)) { return this; } @@ -450,11 +447,11 @@ public Entry withClones(ImmutableOpenMap */ @Nullable public Entry abort() { - final ImmutableOpenMap.Builder shardsBuilder = ImmutableOpenMap.builder(); + final Map shardsBuilder = new HashMap<>(); boolean completed = true; boolean allQueued = true; - for (ObjectObjectCursor shardEntry : shards) { - ShardSnapshotStatus status = shardEntry.value; + for (final Map.Entry shardEntry : shards.entrySet()) { + ShardSnapshotStatus status = shardEntry.getValue(); allQueued &= status.state() == ShardState.QUEUED; if (status.state().completed() == false) { final String nodeId = status.nodeId(); @@ -466,15 +463,15 @@ public Entry abort() { ); } completed &= status.state().completed(); - shardsBuilder.put(shardEntry.key, status); + shardsBuilder.put(shardEntry.getKey(), status); } if (allQueued) { return null; } - return fail(shardsBuilder.build(), completed ? State.SUCCESS : State.ABORTED, ABORTED_FAILURE_TEXT); + return fail(shardsBuilder, completed ? State.SUCCESS : State.ABORTED, ABORTED_FAILURE_TEXT); } - public Entry fail(ImmutableOpenMap shards, State state, String failure) { + public Entry fail(final Map shards, State state, String failure) { return new Entry( snapshot, includeGlobalState, @@ -501,7 +498,7 @@ public Entry fail(ImmutableOpenMap shards, State s * @param shards new shard snapshot states * @return new snapshot entry */ - public Entry withShardStates(ImmutableOpenMap shards) { + public Entry withShardStates(final Map shards) { if (completed(shards.values())) { return new Entry( snapshot, @@ -525,7 +522,7 @@ public Entry withShardStates(ImmutableOpenMap shar * Same as {@link #withShardStates} but does not check if the snapshot completed and thus is only to be used when starting new * shard snapshots on data nodes for a running snapshot. */ - public Entry withStartedShards(ImmutableOpenMap shards) { + public Entry withStartedShards(final Map shards) { final SnapshotsInProgress.Entry updated = new Entry( snapshot, includeGlobalState, @@ -554,7 +551,7 @@ public Snapshot snapshot() { return this.snapshot; } - public ImmutableOpenMap shards() { + public Map shards() { return this.shards; } @@ -611,7 +608,7 @@ public boolean isClone() { return source != null; } - public ImmutableOpenMap clones() { + public Map clones() { return clones; } @@ -678,9 +675,9 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws builder.field(REPOSITORY_STATE_ID, repositoryStateId); builder.startArray(SHARDS); { - for (ObjectObjectCursor shardEntry : shards) { - ShardId shardId = shardEntry.key; - ShardSnapshotStatus status = shardEntry.value; + for (final Map.Entry shardEntry : shards.entrySet()) { + ShardId shardId = shardEntry.getKey(); + ShardSnapshotStatus status = shardEntry.getValue(); builder.startObject(); { builder.field(INDEX, shardId.getIndex()); @@ -705,14 +702,14 @@ public void writeTo(StreamOutput out) throws IOException { out.writeByte(state.value()); out.writeList(indices); out.writeLong(startTime); - out.writeMap(shards); + out.writeMap(shards, (o, v) -> v.writeTo(o), (o, v) -> v.writeTo(o)); out.writeLong(repositoryStateId); out.writeOptionalString(failure); out.writeMap(userMetadata); Version.writeVersion(version, out); out.writeStringCollection(dataStreams); out.writeOptionalWriteable(source); - out.writeMap(clones); + out.writeMap(clones, (o, v) -> v.writeTo(o), (o, v) -> v.writeTo(o)); } @Override @@ -727,18 +724,18 @@ public boolean isFragment() { * @param shards list of shard statuses * @return true if all shards have completed (either successfully or failed), false otherwise */ - public static boolean completed(ObjectContainer shards) { - for (ObjectCursor status : shards) { - if (status.value.state().completed == false) { + public static boolean completed(final Collection shards) { + for (final ShardSnapshotStatus status : shards) { + if (status.state().completed == false) { return false; } } return true; } - private static boolean hasFailures(ImmutableOpenMap clones) { - for (ObjectCursor value : clones.values()) { - if (value.value.state().failed()) { + private static boolean hasFailures(final Map clones) { + for (final ShardSnapshotStatus value : clones.values()) { + if (value.state().failed()) { return true; } } @@ -930,9 +927,9 @@ public static State fromValue(byte value) { private static boolean assertConsistentEntries(List entries) { final Map> assignedShardsByRepo = new HashMap<>(); for (Entry entry : entries) { - for (ObjectObjectCursor shard : entry.shards()) { - if (shard.value.isActive()) { - assert assignedShardsByRepo.computeIfAbsent(entry.repository(), k -> new HashSet<>()).add(shard.key) + for (final Map.Entry shard : entry.shards().entrySet()) { + if (shard.getValue().isActive()) { + assert assignedShardsByRepo.computeIfAbsent(entry.repository(), k -> new HashSet<>()).add(shard.getKey()) : "Found duplicate shard assignments in " + entries; } } diff --git a/server/src/main/java/org/opensearch/cluster/block/ClusterBlocks.java b/server/src/main/java/org/opensearch/cluster/block/ClusterBlocks.java index 167a1cc0fab98..cf77349271eb5 100644 --- a/server/src/main/java/org/opensearch/cluster/block/ClusterBlocks.java +++ b/server/src/main/java/org/opensearch/cluster/block/ClusterBlocks.java @@ -32,13 +32,11 @@ package org.opensearch.cluster.block; -import com.carrotsearch.hppc.cursors.ObjectObjectCursor; import org.opensearch.cluster.AbstractDiffable; import org.opensearch.cluster.Diff; import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.cluster.metadata.MetadataIndexStateService; import org.opensearch.common.Nullable; -import org.opensearch.common.collect.ImmutableOpenMap; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.io.stream.StreamOutput; import org.opensearch.common.util.set.Sets; @@ -64,17 +62,17 @@ * @opensearch.internal */ public class ClusterBlocks extends AbstractDiffable { - public static final ClusterBlocks EMPTY_CLUSTER_BLOCK = new ClusterBlocks(emptySet(), ImmutableOpenMap.of()); + public static final ClusterBlocks EMPTY_CLUSTER_BLOCK = new ClusterBlocks(emptySet(), Map.of()); private final Set global; - private final ImmutableOpenMap> indicesBlocks; + private final Map> indicesBlocks; private final EnumMap levelHolders; - ClusterBlocks(Set global, ImmutableOpenMap> indicesBlocks) { + ClusterBlocks(Set global, final Map> indicesBlocks) { this.global = global; - this.indicesBlocks = indicesBlocks; + this.indicesBlocks = Collections.unmodifiableMap(indicesBlocks); levelHolders = generateLevelHolders(global, indicesBlocks); } @@ -82,7 +80,7 @@ public Set global() { return global; } - public ImmutableOpenMap> indices() { + public Map> indices() { return indicesBlocks; } @@ -90,7 +88,7 @@ public Set global(ClusterBlockLevel level) { return levelHolders.get(level).global(); } - public ImmutableOpenMap> indices(ClusterBlockLevel level) { + public Map> indices(ClusterBlockLevel level) { return levelHolders.get(level).indices(); } @@ -100,7 +98,7 @@ private Set blocksForIndex(ClusterBlockLevel level, String index) private static EnumMap generateLevelHolders( Set global, - ImmutableOpenMap> indicesBlocks + final Map> indicesBlocks ) { EnumMap levelHolders = new EnumMap<>(ClusterBlockLevel.class); @@ -108,11 +106,11 @@ private static EnumMap generateLevelHol Predicate containsLevel = block -> block.contains(level); Set newGlobal = unmodifiableSet(global.stream().filter(containsLevel).collect(toSet())); - ImmutableOpenMap.Builder> indicesBuilder = ImmutableOpenMap.builder(); - for (ObjectObjectCursor> entry : indicesBlocks) { - indicesBuilder.put(entry.key, unmodifiableSet(entry.value.stream().filter(containsLevel).collect(toSet()))); + final Map> indicesBuilder = new HashMap<>(); + for (final Map.Entry> entry : indicesBlocks.entrySet()) { + indicesBuilder.put(entry.getKey(), unmodifiableSet(entry.getValue().stream().filter(containsLevel).collect(toSet()))); } - levelHolders.put(level, new ImmutableLevelHolder(newGlobal, indicesBuilder.build())); + levelHolders.put(level, new ImmutableLevelHolder(newGlobal, indicesBuilder)); } return levelHolders; } @@ -288,9 +286,9 @@ public String toString() { sb.append(" ").append(block); } } - for (ObjectObjectCursor> entry : indices()) { - sb.append(" ").append(entry.key).append(":\n"); - for (ClusterBlock block : entry.value) { + for (final Map.Entry> entry : indices().entrySet()) { + sb.append(" ").append(entry.getKey()).append(":\n"); + for (ClusterBlock block : entry.getValue()) { sb.append(" ").append(block); } } @@ -310,10 +308,7 @@ private static void writeBlockSet(Set blocks, StreamOutput out) th public static ClusterBlocks readFrom(StreamInput in) throws IOException { final Set global = readBlockSet(in); - ImmutableOpenMap> indicesBlocks = in.readImmutableMap( - i -> i.readString().intern(), - ClusterBlocks::readBlockSet - ); + final Map> indicesBlocks = in.readMap(i -> i.readString().intern(), ClusterBlocks::readBlockSet); if (global.isEmpty() && indicesBlocks.isEmpty()) { return EMPTY_CLUSTER_BLOCK; } @@ -337,18 +332,18 @@ public static Diff readDiffFrom(StreamInput in) throws IOExceptio static class ImmutableLevelHolder { private final Set global; - private final ImmutableOpenMap> indices; + private final Map> indices; - ImmutableLevelHolder(Set global, ImmutableOpenMap> indices) { + ImmutableLevelHolder(Set global, final Map> indices) { this.global = global; - this.indices = indices; + this.indices = Collections.unmodifiableMap(indices); } public Set global() { return global; } - public ImmutableOpenMap> indices() { + public Map> indices() { return indices; } } @@ -372,11 +367,11 @@ public Builder() {} public Builder blocks(ClusterBlocks blocks) { global.addAll(blocks.global()); - for (ObjectObjectCursor> entry : blocks.indices()) { - if (!indices.containsKey(entry.key)) { - indices.put(entry.key, new HashSet<>()); + for (final Map.Entry> entry : blocks.indices().entrySet()) { + if (!indices.containsKey(entry.getKey())) { + indices.put(entry.getKey(), new HashSet<>()); } - indices.get(entry.key).addAll(entry.value); + indices.get(entry.getKey()).addAll(entry.getValue()); } return this; } @@ -476,11 +471,11 @@ public ClusterBlocks build() { return EMPTY_CLUSTER_BLOCK; } // We copy the block sets here in case of the builder is modified after build is called - ImmutableOpenMap.Builder> indicesBuilder = ImmutableOpenMap.builder(indices.size()); + final HashMap> indicesBuilder = new HashMap<>(indices.size()); for (Map.Entry> entry : indices.entrySet()) { indicesBuilder.put(entry.getKey(), unmodifiableSet(new HashSet<>(entry.getValue()))); } - return new ClusterBlocks(unmodifiableSet(new HashSet<>(global)), indicesBuilder.build()); + return new ClusterBlocks(unmodifiableSet(new HashSet<>(global)), indicesBuilder); } } } diff --git a/server/src/main/java/org/opensearch/cluster/metadata/Metadata.java b/server/src/main/java/org/opensearch/cluster/metadata/Metadata.java index 67df17d1ca108..ab39086da970a 100644 --- a/server/src/main/java/org/opensearch/cluster/metadata/Metadata.java +++ b/server/src/main/java/org/opensearch/cluster/metadata/Metadata.java @@ -382,7 +382,7 @@ public SortedMap getIndicesLookup() { * @return A map of index name to the list of aliases metadata. If a concrete index does not have matching * aliases then the result will not include the index's key. */ - public ImmutableOpenMap> findAllAliases(final String[] concreteIndices) { + public Map> findAllAliases(final String[] concreteIndices) { return findAliases(Strings.EMPTY_ARRAY, concreteIndices); } @@ -395,7 +395,7 @@ public ImmutableOpenMap> findAllAliases(final String * @return A map of index name to the list of aliases metadata. If a concrete index does not have matching * aliases then the result will not include the index's key. */ - public ImmutableOpenMap> findAliases(final AliasesRequest aliasesRequest, final String[] concreteIndices) { + public Map> findAliases(final AliasesRequest aliasesRequest, final String[] concreteIndices) { return findAliases(aliasesRequest.aliases(), concreteIndices); } @@ -408,11 +408,11 @@ public ImmutableOpenMap> findAliases(final AliasesRe * @return A map of index name to the list of aliases metadata. If a concrete index does not have matching * aliases then the result will not include the index's key. */ - private ImmutableOpenMap> findAliases(final String[] aliases, final String[] concreteIndices) { + private Map> findAliases(final String[] aliases, final String[] concreteIndices) { assert aliases != null; assert concreteIndices != null; if (concreteIndices.length == 0) { - return ImmutableOpenMap.of(); + return Map.of(); } String[] patterns = new String[aliases.length]; boolean[] include = new boolean[aliases.length]; @@ -427,7 +427,7 @@ private ImmutableOpenMap> findAliases(final String[] } } boolean matchAllAliases = patterns.length == 0; - ImmutableOpenMap.Builder> mapBuilder = ImmutableOpenMap.builder(); + final Map> mapBuilder = new HashMap<>(); for (String index : concreteIndices) { IndexMetadata indexMetadata = indices.get(index); List filteredValues = new ArrayList<>(); @@ -455,7 +455,7 @@ private ImmutableOpenMap> findAliases(final String[] mapBuilder.put(index, Collections.unmodifiableList(filteredValues)); } } - return mapBuilder.build(); + return mapBuilder; } /** @@ -1224,7 +1224,6 @@ public Builder templates(Map templates) { public Builder put(String name, ComponentTemplate componentTemplate) { Objects.requireNonNull(componentTemplate, "it is invalid to add a null component template: " + name); - // ಠ_ಠ at ImmutableOpenMap Map existingTemplates = Optional.ofNullable( (ComponentTemplateMetadata) this.customs.get(ComponentTemplateMetadata.TYPE) ).map(ctm -> new HashMap<>(ctm.componentTemplates())).orElse(new HashMap<>()); @@ -1234,7 +1233,6 @@ public Builder put(String name, ComponentTemplate componentTemplate) { } public Builder removeComponentTemplate(String name) { - // ಠ_ಠ at ImmutableOpenMap Map existingTemplates = Optional.ofNullable( (ComponentTemplateMetadata) this.customs.get(ComponentTemplateMetadata.TYPE) ).map(ctm -> new HashMap<>(ctm.componentTemplates())).orElse(new HashMap<>()); @@ -1255,7 +1253,6 @@ public Builder indexTemplates(Map indexTemplate public Builder put(String name, ComposableIndexTemplate indexTemplate) { Objects.requireNonNull(indexTemplate, "it is invalid to add a null index template: " + name); - // ಠ_ಠ at ImmutableOpenMap Map existingTemplates = Optional.ofNullable( (ComposableIndexTemplateMetadata) this.customs.get(ComposableIndexTemplateMetadata.TYPE) ).map(itmd -> new HashMap<>(itmd.indexTemplates())).orElse(new HashMap<>()); @@ -1265,7 +1262,6 @@ public Builder put(String name, ComposableIndexTemplate indexTemplate) { } public Builder removeIndexTemplate(String name) { - // ಠ_ಠ at ImmutableOpenMap Map existingTemplates = Optional.ofNullable( (ComposableIndexTemplateMetadata) this.customs.get(ComposableIndexTemplateMetadata.TYPE) ).map(itmd -> new HashMap<>(itmd.indexTemplates())).orElse(new HashMap<>()); diff --git a/server/src/main/java/org/opensearch/cluster/metadata/MetadataDeleteIndexService.java b/server/src/main/java/org/opensearch/cluster/metadata/MetadataDeleteIndexService.java index 655b5ceb376f5..b23784bcd18aa 100644 --- a/server/src/main/java/org/opensearch/cluster/metadata/MetadataDeleteIndexService.java +++ b/server/src/main/java/org/opensearch/cluster/metadata/MetadataDeleteIndexService.java @@ -47,7 +47,6 @@ import org.opensearch.cluster.service.ClusterManagerTaskThrottler; import org.opensearch.cluster.service.ClusterService; import org.opensearch.common.Priority; -import org.opensearch.common.collect.ImmutableOpenMap; import org.opensearch.common.inject.Inject; import org.opensearch.common.settings.Settings; import org.opensearch.common.util.set.Sets; @@ -57,6 +56,7 @@ import org.opensearch.snapshots.SnapshotsService; import java.util.Arrays; +import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.Map; @@ -185,13 +185,13 @@ public ClusterState deleteIndices(ClusterState currentState, Set indices) ClusterBlocks blocks = clusterBlocksBuilder.build(); // update snapshot restore entries - ImmutableOpenMap customs = currentState.getCustoms(); + Map customs = currentState.getCustoms(); final RestoreInProgress restoreInProgress = currentState.custom(RestoreInProgress.TYPE, RestoreInProgress.EMPTY); RestoreInProgress updatedRestoreInProgress = RestoreService.updateRestoreStateWithDeletedIndices(restoreInProgress, indices); if (updatedRestoreInProgress != restoreInProgress) { - ImmutableOpenMap.Builder builder = ImmutableOpenMap.builder(customs); + final Map builder = new HashMap<>(customs); builder.put(RestoreInProgress.TYPE, updatedRestoreInProgress); - customs = builder.build(); + customs = Collections.unmodifiableMap(builder); } return allocationService.reroute( diff --git a/server/src/main/java/org/opensearch/cluster/routing/RoutingNodes.java b/server/src/main/java/org/opensearch/cluster/routing/RoutingNodes.java index 5cb0ecab1f27c..1f071d0e4bf08 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/RoutingNodes.java +++ b/server/src/main/java/org/opensearch/cluster/routing/RoutingNodes.java @@ -117,8 +117,8 @@ public RoutingNodes(ClusterState clusterState, boolean readOnly) { // fill in the inverse of node -> shards allocated // also fill replicaSet information - for (ObjectCursor indexRoutingTable : routingTable.indicesRouting().values()) { - for (IndexShardRoutingTable indexShard : indexRoutingTable.value) { + for (final IndexRoutingTable indexRoutingTable : routingTable.indicesRouting().values()) { + for (IndexShardRoutingTable indexShard : indexRoutingTable) { assert indexShard.primary != null; for (ShardRouting shard : indexShard) { // to get all the shards belonging to an index, including the replicas, diff --git a/server/src/main/java/org/opensearch/cluster/routing/RoutingTable.java b/server/src/main/java/org/opensearch/cluster/routing/RoutingTable.java index 15bb997bfb05a..065c2e02b1926 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/RoutingTable.java +++ b/server/src/main/java/org/opensearch/cluster/routing/RoutingTable.java @@ -33,8 +33,6 @@ package org.opensearch.cluster.routing; import com.carrotsearch.hppc.IntSet; -import com.carrotsearch.hppc.cursors.ObjectCursor; -import com.carrotsearch.hppc.cursors.ObjectObjectCursor; import org.opensearch.cluster.Diff; import org.opensearch.cluster.Diffable; import org.opensearch.cluster.DiffableUtils; @@ -43,7 +41,6 @@ import org.opensearch.cluster.routing.RecoverySource.SnapshotRecoverySource; import org.opensearch.cluster.routing.RecoverySource.RemoteStoreRecoverySource; import org.opensearch.common.Nullable; -import org.opensearch.common.collect.ImmutableOpenMap; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.io.stream.StreamOutput; import org.opensearch.common.util.iterable.Iterables; @@ -78,11 +75,11 @@ public class RoutingTable implements Iterable, Diffable indicesRouting; + private final Map indicesRouting; - private RoutingTable(long version, ImmutableOpenMap indicesRouting) { + private RoutingTable(long version, final Map indicesRouting) { this.version = version; - this.indicesRouting = indicesRouting; + this.indicesRouting = Collections.unmodifiableMap(indicesRouting); } /** @@ -112,7 +109,7 @@ public long version() { @Override public Iterator iterator() { - return indicesRouting.valuesIt(); + return indicesRouting.values().iterator(); } public boolean hasIndex(String index) { @@ -132,11 +129,11 @@ public IndexRoutingTable index(Index index) { return indicesRouting.get(index.getName()); } - public ImmutableOpenMap indicesRouting() { + public Map indicesRouting() { return indicesRouting; } - public ImmutableOpenMap getIndicesRouting() { + public Map getIndicesRouting() { return indicesRouting(); } @@ -214,7 +211,7 @@ public int shardsMatchingPredicateCount(Predicate predicate) { */ public List allShards() { List shards = new ArrayList<>(); - String[] indices = indicesRouting.keys().toArray(String.class); + String[] indices = indicesRouting.keySet().toArray(new String[0]); for (String index : indices) { List allShardsIndex = allShards(index); shards.addAll(allShardsIndex); @@ -379,8 +376,8 @@ public static RoutingTable readFrom(StreamInput in) throws IOException { public void writeTo(StreamOutput out) throws IOException { out.writeLong(version); out.writeVInt(indicesRouting.size()); - for (ObjectCursor index : indicesRouting.values()) { - index.value.writeTo(out); + for (final IndexRoutingTable index : indicesRouting.values()) { + index.writeTo(out); } } @@ -388,7 +385,7 @@ private static class RoutingTableDiff implements Diff { private final long version; - private final Diff> indicesRouting; + private final Diff> indicesRouting; RoutingTableDiff(RoutingTable before, RoutingTable after) { version = after.version; @@ -400,7 +397,7 @@ private static class RoutingTableDiff implements Diff { RoutingTableDiff(StreamInput in) throws IOException { version = in.readLong(); - indicesRouting = DiffableUtils.readImmutableOpenMapDiff(in, DiffableUtils.getStringKeySerializer(), DIFF_VALUE_READER); + indicesRouting = DiffableUtils.readJdkMapDiff(in, DiffableUtils.getStringKeySerializer(), DIFF_VALUE_READER); } @Override @@ -431,7 +428,7 @@ public static Builder builder(RoutingTable routingTable) { public static class Builder { private long version; - private ImmutableOpenMap.Builder indicesRouting = ImmutableOpenMap.builder(); + private Map indicesRouting = new HashMap<>(); public Builder() { @@ -626,7 +623,7 @@ public RoutingTable build() { if (indicesRouting == null) { throw new IllegalStateException("once build is called the builder cannot be reused"); } - RoutingTable table = new RoutingTable(version, indicesRouting.build()); + RoutingTable table = new RoutingTable(version, indicesRouting); indicesRouting = null; return table; } @@ -635,8 +632,8 @@ public RoutingTable build() { @Override public String toString() { StringBuilder sb = new StringBuilder("routing_table (version ").append(version).append("):\n"); - for (ObjectObjectCursor entry : indicesRouting) { - sb.append(entry.value.prettyPrint()).append('\n'); + for (final IndexRoutingTable entry : indicesRouting.values()) { + sb.append(entry.prettyPrint()).append('\n'); } return sb.toString(); } diff --git a/server/src/main/java/org/opensearch/cluster/routing/allocation/AllocationService.java b/server/src/main/java/org/opensearch/cluster/routing/allocation/AllocationService.java index 5169e63aeb9a5..a5e2175f05c51 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/allocation/AllocationService.java +++ b/server/src/main/java/org/opensearch/cluster/routing/allocation/AllocationService.java @@ -54,7 +54,6 @@ import org.opensearch.cluster.routing.allocation.command.AllocationCommands; import org.opensearch.cluster.routing.allocation.decider.AllocationDeciders; import org.opensearch.cluster.routing.allocation.decider.Decision; -import org.opensearch.common.collect.ImmutableOpenMap; import org.opensearch.gateway.GatewayAllocator; import org.opensearch.gateway.PriorityComparator; import org.opensearch.snapshots.SnapshotsInfoService; @@ -62,6 +61,7 @@ import java.util.ArrayList; import java.util.Collections; import java.util.Comparator; +import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; import java.util.List; @@ -183,9 +183,9 @@ private ClusterState buildResult(ClusterState oldState, RoutingAllocation alloca if (restoreInProgress != null) { RestoreInProgress updatedRestoreInProgress = allocation.updateRestoreInfoWithRoutingChanges(restoreInProgress); if (updatedRestoreInProgress != restoreInProgress) { - ImmutableOpenMap.Builder customsBuilder = ImmutableOpenMap.builder(allocation.getCustoms()); + final Map customsBuilder = new HashMap<>(allocation.getCustoms()); customsBuilder.put(RestoreInProgress.TYPE, updatedRestoreInProgress); - newStateBuilder.customs(customsBuilder.build()); + newStateBuilder.customs(customsBuilder); } } return newStateBuilder.build(); diff --git a/server/src/main/java/org/opensearch/cluster/routing/allocation/DiskThresholdMonitor.java b/server/src/main/java/org/opensearch/cluster/routing/allocation/DiskThresholdMonitor.java index 1dc586cf281b7..9f5402ba2ab91 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/allocation/DiskThresholdMonitor.java +++ b/server/src/main/java/org/opensearch/cluster/routing/allocation/DiskThresholdMonitor.java @@ -60,6 +60,7 @@ import java.util.List; import java.util.Map; import java.util.Set; +import java.util.Spliterators; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; import java.util.function.LongSupplier; @@ -351,8 +352,11 @@ public void onNewInfo(ClusterInfo info) { logger.trace("no reroute required"); listener.onResponse(null); } - final Set indicesToAutoRelease = StreamSupport.stream(state.routingTable().indicesRouting().spliterator(), false) - .map(c -> c.key) + final Set indicesToAutoRelease = StreamSupport.stream( + Spliterators.spliterator(state.routingTable().indicesRouting().entrySet(), 0), + false + ) + .map(c -> c.getKey()) .filter(index -> indicesNotToAutoRelease.contains(index) == false) .filter(index -> state.getBlocks().hasIndexBlock(index, IndexMetadata.INDEX_READ_ONLY_ALLOW_DELETE_BLOCK)) .collect(Collectors.toSet()); diff --git a/server/src/main/java/org/opensearch/cluster/routing/allocation/RoutingAllocation.java b/server/src/main/java/org/opensearch/cluster/routing/allocation/RoutingAllocation.java index 3ac22f5eee362..6dae12a592a6e 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/allocation/RoutingAllocation.java +++ b/server/src/main/java/org/opensearch/cluster/routing/allocation/RoutingAllocation.java @@ -43,7 +43,6 @@ import org.opensearch.cluster.routing.ShardRouting; import org.opensearch.cluster.routing.allocation.decider.AllocationDeciders; import org.opensearch.cluster.routing.allocation.decider.Decision; -import org.opensearch.common.collect.ImmutableOpenMap; import org.opensearch.index.shard.ShardId; import org.opensearch.snapshots.RestoreService.RestoreInProgressUpdater; import org.opensearch.snapshots.SnapshotShardSizeInfo; @@ -75,7 +74,7 @@ public class RoutingAllocation { private final DiscoveryNodes nodes; - private final ImmutableOpenMap customs; + private final Map customs; private final ClusterInfo clusterInfo; @@ -183,7 +182,7 @@ public T custom(String key) { return (T) customs.get(key); } - public ImmutableOpenMap getCustoms() { + public Map getCustoms() { return customs; } diff --git a/server/src/main/java/org/opensearch/cluster/routing/allocation/allocator/LocalShardsBalancer.java b/server/src/main/java/org/opensearch/cluster/routing/allocation/allocator/LocalShardsBalancer.java index 910c89f455c1c..80b4f720bd104 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/allocation/allocator/LocalShardsBalancer.java +++ b/server/src/main/java/org/opensearch/cluster/routing/allocation/allocator/LocalShardsBalancer.java @@ -471,7 +471,7 @@ private void balanceByWeights() { private String[] buildWeightOrderedIndices() { final List localIndices = new ArrayList<>(); - for (String index : allocation.routingTable().indicesRouting().keys().toArray(String.class)) { + for (String index : allocation.routingTable().indicesRouting().keySet().toArray(new String[0])) { if (RoutingPool.LOCAL_ONLY.equals(RoutingPool.getIndexPool(metadata.index(index)))) { localIndices.add(index); } diff --git a/server/src/main/java/org/opensearch/rest/action/admin/indices/RestGetAliasesAction.java b/server/src/main/java/org/opensearch/rest/action/admin/indices/RestGetAliasesAction.java index 6afc0f425992b..214bc4ba418dd 100644 --- a/server/src/main/java/org/opensearch/rest/action/admin/indices/RestGetAliasesAction.java +++ b/server/src/main/java/org/opensearch/rest/action/admin/indices/RestGetAliasesAction.java @@ -32,7 +32,6 @@ package org.opensearch.rest.action.admin.indices; -import com.carrotsearch.hppc.cursors.ObjectObjectCursor; import org.opensearch.action.admin.indices.alias.get.GetAliasesRequest; import org.opensearch.action.admin.indices.alias.get.GetAliasesResponse; import org.opensearch.action.support.IndicesOptions; @@ -40,7 +39,6 @@ import org.opensearch.cluster.metadata.AliasMetadata; import org.opensearch.cluster.metadata.Metadata; import org.opensearch.common.Strings; -import org.opensearch.common.collect.ImmutableOpenMap; import org.opensearch.common.regex.Regex; import org.opensearch.core.xcontent.ToXContent; import org.opensearch.core.xcontent.XContentBuilder; @@ -55,6 +53,7 @@ import java.util.HashSet; import java.util.List; import java.util.Locale; +import java.util.Map; import java.util.Set; import java.util.SortedSet; import java.util.TreeSet; @@ -95,16 +94,16 @@ public String getName() { static RestResponse buildRestResponse( boolean aliasesExplicitlyRequested, String[] requestedAliases, - ImmutableOpenMap> responseAliasMap, + final Map> responseAliasMap, XContentBuilder builder ) throws Exception { final Set indicesToDisplay = new HashSet<>(); final Set returnedAliasNames = new HashSet<>(); - for (final ObjectObjectCursor> cursor : responseAliasMap) { - for (final AliasMetadata aliasMetadata : cursor.value) { + for (final Map.Entry> cursor : responseAliasMap.entrySet()) { + for (final AliasMetadata aliasMetadata : cursor.getValue()) { if (aliasesExplicitlyRequested) { // only display indices that have aliases - indicesToDisplay.add(cursor.key); + indicesToDisplay.add(cursor.getKey()); } returnedAliasNames.add(aliasMetadata.alias()); } @@ -165,13 +164,13 @@ static RestResponse buildRestResponse( builder.field("status", status.getStatus()); } - for (final ObjectObjectCursor> entry : responseAliasMap) { - if (aliasesExplicitlyRequested == false || (aliasesExplicitlyRequested && indicesToDisplay.contains(entry.key))) { - builder.startObject(entry.key); + for (final Map.Entry> entry : responseAliasMap.entrySet()) { + if (aliasesExplicitlyRequested == false || (aliasesExplicitlyRequested && indicesToDisplay.contains(entry.getKey()))) { + builder.startObject(entry.getKey()); { builder.startObject("aliases"); { - for (final AliasMetadata alias : entry.value) { + for (final AliasMetadata alias : entry.getValue()) { AliasMetadata.Builder.toXContent(alias, builder, ToXContent.EMPTY_PARAMS); } } diff --git a/server/src/main/java/org/opensearch/rest/action/cat/RestAliasAction.java b/server/src/main/java/org/opensearch/rest/action/cat/RestAliasAction.java index 7aa91c175dbf0..6df5dc268d666 100644 --- a/server/src/main/java/org/opensearch/rest/action/cat/RestAliasAction.java +++ b/server/src/main/java/org/opensearch/rest/action/cat/RestAliasAction.java @@ -31,7 +31,6 @@ package org.opensearch.rest.action.cat; -import com.carrotsearch.hppc.cursors.ObjectObjectCursor; import org.opensearch.action.admin.indices.alias.get.GetAliasesRequest; import org.opensearch.action.admin.indices.alias.get.GetAliasesResponse; import org.opensearch.action.support.IndicesOptions; @@ -44,6 +43,7 @@ import org.opensearch.rest.action.RestResponseListener; import java.util.List; +import java.util.Map; import static java.util.Arrays.asList; import static java.util.Collections.unmodifiableList; @@ -111,9 +111,9 @@ protected Table getTableWithHeader(RestRequest request) { private Table buildTable(RestRequest request, GetAliasesResponse response) { Table table = getTableWithHeader(request); - for (ObjectObjectCursor> cursor : response.getAliases()) { - String indexName = cursor.key; - for (AliasMetadata aliasMetadata : cursor.value) { + for (final Map.Entry> cursor : response.getAliases().entrySet()) { + String indexName = cursor.getKey(); + for (AliasMetadata aliasMetadata : cursor.getValue()) { table.startRow(); table.addCell(aliasMetadata.alias()); table.addCell(indexName); diff --git a/server/src/main/java/org/opensearch/snapshots/InFlightShardSnapshotStates.java b/server/src/main/java/org/opensearch/snapshots/InFlightShardSnapshotStates.java index 17d77fcc0695b..f31d8c74074a3 100644 --- a/server/src/main/java/org/opensearch/snapshots/InFlightShardSnapshotStates.java +++ b/server/src/main/java/org/opensearch/snapshots/InFlightShardSnapshotStates.java @@ -32,7 +32,6 @@ package org.opensearch.snapshots; -import com.carrotsearch.hppc.cursors.ObjectObjectCursor; import org.opensearch.cluster.SnapshotsInProgress; import org.opensearch.common.Nullable; import org.opensearch.index.shard.ShardId; @@ -73,14 +72,15 @@ public static InFlightShardSnapshotStates forRepo(String repoName, List clone : runningSnapshot.clones()) { - final RepositoryShardId repoShardId = clone.key; - addStateInformation(generations, busyIds, clone.value, repoShardId.shardId(), repoShardId.indexName()); + for (final Map.Entry clone : runningSnapshot.clones() + .entrySet()) { + final RepositoryShardId repoShardId = clone.getKey(); + addStateInformation(generations, busyIds, clone.getValue(), repoShardId.shardId(), repoShardId.indexName()); } } else { - for (ObjectObjectCursor shard : runningSnapshot.shards()) { - final ShardId sid = shard.key; - addStateInformation(generations, busyIds, shard.value, sid.id(), sid.getIndexName()); + for (final Map.Entry shard : runningSnapshot.shards().entrySet()) { + final ShardId sid = shard.getKey(); + addStateInformation(generations, busyIds, shard.getValue(), sid.id(), sid.getIndexName()); } } } diff --git a/server/src/main/java/org/opensearch/snapshots/RestoreService.java b/server/src/main/java/org/opensearch/snapshots/RestoreService.java index d6dafc32f4e63..3fc037d7e8fc9 100644 --- a/server/src/main/java/org/opensearch/snapshots/RestoreService.java +++ b/server/src/main/java/org/opensearch/snapshots/RestoreService.java @@ -34,7 +34,6 @@ import com.carrotsearch.hppc.IntHashSet; import com.carrotsearch.hppc.IntSet; import com.carrotsearch.hppc.cursors.ObjectCursor; -import com.carrotsearch.hppc.cursors.ObjectObjectCursor; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; @@ -79,7 +78,6 @@ import org.opensearch.cluster.service.ClusterService; import org.opensearch.common.Priority; import org.opensearch.common.UUIDs; -import org.opensearch.common.collect.ImmutableOpenMap; import org.opensearch.common.lucene.Lucene; import org.opensearch.common.regex.Regex; import org.opensearch.common.settings.ClusterSettings; @@ -426,13 +424,12 @@ public ClusterState execute(ClusterState currentState) { Metadata.Builder mdBuilder = Metadata.builder(currentState.metadata()); ClusterBlocks.Builder blocks = ClusterBlocks.builder().blocks(currentState.blocks()); RoutingTable.Builder rtBuilder = RoutingTable.builder(currentState.routingTable()); - ImmutableOpenMap shards; + final Map shards; Set aliases = new HashSet<>(); if (indices.isEmpty() == false) { // We have some indices to restore - ImmutableOpenMap.Builder shardsBuilder = ImmutableOpenMap - .builder(); + Map shardsBuilder = new HashMap<>(); for (Map.Entry indexEntry : indices.entrySet()) { String renamedIndexName = indexEntry.getKey(); @@ -592,7 +589,7 @@ public ClusterState execute(ClusterState currentState) { } } - shards = shardsBuilder.build(); + shards = Collections.unmodifiableMap(shardsBuilder); RestoreInProgress.Entry restoreEntry = new RestoreInProgress.Entry( restoreUUID, snapshot, @@ -607,7 +604,7 @@ public ClusterState execute(ClusterState currentState) { ).build() ); } else { - shards = ImmutableOpenMap.of(); + shards = Map.of(); } checkAliasNameConflicts(indices, aliases); @@ -853,19 +850,19 @@ public static RestoreInProgress updateRestoreStateWithDeletedIndices(RestoreInPr boolean changesMade = false; RestoreInProgress.Builder builder = new RestoreInProgress.Builder(); for (RestoreInProgress.Entry entry : oldRestore) { - ImmutableOpenMap.Builder shardsBuilder = null; - for (ObjectObjectCursor cursor : entry.shards()) { - ShardId shardId = cursor.key; + Map shardsBuilder = null; + for (final Map.Entry cursor : entry.shards().entrySet()) { + ShardId shardId = cursor.getKey(); if (deletedIndices.contains(shardId.getIndex())) { changesMade = true; if (shardsBuilder == null) { - shardsBuilder = ImmutableOpenMap.builder(entry.shards()); + shardsBuilder = new HashMap<>(entry.shards()); } shardsBuilder.put(shardId, new ShardRestoreStatus(null, RestoreInProgress.State.FAILURE, "index was deleted")); } } if (shardsBuilder != null) { - ImmutableOpenMap shards = shardsBuilder.build(); + final Map shards = Collections.unmodifiableMap(shardsBuilder); builder.add( new RestoreInProgress.Entry( entry.uuid(), @@ -1004,9 +1001,9 @@ public RestoreInProgress applyChanges(final RestoreInProgress oldRestore) { RestoreInProgress.Builder builder = new RestoreInProgress.Builder(); for (RestoreInProgress.Entry entry : oldRestore) { Map updates = shardChanges.get(entry.uuid()); - ImmutableOpenMap shardStates = entry.shards(); + final Map shardStates = entry.shards(); if (updates != null && updates.isEmpty() == false) { - ImmutableOpenMap.Builder shardsBuilder = ImmutableOpenMap.builder(shardStates); + Map shardsBuilder = new HashMap<>(shardStates); for (Map.Entry shard : updates.entrySet()) { ShardId shardId = shard.getKey(); ShardRestoreStatus status = shardStates.get(shardId); @@ -1015,7 +1012,7 @@ public RestoreInProgress applyChanges(final RestoreInProgress oldRestore) { } } - ImmutableOpenMap shards = shardsBuilder.build(); + final Map shards = Collections.unmodifiableMap(shardsBuilder); RestoreInProgress.State newState = overallState(RestoreInProgress.State.STARTED, shards); builder.add(new RestoreInProgress.Entry(entry.uuid(), entry.snapshot(), newState, entry.indices(), shards)); } else { @@ -1067,9 +1064,9 @@ public ClusterTasksResult execute(final ClusterState currentState, final L if (changed == false) { return resultBuilder.build(currentState); } - ImmutableOpenMap.Builder builder = ImmutableOpenMap.builder(currentState.getCustoms()); + final Map builder = new HashMap<>(currentState.getCustoms()); builder.put(RestoreInProgress.TYPE, restoreInProgressBuilder.build()); - ImmutableOpenMap customs = builder.build(); + final Map customs = Collections.unmodifiableMap(builder); return resultBuilder.build(ClusterState.builder(currentState).customs(customs).build()); } @@ -1102,14 +1099,14 @@ private void cleanupRestoreState(ClusterChangedEvent event) { private static RestoreInProgress.State overallState( RestoreInProgress.State nonCompletedState, - ImmutableOpenMap shards + final Map shards ) { boolean hasFailed = false; - for (ObjectCursor status : shards.values()) { - if (!status.value.state().completed()) { + for (RestoreInProgress.ShardRestoreStatus status : shards.values()) { + if (status.state().completed() == false) { return nonCompletedState; } - if (status.value.state() == RestoreInProgress.State.FAILURE) { + if (status.state() == RestoreInProgress.State.FAILURE) { hasFailed = true; } } @@ -1120,19 +1117,19 @@ private static RestoreInProgress.State overallState( } } - public static boolean completed(ImmutableOpenMap shards) { - for (ObjectCursor status : shards.values()) { - if (!status.value.state().completed()) { + public static boolean completed(final Map shards) { + for (final RestoreInProgress.ShardRestoreStatus status : shards.values()) { + if (status.state().completed() == false) { return false; } } return true; } - public static int failedShards(ImmutableOpenMap shards) { + public static int failedShards(final Map shards) { int failedShards = 0; - for (ObjectCursor status : shards.values()) { - if (status.value.state() == RestoreInProgress.State.FAILURE) { + for (final RestoreInProgress.ShardRestoreStatus status : shards.values()) { + if (status.state() == RestoreInProgress.State.FAILURE) { failedShards++; } } @@ -1214,10 +1211,10 @@ public static boolean failed(SnapshotInfo snapshot, String index) { public static Set restoringIndices(final ClusterState currentState, final Set indicesToCheck) { final Set indices = new HashSet<>(); for (RestoreInProgress.Entry entry : currentState.custom(RestoreInProgress.TYPE, RestoreInProgress.EMPTY)) { - for (ObjectObjectCursor shard : entry.shards()) { - Index index = shard.key.getIndex(); + for (final Map.Entry shard : entry.shards().entrySet()) { + Index index = shard.getKey().getIndex(); if (indicesToCheck.contains(index) - && shard.value.state().completed() == false + && shard.getValue().state().completed() == false && currentState.getMetadata().index(index) != null) { indices.add(index); } diff --git a/server/src/main/java/org/opensearch/snapshots/SnapshotShardsService.java b/server/src/main/java/org/opensearch/snapshots/SnapshotShardsService.java index 4c37af46bd101..48589736166e8 100644 --- a/server/src/main/java/org/opensearch/snapshots/SnapshotShardsService.java +++ b/server/src/main/java/org/opensearch/snapshots/SnapshotShardsService.java @@ -32,7 +32,6 @@ package org.opensearch.snapshots; -import com.carrotsearch.hppc.cursors.ObjectObjectCursor; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; @@ -48,7 +47,6 @@ import org.opensearch.cluster.node.DiscoveryNode; import org.opensearch.cluster.service.ClusterService; import org.opensearch.common.Nullable; -import org.opensearch.common.collect.ImmutableOpenMap; import org.opensearch.common.component.AbstractLifecycleComponent; import org.opensearch.common.concurrent.GatedCloseable; import org.opensearch.common.io.stream.StreamInput; @@ -227,10 +225,10 @@ private void startNewSnapshots(SnapshotsInProgress snapshotsInProgress) { Map startedShards = null; final Snapshot snapshot = entry.snapshot(); Map snapshotShards = shardSnapshots.getOrDefault(snapshot, emptyMap()); - for (ObjectObjectCursor shard : entry.shards()) { + for (final Map.Entry shard : entry.shards().entrySet()) { // Add all new shards to start processing on - final ShardId shardId = shard.key; - final ShardSnapshotStatus shardSnapshotStatus = shard.value; + final ShardId shardId = shard.getKey(); + final ShardSnapshotStatus shardSnapshotStatus = shard.getValue(); if (shardSnapshotStatus.state() == ShardState.INIT && localNodeId.equals(shardSnapshotStatus.nodeId()) && snapshotShards.containsKey(shardId) == false) { @@ -249,13 +247,13 @@ private void startNewSnapshots(SnapshotsInProgress snapshotsInProgress) { // Abort all running shards for this snapshot final Snapshot snapshot = entry.snapshot(); Map snapshotShards = shardSnapshots.getOrDefault(snapshot, emptyMap()); - for (ObjectObjectCursor shard : entry.shards()) { - final IndexShardSnapshotStatus snapshotStatus = snapshotShards.get(shard.key); + for (final Map.Entry shard : entry.shards().entrySet()) { + final IndexShardSnapshotStatus snapshotStatus = snapshotShards.get(shard.getKey()); if (snapshotStatus == null) { // due to CS batching we might have missed the INIT state and straight went into ABORTED // notify cluster-manager that abort has completed by moving to FAILED - if (shard.value.state() == ShardState.ABORTED && localNodeId.equals(shard.value.nodeId())) { - notifyFailedSnapshotShard(snapshot, shard.key, shard.value.reason()); + if (shard.getValue().state() == ShardState.ABORTED && localNodeId.equals(shard.getValue().nodeId())) { + notifyFailedSnapshotShard(snapshot, shard.getKey(), shard.getValue().reason()); } } else { snapshotStatus.abortIfNotCompleted("snapshot has been aborted"); @@ -453,7 +451,7 @@ private void syncShardStatsOnNewMaster(ClusterChangedEvent event) { if (snapshot.state() == State.STARTED || snapshot.state() == State.ABORTED) { Map localShards = currentSnapshotShards(snapshot.snapshot()); if (localShards != null) { - ImmutableOpenMap masterShards = snapshot.shards(); + final Map masterShards = snapshot.shards(); for (Map.Entry localShard : localShards.entrySet()) { ShardId shardId = localShard.getKey(); ShardSnapshotStatus masterShard = masterShards.get(shardId); diff --git a/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java b/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java index 645775c3ec09c..51ca07adddd5a 100644 --- a/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java +++ b/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java @@ -32,8 +32,6 @@ package org.opensearch.snapshots; -import com.carrotsearch.hppc.cursors.ObjectCursor; -import com.carrotsearch.hppc.cursors.ObjectObjectCursor; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; @@ -83,7 +81,6 @@ import org.opensearch.common.Priority; import org.opensearch.common.Strings; import org.opensearch.common.UUIDs; -import org.opensearch.common.collect.ImmutableOpenMap; import org.opensearch.common.collect.Tuple; import org.opensearch.common.component.AbstractLifecycleComponent; import org.opensearch.common.io.stream.StreamInput; @@ -314,7 +311,7 @@ public ClusterState execute(ClusterState currentState) { getInFlightIndexIds(runningSnapshots, repositoryName) ); final Version version = minCompatibleVersion(currentState.nodes().getMinNodeVersion(), repositoryData, null); - ImmutableOpenMap shards = shards( + final Map shards = shards( snapshots, deletionsInProgress, currentState.metadata(), @@ -325,9 +322,9 @@ public ClusterState execute(ClusterState currentState) { ); if (request.partial() == false) { Set missing = new HashSet<>(); - for (ObjectObjectCursor entry : shards) { - if (entry.value.state() == ShardState.MISSING) { - missing.add(entry.key.getIndex().getName()); + for (final Map.Entry entry : shards.entrySet()) { + if (entry.getValue().state() == ShardState.MISSING) { + missing.add(entry.getKey().getIndex().getName()); } } if (missing.isEmpty() == false) { @@ -597,7 +594,7 @@ public ClusterState execute(ClusterState currentState) { final ShardGenerations shardGenerations = repoData.shardGenerations(); for (int i = 0; i < updatedEntries.size(); i++) { if (cloneEntry.snapshot().equals(updatedEntries.get(i).snapshot())) { - final ImmutableOpenMap.Builder clonesBuilder = ImmutableOpenMap.builder(); + final Map clonesBuilder = new HashMap<>(); final InFlightShardSnapshotStates inFlightShardStates = InFlightShardSnapshotStates.forRepo( repoName, snapshotsInProgress.entries() @@ -619,7 +616,7 @@ public ClusterState execute(ClusterState currentState) { } } } - updatedEntry = cloneEntry.withClones(clonesBuilder.build()); + updatedEntry = cloneEntry.withClones(clonesBuilder); updatedEntries.set(i, updatedEntry); changed = true; break; @@ -641,12 +638,12 @@ public void clusterStateProcessed(String source, ClusterState oldState, ClusterS if (updatedEntry != null) { final Snapshot target = updatedEntry.snapshot(); final SnapshotId sourceSnapshot = updatedEntry.source(); - for (ObjectObjectCursor indexClone : updatedEntry.clones()) { - final ShardSnapshotStatus shardStatusBefore = indexClone.value; + for (final Map.Entry indexClone : updatedEntry.clones().entrySet()) { + final ShardSnapshotStatus shardStatusBefore = indexClone.getValue(); if (shardStatusBefore.state() != ShardState.INIT) { continue; } - final RepositoryShardId repoShardId = indexClone.key; + final RepositoryShardId repoShardId = indexClone.getKey(); runReadyClone(target, sourceSnapshot, shardStatusBefore, repoShardId, repository); } } else { @@ -810,21 +807,19 @@ private static ShardGenerations buildGenerations(SnapshotsInProgress.Entry snaps final Map 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()); - builder.put(indexId, c.key.shardId(), c.value.generation()); + snapshot.clones().forEach((id, status) -> { + final IndexId indexId = indexLookup.get(id.indexName()); + builder.put(indexId, id.shardId(), status.generation()); }); } else { - snapshot.shards().forEach(c -> { - if (metadata.index(c.key.getIndex()) == null) { - assert snapshot.partial() : "Index [" - + c.key.getIndex() - + "] was deleted during a snapshot but snapshot was not partial."; + snapshot.shards().forEach((id, status) -> { + if (metadata.index(id.getIndex()) == null) { + assert snapshot.partial() : "Index [" + id.getIndex() + "] was deleted during a snapshot but snapshot was not partial."; return; } - final IndexId indexId = indexLookup.get(c.key.getIndexName()); + final IndexId indexId = indexLookup.get(id.getIndexName()); if (indexId != null) { - builder.put(indexId, c.key.id(), c.value.generation()); + builder.put(indexId, id.id(), status.generation()); } }); } @@ -1003,8 +998,8 @@ private static boolean assertNoDanglingSnapshots(ClusterState state) { final Set reposSeen = new HashSet<>(); for (SnapshotsInProgress.Entry entry : snapshotsInProgress.entries()) { if (reposSeen.add(entry.repository())) { - for (ObjectCursor value : entry.shards().values()) { - if (value.value.equals(ShardSnapshotStatus.UNASSIGNED_QUEUED)) { + for (final ShardSnapshotStatus status : entry.shards().values()) { + if (status.equals(ShardSnapshotStatus.UNASSIGNED_QUEUED)) { assert reposWithRunningDelete.contains(entry.repository()) : "Found shard snapshot waiting to be assigned in [" + entry + "] but it is not blocked by any running delete"; @@ -1073,7 +1068,7 @@ public ClusterState execute(ClusterState currentState) { changed = true; } } else { - ImmutableOpenMap shards = processWaitingShardsAndRemovedNodes( + final Map shards = processWaitingShardsAndRemovedNodes( snapshot.shards(), routingTable, nodes, @@ -1168,17 +1163,17 @@ public void clusterStateProcessed(String source, ClusterState oldState, ClusterS ); } - private static ImmutableOpenMap processWaitingShardsAndRemovedNodes( - ImmutableOpenMap snapshotShards, + private static Map processWaitingShardsAndRemovedNodes( + final Map snapshotShards, RoutingTable routingTable, DiscoveryNodes nodes, Map knownFailures ) { boolean snapshotChanged = false; - ImmutableOpenMap.Builder shards = ImmutableOpenMap.builder(); - for (ObjectObjectCursor shardEntry : snapshotShards) { - ShardSnapshotStatus shardStatus = shardEntry.value; - ShardId shardId = shardEntry.key; + final Map shards = new HashMap<>(); + for (final Map.Entry shardEntry : snapshotShards.entrySet()) { + ShardSnapshotStatus shardStatus = shardEntry.getValue(); + ShardId shardId = shardEntry.getKey(); if (shardStatus.equals(ShardSnapshotStatus.UNASSIGNED_QUEUED)) { // this shard snapshot is waiting for a previous snapshot to finish execution for this shard final ShardSnapshotStatus knownFailure = knownFailures.get(shardId); @@ -1244,7 +1239,7 @@ private static ImmutableOpenMap processWaitingShar } } if (snapshotChanged) { - return shards.build(); + return Collections.unmodifiableMap(shards); } else { return null; } @@ -1253,11 +1248,11 @@ private static ImmutableOpenMap processWaitingShar private static boolean waitingShardsStartedOrUnassigned(SnapshotsInProgress snapshotsInProgress, ClusterChangedEvent event) { for (SnapshotsInProgress.Entry entry : snapshotsInProgress.entries()) { if (entry.state() == State.STARTED) { - for (ObjectObjectCursor shardStatus : entry.shards()) { - if (shardStatus.value.state() != ShardState.WAITING) { + for (final Map.Entry shardStatus : entry.shards().entrySet()) { + if (shardStatus.getValue().state() != ShardState.WAITING) { continue; } - final ShardId shardId = shardStatus.key; + final ShardId shardId = shardStatus.getKey(); if (event.indexRoutingTableChanged(shardId.getIndexName())) { IndexRoutingTable indexShardRoutingTable = event.state().getRoutingTable().index(shardId.getIndex()); if (indexShardRoutingTable == null) { @@ -1286,8 +1281,7 @@ private static boolean removedNodesCleanupNeeded(SnapshotsInProgress snapshotsIn // nothing to do for already completed snapshots return false; } - for (ObjectCursor shardStatus : snapshot.shards().values()) { - final ShardSnapshotStatus shardSnapshotStatus = shardStatus.value; + for (final ShardSnapshotStatus shardSnapshotStatus : snapshot.shards().values()) { if (shardSnapshotStatus.state().completed() == false && removedNodeIds.contains(shardSnapshotStatus.nodeId())) { // Snapshot had an incomplete shard running on a removed node so we need to adjust that shard's snapshot status return true; @@ -1378,9 +1372,9 @@ private void finalizeSnapshotEntry(SnapshotsInProgress.Entry entry, Metadata met final Snapshot snapshot = entry.snapshot(); logger.trace("[{}] finalizing snapshot in repository, state: [{}], failure[{}]", snapshot, entry.state(), failure); ArrayList shardFailures = new ArrayList<>(); - for (ObjectObjectCursor shardStatus : entry.shards()) { - ShardId shardId = shardStatus.key; - ShardSnapshotStatus status = shardStatus.value; + for (final Map.Entry shardStatus : entry.shards().entrySet()) { + ShardId shardId = shardStatus.getKey(); + ShardSnapshotStatus status = shardStatus.getValue(); final ShardState state = status.state(); if (state.failed()) { shardFailures.add(new SnapshotShardFailure(status.nodeId(), shardId, status.reason())); @@ -2088,8 +2082,8 @@ private static boolean isWritingToRepository(SnapshotsInProgress.Entry entry) { // Entry is writing to the repo because it's finalizing on cluster-manager return true; } - for (ObjectCursor value : entry.shards().values()) { - if (value.value.isActive()) { + for (final ShardSnapshotStatus value : entry.shards().values()) { + if (value.isActive()) { // Entry is writing to the repo because it's writing to a shard on a data node or waiting to do so for a concrete shard return true; } @@ -2378,16 +2372,16 @@ private SnapshotsInProgress updatedSnapshotsInProgress(ClusterState currentState final String repoName = deleteEntry.repository(); // Computing the new assignments can be quite costly, only do it once below if actually needed - ImmutableOpenMap shardAssignments = null; + Map shardAssignments = null; for (SnapshotsInProgress.Entry entry : snapshotsInProgress.entries()) { if (entry.repository().equals(repoName)) { if (entry.state().completed() == false) { // Collect waiting shards that in entry that we can assign now that we are done with the deletion final List canBeUpdated = new ArrayList<>(); - for (ObjectObjectCursor value : entry.shards()) { - if (value.value.equals(ShardSnapshotStatus.UNASSIGNED_QUEUED) - && reassignedShardIds.contains(value.key) == false) { - canBeUpdated.add(value.key); + for (final Map.Entry value : entry.shards().entrySet()) { + if (value.getValue().equals(ShardSnapshotStatus.UNASSIGNED_QUEUED) + && reassignedShardIds.contains(value.getKey()) == false) { + canBeUpdated.add(value.getKey()); } } if (canBeUpdated.isEmpty()) { @@ -2405,8 +2399,7 @@ private SnapshotsInProgress updatedSnapshotsInProgress(ClusterState currentState repoName ); } - final ImmutableOpenMap.Builder updatedAssignmentsBuilder = ImmutableOpenMap - .builder(entry.shards()); + final Map updatedAssignmentsBuilder = new HashMap<>(entry.shards()); for (ShardId shardId : canBeUpdated) { final ShardSnapshotStatus updated = shardAssignments.get(shardId); if (updated == null) { @@ -2421,7 +2414,7 @@ private SnapshotsInProgress updatedSnapshotsInProgress(ClusterState currentState updatedAssignmentsBuilder.put(shardId, updated); } } - final SnapshotsInProgress.Entry updatedEntry = entry.withShardStates(updatedAssignmentsBuilder.build()); + final SnapshotsInProgress.Entry updatedEntry = entry.withShardStates(updatedAssignmentsBuilder); snapshotEntries.add(updatedEntry); changed = true; // When all the required shards for a snapshot are missing, the snapshot state will be "completed" @@ -2501,7 +2494,7 @@ private static void completeListenersIgnoringException(@Nullable List shards( + private static Map shards( SnapshotsInProgress snapshotsInProgress, @Nullable SnapshotDeletionsInProgress deletionsInProgress, Metadata metadata, @@ -2510,7 +2503,7 @@ private static ImmutableOpenMap builder = ImmutableOpenMap.builder(); + final Map builder = new HashMap<>(); final ShardGenerations shardGenerations = repositoryData.shardGenerations(); final InFlightShardSnapshotStates inFlightShardStates = InFlightShardSnapshotStates.forRepo( repoName, @@ -2579,7 +2572,7 @@ private static ImmutableOpenMap shards = null; - ImmutableOpenMap.Builder clones = null; + Map shards = null; + Map clones = null; Map indicesLookup = null; // inner loop over all the shard updates that are potentially applicable to the current snapshot entry for (Iterator iterator = unconsumedTasks.iterator(); iterator.hasNext();) { @@ -2763,7 +2756,7 @@ public ClusterManagerTaskThrottler.ThrottlingKey getClusterManagerThrottlingKey( updateSnapshotState.updatedState.state() ); if (clones == null) { - clones = ImmutableOpenMap.builder(entry.clones()); + clones = new HashMap<>(entry.clones()); } changedCount++; clones.put(finishedShardId, updateSnapshotState.updatedState); @@ -2778,7 +2771,7 @@ public ClusterManagerTaskThrottler.ThrottlingKey getClusterManagerThrottlingKey( continue; } if (clones == null) { - clones = ImmutableOpenMap.builder(entry.clones()); + clones = new HashMap<>(entry.clones()); } final ShardSnapshotStatus finishedStatus = updateSnapshotState.updatedState; logger.trace( @@ -2808,7 +2801,7 @@ public ClusterManagerTaskThrottler.ThrottlingKey getClusterManagerThrottlingKey( continue; } if (shards == null) { - shards = ImmutableOpenMap.builder(entry.shards()); + shards = new HashMap<>(entry.shards()); } final ShardSnapshotStatus finishedStatus = updateSnapshotState.updatedState; logger.trace( @@ -2858,7 +2851,7 @@ public ClusterManagerTaskThrottler.ThrottlingKey getClusterManagerThrottlingKey( updateSnapshotState.updatedState.state() ); if (shards == null) { - shards = ImmutableOpenMap.builder(entry.shards()); + shards = new HashMap(entry.shards()); } shards.put(finishedShardId, updateSnapshotState.updatedState); executedTasks.add(updateSnapshotState); @@ -2883,7 +2876,7 @@ public ClusterManagerTaskThrottler.ThrottlingKey getClusterManagerThrottlingKey( continue; } if (clones == null) { - clones = ImmutableOpenMap.builder(entry.clones()); + clones = new HashMap<>(entry.clones()); } final ShardSnapshotStatus finishedStatus = updateSnapshotState.updatedState; logger.trace( @@ -2903,7 +2896,7 @@ public ClusterManagerTaskThrottler.ThrottlingKey getClusterManagerThrottlingKey( continue; } if (shards == null) { - shards = ImmutableOpenMap.builder(entry.shards()); + shards = new HashMap<>(entry.shards()); } final ShardSnapshotStatus finishedStatus = updateSnapshotState.updatedState; logger.trace( @@ -2925,9 +2918,9 @@ public ClusterManagerTaskThrottler.ThrottlingKey getClusterManagerThrottlingKey( + clones + " as well as " + shards; - updatedEntry = entry.withShardStates(shards.build()); + updatedEntry = entry.withShardStates(shards); } else if (clones != null) { - updatedEntry = entry.withClones(clones.build()); + updatedEntry = entry.withClones(clones); } else { updatedEntry = entry; } @@ -3076,13 +3069,13 @@ private void startExecutableClones(SnapshotsInProgress snapshotsInProgress, @Nul for (SnapshotsInProgress.Entry entry : snapshotsInProgress.entries()) { if (entry.isClone() && entry.state() == State.STARTED && (repoName == null || entry.repository().equals(repoName))) { // this is a clone, see if new work is ready - for (ObjectObjectCursor clone : entry.clones()) { - if (clone.value.state() == ShardState.INIT) { + for (final Map.Entry clone : entry.clones().entrySet()) { + if (clone.getValue().state() == ShardState.INIT) { runReadyClone( entry.snapshot(), entry.source(), - clone.value, - clone.key, + clone.getValue(), + clone.getKey(), repositoriesService.repository(entry.repository()) ); } diff --git a/server/src/test/java/org/opensearch/action/admin/indices/alias/get/GetAliasesResponseTests.java b/server/src/test/java/org/opensearch/action/admin/indices/alias/get/GetAliasesResponseTests.java index 98a6aba1932d5..78c5e6dddcde6 100644 --- a/server/src/test/java/org/opensearch/action/admin/indices/alias/get/GetAliasesResponseTests.java +++ b/server/src/test/java/org/opensearch/action/admin/indices/alias/get/GetAliasesResponseTests.java @@ -34,14 +34,15 @@ import org.opensearch.cluster.metadata.AliasMetadata; import org.opensearch.cluster.metadata.AliasMetadata.Builder; -import org.opensearch.common.collect.ImmutableOpenMap; import org.opensearch.common.io.stream.Writeable; import org.opensearch.test.AbstractWireSerializingTestCase; import java.util.ArrayList; +import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; import java.util.List; +import java.util.Map; import java.util.Set; public class GetAliasesResponseTests extends AbstractWireSerializingTestCase { @@ -61,26 +62,26 @@ protected GetAliasesResponse mutateInstance(GetAliasesResponse response) { return new GetAliasesResponse(mutateAliases(response.getAliases())); } - private static ImmutableOpenMap> mutateAliases(ImmutableOpenMap> aliases) { + private static Map> mutateAliases(final Map> aliases) { if (aliases.isEmpty()) { - return createIndicesAliasesMap(1, 3).build(); + return createIndicesAliasesMap(1, 3); } if (randomBoolean()) { - ImmutableOpenMap.Builder> builder = ImmutableOpenMap.builder(aliases); - ImmutableOpenMap> list = createIndicesAliasesMap(1, 2).build(); - list.forEach(e -> builder.put(e.key, e.value)); - return builder.build(); + final Map> builder = new HashMap<>(aliases); + final Map> list = createIndicesAliasesMap(1, 2); + list.forEach((k, v) -> builder.put(k, v)); + return builder; } Set indices = new HashSet<>(); - Iterator keys = aliases.keysIt(); + Iterator keys = aliases.keySet().iterator(); while (keys.hasNext()) { indices.add(keys.next()); } List indicesToBeModified = randomSubsetOf(randomIntBetween(1, indices.size()), indices); - ImmutableOpenMap.Builder> builder = ImmutableOpenMap.builder(); + final Map> builder = new HashMap<>(); for (String index : indices) { List list = new ArrayList<>(aliases.get(index)); @@ -95,15 +96,15 @@ private static ImmutableOpenMap> mutateAliases(Immut } builder.put(index, list); } - return builder.build(); + return builder; } private static GetAliasesResponse createTestItem() { - return new GetAliasesResponse(createIndicesAliasesMap(0, 5).build()); + return new GetAliasesResponse(createIndicesAliasesMap(0, 5)); } - private static ImmutableOpenMap.Builder> createIndicesAliasesMap(int min, int max) { - ImmutableOpenMap.Builder> builder = ImmutableOpenMap.builder(); + private static Map> createIndicesAliasesMap(int min, int max) { + final Map> builder = new HashMap<>(); int indicesNum = randomIntBetween(min, max); for (int i = 0; i < indicesNum; i++) { String index = randomAlphaOfLength(5); diff --git a/server/src/test/java/org/opensearch/action/admin/indices/alias/get/TransportGetAliasesActionTests.java b/server/src/test/java/org/opensearch/action/admin/indices/alias/get/TransportGetAliasesActionTests.java index 46da611d9c2a7..0a223d7b6dc77 100644 --- a/server/src/test/java/org/opensearch/action/admin/indices/alias/get/TransportGetAliasesActionTests.java +++ b/server/src/test/java/org/opensearch/action/admin/indices/alias/get/TransportGetAliasesActionTests.java @@ -36,13 +36,13 @@ import org.opensearch.cluster.metadata.AliasMetadata; import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.cluster.metadata.Metadata; -import org.opensearch.common.collect.ImmutableOpenMap; import org.opensearch.indices.SystemIndexDescriptor; import org.opensearch.indices.SystemIndices; import org.opensearch.test.OpenSearchTestCase; import java.util.Collections; import java.util.List; +import java.util.Map; import static org.hamcrest.Matchers.equalTo; @@ -51,10 +51,8 @@ public class TransportGetAliasesActionTests extends OpenSearchTestCase { public void testPostProcess() { GetAliasesRequest request = new GetAliasesRequest(); - ImmutableOpenMap> aliases = ImmutableOpenMap.>builder() - .fPut("b", Collections.singletonList(new AliasMetadata.Builder("y").build())) - .build(); - ImmutableOpenMap> result = TransportGetAliasesAction.postProcess( + Map> aliases = Map.of("b", Collections.singletonList(new AliasMetadata.Builder("y").build())); + Map> result = TransportGetAliasesAction.postProcess( request, new String[] { "a", "b", "c" }, aliases, @@ -69,9 +67,7 @@ public void testPostProcess() { request = new GetAliasesRequest(); request.replaceAliases("y", "z"); - aliases = ImmutableOpenMap.>builder() - .fPut("b", Collections.singletonList(new AliasMetadata.Builder("y").build())) - .build(); + aliases = Map.of("b", Collections.singletonList(new AliasMetadata.Builder("y").build())); result = TransportGetAliasesAction.postProcess( request, new String[] { "a", "b", "c" }, @@ -86,9 +82,7 @@ public void testPostProcess() { assertThat(result.get("c").size(), equalTo(0)); request = new GetAliasesRequest("y", "z"); - aliases = ImmutableOpenMap.>builder() - .fPut("b", Collections.singletonList(new AliasMetadata.Builder("y").build())) - .build(); + aliases = Map.of("b", Collections.singletonList(new AliasMetadata.Builder("y").build())); result = TransportGetAliasesAction.postProcess( request, new String[] { "a", "b", "c" }, @@ -105,13 +99,15 @@ public void testDeprecationWarningEmittedForTotalWildcard() { ClusterState state = systemIndexTestClusterState(); GetAliasesRequest request = new GetAliasesRequest(); - ImmutableOpenMap> aliases = ImmutableOpenMap.>builder() - .fPut(".b", Collections.singletonList(new AliasMetadata.Builder(".y").build())) - .fPut("c", Collections.singletonList(new AliasMetadata.Builder("d").build())) - .build(); + final Map> aliases = Map.of( + ".b", + Collections.singletonList(new AliasMetadata.Builder(".y").build()), + "c", + Collections.singletonList(new AliasMetadata.Builder("d").build()) + ); final String[] concreteIndices = { "a", ".b", "c" }; assertEquals(state.metadata().findAliases(request, concreteIndices), aliases); - ImmutableOpenMap> result = TransportGetAliasesAction.postProcess( + final Map> result = TransportGetAliasesAction.postProcess( request, concreteIndices, aliases, @@ -134,12 +130,10 @@ public void testDeprecationWarningEmittedWhenSystemIndexIsRequested() { GetAliasesRequest request = new GetAliasesRequest(); request.indices(".b"); - ImmutableOpenMap> aliases = ImmutableOpenMap.>builder() - .fPut(".b", Collections.singletonList(new AliasMetadata.Builder(".y").build())) - .build(); + final Map> aliases = Map.of(".b", Collections.singletonList(new AliasMetadata.Builder(".y").build())); final String[] concreteIndices = { ".b" }; assertEquals(state.metadata().findAliases(request, concreteIndices), aliases); - ImmutableOpenMap> result = TransportGetAliasesAction.postProcess( + final Map> result = TransportGetAliasesAction.postProcess( request, concreteIndices, aliases, @@ -159,12 +153,10 @@ public void testDeprecationWarningEmittedWhenSystemIndexIsRequestedByAlias() { ClusterState state = systemIndexTestClusterState(); GetAliasesRequest request = new GetAliasesRequest(".y"); - ImmutableOpenMap> aliases = ImmutableOpenMap.>builder() - .fPut(".b", Collections.singletonList(new AliasMetadata.Builder(".y").build())) - .build(); + final Map> aliases = Map.of(".b", Collections.singletonList(new AliasMetadata.Builder(".y").build())); final String[] concreteIndices = { "a", ".b", "c" }; assertEquals(state.metadata().findAliases(request, concreteIndices), aliases); - ImmutableOpenMap> result = TransportGetAliasesAction.postProcess( + final Map> result = TransportGetAliasesAction.postProcess( request, concreteIndices, aliases, @@ -184,12 +176,10 @@ public void testDeprecationWarningNotEmittedWhenSystemAccessAllowed() { ClusterState state = systemIndexTestClusterState(); GetAliasesRequest request = new GetAliasesRequest(".y"); - ImmutableOpenMap> aliases = ImmutableOpenMap.>builder() - .fPut(".b", Collections.singletonList(new AliasMetadata.Builder(".y").build())) - .build(); + final Map> aliases = Map.of(".b", Collections.singletonList(new AliasMetadata.Builder(".y").build())); final String[] concreteIndices = { "a", ".b", "c" }; assertEquals(state.metadata().findAliases(request, concreteIndices), aliases); - ImmutableOpenMap> result = TransportGetAliasesAction.postProcess( + final Map> result = TransportGetAliasesAction.postProcess( request, concreteIndices, aliases, @@ -209,12 +199,10 @@ public void testDeprecationWarningNotEmittedWhenOnlyNonsystemIndexRequested() { GetAliasesRequest request = new GetAliasesRequest(); request.indices("c"); - ImmutableOpenMap> aliases = ImmutableOpenMap.>builder() - .fPut("c", Collections.singletonList(new AliasMetadata.Builder("d").build())) - .build(); + final Map> aliases = Map.of("c", Collections.singletonList(new AliasMetadata.Builder("d").build())); final String[] concreteIndices = { "c" }; assertEquals(state.metadata().findAliases(request, concreteIndices), aliases); - ImmutableOpenMap> result = TransportGetAliasesAction.postProcess( + final Map> result = TransportGetAliasesAction.postProcess( request, concreteIndices, aliases, @@ -236,10 +224,10 @@ public void testDeprecationWarningEmittedWhenRequestingNonExistingAliasInSystemP ); GetAliasesRequest request = new GetAliasesRequest(".y"); - ImmutableOpenMap> aliases = ImmutableOpenMap.>builder().build(); + final Map> aliases = Map.of(); final String[] concreteIndices = {}; assertEquals(state.metadata().findAliases(request, concreteIndices), aliases); - ImmutableOpenMap> result = TransportGetAliasesAction.postProcess( + final Map> result = TransportGetAliasesAction.postProcess( request, concreteIndices, aliases, diff --git a/server/src/test/java/org/opensearch/action/admin/indices/datastream/DeleteDataStreamRequestTests.java b/server/src/test/java/org/opensearch/action/admin/indices/datastream/DeleteDataStreamRequestTests.java index e33b873a52f19..c4fd5c553f069 100644 --- a/server/src/test/java/org/opensearch/action/admin/indices/datastream/DeleteDataStreamRequestTests.java +++ b/server/src/test/java/org/opensearch/action/admin/indices/datastream/DeleteDataStreamRequestTests.java @@ -42,7 +42,6 @@ import org.opensearch.cluster.metadata.Metadata; import org.opensearch.cluster.metadata.MetadataDeleteIndexService; import org.opensearch.common.Strings; -import org.opensearch.common.collect.ImmutableOpenMap; import org.opensearch.common.collect.Tuple; import org.opensearch.common.io.stream.Writeable; import org.opensearch.common.settings.Settings; @@ -56,6 +55,7 @@ import java.util.Arrays; import java.util.Collections; import java.util.List; +import java.util.Map; import java.util.Set; import java.util.stream.Collectors; @@ -168,7 +168,7 @@ private SnapshotsInProgress.Entry createEntry(String dataStreamName, String repo Collections.singletonList(dataStreamName), 0, 1, - ImmutableOpenMap.of(), + Map.of(), null, null, null diff --git a/server/src/test/java/org/opensearch/action/admin/indices/get/GetIndexResponseTests.java b/server/src/test/java/org/opensearch/action/admin/indices/get/GetIndexResponseTests.java index 225c435ee1106..94e20af76103e 100644 --- a/server/src/test/java/org/opensearch/action/admin/indices/get/GetIndexResponseTests.java +++ b/server/src/test/java/org/opensearch/action/admin/indices/get/GetIndexResponseTests.java @@ -63,7 +63,7 @@ protected Writeable.Reader instanceReader() { protected GetIndexResponse createTestInstance() { String[] indices = generateRandomStringArray(5, 5, false, false); final Map mappings = new HashMap<>(); - ImmutableOpenMap.Builder> aliases = ImmutableOpenMap.builder(); + final Map> aliases = new HashMap<>(); ImmutableOpenMap.Builder settings = ImmutableOpenMap.builder(); ImmutableOpenMap.Builder defaultSettings = ImmutableOpenMap.builder(); ImmutableOpenMap.Builder dataStreams = ImmutableOpenMap.builder(); @@ -92,6 +92,6 @@ protected GetIndexResponse createTestInstance() { dataStreams.put(index, randomAlphaOfLength(5).toLowerCase(Locale.ROOT)); } } - return new GetIndexResponse(indices, mappings, aliases.build(), settings.build(), defaultSettings.build(), dataStreams.build()); + return new GetIndexResponse(indices, mappings, aliases, settings.build(), defaultSettings.build(), dataStreams.build()); } } diff --git a/server/src/test/java/org/opensearch/cluster/ClusterStateTests.java b/server/src/test/java/org/opensearch/cluster/ClusterStateTests.java index e4c9c14e5f157..42a291ea5a257 100644 --- a/server/src/test/java/org/opensearch/cluster/ClusterStateTests.java +++ b/server/src/test/java/org/opensearch/cluster/ClusterStateTests.java @@ -50,7 +50,6 @@ import org.opensearch.cluster.routing.ShardRoutingState; import org.opensearch.cluster.routing.TestShardRouting; import org.opensearch.common.Strings; -import org.opensearch.common.collect.ImmutableOpenMap; import org.opensearch.common.settings.Settings; import org.opensearch.common.transport.TransportAddress; import org.opensearch.core.xcontent.ToXContent; @@ -66,6 +65,7 @@ import java.net.InetAddress; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collections; import java.util.EnumSet; import java.util.HashMap; import java.util.HashSet; @@ -125,9 +125,9 @@ public void testBuilderRejectsNullCustom() { public void testBuilderRejectsNullInCustoms() { final ClusterState.Builder builder = ClusterState.builder(ClusterName.DEFAULT); final String key = randomAlphaOfLength(10); - final ImmutableOpenMap.Builder mapBuilder = ImmutableOpenMap.builder(); + final Map mapBuilder = new HashMap<>(); mapBuilder.put(key, null); - final ImmutableOpenMap map = mapBuilder.build(); + final Map map = Collections.unmodifiableMap(mapBuilder); assertThat(expectThrows(NullPointerException.class, () -> builder.customs(map)).getMessage(), containsString(key)); } diff --git a/server/src/test/java/org/opensearch/cluster/block/ClusterBlockTests.java b/server/src/test/java/org/opensearch/cluster/block/ClusterBlockTests.java index af82e143faa37..a02af7556c0a2 100644 --- a/server/src/test/java/org/opensearch/cluster/block/ClusterBlockTests.java +++ b/server/src/test/java/org/opensearch/cluster/block/ClusterBlockTests.java @@ -34,7 +34,6 @@ import org.opensearch.Version; import org.opensearch.common.UUIDs; -import org.opensearch.common.collect.ImmutableOpenMap; import org.opensearch.common.io.stream.BytesStreamOutput; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.rest.RestStatus; @@ -43,6 +42,7 @@ import java.util.Arrays; import java.util.Collections; import java.util.List; +import java.util.Map; import static java.util.EnumSet.copyOf; import static org.opensearch.test.VersionUtils.randomVersion; @@ -80,7 +80,7 @@ public void testToStringDanglingComma() { public void testGlobalBlocksCheckedIfNoIndicesSpecified() { ClusterBlock globalBlock = randomClusterBlock(); - ClusterBlocks clusterBlocks = new ClusterBlocks(Collections.singleton(globalBlock), ImmutableOpenMap.of()); + ClusterBlocks clusterBlocks = new ClusterBlocks(Collections.singleton(globalBlock), Map.of()); ClusterBlockException exception = clusterBlocks.indicesBlockedException(randomFrom(globalBlock.levels()), new String[0]); assertNotNull(exception); assertEquals(exception.blocks(), Collections.singleton(globalBlock)); diff --git a/server/src/test/java/org/opensearch/cluster/metadata/MetadataDeleteIndexServiceTests.java b/server/src/test/java/org/opensearch/cluster/metadata/MetadataDeleteIndexServiceTests.java index dbff833cfee60..c309b90f1a777 100644 --- a/server/src/test/java/org/opensearch/cluster/metadata/MetadataDeleteIndexServiceTests.java +++ b/server/src/test/java/org/opensearch/cluster/metadata/MetadataDeleteIndexServiceTests.java @@ -40,7 +40,6 @@ import org.opensearch.cluster.routing.RoutingTable; import org.opensearch.cluster.routing.allocation.AllocationService; import org.opensearch.cluster.service.ClusterService; -import org.opensearch.common.collect.ImmutableOpenMap; import org.opensearch.common.collect.Tuple; import org.opensearch.common.settings.Settings; import org.opensearch.index.Index; @@ -58,6 +57,7 @@ import java.util.HashSet; import java.util.Collections; import java.util.List; +import java.util.Map; import java.util.Locale; import java.util.Set; import java.util.stream.Collectors; @@ -108,7 +108,7 @@ public void testDeleteSnapshotting() { Collections.emptyList(), System.currentTimeMillis(), (long) randomIntBetween(0, 1000), - ImmutableOpenMap.of(), + Map.of(), null, SnapshotInfoTests.randomUserMetadata(), VersionUtils.randomVersion(random()) diff --git a/server/src/test/java/org/opensearch/cluster/metadata/MetadataIndexStateServiceTests.java b/server/src/test/java/org/opensearch/cluster/metadata/MetadataIndexStateServiceTests.java index 70ad47634d2da..ffda58adcd019 100644 --- a/server/src/test/java/org/opensearch/cluster/metadata/MetadataIndexStateServiceTests.java +++ b/server/src/test/java/org/opensearch/cluster/metadata/MetadataIndexStateServiceTests.java @@ -52,7 +52,6 @@ import org.opensearch.cluster.service.ClusterService; import org.opensearch.common.Nullable; import org.opensearch.common.Strings; -import org.opensearch.common.collect.ImmutableOpenMap; import org.opensearch.common.collect.Tuple; import org.opensearch.common.settings.Settings; import org.opensearch.index.Index; @@ -403,7 +402,7 @@ private static ClusterState addBlockedIndex( private static ClusterState addRestoredIndex(final String index, final int numShards, final int numReplicas, final ClusterState state) { ClusterState newState = addOpenedIndex(index, numShards, numReplicas, state); - final ImmutableOpenMap.Builder shardsBuilder = ImmutableOpenMap.builder(); + final Map shardsBuilder = new HashMap<>(); for (ShardRouting shardRouting : newState.routingTable().index(index).randomAllActiveShardsIt()) { shardsBuilder.put(shardRouting.shardId(), new RestoreInProgress.ShardRestoreStatus(shardRouting.currentNodeId())); } @@ -414,7 +413,7 @@ private static ClusterState addRestoredIndex(final String index, final int numSh snapshot, RestoreInProgress.State.INIT, Collections.singletonList(index), - shardsBuilder.build() + shardsBuilder ); return ClusterState.builder(newState).putCustom(RestoreInProgress.TYPE, new RestoreInProgress.Builder().add(entry).build()).build(); } @@ -422,7 +421,7 @@ private static ClusterState addRestoredIndex(final String index, final int numSh private static ClusterState addSnapshotIndex(final String index, final int numShards, final int numReplicas, final ClusterState state) { ClusterState newState = addOpenedIndex(index, numShards, numReplicas, state); - final ImmutableOpenMap.Builder shardsBuilder = ImmutableOpenMap.builder(); + final Map shardsBuilder = new HashMap<>(); for (ShardRouting shardRouting : newState.routingTable().index(index).randomAllActiveShardsIt()) { shardsBuilder.put(shardRouting.shardId(), new SnapshotsInProgress.ShardSnapshotStatus(shardRouting.currentNodeId(), "1")); } @@ -437,7 +436,7 @@ private static ClusterState addSnapshotIndex(final String index, final int numSh Collections.emptyList(), randomNonNegativeLong(), randomLong(), - shardsBuilder.build(), + shardsBuilder, null, SnapshotInfoTests.randomUserMetadata(), VersionUtils.randomVersion(random()) diff --git a/server/src/test/java/org/opensearch/cluster/metadata/MetadataTests.java b/server/src/test/java/org/opensearch/cluster/metadata/MetadataTests.java index 8c2422c7dcf3b..d51613643647c 100644 --- a/server/src/test/java/org/opensearch/cluster/metadata/MetadataTests.java +++ b/server/src/test/java/org/opensearch/cluster/metadata/MetadataTests.java @@ -93,7 +93,7 @@ public void testFindAliases() { .build(); { - ImmutableOpenMap> aliases = metadata.findAliases(new GetAliasesRequest(), Strings.EMPTY_ARRAY); + final Map> aliases = metadata.findAliases(new GetAliasesRequest(), Strings.EMPTY_ARRAY); assertThat(aliases.size(), equalTo(0)); } { @@ -105,7 +105,7 @@ public void testFindAliases() { // replacing with empty aliases behaves as if aliases were unspecified at request building request.replaceAliases(Strings.EMPTY_ARRAY); } - ImmutableOpenMap> aliases = metadata.findAliases(new GetAliasesRequest(), new String[] { "index" }); + final Map> aliases = metadata.findAliases(new GetAliasesRequest(), new String[] { "index" }); assertThat(aliases.size(), equalTo(1)); List aliasMetadataList = aliases.get("index"); assertThat(aliasMetadataList.size(), equalTo(2)); @@ -113,7 +113,7 @@ public void testFindAliases() { assertThat(aliasMetadataList.get(1).alias(), equalTo("alias2")); } { - ImmutableOpenMap> aliases = metadata.findAliases( + final Map> aliases = metadata.findAliases( new GetAliasesRequest("alias*"), new String[] { "index" } ); @@ -124,7 +124,7 @@ public void testFindAliases() { assertThat(aliasMetadataList.get(1).alias(), equalTo("alias2")); } { - ImmutableOpenMap> aliases = metadata.findAliases( + final Map> aliases = metadata.findAliases( new GetAliasesRequest("alias1"), new String[] { "index" } ); @@ -134,7 +134,7 @@ public void testFindAliases() { assertThat(aliasMetadataList.get(0).alias(), equalTo("alias1")); } { - ImmutableOpenMap> aliases = metadata.findAllAliases(new String[] { "index" }); + final Map> aliases = metadata.findAllAliases(new String[] { "index" }); assertThat(aliases.size(), equalTo(1)); List aliasMetadataList = aliases.get("index"); assertThat(aliasMetadataList.size(), equalTo(2)); @@ -142,7 +142,7 @@ public void testFindAliases() { assertThat(aliasMetadataList.get(1).alias(), equalTo("alias2")); } { - ImmutableOpenMap> aliases = metadata.findAllAliases(Strings.EMPTY_ARRAY); + final Map> aliases = metadata.findAllAliases(Strings.EMPTY_ARRAY); assertThat(aliases.size(), equalTo(0)); } } diff --git a/server/src/test/java/org/opensearch/cluster/routing/allocation/AddIncrementallyTests.java b/server/src/test/java/org/opensearch/cluster/routing/allocation/AddIncrementallyTests.java index c2be6dfa60b51..16ff5f753d286 100644 --- a/server/src/test/java/org/opensearch/cluster/routing/allocation/AddIncrementallyTests.java +++ b/server/src/test/java/org/opensearch/cluster/routing/allocation/AddIncrementallyTests.java @@ -32,8 +32,6 @@ package org.opensearch.cluster.routing.allocation; -import com.carrotsearch.hppc.cursors.ObjectCursor; - import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.opensearch.Version; @@ -238,8 +236,8 @@ public void testMinimalRelocationsNoLimit() { } private void assertNumIndexShardsPerNode(ClusterState state, Matcher matcher) { - for (ObjectCursor index : state.routingTable().indicesRouting().keys()) { - assertNumIndexShardsPerNode(state, index.value, matcher); + for (final String index : state.routingTable().indicesRouting().keySet()) { + assertNumIndexShardsPerNode(state, index, matcher); } } @@ -250,10 +248,10 @@ private void assertNumIndexShardsPerNode(ClusterState state, String index, Match } private void assertAtLeastOneIndexShardPerNode(ClusterState state) { - for (ObjectCursor index : state.routingTable().indicesRouting().keys()) { + for (final String index : state.routingTable().indicesRouting().keySet()) { for (RoutingNode node : state.getRoutingNodes()) { - assertThat(node.shardsWithState(index.value, STARTED).size(), Matchers.greaterThanOrEqualTo(1)); + assertThat(node.shardsWithState(index, STARTED).size(), Matchers.greaterThanOrEqualTo(1)); } } diff --git a/server/src/test/java/org/opensearch/cluster/routing/allocation/BalanceConfigurationTests.java b/server/src/test/java/org/opensearch/cluster/routing/allocation/BalanceConfigurationTests.java index 99c0d01a3dcca..b1024b56d6cd3 100644 --- a/server/src/test/java/org/opensearch/cluster/routing/allocation/BalanceConfigurationTests.java +++ b/server/src/test/java/org/opensearch/cluster/routing/allocation/BalanceConfigurationTests.java @@ -32,8 +32,6 @@ package org.opensearch.cluster.routing.allocation; -import com.carrotsearch.hppc.cursors.ObjectCursor; -import com.carrotsearch.hppc.cursors.ObjectObjectCursor; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.apache.lucene.util.ArrayUtil; @@ -69,6 +67,7 @@ import java.util.ArrayList; import java.util.HashSet; import java.util.List; +import java.util.Map; import java.util.Set; import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; @@ -523,12 +522,12 @@ public void testPrimaryBalance_NotSolved_2() { private void verifyPerIndexPrimaryBalance(ClusterState currentState) { RoutingNodes nodes = currentState.getRoutingNodes(); - for (ObjectObjectCursor index : currentState.getRoutingTable().indicesRouting()) { - final int totalPrimaryShards = index.value.primaryShardsActive(); + for (final Map.Entry index : currentState.getRoutingTable().indicesRouting().entrySet()) { + final int totalPrimaryShards = index.getValue().primaryShardsActive(); final int avgPrimaryShardsPerNode = (int) Math.ceil(totalPrimaryShards * 1f / currentState.getRoutingNodes().size()); for (RoutingNode node : nodes) { - final int primaryCount = node.shardsWithState(index.key, STARTED) + final int primaryCount = node.shardsWithState(index.getKey(), STARTED) .stream() .filter(ShardRouting::primary) .collect(Collectors.toList()) @@ -542,8 +541,8 @@ private void verifyPrimaryBalance(ClusterState clusterState) throws Exception { assertBusy(() -> { RoutingNodes nodes = clusterState.getRoutingNodes(); int totalPrimaryShards = 0; - for (ObjectObjectCursor index : clusterState.getRoutingTable().indicesRouting()) { - totalPrimaryShards += index.value.primaryShardsActive(); + for (final IndexRoutingTable index : clusterState.getRoutingTable().indicesRouting().values()) { + totalPrimaryShards += index.primaryShardsActive(); } final int avgPrimaryShardsPerNode = (int) Math.ceil(totalPrimaryShards * 1f / clusterState.getRoutingNodes().size()); for (RoutingNode node : nodes) { @@ -774,10 +773,10 @@ private void assertIndexBalance( final int minAvgNumberOfShards = Math.round(Math.round(Math.floor(avgNumShards - threshold))); final int maxAvgNumberOfShards = Math.round(Math.round(Math.ceil(avgNumShards + threshold))); - for (ObjectCursor index : routingTable.indicesRouting().keys()) { + for (final String index : routingTable.indicesRouting().keySet()) { for (RoutingNode node : nodes) { - assertThat(node.shardsWithState(index.value, STARTED).size(), Matchers.greaterThanOrEqualTo(minAvgNumberOfShards)); - assertThat(node.shardsWithState(index.value, STARTED).size(), Matchers.lessThanOrEqualTo(maxAvgNumberOfShards)); + assertThat(node.shardsWithState(index, STARTED).size(), Matchers.greaterThanOrEqualTo(minAvgNumberOfShards)); + assertThat(node.shardsWithState(index, STARTED).size(), Matchers.lessThanOrEqualTo(maxAvgNumberOfShards)); } } } diff --git a/server/src/test/java/org/opensearch/cluster/routing/allocation/ThrottlingAllocationTests.java b/server/src/test/java/org/opensearch/cluster/routing/allocation/ThrottlingAllocationTests.java index ab6042513af94..0c76b45fc2532 100644 --- a/server/src/test/java/org/opensearch/cluster/routing/allocation/ThrottlingAllocationTests.java +++ b/server/src/test/java/org/opensearch/cluster/routing/allocation/ThrottlingAllocationTests.java @@ -69,7 +69,9 @@ import java.util.ArrayList; import java.util.Collections; +import java.util.HashMap; import java.util.HashSet; +import java.util.Map; import java.util.Set; import java.util.stream.Collectors; @@ -828,10 +830,10 @@ private ClusterState createRecoveryStateAndInitializeAllocations( final RoutingTable routingTable = routingTableBuilder.build(); - final ImmutableOpenMap.Builder restores = ImmutableOpenMap.builder(); + final Map restores = new HashMap<>(); if (snapshotIndices.isEmpty() == false) { // Some indices are restored from snapshot, the RestoreInProgress must be set accordingly - ImmutableOpenMap.Builder restoreShards = ImmutableOpenMap.builder(); + final Map restoreShards = new HashMap<>(); for (ShardRouting shard : routingTable.allShards()) { if (shard.primary() && shard.recoverySource().getType() == RecoverySource.Type.SNAPSHOT) { final ShardId shardId = shard.shardId(); @@ -848,7 +850,7 @@ private ClusterState createRecoveryStateAndInitializeAllocations( snapshot, RestoreInProgress.State.INIT, new ArrayList<>(snapshotIndices), - restoreShards.build() + restoreShards ); restores.put(RestoreInProgress.TYPE, new RestoreInProgress.Builder().add(restore).build()); } @@ -857,7 +859,7 @@ private ClusterState createRecoveryStateAndInitializeAllocations( .nodes(DiscoveryNodes.builder().add(node1)) .metadata(metadataBuilder.build()) .routingTable(routingTable) - .customs(restores.build()) + .customs(restores) .build(); } diff --git a/server/src/test/java/org/opensearch/cluster/routing/allocation/decider/DiskThresholdDeciderTests.java b/server/src/test/java/org/opensearch/cluster/routing/allocation/decider/DiskThresholdDeciderTests.java index da50dd53b7d54..709ad85f7c0b9 100644 --- a/server/src/test/java/org/opensearch/cluster/routing/allocation/decider/DiskThresholdDeciderTests.java +++ b/server/src/test/java/org/opensearch/cluster/routing/allocation/decider/DiskThresholdDeciderTests.java @@ -1344,11 +1344,11 @@ public void testDiskThresholdWithSnapshotShardSizes() { ) .build(); - final ImmutableOpenMap.Builder shards = ImmutableOpenMap.builder(); + final Map shards = new HashMap<>(); shards.put(shardId, new RestoreInProgress.ShardRestoreStatus("node1")); final RestoreInProgress.Builder restores = new RestoreInProgress.Builder().add( - new RestoreInProgress.Entry("_restore_uuid", snapshot, RestoreInProgress.State.INIT, singletonList("test"), shards.build()) + new RestoreInProgress.Entry("_restore_uuid", snapshot, RestoreInProgress.State.INIT, singletonList("test"), shards) ); ClusterState clusterState = ClusterState.builder(new ClusterName(getTestName())) diff --git a/server/src/test/java/org/opensearch/cluster/routing/allocation/decider/RestoreInProgressAllocationDeciderTests.java b/server/src/test/java/org/opensearch/cluster/routing/allocation/decider/RestoreInProgressAllocationDeciderTests.java index 4c0bdb2c900c7..d13a63ade6b9e 100644 --- a/server/src/test/java/org/opensearch/cluster/routing/allocation/decider/RestoreInProgressAllocationDeciderTests.java +++ b/server/src/test/java/org/opensearch/cluster/routing/allocation/decider/RestoreInProgressAllocationDeciderTests.java @@ -52,7 +52,6 @@ import org.opensearch.cluster.routing.UnassignedInfo; import org.opensearch.cluster.routing.allocation.RoutingAllocation; import org.opensearch.common.UUIDs; -import org.opensearch.common.collect.ImmutableOpenMap; import org.opensearch.index.shard.ShardId; import org.opensearch.repositories.IndexId; import org.opensearch.snapshots.Snapshot; @@ -60,6 +59,8 @@ import java.io.IOException; import java.util.Collections; +import java.util.HashMap; +import java.util.Map; import static java.util.Collections.singletonList; @@ -157,7 +158,7 @@ public void testCanAllocatePrimaryExistingInRestoreInProgress() { routingTable = RoutingTable.builder(routingTable).add(newIndexRoutingTable).build(); } - ImmutableOpenMap.Builder shards = ImmutableOpenMap.builder(); + final Map shards = new HashMap<>(); shards.put(primary.shardId(), new RestoreInProgress.ShardRestoreStatus(clusterState.getNodes().getLocalNodeId(), shardState)); Snapshot snapshot = recoverySource.snapshot(); @@ -167,7 +168,7 @@ public void testCanAllocatePrimaryExistingInRestoreInProgress() { snapshot, restoreState, singletonList("test"), - shards.build() + shards ); clusterState = ClusterState.builder(clusterState) diff --git a/server/src/test/java/org/opensearch/cluster/serialization/ClusterSerializationTests.java b/server/src/test/java/org/opensearch/cluster/serialization/ClusterSerializationTests.java index a760bd8e57d1f..ee5615fb94b54 100644 --- a/server/src/test/java/org/opensearch/cluster/serialization/ClusterSerializationTests.java +++ b/server/src/test/java/org/opensearch/cluster/serialization/ClusterSerializationTests.java @@ -50,7 +50,6 @@ import org.opensearch.cluster.routing.RoutingTable; import org.opensearch.cluster.routing.allocation.AllocationService; import org.opensearch.common.UUIDs; -import org.opensearch.common.collect.ImmutableOpenMap; import org.opensearch.common.io.stream.BytesStreamOutput; import org.opensearch.common.io.stream.NamedWriteableAwareStreamInput; import org.opensearch.common.io.stream.NamedWriteableRegistry; @@ -66,6 +65,7 @@ import java.util.Arrays; import java.util.Collections; import java.util.List; +import java.util.Map; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.notNullValue; @@ -162,7 +162,7 @@ public void testSnapshotDeletionsInProgressSerialization() throws Exception { new Snapshot("repo2", new SnapshotId("snap2", UUIDs.randomBase64UUID())), RestoreInProgress.State.STARTED, Collections.singletonList("index_name"), - ImmutableOpenMap.of() + Map.of() ) ).build() ); diff --git a/server/src/test/java/org/opensearch/rest/action/admin/indices/RestGetAliasesActionTests.java b/server/src/test/java/org/opensearch/rest/action/admin/indices/RestGetAliasesActionTests.java index c7974754d7bd0..1b82787e92120 100644 --- a/server/src/test/java/org/opensearch/rest/action/admin/indices/RestGetAliasesActionTests.java +++ b/server/src/test/java/org/opensearch/rest/action/admin/indices/RestGetAliasesActionTests.java @@ -33,7 +33,6 @@ package org.opensearch.rest.action.admin.indices; import org.opensearch.cluster.metadata.AliasMetadata; -import org.opensearch.common.collect.ImmutableOpenMap; import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.common.xcontent.XContentFactory; import org.opensearch.common.xcontent.XContentType; @@ -41,7 +40,9 @@ import org.opensearch.test.OpenSearchTestCase; import java.util.Arrays; +import java.util.HashMap; import java.util.List; +import java.util.Map; import static org.opensearch.rest.RestStatus.OK; import static org.opensearch.rest.RestStatus.NOT_FOUND; @@ -60,16 +61,11 @@ public class RestGetAliasesActionTests extends OpenSearchTestCase { public void testBareRequest() throws Exception { final XContentBuilder xContentBuilder = XContentFactory.contentBuilder(XContentType.JSON); - final ImmutableOpenMap.Builder> openMapBuilder = ImmutableOpenMap.builder(); + final Map> openMapBuilder = new HashMap<>(); final AliasMetadata foobarAliasMetadata = AliasMetadata.builder("foobar").build(); final AliasMetadata fooAliasMetadata = AliasMetadata.builder("foo").build(); openMapBuilder.put("index", Arrays.asList(fooAliasMetadata, foobarAliasMetadata)); - final RestResponse restResponse = RestGetAliasesAction.buildRestResponse( - false, - new String[0], - openMapBuilder.build(), - xContentBuilder - ); + final RestResponse restResponse = RestGetAliasesAction.buildRestResponse(false, new String[0], openMapBuilder, xContentBuilder); assertThat(restResponse.status(), equalTo(OK)); assertThat(restResponse.contentType(), equalTo("application/json; charset=UTF-8")); assertThat(restResponse.content().utf8ToString(), equalTo("{\"index\":{\"aliases\":{\"foo\":{},\"foobar\":{}}}}")); @@ -77,11 +73,11 @@ public void testBareRequest() throws Exception { public void testSimpleAliasWildcardMatchingNothing() throws Exception { final XContentBuilder xContentBuilder = XContentFactory.contentBuilder(XContentType.JSON); - final ImmutableOpenMap.Builder> openMapBuilder = ImmutableOpenMap.builder(); + final Map> openMapBuilder = new HashMap<>(); final RestResponse restResponse = RestGetAliasesAction.buildRestResponse( true, new String[] { "baz*" }, - openMapBuilder.build(), + openMapBuilder, xContentBuilder ); assertThat(restResponse.status(), equalTo(OK)); @@ -91,13 +87,13 @@ public void testSimpleAliasWildcardMatchingNothing() throws Exception { public void testMultipleAliasWildcardsSomeMatching() throws Exception { final XContentBuilder xContentBuilder = XContentFactory.contentBuilder(XContentType.JSON); - final ImmutableOpenMap.Builder> openMapBuilder = ImmutableOpenMap.builder(); + final Map> openMapBuilder = new HashMap<>(); final AliasMetadata aliasMetadata = AliasMetadata.builder("foobar").build(); openMapBuilder.put("index", Arrays.asList(aliasMetadata)); final RestResponse restResponse = RestGetAliasesAction.buildRestResponse( true, new String[] { "baz*", "foobar*" }, - openMapBuilder.build(), + openMapBuilder, xContentBuilder ); assertThat(restResponse.status(), equalTo(OK)); @@ -107,11 +103,11 @@ public void testMultipleAliasWildcardsSomeMatching() throws Exception { public void testAliasWildcardsIncludeAndExcludeAll() throws Exception { final XContentBuilder xContentBuilder = XContentFactory.contentBuilder(XContentType.JSON); - final ImmutableOpenMap.Builder> openMapBuilder = ImmutableOpenMap.builder(); + final Map> openMapBuilder = new HashMap<>(); final RestResponse restResponse = RestGetAliasesAction.buildRestResponse( true, new String[] { "foob*", "-foo*" }, - openMapBuilder.build(), + openMapBuilder, xContentBuilder ); assertThat(restResponse.status(), equalTo(OK)); @@ -121,13 +117,13 @@ public void testAliasWildcardsIncludeAndExcludeAll() throws Exception { public void testAliasWildcardsIncludeAndExcludeSome() throws Exception { final XContentBuilder xContentBuilder = XContentFactory.contentBuilder(XContentType.JSON); - final ImmutableOpenMap.Builder> openMapBuilder = ImmutableOpenMap.builder(); + final Map> openMapBuilder = new HashMap<>(); final AliasMetadata aliasMetadata = AliasMetadata.builder("foo").build(); openMapBuilder.put("index", Arrays.asList(aliasMetadata)); final RestResponse restResponse = RestGetAliasesAction.buildRestResponse( true, new String[] { "foo*", "-foob*" }, - openMapBuilder.build(), + openMapBuilder, xContentBuilder ); assertThat(restResponse.status(), equalTo(OK)); @@ -137,7 +133,7 @@ public void testAliasWildcardsIncludeAndExcludeSome() throws Exception { public void testAliasWildcardsIncludeAndExcludeSomeAndExplicitMissing() throws Exception { final XContentBuilder xContentBuilder = XContentFactory.contentBuilder(XContentType.JSON); - final ImmutableOpenMap.Builder> openMapBuilder = ImmutableOpenMap.builder(); + final Map> openMapBuilder = new HashMap<>(); final AliasMetadata aliasMetadata = AliasMetadata.builder("foo").build(); openMapBuilder.put("index", Arrays.asList(aliasMetadata)); final String[] aliasPattern; @@ -147,12 +143,7 @@ public void testAliasWildcardsIncludeAndExcludeSomeAndExplicitMissing() throws E aliasPattern = new String[] { "foo*", "-foob*", "missing" }; } - final RestResponse restResponse = RestGetAliasesAction.buildRestResponse( - true, - aliasPattern, - openMapBuilder.build(), - xContentBuilder - ); + final RestResponse restResponse = RestGetAliasesAction.buildRestResponse(true, aliasPattern, openMapBuilder, xContentBuilder); assertThat(restResponse.status(), equalTo(NOT_FOUND)); assertThat(restResponse.contentType(), equalTo("application/json; charset=UTF-8")); assertThat( @@ -163,11 +154,11 @@ public void testAliasWildcardsIncludeAndExcludeSomeAndExplicitMissing() throws E public void testAliasWildcardsExcludeExplicitMissing() throws Exception { final XContentBuilder xContentBuilder = XContentFactory.contentBuilder(XContentType.JSON); - final ImmutableOpenMap.Builder> openMapBuilder = ImmutableOpenMap.builder(); + final Map> openMapBuilder = new HashMap<>(); final RestResponse restResponse = RestGetAliasesAction.buildRestResponse( true, new String[] { "foo", "foofoo", "-foo*" }, - openMapBuilder.build(), + openMapBuilder, xContentBuilder ); assertThat(restResponse.status(), equalTo(OK)); diff --git a/server/src/test/java/org/opensearch/snapshots/InternalSnapshotsInfoServiceTests.java b/server/src/test/java/org/opensearch/snapshots/InternalSnapshotsInfoServiceTests.java index 483b5c268d3f9..769a3db0481cd 100644 --- a/server/src/test/java/org/opensearch/snapshots/InternalSnapshotsInfoServiceTests.java +++ b/server/src/test/java/org/opensearch/snapshots/InternalSnapshotsInfoServiceTests.java @@ -54,7 +54,6 @@ import org.opensearch.cluster.service.ClusterApplier; import org.opensearch.cluster.service.ClusterService; import org.opensearch.common.UUIDs; -import org.opensearch.common.collect.ImmutableOpenMap; import org.opensearch.common.settings.Settings; import org.opensearch.index.Index; import org.opensearch.index.shard.ShardId; @@ -72,6 +71,7 @@ import org.junit.Before; import java.util.Collections; +import java.util.HashMap; import java.util.Locale; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; @@ -462,7 +462,7 @@ private ClusterState addUnassignedShards(final ClusterState currentState, String final RestoreInProgress.Builder restores = new RestoreInProgress.Builder( currentState.custom(RestoreInProgress.TYPE, RestoreInProgress.EMPTY) ); - final ImmutableOpenMap.Builder shards = ImmutableOpenMap.builder(); + final Map shards = new HashMap<>(); for (int i = 0; i < indexMetadata.getNumberOfShards(); i++) { shards.put(new ShardId(index, i), new RestoreInProgress.ShardRestoreStatus(clusterService.state().nodes().getLocalNodeId())); } @@ -473,7 +473,7 @@ private ClusterState addUnassignedShards(final ClusterState currentState, String recoverySource.snapshot(), RestoreInProgress.State.INIT, Collections.singletonList(indexName), - shards.build() + shards ) ); diff --git a/server/src/test/java/org/opensearch/snapshots/SnapshotsInProgressSerializationTests.java b/server/src/test/java/org/opensearch/snapshots/SnapshotsInProgressSerializationTests.java index dc5d869c2f6a2..d02b82b45d90e 100644 --- a/server/src/test/java/org/opensearch/snapshots/SnapshotsInProgressSerializationTests.java +++ b/server/src/test/java/org/opensearch/snapshots/SnapshotsInProgressSerializationTests.java @@ -39,7 +39,6 @@ import org.opensearch.cluster.SnapshotsInProgress.Entry; import org.opensearch.cluster.SnapshotsInProgress.ShardState; import org.opensearch.cluster.SnapshotsInProgress.State; -import org.opensearch.common.collect.ImmutableOpenMap; import org.opensearch.common.io.stream.NamedWriteableRegistry; import org.opensearch.common.io.stream.Writeable; import org.opensearch.index.Index; @@ -50,7 +49,10 @@ import java.util.ArrayList; import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; import java.util.List; +import java.util.Map; import java.util.stream.Collectors; public class SnapshotsInProgressSerializationTests extends AbstractDiffableWireSerializationTestCase { @@ -76,7 +78,7 @@ private Entry randomSnapshot() { } long startTime = randomLong(); long repositoryStateId = randomLong(); - ImmutableOpenMap.Builder builder = ImmutableOpenMap.builder(); + Map builder = new HashMap<>(); final List esIndices = indices.stream() .map(i -> new Index(i.getName(), randomAlphaOfLength(10))) .collect(Collectors.toList()); @@ -100,7 +102,7 @@ private Entry randomSnapshot() { ); } } - ImmutableOpenMap shards = builder.build(); + Map shards = Collections.unmodifiableMap(builder); return new Entry( snapshot, includeGlobalState, @@ -172,7 +174,7 @@ protected Custom mutateInstance(Custom instance) { return SnapshotsInProgress.of(entries); } - public static State randomState(ImmutableOpenMap shards) { + public static State randomState(final Map shards) { return SnapshotsInProgress.completed(shards.values()) ? randomFrom(State.SUCCESS, State.FAILED) : randomFrom(State.STARTED, State.INIT, State.ABORTED); diff --git a/server/src/test/java/org/opensearch/snapshots/SnapshotsServiceTests.java b/server/src/test/java/org/opensearch/snapshots/SnapshotsServiceTests.java index 7f96d4842e37d..bbdb71c2024cf 100644 --- a/server/src/test/java/org/opensearch/snapshots/SnapshotsServiceTests.java +++ b/server/src/test/java/org/opensearch/snapshots/SnapshotsServiceTests.java @@ -44,7 +44,6 @@ import org.opensearch.cluster.routing.ShardRoutingState; import org.opensearch.cluster.routing.TestShardRouting; import org.opensearch.common.UUIDs; -import org.opensearch.common.collect.ImmutableOpenMap; import org.opensearch.common.settings.Settings; import org.opensearch.index.Index; import org.opensearch.index.shard.ShardId; @@ -55,6 +54,7 @@ import java.util.Arrays; import java.util.Collections; import java.util.List; +import java.util.Map; import java.util.stream.Collectors; import java.util.stream.StreamSupport; @@ -66,7 +66,7 @@ public class SnapshotsServiceTests extends OpenSearchTestCase { 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.emptyList(), Map.of()); final String indexName1 = "index-1"; final ShardId shardId1 = new ShardId(index(indexName1), 0); @@ -128,7 +128,7 @@ public void testUpdateSnapshotMultipleShards() throws Exception { final SnapshotsInProgress.Entry snapshotSingleShard = snapshotEntry( sn1, Collections.singletonList(indexId1), - ImmutableOpenMap.builder(shardsMap(shardId1, shardInitStatus)).fPut(shardId2, shardInitStatus).build() + Map.of(shardId1, shardInitStatus, shardId2, shardInitStatus) ); assertThat(snapshotSingleShard.state(), is(SnapshotsInProgress.State.STARTED)); @@ -178,7 +178,7 @@ public void testUpdateCloneMultipleShards() throws Exception { final SnapshotsInProgress.Entry cloneMultipleShards = cloneEntry( targetSnapshot, sourceSnapshot.getSnapshotId(), - ImmutableOpenMap.builder(clonesMap(shardId1, shardInitStatus)).fPut(shardId2, shardInitStatus).build() + Map.of(shardId1, shardInitStatus, shardId2, shardInitStatus) ); assertThat(cloneMultipleShards.state(), is(SnapshotsInProgress.State.STARTED)); @@ -408,18 +408,18 @@ private static DiscoveryNodes discoveryNodes(String localNodeId) { return DiscoveryNodes.builder().localNodeId(localNodeId).build(); } - private static ImmutableOpenMap shardsMap( + private static Map shardsMap( ShardId shardId, SnapshotsInProgress.ShardSnapshotStatus shardStatus ) { - return ImmutableOpenMap.builder().fPut(shardId, shardStatus).build(); + return Map.of(shardId, shardStatus); } - private static ImmutableOpenMap clonesMap( + private static Map clonesMap( RepositoryShardId shardId, SnapshotsInProgress.ShardSnapshotStatus shardStatus ) { - return ImmutableOpenMap.builder().fPut(shardId, shardStatus).build(); + return Map.of(shardId, shardStatus); } private static SnapshotsService.ShardSnapshotUpdate successUpdate(Snapshot snapshot, ShardId shardId, String nodeId) { @@ -472,7 +472,7 @@ private static ClusterState applyUpdates(ClusterState state, SnapshotsService.Sh private static SnapshotsInProgress.Entry snapshotEntry( Snapshot snapshot, List indexIds, - ImmutableOpenMap shards + final Map shards ) { return SnapshotsInProgress.startedEntry( snapshot, @@ -491,10 +491,10 @@ private static SnapshotsInProgress.Entry snapshotEntry( private static SnapshotsInProgress.Entry cloneEntry( Snapshot snapshot, SnapshotId source, - ImmutableOpenMap clones + final Map clones ) { - final List indexIds = StreamSupport.stream(clones.keys().spliterator(), false) - .map(k -> k.value.index()) + final List indexIds = StreamSupport.stream(clones.keySet().spliterator(), false) + .map(k -> k.index()) .distinct() .collect(Collectors.toList()); return SnapshotsInProgress.startClone(snapshot, source, indexIds, 1L, randomNonNegativeLong(), Version.CURRENT).withClones(clones); diff --git a/test/framework/src/main/java/org/opensearch/test/InternalTestCluster.java b/test/framework/src/main/java/org/opensearch/test/InternalTestCluster.java index 121003f66fc94..a20db43f6bfb2 100644 --- a/test/framework/src/main/java/org/opensearch/test/InternalTestCluster.java +++ b/test/framework/src/main/java/org/opensearch/test/InternalTestCluster.java @@ -33,7 +33,6 @@ import com.carrotsearch.hppc.ObjectLongMap; import com.carrotsearch.hppc.cursors.IntObjectCursor; -import com.carrotsearch.hppc.cursors.ObjectObjectCursor; import com.carrotsearch.randomizedtesting.RandomizedTest; import com.carrotsearch.randomizedtesting.SeedUtils; import com.carrotsearch.randomizedtesting.generators.RandomNumbers; @@ -1492,8 +1491,8 @@ private IndexShard getShardOrNull(ClusterState clusterState, ShardRouting shardR public void assertSeqNos() throws Exception { assertBusy(() -> { final ClusterState state = clusterService().state(); - for (ObjectObjectCursor indexRoutingTable : state.routingTable().indicesRouting()) { - for (IntObjectCursor indexShardRoutingTable : indexRoutingTable.value.shards()) { + for (final IndexRoutingTable indexRoutingTable : state.routingTable().indicesRouting().values()) { + for (IntObjectCursor indexShardRoutingTable : indexRoutingTable.shards()) { ShardRouting primaryShardRouting = indexShardRoutingTable.value.primaryShard(); final IndexShard primaryShard = getShardOrNull(state, primaryShardRouting); if (primaryShard == null) { @@ -1542,8 +1541,8 @@ public void assertSeqNos() throws Exception { public void assertSameDocIdsOnShards() throws Exception { assertBusy(() -> { ClusterState state = client().admin().cluster().prepareState().get().getState(); - for (ObjectObjectCursor indexRoutingTable : state.routingTable().indicesRouting()) { - for (IntObjectCursor indexShardRoutingTable : indexRoutingTable.value.shards()) { + for (final IndexRoutingTable indexRoutingTable : state.routingTable().indicesRouting().values()) { + for (IntObjectCursor indexShardRoutingTable : indexRoutingTable.shards()) { ShardRouting primaryShardRouting = indexShardRoutingTable.value.primaryShard(); IndexShard primaryShard = getShardOrNull(state, primaryShardRouting); if (primaryShard == null) {