diff --git a/server/src/internalClusterTest/java/org/opensearch/gateway/remote/RemoteRoutingTableServiceIT.java b/server/src/internalClusterTest/java/org/opensearch/gateway/remote/RemoteRoutingTableServiceIT.java index b0d046cbdf3db..9b80e4f907070 100644 --- a/server/src/internalClusterTest/java/org/opensearch/gateway/remote/RemoteRoutingTableServiceIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/gateway/remote/RemoteRoutingTableServiceIT.java @@ -24,6 +24,7 @@ import org.opensearch.test.OpenSearchIntegTestCase; import org.junit.Before; +import java.io.IOException; import java.nio.charset.StandardCharsets; import java.nio.file.Path; import java.util.ArrayList; @@ -46,7 +47,7 @@ public class RemoteRoutingTableServiceIT extends RemoteStoreBaseIntegTestCase { private static final String INDEX_NAME = "test-index"; private static final String INDEX_NAME_1 = "test-index-1"; - BlobPath indexRoutingPath; + List indexRoutingPaths; AtomicInteger indexRoutingFiles = new AtomicInteger(); private final RemoteStoreEnums.PathType pathType = RemoteStoreEnums.PathType.HASHED_PREFIX; @@ -91,7 +92,7 @@ public void testRemoteRoutingTableIndexLifecycle() throws Exception { updateIndexSettings(INDEX_NAME, IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 2); ensureGreen(INDEX_NAME); assertBusy(() -> { - int indexRoutingFilesAfterUpdate = repository.blobStore().blobContainer(indexRoutingPath).listBlobs().size(); + int indexRoutingFilesAfterUpdate = repository.blobStore().blobContainer(indexRoutingPaths.get(0)).listBlobs().size(); // At-least 3 new index routing files will be created as shards will transition from INIT -> UNASSIGNED -> STARTED state assertTrue(indexRoutingFilesAfterUpdate >= indexRoutingFiles.get() + 3); }); @@ -112,6 +113,47 @@ public void testRemoteRoutingTableIndexLifecycle() throws Exception { assertTrue(areRoutingTablesSame(routingTableVersions)); } + public void testRemoteRoutingTableWithMultipleIndex() throws Exception { + BlobStoreRepository repository = prepareClusterAndVerifyRepository(); + + RemoteClusterStateService remoteClusterStateService = internalCluster().getClusterManagerNodeInstance( + RemoteClusterStateService.class + ); + RemoteManifestManager remoteManifestManager = remoteClusterStateService.getRemoteManifestManager(); + Optional latestManifest = remoteManifestManager.getLatestClusterMetadataManifest( + getClusterState().getClusterName().value(), + getClusterState().getMetadata().clusterUUID() + ); + List expectedIndexNames = new ArrayList<>(); + List deletedIndexNames = new ArrayList<>(); + verifyUpdatesInManifestFile(latestManifest, expectedIndexNames, 1, deletedIndexNames, true); + + List routingTables = getRoutingTableFromAllNodes(); + // Verify indices in routing table + Set expectedIndicesInRoutingTable = Set.of(INDEX_NAME); + assertEquals(routingTables.get(0).getIndicesRouting().keySet(), expectedIndicesInRoutingTable); + // Verify routing table across all nodes is equal + assertTrue(areRoutingTablesSame(routingTables)); + + // Create new index + createIndex(INDEX_NAME_1, remoteStoreIndexSettings(1, 5)); + ensureGreen(INDEX_NAME_1); + + latestManifest = remoteManifestManager.getLatestClusterMetadataManifest( + getClusterState().getClusterName().value(), + getClusterState().getMetadata().clusterUUID() + ); + + updateIndexRoutingPaths(repository); + verifyUpdatesInManifestFile(latestManifest, expectedIndexNames, 2, deletedIndexNames, true); + routingTables = getRoutingTableFromAllNodes(); + // Verify indices in routing table + expectedIndicesInRoutingTable = Set.of(INDEX_NAME, INDEX_NAME_1); + assertEquals(routingTables.get(0).getIndicesRouting().keySet(), expectedIndicesInRoutingTable); + // Verify routing table across all nodes is equal + assertTrue(areRoutingTablesSame(routingTables)); + } + public void testRemoteRoutingTableEmptyRoutingTableDiff() throws Exception { prepareClusterAndVerifyRepository(); @@ -166,7 +208,7 @@ public void testRemoteRoutingTableIndexNodeRestart() throws Exception { assertRemoteStoreRepositoryOnAllNodes(REMOTE_ROUTING_TABLE_REPO); assertBusy(() -> { - int indexRoutingFilesAfterNodeDrop = repository.blobStore().blobContainer(indexRoutingPath).listBlobs().size(); + int indexRoutingFilesAfterNodeDrop = repository.blobStore().blobContainer(indexRoutingPaths.get(0)).listBlobs().size(); assertTrue(indexRoutingFilesAfterNodeDrop > indexRoutingFiles.get()); }); @@ -201,7 +243,7 @@ public void testRemoteRoutingTableIndexMasterRestart() throws Exception { assertRemoteStoreRepositoryOnAllNodes(REMOTE_ROUTING_TABLE_REPO); assertBusy(() -> { - int indexRoutingFilesAfterNodeDrop = repository.blobStore().blobContainer(indexRoutingPath).listBlobs().size(); + int indexRoutingFilesAfterNodeDrop = repository.blobStore().blobContainer(indexRoutingPaths.get(0)).listBlobs().size(); assertTrue(indexRoutingFilesAfterNodeDrop > indexRoutingFiles.get()); }); @@ -240,10 +282,14 @@ private BlobStoreRepository prepareClusterAndVerifyRepository() throws Exception BlobPath baseMetadataPath = getBaseMetadataPath(repository); List indexRoutingTables = new ArrayList<>(getClusterState().routingTable().indicesRouting().values()); - indexRoutingPath = getIndexRoutingPath(baseMetadataPath.add(INDEX_ROUTING_TABLE), indexRoutingTables.get(0).getIndex().getUUID()); + indexRoutingPaths = new ArrayList<>(); + for (IndexRoutingTable indexRoutingTable : indexRoutingTables) { + indexRoutingPaths.add(getIndexRoutingPath(baseMetadataPath.add(INDEX_ROUTING_TABLE), indexRoutingTable.getIndex().getUUID())); + } assertBusy(() -> { - indexRoutingFiles.set(repository.blobStore().blobContainer(indexRoutingPath).listBlobs().size()); + int totalRoutingFiles = calculateTotalRoutingFiles(repository); + indexRoutingFiles.set(totalRoutingFiles); // There would be >=3 files as shards will transition from UNASSIGNED -> INIT -> STARTED state assertTrue(indexRoutingFiles.get() >= 3); }); @@ -280,11 +326,19 @@ private void verifyUpdatesInManifestFile( assertTrue(latestManifest.isPresent()); ClusterMetadataManifest manifest = latestManifest.get(); - assertEquals(expectedIndexNames, manifest.getDiffManifest().getIndicesRoutingUpdated()); assertEquals(expectedDeletedIndex, manifest.getDiffManifest().getIndicesDeleted()); assertEquals(expectedIndicesRoutingFilesInManifest, manifest.getIndicesRouting().size()); + + // Check if all paths in manifest.getIndicesRouting() are present in indexRoutingPaths for (ClusterMetadataManifest.UploadedIndexMetadata uploadedFilename : manifest.getIndicesRouting()) { - assertTrue(uploadedFilename.getUploadedFilename().contains(indexRoutingPath.buildAsString())); + boolean pathFound = false; + for (BlobPath indexRoutingPath : indexRoutingPaths) { + if (uploadedFilename.getUploadedFilename().contains(indexRoutingPath.buildAsString())) { + pathFound = true; + break; + } + } + assertTrue("Uploaded file not found in indexRoutingPaths: " + uploadedFilename.getUploadedFilename(), pathFound); } assertEquals(isRoutingTableDiffFileExpected, manifest.getDiffManifest().getIndicesRoutingDiffPath() != null); } @@ -305,6 +359,24 @@ private List getRoutingTableFromAllNodes() throws ExecutionExcepti return routingTables; } + private void updateIndexRoutingPaths(BlobStoreRepository repository) { + BlobPath baseMetadataPath = getBaseMetadataPath(repository); + List indexRoutingTables = new ArrayList<>(getClusterState().routingTable().indicesRouting().values()); + + indexRoutingPaths.clear(); // Clear the list to avoid stale data + for (IndexRoutingTable indexRoutingTable : indexRoutingTables) { + indexRoutingPaths.add(getIndexRoutingPath(baseMetadataPath.add(INDEX_ROUTING_TABLE), indexRoutingTable.getIndex().getUUID())); + } + } + + private int calculateTotalRoutingFiles(BlobStoreRepository repository) throws IOException { + int totalRoutingFiles = 0; + for (BlobPath path : indexRoutingPaths) { + totalRoutingFiles += repository.blobStore().blobContainer(path).listBlobs().size(); + } + return totalRoutingFiles; + } + private boolean areRoutingTablesSame(List routingTables) { if (routingTables == null || routingTables.isEmpty()) { return false; @@ -356,7 +428,6 @@ private void deleteIndexAndVerify(RemoteManifestManager remoteManifestManager) { ); assertTrue(latestManifest.isPresent()); ClusterMetadataManifest manifest = latestManifest.get(); - assertTrue(manifest.getDiffManifest().getIndicesRoutingUpdated().isEmpty()); assertTrue(manifest.getDiffManifest().getIndicesDeleted().contains(INDEX_NAME)); assertTrue(manifest.getIndicesRouting().isEmpty()); } diff --git a/server/src/main/java/org/opensearch/cluster/routing/IndexShardRoutingTable.java b/server/src/main/java/org/opensearch/cluster/routing/IndexShardRoutingTable.java index 00ddef507a162..f782cfc1b060a 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/IndexShardRoutingTable.java +++ b/server/src/main/java/org/opensearch/cluster/routing/IndexShardRoutingTable.java @@ -34,6 +34,8 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; +import org.opensearch.cluster.AbstractDiffable; +import org.opensearch.cluster.Diff; import org.opensearch.cluster.node.DiscoveryNode; import org.opensearch.cluster.node.DiscoveryNodes; import org.opensearch.common.Nullable; @@ -75,7 +77,7 @@ * @opensearch.api */ @PublicApi(since = "1.0.0") -public class IndexShardRoutingTable implements Iterable { +public class IndexShardRoutingTable extends AbstractDiffable implements Iterable { final ShardShuffler shuffler; // Shuffler for weighted round-robin shard routing. This uses rotation to permute shards. @@ -545,6 +547,12 @@ private static List rankShardsAndUpdateStats( return sortedShards; } + @Override + public void writeTo(StreamOutput out) throws IOException { + this.shardId().getIndex().writeTo(out); + Builder.writeToThin(this, out); + } + private static class NodeRankComparator implements Comparator { private final Map nodeRanks; @@ -1067,6 +1075,14 @@ private void populateInitializingShardWeightsMap(WeightedRouting weightedRouting } } + public static IndexShardRoutingTable readFrom(StreamInput in) throws IOException { + return IndexShardRoutingTable.Builder.readFrom(in); + } + + public static Diff readDiffFrom(StreamInput in) throws IOException { + return readDiffFrom(IndexShardRoutingTable::readFrom, in); + } + /** * Builder of an index shard routing table. * 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 f805ed09c2512..544d4f9326fba 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/RoutingTable.java +++ b/server/src/main/java/org/opensearch/cluster/routing/RoutingTable.java @@ -378,6 +378,10 @@ public Diff diff(RoutingTable previousState) { return new RoutingTableDiff(previousState, this); } + public Diff incrementalDiff(RoutingTable previousState) { + return new RoutingTableIncrementalDiff(previousState, this); + } + public static Diff readDiffFrom(StreamInput in) throws IOException { return new RoutingTableDiff(in); } @@ -403,7 +407,7 @@ public void writeTo(StreamOutput out) throws IOException { } } - private static class RoutingTableDiff implements Diff { + private static class RoutingTableDiff implements Diff, StringKeyDiffProvider { private final long version; @@ -432,6 +436,11 @@ public void writeTo(StreamOutput out) throws IOException { out.writeLong(version); indicesRouting.writeTo(out); } + + @Override + public DiffableUtils.MapDiff> provideDiff() { + return (DiffableUtils.MapDiff>) indicesRouting; + } } public static Builder builder() { diff --git a/server/src/main/java/org/opensearch/cluster/routing/RoutingTableIncrementalDiff.java b/server/src/main/java/org/opensearch/cluster/routing/RoutingTableIncrementalDiff.java index 3d75b22a8ed7f..13501a431d9f9 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/RoutingTableIncrementalDiff.java +++ b/server/src/main/java/org/opensearch/cluster/routing/RoutingTableIncrementalDiff.java @@ -9,93 +9,85 @@ package org.opensearch.cluster.routing; import org.opensearch.cluster.Diff; +import org.opensearch.cluster.DiffableUtils; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; +import org.opensearch.core.index.Index; import java.io.IOException; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; import java.util.Map; +import static org.opensearch.cluster.DiffableUtils.MapDiff; + /** * Represents a difference between {@link RoutingTable} objects that can be serialized and deserialized. */ -public class RoutingTableIncrementalDiff implements Diff { +public class RoutingTableIncrementalDiff implements Diff, StringKeyDiffProvider { - private final Map> diffs; + private final Diff> indicesRouting; - /** - * Constructs a new RoutingTableIncrementalDiff with the given differences. - * - * @param diffs a map containing the differences of {@link IndexRoutingTable}. - */ - public RoutingTableIncrementalDiff(Map> diffs) { - this.diffs = diffs; + private final long version; + + private static final DiffableUtils.DiffableValueSerializer CUSTOM_ROUTING_TABLE_DIFFABLE_VALUE_SERIALIZER = + new DiffableUtils.DiffableValueSerializer<>() { + + @Override + public IndexRoutingTable read(StreamInput in, String key) throws IOException { + return IndexRoutingTable.readFrom(in); + } + + @Override + public Diff readDiff(StreamInput in, String key) throws IOException { + return new RoutingTableIncrementalDiff.IndexRoutingTableIncrementalDiff(in); + } + + @Override + public Diff diff(IndexRoutingTable currentState, IndexRoutingTable previousState) { + return new RoutingTableIncrementalDiff.IndexRoutingTableIncrementalDiff( + currentState.getIndex(), + previousState, + currentState + ); + } + }; + + public RoutingTableIncrementalDiff(RoutingTable before, RoutingTable after) { + version = after.version(); + indicesRouting = DiffableUtils.diff( + before.getIndicesRouting(), + after.getIndicesRouting(), + DiffableUtils.getStringKeySerializer(), + CUSTOM_ROUTING_TABLE_DIFFABLE_VALUE_SERIALIZER + ); } - /** - * Gets the map of differences of {@link IndexRoutingTable}. - * - * @return a map containing the differences. - */ - public Map> getDiffs() { - return diffs; + public RoutingTableIncrementalDiff(StreamInput in) throws IOException { + version = in.readLong(); + indicesRouting = DiffableUtils.readJdkMapDiff( + in, + DiffableUtils.getStringKeySerializer(), + CUSTOM_ROUTING_TABLE_DIFFABLE_VALUE_SERIALIZER + ); } - /** - * Reads a {@link RoutingTableIncrementalDiff} from the given {@link StreamInput}. - * - * @param in the input stream to read from. - * @return the deserialized RoutingTableIncrementalDiff. - * @throws IOException if an I/O exception occurs while reading from the stream. - */ public static RoutingTableIncrementalDiff readFrom(StreamInput in) throws IOException { - int size = in.readVInt(); - Map> diffs = new HashMap<>(); - - for (int i = 0; i < size; i++) { - String key = in.readString(); - Diff diff = IndexRoutingTableIncrementalDiff.readFrom(in); - diffs.put(key, diff); - } - return new RoutingTableIncrementalDiff(diffs); + return new RoutingTableIncrementalDiff(in); } - /** - * Applies the differences to the provided {@link RoutingTable}. - * - * @param part the original RoutingTable to which the differences will be applied. - * @return the updated RoutingTable with the applied differences. - */ @Override public RoutingTable apply(RoutingTable part) { - RoutingTable.Builder builder = new RoutingTable.Builder(); - for (IndexRoutingTable indexRoutingTable : part) { - builder.add(indexRoutingTable); // Add existing index routing tables to builder - } - - // Apply the diffs - for (Map.Entry> entry : diffs.entrySet()) { - builder.add(entry.getValue().apply(part.index(entry.getKey()))); - } - - return builder.build(); + return new RoutingTable(version, indicesRouting.apply(part.getIndicesRouting())); } - /** - * Writes the differences to the given {@link StreamOutput}. - * - * @param out the output stream to write to. - * @throws IOException if an I/O exception occurs while writing to the stream. - */ @Override public void writeTo(StreamOutput out) throws IOException { - out.writeVInt(diffs.size()); - for (Map.Entry> entry : diffs.entrySet()) { - out.writeString(entry.getKey()); - entry.getValue().writeTo(out); - } + out.writeLong(version); + indicesRouting.writeTo(out); + } + + @Override + public MapDiff> provideDiff() { + return (MapDiff>) indicesRouting; } /** @@ -103,66 +95,36 @@ public void writeTo(StreamOutput out) throws IOException { */ public static class IndexRoutingTableIncrementalDiff implements Diff { - private final List indexShardRoutingTables; + private final Diff> indexShardRoutingTables; - /** - * Constructs a new IndexShardRoutingTableDiff with the given shard routing tables. - * - * @param indexShardRoutingTables a list of IndexShardRoutingTable representing the differences. - */ - public IndexRoutingTableIncrementalDiff(List indexShardRoutingTables) { - this.indexShardRoutingTables = indexShardRoutingTables; + private final Index index; + + public IndexRoutingTableIncrementalDiff(Index index, IndexRoutingTable before, IndexRoutingTable after) { + this.index = index; + this.indexShardRoutingTables = DiffableUtils.diff(before.getShards(), after.getShards(), DiffableUtils.getIntKeySerializer()); + } + + private static final DiffableUtils.DiffableValueReader DIFF_VALUE_READER = + new DiffableUtils.DiffableValueReader<>(IndexShardRoutingTable::readFrom, IndexShardRoutingTable::readDiffFrom); + + public IndexRoutingTableIncrementalDiff(StreamInput in) throws IOException { + this.index = new Index(in); + this.indexShardRoutingTables = DiffableUtils.readJdkMapDiff(in, DiffableUtils.getIntKeySerializer(), DIFF_VALUE_READER); } - /** - * Applies the differences to the provided {@link IndexRoutingTable}. - * - * @param part the original IndexRoutingTable to which the differences will be applied. - * @return the updated IndexRoutingTable with the applied differences. - */ @Override public IndexRoutingTable apply(IndexRoutingTable part) { - IndexRoutingTable.Builder builder = new IndexRoutingTable.Builder(part.getIndex()); - for (IndexShardRoutingTable shardRoutingTable : part) { - builder.addIndexShard(shardRoutingTable); // Add existing shards to builder - } - - // Apply the diff: update or add the new shard routing tables - for (IndexShardRoutingTable diffShard : indexShardRoutingTables) { - builder.addIndexShard(diffShard); - } - return builder.build(); + return new IndexRoutingTable(index, indexShardRoutingTables.apply(part.getShards())); } - /** - * Writes the differences to the given {@link StreamOutput}. - * - * @param out the output stream to write to. - * @throws IOException if an I/O exception occurs while writing to the stream. - */ @Override public void writeTo(StreamOutput out) throws IOException { - out.writeVInt(indexShardRoutingTables.size()); - for (IndexShardRoutingTable shardRoutingTable : indexShardRoutingTables) { - IndexShardRoutingTable.Builder.writeTo(shardRoutingTable, out); - } + index.writeTo(out); + indexShardRoutingTables.writeTo(out); } - /** - * Reads a {@link IndexRoutingTableIncrementalDiff} from the given {@link StreamInput}. - * - * @param in the input stream to read from. - * @return the deserialized IndexShardRoutingTableDiff. - * @throws IOException if an I/O exception occurs while reading from the stream. - */ public static IndexRoutingTableIncrementalDiff readFrom(StreamInput in) throws IOException { - int size = in.readVInt(); - List indexShardRoutingTables = new ArrayList<>(size); - for (int i = 0; i < size; i++) { - IndexShardRoutingTable shardRoutingTable = IndexShardRoutingTable.Builder.readFrom(in); - indexShardRoutingTables.add(shardRoutingTable); - } - return new IndexRoutingTableIncrementalDiff(indexShardRoutingTables); + return new IndexRoutingTableIncrementalDiff(in); } } } diff --git a/server/src/main/java/org/opensearch/cluster/routing/StringKeyDiffProvider.java b/server/src/main/java/org/opensearch/cluster/routing/StringKeyDiffProvider.java new file mode 100644 index 0000000000000..5d36a238e03ff --- /dev/null +++ b/server/src/main/java/org/opensearch/cluster/routing/StringKeyDiffProvider.java @@ -0,0 +1,33 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.cluster.routing; + +import org.opensearch.cluster.DiffableUtils; + +import java.util.Map; + +/** + * Interface for providing a difference (diff) between two maps with {@code String} keys and values of type {@code V}. + * This interface is used to compute and obtain the difference between two versions of a map, typically used + * in cluster state updates or other scenarios where changes need to be tracked and propagated efficiently. + * + * @param the type of the values in the map + */ +public interface StringKeyDiffProvider { + + /** + * Provides the difference between two versions of a map with {@code String} keys and values of type {@code V}. + * The difference is represented as a {@link DiffableUtils.MapDiff} object, which can be used to apply the + * changes to another map or to serialize the diff. + * + * @return a {@link DiffableUtils.MapDiff} object representing the difference between the maps + */ + DiffableUtils.MapDiff> provideDiff(); + +} diff --git a/server/src/main/java/org/opensearch/cluster/routing/remote/InternalRemoteRoutingTableService.java b/server/src/main/java/org/opensearch/cluster/routing/remote/InternalRemoteRoutingTableService.java index 0f1ff3138ef90..220093b428989 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/remote/InternalRemoteRoutingTableService.java +++ b/server/src/main/java/org/opensearch/cluster/routing/remote/InternalRemoteRoutingTableService.java @@ -13,10 +13,10 @@ import org.apache.logging.log4j.message.ParameterizedMessage; import org.opensearch.action.LatchedActionListener; import org.opensearch.cluster.Diff; -import org.opensearch.cluster.DiffableUtils; import org.opensearch.cluster.routing.IndexRoutingTable; import org.opensearch.cluster.routing.RoutingTable; import org.opensearch.cluster.routing.RoutingTableIncrementalDiff; +import org.opensearch.cluster.routing.StringKeyDiffProvider; import org.opensearch.common.blobstore.BlobPath; import org.opensearch.common.lifecycle.AbstractLifecycleComponent; import org.opensearch.common.remote.RemoteWritableEntityStore; @@ -63,7 +63,7 @@ public class InternalRemoteRoutingTableService extends AbstractLifecycleComponen private final Supplier repositoriesService; private Compressor compressor; private RemoteWritableEntityStore remoteIndexRoutingTableStore; - private RemoteWritableEntityStore remoteRoutingTableDiffStore; + private RemoteWritableEntityStore, RemoteRoutingTableDiff> remoteRoutingTableDiffStore; private final ClusterSettings clusterSettings; private BlobStoreRepository blobStoreRepository; private final ThreadPool threadPool; @@ -95,16 +95,9 @@ public List getIndicesRouting(RoutingTable routingTable) { * @param after current routing table * @return incremental diff of the previous and current routing table */ - public DiffableUtils.MapDiff> getIndicesRoutingMapDiff( - RoutingTable before, - RoutingTable after - ) { - return DiffableUtils.diff( - before.getIndicesRouting(), - after.getIndicesRouting(), - DiffableUtils.getStringKeySerializer(), - CUSTOM_ROUTING_TABLE_DIFFABLE_VALUE_SERIALIZER - ); + @Override + public StringKeyDiffProvider getIndicesRoutingMapDiff(RoutingTable before, RoutingTable after) { + return new RoutingTableIncrementalDiff(before, after); } /** @@ -142,18 +135,16 @@ public void getAsyncIndexRoutingDiffWriteAction( String clusterUUID, long term, long version, - Map> indexRoutingTableDiff, + StringKeyDiffProvider routingTableDiff, LatchedActionListener latchedActionListener ) { - RoutingTableIncrementalDiff routingTableIncrementalDiff = new RoutingTableIncrementalDiff(indexRoutingTableDiff); RemoteRoutingTableDiff remoteRoutingTableDiff = new RemoteRoutingTableDiff( - routingTableIncrementalDiff, + (RoutingTableIncrementalDiff) routingTableDiff, clusterUUID, compressor, term, version ); - ActionListener completionListener = ActionListener.wrap( resp -> latchedActionListener.onResponse(remoteRoutingTableDiff.getUploadedMetadata()), ex -> latchedActionListener.onFailure( @@ -209,15 +200,14 @@ public void getAsyncIndexRoutingReadAction( public void getAsyncIndexRoutingTableDiffReadAction( String clusterUUID, String uploadedFilename, - LatchedActionListener latchedActionListener + LatchedActionListener> latchedActionListener ) { - ActionListener actionListener = ActionListener.wrap( + ActionListener> actionListener = ActionListener.wrap( latchedActionListener::onResponse, latchedActionListener::onFailure ); RemoteRoutingTableDiff remoteRoutingTableDiff = new RemoteRoutingTableDiff(uploadedFilename, clusterUUID, compressor); - remoteRoutingTableDiffStore.readAsync(remoteRoutingTableDiff, actionListener); } diff --git a/server/src/main/java/org/opensearch/cluster/routing/remote/NoopRemoteRoutingTableService.java b/server/src/main/java/org/opensearch/cluster/routing/remote/NoopRemoteRoutingTableService.java index 1ebf3206212a1..17687199c39d6 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/remote/NoopRemoteRoutingTableService.java +++ b/server/src/main/java/org/opensearch/cluster/routing/remote/NoopRemoteRoutingTableService.java @@ -10,16 +10,15 @@ import org.opensearch.action.LatchedActionListener; import org.opensearch.cluster.Diff; -import org.opensearch.cluster.DiffableUtils; import org.opensearch.cluster.routing.IndexRoutingTable; import org.opensearch.cluster.routing.RoutingTable; import org.opensearch.cluster.routing.RoutingTableIncrementalDiff; +import org.opensearch.cluster.routing.StringKeyDiffProvider; import org.opensearch.common.lifecycle.AbstractLifecycleComponent; import org.opensearch.gateway.remote.ClusterMetadataManifest; import java.io.IOException; import java.util.List; -import java.util.Map; /** * Noop impl for RemoteRoutingTableService. @@ -32,16 +31,8 @@ public List getIndicesRouting(RoutingTable routingTable) { } @Override - public DiffableUtils.MapDiff> getIndicesRoutingMapDiff( - RoutingTable before, - RoutingTable after - ) { - return DiffableUtils.diff( - Map.of(), - Map.of(), - DiffableUtils.getStringKeySerializer(), - CUSTOM_ROUTING_TABLE_DIFFABLE_VALUE_SERIALIZER - ); + public StringKeyDiffProvider getIndicesRoutingMapDiff(RoutingTable before, RoutingTable after) { + return new RoutingTableIncrementalDiff(RoutingTable.builder().build(), RoutingTable.builder().build()); } @Override @@ -60,7 +51,7 @@ public void getAsyncIndexRoutingDiffWriteAction( String clusterUUID, long term, long version, - Map> indexRoutingTableDiff, + StringKeyDiffProvider routingTableDiff, LatchedActionListener latchedActionListener ) { // noop @@ -89,7 +80,7 @@ public void getAsyncIndexRoutingReadAction( public void getAsyncIndexRoutingTableDiffReadAction( String clusterUUID, String uploadedFilename, - LatchedActionListener latchedActionListener + LatchedActionListener> latchedActionListener ) { // noop } diff --git a/server/src/main/java/org/opensearch/cluster/routing/remote/RemoteRoutingTableService.java b/server/src/main/java/org/opensearch/cluster/routing/remote/RemoteRoutingTableService.java index 0811a5f3010f4..d7ef3a29aa21f 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/remote/RemoteRoutingTableService.java +++ b/server/src/main/java/org/opensearch/cluster/routing/remote/RemoteRoutingTableService.java @@ -10,20 +10,14 @@ import org.opensearch.action.LatchedActionListener; import org.opensearch.cluster.Diff; -import org.opensearch.cluster.DiffableUtils; import org.opensearch.cluster.routing.IndexRoutingTable; -import org.opensearch.cluster.routing.IndexShardRoutingTable; import org.opensearch.cluster.routing.RoutingTable; -import org.opensearch.cluster.routing.RoutingTableIncrementalDiff; +import org.opensearch.cluster.routing.StringKeyDiffProvider; import org.opensearch.common.lifecycle.LifecycleComponent; -import org.opensearch.core.common.io.stream.StreamInput; -import org.opensearch.core.common.io.stream.StreamOutput; import org.opensearch.gateway.remote.ClusterMetadataManifest; import java.io.IOException; -import java.util.ArrayList; import java.util.List; -import java.util.Map; /** * A Service which provides APIs to upload and download routing table from remote store. @@ -32,38 +26,6 @@ */ public interface RemoteRoutingTableService extends LifecycleComponent { - public static final DiffableUtils.DiffableValueSerializer CUSTOM_ROUTING_TABLE_DIFFABLE_VALUE_SERIALIZER = - new DiffableUtils.DiffableValueSerializer() { - @Override - public IndexRoutingTable read(StreamInput in, String key) throws IOException { - return IndexRoutingTable.readFrom(in); - } - - @Override - public void write(IndexRoutingTable value, StreamOutput out) throws IOException { - value.writeTo(out); - } - - @Override - public Diff readDiff(StreamInput in, String key) throws IOException { - return IndexRoutingTable.readDiffFrom(in); - } - - @Override - public Diff diff(IndexRoutingTable currentState, IndexRoutingTable previousState) { - List diffs = new ArrayList<>(); - for (Map.Entry entry : currentState.getShards().entrySet()) { - Integer index = entry.getKey(); - IndexShardRoutingTable currentShardRoutingTable = entry.getValue(); - IndexShardRoutingTable previousShardRoutingTable = previousState.shard(index); - if (previousShardRoutingTable == null || !previousShardRoutingTable.equals(currentShardRoutingTable)) { - diffs.add(currentShardRoutingTable); - } - } - return new RoutingTableIncrementalDiff.IndexRoutingTableIncrementalDiff(diffs); - } - }; - List getIndicesRouting(RoutingTable routingTable); void getAsyncIndexRoutingReadAction( @@ -75,7 +37,7 @@ void getAsyncIndexRoutingReadAction( void getAsyncIndexRoutingTableDiffReadAction( String clusterUUID, String uploadedFilename, - LatchedActionListener latchedActionListener + LatchedActionListener> latchedActionListener ); List getUpdatedIndexRoutingTableMetadata( @@ -83,10 +45,7 @@ List getUpdatedIndexRoutingTableM List allIndicesRouting ); - DiffableUtils.MapDiff> getIndicesRoutingMapDiff( - RoutingTable before, - RoutingTable after - ); + StringKeyDiffProvider getIndicesRoutingMapDiff(RoutingTable before, RoutingTable after); void getAsyncIndexRoutingWriteAction( String clusterUUID, @@ -100,7 +59,7 @@ void getAsyncIndexRoutingDiffWriteAction( String clusterUUID, long term, long version, - Map> indexRoutingTableDiff, + StringKeyDiffProvider routingTableDiff, LatchedActionListener latchedActionListener ); @@ -110,8 +69,8 @@ List getAllUploadedIndicesRouting List indicesRoutingToDelete ); - public void deleteStaleIndexRoutingPaths(List stalePaths) throws IOException; + void deleteStaleIndexRoutingPaths(List stalePaths) throws IOException; - public void deleteStaleIndexRoutingDiffPaths(List stalePaths) throws IOException; + void deleteStaleIndexRoutingDiffPaths(List stalePaths) throws IOException; } diff --git a/server/src/main/java/org/opensearch/gateway/remote/ClusterMetadataManifest.java b/server/src/main/java/org/opensearch/gateway/remote/ClusterMetadataManifest.java index ca730a73dd89b..8025940b35997 100644 --- a/server/src/main/java/org/opensearch/gateway/remote/ClusterMetadataManifest.java +++ b/server/src/main/java/org/opensearch/gateway/remote/ClusterMetadataManifest.java @@ -45,9 +45,9 @@ public class ClusterMetadataManifest implements Writeable, ToXContentFragment { // also we introduce index routing-metadata, diff and other attributes as part of manifest // required for state publication public static final int CODEC_V3 = 3; // In Codec V3, we have introduced new diff field in diff-manifest's routing_table_diff + public static final int CODEC_V4 = 4; // In Codec V4, we have removed upserts and delete field for routing table in diff manifest - public static final int[] CODEC_VERSIONS = { CODEC_V0, CODEC_V1, CODEC_V2, CODEC_V3 }; - + public static final int[] CODEC_VERSIONS = { CODEC_V0, CODEC_V1, CODEC_V2, CODEC_V3, CODEC_V4 }; private static final ParseField CLUSTER_TERM_FIELD = new ParseField("cluster_term"); private static final ParseField STATE_VERSION_FIELD = new ParseField("state_version"); private static final ParseField CLUSTER_UUID_FIELD = new ParseField("cluster_uuid"); @@ -116,6 +116,10 @@ private static ClusterMetadataManifest.Builder manifestV3Builder(Object[] fields return manifestV2Builder(fields); } + private static ClusterMetadataManifest.Builder manifestV4Builder(Object[] fields) { + return manifestV3Builder(fields); + } + private static long term(Object[] fields) { return (long) fields[0]; } @@ -238,8 +242,13 @@ private static ClusterStateDiffManifest diffManifest(Object[] fields) { fields -> manifestV3Builder(fields).build() ); - private static final ConstructingObjectParser CURRENT_PARSER = PARSER_V3; - public static final int MANIFEST_CURRENT_CODEC_VERSION = CODEC_V3; + private static final ConstructingObjectParser PARSER_V4 = new ConstructingObjectParser<>( + "cluster_metadata_manifest", + fields -> manifestV4Builder(fields).build() + ); + + private static final ConstructingObjectParser CURRENT_PARSER = PARSER_V4; + public static final int MANIFEST_CURRENT_CODEC_VERSION = CODEC_V4; private static final Map VERSION_TO_CODEC_MAPPING; @@ -248,6 +257,7 @@ private static ClusterStateDiffManifest diffManifest(Object[] fields) { declareParser(PARSER_V1, CODEC_V1); declareParser(PARSER_V2, CODEC_V2); declareParser(PARSER_V3, CODEC_V3); + declareParser(PARSER_V4, CODEC_V4); assert Arrays.stream(CODEC_VERSIONS).max().getAsInt() == MANIFEST_CURRENT_CODEC_VERSION; Map versionToCodecMapping = new HashMap<>(); @@ -258,8 +268,10 @@ private static ClusterStateDiffManifest diffManifest(Object[] fields) { versionToCodecMapping.put(version, ClusterMetadataManifest.CODEC_V1); } else if (version.onOrAfter(Version.V_2_15_0) && version.before(Version.V_2_16_0)) { versionToCodecMapping.put(version, ClusterMetadataManifest.CODEC_V2); - } else if (version.onOrAfter(Version.V_2_16_0)) { + } else if (version.onOrAfter(Version.V_2_16_0) && version.before(Version.V_2_17_0)) { versionToCodecMapping.put(version, ClusterMetadataManifest.CODEC_V3); + } else if (version.onOrAfter(Version.V_2_17_0)) { + versionToCodecMapping.put(version, ClusterMetadataManifest.CODEC_V4); } } VERSION_TO_CODEC_MAPPING = Collections.unmodifiableMap(versionToCodecMapping); @@ -860,6 +872,10 @@ public static ClusterMetadataManifest fromXContentV2(XContentParser parser) thro return PARSER_V2.parse(parser, null); } + public static ClusterMetadataManifest fromXContentV3(XContentParser parser) throws IOException { + return PARSER_V3.parse(parser, null); + } + public static ClusterMetadataManifest fromXContent(XContentParser parser) throws IOException { return CURRENT_PARSER.parse(parser, null); } diff --git a/server/src/main/java/org/opensearch/gateway/remote/ClusterStateDiffManifest.java b/server/src/main/java/org/opensearch/gateway/remote/ClusterStateDiffManifest.java index a3b36ddcff1a7..a97b3e1bc6c4b 100644 --- a/server/src/main/java/org/opensearch/gateway/remote/ClusterStateDiffManifest.java +++ b/server/src/main/java/org/opensearch/gateway/remote/ClusterStateDiffManifest.java @@ -13,6 +13,7 @@ import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.cluster.metadata.Metadata; import org.opensearch.cluster.routing.IndexRoutingTable; +import org.opensearch.cluster.routing.StringKeyDiffProvider; import org.opensearch.core.common.Strings; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; @@ -33,6 +34,7 @@ import static org.opensearch.cluster.DiffableUtils.NonDiffableValueSerializer.getAbstractInstance; import static org.opensearch.cluster.DiffableUtils.getStringKeySerializer; import static org.opensearch.core.xcontent.XContentParserUtils.ensureExpectedToken; +import static org.opensearch.gateway.remote.ClusterMetadataManifest.CODEC_V2; import static org.opensearch.gateway.remote.ClusterMetadataManifest.CODEC_V3; /** @@ -81,7 +83,8 @@ public class ClusterStateDiffManifest implements ToXContentFragment, Writeable { public ClusterStateDiffManifest( ClusterState state, ClusterState previousState, - DiffableUtils.MapDiff> routingTableIncrementalDiff, + int codecVersion, + StringKeyDiffProvider routingTableDiff, String indicesRoutingDiffPath ) { fromStateUUID = previousState.stateUUID(); @@ -109,13 +112,14 @@ public ClusterStateDiffManifest( customMetadataUpdated = new ArrayList<>(customDiff.getDiffs().keySet()); customMetadataUpdated.addAll(customDiff.getUpserts().keySet()); customMetadataDeleted = customDiff.getDeletes(); - indicesRoutingUpdated = new ArrayList<>(); indicesRoutingDeleted = new ArrayList<>(); this.indicesRoutingDiffPath = indicesRoutingDiffPath; - if (routingTableIncrementalDiff != null) { - routingTableIncrementalDiff.getUpserts().forEach((k, v) -> indicesRoutingUpdated.add(k)); - indicesRoutingDeleted.addAll(routingTableIncrementalDiff.getDeletes()); + if (codecVersion == CODEC_V2 || codecVersion == CODEC_V3) { + if (routingTableDiff != null && routingTableDiff.provideDiff() != null) { + routingTableDiff.provideDiff().getUpserts().forEach((k, v) -> indicesRoutingUpdated.add(k)); + indicesRoutingDeleted.addAll(routingTableDiff.provideDiff().getDeletes()); + } } hashesOfConsistentSettingsUpdated = !state.metadata() .hashesOfConsistentSettings() @@ -231,18 +235,21 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws builder.endObject(); builder.field(CLUSTER_BLOCKS_UPDATED_FIELD, clusterBlocksUpdated); builder.field(DISCOVERY_NODES_UPDATED_FIELD, discoveryNodesUpdated); - builder.startObject(ROUTING_TABLE_DIFF); - builder.startArray(UPSERTS_FIELD); - for (String index : indicesRoutingUpdated) { - builder.value(index); + if (!indicesRoutingUpdated.isEmpty()) { + builder.startArray(UPSERTS_FIELD); + for (String index : indicesRoutingUpdated) { + builder.value(index); + } + builder.endArray(); } - builder.endArray(); - builder.startArray(DELETES_FIELD); - for (String index : indicesRoutingDeleted) { - builder.value(index); + if (!indicesRoutingDeleted.isEmpty()) { + builder.startArray(DELETES_FIELD); + for (String index : indicesRoutingDeleted) { + builder.value(index); + } + builder.endArray(); } - builder.endArray(); if (indicesRoutingDiffPath != null) { builder.field(DIFF_FIELD, indicesRoutingDiffPath); } @@ -345,10 +352,14 @@ public static ClusterStateDiffManifest fromXContent(XContentParser parser, long parser.nextToken(); switch (currentFieldName) { case UPSERTS_FIELD: - builder.indicesRoutingUpdated(convertListToString(parser.listOrderedMap())); + if (codec_version == CODEC_V2 || codec_version == CODEC_V3) { + builder.indicesRoutingUpdated(convertListToString(parser.listOrderedMap())); + } break; case DELETES_FIELD: - builder.indicesRoutingDeleted(convertListToString(parser.listOrderedMap())); + if (codec_version == CODEC_V2 || codec_version == CODEC_V3) { + builder.indicesRoutingDeleted(convertListToString(parser.listOrderedMap())); + } break; case DIFF_FIELD: if (codec_version >= CODEC_V3) { @@ -466,14 +477,14 @@ public boolean isHashesOfConsistentSettingsUpdated() { return hashesOfConsistentSettingsUpdated; } - public List getIndicesRoutingUpdated() { - return indicesRoutingUpdated; - } - public String getIndicesRoutingDiffPath() { return indicesRoutingDiffPath; } + public List getIndicesRoutingUpdated() { + return indicesRoutingUpdated; + } + public List getIndicesRoutingDeleted() { return indicesRoutingDeleted; } @@ -666,6 +677,11 @@ public Builder discoveryNodesUpdated(boolean discoveryNodesUpdated) { return this; } + public Builder indicesRoutingDiffPath(String indicesRoutingDiffPath) { + this.indicesRoutingDiff = indicesRoutingDiffPath; + return this; + } + public Builder indicesRoutingUpdated(List indicesRoutingUpdated) { this.indicesRoutingUpdated = indicesRoutingUpdated; return this; @@ -676,11 +692,6 @@ public Builder indicesRoutingDeleted(List indicesRoutingDeleted) { return this; } - public Builder indicesRoutingDiffPath(String indicesRoutingDiffPath) { - this.indicesRoutingDiff = indicesRoutingDiffPath; - return this; - } - public Builder clusterStateCustomUpdated(List clusterStateCustomUpdated) { this.clusterStateCustomUpdated = clusterStateCustomUpdated; return this; diff --git a/server/src/main/java/org/opensearch/gateway/remote/RemoteClusterStateService.java b/server/src/main/java/org/opensearch/gateway/remote/RemoteClusterStateService.java index 2b3913ced0144..098a36c7d42c4 100644 --- a/server/src/main/java/org/opensearch/gateway/remote/RemoteClusterStateService.java +++ b/server/src/main/java/org/opensearch/gateway/remote/RemoteClusterStateService.java @@ -28,7 +28,7 @@ import org.opensearch.cluster.node.DiscoveryNodes.Builder; import org.opensearch.cluster.routing.IndexRoutingTable; import org.opensearch.cluster.routing.RoutingTable; -import org.opensearch.cluster.routing.RoutingTableIncrementalDiff; +import org.opensearch.cluster.routing.StringKeyDiffProvider; import org.opensearch.cluster.routing.remote.RemoteRoutingTableService; import org.opensearch.cluster.routing.remote.RemoteRoutingTableServiceFactory; import org.opensearch.cluster.service.ClusterService; @@ -93,6 +93,8 @@ import static org.opensearch.common.util.FeatureFlags.REMOTE_PUBLICATION_EXPERIMENTAL; import static org.opensearch.gateway.PersistedClusterStateService.SLOW_WRITE_LOGGING_THRESHOLD; import static org.opensearch.gateway.remote.ClusterMetadataManifest.CODEC_V2; +import static org.opensearch.gateway.remote.ClusterMetadataManifest.CODEC_V3; +import static org.opensearch.gateway.remote.ClusterMetadataManifest.MANIFEST_CURRENT_CODEC_VERSION; import static org.opensearch.gateway.remote.RemoteClusterStateAttributesManager.CLUSTER_BLOCKS; import static org.opensearch.gateway.remote.RemoteClusterStateAttributesManager.CLUSTER_STATE_ATTRIBUTE; import static org.opensearch.gateway.remote.RemoteClusterStateAttributesManager.DISCOVERY_NODES; @@ -246,6 +248,7 @@ public RemoteClusterStateManifestInfo writeFullMetadata(ClusterState clusterStat ClusterStateDiffManifest clusterStateDiffManifest = new ClusterStateDiffManifest( clusterState, ClusterState.EMPTY_STATE, + MANIFEST_CURRENT_CODEC_VERSION, null, null ); @@ -345,12 +348,18 @@ public RemoteClusterStateManifestInfo writeIncrementalMetadata( } final List indicesRoutingToUpload = new ArrayList<>(); - final DiffableUtils.MapDiff> routingTableIncrementalDiff = - remoteRoutingTableService.getIndicesRoutingMapDiff(previousClusterState.getRoutingTable(), clusterState.getRoutingTable()); - - Map> indexRoutingTableDiffs = routingTableIncrementalDiff.getDiffs(); - routingTableIncrementalDiff.getDiffs().forEach((k, v) -> indicesRoutingToUpload.add(clusterState.getRoutingTable().index(k))); - routingTableIncrementalDiff.getUpserts().forEach((k, v) -> indicesRoutingToUpload.add(v)); + final List deletedIndicesRouting = new ArrayList<>(); + final StringKeyDiffProvider routingTableDiff = remoteRoutingTableService.getIndicesRoutingMapDiff( + previousClusterState.getRoutingTable(), + clusterState.getRoutingTable() + ); + if (routingTableDiff != null && routingTableDiff.provideDiff() != null) { + routingTableDiff.provideDiff() + .getDiffs() + .forEach((k, v) -> indicesRoutingToUpload.add(clusterState.getRoutingTable().index(k))); + routingTableDiff.provideDiff().getUpserts().forEach((k, v) -> indicesRoutingToUpload.add(v)); + deletedIndicesRouting.addAll(routingTableDiff.provideDiff().getDeletes()); + } UploadedMetadataResults uploadedMetadataResults; // For migration case from codec V0 or V1 to V2, we have added null check on metadata attribute files, @@ -387,7 +396,7 @@ public RemoteClusterStateManifestInfo writeIncrementalMetadata( clusterStateCustomsDiff.getUpserts(), updateHashesOfConsistentSettings, indicesRoutingToUpload, - indexRoutingTableDiffs + routingTableDiff ); // update the map if the metadata was uploaded @@ -429,13 +438,14 @@ public RemoteClusterStateManifestInfo writeIncrementalMetadata( uploadedMetadataResults.uploadedIndicesRoutingMetadata = remoteRoutingTableService.getAllUploadedIndicesRouting( previousManifest, uploadedMetadataResults.uploadedIndicesRoutingMetadata, - routingTableIncrementalDiff.getDeletes() + deletedIndicesRouting ); ClusterStateDiffManifest clusterStateDiffManifest = new ClusterStateDiffManifest( clusterState, previousClusterState, - routingTableIncrementalDiff, + MANIFEST_CURRENT_CODEC_VERSION, + routingTableDiff, uploadedMetadataResults.uploadedIndicesRoutingDiffMetadata != null ? uploadedMetadataResults.uploadedIndicesRoutingDiffMetadata.getUploadedFilename() : null @@ -517,14 +527,18 @@ UploadedMetadataResults writeMetadataInParallel( Map clusterStateCustomToUpload, boolean uploadHashesOfConsistentSettings, List indicesRoutingToUpload, - Map> indexRoutingTableDiff + StringKeyDiffProvider routingTableDiff ) throws IOException { assert Objects.nonNull(indexMetadataUploadListeners) : "indexMetadataUploadListeners can not be null"; int totalUploadTasks = indexToUpload.size() + indexMetadataUploadListeners.size() + customToUpload.size() + (uploadCoordinationMetadata ? 1 : 0) + (uploadSettingsMetadata ? 1 : 0) + (uploadTemplateMetadata ? 1 : 0) + (uploadDiscoveryNodes ? 1 : 0) + (uploadClusterBlock ? 1 : 0) + (uploadTransientSettingMetadata ? 1 : 0) + clusterStateCustomToUpload.size() + (uploadHashesOfConsistentSettings ? 1 : 0) + indicesRoutingToUpload.size() - + (indexRoutingTableDiff != null && !indexRoutingTableDiff.isEmpty() ? 1 : 0); + + ((routingTableDiff != null + && routingTableDiff.provideDiff() != null + && (!routingTableDiff.provideDiff().getDiffs().isEmpty() + || !routingTableDiff.provideDiff().getDeletes().isEmpty() + || !routingTableDiff.provideDiff().getUpserts().isEmpty())) ? 1 : 0); CountDownLatch latch = new CountDownLatch(totalUploadTasks); List uploadTasks = Collections.synchronizedList(new ArrayList<>(totalUploadTasks)); Map results = new ConcurrentHashMap<>(totalUploadTasks); @@ -694,13 +708,17 @@ UploadedMetadataResults writeMetadataInParallel( listener ); }); - if (indexRoutingTableDiff != null && !indexRoutingTableDiff.isEmpty()) { + if (routingTableDiff != null + && routingTableDiff.provideDiff() != null + && (!routingTableDiff.provideDiff().getDiffs().isEmpty() + || !routingTableDiff.provideDiff().getDeletes().isEmpty() + || !routingTableDiff.provideDiff().getUpserts().isEmpty())) { uploadTasks.add(RemoteRoutingTableDiff.ROUTING_TABLE_DIFF_FILE); remoteRoutingTableService.getAsyncIndexRoutingDiffWriteAction( clusterState.metadata().clusterUUID(), clusterState.term(), clusterState.version(), - indexRoutingTableDiff, + routingTableDiff, listener ); } @@ -1024,7 +1042,7 @@ ClusterState readClusterStateInParallel( Map clusterStateCustomToRead, boolean readIndexRoutingTableDiff, boolean includeEphemeral - ) throws IOException { + ) { int totalReadTasks = indicesToRead.size() + customToRead.size() + (readCoordinationMetadata ? 1 : 0) + (readSettingsMetadata ? 1 : 0) + (readTemplatesMetadata ? 1 : 0) + (readDiscoveryNodes ? 1 : 0) + (readClusterBlocks ? 1 : 0) @@ -1033,7 +1051,7 @@ ClusterState readClusterStateInParallel( CountDownLatch latch = new CountDownLatch(totalReadTasks); List readResults = Collections.synchronizedList(new ArrayList<>()); List readIndexRoutingTableResults = Collections.synchronizedList(new ArrayList<>()); - AtomicReference readIndexRoutingTableDiffResults = new AtomicReference<>(); + AtomicReference> readIndexRoutingTableDiffResults = new AtomicReference<>(); List exceptionList = Collections.synchronizedList(new ArrayList<>(totalReadTasks)); LatchedActionListener listener = new LatchedActionListener<>(ActionListener.wrap(response -> { @@ -1076,7 +1094,7 @@ ClusterState readClusterStateInParallel( ); } - LatchedActionListener routingTableDiffLatchedActionListener = new LatchedActionListener<>( + LatchedActionListener> routingTableDiffLatchedActionListener = new LatchedActionListener<>( ActionListener.wrap(response -> { logger.debug("Successfully read routing table diff component from remote"); readIndexRoutingTableDiffResults.set(response); @@ -1297,15 +1315,12 @@ ClusterState readClusterStateInParallel( readIndexRoutingTableResults.forEach( indexRoutingTable -> indicesRouting.put(indexRoutingTable.getIndex().getName(), indexRoutingTable) ); - RoutingTableIncrementalDiff routingTableDiff = readIndexRoutingTableDiffResults.get(); + Diff routingTableDiff = readIndexRoutingTableDiffResults.get(); + RoutingTable newRoutingTable = new RoutingTable(manifest.getRoutingTableVersion(), indicesRouting); if (routingTableDiff != null) { - routingTableDiff.getDiffs().forEach((key, diff) -> { - IndexRoutingTable previousIndexRoutingTable = indicesRouting.get(key); - IndexRoutingTable updatedTable = diff.apply(previousIndexRoutingTable); - indicesRouting.put(key, updatedTable); - }); + newRoutingTable = routingTableDiff.apply(previousState.getRoutingTable()); } - clusterStateBuilder.routingTable(new RoutingTable(manifest.getRoutingTableVersion(), indicesRouting)); + clusterStateBuilder.routingTable(newRoutingTable); return clusterStateBuilder.build(); } @@ -1370,8 +1385,7 @@ public ClusterState getClusterStateForManifest( return clusterState; } - public ClusterState getClusterStateUsingDiff(ClusterMetadataManifest manifest, ClusterState previousState, String localNodeId) - throws IOException { + public ClusterState getClusterStateUsingDiff(ClusterMetadataManifest manifest, ClusterState previousState, String localNodeId) { assert manifest.getDiffManifest() != null : "Diff manifest null which is required for downloading cluster state"; final long startTimeNanos = relativeTimeNanosSupplier.getAsLong(); ClusterStateDiffManifest diff = manifest.getDiffManifest(); @@ -1398,9 +1412,11 @@ public ClusterState getClusterStateUsingDiff(ClusterMetadataManifest manifest, C } List updatedIndexRouting = new ArrayList<>(); - updatedIndexRouting.addAll( - remoteRoutingTableService.getUpdatedIndexRoutingTableMetadata(diff.getIndicesRoutingUpdated(), manifest.getIndicesRouting()) - ); + if (manifest.getCodecVersion() == CODEC_V2 || manifest.getCodecVersion() == CODEC_V3) { + updatedIndexRouting.addAll( + remoteRoutingTableService.getUpdatedIndexRoutingTableMetadata(diff.getIndicesRoutingUpdated(), manifest.getIndicesRouting()) + ); + } ClusterState updatedClusterState = readClusterStateInParallel( previousState, @@ -1444,9 +1460,10 @@ public ClusterState getClusterStateUsingDiff(ClusterMetadataManifest manifest, C } HashMap indexRoutingTables = new HashMap<>(updatedClusterState.getRoutingTable().getIndicesRouting()); - - for (String indexName : diff.getIndicesRoutingDeleted()) { - indexRoutingTables.remove(indexName); + if (manifest.getCodecVersion() == CODEC_V2 || manifest.getCodecVersion() == CODEC_V3) { + for (String indexName : diff.getIndicesRoutingDeleted()) { + indexRoutingTables.remove(indexName); + } } ClusterState clusterState = clusterStateBuilder.stateUUID(manifest.getStateUUID()) diff --git a/server/src/main/java/org/opensearch/gateway/remote/model/RemoteClusterMetadataManifest.java b/server/src/main/java/org/opensearch/gateway/remote/model/RemoteClusterMetadataManifest.java index e890c30700eb6..9460e240e1706 100644 --- a/server/src/main/java/org/opensearch/gateway/remote/model/RemoteClusterMetadataManifest.java +++ b/server/src/main/java/org/opensearch/gateway/remote/model/RemoteClusterMetadataManifest.java @@ -52,6 +52,9 @@ public class RemoteClusterMetadataManifest extends AbstractClusterMetadataWritea public static final ChecksumBlobStoreFormat CLUSTER_METADATA_MANIFEST_FORMAT_V2 = new ChecksumBlobStoreFormat<>("cluster-metadata-manifest", METADATA_MANIFEST_NAME_FORMAT, ClusterMetadataManifest::fromXContentV2); + public static final ChecksumBlobStoreFormat CLUSTER_METADATA_MANIFEST_FORMAT_V3 = + new ChecksumBlobStoreFormat<>("cluster-metadata-manifest", METADATA_MANIFEST_NAME_FORMAT, ClusterMetadataManifest::fromXContentV3); + /** * Manifest format compatible with codec v2, where we introduced codec versions/global metadata. */ @@ -151,6 +154,8 @@ private ChecksumBlobStoreFormat getClusterMetadataManif long codecVersion = getManifestCodecVersion(); if (codecVersion == ClusterMetadataManifest.MANIFEST_CURRENT_CODEC_VERSION) { return CLUSTER_METADATA_MANIFEST_FORMAT; + } else if (codecVersion == ClusterMetadataManifest.CODEC_V3) { + return CLUSTER_METADATA_MANIFEST_FORMAT_V3; } else if (codecVersion == ClusterMetadataManifest.CODEC_V2) { return CLUSTER_METADATA_MANIFEST_FORMAT_V2; } else if (codecVersion == ClusterMetadataManifest.CODEC_V1) { diff --git a/server/src/main/java/org/opensearch/gateway/remote/routingtable/RemoteRoutingTableDiff.java b/server/src/main/java/org/opensearch/gateway/remote/routingtable/RemoteRoutingTableDiff.java index 2370417dc14df..b3e0e9e5763b7 100644 --- a/server/src/main/java/org/opensearch/gateway/remote/routingtable/RemoteRoutingTableDiff.java +++ b/server/src/main/java/org/opensearch/gateway/remote/routingtable/RemoteRoutingTableDiff.java @@ -10,6 +10,7 @@ import org.opensearch.cluster.Diff; import org.opensearch.cluster.routing.IndexRoutingTable; +import org.opensearch.cluster.routing.RoutingTable; import org.opensearch.cluster.routing.RoutingTableIncrementalDiff; import org.opensearch.common.io.Streams; import org.opensearch.common.remote.AbstractClusterMetadataWriteableBlobEntity; @@ -22,7 +23,6 @@ import java.io.IOException; import java.io.InputStream; import java.util.List; -import java.util.Map; import static org.opensearch.gateway.remote.RemoteClusterStateUtils.DELIMITER; @@ -30,7 +30,8 @@ * Represents a incremental difference between {@link org.opensearch.cluster.routing.RoutingTable} objects that can be serialized and deserialized. * This class is responsible for writing and reading the differences between RoutingTables to and from an input/output stream. */ -public class RemoteRoutingTableDiff extends AbstractClusterMetadataWriteableBlobEntity { +public class RemoteRoutingTableDiff extends AbstractClusterMetadataWriteableBlobEntity> { + private final RoutingTableIncrementalDiff routingTableIncrementalDiff; private long term; @@ -71,18 +72,6 @@ public RemoteRoutingTableDiff( this.version = version; } - /** - * Constructs a new RemoteRoutingTableDiff with the given differences. - * - * @param routingTableIncrementalDiff a RoutingTableIncrementalDiff object containing the differences of {@link IndexRoutingTable}. - * @param clusterUUID the cluster UUID. - * @param compressor the compressor to be used. - */ - public RemoteRoutingTableDiff(RoutingTableIncrementalDiff routingTableIncrementalDiff, String clusterUUID, Compressor compressor) { - super(clusterUUID, compressor); - this.routingTableIncrementalDiff = routingTableIncrementalDiff; - } - /** * Constructs a new RemoteIndexRoutingTableDiff with the given blob name, cluster UUID, and compressor. * @@ -101,9 +90,8 @@ public RemoteRoutingTableDiff(String blobName, String clusterUUID, Compressor co * * @return a map containing the differences. */ - public Map> getDiffs() { - assert routingTableIncrementalDiff != null; - return routingTableIncrementalDiff.getDiffs(); + public Diff getDiffs() { + return routingTableIncrementalDiff; } @Override @@ -144,7 +132,7 @@ public InputStream serialize() throws IOException { } @Override - public RoutingTableIncrementalDiff deserialize(InputStream in) throws IOException { + public Diff deserialize(InputStream in) throws IOException { return REMOTE_ROUTING_TABLE_DIFF_FORMAT.deserialize(blobName, Streams.readFully(in)); } } diff --git a/server/src/test/java/org/opensearch/cluster/routing/RoutingTableDiffTests.java b/server/src/test/java/org/opensearch/cluster/routing/RoutingTableDiffTests.java new file mode 100644 index 0000000000000..d8c93523c34b9 --- /dev/null +++ b/server/src/test/java/org/opensearch/cluster/routing/RoutingTableDiffTests.java @@ -0,0 +1,325 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.cluster.routing; + +import org.opensearch.Version; +import org.opensearch.cluster.ClusterState; +import org.opensearch.cluster.Diff; +import org.opensearch.cluster.OpenSearchAllocationTestCase; +import org.opensearch.cluster.metadata.IndexMetadata; +import org.opensearch.cluster.metadata.Metadata; +import org.opensearch.cluster.node.DiscoveryNodes; +import org.opensearch.cluster.routing.allocation.AllocationService; +import org.opensearch.cluster.routing.allocation.decider.ThrottlingAllocationDecider; +import org.opensearch.common.settings.Settings; +import org.junit.Before; + +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.is; +import static org.hamcrest.Matchers.not; + +public class RoutingTableDiffTests extends OpenSearchAllocationTestCase { + + private static final String TEST_INDEX_1 = "test1"; + private static final String TEST_INDEX_2 = "test2"; + private static final String TEST_INDEX_3 = "test3"; + private int numberOfShards; + private int numberOfReplicas; + private int shardsPerIndex; + private int totalNumberOfShards; + private static final Settings DEFAULT_SETTINGS = Settings.builder().put(IndexMetadata.SETTING_VERSION_CREATED, Version.CURRENT).build(); + private final AllocationService ALLOCATION_SERVICE = createAllocationService( + Settings.builder() + .put("cluster.routing.allocation.node_concurrent_recoveries", Integer.MAX_VALUE) // don't limit recoveries + .put("cluster.routing.allocation.node_initial_primaries_recoveries", Integer.MAX_VALUE) + .put( + ThrottlingAllocationDecider.CLUSTER_ROUTING_ALLOCATION_NODE_INITIAL_REPLICAS_RECOVERIES_SETTING.getKey(), + Integer.MAX_VALUE + ) + .build() + ); + private ClusterState clusterState; + + @Override + @Before + public void setUp() throws Exception { + super.setUp(); + this.numberOfShards = randomIntBetween(1, 5); + this.numberOfReplicas = randomIntBetween(1, 5); + this.shardsPerIndex = this.numberOfShards * (this.numberOfReplicas + 1); + this.totalNumberOfShards = this.shardsPerIndex * 2; + logger.info("Setup test with {} shards and {} replicas.", this.numberOfShards, this.numberOfReplicas); + RoutingTable emptyRoutingTable = new RoutingTable.Builder().build(); + Metadata metadata = Metadata.builder().put(createIndexMetadata(TEST_INDEX_1)).put(createIndexMetadata(TEST_INDEX_2)).build(); + + RoutingTable testRoutingTable = new RoutingTable.Builder().add( + new IndexRoutingTable.Builder(metadata.index(TEST_INDEX_1).getIndex()).initializeAsNew(metadata.index(TEST_INDEX_1)).build() + ) + .add( + new IndexRoutingTable.Builder(metadata.index(TEST_INDEX_2).getIndex()).initializeAsNew(metadata.index(TEST_INDEX_2)).build() + ) + .build(); + this.clusterState = ClusterState.builder(org.opensearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)) + .metadata(metadata) + .routingTable(testRoutingTable) + .build(); + } + + /** + * puts primary shard indexRoutings into initializing state + */ + private void initPrimaries() { + logger.info("adding {} nodes and performing rerouting", this.numberOfReplicas + 1); + DiscoveryNodes.Builder discoBuilder = DiscoveryNodes.builder(); + for (int i = 0; i < this.numberOfReplicas + 1; i++) { + discoBuilder = discoBuilder.add(newNode("node" + i)); + } + this.clusterState = ClusterState.builder(clusterState).nodes(discoBuilder).build(); + ClusterState rerouteResult = ALLOCATION_SERVICE.reroute(clusterState, "reroute"); + assertThat(rerouteResult, not(equalTo(this.clusterState))); + this.clusterState = rerouteResult; + } + + private void startInitializingShards(String index) { + logger.info("start primary shards for index {}", index); + clusterState = startInitializingShardsAndReroute(ALLOCATION_SERVICE, clusterState, index); + } + + private IndexMetadata.Builder createIndexMetadata(String indexName) { + return new IndexMetadata.Builder(indexName).settings(DEFAULT_SETTINGS) + .numberOfReplicas(this.numberOfReplicas) + .numberOfShards(this.numberOfShards); + } + + public void testRoutingTableUpserts() { + assertThat(clusterState.routingTable().shardsWithState(ShardRoutingState.UNASSIGNED).size(), is(this.totalNumberOfShards)); + initPrimaries(); + int expectedUnassignedShardCount = this.totalNumberOfShards - 2 * this.numberOfShards; + assertThat(clusterState.routingTable().shardsWithState(ShardRoutingState.UNASSIGNED).size(), is(expectedUnassignedShardCount)); + assertThat(clusterState.routingTable().shardsWithState(ShardRoutingState.INITIALIZING).size(), is(2 * this.numberOfShards)); + Metadata metadata = Metadata.builder().put(createIndexMetadata(TEST_INDEX_1)).put(createIndexMetadata(TEST_INDEX_2)).build(); + ClusterState oldClusterState = clusterState; + + // create index routing table for TEST_INDEX_3 + metadata = Metadata.builder(metadata).put(createIndexMetadata(TEST_INDEX_3)).build(); + RoutingTable testRoutingTable = new RoutingTable.Builder(clusterState.routingTable()).add( + new IndexRoutingTable.Builder(metadata.index(TEST_INDEX_3).getIndex()).initializeAsNew(metadata.index(TEST_INDEX_3)).build() + ).build(); + this.clusterState = ClusterState.builder(org.opensearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)) + .metadata(metadata) + .routingTable(testRoutingTable) + .build(); + this.totalNumberOfShards = this.shardsPerIndex * 3; + assertThat( + clusterState.routingTable().shardsWithState(ShardRoutingState.UNASSIGNED).size(), + is(expectedUnassignedShardCount + this.shardsPerIndex) + ); + Diff fullDiff = clusterState.routingTable().diff(oldClusterState.getRoutingTable()); + Diff incrementalDiff = clusterState.routingTable().incrementalDiff(oldClusterState.getRoutingTable()); + RoutingTable newRoutingTable = incrementalDiff.apply(oldClusterState.getRoutingTable()); + for (IndexRoutingTable indexRoutingTable : clusterState.routingTable()) { + assertEquals(clusterState.routingTable().version(), newRoutingTable.version()); + assertEquals(indexRoutingTable, newRoutingTable.index(indexRoutingTable.getIndex())); + } + RoutingTable newRoutingTableWithFullDiff = fullDiff.apply(oldClusterState.getRoutingTable()); + for (IndexRoutingTable indexRoutingTable : clusterState.routingTable()) { + assertEquals(clusterState.routingTable().version(), newRoutingTableWithFullDiff.version()); + assertEquals(indexRoutingTable, newRoutingTableWithFullDiff.index(indexRoutingTable.getIndex())); + } + } + + public void testRoutingTableDeletes() { + assertThat(clusterState.routingTable().shardsWithState(ShardRoutingState.UNASSIGNED).size(), is(this.totalNumberOfShards)); + initPrimaries(); + int expectedUnassignedShardCount = this.totalNumberOfShards - 2 * this.numberOfShards; + assertThat(clusterState.routingTable().shardsWithState(ShardRoutingState.UNASSIGNED).size(), is(expectedUnassignedShardCount)); + assertThat(clusterState.routingTable().shardsWithState(ShardRoutingState.INITIALIZING).size(), is(2 * this.numberOfShards)); + Metadata metadata = Metadata.builder().put(createIndexMetadata(TEST_INDEX_1)).put(createIndexMetadata(TEST_INDEX_2)).build(); + ClusterState oldClusterState = clusterState; + + // delete index routing table for TEST_INDEX_1 + metadata = Metadata.builder(metadata).put(createIndexMetadata(TEST_INDEX_3)).build(); + RoutingTable testRoutingTable = new RoutingTable.Builder(clusterState.routingTable()).remove(TEST_INDEX_1).build(); + this.clusterState = ClusterState.builder(org.opensearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)) + .metadata(metadata) + .routingTable(testRoutingTable) + .build(); + this.totalNumberOfShards = this.shardsPerIndex; + assertThat( + clusterState.routingTable().shardsWithState(ShardRoutingState.UNASSIGNED).size(), + is(expectedUnassignedShardCount - this.numberOfShards * this.numberOfReplicas) + ); + Diff fullDiff = clusterState.routingTable().diff(oldClusterState.getRoutingTable()); + Diff incrementalDiff = clusterState.routingTable().incrementalDiff(oldClusterState.getRoutingTable()); + RoutingTable newRoutingTable = incrementalDiff.apply(oldClusterState.getRoutingTable()); + for (IndexRoutingTable indexRoutingTable : clusterState.routingTable()) { + assertEquals(clusterState.routingTable().version(), newRoutingTable.version()); + assertEquals(indexRoutingTable, newRoutingTable.index(indexRoutingTable.getIndex())); + } + RoutingTable newRoutingTableWithFullDiff = fullDiff.apply(oldClusterState.getRoutingTable()); + for (IndexRoutingTable indexRoutingTable : clusterState.routingTable()) { + assertEquals(clusterState.routingTable().version(), newRoutingTableWithFullDiff.version()); + assertEquals(indexRoutingTable, newRoutingTableWithFullDiff.index(indexRoutingTable.getIndex())); + } + } + + public void testRoutingTableUpsertsWithDiff() { + assertThat(clusterState.routingTable().shardsWithState(ShardRoutingState.UNASSIGNED).size(), is(this.totalNumberOfShards)); + initPrimaries(); + int expectedUnassignedShardCount = this.totalNumberOfShards - 2 * this.numberOfShards; + assertThat(clusterState.routingTable().shardsWithState(ShardRoutingState.UNASSIGNED).size(), is(expectedUnassignedShardCount)); + assertThat(clusterState.routingTable().shardsWithState(ShardRoutingState.INITIALIZING).size(), is(2 * this.numberOfShards)); + Metadata metadata = Metadata.builder().put(createIndexMetadata(TEST_INDEX_1)).put(createIndexMetadata(TEST_INDEX_2)).build(); + ClusterState oldClusterState = clusterState; + + // create index routing table for TEST_INDEX_3 + metadata = Metadata.builder(metadata).put(createIndexMetadata(TEST_INDEX_3)).build(); + RoutingTable testRoutingTable = new RoutingTable.Builder(clusterState.routingTable()).add( + new IndexRoutingTable.Builder(metadata.index(TEST_INDEX_3).getIndex()).initializeAsNew(metadata.index(TEST_INDEX_3)).build() + ).build(); + this.clusterState = ClusterState.builder(org.opensearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)) + .metadata(metadata) + .routingTable(testRoutingTable) + .build(); + this.totalNumberOfShards = this.shardsPerIndex * 3; + assertThat( + clusterState.routingTable().shardsWithState(ShardRoutingState.UNASSIGNED).size(), + is(expectedUnassignedShardCount + this.shardsPerIndex) + ); + initPrimaries(); + clusterState = startRandomInitializingShard(clusterState, ALLOCATION_SERVICE, TEST_INDEX_2); + // assertThat(clusterState.routingTable().shardsWithState(ShardRoutingState.INITIALIZING).size(), is(2 * this.numberOfShards + 1)); + Diff fullDiff = clusterState.routingTable().diff(oldClusterState.getRoutingTable()); + Diff incrementalDiff = clusterState.routingTable().incrementalDiff(oldClusterState.getRoutingTable()); + RoutingTable newRoutingTable = incrementalDiff.apply(oldClusterState.getRoutingTable()); + for (IndexRoutingTable indexRoutingTable : clusterState.routingTable()) { + assertEquals(clusterState.routingTable().version(), newRoutingTable.version()); + assertEquals(indexRoutingTable, newRoutingTable.index(indexRoutingTable.getIndex())); + } + RoutingTable newRoutingTableWithFullDiff = fullDiff.apply(oldClusterState.getRoutingTable()); + for (IndexRoutingTable indexRoutingTable : clusterState.routingTable()) { + assertEquals(clusterState.routingTable().version(), newRoutingTableWithFullDiff.version()); + assertEquals(indexRoutingTable, newRoutingTableWithFullDiff.index(indexRoutingTable.getIndex())); + } + } + + public void testRoutingTableDiffWithReplicaAdded() { + assertThat(clusterState.routingTable().shardsWithState(ShardRoutingState.UNASSIGNED).size(), is(this.totalNumberOfShards)); + initPrimaries(); + int expectedUnassignedShardCount = this.totalNumberOfShards - 2 * this.numberOfShards; + assertThat(clusterState.routingTable().shardsWithState(ShardRoutingState.UNASSIGNED).size(), is(expectedUnassignedShardCount)); + assertThat(clusterState.routingTable().shardsWithState(ShardRoutingState.INITIALIZING).size(), is(2 * this.numberOfShards)); + ClusterState oldClusterState = clusterState; + + // update replica count for TEST_INDEX_1 + RoutingTable updatedRoutingTable = RoutingTable.builder(clusterState.routingTable()) + .updateNumberOfReplicas(this.numberOfReplicas + 1, new String[] { TEST_INDEX_1 }) + .build(); + Metadata metadata = Metadata.builder(clusterState.metadata()) + .updateNumberOfReplicas(this.numberOfReplicas + 1, new String[] { TEST_INDEX_1 }) + .build(); + clusterState = ClusterState.builder(clusterState).routingTable(updatedRoutingTable).metadata(metadata).build(); + assertThat( + clusterState.routingTable().shardsWithState(ShardRoutingState.UNASSIGNED).size(), + is(expectedUnassignedShardCount + this.numberOfShards) + ); + Diff fullDiff = clusterState.routingTable().diff(oldClusterState.getRoutingTable()); + Diff incrementalDiff = clusterState.routingTable().incrementalDiff(oldClusterState.getRoutingTable()); + RoutingTable newRoutingTable = incrementalDiff.apply(oldClusterState.getRoutingTable()); + for (IndexRoutingTable indexRoutingTable : clusterState.routingTable()) { + assertEquals(clusterState.routingTable().version(), newRoutingTable.version()); + assertEquals(indexRoutingTable, newRoutingTable.index(indexRoutingTable.getIndex())); + } + RoutingTable newRoutingTableWithFullDiff = fullDiff.apply(oldClusterState.getRoutingTable()); + for (IndexRoutingTable indexRoutingTable : clusterState.routingTable()) { + assertEquals(clusterState.routingTable().version(), newRoutingTableWithFullDiff.version()); + assertEquals(indexRoutingTable, newRoutingTableWithFullDiff.index(indexRoutingTable.getIndex())); + } + } + + public void testRoutingTableDiffWithReplicaRemoved() { + assertThat(clusterState.routingTable().shardsWithState(ShardRoutingState.UNASSIGNED).size(), is(this.totalNumberOfShards)); + initPrimaries(); + int expectedUnassignedShardCount = this.totalNumberOfShards - 2 * this.numberOfShards; + assertThat(clusterState.routingTable().shardsWithState(ShardRoutingState.UNASSIGNED).size(), is(expectedUnassignedShardCount)); + assertThat(clusterState.routingTable().shardsWithState(ShardRoutingState.INITIALIZING).size(), is(2 * this.numberOfShards)); + ClusterState oldClusterState = clusterState; + + // update replica count for TEST_INDEX_1 + RoutingTable updatedRoutingTable = RoutingTable.builder(clusterState.routingTable()) + .updateNumberOfReplicas(this.numberOfReplicas - 1, new String[] { TEST_INDEX_1 }) + .build(); + Metadata metadata = Metadata.builder(clusterState.metadata()) + .updateNumberOfReplicas(this.numberOfReplicas - 1, new String[] { TEST_INDEX_1 }) + .build(); + clusterState = ClusterState.builder(clusterState).routingTable(updatedRoutingTable).metadata(metadata).build(); + assertThat( + clusterState.routingTable().shardsWithState(ShardRoutingState.UNASSIGNED).size(), + is(expectedUnassignedShardCount - this.numberOfShards) + ); + Diff fullDiff = clusterState.routingTable().diff(oldClusterState.getRoutingTable()); + Diff incrementalDiff = clusterState.routingTable().incrementalDiff(oldClusterState.getRoutingTable()); + RoutingTable newRoutingTable = incrementalDiff.apply(oldClusterState.getRoutingTable()); + for (IndexRoutingTable indexRoutingTable : clusterState.routingTable()) { + assertEquals(clusterState.routingTable().version(), newRoutingTable.version()); + assertEquals(indexRoutingTable, newRoutingTable.index(indexRoutingTable.getIndex())); + } + RoutingTable newRoutingTableWithFullDiff = fullDiff.apply(oldClusterState.getRoutingTable()); + for (IndexRoutingTable indexRoutingTable : clusterState.routingTable()) { + assertEquals(clusterState.routingTable().version(), newRoutingTableWithFullDiff.version()); + assertEquals(indexRoutingTable, newRoutingTableWithFullDiff.index(indexRoutingTable.getIndex())); + } + } + + public void testRoutingTableDiffsWithStartedState() { + assertThat(clusterState.routingTable().shardsWithState(ShardRoutingState.UNASSIGNED).size(), is(this.totalNumberOfShards)); + initPrimaries(); + assertThat( + clusterState.routingTable().shardsWithState(ShardRoutingState.UNASSIGNED).size(), + is(this.totalNumberOfShards - 2 * this.numberOfShards) + ); + assertThat(clusterState.routingTable().shardsWithState(ShardRoutingState.INITIALIZING).size(), is(2 * this.numberOfShards)); + + startInitializingShards(TEST_INDEX_1); + assertThat(clusterState.routingTable().shardsWithState(ShardRoutingState.STARTED).size(), is(this.numberOfShards)); + int initializingExpected = this.numberOfShards + this.numberOfShards * this.numberOfReplicas; + assertThat(clusterState.routingTable().shardsWithState(ShardRoutingState.INITIALIZING).size(), is(initializingExpected)); + assertThat( + clusterState.routingTable().shardsWithState(ShardRoutingState.UNASSIGNED).size(), + is(this.totalNumberOfShards - initializingExpected - this.numberOfShards) + ); + + startInitializingShards(TEST_INDEX_2); + assertThat(clusterState.routingTable().shardsWithState(ShardRoutingState.STARTED).size(), is(2 * this.numberOfShards)); + initializingExpected = 2 * this.numberOfShards * this.numberOfReplicas; + assertThat(clusterState.routingTable().shardsWithState(ShardRoutingState.INITIALIZING).size(), is(initializingExpected)); + assertThat( + clusterState.routingTable().shardsWithState(ShardRoutingState.UNASSIGNED).size(), + is(this.totalNumberOfShards - initializingExpected - 2 * this.numberOfShards) + ); + ClusterState oldClusterState = clusterState; + // start a random replica to change a single shard routing + clusterState = startRandomInitializingShard(clusterState, ALLOCATION_SERVICE); + assertThat(clusterState.routingTable().shardsWithState(ShardRoutingState.INITIALIZING).size(), is(initializingExpected - 1)); + assertThat(clusterState.routingTable().shardsWithState(ShardRoutingState.STARTED).size(), is(2 * this.numberOfShards + 1)); + Diff fullDiff = clusterState.routingTable().diff(oldClusterState.getRoutingTable()); + Diff incrementalDiff = clusterState.routingTable().incrementalDiff(oldClusterState.getRoutingTable()); + RoutingTable newRoutingTable = incrementalDiff.apply(oldClusterState.getRoutingTable()); + for (IndexRoutingTable indexRoutingTable : clusterState.routingTable()) { + assertEquals(clusterState.routingTable().version(), newRoutingTable.version()); + assertEquals(indexRoutingTable, newRoutingTable.index(indexRoutingTable.getIndex())); + } + RoutingTable newRoutingTableWithFullDiff = fullDiff.apply(oldClusterState.getRoutingTable()); + for (IndexRoutingTable indexRoutingTable : clusterState.routingTable()) { + assertEquals(clusterState.routingTable().version(), newRoutingTableWithFullDiff.version()); + assertEquals(indexRoutingTable, newRoutingTableWithFullDiff.index(indexRoutingTable.getIndex())); + } + } + +} diff --git a/server/src/test/java/org/opensearch/cluster/routing/remote/RemoteRoutingTableServiceTests.java b/server/src/test/java/org/opensearch/cluster/routing/remote/RemoteRoutingTableServiceTests.java index 74254f1a1987f..5061de9161ab4 100644 --- a/server/src/test/java/org/opensearch/cluster/routing/remote/RemoteRoutingTableServiceTests.java +++ b/server/src/test/java/org/opensearch/cluster/routing/remote/RemoteRoutingTableServiceTests.java @@ -13,14 +13,13 @@ import org.opensearch.cluster.ClusterName; import org.opensearch.cluster.ClusterState; import org.opensearch.cluster.Diff; -import org.opensearch.cluster.DiffableUtils; import org.opensearch.cluster.coordination.CoordinationMetadata; import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.cluster.metadata.Metadata; import org.opensearch.cluster.routing.IndexRoutingTable; -import org.opensearch.cluster.routing.IndexShardRoutingTable; import org.opensearch.cluster.routing.RoutingTable; import org.opensearch.cluster.routing.RoutingTableIncrementalDiff; +import org.opensearch.cluster.routing.StringKeyDiffProvider; import org.opensearch.cluster.service.ClusterService; import org.opensearch.common.blobstore.BlobContainer; import org.opensearch.common.blobstore.BlobPath; @@ -57,10 +56,8 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Base64; -import java.util.HashMap; import java.util.List; import java.util.Locale; -import java.util.Map; import java.util.concurrent.CountDownLatch; import java.util.function.Supplier; @@ -68,6 +65,7 @@ import static org.opensearch.common.util.FeatureFlags.REMOTE_PUBLICATION_EXPERIMENTAL; import static org.opensearch.gateway.remote.ClusterMetadataManifestTests.randomUploadedIndexMetadataList; +import static org.opensearch.gateway.remote.RemoteClusterStateServiceTests.generateClusterStateWithOneIndex; import static org.opensearch.gateway.remote.RemoteClusterStateUtils.CLUSTER_STATE_PATH_TOKEN; import static org.opensearch.gateway.remote.RemoteClusterStateUtils.DELIMITER; import static org.opensearch.gateway.remote.RemoteClusterStateUtils.PATH_DELIMITER; @@ -187,10 +185,9 @@ public void testGetIndicesRoutingMapDiff() { RoutingTable routingTable = RoutingTable.builder().addAsNew(indexMetadata).build(); - DiffableUtils.MapDiff> diff = remoteRoutingTableService - .getIndicesRoutingMapDiff(routingTable, routingTable); - assertEquals(0, diff.getUpserts().size()); - assertEquals(0, diff.getDeletes().size()); + StringKeyDiffProvider diff = remoteRoutingTableService.getIndicesRoutingMapDiff(routingTable, routingTable); + assertEquals(0, diff.provideDiff().getUpserts().size()); + assertEquals(0, diff.provideDiff().getDeletes().size()); // Reversing order to check for equality without order. IndexRoutingTable indexRouting = routingTable.getIndicesRouting().get(indexName); @@ -201,8 +198,8 @@ public void testGetIndicesRoutingMapDiff() { RoutingTable routingTable2 = RoutingTable.builder().add(indexRoutingTableReversed).build(); diff = remoteRoutingTableService.getIndicesRoutingMapDiff(routingTable, routingTable2); - assertEquals(0, diff.getUpserts().size()); - assertEquals(0, diff.getDeletes().size()); + assertEquals(0, diff.provideDiff().getUpserts().size()); + assertEquals(0, diff.provideDiff().getDeletes().size()); } public void testGetChangedIndicesRouting() { @@ -220,7 +217,10 @@ public void testGetChangedIndicesRouting() { assertEquals( 0, - remoteRoutingTableService.getIndicesRoutingMapDiff(state.getRoutingTable(), state.getRoutingTable()).getUpserts().size() + remoteRoutingTableService.getIndicesRoutingMapDiff(state.getRoutingTable(), state.getRoutingTable()) + .provideDiff() + .getUpserts() + .size() ); // Reversing order to check for equality without order. @@ -234,7 +234,10 @@ public void testGetChangedIndicesRouting() { .build(); assertEquals( 0, - remoteRoutingTableService.getIndicesRoutingMapDiff(state.getRoutingTable(), newState.getRoutingTable()).getUpserts().size() + remoteRoutingTableService.getIndicesRoutingMapDiff(state.getRoutingTable(), newState.getRoutingTable()) + .provideDiff() + .getUpserts() + .size() ); } @@ -259,13 +262,12 @@ public void testGetIndicesRoutingMapDiffIndexAdded() { ).numberOfShards(noOfShards).numberOfReplicas(noOfReplicas).build(); RoutingTable routingTable2 = RoutingTable.builder(routingTable).addAsNew(indexMetadata2).build(); - DiffableUtils.MapDiff> diff = remoteRoutingTableService - .getIndicesRoutingMapDiff(routingTable, routingTable2); - assertEquals(1, diff.getUpserts().size()); - assertNotNull(diff.getUpserts().get(indexName2)); - assertEquals(noOfShards, diff.getUpserts().get(indexName2).getShards().size()); + StringKeyDiffProvider diff = remoteRoutingTableService.getIndicesRoutingMapDiff(routingTable, routingTable2); + assertEquals(1, diff.provideDiff().getUpserts().size()); + assertNotNull(diff.provideDiff().getUpserts().get(indexName2)); + assertEquals(noOfShards, diff.provideDiff().getUpserts().get(indexName2).getShards().size()); - assertEquals(0, diff.getDeletes().size()); + assertEquals(0, diff.provideDiff().getDeletes().size()); } public void testGetIndicesRoutingMapDiffShardChanged() { @@ -289,17 +291,19 @@ public void testGetIndicesRoutingMapDiffShardChanged() { ).numberOfShards(noOfShards + 1).numberOfReplicas(noOfReplicas).build(); RoutingTable routingTable2 = RoutingTable.builder().addAsNew(indexMetadata2).build(); - DiffableUtils.MapDiff> diff = remoteRoutingTableService - .getIndicesRoutingMapDiff(routingTable, routingTable2); - assertEquals(0, diff.getUpserts().size()); - assertEquals(1, diff.getDiffs().size()); - assertNotNull(diff.getDiffs().get(indexName)); - assertEquals(noOfShards + 1, diff.getDiffs().get(indexName).apply(routingTable.indicesRouting().get(indexName)).shards().size()); + StringKeyDiffProvider diff = remoteRoutingTableService.getIndicesRoutingMapDiff(routingTable, routingTable2); + assertEquals(0, diff.provideDiff().getUpserts().size()); + assertEquals(1, diff.provideDiff().getDiffs().size()); + assertNotNull(diff.provideDiff().getDiffs().get(indexName)); + assertEquals( + noOfShards + 1, + diff.provideDiff().getDiffs().get(indexName).apply(routingTable.indicesRouting().get(indexName)).shards().size() + ); assertEquals( noOfReplicas + 1, - diff.getDiffs().get(indexName).apply(routingTable.indicesRouting().get(indexName)).getShards().get(0).getSize() + diff.provideDiff().getDiffs().get(indexName).apply(routingTable.indicesRouting().get(indexName)).getShards().get(0).getSize() ); - assertEquals(0, diff.getDeletes().size()); + assertEquals(0, diff.provideDiff().getDeletes().size()); final IndexMetadata indexMetadata3 = new IndexMetadata.Builder(indexName).settings( Settings.builder() @@ -310,15 +314,18 @@ public void testGetIndicesRoutingMapDiffShardChanged() { RoutingTable routingTable3 = RoutingTable.builder().addAsNew(indexMetadata3).build(); diff = remoteRoutingTableService.getIndicesRoutingMapDiff(routingTable2, routingTable3); - assertEquals(0, diff.getUpserts().size()); - assertEquals(1, diff.getDiffs().size()); - assertNotNull(diff.getDiffs().get(indexName)); - assertEquals(noOfShards + 1, diff.getDiffs().get(indexName).apply(routingTable.indicesRouting().get(indexName)).shards().size()); + assertEquals(0, diff.provideDiff().getUpserts().size()); + assertEquals(1, diff.provideDiff().getDiffs().size()); + assertNotNull(diff.provideDiff().getDiffs().get(indexName)); + assertEquals( + noOfShards + 1, + diff.provideDiff().getDiffs().get(indexName).apply(routingTable.indicesRouting().get(indexName)).shards().size() + ); assertEquals( noOfReplicas + 2, - diff.getDiffs().get(indexName).apply(routingTable.indicesRouting().get(indexName)).getShards().get(0).getSize() + diff.provideDiff().getDiffs().get(indexName).apply(routingTable.indicesRouting().get(indexName)).getShards().get(0).getSize() ); - assertEquals(0, diff.getDeletes().size()); + assertEquals(0, diff.provideDiff().getDeletes().size()); } public void testGetIndicesRoutingMapDiffShardDetailChanged() { @@ -335,13 +342,15 @@ public void testGetIndicesRoutingMapDiffShardDetailChanged() { RoutingTable routingTable = RoutingTable.builder().addAsNew(indexMetadata).build(); RoutingTable routingTable2 = RoutingTable.builder().addAsRecovery(indexMetadata).build(); - DiffableUtils.MapDiff> diff = remoteRoutingTableService - .getIndicesRoutingMapDiff(routingTable, routingTable2); - assertEquals(1, diff.getDiffs().size()); - assertNotNull(diff.getDiffs().get(indexName)); - assertEquals(noOfShards, diff.getDiffs().get(indexName).apply(routingTable.indicesRouting().get(indexName)).shards().size()); - assertEquals(0, diff.getUpserts().size()); - assertEquals(0, diff.getDeletes().size()); + StringKeyDiffProvider diff = remoteRoutingTableService.getIndicesRoutingMapDiff(routingTable, routingTable2); + assertEquals(1, diff.provideDiff().getDiffs().size()); + assertNotNull(diff.provideDiff().getDiffs().get(indexName)); + assertEquals( + noOfShards, + diff.provideDiff().getDiffs().get(indexName).apply(routingTable.indicesRouting().get(indexName)).shards().size() + ); + assertEquals(0, diff.provideDiff().getUpserts().size()); + assertEquals(0, diff.provideDiff().getDeletes().size()); } public void testGetIndicesRoutingMapDiffIndexDeleted() { @@ -363,13 +372,12 @@ public void testGetIndicesRoutingMapDiffIndexDeleted() { ).numberOfShards(between(1, 1000)).numberOfReplicas(randomInt(10)).build(); RoutingTable routingTable2 = RoutingTable.builder().addAsNew(indexMetadata2).build(); - DiffableUtils.MapDiff> diff = remoteRoutingTableService - .getIndicesRoutingMapDiff(routingTable, routingTable2); - assertEquals(1, diff.getUpserts().size()); - assertNotNull(diff.getUpserts().get(indexName2)); + StringKeyDiffProvider diff = remoteRoutingTableService.getIndicesRoutingMapDiff(routingTable, routingTable2); + assertEquals(1, diff.provideDiff().getUpserts().size()); + assertNotNull(diff.provideDiff().getUpserts().get(indexName2)); - assertEquals(1, diff.getDeletes().size()); - assertEquals(indexName, diff.getDeletes().get(0)); + assertEquals(1, diff.provideDiff().getDeletes().size()); + assertEquals(indexName, diff.provideDiff().getDeletes().get(0)); } public void testGetAllUploadedIndicesRouting() { @@ -493,11 +501,17 @@ public void testIndicesRoutingDiffWhenIndexDeleted() { assertEquals( 1, - remoteRoutingTableService.getIndicesRoutingMapDiff(state.getRoutingTable(), updatedRoutingTable).getDeletes().size() + remoteRoutingTableService.getIndicesRoutingMapDiff(state.getRoutingTable(), updatedRoutingTable) + .provideDiff() + .getDeletes() + .size() ); assertEquals( indexNameToDelete, - remoteRoutingTableService.getIndicesRoutingMapDiff(state.getRoutingTable(), updatedRoutingTable).getDeletes().get(0) + remoteRoutingTableService.getIndicesRoutingMapDiff(state.getRoutingTable(), updatedRoutingTable) + .provideDiff() + .getDeletes() + .get(0) ); } @@ -524,19 +538,29 @@ public void testIndicesRoutingDiffWhenIndexDeletedAndAdded() { assertEquals( 1, - remoteRoutingTableService.getIndicesRoutingMapDiff(state.getRoutingTable(), updatedRoutingTable).getDeletes().size() + remoteRoutingTableService.getIndicesRoutingMapDiff(state.getRoutingTable(), updatedRoutingTable) + .provideDiff() + .getDeletes() + .size() ); assertEquals( indexNameToDelete, - remoteRoutingTableService.getIndicesRoutingMapDiff(state.getRoutingTable(), updatedRoutingTable).getDeletes().get(0) + remoteRoutingTableService.getIndicesRoutingMapDiff(state.getRoutingTable(), updatedRoutingTable) + .provideDiff() + .getDeletes() + .get(0) ); assertEquals( 1, - remoteRoutingTableService.getIndicesRoutingMapDiff(state.getRoutingTable(), updatedRoutingTable).getUpserts().size() + remoteRoutingTableService.getIndicesRoutingMapDiff(state.getRoutingTable(), updatedRoutingTable) + .provideDiff() + .getUpserts() + .size() ); assertTrue( remoteRoutingTableService.getIndicesRoutingMapDiff(state.getRoutingTable(), updatedRoutingTable) + .provideDiff() .getUpserts() .containsKey(indexName) ); @@ -571,27 +595,17 @@ public void testGetAsyncIndexRoutingReadAction() throws Exception { public void testGetAsyncIndexRoutingTableDiffReadAction() throws Exception { String indexName = randomAlphaOfLength(randomIntBetween(1, 50)); - ClusterState currentState = createClusterState(indexName); - - // Get the IndexRoutingTable from the current state - IndexRoutingTable indexRoutingTable = currentState.routingTable().index(indexName); - Map shardRoutingTables = indexRoutingTable.getShards(); + ClusterState previousState = generateClusterStateWithOneIndex(indexName, 5, 1, false).build(); + ClusterState currentState = generateClusterStateWithOneIndex(indexName, 5, 2, true).build(); - RoutingTableIncrementalDiff.IndexRoutingTableIncrementalDiff indexRoutingTableDiff = - new RoutingTableIncrementalDiff.IndexRoutingTableIncrementalDiff(new ArrayList<>(shardRoutingTables.values())); - - // Create the map for RoutingTableIncrementalDiff - Map> diffs = new HashMap<>(); - diffs.put(indexName, indexRoutingTableDiff); - - RoutingTableIncrementalDiff diff = new RoutingTableIncrementalDiff(diffs); + RoutingTableIncrementalDiff diff = new RoutingTableIncrementalDiff(previousState.getRoutingTable(), currentState.getRoutingTable()); String uploadedFileName = String.format(Locale.ROOT, "routing-table-diff/" + indexName); when(blobContainer.readBlob(indexName)).thenReturn( REMOTE_ROUTING_TABLE_DIFF_FORMAT.serialize(diff, uploadedFileName, compressor).streamInput() ); - TestCapturingListener listener = new TestCapturingListener<>(); + TestCapturingListener> listener = new TestCapturingListener<>(); CountDownLatch latch = new CountDownLatch(1); remoteRoutingTableService.getAsyncIndexRoutingTableDiffReadAction( @@ -603,8 +617,11 @@ public void testGetAsyncIndexRoutingTableDiffReadAction() throws Exception { assertNull(listener.getFailure()); assertNotNull(listener.getResult()); - RoutingTableIncrementalDiff resultDiff = listener.getResult(); - assertEquals(diff.getDiffs().size(), resultDiff.getDiffs().size()); + Diff resultDiff = listener.getResult(); + assertEquals( + currentState.getRoutingTable().getIndicesRouting(), + resultDiff.apply(previousState.getRoutingTable()).getIndicesRouting() + ); } public void testGetAsyncIndexRoutingWriteAction() throws Exception { @@ -661,20 +678,8 @@ public void testGetAsyncIndexRoutingWriteAction() throws Exception { public void testGetAsyncIndexRoutingDiffWriteAction() throws Exception { String indexName = randomAlphaOfLength(randomIntBetween(1, 50)); - ClusterState currentState = createClusterState(indexName); - - // Get the IndexRoutingTable from the current state - IndexRoutingTable indexRoutingTable = currentState.routingTable().index(indexName); - Map shardRoutingTables = indexRoutingTable.getShards(); - - RoutingTableIncrementalDiff.IndexRoutingTableIncrementalDiff indexRoutingTableDiff = - new RoutingTableIncrementalDiff.IndexRoutingTableIncrementalDiff(new ArrayList<>(shardRoutingTables.values())); - - // Create the map for RoutingTableIncrementalDiff - Map> diffs = new HashMap<>(); - diffs.put(indexName, indexRoutingTableDiff); - - // RoutingTableIncrementalDiff diff = new RoutingTableIncrementalDiff(diffs); + ClusterState previousState = generateClusterStateWithOneIndex(indexName, 5, 1, false).build(); + ClusterState currentState = generateClusterStateWithOneIndex(indexName, 5, 2, true).build(); Iterable remotePath = new BlobPath().add("base-path") .add( @@ -699,7 +704,7 @@ public void testGetAsyncIndexRoutingDiffWriteAction() throws Exception { currentState.metadata().clusterUUID(), currentState.term(), currentState.version(), - diffs, + new RoutingTableIncrementalDiff(previousState.getRoutingTable(), currentState.getRoutingTable()), new LatchedActionListener<>(listener, latch) ); latch.await(); @@ -717,7 +722,7 @@ public void testGetAsyncIndexRoutingDiffWriteAction() throws Exception { assertEquals(4, fileNameTokens.length); assertEquals(ROUTING_TABLE_DIFF_METADATA_PREFIX, fileNameTokens[0]); assertEquals(RemoteStoreUtils.invertLong(1L), fileNameTokens[1]); - assertEquals(RemoteStoreUtils.invertLong(2L), fileNameTokens[2]); + assertEquals(RemoteStoreUtils.invertLong(1L), fileNameTokens[2]); assertThat(RemoteStoreUtils.invertLong(fileNameTokens[3]), lessThanOrEqualTo(System.currentTimeMillis())); } diff --git a/server/src/test/java/org/opensearch/gateway/remote/ClusterMetadataManifestTests.java b/server/src/test/java/org/opensearch/gateway/remote/ClusterMetadataManifestTests.java index 418cfa104a911..6a191d002b971 100644 --- a/server/src/test/java/org/opensearch/gateway/remote/ClusterMetadataManifestTests.java +++ b/server/src/test/java/org/opensearch/gateway/remote/ClusterMetadataManifestTests.java @@ -10,11 +10,11 @@ import org.opensearch.Version; import org.opensearch.cluster.ClusterState; -import org.opensearch.cluster.DiffableUtils; import org.opensearch.cluster.metadata.IndexGraveyard; import org.opensearch.cluster.metadata.RepositoriesMetadata; import org.opensearch.cluster.metadata.WeightedRoutingMetadata; import org.opensearch.cluster.routing.IndexRoutingTable; +import org.opensearch.cluster.routing.StringKeyDiffProvider; import org.opensearch.common.xcontent.json.JsonXContent; import org.opensearch.core.common.bytes.BytesReference; import org.opensearch.core.common.io.stream.NamedWriteableRegistry; @@ -31,7 +31,6 @@ import java.util.Arrays; import java.util.Collections; import java.util.List; -import java.util.Map; import java.util.function.Function; import java.util.stream.Collectors; @@ -41,6 +40,7 @@ import static org.opensearch.gateway.remote.ClusterMetadataManifest.CODEC_V1; import static org.opensearch.gateway.remote.ClusterMetadataManifest.CODEC_V2; import static org.opensearch.gateway.remote.ClusterMetadataManifest.CODEC_V3; +import static org.opensearch.gateway.remote.ClusterMetadataManifest.CODEC_V4; import static org.opensearch.gateway.remote.RemoteClusterStateAttributesManager.CLUSTER_BLOCKS; import static org.opensearch.gateway.remote.RemoteClusterStateAttributesManager.DISCOVERY_NODES; import static org.opensearch.gateway.remote.model.RemoteCoordinationMetadata.COORDINATION_METADATA; @@ -199,6 +199,7 @@ public void testClusterMetadataManifestSerializationEqualsHashCode() { new ClusterStateDiffManifest( RemoteClusterStateServiceTests.generateClusterStateWithOneIndex().build(), ClusterState.EMPTY_STATE, + ClusterMetadataManifest.CODEC_V3, null, "indicesRoutingDiffPath" ) @@ -533,6 +534,7 @@ public void testClusterMetadataManifestXContentV2() throws IOException { new ClusterStateDiffManifest( RemoteClusterStateServiceTests.generateClusterStateWithOneIndex().build(), ClusterState.EMPTY_STATE, + CODEC_V2, null, null ) @@ -552,9 +554,7 @@ public void testClusterMetadataManifestXContentV2() throws IOException { public void testClusterMetadataManifestXContentV3() throws IOException { UploadedIndexMetadata uploadedIndexMetadata = new UploadedIndexMetadata("test-index", "test-uuid", "/test/upload/path"); UploadedMetadataAttribute uploadedMetadataAttribute = new UploadedMetadataAttribute("attribute_name", "testing_attribute"); - final DiffableUtils.MapDiff> routingTableIncrementalDiff = Mockito.mock( - DiffableUtils.MapDiff.class - ); + final StringKeyDiffProvider routingTableIncrementalDiff = Mockito.mock(StringKeyDiffProvider.class); ClusterMetadataManifest originalManifest = ClusterMetadataManifest.builder() .clusterTerm(1L) .stateVersion(1L) @@ -599,6 +599,7 @@ public void testClusterMetadataManifestXContentV3() throws IOException { new ClusterStateDiffManifest( RemoteClusterStateServiceTests.generateClusterStateWithOneIndex().build(), ClusterState.EMPTY_STATE, + CODEC_V3, routingTableIncrementalDiff, uploadedMetadataAttribute.getUploadedFilename() ) @@ -670,6 +671,71 @@ public void testClusterMetadataManifestXContentV2WithoutEphemeral() throws IOExc } } + public void testClusterMetadataManifestXContentV4() throws IOException { + UploadedIndexMetadata uploadedIndexMetadata = new UploadedIndexMetadata("test-index", "test-uuid", "/test/upload/path"); + UploadedMetadataAttribute uploadedMetadataAttribute = new UploadedMetadataAttribute("attribute_name", "testing_attribute"); + final StringKeyDiffProvider routingTableIncrementalDiff = Mockito.mock(StringKeyDiffProvider.class); + ClusterMetadataManifest originalManifest = ClusterMetadataManifest.builder() + .clusterTerm(1L) + .stateVersion(1L) + .clusterUUID("test-cluster-uuid") + .stateUUID("test-state-uuid") + .opensearchVersion(Version.CURRENT) + .nodeId("test-node-id") + .committed(false) + .codecVersion(ClusterMetadataManifest.CODEC_V4) + .indices(Collections.singletonList(uploadedIndexMetadata)) + .previousClusterUUID("prev-cluster-uuid") + .clusterUUIDCommitted(true) + .coordinationMetadata(uploadedMetadataAttribute) + .settingMetadata(uploadedMetadataAttribute) + .templatesMetadata(uploadedMetadataAttribute) + .customMetadataMap( + Collections.unmodifiableList( + Arrays.asList( + new UploadedMetadataAttribute( + CUSTOM_METADATA + CUSTOM_DELIMITER + RepositoriesMetadata.TYPE, + "custom--repositories-file" + ), + new UploadedMetadataAttribute( + CUSTOM_METADATA + CUSTOM_DELIMITER + IndexGraveyard.TYPE, + "custom--index_graveyard-file" + ), + new UploadedMetadataAttribute( + CUSTOM_METADATA + CUSTOM_DELIMITER + WeightedRoutingMetadata.TYPE, + "custom--weighted_routing_netadata-file" + ) + ) + ).stream().collect(Collectors.toMap(UploadedMetadataAttribute::getAttributeName, Function.identity())) + ) + .routingTableVersion(1L) + .indicesRouting(Collections.singletonList(uploadedIndexMetadata)) + .discoveryNodesMetadata(uploadedMetadataAttribute) + .clusterBlocksMetadata(uploadedMetadataAttribute) + .transientSettingsMetadata(uploadedMetadataAttribute) + .hashesOfConsistentSettings(uploadedMetadataAttribute) + .clusterStateCustomMetadataMap(Collections.emptyMap()) + .diffManifest( + new ClusterStateDiffManifest( + RemoteClusterStateServiceTests.generateClusterStateWithOneIndex().build(), + ClusterState.EMPTY_STATE, + CODEC_V4, + routingTableIncrementalDiff, + uploadedMetadataAttribute.getUploadedFilename() + ) + ) + .build(); + final XContentBuilder builder = JsonXContent.contentBuilder(); + builder.startObject(); + originalManifest.toXContent(builder, ToXContent.EMPTY_PARAMS); + builder.endObject(); + + try (XContentParser parser = createParser(JsonXContent.jsonXContent, BytesReference.bytes(builder))) { + final ClusterMetadataManifest fromXContentManifest = ClusterMetadataManifest.fromXContent(parser); + assertEquals(originalManifest, fromXContentManifest); + } + } + public static List randomUploadedIndexMetadataList() { final int size = randomIntBetween(1, 10); final List uploadedIndexMetadataList = new ArrayList<>(size); @@ -722,7 +788,7 @@ public void testGetCodecForVersion() { assertEquals(CODEC_V1, ClusterMetadataManifest.getCodecForVersion(Version.V_2_13_0)); assertEquals(CODEC_V2, ClusterMetadataManifest.getCodecForVersion(Version.V_2_15_0)); assertEquals(CODEC_V3, ClusterMetadataManifest.getCodecForVersion(Version.V_2_16_0)); - assertEquals(CODEC_V3, ClusterMetadataManifest.getCodecForVersion(Version.V_2_17_0)); + assertEquals(CODEC_V4, ClusterMetadataManifest.getCodecForVersion(Version.V_2_17_0)); } private UploadedIndexMetadata randomlyChangingUploadedIndexMetadata(UploadedIndexMetadata uploadedIndexMetadata) { diff --git a/server/src/test/java/org/opensearch/gateway/remote/RemoteClusterStateServiceTests.java b/server/src/test/java/org/opensearch/gateway/remote/RemoteClusterStateServiceTests.java index d57cc17acedcb..835087632d0d0 100644 --- a/server/src/test/java/org/opensearch/gateway/remote/RemoteClusterStateServiceTests.java +++ b/server/src/test/java/org/opensearch/gateway/remote/RemoteClusterStateServiceTests.java @@ -24,8 +24,8 @@ import org.opensearch.cluster.metadata.TemplatesMetadata; import org.opensearch.cluster.node.DiscoveryNode; import org.opensearch.cluster.node.DiscoveryNodes; -import org.opensearch.cluster.routing.IndexRoutingTable; import org.opensearch.cluster.routing.RoutingTable; +import org.opensearch.cluster.routing.StringKeyDiffProvider; import org.opensearch.cluster.routing.remote.InternalRemoteRoutingTableService; import org.opensearch.cluster.routing.remote.NoopRemoteRoutingTableService; import org.opensearch.cluster.service.ClusterService; @@ -546,14 +546,14 @@ public void testTimeoutWhileWritingManifestFile() throws IOException { anyMap(), anyBoolean(), anyList(), - anyMap() + any() ) ).thenReturn(new RemoteClusterStateUtils.UploadedMetadataResults()); RemoteStateTransferException ex = expectThrows( RemoteStateTransferException.class, () -> spiedService.writeFullMetadata(clusterState, randomAlphaOfLength(10), MANIFEST_CURRENT_CODEC_VERSION) ); - assertTrue(ex.getMessage().contains("Timed out waiting for transfer of following metadata to complete")); + assertTrue(ex.getMessage().contains("Timed out waiting for transfer")); } public void testWriteFullMetadataInParallelFailureForIndexMetadata() throws IOException { @@ -697,7 +697,7 @@ public void testWriteIncrementalMetadataSuccess() throws IOException { eq(Collections.emptyMap()), eq(false), eq(Collections.emptyList()), - eq(Collections.emptyMap()) + Mockito.any(StringKeyDiffProvider.class) ); assertThat(manifestInfo.getManifestFileName(), notNullValue()); @@ -778,7 +778,7 @@ public void testWriteIncrementalMetadataSuccessWhenPublicationEnabled() throws I eq(Collections.emptyMap()), eq(true), anyList(), - eq(Collections.emptyMap()) + Mockito.any(StringKeyDiffProvider.class) ); assertThat(manifestInfo.getManifestFileName(), notNullValue()); @@ -1256,11 +1256,6 @@ public void testGetClusterStateUsingDiff() throws IOException { ClusterState.Custom originalClusterStateCustom = clusterState.customs().get(clusterStateCustomName); assertNotEquals(originalClusterStateCustom, updateClusterStateCustom); }); - diffManifest.getIndicesRoutingUpdated().forEach(indexName -> { - IndexRoutingTable updatedIndexRoutingTable = updatedClusterState.getRoutingTable().getIndicesRouting().get(indexName); - IndexRoutingTable originalIndexingRoutingTable = clusterState.getRoutingTable().getIndicesRouting().get(indexName); - assertNotEquals(originalIndexingRoutingTable, updatedIndexRoutingTable); - }); diffManifest.getIndicesDeleted() .forEach(indexName -> { assertFalse(updatedClusterState.metadata().getIndices().containsKey(indexName)); }); diffManifest.getCustomMetadataDeleted().forEach(customMetadataName -> { @@ -1269,9 +1264,6 @@ public void testGetClusterStateUsingDiff() throws IOException { diffManifest.getClusterStateCustomDeleted().forEach(clusterStateCustomName -> { assertFalse(updatedClusterState.customs().containsKey(clusterStateCustomName)); }); - diffManifest.getIndicesRoutingDeleted().forEach(indexName -> { - assertFalse(updatedClusterState.getRoutingTable().getIndicesRouting().containsKey(indexName)); - }); } public void testReadClusterStateInParallel_TimedOut() throws IOException { @@ -2858,9 +2850,7 @@ public void testWriteIncrementalMetadataSuccessWithRoutingTableDiffNull() throws INDEX_ROUTING_METADATA_PREFIX ); indices.add(uploadedIndiceRoutingMetadata); - final ClusterState previousClusterState = generateClusterStateWithOneIndex("test-index2", 5, 1, false).nodes( - nodesWithLocalNodeClusterManager() - ).build(); + final ClusterState previousClusterState = clusterState; final ClusterMetadataManifest previousManifest = ClusterMetadataManifest.builder().indices(indices).build(); when((blobStoreRepository.basePath())).thenReturn(BlobPath.cleanPath().add("base-path")); @@ -2889,9 +2879,6 @@ public void testWriteIncrementalMetadataSuccessWithRoutingTableDiffNull() throws assertThat(manifest.getClusterUUID(), is(expectedManifest.getClusterUUID())); assertThat(manifest.getStateUUID(), is(expectedManifest.getStateUUID())); assertThat(manifest.getRoutingTableVersion(), is(expectedManifest.getRoutingTableVersion())); - assertThat(manifest.getIndicesRouting().get(0).getIndexName(), is(uploadedIndiceRoutingMetadata.getIndexName())); - assertThat(manifest.getIndicesRouting().get(0).getIndexUUID(), is(uploadedIndiceRoutingMetadata.getIndexUUID())); - assertThat(manifest.getIndicesRouting().get(0).getUploadedFilename(), notNullValue()); assertThat(manifest.getDiffManifest().getIndicesRoutingDiffPath(), nullValue()); } diff --git a/server/src/test/java/org/opensearch/gateway/remote/model/ClusterStateDiffManifestTests.java b/server/src/test/java/org/opensearch/gateway/remote/model/ClusterStateDiffManifestTests.java index f89619a09cd52..e716822939a58 100644 --- a/server/src/test/java/org/opensearch/gateway/remote/model/ClusterStateDiffManifestTests.java +++ b/server/src/test/java/org/opensearch/gateway/remote/model/ClusterStateDiffManifestTests.java @@ -10,7 +10,6 @@ import org.opensearch.Version; import org.opensearch.cluster.ClusterState; -import org.opensearch.cluster.DiffableUtils; import org.opensearch.cluster.coordination.CoordinationMetadata; import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.cluster.metadata.IndexTemplateMetadata; @@ -19,6 +18,8 @@ import org.opensearch.cluster.node.DiscoveryNode; import org.opensearch.cluster.node.DiscoveryNodes; import org.opensearch.cluster.routing.IndexRoutingTable; +import org.opensearch.cluster.routing.RoutingTableIncrementalDiff; +import org.opensearch.cluster.routing.StringKeyDiffProvider; import org.opensearch.common.settings.Settings; import org.opensearch.common.xcontent.json.JsonXContent; import org.opensearch.core.common.bytes.BytesReference; @@ -42,9 +43,9 @@ import static java.util.stream.Collectors.toList; import static org.opensearch.Version.CURRENT; import static org.opensearch.cluster.ClusterState.EMPTY_STATE; -import static org.opensearch.cluster.routing.remote.RemoteRoutingTableService.CUSTOM_ROUTING_TABLE_DIFFABLE_VALUE_SERIALIZER; import static org.opensearch.core.common.transport.TransportAddress.META_ADDRESS; import static org.opensearch.gateway.remote.ClusterMetadataManifest.CODEC_V3; +import static org.opensearch.gateway.remote.ClusterMetadataManifest.CODEC_V4; import static org.opensearch.gateway.remote.RemoteClusterStateServiceTests.generateClusterStateWithOneIndex; import static org.opensearch.gateway.remote.RemoteClusterStateServiceTests.nodesWithLocalNodeClusterManager; import static org.opensearch.gateway.remote.model.RemoteClusterBlocksTests.randomClusterBlocks; @@ -120,7 +121,7 @@ public void testClusterStateDiffManifestXContent() throws IOException { diffManifest.toXContent(builder, ToXContent.EMPTY_PARAMS); builder.endObject(); try (XContentParser parser = createParser(JsonXContent.jsonXContent, BytesReference.bytes(builder))) { - final ClusterStateDiffManifest parsedManifest = ClusterStateDiffManifest.fromXContent(parser, CODEC_V3); + final ClusterStateDiffManifest parsedManifest = ClusterStateDiffManifest.fromXContent(parser, CODEC_V4); assertEquals(diffManifest, parsedManifest); } } @@ -132,7 +133,7 @@ public void testClusterStateWithRoutingTableDiffInDiffManifestXContent() throws ClusterState updatedState = generateClusterStateWithOneIndex("test-index", 5, 2, false).nodes(nodesWithLocalNodeClusterManager()) .build(); - ClusterStateDiffManifest diffManifest = verifyRoutingTableDiffManifest(initialState, updatedState); + ClusterStateDiffManifest diffManifest = verifyRoutingTableDiffManifest(initialState, updatedState, CODEC_V3); final XContentBuilder builder = JsonXContent.contentBuilder(); builder.startObject(); diffManifest.toXContent(builder, ToXContent.EMPTY_PARAMS); @@ -143,14 +144,14 @@ public void testClusterStateWithRoutingTableDiffInDiffManifestXContent() throws } } - public void testClusterStateWithRoutingTableDiffInDiffManifestXContent1() throws IOException { + public void testClusterStateWithRoutingTableDiffInDiffManifestXContentWithDeletes() throws IOException { ClusterState initialState = generateClusterStateWithOneIndex("test-index", 5, 1, true).nodes(nodesWithLocalNodeClusterManager()) .build(); ClusterState updatedState = generateClusterStateWithOneIndex("test-index-1", 5, 2, false).nodes(nodesWithLocalNodeClusterManager()) .build(); - ClusterStateDiffManifest diffManifest = verifyRoutingTableDiffManifest(initialState, updatedState); + ClusterStateDiffManifest diffManifest = verifyRoutingTableDiffManifest(initialState, updatedState, CODEC_V3); final XContentBuilder builder = JsonXContent.contentBuilder(); builder.startObject(); diffManifest.toXContent(builder, ToXContent.EMPTY_PARAMS); @@ -161,26 +162,60 @@ public void testClusterStateWithRoutingTableDiffInDiffManifestXContent1() throws } } - private ClusterStateDiffManifest verifyRoutingTableDiffManifest(ClusterState previousState, ClusterState currentState) { - // Create initial and updated IndexRoutingTable maps - Map initialRoutingTableMap = previousState.getRoutingTable().indicesRouting(); - Map updatedRoutingTableMap = currentState.getRoutingTable().indicesRouting(); + public void testClusterStateWithRoutingTableDiffInDiffManifestXContentV4() throws IOException { + ClusterState initialState = generateClusterStateWithOneIndex("test-index", 5, 1, true).nodes(nodesWithLocalNodeClusterManager()) + .build(); - DiffableUtils.MapDiff> routingTableIncrementalDiff = DiffableUtils.diff( - initialRoutingTableMap, - updatedRoutingTableMap, - DiffableUtils.getStringKeySerializer(), - CUSTOM_ROUTING_TABLE_DIFFABLE_VALUE_SERIALIZER + ClusterState updatedState = generateClusterStateWithOneIndex("test-index", 5, 2, false).nodes(nodesWithLocalNodeClusterManager()) + .build(); + + ClusterStateDiffManifest diffManifest = verifyRoutingTableDiffManifest(initialState, updatedState, CODEC_V4); + final XContentBuilder builder = JsonXContent.contentBuilder(); + builder.startObject(); + diffManifest.toXContent(builder, ToXContent.EMPTY_PARAMS); + builder.endObject(); + try (XContentParser parser = createParser(JsonXContent.jsonXContent, BytesReference.bytes(builder))) { + final ClusterStateDiffManifest parsedManifest = ClusterStateDiffManifest.fromXContent(parser, CODEC_V4); + assertEquals(diffManifest, parsedManifest); + } + } + + public void testClusterStateWithRoutingTableDiffInDiffManifestXContentWithDeletesV4() throws IOException { + ClusterState initialState = generateClusterStateWithOneIndex("test-index", 5, 1, true).nodes(nodesWithLocalNodeClusterManager()) + .build(); + + ClusterState updatedState = generateClusterStateWithOneIndex("test-index-1", 5, 2, false).nodes(nodesWithLocalNodeClusterManager()) + .build(); + + ClusterStateDiffManifest diffManifest = verifyRoutingTableDiffManifest(initialState, updatedState, CODEC_V4); + final XContentBuilder builder = JsonXContent.contentBuilder(); + builder.startObject(); + diffManifest.toXContent(builder, ToXContent.EMPTY_PARAMS); + builder.endObject(); + try (XContentParser parser = createParser(JsonXContent.jsonXContent, BytesReference.bytes(builder))) { + final ClusterStateDiffManifest parsedManifest = ClusterStateDiffManifest.fromXContent(parser, CODEC_V4); + assertEquals(diffManifest, parsedManifest); + } + } + + private ClusterStateDiffManifest verifyRoutingTableDiffManifest( + ClusterState previousState, + ClusterState currentState, + int codecVersion + ) { + // Create initial and updated IndexRoutingTable maps + StringKeyDiffProvider routingTableDiff = new RoutingTableIncrementalDiff( + previousState.getRoutingTable(), + currentState.getRoutingTable() ); ClusterStateDiffManifest manifest = new ClusterStateDiffManifest( currentState, previousState, - routingTableIncrementalDiff, + codecVersion, + routingTableDiff, "indicesRoutingDiffPath" ); assertEquals("indicesRoutingDiffPath", manifest.getIndicesRoutingDiffPath()); - assertEquals(routingTableIncrementalDiff.getUpserts().size(), manifest.getIndicesRoutingUpdated().size()); - assertEquals(routingTableIncrementalDiff.getDeletes().size(), manifest.getIndicesRoutingDeleted().size()); return manifest; } @@ -256,7 +291,7 @@ private ClusterStateDiffManifest updateAndVerifyState( } ClusterState updatedClusterState = clusterStateBuilder.metadata(metadataBuilder.build()).build(); - ClusterStateDiffManifest manifest = new ClusterStateDiffManifest(updatedClusterState, initialState, null, null); + ClusterStateDiffManifest manifest = new ClusterStateDiffManifest(updatedClusterState, initialState, CODEC_V4, null, null); assertEquals(indicesToAdd.stream().map(im -> im.getIndex().getName()).collect(toList()), manifest.getIndicesUpdated()); assertEquals(indicesToRemove, manifest.getIndicesDeleted()); assertEquals(new ArrayList<>(customsToAdd.keySet()), manifest.getCustomMetadataUpdated()); diff --git a/server/src/test/java/org/opensearch/gateway/remote/routingtable/RemoteIndexRoutingTableDiffTests.java b/server/src/test/java/org/opensearch/gateway/remote/routingtable/RemoteIndexRoutingTableDiffTests.java index 6ffa7fc5cded8..5dcc3127f2f5d 100644 --- a/server/src/test/java/org/opensearch/gateway/remote/routingtable/RemoteIndexRoutingTableDiffTests.java +++ b/server/src/test/java/org/opensearch/gateway/remote/routingtable/RemoteIndexRoutingTableDiffTests.java @@ -8,11 +8,8 @@ package org.opensearch.gateway.remote.routingtable; -import org.opensearch.Version; +import org.opensearch.cluster.ClusterState; import org.opensearch.cluster.Diff; -import org.opensearch.cluster.metadata.IndexMetadata; -import org.opensearch.cluster.metadata.Metadata; -import org.opensearch.cluster.routing.IndexRoutingTable; import org.opensearch.cluster.routing.RoutingTable; import org.opensearch.cluster.routing.RoutingTableIncrementalDiff; import org.opensearch.common.blobstore.BlobPath; @@ -35,14 +32,12 @@ import java.io.IOException; import java.io.InputStream; -import java.util.HashMap; import java.util.List; -import java.util.Map; +import static org.opensearch.gateway.remote.RemoteClusterStateServiceTests.generateClusterStateWithOneIndex; import static org.opensearch.gateway.remote.routingtable.RemoteRoutingTableDiff.ROUTING_TABLE_DIFF_FILE; import static org.opensearch.gateway.remote.routingtable.RemoteRoutingTableDiff.ROUTING_TABLE_DIFF_METADATA_PREFIX; import static org.opensearch.gateway.remote.routingtable.RemoteRoutingTableDiff.ROUTING_TABLE_DIFF_PATH_TOKEN; -import static org.hamcrest.Matchers.greaterThan; import static org.hamcrest.Matchers.is; import static org.hamcrest.Matchers.lessThanOrEqualTo; import static org.hamcrest.Matchers.nullValue; @@ -87,22 +82,14 @@ public void tearDown() throws Exception { } public void testClusterUUID() { - Map> diffs = new HashMap<>(); String indexName = randomAlphaOfLength(randomIntBetween(1, 50)); - int numberOfShards = randomIntBetween(1, 10); - int numberOfReplicas = randomIntBetween(1, 10); - - IndexMetadata indexMetadata = IndexMetadata.builder(indexName) - .settings(settings(Version.CURRENT)) - .numberOfShards(numberOfShards) - .numberOfReplicas(numberOfReplicas) - .build(); + ClusterState previousState = generateClusterStateWithOneIndex(indexName, 5, 1, false).build(); + ClusterState currentState = generateClusterStateWithOneIndex(indexName, 5, 2, true).build(); - IndexRoutingTable indexRoutingTable = IndexRoutingTable.builder(indexMetadata.getIndex()).initializeAsNew(indexMetadata).build(); - - diffs.put(indexName, indexRoutingTable.diff(indexRoutingTable)); - - RoutingTableIncrementalDiff routingTableIncrementalDiff = new RoutingTableIncrementalDiff(diffs); + RoutingTableIncrementalDiff routingTableIncrementalDiff = new RoutingTableIncrementalDiff( + previousState.getRoutingTable(), + currentState.getRoutingTable() + ); RemoteRoutingTableDiff remoteDiffForUpload = new RemoteRoutingTableDiff( routingTableIncrementalDiff, @@ -118,21 +105,14 @@ public void testClusterUUID() { } public void testFullBlobName() { - Map> diffs = new HashMap<>(); String indexName = randomAlphaOfLength(randomIntBetween(1, 50)); - int numberOfShards = randomIntBetween(1, 10); - int numberOfReplicas = randomIntBetween(1, 10); - - IndexMetadata indexMetadata = IndexMetadata.builder(indexName) - .settings(settings(Version.CURRENT)) - .numberOfShards(numberOfShards) - .numberOfReplicas(numberOfReplicas) - .build(); + ClusterState previousState = generateClusterStateWithOneIndex(indexName, 5, 1, false).build(); + ClusterState currentState = generateClusterStateWithOneIndex(indexName, 5, 2, true).build(); - IndexRoutingTable indexRoutingTable = IndexRoutingTable.builder(indexMetadata.getIndex()).initializeAsNew(indexMetadata).build(); - - diffs.put(indexName, indexRoutingTable.diff(indexRoutingTable)); - RoutingTableIncrementalDiff routingTableIncrementalDiff = new RoutingTableIncrementalDiff(diffs); + RoutingTableIncrementalDiff routingTableIncrementalDiff = new RoutingTableIncrementalDiff( + previousState.getRoutingTable(), + currentState.getRoutingTable() + ); RemoteRoutingTableDiff remoteDiffForUpload = new RemoteRoutingTableDiff( routingTableIncrementalDiff, @@ -148,21 +128,14 @@ public void testFullBlobName() { } public void testBlobFileName() { - Map> diffs = new HashMap<>(); String indexName = randomAlphaOfLength(randomIntBetween(1, 50)); - int numberOfShards = randomIntBetween(1, 10); - int numberOfReplicas = randomIntBetween(1, 10); - - IndexMetadata indexMetadata = IndexMetadata.builder(indexName) - .settings(settings(Version.CURRENT)) - .numberOfShards(numberOfShards) - .numberOfReplicas(numberOfReplicas) - .build(); - - IndexRoutingTable indexRoutingTable = IndexRoutingTable.builder(indexMetadata.getIndex()).initializeAsNew(indexMetadata).build(); + ClusterState previousState = generateClusterStateWithOneIndex(indexName, 5, 1, false).build(); + ClusterState currentState = generateClusterStateWithOneIndex(indexName, 5, 2, true).build(); - diffs.put(indexName, indexRoutingTable.diff(indexRoutingTable)); - RoutingTableIncrementalDiff routingTableIncrementalDiff = new RoutingTableIncrementalDiff(diffs); + RoutingTableIncrementalDiff routingTableIncrementalDiff = new RoutingTableIncrementalDiff( + previousState.getRoutingTable(), + currentState.getRoutingTable() + ); RemoteRoutingTableDiff remoteDiffForUpload = new RemoteRoutingTableDiff( routingTableIncrementalDiff, @@ -178,21 +151,14 @@ public void testBlobFileName() { } public void testBlobPathParameters() { - Map> diffs = new HashMap<>(); String indexName = randomAlphaOfLength(randomIntBetween(1, 50)); - int numberOfShards = randomIntBetween(1, 10); - int numberOfReplicas = randomIntBetween(1, 10); + ClusterState previousState = generateClusterStateWithOneIndex(indexName, 5, 1, false).build(); + ClusterState currentState = generateClusterStateWithOneIndex(indexName, 5, 2, true).build(); - IndexMetadata indexMetadata = IndexMetadata.builder(indexName) - .settings(settings(Version.CURRENT)) - .numberOfShards(numberOfShards) - .numberOfReplicas(numberOfReplicas) - .build(); - - IndexRoutingTable indexRoutingTable = IndexRoutingTable.builder(indexMetadata.getIndex()).initializeAsNew(indexMetadata).build(); - - diffs.put(indexName, indexRoutingTable.diff(indexRoutingTable)); - RoutingTableIncrementalDiff routingTableIncrementalDiff = new RoutingTableIncrementalDiff(diffs); + RoutingTableIncrementalDiff routingTableIncrementalDiff = new RoutingTableIncrementalDiff( + previousState.getRoutingTable(), + currentState.getRoutingTable() + ); RemoteRoutingTableDiff remoteDiffForUpload = new RemoteRoutingTableDiff( routingTableIncrementalDiff, @@ -210,21 +176,14 @@ public void testBlobPathParameters() { } public void testGenerateBlobFileName() { - Map> diffs = new HashMap<>(); String indexName = randomAlphaOfLength(randomIntBetween(1, 50)); - int numberOfShards = randomIntBetween(1, 10); - int numberOfReplicas = randomIntBetween(1, 10); - - IndexMetadata indexMetadata = IndexMetadata.builder(indexName) - .settings(settings(Version.CURRENT)) - .numberOfShards(numberOfShards) - .numberOfReplicas(numberOfReplicas) - .build(); + ClusterState previousState = generateClusterStateWithOneIndex(indexName, 5, 1, false).build(); + ClusterState currentState = generateClusterStateWithOneIndex(indexName, 5, 2, true).build(); - IndexRoutingTable indexRoutingTable = IndexRoutingTable.builder(indexMetadata.getIndex()).initializeAsNew(indexMetadata).build(); - - diffs.put(indexName, indexRoutingTable.diff(indexRoutingTable)); - RoutingTableIncrementalDiff routingTableIncrementalDiff = new RoutingTableIncrementalDiff(diffs); + RoutingTableIncrementalDiff routingTableIncrementalDiff = new RoutingTableIncrementalDiff( + previousState.getRoutingTable(), + currentState.getRoutingTable() + ); RemoteRoutingTableDiff remoteDiffForUpload = new RemoteRoutingTableDiff( routingTableIncrementalDiff, @@ -243,21 +202,14 @@ public void testGenerateBlobFileName() { } public void testGetUploadedMetadata() throws IOException { - Map> diffs = new HashMap<>(); String indexName = randomAlphaOfLength(randomIntBetween(1, 50)); - int numberOfShards = randomIntBetween(1, 10); - int numberOfReplicas = randomIntBetween(1, 10); - - IndexMetadata indexMetadata = IndexMetadata.builder(indexName) - .settings(settings(Version.CURRENT)) - .numberOfShards(numberOfShards) - .numberOfReplicas(numberOfReplicas) - .build(); - - IndexRoutingTable indexRoutingTable = IndexRoutingTable.builder(indexMetadata.getIndex()).initializeAsNew(indexMetadata).build(); + ClusterState previousState = generateClusterStateWithOneIndex(indexName, 5, 1, false).build(); + ClusterState currentState = generateClusterStateWithOneIndex(indexName, 5, 2, true).build(); - diffs.put(indexName, indexRoutingTable.diff(indexRoutingTable)); - RoutingTableIncrementalDiff routingTableIncrementalDiff = new RoutingTableIncrementalDiff(diffs); + RoutingTableIncrementalDiff routingTableIncrementalDiff = new RoutingTableIncrementalDiff( + previousState.getRoutingTable(), + currentState.getRoutingTable() + ); RemoteRoutingTableDiff remoteDiffForUpload = new RemoteRoutingTableDiff( routingTableIncrementalDiff, @@ -277,41 +229,35 @@ public void testStreamOperations() throws IOException { int numberOfShards = randomIntBetween(1, 10); int numberOfReplicas = randomIntBetween(1, 10); - Metadata metadata = Metadata.builder() - .put( - IndexMetadata.builder(indexName) - .settings(settings(Version.CURRENT)) - .numberOfShards(numberOfShards) - .numberOfReplicas(numberOfReplicas) - ) - .build(); - - RoutingTable initialRoutingTable = RoutingTable.builder().addAsNew(metadata.index(indexName)).build(); - Map> diffs = new HashMap<>(); - - initialRoutingTable.getIndicesRouting().values().forEach(indexRoutingTable -> { - diffs.put(indexName, indexRoutingTable.diff(indexRoutingTable)); - RoutingTableIncrementalDiff routingTableIncrementalDiff = new RoutingTableIncrementalDiff(diffs); - - RemoteRoutingTableDiff remoteDiffForUpload = new RemoteRoutingTableDiff( - routingTableIncrementalDiff, - clusterUUID, - compressor, - STATE_TERM, - STATE_VERSION - ); - - assertThrows(AssertionError.class, remoteDiffForUpload::getUploadedMetadata); - - try (InputStream inputStream = remoteDiffForUpload.serialize()) { - remoteDiffForUpload.setFullBlobName(BlobPath.cleanPath()); - assertThat(inputStream.available(), greaterThan(0)); - - routingTableIncrementalDiff = remoteDiffForUpload.deserialize(inputStream); - assertEquals(remoteDiffForUpload.getDiffs().size(), routingTableIncrementalDiff.getDiffs().size()); - } catch (IOException e) { - throw new RuntimeException(e); - } - }); + ClusterState previousState = generateClusterStateWithOneIndex(indexName, numberOfShards, numberOfReplicas, false).build(); + ClusterState currentState = generateClusterStateWithOneIndex(indexName, numberOfShards, numberOfReplicas + 1, true).build(); + + RoutingTableIncrementalDiff routingTableIncrementalDiff = new RoutingTableIncrementalDiff( + previousState.getRoutingTable(), + currentState.getRoutingTable() + ); + + RemoteRoutingTableDiff remoteDiffForUpload = new RemoteRoutingTableDiff( + routingTableIncrementalDiff, + clusterUUID, + compressor, + STATE_TERM, + STATE_VERSION + ); + + // Serialize the remote diff + InputStream inputStream = remoteDiffForUpload.serialize(); + + // Create a new instance for deserialization + RemoteRoutingTableDiff remoteDiffForDownload = new RemoteRoutingTableDiff(TEST_BLOB_NAME, clusterUUID, compressor); + + // Deserialize the remote diff + Diff deserializedDiff = remoteDiffForDownload.deserialize(inputStream); + + // Assert that the indices routing table created from routingTableIncrementalDiff and deserializedDiff is equal + assertEquals( + routingTableIncrementalDiff.apply(previousState.getRoutingTable()).getIndicesRouting(), + deserializedDiff.apply(previousState.getRoutingTable()).getIndicesRouting() + ); } } diff --git a/test/framework/src/main/java/org/opensearch/cluster/OpenSearchAllocationTestCase.java b/test/framework/src/main/java/org/opensearch/cluster/OpenSearchAllocationTestCase.java index f54ba36203684..bc42993ac2096 100644 --- a/test/framework/src/main/java/org/opensearch/cluster/OpenSearchAllocationTestCase.java +++ b/test/framework/src/main/java/org/opensearch/cluster/OpenSearchAllocationTestCase.java @@ -180,6 +180,19 @@ protected static DiscoveryNode newNode(String nodeId, Version version) { protected static ClusterState startRandomInitializingShard(ClusterState clusterState, AllocationService strategy) { List initializingShards = clusterState.getRoutingNodes().shardsWithState(INITIALIZING); + return startInitialisingShardsAndReroute(strategy, clusterState, initializingShards); + } + + protected static ClusterState startRandomInitializingShard(ClusterState clusterState, AllocationService strategy, String index) { + List initializingShards = clusterState.getRoutingNodes().shardsWithState(index, INITIALIZING); + return startInitialisingShardsAndReroute(strategy, clusterState, initializingShards); + } + + private static ClusterState startInitialisingShardsAndReroute( + AllocationService strategy, + ClusterState clusterState, + List initializingShards + ) { if (initializingShards.isEmpty()) { return clusterState; }