diff --git a/libs/core/src/main/java/org/opensearch/core/common/transport/TransportAddress.java b/libs/core/src/main/java/org/opensearch/core/common/transport/TransportAddress.java index 3b5fbb7d76307..b7ab988ce0404 100644 --- a/libs/core/src/main/java/org/opensearch/core/common/transport/TransportAddress.java +++ b/libs/core/src/main/java/org/opensearch/core/common/transport/TransportAddress.java @@ -162,4 +162,14 @@ public String toString() { public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { return builder.value(toString()); } + + public static TransportAddress fromString(String address) throws UnknownHostException { + String[] addressSplit = address.split(":"); + if (addressSplit.length != 2) { + throw new IllegalArgumentException("address must be of the form [hostname/ip]:[port]"); + } + String hostname = addressSplit[0]; + int port = Integer.parseInt(addressSplit[1]); + return new TransportAddress(InetAddress.getByName(hostname), port); + } } diff --git a/libs/core/src/main/java/org/opensearch/core/xcontent/XContentParserUtils.java b/libs/core/src/main/java/org/opensearch/core/xcontent/XContentParserUtils.java index b10be393f9adb..e0128a036148e 100644 --- a/libs/core/src/main/java/org/opensearch/core/xcontent/XContentParserUtils.java +++ b/libs/core/src/main/java/org/opensearch/core/xcontent/XContentParserUtils.java @@ -38,6 +38,8 @@ import org.opensearch.core.xcontent.XContentParser.Token; import java.io.IOException; +import java.util.ArrayList; +import java.util.List; import java.util.Locale; import java.util.function.Consumer; @@ -178,4 +180,14 @@ public static void parseTypedKeysObject(XContentParser parser, String delimi throw new ParsingException(parser.getTokenLocation(), "Failed to parse object: empty key"); } } + + public static List parseStringList(XContentParser parser) throws IOException { + List valueList = new ArrayList<>(); + ensureExpectedToken(Token.START_ARRAY, parser.currentToken(), parser); + while (parser.nextToken() != Token.END_ARRAY) { + ensureExpectedToken(Token.VALUE_STRING, parser.currentToken(), parser); + valueList.add(parser.text()); + } + return valueList; + } } diff --git a/server/src/internalClusterTest/java/org/opensearch/gateway/remote/RemoteClusterStateCleanupManagerIT.java b/server/src/internalClusterTest/java/org/opensearch/gateway/remote/RemoteClusterStateCleanupManagerIT.java new file mode 100644 index 0000000000000..b9dac874ed0c0 --- /dev/null +++ b/server/src/internalClusterTest/java/org/opensearch/gateway/remote/RemoteClusterStateCleanupManagerIT.java @@ -0,0 +1,139 @@ +/* + * 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.gateway.remote; + +import org.opensearch.action.admin.cluster.settings.ClusterUpdateSettingsResponse; +import org.opensearch.common.blobstore.BlobPath; +import org.opensearch.common.settings.Settings; +import org.opensearch.remotestore.RemoteStoreBaseIntegTestCase; +import org.opensearch.repositories.RepositoriesService; +import org.opensearch.repositories.blobstore.BlobStoreRepository; +import org.opensearch.test.OpenSearchIntegTestCase; + +import java.nio.charset.StandardCharsets; +import java.util.Base64; +import java.util.Map; +import java.util.concurrent.TimeUnit; + +import static org.opensearch.gateway.remote.RemoteClusterStateCleanupManager.CLUSTER_STATE_CLEANUP_INTERVAL_DEFAULT; +import static org.opensearch.gateway.remote.RemoteClusterStateCleanupManager.REMOTE_CLUSTER_STATE_CLEANUP_INTERVAL_SETTING; +import static org.opensearch.gateway.remote.RemoteClusterStateCleanupManager.RETAINED_MANIFESTS; +import static org.opensearch.gateway.remote.RemoteClusterStateCleanupManager.SKIP_CLEANUP_STATE_CHANGES; +import static org.opensearch.gateway.remote.RemoteClusterStateService.REMOTE_CLUSTER_STATE_ENABLED_SETTING; +import static org.opensearch.indices.IndicesService.CLUSTER_DEFAULT_INDEX_REFRESH_INTERVAL_SETTING; + +@OpenSearchIntegTestCase.ClusterScope(scope = OpenSearchIntegTestCase.Scope.TEST, numDataNodes = 0) +public class RemoteClusterStateCleanupManagerIT extends RemoteStoreBaseIntegTestCase { + + private static final String INDEX_NAME = "test-index"; + + @Override + protected Settings nodeSettings(int nodeOrdinal) { + return Settings.builder().put(super.nodeSettings(nodeOrdinal)).put(REMOTE_CLUSTER_STATE_ENABLED_SETTING.getKey(), true).build(); + } + + private Map initialTestSetup(int shardCount, int replicaCount, int dataNodeCount, int clusterManagerNodeCount) { + prepareCluster(clusterManagerNodeCount, dataNodeCount, INDEX_NAME, replicaCount, shardCount); + Map indexStats = indexData(1, false, INDEX_NAME); + assertEquals(shardCount * (replicaCount + 1), getNumShards(INDEX_NAME).totalNumShards); + ensureGreen(INDEX_NAME); + return indexStats; + } + + public void testRemoteCleanupTaskUpdated() { + int shardCount = randomIntBetween(1, 2); + int replicaCount = 1; + int dataNodeCount = shardCount * (replicaCount + 1); + int clusterManagerNodeCount = 1; + + initialTestSetup(shardCount, replicaCount, dataNodeCount, clusterManagerNodeCount); + RemoteClusterStateCleanupManager remoteClusterStateCleanupManager = internalCluster().getClusterManagerNodeInstance( + RemoteClusterStateCleanupManager.class + ); + + assertEquals(CLUSTER_STATE_CLEANUP_INTERVAL_DEFAULT, remoteClusterStateCleanupManager.getStaleFileDeletionTask().getInterval()); + assertTrue(remoteClusterStateCleanupManager.getStaleFileDeletionTask().isScheduled()); + + // now disable + client().admin() + .cluster() + .prepareUpdateSettings() + .setPersistentSettings(Settings.builder().put(REMOTE_CLUSTER_STATE_CLEANUP_INTERVAL_SETTING.getKey(), -1)) + .get(); + + assertEquals(-1, remoteClusterStateCleanupManager.getStaleFileDeletionTask().getInterval().getMillis()); + assertFalse(remoteClusterStateCleanupManager.getStaleFileDeletionTask().isScheduled()); + + // now set Clean up interval to 1 min + client().admin() + .cluster() + .prepareUpdateSettings() + .setPersistentSettings(Settings.builder().put(REMOTE_CLUSTER_STATE_CLEANUP_INTERVAL_SETTING.getKey(), "1m")) + .get(); + assertEquals(1, remoteClusterStateCleanupManager.getStaleFileDeletionTask().getInterval().getMinutes()); + } + + public void testRemoteCleanupDeleteStale() throws Exception { + int shardCount = randomIntBetween(1, 2); + int replicaCount = 1; + int dataNodeCount = shardCount * (replicaCount + 1); + int clusterManagerNodeCount = 1; + + initialTestSetup(shardCount, replicaCount, dataNodeCount, clusterManagerNodeCount); + + // set cleanup interval to 100 ms to make the test faster + ClusterUpdateSettingsResponse response = client().admin() + .cluster() + .prepareUpdateSettings() + .setPersistentSettings(Settings.builder().put(REMOTE_CLUSTER_STATE_CLEANUP_INTERVAL_SETTING.getKey(), "100ms")) + .get(); + + assertTrue(response.isAcknowledged()); + + // update cluster state 21 times to ensure that clean up has run after this will upload 42 manifest files + // to repository, if manifest files are less than that it means clean up has run + updateClusterStateNTimes(RETAINED_MANIFESTS + SKIP_CLEANUP_STATE_CHANGES + 1); + + RepositoriesService repositoriesService = internalCluster().getClusterManagerNodeInstance(RepositoriesService.class); + BlobStoreRepository repository = (BlobStoreRepository) repositoriesService.repository(REPOSITORY_NAME); + BlobPath baseMetadataPath = repository.basePath() + .add( + Base64.getUrlEncoder() + .withoutPadding() + .encodeToString(getClusterState().getClusterName().value().getBytes(StandardCharsets.UTF_8)) + ) + .add("cluster-state") + .add(getClusterState().metadata().clusterUUID()); + BlobPath manifestContainerPath = baseMetadataPath.add("manifest"); + + assertBusy(() -> { + int manifestFiles = repository.blobStore().blobContainer(manifestContainerPath).listBlobsByPrefix("manifest").size(); + logger.info("number of current manifest file: {}", manifestFiles); + // we can't guarantee that we have same number of manifest as Retained manifest in our repo as there can be other queued task + // other than replica count change which can upload new manifest files, that's why we check that number of manifests is between + // Retained manifests and Retained manifests + 2 * Skip cleanup state changes (each cluster state update uploads 2 manifests) + assertTrue( + "Current number of manifest files: " + manifestFiles, + manifestFiles >= RETAINED_MANIFESTS && manifestFiles < RETAINED_MANIFESTS + 2 * SKIP_CLEANUP_STATE_CHANGES + ); + }, 500, TimeUnit.MILLISECONDS); + } + + private void updateClusterStateNTimes(int n) { + int newReplicaCount = randomIntBetween(0, 3); + for (int i = n; i > 0; i--) { + ClusterUpdateSettingsResponse response = client().admin() + .cluster() + .prepareUpdateSettings() + .setPersistentSettings(Settings.builder().put(CLUSTER_DEFAULT_INDEX_REFRESH_INTERVAL_SETTING.getKey(), i, TimeUnit.SECONDS)) + .get(); + assertTrue(response.isAcknowledged()); + } + } +} diff --git a/server/src/internalClusterTest/java/org/opensearch/gateway/remote/RemoteClusterStateServiceIT.java b/server/src/internalClusterTest/java/org/opensearch/gateway/remote/RemoteClusterStateServiceIT.java index 42120aa32eb47..7620738c9a30d 100644 --- a/server/src/internalClusterTest/java/org/opensearch/gateway/remote/RemoteClusterStateServiceIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/gateway/remote/RemoteClusterStateServiceIT.java @@ -10,7 +10,6 @@ import org.opensearch.action.admin.cluster.node.stats.NodesStatsRequest; import org.opensearch.action.admin.cluster.node.stats.NodesStatsResponse; -import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.common.blobstore.BlobPath; import org.opensearch.common.settings.Settings; import org.opensearch.discovery.DiscoveryStats; @@ -28,13 +27,13 @@ import java.util.stream.Collectors; import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_NUMBER_OF_REPLICAS; -import static org.opensearch.gateway.remote.RemoteClusterStateService.COORDINATION_METADATA; -import static org.opensearch.gateway.remote.RemoteClusterStateService.CUSTOM_METADATA; -import static org.opensearch.gateway.remote.RemoteClusterStateService.DELIMITER; -import static org.opensearch.gateway.remote.RemoteClusterStateService.METADATA_FILE_PREFIX; import static org.opensearch.gateway.remote.RemoteClusterStateService.REMOTE_CLUSTER_STATE_ENABLED_SETTING; -import static org.opensearch.gateway.remote.RemoteClusterStateService.SETTING_METADATA; -import static org.opensearch.gateway.remote.RemoteClusterStateService.TEMPLATES_METADATA; +import static org.opensearch.gateway.remote.RemoteClusterStateUtils.DELIMITER; +import static org.opensearch.gateway.remote.RemoteClusterStateUtils.METADATA_FILE_PREFIX; +import static org.opensearch.gateway.remote.model.RemoteCoordinationMetadata.COORDINATION_METADATA; +import static org.opensearch.gateway.remote.model.RemoteCustomMetadata.CUSTOM_METADATA; +import static org.opensearch.gateway.remote.model.RemotePersistentSettingsMetadata.SETTING_METADATA; +import static org.opensearch.gateway.remote.model.RemoteTemplatesMetadata.TEMPLATES_METADATA; @OpenSearchIntegTestCase.ClusterScope(scope = OpenSearchIntegTestCase.Scope.TEST, numDataNodes = 0) public class RemoteClusterStateServiceIT extends RemoteStoreBaseIntegTestCase { @@ -51,16 +50,6 @@ protected Settings nodeSettings(int nodeOrdinal) { return Settings.builder().put(super.nodeSettings(nodeOrdinal)).put(REMOTE_CLUSTER_STATE_ENABLED_SETTING.getKey(), true).build(); } - private void prepareCluster(int numClusterManagerNodes, int numDataOnlyNodes, String indices, int replicaCount, int shardCount) { - internalCluster().startClusterManagerOnlyNodes(numClusterManagerNodes); - internalCluster().startDataOnlyNodes(numDataOnlyNodes); - for (String index : indices.split(",")) { - createIndex(index, remoteStoreIndexSettings(replicaCount, shardCount)); - ensureYellowAndNoInitializingShards(index); - ensureGreen(index); - } - } - private Map initialTestSetup(int shardCount, int replicaCount, int dataNodeCount, int clusterManagerNodeCount) { prepareCluster(clusterManagerNodeCount, dataNodeCount, INDEX_NAME, replicaCount, shardCount); Map indexStats = indexData(1, false, INDEX_NAME); @@ -69,49 +58,6 @@ private Map initialTestSetup(int shardCount, int replicaCount, int return indexStats; } - public void testFullClusterRestoreStaleDelete() throws Exception { - int shardCount = randomIntBetween(1, 2); - int replicaCount = 1; - int dataNodeCount = shardCount * (replicaCount + 1); - int clusterManagerNodeCount = 1; - - initialTestSetup(shardCount, replicaCount, dataNodeCount, clusterManagerNodeCount); - setReplicaCount(0); - setReplicaCount(2); - setReplicaCount(0); - setReplicaCount(1); - setReplicaCount(0); - setReplicaCount(1); - setReplicaCount(0); - setReplicaCount(2); - setReplicaCount(0); - - RemoteClusterStateService remoteClusterStateService = internalCluster().getClusterManagerNodeInstance( - RemoteClusterStateService.class - ); - - RepositoriesService repositoriesService = internalCluster().getClusterManagerNodeInstance(RepositoriesService.class); - - BlobStoreRepository repository = (BlobStoreRepository) repositoriesService.repository(REPOSITORY_NAME); - BlobPath baseMetadataPath = repository.basePath() - .add( - Base64.getUrlEncoder() - .withoutPadding() - .encodeToString(getClusterState().getClusterName().value().getBytes(StandardCharsets.UTF_8)) - ) - .add("cluster-state") - .add(getClusterState().metadata().clusterUUID()); - - assertEquals(10, repository.blobStore().blobContainer(baseMetadataPath.add("manifest")).listBlobsByPrefix("manifest").size()); - - Map indexMetadataMap = remoteClusterStateService.getLatestClusterState( - cluster().getClusterName(), - getClusterState().metadata().clusterUUID() - ).getMetadata().getIndices(); - assertEquals(0, indexMetadataMap.values().stream().findFirst().get().getNumberOfReplicas()); - assertEquals(shardCount, indexMetadataMap.values().stream().findFirst().get().getNumberOfShards()); - } - public void testRemoteStateStats() { int shardCount = randomIntBetween(1, 2); int replicaCount = 1; @@ -249,4 +195,17 @@ private void setReplicaCount(int replicaCount) { .setSettings(Settings.builder().put(SETTING_NUMBER_OF_REPLICAS, replicaCount)) .get(); } + + private int updateReplicaCountNTimes(int n, int initialCount) { + int newReplicaCount = randomIntBetween(0, 3); + ; + for (int i = 0; i < n; i++) { + while (newReplicaCount == initialCount) { + newReplicaCount = randomIntBetween(0, 3); + } + setReplicaCount(newReplicaCount); + initialCount = newReplicaCount; + } + return newReplicaCount; + } } diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreBaseIntegTestCase.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreBaseIntegTestCase.java index 740aee69f7d80..64efcee6ef1b5 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreBaseIntegTestCase.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreBaseIntegTestCase.java @@ -350,4 +350,14 @@ protected void restore(boolean restoreAllShards, String... indices) { PlainActionFuture.newFuture() ); } + + protected void prepareCluster(int numClusterManagerNodes, int numDataOnlyNodes, String indices, int replicaCount, int shardCount) { + internalCluster().startClusterManagerOnlyNodes(numClusterManagerNodes); + internalCluster().startDataOnlyNodes(numDataOnlyNodes); + for (String index : indices.split(",")) { + createIndex(index, remoteStoreIndexSettings(replicaCount, shardCount)); + ensureYellowAndNoInitializingShards(index); + ensureGreen(index); + } + } } diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreClusterStateRestoreIT.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreClusterStateRestoreIT.java index b22817ef19d1b..a4ed908e0e362 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreClusterStateRestoreIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreClusterStateRestoreIT.java @@ -36,6 +36,7 @@ import org.opensearch.gateway.remote.ClusterMetadataManifest; import org.opensearch.gateway.remote.ClusterMetadataManifest.UploadedIndexMetadata; import org.opensearch.gateway.remote.RemoteClusterStateService; +import org.opensearch.gateway.remote.RemoteClusterStateUtils; import org.opensearch.test.InternalTestCluster; import org.opensearch.test.OpenSearchIntegTestCase; import org.junit.Before; @@ -326,9 +327,7 @@ public void testFullClusterRestoreManifestFilePointsToInvalidIndexMetadataPathTh // Step - 3 Delete index metadata file in remote try { Files.move( - segmentRepoPath.resolve( - RemoteClusterStateService.encodeString(clusterName) + "/cluster-state/" + prevClusterUUID + "/index" - ), + segmentRepoPath.resolve(RemoteClusterStateUtils.encodeString(clusterName) + "/cluster-state/" + prevClusterUUID + "/index"), segmentRepoPath.resolve("cluster-state/") ); } catch (IOException e) { @@ -354,7 +353,7 @@ public void testRemoteStateFullRestart() throws Exception { try { Files.move( segmentRepoPath.resolve( - RemoteClusterStateService.encodeString(clusterService().state().getClusterName().value()) + RemoteClusterStateUtils.encodeString(clusterService().state().getClusterName().value()) + "/cluster-state/" + prevClusterUUID + "/manifest" diff --git a/server/src/main/java/org/opensearch/cluster/ClusterModule.java b/server/src/main/java/org/opensearch/cluster/ClusterModule.java index f56c906db1002..d8821bb721caa 100644 --- a/server/src/main/java/org/opensearch/cluster/ClusterModule.java +++ b/server/src/main/java/org/opensearch/cluster/ClusterModule.java @@ -320,6 +320,35 @@ public static List getNamedXWriteables() { DecommissionAttributeMetadata::fromXContent ) ); + // Cluster State + entries.add( + new NamedXContentRegistry.Entry( + ClusterState.Custom.class, + new ParseField(SnapshotsInProgress.TYPE), + SnapshotsInProgress::fromXContent + ) + ); + entries.add( + new NamedXContentRegistry.Entry( + ClusterState.Custom.class, + new ParseField(RestoreInProgress.TYPE), + RestoreInProgress::fromXContent + ) + ); + entries.add( + new NamedXContentRegistry.Entry( + ClusterState.Custom.class, + new ParseField(SnapshotDeletionsInProgress.TYPE), + SnapshotDeletionsInProgress::fromXContent + ) + ); + entries.add( + new NamedXContentRegistry.Entry( + ClusterState.Custom.class, + new ParseField(RepositoryCleanupInProgress.TYPE), + RepositoryCleanupInProgress::fromXContent + ) + ); return entries; } diff --git a/server/src/main/java/org/opensearch/cluster/ClusterState.java b/server/src/main/java/org/opensearch/cluster/ClusterState.java index 9e63f961d241d..ef3695b8d8c20 100644 --- a/server/src/main/java/org/opensearch/cluster/ClusterState.java +++ b/server/src/main/java/org/opensearch/cluster/ClusterState.java @@ -61,6 +61,7 @@ import org.opensearch.core.common.io.stream.VersionedNamedWriteable; import org.opensearch.core.xcontent.ToXContentFragment; import org.opensearch.core.xcontent.XContentBuilder; +import org.opensearch.core.xcontent.XContentParser; import org.opensearch.discovery.Discovery; import java.io.IOException; @@ -154,6 +155,9 @@ default boolean isPrivate() { return false; } + static Custom fromXContent(XContentParser parser, String name) throws IOException { + return parser.namedObject(Custom.class, name, null); + } } private static final NamedDiffableValueSerializer CUSTOM_VALUE_SERIALIZER = new NamedDiffableValueSerializer<>(Custom.class); @@ -496,38 +500,12 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws } if (metrics.contains(Metric.BLOCKS)) { - builder.startObject("blocks"); - - if (blocks().global().isEmpty() == false) { - builder.startObject("global"); - for (ClusterBlock block : blocks().global()) { - block.toXContent(builder, params); - } - builder.endObject(); - } - - if (blocks().indices().isEmpty() == false) { - builder.startObject("indices"); - for (final Map.Entry> entry : blocks().indices().entrySet()) { - builder.startObject(entry.getKey()); - for (ClusterBlock block : entry.getValue()) { - block.toXContent(builder, params); - } - builder.endObject(); - } - builder.endObject(); - } - - builder.endObject(); + blocks().toXContent(builder, params); } // nodes if (metrics.contains(Metric.NODES)) { - builder.startObject("nodes"); - for (DiscoveryNode node : nodes) { - node.toXContent(builder, params); - } - builder.endObject(); + nodes.toXContent(builder, params); } // meta data diff --git a/server/src/main/java/org/opensearch/cluster/RepositoryCleanupInProgress.java b/server/src/main/java/org/opensearch/cluster/RepositoryCleanupInProgress.java index 72a3519aca6f8..636db24d1dbad 100644 --- a/server/src/main/java/org/opensearch/cluster/RepositoryCleanupInProgress.java +++ b/server/src/main/java/org/opensearch/cluster/RepositoryCleanupInProgress.java @@ -33,12 +33,15 @@ import org.opensearch.LegacyESVersion; import org.opensearch.Version; +import org.opensearch.cluster.metadata.Metadata; import org.opensearch.core.common.Strings; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; import org.opensearch.core.common.io.stream.Writeable; import org.opensearch.core.xcontent.MediaTypeRegistry; import org.opensearch.core.xcontent.XContentBuilder; +import org.opensearch.core.xcontent.XContentParser; +import org.opensearch.core.xcontent.XContentParserUtils; import org.opensearch.repositories.RepositoryOperation; import java.io.IOException; @@ -101,6 +104,9 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws builder.startObject(); { builder.field("repository", entry.repository); + if (params.param(Metadata.CONTEXT_MODE_PARAM, Metadata.CONTEXT_MODE_API).equals(Metadata.CONTEXT_MODE_GATEWAY)) { + builder.field("repository_state_id", entry.repositoryStateId); + } // else we don't serialize it } builder.endObject(); } @@ -108,6 +114,36 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws return builder; } + public static RepositoryCleanupInProgress fromXContent(XContentParser parser) throws IOException { + if (parser.currentToken() == null) { + parser.nextToken(); + } + XContentParserUtils.ensureFieldName(parser, parser.currentToken(), TYPE); + parser.nextToken(); + XContentParserUtils.ensureExpectedToken(XContentParser.Token.START_ARRAY, parser.currentToken(), parser); + List entries = new ArrayList<>(); + while (parser.nextToken() != XContentParser.Token.END_ARRAY) { + XContentParserUtils.ensureExpectedToken(XContentParser.Token.START_OBJECT, parser.nextToken(), parser); + String repository = null; + long repositoryStateId = -1L; + while (parser.nextToken() != XContentParser.Token.END_OBJECT) { + XContentParserUtils.ensureExpectedToken(XContentParser.Token.FIELD_NAME, parser.currentToken(), parser); + String currentFieldName = parser.currentName(); + parser.nextToken(); + if ("repository".equals(currentFieldName)) { + repository = parser.text(); + } else if ("repository_state_id".equals(currentFieldName)) { + // only XContent parsed with {@link Metadata.CONTEXT_MODE_GATEWAY} will have the repository state id and can be deserialized + repositoryStateId = parser.longValue(); + } else { + throw new IllegalArgumentException("unknown field [" + currentFieldName + "]"); + } + } + entries.add(new Entry(repository, repositoryStateId)); + } + return new RepositoryCleanupInProgress(entries); + } + @Override public String toString() { return Strings.toString(MediaTypeRegistry.JSON, this); diff --git a/server/src/main/java/org/opensearch/cluster/RestoreInProgress.java b/server/src/main/java/org/opensearch/cluster/RestoreInProgress.java index 769f97373f7b7..a1fcbab3cbb21 100644 --- a/server/src/main/java/org/opensearch/cluster/RestoreInProgress.java +++ b/server/src/main/java/org/opensearch/cluster/RestoreInProgress.java @@ -34,16 +34,22 @@ import org.opensearch.Version; import org.opensearch.cluster.ClusterState.Custom; +import org.opensearch.cluster.metadata.Metadata; import org.opensearch.common.annotation.PublicApi; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; import org.opensearch.core.common.io.stream.Writeable; import org.opensearch.core.index.shard.ShardId; import org.opensearch.core.xcontent.ToXContent; +import org.opensearch.core.xcontent.ToXContentFragment; import org.opensearch.core.xcontent.XContentBuilder; +import org.opensearch.core.xcontent.XContentParser; +import org.opensearch.core.xcontent.XContentParserUtils; import org.opensearch.snapshots.Snapshot; +import org.opensearch.snapshots.SnapshotId; import java.io.IOException; +import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; import java.util.Iterator; @@ -52,6 +58,8 @@ import java.util.Objects; import java.util.UUID; +import static org.opensearch.core.xcontent.XContentParserUtils.ensureExpectedToken; + /** * Meta data about restore processes that are currently executing * @@ -144,7 +152,7 @@ public RestoreInProgress build() { * @opensearch.api */ @PublicApi(since = "1.0.0") - public static class Entry { + public static class Entry implements ToXContentFragment { private final String uuid; private final State state; private final Snapshot snapshot; @@ -236,6 +244,135 @@ public boolean equals(Object o) { public int hashCode() { return Objects.hash(uuid, snapshot, state, indices, shards); } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + boolean isGatewayXContent = params.param(Metadata.CONTEXT_MODE_PARAM, Metadata.CONTEXT_MODE_API) + .equals(Metadata.CONTEXT_MODE_GATEWAY); + builder.startObject(); + builder.field("snapshot", snapshot().getSnapshotId().getName()); + builder.field("repository", snapshot().getRepository()); + builder.field("state", state()); + if (isGatewayXContent) { + builder.field("snapshot_uuid", snapshot().getSnapshotId().getUUID()); + builder.field("uuid", uuid()); + } + builder.startArray("indices"); + { + for (String index : indices()) { + builder.value(index); + } + } + builder.endArray(); + builder.startArray("shards"); + { + for (final Map.Entry shardEntry : shards.entrySet()) { + ShardId shardId = shardEntry.getKey(); + ShardRestoreStatus status = shardEntry.getValue(); + builder.startObject(); + { + builder.field("index", shardId.getIndex()); + builder.field("shard", shardId.getId()); + builder.field("state", status.state()); + if (isGatewayXContent) { + builder.field("index_uuid", shardId.getIndex().getUUID()); + if (status.nodeId() != null) builder.field("node_id", status.nodeId()); + if (status.reason() != null) builder.field("reason", status.reason()); + } + } + builder.endObject(); + } + } + + builder.endArray(); + builder.endObject(); + return builder; + } + + public static Entry fromXContent(XContentParser parser) throws IOException { + if (parser.currentToken() == null) { + parser.nextToken(); + } + ensureExpectedToken(XContentParser.Token.START_OBJECT, parser.currentToken(), parser); + String snapshotName = null; + String snapshotRepository = null; + String snapshotUUID = null; + int state = -1; + String uuid = null; + List indices = new ArrayList<>(); + Map shards = new HashMap<>(); + while (parser.nextToken() != XContentParser.Token.END_OBJECT) { + ensureExpectedToken(XContentParser.Token.FIELD_NAME, parser.currentToken(), parser); + String currentFieldName = parser.currentName(); + parser.nextToken(); + switch (currentFieldName) { + case "snapshot": + snapshotName = parser.text(); + break; + case "repository": + snapshotRepository = parser.text(); + break; + case "state": + state = parser.intValue(); + break; + case "snapshot_uuid": + snapshotUUID = parser.text(); + break; + case "uuid": + uuid = parser.text(); + break; + case "indices": + ensureExpectedToken(XContentParser.Token.START_ARRAY, parser.currentToken(), parser); + while (parser.nextToken() != XContentParser.Token.END_ARRAY) { + indices.add(parser.text()); + } + break; + case "shards": + ensureExpectedToken(XContentParser.Token.START_ARRAY, parser.currentToken(), parser); + while (parser.nextToken() != XContentParser.Token.END_ARRAY) { + String indexName = null; + String indexUUID = null; + int shardId = -1; + int restoreState = -1; + String nodeId = null; + String reason = null; + ensureExpectedToken(XContentParser.Token.START_OBJECT, parser.currentToken(), parser); + while (parser.nextToken() != XContentParser.Token.END_OBJECT) { + ensureExpectedToken(XContentParser.Token.FIELD_NAME, parser.currentToken(), parser); + String currentShardFieldName = parser.currentName(); + parser.nextToken(); + switch (currentShardFieldName) { + case "index": + indexName = parser.text(); + break; + case "shard": + shardId = parser.intValue(); + break; + case "state": + restoreState = parser.intValue(); + break; + case "index_uuid": + indexUUID = parser.text(); + break; + case "node_id": + nodeId = parser.text(); + break; + case "reason": + reason = parser.text(); + break; + default: + throw new IllegalArgumentException("unknown field [" + currentFieldName + "]"); + } + } + shards.put(new ShardId(indexName, indexUUID, shardId), new ShardRestoreStatus(nodeId, State.fromValue((byte) restoreState), reason)); + } + break; + default: + throw new IllegalArgumentException("unknown field [" + currentFieldName + "]"); + } + } + return new Entry(uuid, new Snapshot(snapshotRepository, new SnapshotId(snapshotName, snapshotUUID)), State.fromValue((byte) state), indices, shards); + } } /** @@ -495,46 +632,57 @@ public void writeTo(StreamOutput out) throws IOException { public XContentBuilder toXContent(XContentBuilder builder, ToXContent.Params params) throws IOException { builder.startArray("snapshots"); for (final Entry entry : entries.values()) { - toXContent(entry, builder); + toXContent(entry, builder, ToXContent.EMPTY_PARAMS); } builder.endArray(); return builder; } + public static RestoreInProgress fromXContent(XContentParser parser) throws IOException { + if (parser.currentToken() == null) { + parser.nextToken(); + } + final Map entries = new HashMap<>(); + XContentParserUtils.ensureFieldName(parser, parser.currentToken(), "snapshots"); + while (parser.nextToken() != XContentParser.Token.END_ARRAY) { + final Entry entry = Entry.fromXContent(parser); + entries.put(entry.uuid, entry); + } + return new RestoreInProgress(entries); + } + /** * Serializes single restore operation * * @param entry restore operation metadata * @param builder XContent builder + * @param params */ - public void toXContent(Entry entry, XContentBuilder builder) throws IOException { - builder.startObject(); - builder.field("snapshot", entry.snapshot().getSnapshotId().getName()); - builder.field("repository", entry.snapshot().getRepository()); - builder.field("state", entry.state()); - builder.startArray("indices"); - { - for (String index : entry.indices()) { - builder.value(index); - } - } - builder.endArray(); - builder.startArray("shards"); - { - for (final Map.Entry shardEntry : entry.shards.entrySet()) { - ShardId shardId = shardEntry.getKey(); - ShardRestoreStatus status = shardEntry.getValue(); - builder.startObject(); - { - builder.field("index", shardId.getIndex()); - builder.field("shard", shardId.getId()); - builder.field("state", status.state()); - } - builder.endObject(); - } - } - - builder.endArray(); - builder.endObject(); + public void toXContent(Entry entry, XContentBuilder builder, ToXContent.Params params) throws IOException { + entry.toXContent(builder, params); } } + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/server/src/main/java/org/opensearch/cluster/SnapshotDeletionsInProgress.java b/server/src/main/java/org/opensearch/cluster/SnapshotDeletionsInProgress.java index e33245e02f75c..b59e0d8214932 100644 --- a/server/src/main/java/org/opensearch/cluster/SnapshotDeletionsInProgress.java +++ b/server/src/main/java/org/opensearch/cluster/SnapshotDeletionsInProgress.java @@ -34,12 +34,14 @@ import org.opensearch.Version; import org.opensearch.cluster.ClusterState.Custom; +import org.opensearch.cluster.metadata.Metadata; import org.opensearch.common.UUIDs; import org.opensearch.common.unit.TimeValue; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; import org.opensearch.core.common.io.stream.Writeable; import org.opensearch.core.xcontent.XContentBuilder; +import org.opensearch.core.xcontent.XContentParser; import org.opensearch.repositories.RepositoryOperation; import org.opensearch.snapshots.SnapshotId; @@ -52,6 +54,9 @@ import java.util.Objects; import java.util.Set; +import static org.opensearch.core.xcontent.XContentParserUtils.ensureExpectedToken; +import static org.opensearch.core.xcontent.XContentParserUtils.ensureFieldName; + /** * A class that represents the snapshot deletions that are in progress in the cluster. * @@ -72,7 +77,7 @@ private SnapshotDeletionsInProgress(List entries) { assert assertNoConcurrentDeletionsForSameRepository(entries); } - public static SnapshotDeletionsInProgress of(List entries) { + public static SnapshotDeletionsInProgress of(List entries) { if (entries.isEmpty()) { return EMPTY; } @@ -190,11 +195,22 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws builder.field("repository", entry.repository()); builder.startArray("snapshots"); for (SnapshotId snapshot : entry.snapshots) { - builder.value(snapshot.getName()); + if (params.param(Metadata.CONTEXT_MODE_PARAM, Metadata.CONTEXT_MODE_API).equals(Metadata.CONTEXT_MODE_GATEWAY)) { + builder.startObject(); + builder.field("name", snapshot.getName()); + builder.field("uuid", snapshot.getUUID()); + builder.endObject(); + } else { + builder.value(snapshot.getName()); + } } builder.endArray(); builder.humanReadableField("start_time_millis", "start_time", new TimeValue(entry.startTime)); builder.field("repository_state_id", entry.repositoryStateId); + if (params.param(Metadata.CONTEXT_MODE_PARAM, Metadata.CONTEXT_MODE_API).equals(Metadata.CONTEXT_MODE_GATEWAY)) { + builder.field("state", entry.state().value); + builder.field("uuid", entry.uuid()); + } // else we don't serialize it } builder.endObject(); } @@ -202,6 +218,80 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws return builder; } + public static SnapshotDeletionsInProgress fromXContent(XContentParser parser) throws IOException { + if (parser.currentToken() == null) { // fresh parser? move to the first token + parser.nextToken(); + } + if (parser.currentToken() == XContentParser.Token.START_OBJECT) { + parser.nextToken(); + } + ensureFieldName(parser, parser.currentToken(), TYPE); + parser.nextToken(); + ensureExpectedToken(XContentParser.Token.START_ARRAY, parser.currentToken(), parser); + List entries = new ArrayList<>(); + while ((parser.nextToken()) != XContentParser.Token.END_ARRAY) { + ensureExpectedToken(XContentParser.Token.START_OBJECT, parser.currentToken(), parser); + String repository = null; + long repositoryStateId = -1; + byte stateValue = -1; + List snapshotIds = new ArrayList<>(); + TimeValue startTime = null; + String entryUUID = null; + while ((parser.nextToken()) != XContentParser.Token.END_OBJECT) { + ensureExpectedToken(XContentParser.Token.FIELD_NAME, parser.currentToken(), parser); + final String fieldName = parser.currentName(); + parser.nextToken(); + switch (fieldName) { + case "repository": + repository = parser.text(); + break; + case "snapshots": + ensureExpectedToken(XContentParser.Token.START_ARRAY, parser.currentToken(), parser); + while ((parser.nextToken()) != XContentParser.Token.END_ARRAY) { + String name = null; + String uuid = null; + ensureExpectedToken(XContentParser.Token.START_OBJECT, parser.currentToken(), parser); + while ((parser.nextToken()) != XContentParser.Token.END_OBJECT) { + ensureExpectedToken(XContentParser.Token.FIELD_NAME, parser.currentToken(), parser); + final String currentFieldName = parser.currentName(); + parser.nextToken(); + if ("name".equals(currentFieldName)) { + name = parser.text(); + } else if ("uuid".equals(currentFieldName)) { + uuid = parser.text(); + } else { + throw new IllegalArgumentException("unknown field [" + currentFieldName + "]"); + } + } + snapshotIds.add(new SnapshotId(name, uuid)); + } + break; + case "start_time_millis": + startTime = TimeValue.timeValueMillis(parser.longValue()); + break; + case "start_time": + startTime = TimeValue.parseTimeValue(parser.text(), "start_time"); + break; + case "repository_state_id": + repositoryStateId = parser.longValue(); + break; + case "state": + stateValue = (byte) parser.intValue(); + break; + case "uuid": + entryUUID = parser.text(); + break; + default: + throw new IllegalArgumentException("unknown field [" + fieldName + "]"); + } + } + assert startTime != null; + entries.add(new Entry(snapshotIds, repository, startTime.millis(), repositoryStateId, State.fromValue(stateValue), entryUUID)); + } + return SnapshotDeletionsInProgress.of(entries); + } + + @Override public String toString() { StringBuilder builder = new StringBuilder("SnapshotDeletionsInProgress["); @@ -389,6 +479,17 @@ public static State readFrom(StreamInput in) throws IOException { } } + public static State fromValue(byte value) { + switch (value) { + case 0: + return WAITING; + case 1: + return STARTED; + default: + throw new IllegalArgumentException("No snapshot delete state for value [" + value + "]"); + } + } + @Override public void writeTo(StreamOutput out) throws IOException { out.writeByte(value); diff --git a/server/src/main/java/org/opensearch/cluster/SnapshotsInProgress.java b/server/src/main/java/org/opensearch/cluster/SnapshotsInProgress.java index d658f38430dd9..9a3f582995814 100644 --- a/server/src/main/java/org/opensearch/cluster/SnapshotsInProgress.java +++ b/server/src/main/java/org/opensearch/cluster/SnapshotsInProgress.java @@ -34,6 +34,7 @@ import org.opensearch.Version; import org.opensearch.cluster.ClusterState.Custom; +import org.opensearch.cluster.metadata.Metadata; import org.opensearch.common.Nullable; import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.unit.TimeValue; @@ -41,10 +42,12 @@ import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; import org.opensearch.core.common.io.stream.Writeable; +import org.opensearch.core.index.Index; import org.opensearch.core.index.shard.ShardId; import org.opensearch.core.xcontent.MediaTypeRegistry; import org.opensearch.core.xcontent.ToXContent; import org.opensearch.core.xcontent.XContentBuilder; +import org.opensearch.core.xcontent.XContentParser; import org.opensearch.repositories.IndexId; import org.opensearch.repositories.RepositoryOperation; import org.opensearch.repositories.RepositoryShardId; @@ -53,6 +56,7 @@ import org.opensearch.snapshots.SnapshotId; import java.io.IOException; +import java.util.ArrayList; import java.util.Collection; import java.util.Collections; import java.util.HashMap; @@ -63,6 +67,11 @@ import java.util.Set; import java.util.stream.Collectors; +import static org.opensearch.common.xcontent.XContentUtils.readValue; +import static org.opensearch.core.xcontent.XContentParserUtils.ensureExpectedToken; +import static org.opensearch.core.xcontent.XContentParserUtils.ensureFieldName; +import static org.opensearch.core.xcontent.XContentParserUtils.parseStringList; + /** * Meta data about snapshots that are currently executing * @@ -732,16 +741,227 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws builder.field(SHARD, shardId.getId()); builder.field(STATE, status.state()); builder.field(NODE, status.nodeId()); + if (params.param(Metadata.CONTEXT_MODE_PARAM, Metadata.CONTEXT_MODE_API).equals(Metadata.CONTEXT_MODE_GATEWAY)) { + if (status.generation() != null) builder.field(GENERATION, status.generation()); + if (status.reason() != null) builder.field(REASON, status.reason()); + } } builder.endObject(); } } builder.endArray(); builder.array(DATA_STREAMS, dataStreams.toArray(new String[0])); + if (params.param(Metadata.CONTEXT_MODE_PARAM, Metadata.CONTEXT_MODE_API).equals(Metadata.CONTEXT_MODE_GATEWAY)) { + builder.field(VERSION, version); + if (failure != null) builder.field(FAILURE, failure); + if (source != null) { + builder.field(SOURCE, source); + } + builder.field(USER_METADATA, userMetadata); + builder.startArray(CLONES); + for (RepositoryShardId shardId : clones.keySet()) { + builder.startObject(); + builder.field(INDEX, shardId.index().getName()); + builder.field(INDEX_ID, shardId.index().getId()); + builder.field(SHARD, shardId.shardId()); + ShardSnapshotStatus status = clones.get(shardId); + if (status.nodeId() != null) builder.field(NODE, status.nodeId()); + builder.field(STATE, status.state()); + if (status.generation() != null) builder.field(GENERATION, status.generation()); + if (status.reason() != null) builder.field(REASON, status.reason()); + builder.endObject(); + } + builder.endArray(); + builder.field(REMOTE_STORE_INDEX_SHALLOW_COPY, remoteStoreIndexShallowCopy); + } builder.endObject(); return builder; } + public static Entry fromXContent(XContentParser parser) throws IOException { + String repository = null; + String snapshotName = null; + String snapshotUUID = null; + boolean includeGlobalState = false; + boolean partial = false; + String failure = null; + Version version = null; + SnapshotId source = null; + Map metadata = null; + State state = null; + List indices = new ArrayList<>(); + long startTime = 0; + long repositoryStateId = -1L; + Map shards = new HashMap<>(); + List dataStreams = new ArrayList<>(); + Map clones = new HashMap<>(); + boolean remoteStoreIndexShallowCopy = false; + ensureExpectedToken(XContentParser.Token.START_OBJECT, parser.currentToken(), parser); + while (parser.nextToken() != XContentParser.Token.END_OBJECT) { + String currentFieldName = parser.currentName(); + parser.nextToken(); + switch (currentFieldName) { + case REPOSITORY: + repository = parser.text(); + break; + case SNAPSHOT: + snapshotName = parser.text(); + break; + case UUID: + snapshotUUID = parser.text(); + break; + case INCLUDE_GLOBAL_STATE: + includeGlobalState = parser.booleanValue(); + break; + case PARTIAL: + partial = parser.booleanValue(); + break; + case STATE: + state = State.fromString(parser.text()); + break; + case INDICES: + ensureExpectedToken(XContentParser.Token.START_ARRAY, parser.currentToken(), parser); + while (parser.nextToken() != XContentParser.Token.END_ARRAY) { + indices.add(IndexId.fromXContent(parser)); + } + break; + case START_TIME_MILLIS: + startTime = parser.longValue(); + break; + case START_TIME: + startTime = TimeValue.parseTimeValue(parser.text(), null, currentFieldName).millis(); + break; + case REPOSITORY_STATE_ID: + repositoryStateId = parser.longValue(); + break; + case SHARDS: + ensureExpectedToken(XContentParser.Token.START_ARRAY, parser.currentToken(), parser); + while (parser.nextToken() != XContentParser.Token.END_ARRAY) { + Index index = null; + int shardId = -1; + String nodeId = null; + ShardState shardState = null; + String reason = null; + String generation = null; + ensureExpectedToken(XContentParser.Token.START_OBJECT, parser.currentToken(), parser); + while (parser.nextToken() != XContentParser.Token.END_OBJECT) { + final String currentShardField = parser.currentName(); + parser.nextToken(); + switch (currentShardField) { + case INDEX: + index = Index.fromXContent(parser); + break; + case SHARD: + shardId = parser.intValue(); + break; + case NODE: + nodeId = (String) readValue(parser, parser.currentToken()); + break; + case STATE: + shardState = ShardState.fromString(parser.text()); + break; + case REASON: + reason = parser.text(); + break; + case GENERATION: + generation = parser.text(); + break; + default: + throw new IllegalArgumentException("unknown field [" + currentShardField + "]"); + } + } + shards.put(new ShardId(index, shardId), + reason != null ? new ShardSnapshotStatus(nodeId, shardState, reason, generation) : + new ShardSnapshotStatus(nodeId, shardState, generation)); + } + break; + case DATA_STREAMS: + dataStreams = parseStringList(parser); + break; + case FAILURE: + failure = parser.text(); + break; + case SOURCE: + source = SnapshotId.fromXContent(parser); + break; + case USER_METADATA: + metadata = parser.map(); + break; + case CLONES: + ensureExpectedToken(XContentParser.Token.START_ARRAY, parser.currentToken(), parser); + while (parser.nextToken() != XContentParser.Token.END_ARRAY) { + String index = null; + String indexId = null; + int shardId = -1; + byte snapshotShardStatus = -1; + String nodeId = null; + String reason = null; + String generation = null; + ensureExpectedToken(XContentParser.Token.START_OBJECT, parser.currentToken(), parser); + while (parser.nextToken() != XContentParser.Token.END_OBJECT) { + currentFieldName = parser.currentName(); + parser.nextToken(); + switch (currentFieldName) { + case INDEX: + index = parser.text(); + break; + case INDEX_ID: + indexId = parser.text(); + break; + case SHARD: + shardId = parser.intValue(); + break; + case STATE: + snapshotShardStatus = (byte) parser.intValue(); + break; + case NODE: + nodeId = parser.text(); + break; + case REASON: + reason = parser.text(); + break; + case GENERATION: + generation = parser.text(); + break; + default: + throw new IllegalArgumentException("unknown field [" + currentFieldName + "]"); + } + } + clones.put(new RepositoryShardId(new IndexId(index, indexId), shardId), + reason != null ? new ShardSnapshotStatus(nodeId, ShardState.fromValue(snapshotShardStatus), reason, generation) : + new ShardSnapshotStatus(nodeId, ShardState.fromValue(snapshotShardStatus), generation)); + } + break; + case VERSION: + version = Version.fromString(parser.text()); + break; + case REMOTE_STORE_INDEX_SHALLOW_COPY: + remoteStoreIndexShallowCopy = parser.booleanValue(); + break; + default: + throw new IllegalArgumentException("unknown field [" + currentFieldName + "]"); + } + } + Snapshot snapshot = new Snapshot(repository, new SnapshotId(snapshotName, snapshotUUID)); + return new Entry( + snapshot, + includeGlobalState, + partial, + state, + indices, + dataStreams, + startTime, + repositoryStateId, + shards, + failure, + metadata, + version, + source, + clones, + remoteStoreIndexShallowCopy + ); + } + @Override public void writeTo(StreamOutput out) throws IOException { snapshot.writeTo(out); @@ -980,6 +1200,25 @@ public static State fromValue(byte value) { throw new IllegalArgumentException("No snapshot state for value [" + value + "]"); } } + + public static State fromString(String value) { + switch(value) { + case "INIT": + return INIT; + case "STARTED": + return STARTED; + case "SUCCESS": + return SUCCESS; + case "FAILED": + return FAILED; + case "ABORTED": + return ABORTED; + case "PARTIAL": + return PARTIAL; + default: + throw new IllegalArgumentException("No snapshot state for value [" + value + "]"); + } + } } private final List entries; @@ -1057,6 +1296,12 @@ public void writeTo(StreamOutput out) throws IOException { private static final String INCLUDE_GLOBAL_STATE = "include_global_state"; private static final String PARTIAL = "partial"; private static final String STATE = "state"; + private static final String VERSION = "version"; + private static final String FAILURE = "failure"; + private static final String SOURCE = "source"; + private static final String USER_METADATA = "user_metadata"; + private static final String CLONES = "clones"; + private static final String REMOTE_STORE_INDEX_SHALLOW_COPY = "remote_store_index_shallow_copy"; private static final String INDICES = "indices"; private static final String DATA_STREAMS = "data_streams"; private static final String START_TIME_MILLIS = "start_time_millis"; @@ -1064,6 +1309,10 @@ public void writeTo(StreamOutput out) throws IOException { private static final String REPOSITORY_STATE_ID = "repository_state_id"; private static final String SHARDS = "shards"; private static final String INDEX = "index"; + private static final String INDEX_ID = "index_id"; + private static final String INDEX_UUID = "index_uuid"; + private static final String GENERATION = "generation"; + private static final String REASON = "reason"; private static final String SHARD = "shard"; private static final String NODE = "node"; @@ -1077,6 +1326,22 @@ public XContentBuilder toXContent(XContentBuilder builder, ToXContent.Params par return builder; } + public static SnapshotsInProgress fromXContent(XContentParser parser) throws IOException { + if (parser.currentToken() == null) { // fresh parser? move to the first token + parser.nextToken(); + } + if (parser.currentToken() == XContentParser.Token.START_OBJECT) { + parser.nextToken(); + } + ensureFieldName(parser, parser.currentToken(), SNAPSHOTS); + ensureExpectedToken(XContentParser.Token.START_ARRAY, parser.nextToken(), parser); + List entries = new ArrayList<>(); + while (parser.nextToken() != XContentParser.Token.END_ARRAY) { + entries.add(Entry.fromXContent(parser)); + } + return SnapshotsInProgress.of(entries); + } + /** * The shard state. * @@ -1137,5 +1402,26 @@ public static ShardState fromValue(byte value) { throw new IllegalArgumentException("No shard snapshot state for value [" + value + "]"); } } + + public static ShardState fromString(String state) { + switch (state) { + case "INIT": + return INIT; + case "SUCCESS": + return SUCCESS; + case "FAILED": + return FAILED; + case "ABORTED": + return ABORTED; + case "MISSING": + return MISSING; + case "WAITING": + return WAITING; + case "QUEUED": + return QUEUED; + default: + throw new IllegalArgumentException("No shard snapshot state for value [" + state + "]"); + } + } } } diff --git a/server/src/main/java/org/opensearch/cluster/block/ClusterBlock.java b/server/src/main/java/org/opensearch/cluster/block/ClusterBlock.java index 5fa897c0b1185..c958d7752cdbb 100644 --- a/server/src/main/java/org/opensearch/cluster/block/ClusterBlock.java +++ b/server/src/main/java/org/opensearch/cluster/block/ClusterBlock.java @@ -32,19 +32,26 @@ package org.opensearch.cluster.block; +import org.opensearch.cluster.metadata.Metadata; import org.opensearch.common.Nullable; import org.opensearch.common.annotation.PublicApi; +import org.opensearch.common.util.set.Sets; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; import org.opensearch.core.common.io.stream.Writeable; import org.opensearch.core.rest.RestStatus; import org.opensearch.core.xcontent.ToXContentFragment; import org.opensearch.core.xcontent.XContentBuilder; +import org.opensearch.core.xcontent.XContentParser; import java.io.IOException; import java.util.EnumSet; import java.util.Locale; import java.util.Objects; +import java.util.Set; + +import static org.opensearch.cluster.metadata.Metadata.CONTEXT_MODE_API; +import static org.opensearch.cluster.metadata.Metadata.CONTEXT_MODE_PARAM; /** * Blocks the cluster for concurrency @@ -54,6 +61,20 @@ @PublicApi(since = "1.0.0") public class ClusterBlock implements Writeable, ToXContentFragment { + static final String KEY_UUID = "uuid"; + static final String KEY_DESCRIPTION = "description"; + static final String KEY_RETRYABLE = "retryable"; + static final String KEY_DISABLE_STATE_PERSISTENCE = "disable_state_persistence"; + static final String KEY_LEVELS = "levels"; + static final String KEY_STATUS = "status"; + static final String KEY_ALLOW_RELEASE_RESOURCES = "allow_release_resources"; + private static final Set VALID_FIELDS = Sets.newHashSet( + KEY_UUID, + KEY_DESCRIPTION, + KEY_RETRYABLE, + KEY_DISABLE_STATE_PERSISTENCE, + KEY_LEVELS + ); private final int id; @Nullable private final String uuid; @@ -154,24 +175,99 @@ public boolean disableStatePersistence() { @Override public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + Metadata.XContentContext context = Metadata.XContentContext.valueOf(params.param(CONTEXT_MODE_PARAM, CONTEXT_MODE_API)); builder.startObject(Integer.toString(id)); if (uuid != null) { - builder.field("uuid", uuid); + builder.field(KEY_UUID, uuid); } - builder.field("description", description); - builder.field("retryable", retryable); + builder.field(KEY_DESCRIPTION, description); + builder.field(KEY_RETRYABLE, retryable); if (disableStatePersistence) { - builder.field("disable_state_persistence", disableStatePersistence); + builder.field(KEY_DISABLE_STATE_PERSISTENCE, disableStatePersistence); } - builder.startArray("levels"); + builder.startArray(KEY_LEVELS); for (ClusterBlockLevel level : levels) { builder.value(level.name().toLowerCase(Locale.ROOT)); } builder.endArray(); + if (context == Metadata.XContentContext.GATEWAY) { + builder.field(KEY_STATUS, status); + builder.field(KEY_ALLOW_RELEASE_RESOURCES, allowReleaseResources); + } builder.endObject(); return builder; } + public static ClusterBlock fromXContent(XContentParser parser, int id) throws IOException { + String uuid = null; + String description = null; + boolean retryable = false; + boolean disableStatePersistence = false; + RestStatus status = null; + boolean allowReleaseResources = false; + EnumSet levels = EnumSet.noneOf(ClusterBlockLevel.class); + String currentFieldName = skipBlockID(parser); + XContentParser.Token token; + while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) { + if (token == XContentParser.Token.FIELD_NAME) { + currentFieldName = parser.currentName(); + } else if (token.isValue()) { + switch (Objects.requireNonNull(currentFieldName)) { + case KEY_UUID: + uuid = parser.text(); + break; + case KEY_DESCRIPTION: + description = parser.text(); + break; + case KEY_RETRYABLE: + retryable = parser.booleanValue(); + break; + case KEY_DISABLE_STATE_PERSISTENCE: + disableStatePersistence = parser.booleanValue(); + break; + case KEY_STATUS: + status = RestStatus.valueOf(parser.text()); + break; + case KEY_ALLOW_RELEASE_RESOURCES: + allowReleaseResources = parser.booleanValue(); + break; + default: + throw new IllegalArgumentException("unknown field [" + currentFieldName + "]"); + } + } else if (token == XContentParser.Token.START_ARRAY) { + if (currentFieldName.equals(KEY_LEVELS)) { + while ((token = parser.nextToken()) != XContentParser.Token.END_ARRAY) { + levels.add(ClusterBlockLevel.fromString(parser.text(), Locale.ROOT)); + } + } else { + throw new IllegalArgumentException("unknown field [" + currentFieldName + "]"); + } + } else { + throw new IllegalArgumentException("unexpected token [" + token + "]"); + } + } + return new ClusterBlock(id, uuid, description, retryable, disableStatePersistence, allowReleaseResources, status, levels); + } + + private static String skipBlockID(XContentParser parser) throws IOException { + if (parser.currentToken() == null) { + parser.nextToken(); + } + if (parser.currentToken() == XContentParser.Token.START_OBJECT) { + parser.nextToken(); + if (parser.currentToken() == XContentParser.Token.FIELD_NAME) { + String currentFieldName = parser.currentName(); + if (VALID_FIELDS.contains(currentFieldName)) { + return currentFieldName; + } else { + // we have hit block id, just move on + parser.nextToken(); + } + } + } + return null; + } + @Override public void writeTo(StreamOutput out) throws IOException { out.writeVInt(id); diff --git a/server/src/main/java/org/opensearch/cluster/block/ClusterBlockLevel.java b/server/src/main/java/org/opensearch/cluster/block/ClusterBlockLevel.java index 5d3bf94aedb19..4940234c21ba6 100644 --- a/server/src/main/java/org/opensearch/cluster/block/ClusterBlockLevel.java +++ b/server/src/main/java/org/opensearch/cluster/block/ClusterBlockLevel.java @@ -35,6 +35,7 @@ import org.opensearch.common.annotation.PublicApi; import java.util.EnumSet; +import java.util.Locale; /** * What level to block the cluster @@ -51,4 +52,11 @@ public enum ClusterBlockLevel { public static final EnumSet ALL = EnumSet.allOf(ClusterBlockLevel.class); public static final EnumSet READ_WRITE = EnumSet.of(READ, WRITE); + + /* + * This method is used to convert a string to a ClusterBlockLevel. + * */ + public static ClusterBlockLevel fromString(String level, Locale locale) { + return ClusterBlockLevel.valueOf(level.toUpperCase(locale)); + } } diff --git a/server/src/main/java/org/opensearch/cluster/block/ClusterBlocks.java b/server/src/main/java/org/opensearch/cluster/block/ClusterBlocks.java index 02a20b7681ba7..e783ac1578d8f 100644 --- a/server/src/main/java/org/opensearch/cluster/block/ClusterBlocks.java +++ b/server/src/main/java/org/opensearch/cluster/block/ClusterBlocks.java @@ -42,6 +42,11 @@ import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; import org.opensearch.core.rest.RestStatus; +import org.opensearch.core.xcontent.ToXContent; +import org.opensearch.core.xcontent.ToXContentFragment; +import org.opensearch.core.xcontent.XContentBuilder; +import org.opensearch.core.xcontent.XContentParser; +import org.opensearch.core.xcontent.XContentParserUtils; import java.io.IOException; import java.util.Collections; @@ -62,7 +67,7 @@ * @opensearch.api */ @PublicApi(since = "1.0.0") -public class ClusterBlocks extends AbstractDiffable { +public class ClusterBlocks extends AbstractDiffable implements ToXContentFragment { public static final ClusterBlocks EMPTY_CLUSTER_BLOCK = new ClusterBlocks(emptySet(), Map.of()); private final Set global; @@ -325,6 +330,16 @@ public static Diff readDiffFrom(StreamInput in) throws IOExceptio return AbstractDiffable.readDiffFrom(ClusterBlocks::readFrom, in); } + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + Builder.toXContext(this, builder, params); + return builder; + } + + public static ClusterBlocks fromXContent(XContentParser parser) throws IOException { + return Builder.fromXContent(parser); + } + /** * An immutable level holder. * @@ -426,10 +441,16 @@ public Builder removeGlobalBlock(int blockId) { } public Builder addIndexBlock(String index, ClusterBlock block) { + prepareIndexForBlocks(index); + indices.get(index).add(block); + return this; + } + + // initialize an index adding further blocks + private Builder prepareIndexForBlocks(String index) { if (!indices.containsKey(index)) { indices.put(index, new HashSet<>()); } - indices.get(index).add(block); return this; } @@ -479,5 +500,82 @@ public ClusterBlocks build() { } return new ClusterBlocks(unmodifiableSet(new HashSet<>(global)), indicesBuilder); } + + public static void toXContext(ClusterBlocks blocks, XContentBuilder builder, ToXContent.Params params) throws IOException { + builder.startObject("blocks"); + if (blocks.global().isEmpty() == false) { + builder.startObject("global"); + for (ClusterBlock block : blocks.global()) { + block.toXContent(builder, params); + } + builder.endObject(); + } + + if (blocks.indices().isEmpty() == false) { + builder.startObject("indices"); + for (Map.Entry> entry : blocks.indices().entrySet()) { + builder.startObject(entry.getKey()); + for (ClusterBlock block : entry.getValue()) { + block.toXContent(builder, params); + } + builder.endObject(); + } + builder.endObject(); + } + builder.endObject(); + } + + public static ClusterBlocks fromXContent(XContentParser parser) throws IOException { + Builder builder = new Builder(); + String currentFieldName = skipBlocksField(parser); + XContentParser.Token token; + while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) { + XContentParserUtils.ensureExpectedToken(XContentParser.Token.FIELD_NAME, token, parser); + currentFieldName = parser.currentName(); + parser.nextToken(); + switch (currentFieldName) { + case "global": + while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) { + currentFieldName = parser.currentName(); + parser.nextToken(); + builder.addGlobalBlock(ClusterBlock.fromXContent(parser, Integer.parseInt(currentFieldName))); + } + break; + case "indices": + while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) { + String indexName = parser.currentName(); + parser.nextToken(); + // prepare for this index as we want to add this to ClusterBlocks even if there is no Block associated with it + builder.prepareIndexForBlocks(indexName); + while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) { + currentFieldName = parser.currentName(); + parser.nextToken(); + builder.addIndexBlock(indexName, ClusterBlock.fromXContent(parser, Integer.parseInt(currentFieldName))); + } + } + break; + default: + throw new IllegalArgumentException("unknown field [" + currentFieldName + "]"); + } + } + return builder.build(); + } + + private static String skipBlocksField(XContentParser parser) throws IOException { + if (parser.currentToken() == null) { + parser.nextToken(); + } + if (parser.currentToken() == XContentParser.Token.START_OBJECT) { + parser.nextToken(); + if (parser.currentToken() == XContentParser.Token.FIELD_NAME) { + if ("blocks".equals(parser.currentName())) { + parser.nextToken(); + } else { + return parser.currentName(); + } + } + } + return null; + } } } diff --git a/server/src/main/java/org/opensearch/cluster/coordination/CoordinationState.java b/server/src/main/java/org/opensearch/cluster/coordination/CoordinationState.java index 987a3e3ffa7d3..ede89b6d04cfe 100644 --- a/server/src/main/java/org/opensearch/cluster/coordination/CoordinationState.java +++ b/server/src/main/java/org/opensearch/cluster/coordination/CoordinationState.java @@ -50,6 +50,7 @@ import java.util.Map; import java.util.Optional; import java.util.Set; +import org.opensearch.node.remotestore.RemoteStoreNodeAttribute; import static org.opensearch.cluster.coordination.Coordinator.ZEN1_BWC_TERM; import static org.opensearch.node.remotestore.RemoteStoreNodeAttribute.isRemoteStoreClusterStateEnabled; @@ -79,6 +80,7 @@ public class CoordinationState { private VotingConfiguration lastPublishedConfiguration; private VoteCollection publishVotes; private final boolean isRemoteStateEnabled; + private final boolean isRemotePublicationEnabled; public CoordinationState( DiscoveryNode localNode, @@ -102,6 +104,11 @@ public CoordinationState( .getLastAcceptedConfiguration(); this.publishVotes = new VoteCollection(); this.isRemoteStateEnabled = isRemoteStoreClusterStateEnabled(settings); + this.isRemotePublicationEnabled = RemoteStoreNodeAttribute.isRemotePublicationEnabled(settings); + } + + public boolean isRemotePublicationEnabled() { + return isRemotePublicationEnabled; } public long getCurrentTerm() { diff --git a/server/src/main/java/org/opensearch/cluster/coordination/Coordinator.java b/server/src/main/java/org/opensearch/cluster/coordination/Coordinator.java index 3d74feddfa261..4a258a92d0ca6 100644 --- a/server/src/main/java/org/opensearch/cluster/coordination/Coordinator.java +++ b/server/src/main/java/org/opensearch/cluster/coordination/Coordinator.java @@ -48,6 +48,7 @@ import org.opensearch.cluster.coordination.CoordinationState.VoteCollection; import org.opensearch.cluster.coordination.FollowersChecker.FollowerCheckRequest; import org.opensearch.cluster.coordination.JoinHelper.InitialJoinAccumulator; +import org.opensearch.cluster.coordination.PersistedStateRegistry.PersistedStateType; import org.opensearch.cluster.metadata.Metadata; import org.opensearch.cluster.node.DiscoveryNode; import org.opensearch.cluster.node.DiscoveryNodes; @@ -84,6 +85,8 @@ import org.opensearch.discovery.PeerFinder; import org.opensearch.discovery.SeedHostsProvider; import org.opensearch.discovery.SeedHostsResolver; +import org.opensearch.gateway.GatewayMetaState.RemotePersistedState; +import org.opensearch.gateway.remote.RemoteClusterStateService; import org.opensearch.monitor.NodeHealthService; import org.opensearch.monitor.StatusInfo; import org.opensearch.node.remotestore.RemoteStoreNodeService; @@ -207,7 +210,8 @@ public Coordinator( ElectionStrategy electionStrategy, NodeHealthService nodeHealthService, PersistedStateRegistry persistedStateRegistry, - RemoteStoreNodeService remoteStoreNodeService + RemoteStoreNodeService remoteStoreNodeService, + RemoteClusterStateService remoteClusterStateService ) { this.settings = settings; this.transportService = transportService; @@ -259,7 +263,8 @@ public Coordinator( transportService, namedWriteableRegistry, this::handlePublishRequest, - this::handleApplyCommit + this::handleApplyCommit, + remoteClusterStateService ); this.leaderChecker = new LeaderChecker(settings, clusterSettings, transportService, this::onLeaderFailure, nodeHealthService); this.followersChecker = new FollowersChecker( @@ -1320,7 +1325,9 @@ assert getLocalNode().equals(clusterState.getNodes().get(getLocalNode().getId()) + clusterState; final PublicationTransportHandler.PublicationContext publicationContext = publicationHandler.newPublicationContext( - clusterChangedEvent + clusterChangedEvent, + coordinationState.get().isRemotePublicationEnabled(), + persistedStateRegistry ); final PublishRequest publishRequest = coordinationState.get().handleClientValue(clusterState); diff --git a/server/src/main/java/org/opensearch/cluster/coordination/PublicationTransportHandler.java b/server/src/main/java/org/opensearch/cluster/coordination/PublicationTransportHandler.java index 1fdaeead0d28d..545667ce75556 100644 --- a/server/src/main/java/org/opensearch/cluster/coordination/PublicationTransportHandler.java +++ b/server/src/main/java/org/opensearch/cluster/coordination/PublicationTransportHandler.java @@ -31,6 +31,9 @@ package org.opensearch.cluster.coordination; +import java.util.Locale; +import java.util.Optional; +import java.util.function.Supplier; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; @@ -40,6 +43,8 @@ import org.opensearch.cluster.ClusterState; import org.opensearch.cluster.Diff; import org.opensearch.cluster.IncompatibleClusterStateVersionException; +import org.opensearch.cluster.coordination.CoordinationState.PersistedState; +import org.opensearch.cluster.coordination.PersistedStateRegistry.PersistedStateType; import org.opensearch.cluster.node.DiscoveryNode; import org.opensearch.cluster.node.DiscoveryNodes; import org.opensearch.core.action.ActionListener; @@ -47,6 +52,9 @@ import org.opensearch.core.common.io.stream.NamedWriteableRegistry; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.transport.TransportResponse; +import org.opensearch.gateway.GatewayMetaState.RemotePersistedState; +import org.opensearch.gateway.remote.ClusterMetadataManifest; +import org.opensearch.gateway.remote.RemoteClusterStateService; import org.opensearch.threadpool.ThreadPool; import org.opensearch.transport.BytesTransportRequest; import org.opensearch.transport.TransportChannel; @@ -74,6 +82,7 @@ public class PublicationTransportHandler { private static final Logger logger = LogManager.getLogger(PublicationTransportHandler.class); public static final String PUBLISH_STATE_ACTION_NAME = "internal:cluster/coordination/publish_state"; + public static final String PUBLISH_REMOTE_STATE_ACTION_NAME = "internal:cluster/coordination/publish_remote_state"; public static final String COMMIT_STATE_ACTION_NAME = "internal:cluster/coordination/commit_state"; private final TransportService transportService; @@ -97,16 +106,19 @@ public class PublicationTransportHandler { private final TransportRequestOptions stateRequestOptions = TransportRequestOptions.builder() .withType(TransportRequestOptions.Type.STATE) .build(); + private final RemoteClusterStateService remoteClusterStateService; public PublicationTransportHandler( TransportService transportService, NamedWriteableRegistry namedWriteableRegistry, Function handlePublishRequest, - BiConsumer> handleApplyCommit + BiConsumer> handleApplyCommit, + RemoteClusterStateService remoteClusterStateService ) { this.transportService = transportService; this.namedWriteableRegistry = namedWriteableRegistry; this.handlePublishRequest = handlePublishRequest; + this.remoteClusterStateService = remoteClusterStateService; transportService.registerRequestHandler( PUBLISH_STATE_ACTION_NAME, @@ -117,6 +129,15 @@ public PublicationTransportHandler( (request, channel, task) -> channel.sendResponse(handleIncomingPublishRequest(request)) ); + transportService.registerRequestHandler( + PUBLISH_REMOTE_STATE_ACTION_NAME, + ThreadPool.Names.GENERIC, + false, + false, + RemotePublishRequest::new, + (request, channel, task) -> channel.sendResponse(handleIncomingRemotePublishRequest(request)) + ); + transportService.registerRequestHandler( COMMIT_STATE_ACTION_NAME, ThreadPool.Names.GENERIC, @@ -211,6 +232,49 @@ private PublishWithJoinResponse handleIncomingPublishRequest(BytesTransportReque } } + private PublishWithJoinResponse handleIncomingRemotePublishRequest(RemotePublishRequest request) throws IOException { + if (transportService.getLocalNode().equals(request.getSourceNode())) { + return acceptStateOnLocalNode(request); + } +// final Optional manifestOptional = remoteClusterStateService.getClusterMetadataManifestByTermVersion(request.getClusterName(), request.getClusterUUID(), request.term, request.version); +// if (manifestOptional.isPresent() == false) { +// throw new IllegalStateException( +// String.format(Locale.ROOT, "Manifest is not present for term - %s version - %s", request.term, request.version) +// ); +// } + ClusterMetadataManifest manifest = remoteClusterStateService.getClusterMetadataManifestByFileName(request.getClusterUUID(), request.getManifestFile()); + boolean applyFullState = false; + final ClusterState lastSeen = lastSeenClusterState.get(); + if (lastSeen == null) { + logger.debug(() -> "Diff cannot be applied as there is no last cluster state"); + applyFullState = true; + } else if (manifest.getDiffManifest() == null) { + logger.trace(() -> "There is no diff in the manifest"); + applyFullState = true; + } else if (manifest.getDiffManifest().getFromStateUUID().equals(lastSeen.stateUUID()) == false) { + logger.debug(() -> "Last cluster state not compatible with the diff"); + applyFullState = true; + } + + if (applyFullState == true) { + logger.info(() -> new ParameterizedMessage("Downloading full cluster state for term {}, version {}, stateUUID {}", manifest.getClusterTerm(), manifest.getStateVersion(), + manifest.getStateUUID())); + ClusterState clusterState = remoteClusterStateService.getClusterStateForManifest(request.getClusterName(), manifest, transportService.getLocalNode().getId(), true); + fullClusterStateReceivedCount.incrementAndGet(); + final PublishWithJoinResponse response = acceptState(clusterState); + lastSeenClusterState.set(clusterState); + return response; + } else { + logger.info(() -> new ParameterizedMessage("Downloading diff cluster state for term {}, version {}, previousUUID {}, current UUID {}", manifest.getClusterTerm(), + manifest.getStateVersion(), manifest.getDiffManifest().getFromStateUUID(), manifest.getStateUUID())); + ClusterState clusterState = remoteClusterStateService.getClusterStateUsingDiff(request.getClusterName(), manifest, lastSeen, transportService.getLocalNode().getId()); + compatibleClusterStateDiffReceivedCount.incrementAndGet(); + final PublishWithJoinResponse response = acceptState(clusterState); + lastSeenClusterState.compareAndSet(lastSeen, clusterState); + return response; + } + } + private PublishWithJoinResponse acceptState(ClusterState incomingState) { // if the state is coming from the current node, use original request instead (see currentPublishRequestToSelf for explanation) if (transportService.getLocalNode().equals(incomingState.nodes().getClusterManagerNode())) { @@ -224,8 +288,20 @@ private PublishWithJoinResponse acceptState(ClusterState incomingState) { return handlePublishRequest.apply(new PublishRequest(incomingState)); } - public PublicationContext newPublicationContext(ClusterChangedEvent clusterChangedEvent) { - final PublicationContext publicationContext = new PublicationContext(clusterChangedEvent); + private PublishWithJoinResponse acceptStateOnLocalNode(RemotePublishRequest remotePublishRequest) { + final PublishRequest publishRequest = currentPublishRequestToSelf.get(); + if (publishRequest == null || publishRequest.getAcceptedState().coordinationMetadata().term() != remotePublishRequest.term + || publishRequest.getAcceptedState().version() != remotePublishRequest.version) { + throw new IllegalStateException("publication to self failed for " + remotePublishRequest); + } + PublishWithJoinResponse publishWithJoinResponse = handlePublishRequest.apply(publishRequest); + lastSeenClusterState.set(publishRequest.getAcceptedState()); + return publishWithJoinResponse; + } + + public PublicationContext newPublicationContext(ClusterChangedEvent clusterChangedEvent, boolean isRemotePublicationEnabled, + PersistedStateRegistry persistedStateRegistry) { + final PublicationContext publicationContext = new PublicationContext(clusterChangedEvent, isRemotePublicationEnabled, persistedStateRegistry); // Build the serializations we expect to need now, early in the process, so that an error during serialization fails the publication // straight away. This isn't watertight since we send diffs on a best-effort basis and may fall back to sending a full state (and @@ -270,12 +346,16 @@ public class PublicationContext { private final boolean sendFullVersion; private final Map serializedStates = new HashMap<>(); private final Map serializedDiffs = new HashMap<>(); + private final boolean sendRemoteState; + private final PersistedStateRegistry persistedStateRegistry; - PublicationContext(ClusterChangedEvent clusterChangedEvent) { + PublicationContext(ClusterChangedEvent clusterChangedEvent, boolean isRemotePublicationEnabled, PersistedStateRegistry persistedStateRegistry) { discoveryNodes = clusterChangedEvent.state().nodes(); newState = clusterChangedEvent.state(); previousState = clusterChangedEvent.previousState(); sendFullVersion = previousState.getBlocks().disableStatePersistence(); + sendRemoteState = isRemotePublicationEnabled; + this.persistedStateRegistry = persistedStateRegistry; } void buildDiffAndSerializeStates() { @@ -339,7 +419,9 @@ public void onFailure(Exception e) { } else { responseActionListener = listener; } - if (sendFullVersion || previousState.nodes().nodeExists(destination) == false) { + if (sendRemoteState && destination.isRemoteClusterStateEnabled() && destination.isRemoteRoutingTableEnabled()) { + sendRemoteClusterState(destination, publishRequest.getAcceptedState(), responseActionListener); + } else if (sendFullVersion || previousState.nodes().nodeExists(destination) == false) { logger.trace("sending full cluster state version [{}] to [{}]", newState.version(), destination); sendFullClusterState(destination, responseActionListener); } else { @@ -384,6 +466,44 @@ public String executor() { ); } + private void sendRemoteClusterState(final DiscoveryNode destination, final ClusterState clusterState, final ActionListener listener) { + try { + final String manifestFileName = ((RemotePersistedState) persistedStateRegistry.getPersistedState(PersistedStateType.REMOTE)).getLastUploadedManifestFile(); + final RemotePublishRequest remotePublishRequest = new RemotePublishRequest(discoveryNodes.getLocalNode(), clusterState.term(), + clusterState.getVersion(), clusterState.getClusterName().value(), clusterState.metadata().clusterUUID(), manifestFileName); + final Consumer transportExceptionHandler = exp -> { + logger.debug(() -> new ParameterizedMessage("failed to send remote cluster state to {}", destination), exp); + listener.onFailure(exp); + }; + final TransportResponseHandler responseHandler = new TransportResponseHandler<>() { + + @Override + public PublishWithJoinResponse read(StreamInput in) throws IOException { + return new PublishWithJoinResponse(in); + } + + @Override + public void handleResponse(PublishWithJoinResponse response) { + listener.onResponse(response); + } + + @Override + public void handleException(TransportException exp) { + transportExceptionHandler.accept(exp); + } + + @Override + public String executor() { + return ThreadPool.Names.GENERIC; + } + }; + transportService.sendRequest(destination, PUBLISH_REMOTE_STATE_ACTION_NAME, remotePublishRequest, stateRequestOptions, responseHandler); + } catch (Exception e) { + logger.warn(() -> new ParameterizedMessage("error sending remote cluster state to {}", destination), e); + listener.onFailure(e); + } + } + private void sendFullClusterState(DiscoveryNode destination, ActionListener listener) { BytesReference bytes = serializedStates.get(destination.getVersion()); if (bytes == null) { diff --git a/server/src/main/java/org/opensearch/cluster/coordination/RemotePublishRequest.java b/server/src/main/java/org/opensearch/cluster/coordination/RemotePublishRequest.java new file mode 100644 index 0000000000000..68dea53372c45 --- /dev/null +++ b/server/src/main/java/org/opensearch/cluster/coordination/RemotePublishRequest.java @@ -0,0 +1,62 @@ +/* + * 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.coordination; + +import org.opensearch.cluster.node.DiscoveryNode; +import org.opensearch.core.common.io.stream.StreamInput; +import org.opensearch.core.common.io.stream.StreamOutput; + +import java.io.IOException; + +public class RemotePublishRequest extends TermVersionRequest { + + private final String clusterName; + private final String clusterUUID; + private final String manifestFile; + + public RemotePublishRequest(DiscoveryNode sourceNode, long term, long version, String clusterName, String clusterUUID, String manifestFile) { + super(sourceNode, term, version); + this.clusterName = clusterName; + this.clusterUUID = clusterUUID; + this.manifestFile = manifestFile; + } + + public RemotePublishRequest(StreamInput in) throws IOException { + super(in); + this.clusterName = in.readString(); + this.clusterUUID = in.readString(); + this.manifestFile = in.readString(); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + out.writeString(clusterName); + out.writeString(clusterUUID); + out.writeString(manifestFile); + } + + @Override + public String toString() { + return "RemotePublishRequest{" + "term=" + term + ", version=" + version + ", clusterName=" + clusterName + ", clusterUUID=" + clusterUUID + + ", sourceNode=" + sourceNode + ", manifestFile=" + manifestFile + '}'; + } + + public String getClusterName() { + return clusterName; + } + + public String getClusterUUID() { + return clusterUUID; + } + + public String getManifestFile() { + return manifestFile; + } +} diff --git a/server/src/main/java/org/opensearch/cluster/metadata/DiffableStringMap.java b/server/src/main/java/org/opensearch/cluster/metadata/DiffableStringMap.java index a8102182576ff..57f3051b96226 100644 --- a/server/src/main/java/org/opensearch/cluster/metadata/DiffableStringMap.java +++ b/server/src/main/java/org/opensearch/cluster/metadata/DiffableStringMap.java @@ -37,6 +37,9 @@ import org.opensearch.common.annotation.PublicApi; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; +import org.opensearch.core.xcontent.ToXContentObject; +import org.opensearch.core.xcontent.XContentBuilder; +import org.opensearch.core.xcontent.XContentParser; import java.io.IOException; import java.util.AbstractMap; @@ -46,6 +49,11 @@ import java.util.List; import java.util.Map; import java.util.Set; +import java.util.stream.Collectors; + +import static org.opensearch.core.xcontent.XContentParser.Token.END_OBJECT; +import static org.opensearch.core.xcontent.XContentParserUtils.ensureExpectedToken; +import static org.opensearch.core.xcontent.XContentParserUtils.ensureFieldName; /** * This is a {@code Map} that implements AbstractDiffable so it @@ -54,7 +62,7 @@ * @opensearch.api */ @PublicApi(since = "1.0.0") -public class DiffableStringMap extends AbstractMap implements Diffable { +public class DiffableStringMap extends AbstractMap implements Diffable, ToXContentObject { public static final DiffableStringMap EMPTY = new DiffableStringMap(Collections.emptyMap()); @@ -66,7 +74,7 @@ public static DiffableStringMap readFrom(StreamInput in) throws IOException { return map.isEmpty() ? EMPTY : new DiffableStringMap(map); } - DiffableStringMap(final Map map) { + public DiffableStringMap(final Map map) { this.innerMap = Collections.unmodifiableMap(map); } @@ -90,6 +98,28 @@ public static Diff readDiffFrom(StreamInput in) throws IOExce return new DiffableStringMapDiff(in); } + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.startObject("diffable_string_map"); + builder.field("inner_map", innerMap); + builder.endObject(); + return builder; + } + + public static DiffableStringMap fromXContent(XContentParser parser) throws IOException { + if (parser.currentToken() == null) { // fresh parser? move to next token + parser.nextToken(); + } + ensureFieldName(parser, parser.currentToken(), "diffable_string_map"); + ensureExpectedToken(XContentParser.Token.START_OBJECT, parser.currentToken(), parser); + parser.nextToken(); + ensureFieldName(parser, parser.currentToken(), "inner_map"); + parser.nextToken(); + Map innerMap = parser.mapOrdered(); + ensureExpectedToken(END_OBJECT, parser.currentToken(), parser); + return new DiffableStringMap(innerMap.entrySet().stream().collect(Collectors.toMap(Entry::getKey, entry -> (String) entry.getValue()))); + } + /** * Represents differences between two DiffableStringMaps. * diff --git a/server/src/main/java/org/opensearch/cluster/metadata/IndexMetadata.java b/server/src/main/java/org/opensearch/cluster/metadata/IndexMetadata.java index 9e7fe23f29872..2c305d1b2e69f 100644 --- a/server/src/main/java/org/opensearch/cluster/metadata/IndexMetadata.java +++ b/server/src/main/java/org/opensearch/cluster/metadata/IndexMetadata.java @@ -757,6 +757,10 @@ public String getIndexUUID() { return index.getUUID(); } + public String getIndexName() { + return index.getName(); + } + /** * Test whether the current index UUID is the same as the given one. Returns true if either are _na_ */ diff --git a/server/src/main/java/org/opensearch/cluster/metadata/Metadata.java b/server/src/main/java/org/opensearch/cluster/metadata/Metadata.java index e1aa5626f36c1..81416a79ef9c3 100644 --- a/server/src/main/java/org/opensearch/cluster/metadata/Metadata.java +++ b/server/src/main/java/org/opensearch/cluster/metadata/Metadata.java @@ -973,10 +973,19 @@ public static boolean isSettingsMetadataEqual(Metadata metadata1, Metadata metad return metadata1.persistentSettings.equals(metadata2.persistentSettings); } + public static boolean isTransientSettingsMetadataEqual(Metadata metadata1, Metadata metadata2) { + return metadata1.transientSettings.equals(metadata2.transientSettings); + } + public static boolean isTemplatesMetadataEqual(Metadata metadata1, Metadata metadata2) { return metadata1.templates.equals(metadata2.templates); } + public static boolean isHashesOfConsistentSettingsEqual(Metadata metadata1, Metadata metadata2) { + return metadata1.hashesOfConsistentSettings.equals(metadata2.hashesOfConsistentSettings); + } + + public static boolean isCustomMetadataEqual(Metadata metadata1, Metadata metadata2) { int customCount1 = 0; for (Map.Entry cursor : metadata1.customs.entrySet()) { diff --git a/server/src/main/java/org/opensearch/cluster/node/DiscoveryNode.java b/server/src/main/java/org/opensearch/cluster/node/DiscoveryNode.java index 5226e9570ac14..85c3e97ca71ad 100644 --- a/server/src/main/java/org/opensearch/cluster/node/DiscoveryNode.java +++ b/server/src/main/java/org/opensearch/cluster/node/DiscoveryNode.java @@ -33,6 +33,7 @@ package org.opensearch.cluster.node; import org.opensearch.Version; +import org.opensearch.cluster.metadata.Metadata; import org.opensearch.common.UUIDs; import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.settings.Setting; @@ -43,6 +44,7 @@ import org.opensearch.core.common.transport.TransportAddress; import org.opensearch.core.xcontent.ToXContentFragment; import org.opensearch.core.xcontent.XContentBuilder; +import org.opensearch.core.xcontent.XContentParser; import org.opensearch.node.Node; import java.io.IOException; @@ -60,8 +62,12 @@ import java.util.stream.Collectors; import java.util.stream.Stream; +import static org.opensearch.cluster.metadata.Metadata.CONTEXT_MODE_API; +import static org.opensearch.cluster.metadata.Metadata.CONTEXT_MODE_PARAM; import static org.opensearch.node.NodeRoleSettings.NODE_ROLES_SETTING; +import static org.opensearch.node.remotestore.RemoteStoreNodeAttribute.REMOTE_STORE_CLUSTER_STATE_REPOSITORY_NAME_ATTRIBUTE_KEY; import static org.opensearch.node.remotestore.RemoteStoreNodeAttribute.REMOTE_STORE_NODE_ATTRIBUTE_KEY_PREFIX; +import static org.opensearch.node.remotestore.RemoteStoreNodeAttribute.REMOTE_STORE_ROUTING_TABLE_REPOSITORY_NAME_ATTRIBUTE_KEY; /** * A discovery node represents a node that is part of the cluster. @@ -72,6 +78,14 @@ public class DiscoveryNode implements Writeable, ToXContentFragment { static final String COORDINATING_ONLY = "coordinating_only"; + static final String KEY_NAME = "name"; + static final String KEY_EPHEMERAL_ID = "ephemeral_id"; + static final String KEY_HOST_NAME = "host_name"; + static final String KEY_HOST_ADDRESS = "host_address"; + static final String KEY_TRANSPORT_ADDRESS = "transport_address"; + static final String KEY_ATTRIBUTES = "attributes"; + static final String KEY_VERSION = "version"; + static final String KEY_ROLES = "roles"; public static boolean nodeRequiresLocalStorage(Settings settings) { boolean localStorageEnable = Node.NODE_LOCAL_STORAGE_SETTING.get(settings); @@ -470,6 +484,25 @@ public boolean isRemoteStoreNode() { return this.getAttributes().keySet().stream().anyMatch(key -> key.startsWith(REMOTE_STORE_NODE_ATTRIBUTE_KEY_PREFIX)); } + /** + * Returns whether the node is a remote cluster state enabled node. + * @return true if the node contains remote cluster state node attribute, false otherwise + */ + public boolean isRemoteClusterStateEnabled() { + return this.getAttributes() + .keySet() + .stream() + .anyMatch(key -> (key.equals(REMOTE_STORE_CLUSTER_STATE_REPOSITORY_NAME_ATTRIBUTE_KEY))); + } + + /** + * Returns whether remote routing table is enabled on the node + * @return true if the node contains remote routing table node attributes, false otherwise + */ + public boolean isRemoteRoutingTableEnabled() { + return this.getAttributes().keySet().stream().anyMatch(key -> key.equals(REMOTE_STORE_ROUTING_TABLE_REPOSITORY_NAME_ATTRIBUTE_KEY)); + } + /** * Returns a set of all the roles that the node has. The roles are returned in sorted order by the role name. *

@@ -544,21 +577,97 @@ public String toString() { @Override public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + Metadata.XContentContext context = Metadata.XContentContext.valueOf(params.param(CONTEXT_MODE_PARAM, CONTEXT_MODE_API)); builder.startObject(getId()); - builder.field("name", getName()); - builder.field("ephemeral_id", getEphemeralId()); - builder.field("transport_address", getAddress().toString()); + builder.field(KEY_NAME, getName()); + builder.field(KEY_EPHEMERAL_ID, getEphemeralId()); + builder.field(KEY_TRANSPORT_ADDRESS, getAddress().toString()); - builder.startObject("attributes"); + builder.startObject(KEY_ATTRIBUTES); for (Map.Entry entry : attributes.entrySet()) { builder.field(entry.getKey(), entry.getValue()); } builder.endObject(); + if (context == Metadata.XContentContext.GATEWAY) { + builder.field(KEY_HOST_NAME, getHostName()); + builder.field(KEY_HOST_ADDRESS, getHostAddress()); + builder.field(KEY_VERSION, getVersion().toString()); + builder.startArray(KEY_ROLES); + for (DiscoveryNodeRole role : roles) { + builder.value(role.roleName()); + } + builder.endArray(); + } builder.endObject(); return builder; } + public static DiscoveryNode fromXContent(XContentParser parser, String nodeId) throws IOException { + if (parser.currentToken() == null) { + parser.nextToken(); + } + if (parser.currentToken() == XContentParser.Token.START_OBJECT) { + parser.nextToken(); + } + if (parser.currentToken() != XContentParser.Token.FIELD_NAME) { + throw new IllegalArgumentException("expected field name but got a " + parser.currentToken()); + } + String nodeName = null; + String hostName = null; + String hostAddress = null; + String ephemeralId = null; + TransportAddress transportAddress = null; + Map attributes = new HashMap<>(); + Set roles = new HashSet<>(); + Version version = null; + String currentFieldName = parser.currentName(); + // token should be start object at this point + // XContentParser.Token token = parser.nextToken(); + // if (token != XContentParser.Token.START_OBJECT) { + // throw new IllegalArgumentException("expected object but got a " + token); + // } + XContentParser.Token token; + while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) { + if (token == XContentParser.Token.FIELD_NAME) { + currentFieldName = parser.currentName(); + } else if (token.isValue()) { + if (KEY_NAME.equals(currentFieldName)) { + nodeName = parser.text(); + } else if (KEY_EPHEMERAL_ID.equals(currentFieldName)) { + ephemeralId = parser.text(); + } else if (KEY_TRANSPORT_ADDRESS.equals(currentFieldName)) { + transportAddress = TransportAddress.fromString(parser.text()); + } else if (KEY_HOST_NAME.equals(currentFieldName)) { + hostName = parser.text(); + } else if (KEY_HOST_ADDRESS.equals(currentFieldName)) { + hostAddress = parser.text(); + } else if (KEY_VERSION.equals(currentFieldName)) { + version = Version.fromString(parser.text()); + } + } else if (token == XContentParser.Token.START_OBJECT) { + if (KEY_ATTRIBUTES.equals(currentFieldName)) { + while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) { + if (token == XContentParser.Token.FIELD_NAME) { + currentFieldName = parser.currentName(); + } else if (token.isValue()) { + attributes.put(currentFieldName, parser.text()); + } + } + } + } else if (token == XContentParser.Token.START_ARRAY) { + if (KEY_ROLES.equals(currentFieldName)) { + while ((token = parser.nextToken()) != XContentParser.Token.END_ARRAY) { + roles.add(getRoleFromRoleName(parser.text())); + } + } + } else { + throw new IllegalArgumentException("unexpected token " + token); + } + } + return new DiscoveryNode(nodeName, nodeId, ephemeralId, hostName, hostAddress, transportAddress, attributes, roles, version); + } + private static Map rolesToMap(final Stream roles) { return Collections.unmodifiableMap(roles.collect(Collectors.toMap(DiscoveryNodeRole::roleName, Function.identity()))); } diff --git a/server/src/main/java/org/opensearch/cluster/node/DiscoveryNodes.java b/server/src/main/java/org/opensearch/cluster/node/DiscoveryNodes.java index 2ebcd8096893d..c5ecc2a188079 100644 --- a/server/src/main/java/org/opensearch/cluster/node/DiscoveryNodes.java +++ b/server/src/main/java/org/opensearch/cluster/node/DiscoveryNodes.java @@ -35,6 +35,7 @@ import org.opensearch.Version; import org.opensearch.cluster.AbstractDiffable; import org.opensearch.cluster.Diff; +import org.opensearch.cluster.metadata.Metadata; import org.opensearch.common.Booleans; import org.opensearch.common.Nullable; import org.opensearch.common.annotation.PublicApi; @@ -44,6 +45,10 @@ import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; import org.opensearch.core.common.transport.TransportAddress; +import org.opensearch.core.xcontent.ToXContentFragment; +import org.opensearch.core.xcontent.XContentBuilder; +import org.opensearch.core.xcontent.XContentParser; +import org.opensearch.core.xcontent.XContentParserUtils; import java.io.IOException; import java.util.ArrayList; @@ -59,6 +64,9 @@ import java.util.stream.Stream; import java.util.stream.StreamSupport; +import static org.opensearch.cluster.metadata.Metadata.CONTEXT_MODE_API; +import static org.opensearch.cluster.metadata.Metadata.CONTEXT_MODE_PARAM; + /** * This class holds all {@link DiscoveryNode} in the cluster and provides convenience methods to * access, modify merge / diff discovery nodes. @@ -66,7 +74,7 @@ * @opensearch.api */ @PublicApi(since = "1.0.0") -public class DiscoveryNodes extends AbstractDiffable implements Iterable { +public class DiscoveryNodes extends AbstractDiffable implements Iterable, ToXContentFragment { public static final DiscoveryNodes EMPTY_NODES = builder().build(); @@ -566,6 +574,64 @@ public String toString() { return sb.toString(); } + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.startObject("nodes"); + for (DiscoveryNode node : this) { + node.toXContent(builder, params); + } + builder.endObject(); + Metadata.XContentContext context = Metadata.XContentContext.valueOf(params.param(CONTEXT_MODE_PARAM, CONTEXT_MODE_API)); + if (context == Metadata.XContentContext.GATEWAY && clusterManagerNodeId != null) { + builder.field("cluster_manager", clusterManagerNodeId); + } + return builder; + } + + public static DiscoveryNodes fromXContent(XContentParser parser) throws IOException { + Builder builder = new Builder(); + if (parser.currentToken() == null) { + parser.nextToken(); + } + if (parser.currentToken() == XContentParser.Token.START_OBJECT) { + parser.nextToken(); + } + XContentParserUtils.ensureExpectedToken(XContentParser.Token.FIELD_NAME, parser.currentToken(), parser); + XContentParser.Token token; + String currentFieldName = parser.currentName(); + while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) { + if (token == XContentParser.Token.FIELD_NAME) { + currentFieldName = parser.currentName(); + } else if (token == XContentParser.Token.START_OBJECT) { + if ("nodes".equals(currentFieldName)) { + while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) { + if (token == XContentParser.Token.FIELD_NAME) { + currentFieldName = parser.currentName(); + } else if (token == XContentParser.Token.START_OBJECT) { + String nodeId = currentFieldName; + DiscoveryNode node = DiscoveryNode.fromXContent(parser, nodeId); + builder.add(node); + } + } + } else { + throw new IllegalArgumentException("unexpected object field " + currentFieldName); + } + } else if (token.isValue()) { + if ("cluster_manager".equals(currentFieldName)) { + String clusterManagerNodeId = parser.text(); + if (clusterManagerNodeId != null) { + builder.clusterManagerNodeId(clusterManagerNodeId); + } + } else { + throw new IllegalArgumentException("unexpected value field " + currentFieldName); + } + } else { + throw new IllegalArgumentException("unexpected token " + token); + } + } + return builder.build(); + } + /** * Delta between nodes. * 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 fd8cbea42c12f..2c0f8824ca4fc 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/IndexShardRoutingTable.java +++ b/server/src/main/java/org/opensearch/cluster/routing/IndexShardRoutingTable.java @@ -738,7 +738,7 @@ public boolean equals(Object o) { IndexShardRoutingTable that = (IndexShardRoutingTable) o; if (!shardId.equals(that.shardId)) return false; - if (!shards.equals(that.shards)) return false; + if(!new HashSet<>(shards).equals(new HashSet<>(that.shards))) return false; return true; } 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 e4095a84be081..6c7b94f316da2 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/RoutingTable.java +++ b/server/src/main/java/org/opensearch/cluster/routing/RoutingTable.java @@ -79,7 +79,7 @@ public class RoutingTable implements Iterable, Diffable indicesRouting; - private RoutingTable(long version, final Map indicesRouting) { + public RoutingTable(long version, final Map indicesRouting) { this.version = version; this.indicesRouting = Collections.unmodifiableMap(indicesRouting); } 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 ba2208e17df1f..83c756c98af14 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,19 +10,58 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; +import org.apache.lucene.store.IndexInput; +import org.opensearch.action.LatchedActionListener; +import org.opensearch.cluster.ClusterState; +import org.opensearch.cluster.DiffableUtils; +import org.opensearch.cluster.routing.IndexRoutingTable; +import org.opensearch.cluster.routing.RoutingTable; +import org.opensearch.common.CheckedRunnable; +import org.opensearch.common.blobstore.AsyncMultiStreamBlobContainer; +import org.opensearch.common.blobstore.BlobContainer; +import org.opensearch.common.blobstore.BlobPath; +import org.opensearch.common.blobstore.stream.write.WritePriority; +import org.opensearch.common.blobstore.transfer.RemoteTransferContainer; +import org.opensearch.common.blobstore.transfer.stream.OffsetRangeIndexInputStream; import org.opensearch.common.lifecycle.AbstractLifecycleComponent; + +import org.opensearch.common.lucene.store.ByteArrayIndexInput; +import org.opensearch.common.settings.Setting; import org.opensearch.common.settings.Settings; import org.opensearch.common.util.io.IOUtils; +import org.opensearch.core.action.ActionListener; +import org.opensearch.core.common.io.stream.StreamInput; +import org.opensearch.core.common.io.stream.StreamOutput; +import org.opensearch.core.index.Index; +import org.opensearch.gateway.remote.ClusterMetadataManifest; +import org.opensearch.gateway.remote.RemoteClusterStateUtils; +import org.opensearch.gateway.remote.routingtable.IndexRoutingTableInputStream; +import org.opensearch.gateway.remote.routingtable.IndexRoutingTableInputStreamReader; +import org.opensearch.index.remote.RemoteStoreEnums; +import org.opensearch.index.remote.RemoteStorePathStrategy; +import org.opensearch.index.remote.RemoteStoreUtils; import org.opensearch.node.Node; import org.opensearch.node.remotestore.RemoteStoreNodeAttribute; import org.opensearch.repositories.RepositoriesService; import org.opensearch.repositories.Repository; import org.opensearch.repositories.blobstore.BlobStoreRepository; +import org.opensearch.threadpool.ThreadPool; import java.io.IOException; + +import java.io.InputStream; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.ExecutorService; +import java.util.function.Function; import java.util.function.Supplier; +import java.util.stream.Collectors; -import static org.opensearch.node.remotestore.RemoteStoreNodeAttribute.isRemoteRoutingTableEnabled; +import static org.opensearch.gateway.remote.RemoteClusterStateUtils.getCusterMetadataBasePath; +import static org.opensearch.node.remotestore.RemoteStoreNodeAttribute.isRemotePublicationEnabled; /** * A Service which provides APIs to upload and download routing table from remote store. @@ -31,19 +70,227 @@ */ public class RemoteRoutingTableService extends AbstractLifecycleComponent { + /** + * Cluster setting to specify if routing table should be published to remote store + */ + public static final Setting REMOTE_ROUTING_TABLE_ENABLED_SETTING = Setting.boolSetting( + "cluster.remote_store.routing.enabled", + true, + Setting.Property.NodeScope, + Setting.Property.Final + ); + public static final String INDEX_ROUTING_PATH_TOKEN = "index-routing"; + public static final String ROUTING_TABLE = "routing-table"; + public static final String INDEX_ROUTING_FILE_PREFIX = "index_routing"; + public static final String DELIMITER = "__"; + public static final String INDEX_ROUTING_METADATA_PREFIX = "indexRouting--"; private static final Logger logger = LogManager.getLogger(RemoteRoutingTableService.class); private final Settings settings; private final Supplier repositoriesService; private BlobStoreRepository blobStoreRepository; + private final ThreadPool threadPool; + + private static final DiffableUtils.NonDiffableValueSerializer CUSTOM_ROUTING_TABLE_VALUE_SERIALIZER = new DiffableUtils.NonDiffableValueSerializer() { + @Override + public void write(IndexRoutingTable value, StreamOutput out) throws IOException { + value.writeTo(out); + } - public RemoteRoutingTableService(Supplier repositoriesService, Settings settings) { - assert isRemoteRoutingTableEnabled(settings) : "Remote routing table is not enabled"; + @Override + public IndexRoutingTable read(StreamInput in, String key) throws IOException { + return IndexRoutingTable.readFrom(in); + } + }; + + + public RemoteRoutingTableService(Supplier repositoriesService, + Settings settings, + ThreadPool threadPool) { + assert isRemotePublicationEnabled(settings) : "Remote routing table is not enabled"; this.repositoriesService = repositoriesService; this.settings = settings; + this.threadPool = threadPool; } + + public CheckedRunnable getIndexRoutingAsyncAction( + ClusterState clusterState, + IndexRoutingTable indexRouting, + LatchedActionListener latchedActionListener + ) throws IOException { + + BlobPath custerMetadataBasePath = getCusterMetadataBasePath(blobStoreRepository, clusterState.getClusterName().value(), + clusterState.metadata().clusterUUID()).add(INDEX_ROUTING_PATH_TOKEN); + logger.info("custerMetadataBasePath {}", custerMetadataBasePath); + + BlobPath path = RemoteStoreEnums.PathType.HASHED_PREFIX.path(RemoteStorePathStrategy.PathInput.builder() + .basePath(custerMetadataBasePath) + .indexUUID(indexRouting.getIndex().getUUID()) + .build(), + RemoteStoreEnums.PathHashAlgorithm.FNV_1A_BASE64); + logger.info("path from prefix hasd {}", path); + final BlobContainer blobContainer = blobStoreRepository.blobStore().blobContainer(path); + + final String fileName = getIndexRoutingFileName(); + logger.info("fileName {}", fileName); + + ActionListener completionListener = ActionListener.wrap( + resp -> latchedActionListener.onResponse( + new ClusterMetadataManifest.UploadedIndexMetadata( + + indexRouting.getIndex().getName(), + indexRouting.getIndex().getUUID(), + path.buildAsString() + fileName, + INDEX_ROUTING_METADATA_PREFIX + ) + ), + ex -> latchedActionListener.onFailure(new RemoteClusterStateUtils.RemoteStateTransferException(indexRouting.getIndex().toString(), ex)) + ); + + if (blobContainer instanceof AsyncMultiStreamBlobContainer == false) { + logger.info("TRYING FILE UPLOAD"); + + return () -> { + logger.info("Going to upload {}", indexRouting.prettyPrint()); + + uploadIndex(indexRouting, fileName , blobContainer); + logger.info("upload done {}", indexRouting.prettyPrint()); + + completionListener.onResponse(null); + logger.info("response done {}", indexRouting.prettyPrint()); + + }; + } + + logger.info("TRYING S3 UPLOAD"); + + //TODO: Integrate with S3AsyncCrtClient for using buffered stream directly with putObject. + try ( + InputStream indexRoutingStream = new IndexRoutingTableInputStream(indexRouting); + IndexInput input = new ByteArrayIndexInput("indexrouting", indexRoutingStream.readAllBytes())) { + //todo reenable checksum +// long expectedChecksum; +// try { +// expectedChecksum = checksumOfChecksum(input.clone(), 8); +// } catch (Exception e) { +// throw e; +// } + try ( + RemoteTransferContainer remoteTransferContainer = new RemoteTransferContainer( + fileName, + fileName, + input.length(), + true, + WritePriority.URGENT, + (size, position) -> new OffsetRangeIndexInputStream(input, size, position), +// expectedChecksum, +// ((AsyncMultiStreamBlobContainer) blobContainer).remoteIntegrityCheckSupported() + null, + false + ) + ) { + return () -> ((AsyncMultiStreamBlobContainer) blobContainer).asyncBlobUpload(remoteTransferContainer.createWriteContext(), completionListener); + } catch (IOException e) { + e.printStackTrace(); + return null; + } + } + } + + + public List getAllUploadedIndicesRouting(ClusterMetadataManifest previousManifest, List indicesRoutingToUpload, Set indicesRoutingToDelete) { + final Map allUploadedIndicesRouting = previousManifest.getIndicesRouting() + .stream() + .collect(Collectors.toMap(ClusterMetadataManifest.UploadedIndexMetadata::getIndexName, Function.identity())); + + indicesRoutingToUpload.forEach( + uploadedIndexRouting -> allUploadedIndicesRouting.put(uploadedIndexRouting.getIndexName(), uploadedIndexRouting) + ); + + indicesRoutingToDelete.forEach(allUploadedIndicesRouting::remove); + + logger.info("allUploadedIndicesRouting ROUTING {}", allUploadedIndicesRouting); + + return new ArrayList<>(allUploadedIndicesRouting.values()); + } + + private void uploadIndex(IndexRoutingTable indexRouting, String fileName, BlobContainer container) { + logger.info("Starting write"); + + try { + InputStream indexRoutingStream = new IndexRoutingTableInputStream(indexRouting); + container.writeBlob(fileName, indexRoutingStream, 4096, true); + logger.info("SUccessful write"); + } catch (IOException e) { + logger.error("Failed to write {}", e); + } + } + + private String getIndexRoutingFileName() { + return String.join( + DELIMITER, + INDEX_ROUTING_FILE_PREFIX, + RemoteStoreUtils.invertLong(System.currentTimeMillis()) + ); + + } + + public CheckedRunnable getAsyncIndexMetadataReadAction( + String uploadedFilename, + Index index, + LatchedActionListener latchedActionListener) { + int idx = uploadedFilename.lastIndexOf("/"); + String blobFileName = uploadedFilename.substring(idx+1); + BlobContainer blobContainer = blobStoreRepository.blobStore().blobContainer( BlobPath.cleanPath().add(uploadedFilename.substring(0,idx))); + + return () -> readAsync( + blobContainer, + blobFileName, + threadPool.executor(ThreadPool.Names.GENERIC), + ActionListener.wrap(response -> latchedActionListener.onResponse(response.readIndexRoutingTable(index)), latchedActionListener::onFailure) + ); + } + + public void readAsync(BlobContainer blobContainer, String name, ExecutorService executorService, ActionListener listener) throws IOException { + executorService.execute(() -> { + try { + listener.onResponse(read(blobContainer, name)); + } catch (Exception e) { + listener.onFailure(e); + } + }); + } + + public IndexRoutingTableInputStreamReader read(BlobContainer blobContainer, String path) { + try { + return new IndexRoutingTableInputStreamReader(blobContainer.readBlob(path)); + } catch (IOException e) { + logger.info("RoutingTable read failed with error: {}", e.toString()); + } + return null; + } + + public List getUpdatedIndexRoutingTableMetadata(List updatedIndicesRouting, List allIndicesRouting) { + return updatedIndicesRouting.stream().map(idx -> { + Optional uploadedIndexMetadataOptional = allIndicesRouting.stream().filter(idx2 -> idx2.getIndexName().equals(idx)).findFirst(); + assert uploadedIndexMetadataOptional.isPresent() == true; + return uploadedIndexMetadataOptional.get(); + }).collect(Collectors.toList()); + } + + + public static DiffableUtils.MapDiff> getIndicesRoutingMapDiff(RoutingTable before, RoutingTable after) { + return DiffableUtils.diff( + before.getIndicesRouting(), + after.getIndicesRouting(), + DiffableUtils.getStringKeySerializer(), + CUSTOM_ROUTING_TABLE_VALUE_SERIALIZER + ); + } + + @Override - protected void doClose() throws IOException { + public void doClose() throws IOException { if (blobStoreRepository != null) { IOUtils.close(blobStoreRepository); } @@ -51,7 +298,7 @@ protected void doClose() throws IOException { @Override protected void doStart() { - assert isRemoteRoutingTableEnabled(settings) == true : "Remote routing table is not enabled"; + assert isRemotePublicationEnabled(settings) == true : "Remote routing table is not enabled"; final String remoteStoreRepo = settings.get( Node.NODE_ATTRIBUTES.getKey() + RemoteStoreNodeAttribute.REMOTE_STORE_ROUTING_TABLE_REPOSITORY_NAME_ATTRIBUTE_KEY ); diff --git a/server/src/main/java/org/opensearch/common/io/Streams.java b/server/src/main/java/org/opensearch/common/io/Streams.java index f0d8f8b8bedbf..bb70d1901e6a7 100644 --- a/server/src/main/java/org/opensearch/common/io/Streams.java +++ b/server/src/main/java/org/opensearch/common/io/Streams.java @@ -44,10 +44,12 @@ import java.io.InputStreamReader; import java.io.OutputStream; import java.io.Reader; +import java.io.SequenceInputStream; import java.io.StringWriter; import java.io.Writer; import java.nio.charset.StandardCharsets; import java.util.ArrayList; +import java.util.Collections; import java.util.List; import java.util.Objects; import java.util.function.Consumer; @@ -221,6 +223,11 @@ public static BytesReference readFully(InputStream in) throws IOException { return out.bytes(); } + public static BytesReference readStreamList(List streams) throws IOException { + SequenceInputStream sis = new SequenceInputStream(Collections.enumeration(streams)); + return readFully(sis); + } + /** * Limits the given input stream to the provided number of bytes */ diff --git a/server/src/main/java/org/opensearch/index/translog/BufferedChecksumStreamInput.java b/server/src/main/java/org/opensearch/common/io/stream/BufferedChecksumStreamInput.java similarity index 99% rename from server/src/main/java/org/opensearch/index/translog/BufferedChecksumStreamInput.java rename to server/src/main/java/org/opensearch/common/io/stream/BufferedChecksumStreamInput.java index f75f27b7bcb91..f3341712275f9 100644 --- a/server/src/main/java/org/opensearch/index/translog/BufferedChecksumStreamInput.java +++ b/server/src/main/java/org/opensearch/common/io/stream/BufferedChecksumStreamInput.java @@ -30,7 +30,7 @@ * GitHub history for details. */ -package org.opensearch.index.translog; +package org.opensearch.common.io.stream; import org.apache.lucene.store.BufferedChecksum; import org.apache.lucene.util.BitUtil; diff --git a/server/src/main/java/org/opensearch/index/translog/BufferedChecksumStreamOutput.java b/server/src/main/java/org/opensearch/common/io/stream/BufferedChecksumStreamOutput.java similarity index 98% rename from server/src/main/java/org/opensearch/index/translog/BufferedChecksumStreamOutput.java rename to server/src/main/java/org/opensearch/common/io/stream/BufferedChecksumStreamOutput.java index 9e96664c79cc5..254f228f1c739 100644 --- a/server/src/main/java/org/opensearch/index/translog/BufferedChecksumStreamOutput.java +++ b/server/src/main/java/org/opensearch/common/io/stream/BufferedChecksumStreamOutput.java @@ -30,7 +30,7 @@ * GitHub history for details. */ -package org.opensearch.index.translog; +package org.opensearch.common.io.stream; import org.apache.lucene.store.BufferedChecksum; import org.opensearch.common.annotation.PublicApi; diff --git a/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java b/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java index 7814518af471b..bd93f9d925641 100644 --- a/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java +++ b/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java @@ -77,6 +77,7 @@ import org.opensearch.cluster.routing.allocation.decider.SameShardAllocationDecider; import org.opensearch.cluster.routing.allocation.decider.ShardsLimitAllocationDecider; import org.opensearch.cluster.routing.allocation.decider.ThrottlingAllocationDecider; +import org.opensearch.cluster.routing.remote.RemoteRoutingTableService; import org.opensearch.cluster.service.ClusterApplierService; import org.opensearch.cluster.service.ClusterManagerService; import org.opensearch.cluster.service.ClusterManagerTaskThrottler; @@ -104,7 +105,11 @@ import org.opensearch.gateway.GatewayService; import org.opensearch.gateway.PersistedClusterStateService; import org.opensearch.gateway.ShardsBatchGatewayAllocator; +import org.opensearch.gateway.remote.RemoteClusterStateCleanupManager; import org.opensearch.gateway.remote.RemoteClusterStateService; +import org.opensearch.gateway.remote.RemoteGlobalMetadataManager; +import org.opensearch.gateway.remote.RemoteIndexMetadataManager; +import org.opensearch.gateway.remote.RemoteManifestManager; import org.opensearch.http.HttpTransportSettings; import org.opensearch.index.IndexModule; import org.opensearch.index.IndexSettings; @@ -711,10 +716,13 @@ public void apply(Settings value, Settings current, Settings previous) { SearchRequestSlowLog.CLUSTER_SEARCH_REQUEST_SLOWLOG_LEVEL, // Remote cluster state settings + RemoteClusterStateCleanupManager.REMOTE_CLUSTER_STATE_CLEANUP_INTERVAL_SETTING, RemoteClusterStateService.REMOTE_CLUSTER_STATE_ENABLED_SETTING, - RemoteClusterStateService.INDEX_METADATA_UPLOAD_TIMEOUT_SETTING, - RemoteClusterStateService.GLOBAL_METADATA_UPLOAD_TIMEOUT_SETTING, - RemoteClusterStateService.METADATA_MANIFEST_UPLOAD_TIMEOUT_SETTING, + RemoteClusterStateService.REMOTE_STATE_READ_TIMEOUT_SETTING, + RemoteClusterStateService.REMOTE_STATE_READ_TIMEOUT_SETTING, + RemoteIndexMetadataManager.INDEX_METADATA_UPLOAD_TIMEOUT_SETTING, + RemoteGlobalMetadataManager.GLOBAL_METADATA_UPLOAD_TIMEOUT_SETTING, + RemoteManifestManager.METADATA_MANIFEST_UPLOAD_TIMEOUT_SETTING, RemoteStoreNodeService.REMOTE_STORE_COMPATIBILITY_MODE_SETTING, RemoteStoreNodeService.MIGRATION_DIRECTION_SETTING, IndicesService.CLUSTER_REMOTE_INDEX_RESTRICT_ASYNC_DURABILITY_SETTING, @@ -741,7 +749,10 @@ public void apply(Settings value, Settings current, Settings previous) { RemoteStoreSettings.CLUSTER_REMOTE_STORE_PATH_TYPE_SETTING, RemoteStoreSettings.CLUSTER_REMOTE_STORE_PATH_HASH_ALGORITHM_SETTING, RemoteStoreSettings.CLUSTER_REMOTE_MAX_TRANSLOG_READERS, - RemoteStoreSettings.CLUSTER_REMOTE_STORE_TRANSLOG_METADATA + RemoteStoreSettings.CLUSTER_REMOTE_STORE_TRANSLOG_METADATA, + + // Remote Routing table settings + RemoteRoutingTableService.REMOTE_ROUTING_TABLE_ENABLED_SETTING ) ) ); diff --git a/server/src/main/java/org/opensearch/discovery/DiscoveryModule.java b/server/src/main/java/org/opensearch/discovery/DiscoveryModule.java index 288371aa240a0..2edb9a61a85a8 100644 --- a/server/src/main/java/org/opensearch/discovery/DiscoveryModule.java +++ b/server/src/main/java/org/opensearch/discovery/DiscoveryModule.java @@ -52,6 +52,7 @@ import org.opensearch.core.common.io.stream.NamedWriteableRegistry; import org.opensearch.core.common.transport.TransportAddress; import org.opensearch.gateway.GatewayMetaState; +import org.opensearch.gateway.remote.RemoteClusterStateService; import org.opensearch.monitor.NodeHealthService; import org.opensearch.node.remotestore.RemoteStoreNodeService; import org.opensearch.plugins.DiscoveryPlugin; @@ -133,7 +134,8 @@ public DiscoveryModule( RerouteService rerouteService, NodeHealthService nodeHealthService, PersistedStateRegistry persistedStateRegistry, - RemoteStoreNodeService remoteStoreNodeService + RemoteStoreNodeService remoteStoreNodeService, + RemoteClusterStateService remoteClusterStateService ) { final Collection> joinValidators = new ArrayList<>(); final Map> hostProviders = new HashMap<>(); @@ -211,7 +213,8 @@ public DiscoveryModule( electionStrategy, nodeHealthService, persistedStateRegistry, - remoteStoreNodeService + remoteStoreNodeService, + remoteClusterStateService ); } else { throw new IllegalArgumentException("Unknown discovery type [" + discoveryType + "]"); diff --git a/server/src/main/java/org/opensearch/gateway/GatewayMetaState.java b/server/src/main/java/org/opensearch/gateway/GatewayMetaState.java index c3056276706a0..1cc0c9cbb6642 100644 --- a/server/src/main/java/org/opensearch/gateway/GatewayMetaState.java +++ b/server/src/main/java/org/opensearch/gateway/GatewayMetaState.java @@ -64,6 +64,7 @@ import org.opensearch.env.NodeMetadata; import org.opensearch.gateway.remote.ClusterMetadataManifest; import org.opensearch.gateway.remote.RemoteClusterStateService; +import org.opensearch.gateway.remote.model.RemoteUploadDetails; import org.opensearch.index.recovery.RemoteStoreRestoreService; import org.opensearch.index.recovery.RemoteStoreRestoreService.RemoteRestoreResult; import org.opensearch.node.Node; @@ -665,6 +666,8 @@ public static class RemotePersistedState implements PersistedState { private ClusterState lastAcceptedState; private ClusterMetadataManifest lastAcceptedManifest; + + private String lastUploadedManifestFile; private final RemoteClusterStateService remoteClusterStateService; private String previousClusterUUID; @@ -690,10 +693,14 @@ public void setCurrentTerm(long currentTerm) { // But for RemotePersistedState, the state is only pushed by the active cluster. So this method is not required. } + public String getLastUploadedManifestFile() { + return lastUploadedManifestFile; + } + @Override public void setLastAcceptedState(ClusterState clusterState) { try { - final ClusterMetadataManifest manifest; + final RemoteUploadDetails manifestDetails; if (shouldWriteFullClusterState(clusterState)) { final Optional latestManifest = remoteClusterStateService.getLatestClusterMetadataManifest( clusterState.getClusterName().value(), @@ -711,15 +718,16 @@ public void setLastAcceptedState(ClusterState clusterState) { clusterState.metadata().clusterUUID() ); } - manifest = remoteClusterStateService.writeFullMetadata(clusterState, previousClusterUUID); + manifestDetails = remoteClusterStateService.writeFullMetadata(clusterState, previousClusterUUID); } else { assert verifyManifestAndClusterState(lastAcceptedManifest, lastAcceptedState) == true : "Previous manifest and previous ClusterState are not in sync"; - manifest = remoteClusterStateService.writeIncrementalMetadata(lastAcceptedState, clusterState, lastAcceptedManifest); + manifestDetails = remoteClusterStateService.writeIncrementalMetadata(lastAcceptedState, clusterState, lastAcceptedManifest); } - assert verifyManifestAndClusterState(manifest, clusterState) == true : "Manifest and ClusterState are not in sync"; - lastAcceptedManifest = manifest; + assert verifyManifestAndClusterState(manifestDetails.getClusterMetadataManifest(), clusterState) == true : "Manifest and ClusterState are not in sync"; + lastAcceptedManifest = manifestDetails.getClusterMetadataManifest(); lastAcceptedState = clusterState; + lastUploadedManifestFile = manifestDetails.getManifestFileName(); } catch (Exception e) { remoteClusterStateService.writeMetadataFailed(); handleExceptionOnWrite(e); @@ -767,12 +775,13 @@ public void markLastAcceptedStateAsCommitted() { metadataBuilder.clusterUUIDCommitted(true); clusterState = ClusterState.builder(lastAcceptedState).metadata(metadataBuilder).build(); } - final ClusterMetadataManifest committedManifest = remoteClusterStateService.markLastStateAsCommitted( + final RemoteUploadDetails committedManifestDetails = remoteClusterStateService.markLastStateAsCommitted( clusterState, lastAcceptedManifest ); - lastAcceptedManifest = committedManifest; + lastAcceptedManifest = committedManifestDetails.getClusterMetadataManifest(); lastAcceptedState = clusterState; + lastUploadedManifestFile = committedManifestDetails.getManifestFileName(); } catch (Exception e) { handleExceptionOnWrite(e); } 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 bf02c73ca560b..92bec556afe97 100644 --- a/server/src/main/java/org/opensearch/gateway/remote/ClusterMetadataManifest.java +++ b/server/src/main/java/org/opensearch/gateway/remote/ClusterMetadataManifest.java @@ -8,6 +8,8 @@ package org.opensearch.gateway.remote; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.opensearch.Version; import org.opensearch.core.ParseField; import org.opensearch.core.common.Strings; @@ -41,6 +43,7 @@ public class ClusterMetadataManifest implements Writeable, ToXContentFragment { public static final int CODEC_V0 = 0; // Older codec version, where we haven't introduced codec versions for manifest. public static final int CODEC_V1 = 1; // In Codec V1 we have introduced global-metadata and codec version in Manifest file. public static final int CODEC_V2 = 2; // In Codec V2, there are seperate metadata files rather than a single global metadata file. + public static final int CODEC_V3 = 3; // In Codec V3, we introduced index routing-metadata, diff as part of manifest. private static final ParseField CLUSTER_TERM_FIELD = new ParseField("cluster_term"); private static final ParseField STATE_VERSION_FIELD = new ParseField("state_version"); @@ -54,10 +57,19 @@ public class ClusterMetadataManifest implements Writeable, ToXContentFragment { private static final ParseField INDICES_FIELD = new ParseField("indices"); private static final ParseField PREVIOUS_CLUSTER_UUID = new ParseField("previous_cluster_uuid"); private static final ParseField CLUSTER_UUID_COMMITTED = new ParseField("cluster_uuid_committed"); + private static final ParseField METADATA_VERSION = new ParseField("metadata_version"); private static final ParseField UPLOADED_COORDINATOR_METADATA = new ParseField("uploaded_coordinator_metadata"); private static final ParseField UPLOADED_SETTINGS_METADATA = new ParseField("uploaded_settings_metadata"); + private static final ParseField UPLOADED_TRANSIENT_SETTINGS_METADATA = new ParseField("uploaded_transient_settings_metadata"); private static final ParseField UPLOADED_TEMPLATES_METADATA = new ParseField("uploaded_templates_metadata"); private static final ParseField UPLOADED_CUSTOM_METADATA = new ParseField("uploaded_custom_metadata"); + private static final ParseField UPLOADED_DISCOVERY_NODES_METADATA = new ParseField("uploaded_discovery_nodes_metadata"); + private static final ParseField UPLOADED_CLUSTER_BLOCKS_METADATA = new ParseField("uploaded_cluster_blocks_metadata"); + private static final ParseField UPLOADED_HASHES_OF_CONSISTENT_SETTINGS_METADATA = new ParseField("uploaded_hashes_of_consistent_settings_metadata"); + private static final ParseField UPLOADED_CLUSTER_STATE_CUSTOM_METADATA = new ParseField("uploaded_cluster_state_custom_metadata"); + private static final ParseField DIFF_MANIFEST = new ParseField("diff_manifest"); + private static final ParseField ROUTING_TABLE_VERSION_FIELD = new ParseField("routing_table_version"); + private static final ParseField INDICES_ROUTING_FIELD = new ParseField("indices_routing"); private static ClusterMetadataManifest.Builder manifestV0Builder(Object[] fields) { return ClusterMetadataManifest.builder() @@ -86,6 +98,19 @@ private static ClusterMetadataManifest.Builder manifestV2Builder(Object[] fields .customMetadataMap(customMetadata(fields)); } + private static ClusterMetadataManifest.Builder manifestV3Builder(Object[] fields) { + return manifestV2Builder(fields) + .discoveryNodesMetadata(discoveryNodesMetadata(fields)) + .clusterBlocksMetadata(clusterBlocksMetadata(fields)) + .diffManifest(diffManifest(fields)) + .routingTableVersion(routingTableVersion(fields)) + .indicesRouting(indicesRouting(fields)) + .metadataVersion(metadataVersion(fields)) + .transientSettingsMetadata(transientSettingsMetadata(fields)) + .hashesOfConsistentSettings(hashesOfConsistentSettings(fields)) + .clusterStateCustomMetadataMap(clusterStateCustomMetadata(fields)); + } + private static long term(Object[] fields) { return (long) fields[0]; } @@ -151,6 +176,43 @@ private static Map customMetadata(Object[] fi return customs.stream().collect(Collectors.toMap(UploadedMetadataAttribute::getAttributeName, Function.identity())); } + private static UploadedMetadataAttribute discoveryNodesMetadata(Object[] fields) { + return (UploadedMetadataAttribute) fields[15]; + } + + private static UploadedMetadataAttribute clusterBlocksMetadata(Object[] fields) { + return (UploadedMetadataAttribute) fields[16]; + } + + private static ClusterStateDiffManifest diffManifest(Object[] fields) { + return (ClusterStateDiffManifest) fields[17]; + } + + private static long routingTableVersion(Object[] fields) { + return (long) fields[18]; + } + + private static List indicesRouting(Object[] fields) { + return (List) fields[19]; + } + + private static long metadataVersion(Object[] fields) { + return (long) fields[20]; + } + + private static UploadedMetadataAttribute transientSettingsMetadata(Object[] fields) { + return (UploadedMetadataAttribute) fields[21]; + } + + private static UploadedMetadataAttribute hashesOfConsistentSettings(Object[] fields) { + return (UploadedMetadataAttribute) fields[22]; + } + + private static Map clusterStateCustomMetadata(Object[] fields) { + List customs = (List) fields[23]; + return customs.stream().collect(Collectors.toMap(UploadedMetadataAttribute::getAttributeName, Function.identity())); + } + private static final ConstructingObjectParser PARSER_V0 = new ConstructingObjectParser<>( "cluster_metadata_manifest", fields -> manifestV0Builder(fields).build() @@ -166,12 +228,18 @@ private static Map customMetadata(Object[] fi fields -> manifestV2Builder(fields).build() ); - private static final ConstructingObjectParser CURRENT_PARSER = PARSER_V2; + private static final ConstructingObjectParser PARSER_V3 = new ConstructingObjectParser<>( + "cluster_metadata_manifest", + fields -> manifestV3Builder(fields).build() + ); + + private static final ConstructingObjectParser CURRENT_PARSER = PARSER_V3; static { declareParser(PARSER_V0, CODEC_V0); declareParser(PARSER_V1, CODEC_V1); declareParser(PARSER_V2, CODEC_V2); + declareParser(PARSER_V3, CODEC_V3); } private static void declareParser(ConstructingObjectParser parser, long codec_version) { @@ -216,13 +284,56 @@ private static void declareParser(ConstructingObjectParser= CODEC_V3) { + parser.declareNamedObject( + ConstructingObjectParser.optionalConstructorArg(), + UploadedMetadataAttribute.PARSER, + UPLOADED_DISCOVERY_NODES_METADATA + ); + parser.declareNamedObject( + ConstructingObjectParser.optionalConstructorArg(), + UploadedMetadataAttribute.PARSER, + UPLOADED_CLUSTER_BLOCKS_METADATA + ); + parser.declareObject( + ConstructingObjectParser.constructorArg(), + (p, c) -> ClusterStateDiffManifest.fromXContent(p), + DIFF_MANIFEST + ); + parser.declareLong(ConstructingObjectParser.constructorArg(), ROUTING_TABLE_VERSION_FIELD); + parser.declareObjectArray( + ConstructingObjectParser.constructorArg(), + (p, c) -> UploadedIndexMetadata.fromXContent(p), + INDICES_ROUTING_FIELD + ); + parser.declareLong(ConstructingObjectParser.constructorArg(), METADATA_VERSION); + parser.declareNamedObject( + ConstructingObjectParser.optionalConstructorArg(), + UploadedMetadataAttribute.PARSER, + UPLOADED_TRANSIENT_SETTINGS_METADATA + ); + parser.declareNamedObject( + ConstructingObjectParser.optionalConstructorArg(), + UploadedMetadataAttribute.PARSER, + UPLOADED_HASHES_OF_CONSISTENT_SETTINGS_METADATA + ); + parser.declareNamedObjects( + ConstructingObjectParser.optionalConstructorArg(), + UploadedMetadataAttribute.PARSER, + UPLOADED_CLUSTER_STATE_CUSTOM_METADATA + ); + } } private final int codecVersion; private final String globalMetadataFileName; + private final long metadataVersion; private final UploadedMetadataAttribute uploadedCoordinationMetadata; private final UploadedMetadataAttribute uploadedSettingsMetadata; + private final UploadedMetadataAttribute uploadedTransientSettingsMetadata; private final UploadedMetadataAttribute uploadedTemplatesMetadata; + private final UploadedMetadataAttribute uploadedDiscoveryNodesMetadata; + private final UploadedMetadataAttribute uploadedClusterBlocksMetadata; private final Map uploadedCustomMetadataMap; private final List indices; private final long clusterTerm; @@ -234,6 +345,11 @@ private static void declareParser(ConstructingObjectParser indicesRouting; + private final UploadedMetadataAttribute uploadedHashesOfConsistentSettings; + private final Map uploadedClusterStateCustomMap; public List getIndices() { return indices; @@ -283,6 +399,12 @@ public String getGlobalMetadataFileName() { return globalMetadataFileName; } + private static final Logger logger = LogManager.getLogger(ClusterMetadataManifest.class); + + public long getMetadataVersion() { + return metadataVersion; + } + public UploadedMetadataAttribute getCoordinationMetadata() { return uploadedCoordinationMetadata; } @@ -291,14 +413,50 @@ public UploadedMetadataAttribute getSettingsMetadata() { return uploadedSettingsMetadata; } + public UploadedMetadataAttribute getTransientSettingsMetadata() { + return uploadedTransientSettingsMetadata; + } + public UploadedMetadataAttribute getTemplatesMetadata() { return uploadedTemplatesMetadata; } + public UploadedMetadataAttribute getDiscoveryNodesMetadata() { + return uploadedDiscoveryNodesMetadata; + } + + public UploadedMetadataAttribute getClusterBlocksMetadata() { + return uploadedClusterBlocksMetadata; + } + + public ClusterStateDiffManifest getDiffManifest() { + return diffManifest; + } + public Map getCustomMetadataMap() { return uploadedCustomMetadataMap; } + public UploadedMetadataAttribute getDiscoverNodeMetadata() { + return uploadedDiscoveryNodesMetadata; + } + + public Map getClusterStateCustomMap() { + return uploadedClusterStateCustomMap; + } + + public UploadedMetadataAttribute getHashesOfConsistentSettings() { + return uploadedHashesOfConsistentSettings; + } + + public long getRoutingTableVersion() { + return routingTableVersion; + } + + public List getIndicesRouting() { + return indicesRouting; + } + public boolean hasMetadataAttributesFiles() { return uploadedCoordinationMetadata != null || uploadedSettingsMetadata != null @@ -322,7 +480,16 @@ public ClusterMetadataManifest( UploadedMetadataAttribute uploadedCoordinationMetadata, UploadedMetadataAttribute uploadedSettingsMetadata, UploadedMetadataAttribute uploadedTemplatesMetadata, - Map uploadedCustomMetadataMap + Map uploadedCustomMetadataMap, + UploadedMetadataAttribute discoveryNodesMetadata, + UploadedMetadataAttribute clusterBlocksMetadata, + ClusterStateDiffManifest diffManifest, + long routingTableVersion, + List indicesRouting, + long metadataVersion, + UploadedMetadataAttribute uploadedTransientSettingsMetadata, + UploadedMetadataAttribute uploadedHashesOfConsistentSettings, + Map uploadedClusterStateCustomMap ) { this.clusterTerm = clusterTerm; this.stateVersion = version; @@ -342,6 +509,17 @@ public ClusterMetadataManifest( this.uploadedCustomMetadataMap = Collections.unmodifiableMap( uploadedCustomMetadataMap != null ? uploadedCustomMetadataMap : new HashMap<>() ); + this.uploadedDiscoveryNodesMetadata = discoveryNodesMetadata; + this.uploadedClusterBlocksMetadata = clusterBlocksMetadata; + this.diffManifest = diffManifest; + this.routingTableVersion = routingTableVersion; + this.indicesRouting = Collections.unmodifiableList(indicesRouting); + this.metadataVersion = metadataVersion; + this.uploadedTransientSettingsMetadata = uploadedTransientSettingsMetadata; + this.uploadedHashesOfConsistentSettings = uploadedHashesOfConsistentSettings; + this.uploadedClusterStateCustomMap = Collections.unmodifiableMap( + uploadedClusterStateCustomMap != null ? uploadedClusterStateCustomMap : new HashMap<>() + ); } public ClusterMetadataManifest(StreamInput in) throws IOException { @@ -355,6 +533,7 @@ public ClusterMetadataManifest(StreamInput in) throws IOException { this.indices = Collections.unmodifiableList(in.readList(UploadedIndexMetadata::new)); this.previousClusterUUID = in.readString(); this.clusterUUIDCommitted = in.readBoolean(); + if (in.getVersion().onOrAfter(Version.V_3_0_0)) { this.codecVersion = in.readInt(); this.uploadedCoordinationMetadata = new UploadedMetadataAttribute(in); @@ -364,6 +543,18 @@ public ClusterMetadataManifest(StreamInput in) throws IOException { in.readMap(StreamInput::readString, UploadedMetadataAttribute::new) ); this.globalMetadataFileName = null; + // ToDo: change the version check and initialize these + this.uploadedDiscoveryNodesMetadata = null; + this.uploadedClusterBlocksMetadata = null; + this.diffManifest = null; + this.routingTableVersion = in.readLong(); + this.indicesRouting = Collections.unmodifiableList(in.readList(UploadedIndexMetadata::new)); + this.metadataVersion = in.readLong(); + this.uploadedTransientSettingsMetadata = new UploadedMetadataAttribute(in); + this.uploadedHashesOfConsistentSettings = new UploadedMetadataAttribute(in); + this.uploadedClusterStateCustomMap = Collections.unmodifiableMap( + in.readMap(StreamInput::readString, UploadedMetadataAttribute::new) + ); } else if (in.getVersion().onOrAfter(Version.V_2_12_0)) { this.codecVersion = in.readInt(); this.globalMetadataFileName = in.readString(); @@ -371,6 +562,15 @@ public ClusterMetadataManifest(StreamInput in) throws IOException { this.uploadedSettingsMetadata = null; this.uploadedTemplatesMetadata = null; this.uploadedCustomMetadataMap = null; + this.routingTableVersion = -1; + this.indicesRouting = null; + this.uploadedDiscoveryNodesMetadata = null; + this.uploadedClusterBlocksMetadata = null; + this.diffManifest = null; + this.metadataVersion = -1; + this.uploadedTransientSettingsMetadata = null; + this.uploadedHashesOfConsistentSettings = null; + this.uploadedClusterStateCustomMap = null; } else { this.codecVersion = CODEC_V0; // Default codec this.globalMetadataFileName = null; @@ -378,6 +578,15 @@ public ClusterMetadataManifest(StreamInput in) throws IOException { this.uploadedSettingsMetadata = null; this.uploadedTemplatesMetadata = null; this.uploadedCustomMetadataMap = null; + this.routingTableVersion = -1; + this.indicesRouting = null; + this.uploadedDiscoveryNodesMetadata = null; + this.uploadedClusterBlocksMetadata = null; + this.diffManifest = null; + this.metadataVersion = -1; + this.uploadedTransientSettingsMetadata = null; + this.uploadedHashesOfConsistentSettings = null; + this.uploadedClusterStateCustomMap = null; } } @@ -391,6 +600,7 @@ public static Builder builder(ClusterMetadataManifest manifest) { @Override public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + logger.info("CLUSTER_TERM_FIELD {} : {}", CLUSTER_TERM_FIELD.getPreferredName(), getClusterTerm()); builder.field(CLUSTER_TERM_FIELD.getPreferredName(), getClusterTerm()) .field(STATE_VERSION_FIELD.getPreferredName(), getStateVersion()) .field(CLUSTER_UUID_FIELD.getPreferredName(), getClusterUUID()) @@ -401,7 +611,9 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws builder.startArray(INDICES_FIELD.getPreferredName()); { for (UploadedIndexMetadata uploadedIndexMetadata : indices) { + builder.startObject(); uploadedIndexMetadata.toXContent(builder, params); + builder.endObject(); } } builder.endArray(); @@ -433,6 +645,49 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws builder.field(CODEC_VERSION_FIELD.getPreferredName(), getCodecVersion()); builder.field(GLOBAL_METADATA_FIELD.getPreferredName(), getGlobalMetadataFileName()); } + if (onOrAfterCodecVersion(CODEC_V3)) { + if (getDiscoveryNodesMetadata() != null) { + builder.startObject(UPLOADED_DISCOVERY_NODES_METADATA.getPreferredName()); + getDiscoveryNodesMetadata().toXContent(builder, params); + builder.endObject(); + } + if (getClusterBlocksMetadata() != null) { + builder.startObject(UPLOADED_CLUSTER_BLOCKS_METADATA.getPreferredName()); + getClusterBlocksMetadata().toXContent(builder, params); + builder.endObject(); + } + if (getTransientSettingsMetadata() != null) { + builder.startObject(UPLOADED_TRANSIENT_SETTINGS_METADATA.getPreferredName()); + getTransientSettingsMetadata().toXContent(builder, params); + builder.endObject(); + } + if (getDiffManifest() != null) { + builder.startObject(DIFF_MANIFEST.getPreferredName()); + getDiffManifest().toXContent(builder, params); + builder.endObject(); + } + builder.field(METADATA_VERSION.getPreferredName(), getMetadataVersion()); + builder.field(ROUTING_TABLE_VERSION_FIELD.getPreferredName(), getRoutingTableVersion()); + builder.startArray(INDICES_ROUTING_FIELD.getPreferredName()); + { + for (UploadedIndexMetadata uploadedIndexMetadata : indicesRouting) { + builder.startObject(); + uploadedIndexMetadata.toXContent(builder, params); + builder.endObject(); + } + } + builder.endArray(); + builder.startObject(UPLOADED_CLUSTER_STATE_CUSTOM_METADATA.getPreferredName()); + for (UploadedMetadataAttribute attribute : getClusterStateCustomMap().values()) { + attribute.toXContent(builder, params); + } + builder.endObject(); + if (getHashesOfConsistentSettings() != null) { + builder.startObject(UPLOADED_HASHES_OF_CONSISTENT_SETTINGS_METADATA.getPreferredName()); + getHashesOfConsistentSettings().toXContent(builder, params); + builder.endObject(); + } + } return builder; } @@ -454,6 +709,12 @@ public void writeTo(StreamOutput out) throws IOException { uploadedSettingsMetadata.writeTo(out); uploadedTemplatesMetadata.writeTo(out); out.writeMap(uploadedCustomMetadataMap, StreamOutput::writeString, (o, v) -> v.writeTo(o)); + out.writeLong(routingTableVersion); + out.writeCollection(indicesRouting); + out.writeLong(metadataVersion); + uploadedTransientSettingsMetadata.writeTo(out); + uploadedHashesOfConsistentSettings.writeTo(out); + out.writeMap(uploadedClusterStateCustomMap, StreamOutput::writeString, (o, v) -> v.writeTo(o)); } else if (out.getVersion().onOrAfter(Version.V_2_12_0)) { out.writeInt(codecVersion); out.writeString(globalMetadataFileName); @@ -462,6 +723,7 @@ public void writeTo(StreamOutput out) throws IOException { @Override public boolean equals(Object o) { + // ToDo: update this method to contain new attributes if (this == o) { return true; } @@ -480,11 +742,14 @@ public boolean equals(Object o) { && Objects.equals(previousClusterUUID, that.previousClusterUUID) && Objects.equals(clusterUUIDCommitted, that.clusterUUIDCommitted) && Objects.equals(globalMetadataFileName, that.globalMetadataFileName) - && Objects.equals(codecVersion, that.codecVersion); + && Objects.equals(codecVersion, that.codecVersion) + && Objects.equals(routingTableVersion, that.routingTableVersion) + && Objects.equals(indicesRouting, that.indicesRouting); } @Override public int hashCode() { + // ToDo: update this method to contain new attributes return Objects.hash( codecVersion, globalMetadataFileName, @@ -497,7 +762,9 @@ public int hashCode() { nodeId, committed, previousClusterUUID, - clusterUUIDCommitted + clusterUUIDCommitted, + routingTableVersion, + indicesRouting ); } @@ -530,8 +797,12 @@ public static ClusterMetadataManifest fromXContent(XContentParser parser) throws public static class Builder { private String globalMetadataFileName; + private long metadataVersion; + private UploadedMetadataAttribute discoveryNodesMetadata; + private UploadedMetadataAttribute clusterBlocksMetadata; private UploadedMetadataAttribute coordinationMetadata; private UploadedMetadataAttribute settingsMetadata; + private UploadedMetadataAttribute transientSettingsMetadata; private UploadedMetadataAttribute templatesMetadata; private Map customMetadataMap; private int codecVersion; @@ -545,12 +816,32 @@ public static class Builder { private String previousClusterUUID; private boolean committed; private boolean clusterUUIDCommitted; + private ClusterStateDiffManifest diffManifest; + private long routingTableVersion; + private List indicesRouting; + private UploadedMetadataAttribute hashesOfConsistentSettings; + private Map clusterStateCustomMetadataMap; public Builder indices(List indices) { this.indices = indices; return this; } + public Builder routingTableVersion(long routingTableVersion) { + this.routingTableVersion = routingTableVersion; + return this; + } + + public Builder metadataVersion(long metadataVersion) { + this.metadataVersion = metadataVersion; + return this; + } + + public Builder indicesRouting(List indicesRouting) { + this.indicesRouting = indicesRouting; + return this; + } + public Builder codecVersion(int codecVersion) { this.codecVersion = codecVersion; return this; @@ -571,6 +862,11 @@ public Builder settingMetadata(UploadedMetadataAttribute settingsMetadata) { return this; } + public Builder transientSettingsMetadata(UploadedMetadataAttribute settingsMetadata) { + this.transientSettingsMetadata = settingsMetadata; + return this; + } + public Builder templatesMetadata(UploadedMetadataAttribute templatesMetadata) { this.templatesMetadata = templatesMetadata; return this; @@ -625,6 +921,10 @@ public List getIndices() { return indices; } + public List getIndicesRouting() { + return indicesRouting; + } + public Builder previousClusterUUID(String previousClusterUUID) { this.previousClusterUUID = previousClusterUUID; return this; @@ -635,9 +935,36 @@ public Builder clusterUUIDCommitted(boolean clusterUUIDCommitted) { return this; } + public Builder discoveryNodesMetadata(UploadedMetadataAttribute discoveryNodesMetadata) { + this.discoveryNodesMetadata = discoveryNodesMetadata; + return this; + } + + public Builder clusterBlocksMetadata(UploadedMetadataAttribute clusterBlocksMetadata) { + this.clusterBlocksMetadata = clusterBlocksMetadata; + return this; + } + + public Builder diffManifest(ClusterStateDiffManifest diffManifest) { + this.diffManifest = diffManifest; + return this; + } + + public Builder hashesOfConsistentSettings(UploadedMetadataAttribute hashesOfConsistentSettings) { + this.hashesOfConsistentSettings = hashesOfConsistentSettings; + return this; + } + + public Builder clusterStateCustomMetadataMap(Map clusterStateCustomMetadataMap) { + this.clusterStateCustomMetadataMap = clusterStateCustomMetadataMap; + return this; + } + public Builder() { indices = new ArrayList<>(); customMetadataMap = new HashMap<>(); + indicesRouting = new ArrayList<>(); + clusterStateCustomMetadataMap = new HashMap<>(); } public Builder(ClusterMetadataManifest manifest) { @@ -657,6 +984,11 @@ public Builder(ClusterMetadataManifest manifest) { this.indices = new ArrayList<>(manifest.indices); this.previousClusterUUID = manifest.previousClusterUUID; this.clusterUUIDCommitted = manifest.clusterUUIDCommitted; + this.diffManifest = manifest.diffManifest; + this.routingTableVersion = manifest.routingTableVersion; + this.indicesRouting = new ArrayList<>(manifest.indicesRouting); + this.hashesOfConsistentSettings = manifest.uploadedHashesOfConsistentSettings; + this.clusterStateCustomMetadataMap = manifest.uploadedClusterStateCustomMap; } public ClusterMetadataManifest build() { @@ -676,7 +1008,16 @@ public ClusterMetadataManifest build() { coordinationMetadata, settingsMetadata, templatesMetadata, - customMetadataMap + customMetadataMap, + discoveryNodesMetadata, + clusterBlocksMetadata, + diffManifest, + routingTableVersion, + indicesRouting, + metadataVersion, + transientSettingsMetadata, + hashesOfConsistentSettings, + clusterStateCustomMetadataMap ); } @@ -711,6 +1052,7 @@ public static class UploadedIndexMetadata implements UploadedMetadata, Writeable private static final ParseField INDEX_NAME_FIELD = new ParseField("index_name"); private static final ParseField INDEX_UUID_FIELD = new ParseField("index_uuid"); private static final ParseField UPLOADED_FILENAME_FIELD = new ParseField("uploaded_filename"); + private static final ParseField COMPONENT_PREFIX_FIELD = new ParseField("component_prefix"); private static String indexName(Object[] fields) { return (String) fields[0]; @@ -724,23 +1066,34 @@ private static String uploadedFilename(Object[] fields) { return (String) fields[2]; } + private static String componentPrefix(Object[] fields) { + return (String) fields[3]; + } + private static final ConstructingObjectParser PARSER = new ConstructingObjectParser<>( "uploaded_index_metadata", - fields -> new UploadedIndexMetadata(indexName(fields), indexUUID(fields), uploadedFilename(fields)) + fields -> new UploadedIndexMetadata(indexName(fields), indexUUID(fields), uploadedFilename(fields), componentPrefix(fields)) ); static { PARSER.declareString(ConstructingObjectParser.constructorArg(), INDEX_NAME_FIELD); PARSER.declareString(ConstructingObjectParser.constructorArg(), INDEX_UUID_FIELD); PARSER.declareString(ConstructingObjectParser.constructorArg(), UPLOADED_FILENAME_FIELD); + PARSER.declareString(ConstructingObjectParser.constructorArg(), COMPONENT_PREFIX_FIELD); } - static final String COMPONENT_PREFIX = "index--"; + public static final String COMPONENT_PREFIX = "index--"; + private final String componentPrefix; private final String indexName; private final String indexUUID; private final String uploadedFilename; public UploadedIndexMetadata(String indexName, String indexUUID, String uploadedFileName) { + this( indexName,indexUUID,uploadedFileName, COMPONENT_PREFIX); + } + + public UploadedIndexMetadata(String indexName, String indexUUID, String uploadedFileName, String componentPrefix) { + this.componentPrefix = componentPrefix; this.indexName = indexName; this.indexUUID = indexUUID; this.uploadedFilename = uploadedFileName; @@ -750,6 +1103,7 @@ public UploadedIndexMetadata(StreamInput in) throws IOException { this.indexName = in.readString(); this.indexUUID = in.readString(); this.uploadedFilename = in.readString(); + this.componentPrefix = in.readString(); } public String getUploadedFilePath() { @@ -758,12 +1112,12 @@ public String getUploadedFilePath() { @Override public String getComponent() { - return COMPONENT_PREFIX + getIndexName(); + return componentPrefix + getIndexName(); } public String getUploadedFilename() { - String[] splitPath = uploadedFilename.split("/"); - return splitPath[splitPath.length - 1]; + // todo see how this change affects existing users + return uploadedFilename; } public String getIndexName() { @@ -774,13 +1128,18 @@ public String getIndexUUID() { return indexUUID; } + public String getComponentPrefix() { + return componentPrefix; + } + @Override public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { - return builder.startObject() + return builder .field(INDEX_NAME_FIELD.getPreferredName(), getIndexName()) .field(INDEX_UUID_FIELD.getPreferredName(), getIndexUUID()) .field(UPLOADED_FILENAME_FIELD.getPreferredName(), getUploadedFilePath()) - .endObject(); + .field(COMPONENT_PREFIX_FIELD.getPreferredName(), getComponentPrefix()); + } @Override @@ -788,6 +1147,7 @@ public void writeTo(StreamOutput out) throws IOException { out.writeString(indexName); out.writeString(indexUUID); out.writeString(uploadedFilename); + out.writeString(componentPrefix); } @Override @@ -801,12 +1161,14 @@ public boolean equals(Object o) { final UploadedIndexMetadata that = (UploadedIndexMetadata) o; return Objects.equals(indexName, that.indexName) && Objects.equals(indexUUID, that.indexUUID) - && Objects.equals(uploadedFilename, that.uploadedFilename); + && Objects.equals(uploadedFilename, that.uploadedFilename) + && Objects.equals(componentPrefix, that.componentPrefix); + } @Override public int hashCode() { - return Objects.hash(indexName, indexUUID, uploadedFilename); + return Objects.hash(indexName, indexUUID, uploadedFilename, componentPrefix); } @Override diff --git a/server/src/main/java/org/opensearch/gateway/remote/ClusterStateDiffManifest.java b/server/src/main/java/org/opensearch/gateway/remote/ClusterStateDiffManifest.java new file mode 100644 index 0000000000000..ee883b56ae40b --- /dev/null +++ b/server/src/main/java/org/opensearch/gateway/remote/ClusterStateDiffManifest.java @@ -0,0 +1,614 @@ +/* + * 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.gateway.remote; + +import org.apache.lucene.util.packed.DirectMonotonicReader; +import org.opensearch.cluster.ClusterState; +import org.opensearch.cluster.DiffableUtils; +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.remote.RemoteRoutingTableService; +import org.opensearch.core.common.Strings; +import org.opensearch.core.xcontent.MediaTypeRegistry; +import org.opensearch.core.xcontent.ToXContentObject; +import org.opensearch.core.xcontent.XContentBuilder; +import org.opensearch.core.xcontent.XContentParseException; +import org.opensearch.core.xcontent.XContentParser; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.opensearch.core.xcontent.XContentParserUtils.ensureExpectedToken; +import static org.opensearch.core.xcontent.XContentParserUtils.parseStringList; + +public class ClusterStateDiffManifest implements ToXContentObject { + private static final String FROM_STATE_UUID_FIELD = "from_state_uuid"; + private static final String TO_STATE_UUID_FIELD = "to_state_uuid"; + private static final String METADATA_DIFF_FIELD = "metadata_diff"; + private static final String COORDINATION_METADATA_UPDATED_FIELD = "coordination_metadata_diff"; + private static final String SETTINGS_METADATA_UPDATED_FIELD = "settings_metadata_diff"; + private static final String TRANSIENT_SETTINGS_METADATA_UPDATED_FIELD = "transient_settings_metadata_diff"; + private static final String TEMPLATES_METADATA_UPDATED_FIELD = "templates_metadata_diff"; + private static final String HASHES_OF_CONSISTENT_SETTINGS_UPDATED_FIELD = "hashes_of_consistent_settings_diff"; + private static final String INDICES_DIFF_FIELD = "indices_diff"; + private static final String METADATA_CUSTOM_DIFF_FIELD = "metadata_custom_diff"; + private static final String UPSERTS_FIELD = "upserts"; + private static final String DELETES_FIELD = "deletes"; + private static final String CLUSTER_BLOCKS_UPDATED_FIELD = "cluster_blocks_diff"; + private static final String DISCOVERY_NODES_UPDATED_FIELD = "discovery_nodes_diff"; + private static final String ROUTING_TABLE_DIFF = "routing_table_diff"; + private static final String CLUSTER_STATE_CUSTOM_DIFF_FIELD = "cluster_state_custom_diff"; + private final String fromStateUUID; + private final String toStateUUID; + private final boolean coordinationMetadataUpdated; + private final boolean settingsMetadataUpdated; + private final boolean transientSettingsMetadataUpdate; + private final boolean templatesMetadataUpdated; + private List customMetadataUpdated; + private final List customMetadataDeleted; + private final List indicesUpdated; + private final List indicesDeleted; + private final boolean clusterBlocksUpdated; + private final boolean discoveryNodesUpdated; + private final List indicesRoutingUpdated; + private final List indicesRoutingDeleted; + private final boolean hashesOfConsistentSettingsUpdated; + private final List clusterStateCustomUpdated; + private final List clusterStateCustomDeleted; + + ClusterStateDiffManifest(ClusterState state, ClusterState previousState) { + fromStateUUID = previousState.stateUUID(); + toStateUUID = state.stateUUID(); + coordinationMetadataUpdated = !Metadata.isCoordinationMetadataEqual(state.metadata(), previousState.metadata()); + settingsMetadataUpdated = !Metadata.isSettingsMetadataEqual(state.metadata(), previousState.metadata()); + transientSettingsMetadataUpdate = !Metadata.isTransientSettingsMetadataEqual(state.metadata(), previousState.metadata()); + templatesMetadataUpdated = !Metadata.isTemplatesMetadataEqual(state.metadata(), previousState.metadata()); + indicesDeleted = findRemovedIndices(state.metadata().indices(), previousState.metadata().indices()); + indicesUpdated = findUpdatedIndices(state.metadata().indices(), previousState.metadata().indices()); + clusterBlocksUpdated = !state.blocks().equals(previousState.blocks()); + discoveryNodesUpdated = state.nodes().delta(previousState.nodes()).hasChanges(); + customMetadataUpdated = new ArrayList<>(); + for (String custom : state.metadata().customs().keySet()) { + if (!previousState.metadata().customs().containsKey(custom) || !state.metadata().customs().get(custom).equals(previousState.metadata().customs().get(custom))) { + customMetadataUpdated.add(custom); + } + } + customMetadataDeleted = new ArrayList<>(); + for (String custom : previousState.metadata().customs().keySet()) { + if (state.metadata().customs().get(custom) == null) { + customMetadataDeleted.add(custom); + } + } + + DiffableUtils.MapDiff> routingTableDiff = RemoteRoutingTableService.getIndicesRoutingMapDiff(previousState.getRoutingTable(), + state.getRoutingTable()); + + indicesRoutingUpdated = new ArrayList<>(); + routingTableDiff.getUpserts().forEach((k,v) -> indicesRoutingUpdated.add(k)); + + indicesRoutingDeleted = routingTableDiff.getDeletes(); + hashesOfConsistentSettingsUpdated = !state.metadata().hashesOfConsistentSettings().equals(previousState.metadata().hashesOfConsistentSettings()); + clusterStateCustomUpdated = new ArrayList<>(); + clusterStateCustomDeleted = new ArrayList<>(); + for (String custom : state.customs().keySet()) { + if (!previousState.customs().containsKey(custom) || !state.customs().get(custom).equals(previousState.customs().get(custom))) { + clusterStateCustomUpdated.add(custom); + } + } + for (String custom : previousState.customs().keySet()) { + if (state.customs().get(custom) == null) { + clusterStateCustomDeleted.add(custom); + } + } + } + + public ClusterStateDiffManifest( + String fromStateUUID, + String toStateUUID, + boolean coordinationMetadataUpdated, + boolean settingsMetadataUpdated, + boolean transientSettingsMetadataUpdate, + boolean templatesMetadataUpdated, + List customMetadataUpdated, + List customMetadataDeleted, + List indicesUpdated, + List indicesDeleted, + boolean clusterBlocksUpdated, + boolean discoveryNodesUpdated, + ListindicesRoutingUpdated, + ListindicesRoutingDeleted, + boolean hashesOfConsistentSettingsUpdated, + List clusterStateCustomUpdated, + List clusterStateCustomDeleted + ) { + this.fromStateUUID = fromStateUUID; + this.toStateUUID = toStateUUID; + this.coordinationMetadataUpdated = coordinationMetadataUpdated; + this.settingsMetadataUpdated = settingsMetadataUpdated; + this.transientSettingsMetadataUpdate = transientSettingsMetadataUpdate; + this.templatesMetadataUpdated = templatesMetadataUpdated; + this.customMetadataUpdated = customMetadataUpdated; + this.customMetadataDeleted = customMetadataDeleted; + this.indicesUpdated = indicesUpdated; + this.indicesDeleted = indicesDeleted; + this.clusterBlocksUpdated = clusterBlocksUpdated; + this.discoveryNodesUpdated = discoveryNodesUpdated; + this.indicesRoutingUpdated = indicesRoutingUpdated; + this.indicesRoutingDeleted = indicesRoutingDeleted; + this.hashesOfConsistentSettingsUpdated = hashesOfConsistentSettingsUpdated; + this.clusterStateCustomUpdated = clusterStateCustomUpdated; + this.clusterStateCustomDeleted = clusterStateCustomDeleted; + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + { + builder.field(FROM_STATE_UUID_FIELD, fromStateUUID); + builder.field(TO_STATE_UUID_FIELD, toStateUUID); + builder.startObject(METADATA_DIFF_FIELD); + { + builder.field(COORDINATION_METADATA_UPDATED_FIELD, coordinationMetadataUpdated); + builder.field(SETTINGS_METADATA_UPDATED_FIELD, settingsMetadataUpdated); + builder.field(TRANSIENT_SETTINGS_METADATA_UPDATED_FIELD, transientSettingsMetadataUpdate); + builder.field(TEMPLATES_METADATA_UPDATED_FIELD, templatesMetadataUpdated); + builder.startObject(INDICES_DIFF_FIELD); + builder.startArray(UPSERTS_FIELD); + for (String index : indicesUpdated) { + builder.value(index); + } + builder.endArray(); + builder.startArray(DELETES_FIELD); + for (String index : indicesDeleted) { + builder.value(index); + } + builder.endArray(); + builder.endObject(); + builder.startObject(METADATA_CUSTOM_DIFF_FIELD); + builder.startArray(UPSERTS_FIELD); + for (String custom : customMetadataUpdated) { + builder.value(custom); + } + builder.endArray(); + builder.startArray(DELETES_FIELD); + for (String custom : customMetadataDeleted) { + builder.value(custom); + } + builder.endArray(); + builder.endObject(); + builder.field(HASHES_OF_CONSISTENT_SETTINGS_UPDATED_FIELD, hashesOfConsistentSettingsUpdated); + } + 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); + } + builder.endArray(); + builder.startArray(DELETES_FIELD); + for (String index : indicesRoutingDeleted) { + builder.value(index); + } + builder.endArray(); + builder.endObject(); + builder.startObject(CLUSTER_STATE_CUSTOM_DIFF_FIELD); + builder.startArray(UPSERTS_FIELD); + for (String custom : clusterStateCustomUpdated) { + builder.value(custom); + } + builder.endArray(); + builder.startArray(DELETES_FIELD); + for (String custom : clusterStateCustomDeleted) { + builder.value(custom); + } + builder.endArray(); + builder.endObject(); + + } + return builder; + } + + public static ClusterStateDiffManifest fromXContent(XContentParser parser) throws IOException { + Builder builder = new Builder(); + if (parser.currentToken() == null) { // fresh parser? move to next token + parser.nextToken(); + } + if (parser.currentToken() == XContentParser.Token.START_OBJECT) { + parser.nextToken(); + } + ensureExpectedToken(XContentParser.Token.FIELD_NAME, parser.currentToken(), parser); + String currentFieldName = parser.currentName(); + XContentParser.Token token; + while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) { + if (token == XContentParser.Token.FIELD_NAME) { + currentFieldName = parser.currentName(); + } else if (token == XContentParser.Token.START_OBJECT) { + if (currentFieldName.equals(METADATA_DIFF_FIELD)) { + while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) { + currentFieldName = parser.currentName(); + token = parser.nextToken(); + if (token.isValue()) { + switch (currentFieldName) { + case COORDINATION_METADATA_UPDATED_FIELD: + builder.coordinationMetadataUpdated(parser.booleanValue()); + break; + case SETTINGS_METADATA_UPDATED_FIELD: + builder.settingsMetadataUpdated(parser.booleanValue()); + break; + case TRANSIENT_SETTINGS_METADATA_UPDATED_FIELD: + builder.transientSettingsMetadataUpdate(parser.booleanValue()); + break; + case TEMPLATES_METADATA_UPDATED_FIELD: + builder.templatesMetadataUpdated(parser.booleanValue()); + break; + case HASHES_OF_CONSISTENT_SETTINGS_UPDATED_FIELD: + builder.hashesOfConsistentSettingsUpdated(parser.booleanValue()); + break; + default: + throw new XContentParseException("Unexpected field [" + currentFieldName + "]"); + } + } else if (token == XContentParser.Token.START_OBJECT) { + if (currentFieldName.equals(INDICES_DIFF_FIELD)) { + while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) { + currentFieldName = parser.currentName(); + token = parser.nextToken(); + switch (currentFieldName) { + case UPSERTS_FIELD: + builder.indicesUpdated(parseStringList(parser)); + break; + case DELETES_FIELD: + builder.indicesDeleted(parseStringList(parser)); + break; + default: + throw new XContentParseException("Unexpected field [" + currentFieldName + "]"); + } + } + } else if (currentFieldName.equals(METADATA_CUSTOM_DIFF_FIELD)) { + while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) { + currentFieldName = parser.currentName(); + token = parser.nextToken(); + switch (currentFieldName) { + case UPSERTS_FIELD: + builder.customMetadataUpdated(parseStringList(parser)); + break; + case DELETES_FIELD: + builder.customMetadataDeleted(parseStringList(parser)); + break; + default: + throw new XContentParseException("Unexpected field [" + currentFieldName + "]"); + } + } + } else { + throw new XContentParseException("Unexpected field [" + currentFieldName + "]"); + } + } else { + throw new XContentParseException("Unexpected token [" + token + "]"); + } + } + } else if (currentFieldName.equals(ROUTING_TABLE_DIFF)) { + while ((parser.nextToken()) != XContentParser.Token.END_OBJECT) { + currentFieldName = parser.currentName(); + parser.nextToken(); + switch (currentFieldName) { + case UPSERTS_FIELD: + builder.indicesRoutingUpdated(parseStringList(parser)); + break; + case DELETES_FIELD: + builder.indicesRoutingDeleted(parseStringList(parser)); + break; + default: + throw new XContentParseException("Unexpected field [" + currentFieldName + "]"); + } + } + } else if (currentFieldName.equals(CLUSTER_STATE_CUSTOM_DIFF_FIELD)) { + while ((parser.nextToken()) != XContentParser.Token.END_OBJECT) { + currentFieldName = parser.currentName(); + parser.nextToken(); + switch (currentFieldName) { + case UPSERTS_FIELD: + builder.clusterStateCustomUpdated(parseStringList(parser)); + break; + case DELETES_FIELD: + builder.clusterStateCustomDeleted(parseStringList(parser)); + break; + default: + throw new XContentParseException("Unexpected field [" + currentFieldName + "]"); + } + } + } else { + throw new XContentParseException("Unexpected field [" + currentFieldName + "]"); + } + } else if (token.isValue()) { + switch (currentFieldName) { + case FROM_STATE_UUID_FIELD: + builder.fromStateUUID(parser.text()); + break; + case TO_STATE_UUID_FIELD: + builder.toStateUUID(parser.text()); + break; + case CLUSTER_BLOCKS_UPDATED_FIELD: + builder.clusterBlocksUpdated(parser.booleanValue()); + break; + case DISCOVERY_NODES_UPDATED_FIELD: + builder.discoveryNodesUpdated(parser.booleanValue()); + break; + default: + throw new XContentParseException("Unexpected field [" + currentFieldName + "]"); + } + } else { + throw new XContentParseException("Unexpected token [" + token + "]"); + } + } + return builder.build(); + } + + @Override + public String toString() { + return Strings.toString(MediaTypeRegistry.JSON, this); + } + + public List findRemovedIndices(Map indices, Map previousIndices) { + List removedIndices = new ArrayList<>(); + for (String index : previousIndices.keySet()) { + // index present in previous state but not in current + if (!indices.containsKey(index)) { + removedIndices.add(index); + } + } + return removedIndices; + } + + public List findUpdatedIndices(Map indices, Map previousIndices) { + List updatedIndices = new ArrayList<>(); + for (String index : indices.keySet()) { + if (!previousIndices.containsKey(index)) { + updatedIndices.add(index); + } else if (previousIndices.get(index).getVersion() != indices.get(index).getVersion()) { + updatedIndices.add(index); + } + } + return updatedIndices; + } + + public List getIndicesRoutingDeleted(RoutingTable previousRoutingTable, RoutingTable currentRoutingTable) { + List deletedIndicesRouting = new ArrayList<>(); + for(IndexRoutingTable previousIndexRouting: previousRoutingTable.getIndicesRouting().values()) { + if(!currentRoutingTable.getIndicesRouting().containsKey(previousIndexRouting.getIndex().getName())) { + // Latest Routing Table does not have entry for the index which means the index is deleted + deletedIndicesRouting.add(previousIndexRouting.getIndex().getName()); + } + } + return deletedIndicesRouting; + } + + public List getIndicesRoutingUpdated(RoutingTable previousRoutingTable, RoutingTable currentRoutingTable) { + List updatedIndicesRouting = new ArrayList<>(); + for(IndexRoutingTable currentIndicesRouting: currentRoutingTable.getIndicesRouting().values()) { + if(!previousRoutingTable.getIndicesRouting().containsKey(currentIndicesRouting.getIndex().getName())) { + // Latest Routing Table does not have entry for the index which means the index is created + updatedIndicesRouting.add(currentIndicesRouting.getIndex().getName()); + } else { + if(previousRoutingTable.getIndicesRouting().get(currentIndicesRouting.getIndex().getName()).equals(currentIndicesRouting)) { + // if the latest routing table has the same routing table as the previous routing table, then the index is not updated + continue; + } + updatedIndicesRouting.add(currentIndicesRouting.getIndex().getName()); + } + } + return updatedIndicesRouting; + } + + public String getFromStateUUID() { + return fromStateUUID; + } + + public String getToStateUUID() { + return toStateUUID; + } + + public boolean isCoordinationMetadataUpdated() { + return coordinationMetadataUpdated; + } + + public boolean isSettingsMetadataUpdated() { + return settingsMetadataUpdated; + } + + public boolean isTransientSettingsMetadataUpdated() { + return transientSettingsMetadataUpdate; + } + + public boolean isTemplatesMetadataUpdated() { + return templatesMetadataUpdated; + } + + public List getCustomMetadataUpdated() { + return customMetadataUpdated; + } + + public List getCustomMetadataDeleted() { + return customMetadataDeleted; + } + + public List getIndicesUpdated() { + return indicesUpdated; + } + + public List getIndicesDeleted() { + return indicesDeleted; + } + + public boolean isClusterBlocksUpdated() { + return clusterBlocksUpdated; + } + + public boolean isDiscoveryNodesUpdated() { + return discoveryNodesUpdated; + } + + public boolean isHashesOfConsistentSettingsUpdated() { + return hashesOfConsistentSettingsUpdated; + } + + public List getIndicesRoutingUpdated() { + return indicesRoutingUpdated; + } + + public List getIndicesRoutingDeleted() { + return indicesRoutingDeleted; + } + + public List getClusterStateCustomUpdated() { + return clusterStateCustomUpdated; + } + + public List getClusterStateCustomDeleted() { + return clusterStateCustomDeleted; + } + + public static Builder builder() { + return new Builder(); + } + + public static class Builder { + private String fromStateUUID; + private String toStateUUID; + private boolean coordinationMetadataUpdated; + private boolean settingsMetadataUpdated; + private boolean transientSettingsMetadataUpdated; + private boolean templatesMetadataUpdated; + private List customMetadataUpdated; + private List customMetadataDeleted; + private List indicesUpdated; + private List indicesDeleted; + private boolean clusterBlocksUpdated; + private boolean discoveryNodesUpdated; + private List indicesRoutingUpdated; + private List indicesRoutingDeleted; + private boolean hashesOfConsistentSettingsUpdated; + private List clusterStateCustomUpdated; + private List clusterStateCustomDeleted; + + public Builder() {} + + public Builder fromStateUUID(String fromStateUUID) { + this.fromStateUUID = fromStateUUID; + return this; + } + + public Builder toStateUUID(String toStateUUID) { + this.toStateUUID = toStateUUID; + return this; + } + + public Builder coordinationMetadataUpdated(boolean coordinationMetadataUpdated) { + this.coordinationMetadataUpdated = coordinationMetadataUpdated; + return this; + } + + public Builder settingsMetadataUpdated(boolean settingsMetadataUpdated) { + this.settingsMetadataUpdated = settingsMetadataUpdated; + return this; + } + + public Builder transientSettingsMetadataUpdate(boolean settingsMetadataUpdated) { + this.transientSettingsMetadataUpdated = settingsMetadataUpdated; + return this; + } + + public Builder templatesMetadataUpdated(boolean templatesMetadataUpdated) { + this.templatesMetadataUpdated = templatesMetadataUpdated; + return this; + } + + public Builder hashesOfConsistentSettingsUpdated(boolean hashesOfConsistentSettingsUpdated) { + this.hashesOfConsistentSettingsUpdated = hashesOfConsistentSettingsUpdated; + return this; + } + + public Builder customMetadataUpdated(List customMetadataUpdated) { + this.customMetadataUpdated = customMetadataUpdated; + return this; + } + + public Builder customMetadataDeleted(List customMetadataDeleted) { + this.customMetadataDeleted = customMetadataDeleted; + return this; + } + + public Builder indicesUpdated(List indicesUpdated) { + this.indicesUpdated = indicesUpdated; + return this; + } + + public Builder indicesDeleted(List indicesDeleted) { + this.indicesDeleted = indicesDeleted; + return this; + } + + public Builder clusterBlocksUpdated(boolean clusterBlocksUpdated) { + this.clusterBlocksUpdated = clusterBlocksUpdated; + return this; + } + + public Builder discoveryNodesUpdated(boolean discoveryNodesUpdated) { + this.discoveryNodesUpdated = discoveryNodesUpdated; + return this; + } + + public Builder indicesRoutingUpdated(List indicesRoutingUpdated) { + this.indicesRoutingUpdated = indicesRoutingUpdated; + return this; + } + + public Builder indicesRoutingDeleted(List indicesRoutingDeleted) { + this.indicesRoutingDeleted = indicesRoutingDeleted; + return this; + } + + public Builder clusterStateCustomUpdated(List clusterStateCustomUpdated) { + this.clusterStateCustomUpdated = clusterStateCustomUpdated; + return this; + } + + public Builder clusterStateCustomDeleted(List clusterStateCustomDeleted) { + this.clusterStateCustomDeleted = clusterStateCustomDeleted; + return this; + } + + public ClusterStateDiffManifest build() { + return new ClusterStateDiffManifest( + fromStateUUID, + toStateUUID, + coordinationMetadataUpdated, + settingsMetadataUpdated, + transientSettingsMetadataUpdated, + templatesMetadataUpdated, + customMetadataUpdated, + customMetadataDeleted, + indicesUpdated, + indicesDeleted, + clusterBlocksUpdated, + discoveryNodesUpdated, + indicesRoutingUpdated, + indicesRoutingDeleted, + hashesOfConsistentSettingsUpdated, + clusterStateCustomUpdated, + clusterStateCustomDeleted + ); + } + } +} diff --git a/server/src/main/java/org/opensearch/gateway/remote/RemoteClusterStateAttributesManager.java b/server/src/main/java/org/opensearch/gateway/remote/RemoteClusterStateAttributesManager.java new file mode 100644 index 0000000000000..e7d497bf2ee83 --- /dev/null +++ b/server/src/main/java/org/opensearch/gateway/remote/RemoteClusterStateAttributesManager.java @@ -0,0 +1,151 @@ +/* + * 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.gateway.remote; + +import java.io.IOException; +import org.opensearch.action.LatchedActionListener; +import org.opensearch.cluster.ClusterState; +import org.opensearch.cluster.ClusterState.Custom; +import org.opensearch.cluster.block.ClusterBlocks; +import org.opensearch.cluster.node.DiscoveryNodes; +import org.opensearch.common.CheckedRunnable; +import org.opensearch.core.action.ActionListener; +import org.opensearch.core.common.io.stream.NamedWriteableRegistry; +import org.opensearch.core.compress.Compressor; +import org.opensearch.core.xcontent.NamedXContentRegistry; +import org.opensearch.core.xcontent.ToXContent; +import org.opensearch.gateway.remote.RemoteClusterStateUtils.RemoteStateTransferException; +import org.opensearch.common.remote.AbstractRemoteWritableBlobEntity; +import org.opensearch.gateway.remote.model.RemoteClusterStateBlobStore; +import org.opensearch.gateway.remote.model.RemoteClusterBlocks; +import org.opensearch.gateway.remote.model.RemoteClusterStateCustoms; +import org.opensearch.gateway.remote.model.RemoteDiscoveryNodes; +import org.opensearch.gateway.remote.model.RemoteReadResult; + +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; + +import static org.opensearch.gateway.remote.model.RemoteClusterStateCustoms.CLUSTER_STATE_CUSTOM; +import static org.opensearch.gateway.remote.model.RemoteCustomMetadata.CUSTOM_DELIMITER; + +public class RemoteClusterStateAttributesManager { + public static final String CLUSTER_STATE_ATTRIBUTE = "cluster_state_attribute"; + public static final String DISCOVERY_NODES = "nodes"; + public static final String CLUSTER_BLOCKS = "blocks"; + public static final int CLUSTER_STATE_ATTRIBUTES_CURRENT_CODEC_VERSION = 1; + private final RemoteClusterStateBlobStore clusterBlocksBlobStore; + private final RemoteClusterStateBlobStore discoveryNodesBlobStore; + private final RemoteClusterStateBlobStore customsBlobStore; + private final Compressor compressor; + private final NamedXContentRegistry namedXContentRegistry; + private final NamedWriteableRegistry namedWriteableRegistry; + + RemoteClusterStateAttributesManager( + RemoteClusterStateBlobStore clusterBlocksBlobStore, + RemoteClusterStateBlobStore discoveryNodesBlobStore, + RemoteClusterStateBlobStore customsBlobStore, + Compressor compressor, + NamedXContentRegistry namedXContentRegistry, + NamedWriteableRegistry namedWriteableRegistry + ) { + this.clusterBlocksBlobStore = clusterBlocksBlobStore; + this.discoveryNodesBlobStore = discoveryNodesBlobStore; + this.customsBlobStore = customsBlobStore; + this.compressor = compressor; + this.namedXContentRegistry = namedXContentRegistry; + this.namedWriteableRegistry = namedWriteableRegistry; + } + + /** + * Allows async upload of Cluster State Attribute components to remote + */ + CheckedRunnable getAsyncMetadataWriteAction( + ClusterState clusterState, + String component, + ToXContent componentData, + LatchedActionListener latchedActionListener + ) { + if (componentData instanceof DiscoveryNodes) { + RemoteDiscoveryNodes remoteObject = new RemoteDiscoveryNodes((DiscoveryNodes)componentData, clusterState.version(), clusterState.metadata().clusterUUID(), compressor, namedXContentRegistry); + return () -> discoveryNodesBlobStore.writeAsync(remoteObject, getActionListener(component, remoteObject, latchedActionListener)); + } else if (componentData instanceof ClusterBlocks) { + RemoteClusterBlocks remoteObject = new RemoteClusterBlocks((ClusterBlocks) componentData, clusterState.version(), clusterState.metadata().clusterUUID(), compressor, namedXContentRegistry); + return () -> clusterBlocksBlobStore.writeAsync(remoteObject, getActionListener(component, remoteObject, latchedActionListener)); + } else if (componentData instanceof ClusterState.Custom) { + RemoteClusterStateCustoms remoteObject = new RemoteClusterStateCustoms( + (ClusterState.Custom) componentData, + component, + clusterState.version(), + clusterState.metadata().clusterUUID(), + compressor, + namedXContentRegistry, + namedWriteableRegistry + ); + return () -> customsBlobStore.writeAsync(remoteObject, getActionListener(component, remoteObject, latchedActionListener)); + } else { + throw new RemoteStateTransferException("Remote object not found for "+ componentData.getClass()); + } + } + + private ActionListener getActionListener(String component, AbstractRemoteWritableBlobEntity remoteObject, LatchedActionListener latchedActionListener) { + return ActionListener.wrap( + resp -> latchedActionListener.onResponse( + remoteObject.getUploadedMetadata() + ), + ex -> latchedActionListener.onFailure(new RemoteClusterStateUtils.RemoteStateTransferException(component, ex)) + ); + } + + public CheckedRunnable getAsyncMetadataReadAction( + String clusterUUID, + String component, + String componentName, + String uploadedFilename, + LatchedActionListener listener + ) { + final ActionListener actionListener = ActionListener.wrap(response -> listener.onResponse(new RemoteReadResult((ToXContent) response, CLUSTER_STATE_ATTRIBUTE, component)), listener::onFailure); + if (component.equals(RemoteDiscoveryNodes.DISCOVERY_NODES)) { + RemoteDiscoveryNodes remoteDiscoveryNodes = new RemoteDiscoveryNodes(uploadedFilename, clusterUUID, compressor, namedXContentRegistry); + return () -> discoveryNodesBlobStore.readAsync(remoteDiscoveryNodes, actionListener); + } else if (component.equals(RemoteClusterBlocks.CLUSTER_BLOCKS)) { + RemoteClusterBlocks remoteClusterBlocks = new RemoteClusterBlocks(uploadedFilename, clusterUUID, compressor, namedXContentRegistry); + return () -> clusterBlocksBlobStore.readAsync(remoteClusterBlocks, actionListener); + } else if (component.equals(CLUSTER_STATE_CUSTOM)) { + final ActionListener customActionListener = ActionListener.wrap(response -> listener.onResponse(new RemoteReadResult((ToXContent) response, CLUSTER_STATE_ATTRIBUTE, String.join(CUSTOM_DELIMITER, component, componentName))), listener::onFailure); + RemoteClusterStateCustoms remoteClusterStateCustoms = new RemoteClusterStateCustoms( + uploadedFilename, + componentName, + clusterUUID, + compressor, + namedXContentRegistry, + namedWriteableRegistry + ); + return () -> customsBlobStore.readAsync(remoteClusterStateCustoms, customActionListener); + } else { + throw new RemoteStateTransferException("Remote object not found for "+ component); + } + } + + public Map getUpdatedCustoms(ClusterState clusterState, ClusterState previousClusterState) { + Map updatedCustoms = new HashMap<>(); + Set currentCustoms = new HashSet<>(clusterState.customs().keySet()); + for (Map.Entry entry : previousClusterState.customs().entrySet()) { + if (currentCustoms.contains(entry.getKey()) && !entry.getValue().equals(clusterState.customs().get(entry.getKey()))) { + updatedCustoms.put(entry.getKey(), clusterState.customs().get(entry.getKey())); + } + currentCustoms.remove(entry.getKey()); + } + for (String custom : currentCustoms) { + updatedCustoms.put(custom, clusterState.customs().get(custom)); + } + return updatedCustoms; + } +} diff --git a/server/src/main/java/org/opensearch/gateway/remote/RemoteClusterStateCleanupManager.java b/server/src/main/java/org/opensearch/gateway/remote/RemoteClusterStateCleanupManager.java new file mode 100644 index 0000000000000..f650678358065 --- /dev/null +++ b/server/src/main/java/org/opensearch/gateway/remote/RemoteClusterStateCleanupManager.java @@ -0,0 +1,403 @@ +/* + * 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.gateway.remote; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.apache.logging.log4j.message.ParameterizedMessage; +import org.apache.logging.log4j.util.Strings; +import org.opensearch.cluster.ClusterState; +import org.opensearch.cluster.service.ClusterApplierService; +import org.opensearch.cluster.service.ClusterService; +import org.opensearch.common.blobstore.BlobMetadata; +import org.opensearch.common.blobstore.BlobPath; +import org.opensearch.common.settings.ClusterSettings; +import org.opensearch.common.settings.Setting; +import org.opensearch.common.unit.TimeValue; +import org.opensearch.common.util.concurrent.AbstractAsyncTask; +import org.opensearch.core.action.ActionListener; +import org.opensearch.index.translog.transfer.BlobStoreTransferService; +import org.opensearch.threadpool.ThreadPool; + +import java.io.Closeable; +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Locale; +import java.util.Set; +import java.util.concurrent.atomic.AtomicBoolean; + +import static org.opensearch.gateway.remote.model.RemoteClusterMetadataManifest.MANIFEST; + +/** + * A Manager which provides APIs to clean up stale cluster state files and runs an async stale cleanup task + * + * @opensearch.internal + */ +public class RemoteClusterStateCleanupManager implements Closeable { + + public static final int RETAINED_MANIFESTS = 10; + public static final int SKIP_CLEANUP_STATE_CHANGES = 10; + public static final TimeValue CLUSTER_STATE_CLEANUP_INTERVAL_DEFAULT = TimeValue.timeValueSeconds(15); + public static final TimeValue CLUSTER_STATE_CLEANUP_INTERVAL_MINIMUM = TimeValue.MINUS_ONE; + + /** + * Setting to specify the interval to do run stale file cleanup job + * Min value -1 indicates that the stale file cleanup job should be disabled + */ + public static final Setting REMOTE_CLUSTER_STATE_CLEANUP_INTERVAL_SETTING = Setting.timeSetting( + "cluster.remote_store.state.cleanup_interval", + CLUSTER_STATE_CLEANUP_INTERVAL_DEFAULT, + CLUSTER_STATE_CLEANUP_INTERVAL_MINIMUM, + Setting.Property.NodeScope, + Setting.Property.Dynamic + ); + private static final Logger logger = LogManager.getLogger(RemoteClusterStateCleanupManager.class); + private final RemoteClusterStateService remoteClusterStateService; + private final RemotePersistenceStats remoteStateStats; + private BlobStoreTransferService blobStoreTransferService; + private TimeValue staleFileCleanupInterval; + private final AtomicBoolean deleteStaleMetadataRunning = new AtomicBoolean(false); + private AsyncStaleFileDeletion staleFileDeletionTask; + private long lastCleanupAttemptStateVersion; + private final ThreadPool threadpool; + private final ClusterApplierService clusterApplierService; + + public RemoteClusterStateCleanupManager(RemoteClusterStateService remoteClusterStateService, ClusterService clusterService) { + this.remoteClusterStateService = remoteClusterStateService; + this.remoteStateStats = remoteClusterStateService.getStats(); + ClusterSettings clusterSettings = clusterService.getClusterSettings(); + this.clusterApplierService = clusterService.getClusterApplierService(); + this.staleFileCleanupInterval = clusterSettings.get(REMOTE_CLUSTER_STATE_CLEANUP_INTERVAL_SETTING); + this.threadpool = remoteClusterStateService.getThreadpool(); + // initialize with 0, a cleanup will be done when this node is elected master node and version is incremented more than threshold + this.lastCleanupAttemptStateVersion = 0; + clusterSettings.addSettingsUpdateConsumer(REMOTE_CLUSTER_STATE_CLEANUP_INTERVAL_SETTING, this::updateCleanupInterval); + } + + void start() { + staleFileDeletionTask = new AsyncStaleFileDeletion(this); + } + + @Override + public void close() throws IOException { + if (staleFileDeletionTask != null) { + staleFileDeletionTask.close(); + } + } + + private BlobStoreTransferService getBlobStoreTransferService() { + if (blobStoreTransferService == null) { + blobStoreTransferService = new BlobStoreTransferService(remoteClusterStateService.getBlobStore(), threadpool); + } + return blobStoreTransferService; + } + + private void updateCleanupInterval(TimeValue updatedInterval) { + this.staleFileCleanupInterval = updatedInterval; + logger.info("updated remote state cleanup interval to {}", updatedInterval); + // After updating the interval, we need to close the current task and create a new one which will run with updated interval + if (staleFileDeletionTask != null && !staleFileDeletionTask.getInterval().equals(updatedInterval)) { + staleFileDeletionTask.setInterval(updatedInterval); + } + } + + // visible for testing + void cleanUpStaleFiles() { + ClusterState currentAppliedState = clusterApplierService.state(); + if (currentAppliedState.nodes().isLocalNodeElectedClusterManager()) { + long cleanUpAttemptStateVersion = currentAppliedState.version(); + assert Strings.isNotEmpty(currentAppliedState.getClusterName().value()) : "cluster name is not set"; + assert Strings.isNotEmpty(currentAppliedState.metadata().clusterUUID()) : "cluster uuid is not set"; + if (cleanUpAttemptStateVersion - lastCleanupAttemptStateVersion > SKIP_CLEANUP_STATE_CHANGES) { + logger.info( + "Cleaning up stale remote state files for cluster [{}] with uuid [{}]. Last clean was done before {} updates", + currentAppliedState.getClusterName().value(), + currentAppliedState.metadata().clusterUUID(), + cleanUpAttemptStateVersion - lastCleanupAttemptStateVersion + ); + this.deleteStaleClusterMetadata( + currentAppliedState.getClusterName().value(), + currentAppliedState.metadata().clusterUUID(), + RETAINED_MANIFESTS + ); + lastCleanupAttemptStateVersion = cleanUpAttemptStateVersion; + } else { + logger.debug( + "Skipping cleanup of stale remote state files for cluster [{}] with uuid [{}]. Last clean was done before {} updates, which is less than threshold {}", + currentAppliedState.getClusterName().value(), + currentAppliedState.metadata().clusterUUID(), + cleanUpAttemptStateVersion - lastCleanupAttemptStateVersion, + SKIP_CLEANUP_STATE_CHANGES + ); + } + } else { + logger.debug("Skipping cleanup task as local node is not elected Cluster Manager"); + } + } + + private void addStaleGlobalMetadataPath(String fileName, Set filesToKeep, Set staleGlobalMetadataPaths) { + if (!filesToKeep.contains(fileName)) { + staleGlobalMetadataPaths.add(fileName); + } + } + + // visible for testing + void deleteClusterMetadata( + String clusterName, + String clusterUUID, + List activeManifestBlobMetadata, + List staleManifestBlobMetadata + ) { + try { + Set filesToKeep = new HashSet<>(); + Set staleManifestPaths = new HashSet<>(); + Set staleIndexMetadataPaths = new HashSet<>(); + Set staleGlobalMetadataPaths = new HashSet<>(); + Set staleIndexRoutingPaths = new HashSet<>(); + activeManifestBlobMetadata.forEach(blobMetadata -> { + ClusterMetadataManifest clusterMetadataManifest = remoteClusterStateService.getRemoteManifestManager().fetchRemoteClusterMetadataManifest( + clusterName, + clusterUUID, + blobMetadata.name() + ); + clusterMetadataManifest.getIndices() + .forEach(uploadedIndexMetadata -> filesToKeep.add(RemoteClusterStateUtils.getFormattedFileName(uploadedIndexMetadata.getUploadedFilename(), clusterMetadataManifest.getCodecVersion()))); + if (clusterMetadataManifest.getCodecVersion() == ClusterMetadataManifest.CODEC_V1) { + filesToKeep.add(RemoteClusterStateUtils.getFormattedFileName(clusterMetadataManifest.getGlobalMetadataFileName(), clusterMetadataManifest.getCodecVersion())); + } else if (clusterMetadataManifest.getCodecVersion() >= ClusterMetadataManifest.CODEC_V2) { + filesToKeep.add(clusterMetadataManifest.getCoordinationMetadata().getUploadedFilename()); + filesToKeep.add(clusterMetadataManifest.getSettingsMetadata().getUploadedFilename()); + filesToKeep.add(clusterMetadataManifest.getTemplatesMetadata().getUploadedFilename()); + clusterMetadataManifest.getCustomMetadataMap().values().forEach(attribute -> filesToKeep.add(attribute.getUploadedFilename())); + } + if (clusterMetadataManifest.getCodecVersion() >= ClusterMetadataManifest.CODEC_V3) { + clusterMetadataManifest.getIndicesRouting() + .forEach(uploadedIndicesRouting -> filesToKeep.add(uploadedIndicesRouting.getUploadedFilename())); + } + }); + staleManifestBlobMetadata.forEach(blobMetadata -> { + ClusterMetadataManifest clusterMetadataManifest = remoteClusterStateService.getRemoteManifestManager().fetchRemoteClusterMetadataManifest( + clusterName, + clusterUUID, + blobMetadata.name() + ); + staleManifestPaths.add(remoteClusterStateService.getRemoteManifestManager().getManifestFolderPath(clusterName, clusterUUID).buildAsString() + blobMetadata.name()); + if (clusterMetadataManifest.getCodecVersion() == ClusterMetadataManifest.CODEC_V1) { + addStaleGlobalMetadataPath(RemoteClusterStateUtils.getFormattedFileName(clusterMetadataManifest.getGlobalMetadataFileName(), clusterMetadataManifest.getCodecVersion()), filesToKeep, staleGlobalMetadataPaths); + } else if (clusterMetadataManifest.getCodecVersion() >= ClusterMetadataManifest.CODEC_V2) { + addStaleGlobalMetadataPath(clusterMetadataManifest.getCoordinationMetadata().getUploadedFilename(), filesToKeep, staleGlobalMetadataPaths); + addStaleGlobalMetadataPath(clusterMetadataManifest.getSettingsMetadata().getUploadedFilename(), filesToKeep, staleGlobalMetadataPaths); + addStaleGlobalMetadataPath(clusterMetadataManifest.getTemplatesMetadata().getUploadedFilename(), filesToKeep, staleGlobalMetadataPaths); + clusterMetadataManifest.getCustomMetadataMap().values().forEach(attribute -> { + addStaleGlobalMetadataPath(attribute.getUploadedFilename(), filesToKeep, staleGlobalMetadataPaths); + }); + } + if (clusterMetadataManifest.getCodecVersion() >= ClusterMetadataManifest.CODEC_V3) { + clusterMetadataManifest.getIndicesRouting().forEach(uploadedIndicesRouting -> { + if (filesToKeep.contains(uploadedIndicesRouting.getUploadedFilename()) == false){ + staleIndexRoutingPaths.add(new BlobPath().buildAsString() + uploadedIndicesRouting.getUploadedFilename()); + logger.debug("Indices routing paths in stale manifest: {}", uploadedIndicesRouting.getUploadedFilename()); + } + }); + } + + clusterMetadataManifest.getIndices().forEach(uploadedIndexMetadata -> { + if (filesToKeep.contains(uploadedIndexMetadata.getUploadedFilename()) == false) { + staleIndexMetadataPaths.add( + RemoteClusterStateUtils.getFormattedFileName(uploadedIndexMetadata.getUploadedFilename(), clusterMetadataManifest.getCodecVersion()) + ); + } + }); + }); + + if (staleManifestPaths.isEmpty()) { + logger.debug("No stale Remote Cluster Metadata files found"); + return; + } + + deleteStalePaths(new ArrayList<>(staleGlobalMetadataPaths)); + deleteStalePaths(new ArrayList<>(staleIndexMetadataPaths)); + deleteStalePaths(new ArrayList<>(staleManifestPaths)); + deleteStaleIndexRoutingPaths(new ArrayList<>(staleIndexRoutingPaths)); + } catch (IllegalStateException e) { + logger.error("Error while fetching Remote Cluster Metadata manifests", e); + } catch (IOException e) { + logger.error("Error while deleting stale Remote Cluster Metadata files", e); + remoteStateStats.cleanUpAttemptFailed(); + } catch (Exception e) { + logger.error("Unexpected error while deleting stale Remote Cluster Metadata files", e); + remoteStateStats.cleanUpAttemptFailed(); + } + } + + /** + * Deletes older than last {@code versionsToRetain} manifests. Also cleans up unreferenced IndexMetadata associated with older manifests + * + * @param clusterName name of the cluster + * @param clusterUUID uuid of cluster state to refer to in remote + * @param manifestsToRetain no of latest manifest files to keep in remote + */ + // package private for testing + void deleteStaleClusterMetadata(String clusterName, String clusterUUID, int manifestsToRetain) { + if (deleteStaleMetadataRunning.compareAndSet(false, true) == false) { + logger.info("Delete stale cluster metadata task is already in progress."); + return; + } + try { + getBlobStoreTransferService().listAllInSortedOrderAsync( + ThreadPool.Names.REMOTE_PURGE, + remoteClusterStateService.getRemoteManifestManager().getManifestFolderPath(clusterName, clusterUUID), + MANIFEST, + Integer.MAX_VALUE, + new ActionListener<>() { + @Override + public void onResponse(List blobMetadata) { + if (blobMetadata.size() > manifestsToRetain) { + deleteClusterMetadata( + clusterName, + clusterUUID, + blobMetadata.subList(0, manifestsToRetain), + blobMetadata.subList(manifestsToRetain, blobMetadata.size()) + ); + } + deleteStaleMetadataRunning.set(false); + } + + @Override + public void onFailure(Exception e) { + logger.error( + new ParameterizedMessage( + "Exception occurred while deleting Remote Cluster Metadata for clusterUUIDs {}", + clusterUUID + ) + ); + deleteStaleMetadataRunning.set(false); + } + } + ); + } catch (Exception e) { + deleteStaleMetadataRunning.set(false); + throw e; + } + } + + /** + * Purges all remote cluster state against provided cluster UUIDs + * + * @param clusterName name of the cluster + * @param clusterUUIDs clusteUUIDs for which the remote state needs to be purged + */ + void deleteStaleUUIDsClusterMetadata(String clusterName, List clusterUUIDs) { + clusterUUIDs.forEach( + clusterUUID -> getBlobStoreTransferService().deleteAsync( + ThreadPool.Names.REMOTE_PURGE, + RemoteClusterStateUtils.getCusterMetadataBasePath(remoteClusterStateService.getBlobStoreRepository(), clusterName, clusterUUID), + new ActionListener<>() { + @Override + public void onResponse(Void unused) { + logger.info("Deleted all remote cluster metadata for cluster UUID - {}", clusterUUID); + } + + @Override + public void onFailure(Exception e) { + logger.error( + new ParameterizedMessage( + "Exception occurred while deleting all remote cluster metadata for cluster UUID {}", + clusterUUID + ), + e + ); + remoteStateStats.cleanUpAttemptFailed(); + } + } + ) + ); + } + + // package private for testing + void deleteStalePaths(List stalePaths) throws IOException { + logger.debug(String.format(Locale.ROOT, "Deleting stale files from remote - %s", stalePaths)); + getBlobStoreTransferService().deleteBlobs( + BlobPath.cleanPath(), + stalePaths + ); + } + + void deleteStaleIndexRoutingPaths(List stalePaths) throws IOException { + logger.debug(String.format(Locale.ROOT, "Deleting stale index routing files from remote - %s", stalePaths)); + getBlobStoreTransferService().deleteBlobs( + BlobPath.cleanPath(), + stalePaths + ); + } + + /** + * Purges all remote cluster state against provided cluster UUIDs + * @param clusterState current state of the cluster + * @param committedManifest last committed ClusterMetadataManifest + */ + public void deleteStaleClusterUUIDs(ClusterState clusterState, ClusterMetadataManifest committedManifest) { + threadpool.executor(ThreadPool.Names.REMOTE_PURGE).execute(() -> { + String clusterName = clusterState.getClusterName().value(); + logger.debug("Deleting stale cluster UUIDs data from remote [{}]", clusterName); + Set allClustersUUIDsInRemote; + try { + allClustersUUIDsInRemote = new HashSet<>( + remoteClusterStateService.getAllClusterUUIDs(clusterState.getClusterName().value()) + ); + } catch (IOException e) { + logger.info(String.format(Locale.ROOT, "Error while fetching all cluster UUIDs for [%s]", clusterName)); + return; + } + // Retain last 2 cluster uuids data + allClustersUUIDsInRemote.remove(committedManifest.getClusterUUID()); + allClustersUUIDsInRemote.remove(committedManifest.getPreviousClusterUUID()); + deleteStaleUUIDsClusterMetadata(clusterName, new ArrayList<>(allClustersUUIDsInRemote)); + }); + } + + public TimeValue getStaleFileCleanupInterval() { + return this.staleFileCleanupInterval; + } + + AsyncStaleFileDeletion getStaleFileDeletionTask() { // for testing + return this.staleFileDeletionTask; + } + + RemotePersistenceStats getStats() { + return this.remoteStateStats; + } + + static final class AsyncStaleFileDeletion extends AbstractAsyncTask { + private final RemoteClusterStateCleanupManager remoteClusterStateCleanupManager; + + AsyncStaleFileDeletion(RemoteClusterStateCleanupManager remoteClusterStateCleanupManager) { + super( + logger, + remoteClusterStateCleanupManager.threadpool, + remoteClusterStateCleanupManager.getStaleFileCleanupInterval(), + true + ); + this.remoteClusterStateCleanupManager = remoteClusterStateCleanupManager; + rescheduleIfNecessary(); + } + + @Override + protected boolean mustReschedule() { + return true; + } + + @Override + protected void runInternal() { + remoteClusterStateCleanupManager.cleanUpStaleFiles(); + } + } +} 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 1a67f3cf25bbf..d7461f9f06475 100644 --- a/server/src/main/java/org/opensearch/gateway/remote/RemoteClusterStateService.java +++ b/server/src/main/java/org/opensearch/gateway/remote/RemoteClusterStateService.java @@ -8,22 +8,71 @@ package org.opensearch.gateway.remote; +import static java.util.Collections.emptyList; +import static java.util.Collections.emptyMap; +import static org.opensearch.gateway.PersistedClusterStateService.SLOW_WRITE_LOGGING_THRESHOLD; +import static org.opensearch.gateway.remote.ClusterMetadataManifest.CODEC_V1; +import static org.opensearch.gateway.remote.ClusterMetadataManifest.CODEC_V2; +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; +import static org.opensearch.gateway.remote.RemoteClusterStateUtils.DELIMITER; +import static org.opensearch.gateway.remote.RemoteClusterStateUtils.UploadedMetadataResults; +import static org.opensearch.gateway.remote.RemoteClusterStateUtils.clusterUUIDContainer; +import static org.opensearch.gateway.remote.model.RemoteCoordinationMetadata.COORDINATION_METADATA; +import static org.opensearch.gateway.remote.model.RemoteCustomMetadata.CUSTOM_DELIMITER; +import static org.opensearch.gateway.remote.model.RemoteCustomMetadata.CUSTOM_METADATA; +import static org.opensearch.gateway.remote.model.RemotePersistentSettingsMetadata.SETTING_METADATA; +import static org.opensearch.gateway.remote.model.RemoteTemplatesMetadata.TEMPLATES_METADATA; +import static org.opensearch.gateway.remote.model.RemoteClusterStateCustoms.CLUSTER_STATE_CUSTOM; +import static org.opensearch.gateway.remote.model.RemoteHashesOfConsistentSettings.HASHES_OF_CONSISTENT_SETTINGS; +import static org.opensearch.gateway.remote.model.RemoteIndexMetadata.INDEX_PATH_TOKEN; +import static org.opensearch.gateway.remote.model.RemoteTransientSettingsMetadata.TRANSIENT_SETTING_METADATA; +import static org.opensearch.node.remotestore.RemoteStoreNodeAttribute.isRemotePublicationEnabled; +import static org.opensearch.node.remotestore.RemoteStoreNodeAttribute.isRemoteStoreClusterStateEnabled; + +import java.io.Closeable; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Locale; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Function; +import java.util.function.LongSupplier; +import java.util.function.Supplier; +import java.util.stream.Collectors; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; -import org.opensearch.Version; import org.opensearch.action.LatchedActionListener; +import org.opensearch.cluster.ClusterName; import org.opensearch.cluster.ClusterState; +import org.opensearch.cluster.DiffableUtils; +import org.opensearch.cluster.block.ClusterBlocks; import org.opensearch.cluster.coordination.CoordinationMetadata; +import org.opensearch.cluster.metadata.DiffableStringMap; import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.cluster.metadata.Metadata; +import org.opensearch.cluster.metadata.Metadata.Custom; import org.opensearch.cluster.metadata.TemplatesMetadata; +import org.opensearch.cluster.node.DiscoveryNodes; +import org.opensearch.cluster.routing.IndexRoutingTable; +import org.opensearch.cluster.routing.RoutingTable; import org.opensearch.cluster.routing.remote.RemoteRoutingTableService; +import org.opensearch.cluster.service.ClusterService; import org.opensearch.common.CheckedRunnable; import org.opensearch.common.Nullable; import org.opensearch.common.blobstore.BlobContainer; -import org.opensearch.common.blobstore.BlobMetadata; -import org.opensearch.common.blobstore.BlobPath; +import org.opensearch.common.blobstore.BlobStore; +import org.opensearch.common.collect.Tuple; import org.opensearch.common.settings.ClusterSettings; import org.opensearch.common.settings.Setting; import org.opensearch.common.settings.Setting.Property; @@ -31,46 +80,35 @@ import org.opensearch.common.unit.TimeValue; import org.opensearch.common.util.io.IOUtils; import org.opensearch.core.action.ActionListener; +import org.opensearch.core.common.io.stream.NamedWriteableRegistry; +import org.opensearch.core.index.Index; import org.opensearch.core.xcontent.ToXContent; import org.opensearch.gateway.remote.ClusterMetadataManifest.UploadedIndexMetadata; import org.opensearch.gateway.remote.ClusterMetadataManifest.UploadedMetadataAttribute; -import org.opensearch.index.remote.RemoteStoreUtils; +import org.opensearch.gateway.remote.model.RemoteClusterBlocks; +import org.opensearch.gateway.remote.model.RemoteClusterMetadataManifest; +import org.opensearch.gateway.remote.model.RemoteClusterStateCustoms; +import org.opensearch.gateway.remote.model.RemoteCoordinationMetadata; +import org.opensearch.gateway.remote.model.RemoteCustomMetadata; +import org.opensearch.gateway.remote.model.RemoteDiscoveryNodes; +import org.opensearch.gateway.remote.model.RemoteGlobalMetadata; +import org.opensearch.gateway.remote.model.RemoteHashesOfConsistentSettings; +import org.opensearch.gateway.remote.model.RemoteIndexMetadata; +import org.opensearch.gateway.remote.model.RemotePersistentSettingsMetadata; +import org.opensearch.gateway.remote.model.RemoteReadResult; +import org.opensearch.gateway.remote.model.RemoteTemplatesMetadata; +import org.opensearch.gateway.remote.model.RemoteTransientSettingsMetadata; +import org.opensearch.gateway.remote.model.RemoteClusterStateBlobStore; +import org.opensearch.common.remote.RemoteWritableEntityStore; +import org.opensearch.gateway.remote.model.RemoteUploadDetails; import org.opensearch.index.translog.transfer.BlobStoreTransferService; import org.opensearch.node.Node; import org.opensearch.node.remotestore.RemoteStoreNodeAttribute; import org.opensearch.repositories.RepositoriesService; import org.opensearch.repositories.Repository; import org.opensearch.repositories.blobstore.BlobStoreRepository; -import org.opensearch.repositories.blobstore.ChecksumBlobStoreFormat; import org.opensearch.threadpool.ThreadPool; -import java.io.Closeable; -import java.io.IOException; -import java.nio.charset.StandardCharsets; -import java.util.ArrayList; -import java.util.Base64; -import java.util.Collections; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Locale; -import java.util.Map; -import java.util.Objects; -import java.util.Optional; -import java.util.Set; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicReference; -import java.util.function.Function; -import java.util.function.LongSupplier; -import java.util.function.Supplier; -import java.util.stream.Collectors; - -import static java.util.Objects.requireNonNull; -import static org.opensearch.gateway.PersistedClusterStateService.SLOW_WRITE_LOGGING_THRESHOLD; -import static org.opensearch.node.remotestore.RemoteStoreNodeAttribute.isRemoteRoutingTableEnabled; -import static org.opensearch.node.remotestore.RemoteStoreNodeAttribute.isRemoteStoreClusterStateEnabled; /** * A Service which provides APIs to upload and download cluster metadata from remote store. @@ -79,101 +117,10 @@ */ public class RemoteClusterStateService implements Closeable { - public static final String METADATA_NAME_FORMAT = "%s.dat"; - - public static final String METADATA_MANIFEST_NAME_FORMAT = "%s"; - public static final int RETAINED_MANIFESTS = 10; - public static final String DELIMITER = "__"; - public static final String CUSTOM_DELIMITER = "--"; - private static final Logger logger = LogManager.getLogger(RemoteClusterStateService.class); - public static final TimeValue INDEX_METADATA_UPLOAD_TIMEOUT_DEFAULT = TimeValue.timeValueMillis(20000); - - public static final TimeValue GLOBAL_METADATA_UPLOAD_TIMEOUT_DEFAULT = TimeValue.timeValueMillis(20000); - - public static final TimeValue METADATA_MANIFEST_UPLOAD_TIMEOUT_DEFAULT = TimeValue.timeValueMillis(20000); - - public static final Setting INDEX_METADATA_UPLOAD_TIMEOUT_SETTING = Setting.timeSetting( - "cluster.remote_store.state.index_metadata.upload_timeout", - INDEX_METADATA_UPLOAD_TIMEOUT_DEFAULT, - Setting.Property.Dynamic, - Setting.Property.NodeScope - ); - - public static final Setting GLOBAL_METADATA_UPLOAD_TIMEOUT_SETTING = Setting.timeSetting( - "cluster.remote_store.state.global_metadata.upload_timeout", - GLOBAL_METADATA_UPLOAD_TIMEOUT_DEFAULT, - Setting.Property.Dynamic, - Setting.Property.NodeScope - ); - - public static final Setting METADATA_MANIFEST_UPLOAD_TIMEOUT_SETTING = Setting.timeSetting( - "cluster.remote_store.state.metadata_manifest.upload_timeout", - METADATA_MANIFEST_UPLOAD_TIMEOUT_DEFAULT, - Setting.Property.Dynamic, - Setting.Property.NodeScope - ); - - public static final ChecksumBlobStoreFormat INDEX_METADATA_FORMAT = new ChecksumBlobStoreFormat<>( - "index-metadata", - METADATA_NAME_FORMAT, - IndexMetadata::fromXContent - ); - - public static final ChecksumBlobStoreFormat GLOBAL_METADATA_FORMAT = new ChecksumBlobStoreFormat<>( - "metadata", - METADATA_NAME_FORMAT, - Metadata::fromXContent - ); - - public static final ChecksumBlobStoreFormat COORDINATION_METADATA_FORMAT = new ChecksumBlobStoreFormat<>( - "coordination", - METADATA_NAME_FORMAT, - CoordinationMetadata::fromXContent - ); - - public static final ChecksumBlobStoreFormat SETTINGS_METADATA_FORMAT = new ChecksumBlobStoreFormat<>( - "settings", - METADATA_NAME_FORMAT, - Settings::fromXContent - ); - - public static final ChecksumBlobStoreFormat TEMPLATES_METADATA_FORMAT = new ChecksumBlobStoreFormat<>( - "templates", - METADATA_NAME_FORMAT, - TemplatesMetadata::fromXContent - ); - - public static final ChecksumBlobStoreFormat CUSTOM_METADATA_FORMAT = new ChecksumBlobStoreFormat<>( - "custom", - METADATA_NAME_FORMAT, - null // no need of reader here, as this object is only used to write/serialize the object - ); - - /** - * Manifest format compatible with older codec v0, where codec version was missing. - */ - public static final ChecksumBlobStoreFormat CLUSTER_METADATA_MANIFEST_FORMAT_V0 = - new ChecksumBlobStoreFormat<>("cluster-metadata-manifest", METADATA_MANIFEST_NAME_FORMAT, ClusterMetadataManifest::fromXContentV0); - - /** - * Manifest format compatible with older codec v1, where codec versions/global metadata was introduced. - */ - public static final ChecksumBlobStoreFormat CLUSTER_METADATA_MANIFEST_FORMAT_V1 = - new ChecksumBlobStoreFormat<>("cluster-metadata-manifest", METADATA_MANIFEST_NAME_FORMAT, ClusterMetadataManifest::fromXContentV1); - - /** - * Manifest format compatible with codec v2, where global metadata file is replaced with multiple metadata attribute files - */ - public static final ChecksumBlobStoreFormat CLUSTER_METADATA_MANIFEST_FORMAT = new ChecksumBlobStoreFormat<>( - "cluster-metadata-manifest", - METADATA_MANIFEST_NAME_FORMAT, - ClusterMetadataManifest::fromXContent - ); - /** * Used to specify if cluster state metadata should be published to remote store */ @@ -184,18 +131,16 @@ public class RemoteClusterStateService implements Closeable { Property.Final ); - public static final String CLUSTER_STATE_PATH_TOKEN = "cluster-state"; - public static final String INDEX_PATH_TOKEN = "index"; - public static final String GLOBAL_METADATA_PATH_TOKEN = "global-metadata"; - public static final String MANIFEST_PATH_TOKEN = "manifest"; - public static final String MANIFEST_FILE_PREFIX = "manifest"; - public static final String METADATA_FILE_PREFIX = "metadata"; - public static final String COORDINATION_METADATA = "coordination"; - public static final String SETTING_METADATA = "settings"; - public static final String TEMPLATES_METADATA = "templates"; - public static final String CUSTOM_METADATA = "custom"; - public static final int SPLITED_MANIFEST_FILE_LENGTH = 6; // file name manifest__term__version__C/P__timestamp__codecversion + public static final TimeValue REMOTE_STATE_READ_TIMEOUT_DEFAULT = TimeValue.timeValueMillis(20000); + public static final Setting REMOTE_STATE_READ_TIMEOUT_SETTING = Setting.timeSetting( + "cluster.remote_store.state.read_timeout", + REMOTE_STATE_READ_TIMEOUT_DEFAULT, + Setting.Property.Dynamic, + Setting.Property.NodeScope + ); + + private TimeValue remoteStateReadTimeout; private final String nodeId; private final Supplier repositoriesService; private final Settings settings; @@ -207,19 +152,19 @@ public class RemoteClusterStateService implements Closeable { private Optional remoteRoutingTableService; private volatile TimeValue slowWriteLoggingThreshold; - private volatile TimeValue indexMetadataUploadTimeout; - private volatile TimeValue globalMetadataUploadTimeout; - private volatile TimeValue metadataManifestUploadTimeout; - - private final AtomicBoolean deleteStaleMetadataRunning = new AtomicBoolean(false); private final RemotePersistenceStats remoteStateStats; + private RemoteClusterStateCleanupManager remoteClusterStateCleanupManager; + private RemoteIndexMetadataManager remoteIndexMetadataManager; + private RemoteGlobalMetadataManager remoteGlobalMetadataManager; + private RemoteClusterStateAttributesManager remoteClusterStateAttributesManager; + private RemoteManifestManager remoteManifestManager; + private ClusterSettings clusterSettings; + private NamedWriteableRegistry namedWriteableRegistry; private final String CLUSTER_STATE_UPLOAD_TIME_LOG_STRING = "writing cluster state for version [{}] took [{}ms]"; private final String METADATA_UPDATE_LOG_STRING = "wrote metadata for [{}] indices and skipped [{}] unchanged " + "indices, coordination metadata updated : [{}], settings metadata updated : [{}], templates metadata " + "updated : [{}], custom metadata updated : [{}]"; public static final int INDEX_METADATA_CURRENT_CODEC_VERSION = 1; - public static final int MANIFEST_CURRENT_CODEC_VERSION = ClusterMetadataManifest.CODEC_V2; - public static final int GLOBAL_METADATA_CURRENT_CODEC_VERSION = 2; // ToXContent Params with gateway mode. // We are using gateway context mode to persist all custom metadata. @@ -235,37 +180,32 @@ public RemoteClusterStateService( String nodeId, Supplier repositoriesService, Settings settings, - ClusterSettings clusterSettings, + ClusterService clusterService, LongSupplier relativeTimeNanosSupplier, ThreadPool threadPool, - List indexMetadataUploadListeners + List indexMetadataUploadListeners, + NamedWriteableRegistry namedWriteableRegistry ) { assert isRemoteStoreClusterStateEnabled(settings) : "Remote cluster state is not enabled"; + logger.info("REMOTE STATE ENABLED"); + this.nodeId = nodeId; this.repositoriesService = repositoriesService; this.settings = settings; this.relativeTimeNanosSupplier = relativeTimeNanosSupplier; this.threadpool = threadPool; + clusterSettings = clusterService.getClusterSettings(); this.slowWriteLoggingThreshold = clusterSettings.get(SLOW_WRITE_LOGGING_THRESHOLD); - this.indexMetadataUploadTimeout = clusterSettings.get(INDEX_METADATA_UPLOAD_TIMEOUT_SETTING); - this.globalMetadataUploadTimeout = clusterSettings.get(GLOBAL_METADATA_UPLOAD_TIMEOUT_SETTING); - this.metadataManifestUploadTimeout = clusterSettings.get(METADATA_MANIFEST_UPLOAD_TIMEOUT_SETTING); clusterSettings.addSettingsUpdateConsumer(SLOW_WRITE_LOGGING_THRESHOLD, this::setSlowWriteLoggingThreshold); - clusterSettings.addSettingsUpdateConsumer(INDEX_METADATA_UPLOAD_TIMEOUT_SETTING, this::setIndexMetadataUploadTimeout); - clusterSettings.addSettingsUpdateConsumer(GLOBAL_METADATA_UPLOAD_TIMEOUT_SETTING, this::setGlobalMetadataUploadTimeout); - clusterSettings.addSettingsUpdateConsumer(METADATA_MANIFEST_UPLOAD_TIMEOUT_SETTING, this::setMetadataManifestUploadTimeout); + this.remoteStateReadTimeout = clusterSettings.get(REMOTE_STATE_READ_TIMEOUT_SETTING); + clusterSettings.addSettingsUpdateConsumer(REMOTE_STATE_READ_TIMEOUT_SETTING, this::setRemoteClusterStateEnabled); this.remoteStateStats = new RemotePersistenceStats(); - this.indexMetadataUploadListeners = indexMetadataUploadListeners; - this.remoteRoutingTableService = isRemoteRoutingTableEnabled(settings) - ? Optional.of(new RemoteRoutingTableService(repositoriesService, settings)) + this.namedWriteableRegistry = namedWriteableRegistry; + this.remoteRoutingTableService = isRemotePublicationEnabled(settings) + ? Optional.of(new RemoteRoutingTableService(repositoriesService, settings, threadPool)) : Optional.empty(); - } - - private BlobStoreTransferService getBlobStoreTransferService() { - if (blobStoreTransferService == null) { - blobStoreTransferService = new BlobStoreTransferService(blobStoreRepository.blobStore(), threadpool); - } - return blobStoreTransferService; + this.remoteClusterStateCleanupManager = new RemoteClusterStateCleanupManager(this, clusterService); + this.indexMetadataUploadListeners = indexMetadataUploadListeners; } /** @@ -275,7 +215,8 @@ private BlobStoreTransferService getBlobStoreTransferService() { * @return A manifest object which contains the details of uploaded entity metadata. */ @Nullable - public ClusterMetadataManifest writeFullMetadata(ClusterState clusterState, String previousClusterUUID) throws IOException { + public RemoteUploadDetails writeFullMetadata(ClusterState clusterState, String previousClusterUUID) throws IOException { + logger.info("WRITING FULL STATE"); final long startTimeNanos = relativeTimeNanosSupplier.getAsLong(); if (clusterState.nodes().isLocalNodeElectedClusterManager() == false) { logger.error("Local node is not elected cluster manager. Exiting"); @@ -285,22 +226,28 @@ public ClusterMetadataManifest writeFullMetadata(ClusterState clusterState, Stri UploadedMetadataResults uploadedMetadataResults = writeMetadataInParallel( clusterState, new ArrayList<>(clusterState.metadata().indices().values()), - Collections.emptyMap(), + emptyMap(), clusterState.metadata().customs(), true, true, + true, + true, + true, + new ArrayList<>(clusterState.getRoutingTable().indicesRouting().values()), + true, + clusterState.customs(), true ); - final ClusterMetadataManifest manifest = uploadManifest( + + final RemoteUploadDetails manifestDetails = remoteManifestManager.uploadManifest( clusterState, - uploadedMetadataResults.uploadedIndexMetadata, + uploadedMetadataResults, previousClusterUUID, - uploadedMetadataResults.uploadedCoordinationMetadata, - uploadedMetadataResults.uploadedSettingsMetadata, - uploadedMetadataResults.uploadedTemplatesMetadata, - uploadedMetadataResults.uploadedCustomMetadataMap, + new ClusterStateDiffManifest(clusterState, ClusterState.EMPTY_STATE), false ); + + logger.info("MANIFEST IN FULL STATE {}", manifestDetails.getClusterMetadataManifest()); final long durationMillis = TimeValue.nsecToMSec(relativeTimeNanosSupplier.getAsLong() - startTimeNanos); remoteStateStats.stateSucceeded(); remoteStateStats.stateTook(durationMillis); @@ -318,7 +265,7 @@ public ClusterMetadataManifest writeFullMetadata(ClusterState clusterState, Stri uploadedMetadataResults.uploadedIndexMetadata.size() ); } - return manifest; + return manifestDetails; } /** @@ -329,11 +276,13 @@ public ClusterMetadataManifest writeFullMetadata(ClusterState clusterState, Stri * @return The uploaded ClusterMetadataManifest file */ @Nullable - public ClusterMetadataManifest writeIncrementalMetadata( + public RemoteUploadDetails writeIncrementalMetadata( ClusterState previousClusterState, ClusterState clusterState, ClusterMetadataManifest previousManifest ) throws IOException { + logger.info("WRITING INCREMENTAL STATE"); + final long startTimeNanos = relativeTimeNanosSupplier.getAsLong(); if (clusterState.nodes().isLocalNodeElectedClusterManager() == false) { logger.error("Local node is not elected cluster manager. Exiting"); @@ -342,7 +291,10 @@ public ClusterMetadataManifest writeIncrementalMetadata( assert previousClusterState.metadata().coordinationMetadata().term() == clusterState.metadata().coordinationMetadata().term(); final Map customsToBeDeletedFromRemote = new HashMap<>(previousManifest.getCustomMetadataMap()); - final Map customsToUpload = getUpdatedCustoms(clusterState, previousClusterState); + final Map clusterStateCustomsToBeDeleted = new HashMap<>(previousManifest.getClusterStateCustomMap()); + final Map customsToUpload = remoteGlobalMetadataManager.getUpdatedCustoms(clusterState, previousClusterState); + final Map clusterStateCustomsToUpload = remoteClusterStateAttributesManager.getUpdatedCustoms(clusterState, + previousClusterState); final Map allUploadedCustomMap = new HashMap<>(previousManifest.getCustomMetadataMap()); for (final String custom : clusterState.metadata().customs().keySet()) { // remove all the customs which are present currently @@ -350,7 +302,10 @@ public ClusterMetadataManifest writeIncrementalMetadata( } final Map indicesToBeDeletedFromRemote = new HashMap<>(previousClusterState.metadata().indices()); - + for (final String custom : clusterState.customs().keySet()) { + // remove all the custom which are present currently + clusterStateCustomsToBeDeleted.remove(custom); + } int numIndicesUpdated = 0; int numIndicesUnchanged = 0; final Map allUploadedIndexMetadata = previousManifest.getIndices() @@ -380,6 +335,14 @@ public ClusterMetadataManifest writeIncrementalMetadata( // index present in current cluster state indicesToBeDeletedFromRemote.remove(indexMetadata.getIndex().getName()); } + + List indicesRoutingToUpload = new ArrayList<>(); + DiffableUtils.MapDiff> routingTableDiff = null; + if (remoteRoutingTableService != null) { + routingTableDiff = RemoteRoutingTableService.getIndicesRoutingMapDiff(previousClusterState.getRoutingTable(), clusterState.getRoutingTable()); + routingTableDiff.getUpserts().forEach((k, v) -> indicesRoutingToUpload.add(v)); + } + UploadedMetadataResults uploadedMetadataResults; // For migration case from codec V0 or V1 to V2, we have added null check on metadata attribute files, // If file is empty and codec is 1 then write global metadata. @@ -389,8 +352,21 @@ public ClusterMetadataManifest writeIncrementalMetadata( ; boolean updateSettingsMetadata = firstUploadForSplitGlobalMetadata || Metadata.isSettingsMetadataEqual(previousClusterState.metadata(), clusterState.metadata()) == false; + boolean updateTransientSettingsMetadata = firstUploadForSplitGlobalMetadata + || Metadata.isTransientSettingsMetadataEqual(previousClusterState.metadata(), clusterState.metadata()) == false; boolean updateTemplatesMetadata = firstUploadForSplitGlobalMetadata || Metadata.isTemplatesMetadataEqual(previousClusterState.metadata(), clusterState.metadata()) == false; + // ToDo: check if these needs to be updated or not + final boolean updateDiscoveryNodes = clusterState.getNodes().delta(previousClusterState.getNodes()).hasChanges(); + final boolean updateClusterBlocks = !clusterState.blocks().equals(previousClusterState.blocks()); + final boolean updateHashesOfConsistentSettings = firstUploadForSplitGlobalMetadata + || Metadata.isHashesOfConsistentSettingsEqual(previousClusterState.metadata(), clusterState.metadata()) == false; + + // Write Index Metadata + final Map previousStateIndexMetadataByName = new HashMap<>(); + for (final IndexMetadata indexMetadata : previousClusterState.metadata().indices().values()) { + previousStateIndexMetadataByName.put(indexMetadata.getIndex().getName(), indexMetadata); + } uploadedMetadataResults = writeMetadataInParallel( clusterState, @@ -399,7 +375,13 @@ public ClusterMetadataManifest writeIncrementalMetadata( firstUploadForSplitGlobalMetadata ? clusterState.metadata().customs() : customsToUpload, updateCoordinationMetadata, updateSettingsMetadata, - updateTemplatesMetadata + updateTemplatesMetadata, + updateDiscoveryNodes, + updateClusterBlocks, + indicesRoutingToUpload, + updateTransientSettingsMetadata, + clusterStateCustomsToUpload, + updateHashesOfConsistentSettings ); // update the map if the metadata was uploaded @@ -410,27 +392,60 @@ public ClusterMetadataManifest writeIncrementalMetadata( // remove the data for removed custom/indices customsToBeDeletedFromRemote.keySet().forEach(allUploadedCustomMap::remove); indicesToBeDeletedFromRemote.keySet().forEach(allUploadedIndexMetadata::remove); + clusterStateCustomsToBeDeleted.keySet().forEach(allUploadedCustomMap::remove); + + List allUploadedIndicesRouting = new ArrayList<>(); + if (remoteRoutingTableService.isPresent()) { + allUploadedIndicesRouting = remoteRoutingTableService.get().getAllUploadedIndicesRouting(previousManifest, + uploadedMetadataResults.uploadedIndicesRoutingMetadata, indicesToBeDeletedFromRemote.keySet()); + } + + if (!updateCoordinationMetadata) { + uploadedMetadataResults.uploadedCoordinationMetadata = previousManifest.getCoordinationMetadata(); + } + if (!updateSettingsMetadata) { + uploadedMetadataResults.uploadedSettingsMetadata = previousManifest.getSettingsMetadata(); + } + if (!updateTransientSettingsMetadata) { + uploadedMetadataResults.uploadedTransientSettingsMetadata = previousManifest.getTransientSettingsMetadata(); + } + if (!updateTemplatesMetadata) { + uploadedMetadataResults.uploadedTemplatesMetadata = previousManifest.getTemplatesMetadata(); + } + if (!updateDiscoveryNodes && !firstUploadForSplitGlobalMetadata) { + uploadedMetadataResults.uploadedDiscoveryNodes = previousManifest.getDiscoveryNodesMetadata(); + } + if (!updateClusterBlocks && !firstUploadForSplitGlobalMetadata) { + uploadedMetadataResults.uploadedClusterBlocks = previousManifest.getClusterBlocksMetadata(); + } + if (!updateHashesOfConsistentSettings && !firstUploadForSplitGlobalMetadata) { + uploadedMetadataResults.uploadedHashesOfConsistentSettings = previousManifest.getHashesOfConsistentSettings(); + } + if (!firstUploadForSplitGlobalMetadata && customsToUpload.isEmpty()) { + uploadedMetadataResults.uploadedCustomMetadataMap = previousManifest.getCustomMetadataMap(); + } + if (!firstUploadForSplitGlobalMetadata && clusterStateCustomsToUpload.isEmpty()) { + uploadedMetadataResults.uploadedClusterStateCustomMetadataMap = previousManifest.getClusterStateCustomMap(); + } + uploadedMetadataResults.uploadedIndexMetadata = new ArrayList<>(allUploadedIndexMetadata.values()); + uploadedMetadataResults.uploadedIndicesRoutingMetadata = allUploadedIndicesRouting; - final ClusterMetadataManifest manifest = uploadManifest( + final RemoteUploadDetails manifestDetails = remoteManifestManager.uploadManifest( clusterState, - new ArrayList<>(allUploadedIndexMetadata.values()), + uploadedMetadataResults, previousManifest.getPreviousClusterUUID(), - updateCoordinationMetadata ? uploadedMetadataResults.uploadedCoordinationMetadata : previousManifest.getCoordinationMetadata(), - updateSettingsMetadata ? uploadedMetadataResults.uploadedSettingsMetadata : previousManifest.getSettingsMetadata(), - updateTemplatesMetadata ? uploadedMetadataResults.uploadedTemplatesMetadata : previousManifest.getTemplatesMetadata(), - firstUploadForSplitGlobalMetadata || !customsToUpload.isEmpty() - ? allUploadedCustomMap - : previousManifest.getCustomMetadataMap(), + new ClusterStateDiffManifest(clusterState, previousClusterState), false ); - deleteStaleClusterMetadata(clusterState.getClusterName().value(), clusterState.metadata().clusterUUID(), RETAINED_MANIFESTS); + + logger.info("MANIFEST IN INC STATE {}", manifestDetails.getClusterMetadataManifest()); final long durationMillis = TimeValue.nsecToMSec(relativeTimeNanosSupplier.getAsLong() - startTimeNanos); remoteStateStats.stateSucceeded(); remoteStateStats.stateTook(durationMillis); ParameterizedMessage clusterStateUploadTimeMessage = new ParameterizedMessage( CLUSTER_STATE_UPLOAD_TIME_LOG_STRING, - manifest.getStateVersion(), + manifestDetails.getClusterMetadataManifest().getStateVersion(), durationMillis ); ParameterizedMessage metadataUpdateMessage = new ParameterizedMessage( @@ -444,15 +459,35 @@ public ClusterMetadataManifest writeIncrementalMetadata( ); if (durationMillis >= slowWriteLoggingThreshold.getMillis()) { logger.warn( - "{} which is above the warn threshold of [{}]; {}", - clusterStateUploadTimeMessage, + "writing cluster state took [{}ms] which is above the warn threshold of [{}]; " + + "wrote metadata for [{}] indices and skipped [{}] unchanged indices, coordination metadata updated : [{}], " + + "settings metadata updated : [{}], templates metadata updated : [{}], custom metadata updated : [{}]", + durationMillis, slowWriteLoggingThreshold, - metadataUpdateMessage + numIndicesUpdated, + numIndicesUnchanged, + updateCoordinationMetadata, + updateSettingsMetadata, + updateTemplatesMetadata, + customsToUpload.size() ); } else { logger.info("{}; {}", clusterStateUploadTimeMessage, metadataUpdateMessage); + logger.info( + "writing cluster state for version [{}] took [{}ms]; " + + "wrote metadata for [{}] indices and skipped [{}] unchanged indices, coordination metadata updated : [{}], " + + "settings metadata updated : [{}], templates metadata updated : [{}], custom metadata updated : [{}]", + manifestDetails.getClusterMetadataManifest().getStateVersion(), + durationMillis, + numIndicesUpdated, + numIndicesUnchanged, + updateCoordinationMetadata, + updateSettingsMetadata, + updateTemplatesMetadata, + customsToUpload.size() + ); } - return manifest; + return manifestDetails; } private UploadedMetadataResults writeMetadataInParallel( @@ -462,19 +497,27 @@ private UploadedMetadataResults writeMetadataInParallel( Map customToUpload, boolean uploadCoordinationMetadata, boolean uploadSettingsMetadata, - boolean uploadTemplateMetadata + boolean uploadTemplateMetadata, + boolean uploadDiscoveryNodes, + boolean uploadClusterBlock, + List indicesRoutingToUpload, + boolean uploadTransientSettingMetadata, + Map clusterStateCustomToUpload, + boolean uploadHashesOfConsistentSettings ) 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); + int totalUploadTasks = + indexToUpload.size() + indexMetadataUploadListeners.size() + customToUpload.size() + (uploadCoordinationMetadata ? 1 : 0) + (uploadSettingsMetadata + ? 1 : 0) + (uploadTemplateMetadata ? 1 : 0) + (uploadDiscoveryNodes ? 1 : 0) + (uploadClusterBlock ? 1 : 0) + indicesRoutingToUpload.size() + + (uploadTransientSettingMetadata ? 1 : 0) + clusterStateCustomToUpload.size(); + // + (uploadHashesOfConsistentSettings ? 1 : 0); CountDownLatch latch = new CountDownLatch(totalUploadTasks); - Map> uploadTasks = new HashMap<>(totalUploadTasks); - Map results = new HashMap<>(totalUploadTasks); + Map> uploadTasks = new ConcurrentHashMap<>(totalUploadTasks); + Map results = new ConcurrentHashMap<>(totalUploadTasks); List exceptionList = Collections.synchronizedList(new ArrayList<>(totalUploadTasks)); LatchedActionListener listener = new LatchedActionListener<>( ActionListener.wrap((ClusterMetadataManifest.UploadedMetadata uploadedMetadata) -> { - logger.trace(String.format(Locale.ROOT, "Metadata component %s uploaded successfully.", uploadedMetadata.getComponent())); + logger.info(String.format(Locale.ROOT, "Metadata component %s uploaded successfully.", uploadedMetadata.getComponent())); results.put(uploadedMetadata.getComponent(), uploadedMetadata); }, ex -> { logger.error( @@ -489,59 +532,135 @@ private UploadedMetadataResults writeMetadataInParallel( if (uploadSettingsMetadata) { uploadTasks.put( SETTING_METADATA, - getAsyncMetadataWriteAction( - clusterState, - SETTING_METADATA, - SETTINGS_METADATA_FORMAT, + remoteGlobalMetadataManager.getAsyncMetadataWriteAction( clusterState.metadata().persistentSettings(), - listener + clusterState.metadata().version(), + clusterState.metadata().clusterUUID(), + listener, + null + ) + ); + } + if (uploadTransientSettingMetadata) { + uploadTasks.put( + TRANSIENT_SETTING_METADATA, + remoteGlobalMetadataManager.getAsyncMetadataWriteAction( + clusterState.metadata().transientSettings(), + clusterState.metadata().version(), + clusterState.metadata().clusterUUID(), + listener, + TRANSIENT_SETTING_METADATA ) ); } if (uploadCoordinationMetadata) { uploadTasks.put( COORDINATION_METADATA, - getAsyncMetadataWriteAction( - clusterState, - COORDINATION_METADATA, - COORDINATION_METADATA_FORMAT, + remoteGlobalMetadataManager.getAsyncMetadataWriteAction( clusterState.metadata().coordinationMetadata(), - listener + clusterState.metadata().version(), + clusterState.metadata().clusterUUID(), + listener, + null ) ); } if (uploadTemplateMetadata) { uploadTasks.put( TEMPLATES_METADATA, - getAsyncMetadataWriteAction( - clusterState, - TEMPLATES_METADATA, - TEMPLATES_METADATA_FORMAT, + remoteGlobalMetadataManager.getAsyncMetadataWriteAction( clusterState.metadata().templatesMetadata(), + clusterState.metadata().version(), + clusterState.metadata().clusterUUID(), + listener, + null + ) + ); + } + if (uploadDiscoveryNodes) { + uploadTasks.put( + DISCOVERY_NODES, + remoteClusterStateAttributesManager.getAsyncMetadataWriteAction( + clusterState, + DISCOVERY_NODES, + clusterState.nodes(), + listener + ) + ); + } + if (uploadClusterBlock) { + uploadTasks.put( + CLUSTER_BLOCKS, + remoteClusterStateAttributesManager.getAsyncMetadataWriteAction( + clusterState, + CLUSTER_BLOCKS, + clusterState.blocks(), listener ) ); } +// if (uploadHashesOfConsistentSettings) { +// uploadTasks.put( +// HASHES_OF_CONSISTENT_SETTINGS, +// remoteGlobalMetadataManager.getAsyncMetadataWriteAction( +// new DiffableStringMap(clusterState.metadata().hashesOfConsistentSettings()), +// clusterState.metadata().version(), +// clusterState.metadata().clusterUUID(), +// listener, +// null +// ) +// ); +// } customToUpload.forEach((key, value) -> { String customComponent = String.join(CUSTOM_DELIMITER, CUSTOM_METADATA, key); uploadTasks.put( customComponent, - getAsyncMetadataWriteAction(clusterState, customComponent, CUSTOM_METADATA_FORMAT, value, listener) + remoteGlobalMetadataManager.getAsyncMetadataWriteAction( + value, + clusterState.metadata().version(), + clusterState.metadata().clusterUUID(), + listener, + key + ) ); }); indexToUpload.forEach(indexMetadata -> { - uploadTasks.put(indexMetadata.getIndex().getName(), getIndexMetadataAsyncAction(clusterState, indexMetadata, listener)); + uploadTasks.put( + indexMetadata.getIndexName(), + remoteIndexMetadataManager.getIndexMetadataAsyncAction(indexMetadata, clusterState.metadata().clusterUUID(), listener) + ); + }); + clusterStateCustomToUpload.forEach((key, value) -> { + uploadTasks.put( + key, + remoteClusterStateAttributesManager.getAsyncMetadataWriteAction( + clusterState, + key, + value, + listener + ) + ); + }); + + indicesRoutingToUpload.forEach(indexRoutingTable -> { + try { + uploadTasks.put( + indexRoutingTable.getIndex().getName() + "--indexRouting", + remoteRoutingTableService.get().getIndexRoutingAsyncAction(clusterState, indexRoutingTable, listener) + ); + } catch (IOException e) { + e.printStackTrace(); + } }); // start async upload of all required metadata files for (CheckedRunnable uploadTask : uploadTasks.values()) { uploadTask.run(); } - invokeIndexMetadataUploadListeners(indexToUpload, prevIndexMetadataByName, latch, exceptionList); try { - if (latch.await(getGlobalMetadataUploadTimeout().millis(), TimeUnit.MILLISECONDS) == false) { + if (latch.await(remoteGlobalMetadataManager.getGlobalMetadataUploadTimeout().millis(), TimeUnit.MILLISECONDS) == false) { // TODO: We should add metrics where transfer is timing out. [Issue: #10687] RemoteStateTransferException ex = new RemoteStateTransferException( String.format( @@ -579,25 +698,43 @@ private UploadedMetadataResults writeMetadataInParallel( } UploadedMetadataResults response = new UploadedMetadataResults(); results.forEach((name, uploadedMetadata) -> { - if (name.contains(CUSTOM_METADATA)) { + if (uploadedMetadata.getClass().equals(UploadedIndexMetadata.class) && + uploadedMetadata.getComponent().contains(RemoteRoutingTableService.INDEX_ROUTING_METADATA_PREFIX)) { + response.uploadedIndicesRoutingMetadata.add((UploadedIndexMetadata) uploadedMetadata); + } else if (name.contains(CUSTOM_METADATA)) { // component name for custom metadata will look like custom-- String custom = name.split(DELIMITER)[0].split(CUSTOM_DELIMITER)[1]; response.uploadedCustomMetadataMap.put( custom, new UploadedMetadataAttribute(custom, uploadedMetadata.getUploadedFilename()) ); - } else if (COORDINATION_METADATA.equals(name)) { + } else if (name.contains(CLUSTER_STATE_CUSTOM)) { + String custom = name.split(DELIMITER)[0].split(CUSTOM_DELIMITER)[1]; + response.uploadedClusterStateCustomMetadataMap.put( + custom, + new UploadedMetadataAttribute(custom, uploadedMetadata.getUploadedFilename()) + ); + } else if (RemoteCoordinationMetadata.COORDINATION_METADATA.equals(name)) { response.uploadedCoordinationMetadata = (UploadedMetadataAttribute) uploadedMetadata; - } else if (SETTING_METADATA.equals(name)) { + } else if (RemotePersistentSettingsMetadata.SETTING_METADATA.equals(name)) { response.uploadedSettingsMetadata = (UploadedMetadataAttribute) uploadedMetadata; - } else if (TEMPLATES_METADATA.equals(name)) { + } else if (RemoteTransientSettingsMetadata.TRANSIENT_SETTING_METADATA.equals(name)) { + response.uploadedTransientSettingsMetadata = (UploadedMetadataAttribute) uploadedMetadata; + } else if (RemoteTemplatesMetadata.TEMPLATES_METADATA.equals(name)) { response.uploadedTemplatesMetadata = (UploadedMetadataAttribute) uploadedMetadata; } else if (name.contains(UploadedIndexMetadata.COMPONENT_PREFIX)) { response.uploadedIndexMetadata.add((UploadedIndexMetadata) uploadedMetadata); + } else if (RemoteDiscoveryNodes.DISCOVERY_NODES.equals(uploadedMetadata.getComponent())) { + response.uploadedDiscoveryNodes = (UploadedMetadataAttribute) uploadedMetadata; + } else if (RemoteClusterBlocks.CLUSTER_BLOCKS.equals(uploadedMetadata.getComponent())) { + response.uploadedClusterBlocks = (UploadedMetadataAttribute) uploadedMetadata; + } else if (RemoteHashesOfConsistentSettings.HASHES_OF_CONSISTENT_SETTINGS.equals(uploadedMetadata.getComponent())) { + response.uploadedHashesOfConsistentSettings = (UploadedMetadataAttribute) uploadedMetadata; } else { - throw new IllegalStateException("Unknown metadata component name " + name); + throw new IllegalStateException("Unexpected metadata component " + uploadedMetadata.getComponent()); } }); + logger.info("response {}", response.uploadedIndicesRoutingMetadata.toString()); return response; } @@ -658,77 +795,12 @@ private ActionListener getIndexMetadataUploadActionListener( ); } - /** - * Allows async Upload of IndexMetadata to remote - * - * @param clusterState current ClusterState - * @param indexMetadata {@link IndexMetadata} to upload - * @param latchedActionListener listener to respond back on after upload finishes - */ - private CheckedRunnable getIndexMetadataAsyncAction( - ClusterState clusterState, - IndexMetadata indexMetadata, - LatchedActionListener latchedActionListener - ) { - final BlobContainer indexMetadataContainer = indexMetadataContainer( - clusterState.getClusterName().value(), - clusterState.metadata().clusterUUID(), - indexMetadata.getIndexUUID() - ); - final String indexMetadataFilename = indexMetadataFileName(indexMetadata); - ActionListener completionListener = ActionListener.wrap( - resp -> latchedActionListener.onResponse( - new UploadedIndexMetadata( - indexMetadata.getIndex().getName(), - indexMetadata.getIndexUUID(), - indexMetadataContainer.path().buildAsString() + indexMetadataFilename - ) - ), - ex -> latchedActionListener.onFailure(new RemoteStateTransferException(indexMetadata.getIndex().toString(), ex)) - ); - - return () -> INDEX_METADATA_FORMAT.writeAsyncWithUrgentPriority( - indexMetadata, - indexMetadataContainer, - indexMetadataFilename, - blobStoreRepository.getCompressor(), - completionListener, - FORMAT_PARAMS - ); - } - - /** - * Allows async upload of Metadata components to remote - */ - - private CheckedRunnable getAsyncMetadataWriteAction( - ClusterState clusterState, - String component, - ChecksumBlobStoreFormat componentMetadataBlobStore, - ToXContent componentMetadata, - LatchedActionListener latchedActionListener - ) { - final BlobContainer globalMetadataContainer = globalMetadataContainer( - clusterState.getClusterName().value(), - clusterState.metadata().clusterUUID() - ); - final String componentMetadataFilename = metadataAttributeFileName(component, clusterState.metadata().version()); - ActionListener completionListener = ActionListener.wrap( - resp -> latchedActionListener.onResponse(new UploadedMetadataAttribute(component, componentMetadataFilename)), - ex -> latchedActionListener.onFailure(new RemoteStateTransferException(component, ex)) - ); - return () -> componentMetadataBlobStore.writeAsyncWithUrgentPriority( - componentMetadata, - globalMetadataContainer, - componentMetadataFilename, - blobStoreRepository.getCompressor(), - completionListener, - FORMAT_PARAMS - ); + public RemoteManifestManager getRemoteManifestManager() { + return remoteManifestManager; } @Nullable - public ClusterMetadataManifest markLastStateAsCommitted(ClusterState clusterState, ClusterMetadataManifest previousManifest) + public RemoteUploadDetails markLastStateAsCommitted(ClusterState clusterState, ClusterMetadataManifest previousManifest) throws IOException { assert clusterState != null : "Last accepted cluster state is not set"; if (clusterState.nodes().isLocalNodeElectedClusterManager() == false) { @@ -736,22 +808,56 @@ public ClusterMetadataManifest markLastStateAsCommitted(ClusterState clusterStat return null; } assert previousManifest != null : "Last cluster metadata manifest is not set"; - ClusterMetadataManifest committedManifest = uploadManifest( - clusterState, + UploadedMetadataResults uploadedMetadataResults = new UploadedMetadataResults( previousManifest.getIndices(), - previousManifest.getPreviousClusterUUID(), + previousManifest.getCustomMetadataMap(), previousManifest.getCoordinationMetadata(), previousManifest.getSettingsMetadata(), + previousManifest.getTransientSettingsMetadata(), previousManifest.getTemplatesMetadata(), - previousManifest.getCustomMetadataMap(), + previousManifest.getDiscoverNodeMetadata(), + previousManifest.getClusterBlocksMetadata(), + previousManifest.getIndicesRouting(), + previousManifest.getHashesOfConsistentSettings(), + previousManifest.getClusterStateCustomMap() + ); + + RemoteUploadDetails committedManifestDetails = remoteManifestManager.uploadManifest( + clusterState, + uploadedMetadataResults, + previousManifest.getPreviousClusterUUID(), + previousManifest.getDiffManifest(), true ); - deleteStaleClusterUUIDs(clusterState, committedManifest); - return committedManifest; + if (!previousManifest.isClusterUUIDCommitted() && committedManifestDetails.getClusterMetadataManifest().isClusterUUIDCommitted()) { + remoteClusterStateCleanupManager.deleteStaleClusterUUIDs(clusterState, committedManifestDetails.getClusterMetadataManifest()); + } + + return committedManifestDetails; + } + + /** + * Fetch latest ClusterMetadataManifest from remote state store + * + * @param clusterUUID uuid of cluster state to refer to in remote + * @param clusterName name of the cluster + * @return ClusterMetadataManifest + */ + public Optional getLatestClusterMetadataManifest(String clusterName, String clusterUUID) { + return remoteManifestManager.getLatestClusterMetadataManifest(clusterName, clusterUUID); + } + + public Optional getClusterMetadataManifestByTermVersion(String clusterName, String clusterUUID, long term, long version) { + return remoteManifestManager.getClusterMetadataManifestByTermVersion(clusterName, clusterUUID, term, version); + } + + public ClusterMetadataManifest getClusterMetadataManifestByFileName(String clusterUUID, String fileName) { + return remoteManifestManager.getRemoteClusterMetadataManifestByFileName(clusterUUID, fileName); } @Override public void close() throws IOException { + remoteClusterStateCleanupManager.close(); if (blobStoreRepository != null) { IOUtils.close(blobStoreRepository); } @@ -770,461 +876,482 @@ public void start() { assert repository instanceof BlobStoreRepository : "Repository should be instance of BlobStoreRepository"; blobStoreRepository = (BlobStoreRepository) repository; this.remoteRoutingTableService.ifPresent(RemoteRoutingTableService::start); + String clusterName = ClusterName.CLUSTER_NAME_SETTING.get(settings).value(); + RemoteWritableEntityStore globalMetadataBlobStore = new RemoteClusterStateBlobStore<>(getBlobStoreTransferService(), + blobStoreRepository, clusterName, threadpool, ThreadPool.Names.GENERIC); + RemoteClusterStateBlobStore coordinationMetadataBlobStore = new RemoteClusterStateBlobStore<>( + getBlobStoreTransferService(), blobStoreRepository, clusterName, threadpool, ThreadPool.Names.GENERIC); + RemoteClusterStateBlobStore transientSettingsBlobStore = new RemoteClusterStateBlobStore<>( + getBlobStoreTransferService(), blobStoreRepository, clusterName, threadpool, ThreadPool.Names.GENERIC); + RemoteClusterStateBlobStore persistentSettingsBlobStore = new RemoteClusterStateBlobStore<>( + getBlobStoreTransferService(), blobStoreRepository, clusterName, threadpool, ThreadPool.Names.GENERIC); + RemoteClusterStateBlobStore templatesMetadataBlobStore = new RemoteClusterStateBlobStore<>( + getBlobStoreTransferService(), blobStoreRepository, clusterName, threadpool, ThreadPool.Names.GENERIC); + RemoteClusterStateBlobStore customMetadataBlobStore = new RemoteClusterStateBlobStore<>(getBlobStoreTransferService(), + blobStoreRepository, clusterName, threadpool, ThreadPool.Names.GENERIC); + RemoteClusterStateBlobStore hashesOfConsistentSettingsBlobStore = + new RemoteClusterStateBlobStore<>( + getBlobStoreTransferService(), blobStoreRepository, clusterName, threadpool, ThreadPool.Names.GENERIC); + remoteGlobalMetadataManager = new RemoteGlobalMetadataManager(clusterSettings, globalMetadataBlobStore, coordinationMetadataBlobStore, + transientSettingsBlobStore, persistentSettingsBlobStore, templatesMetadataBlobStore, customMetadataBlobStore, hashesOfConsistentSettingsBlobStore, + blobStoreRepository.getCompressor(), blobStoreRepository.getNamedXContentRegistry()); + RemoteClusterStateBlobStore indexMetadataBlobStore = new RemoteClusterStateBlobStore<>( + getBlobStoreTransferService(), blobStoreRepository, clusterName, threadpool, ThreadPool.Names.GENERIC); + remoteIndexMetadataManager = new RemoteIndexMetadataManager(indexMetadataBlobStore, clusterSettings, blobStoreRepository.getCompressor(), + blobStoreRepository.getNamedXContentRegistry()); + RemoteClusterStateBlobStore clusterBlocksBlobStore = new RemoteClusterStateBlobStore<>( + getBlobStoreTransferService(), blobStoreRepository, clusterName, threadpool, ThreadPool.Names.GENERIC); + RemoteClusterStateBlobStore discoveryNodesBlobStore = new RemoteClusterStateBlobStore<>( + getBlobStoreTransferService(), blobStoreRepository, clusterName, threadpool, ThreadPool.Names.GENERIC); + RemoteClusterStateBlobStore clusterStateCustomsBlobStore = new RemoteClusterStateBlobStore<>( + getBlobStoreTransferService(), blobStoreRepository, clusterName, threadpool, ThreadPool.Names.GENERIC); + remoteClusterStateAttributesManager = new RemoteClusterStateAttributesManager(clusterBlocksBlobStore, discoveryNodesBlobStore, + clusterStateCustomsBlobStore, blobStoreRepository.getCompressor(), blobStoreRepository.getNamedXContentRegistry(), namedWriteableRegistry); + RemoteClusterStateBlobStore manifestBlobStore = new RemoteClusterStateBlobStore<>( + getBlobStoreTransferService(), blobStoreRepository, clusterName, threadpool, ThreadPool.Names.GENERIC); + remoteManifestManager = new RemoteManifestManager(manifestBlobStore, clusterSettings, nodeId, blobStoreRepository.getCompressor(), + blobStoreRepository.getNamedXContentRegistry(), blobStoreRepository); + remoteClusterStateCleanupManager.start(); } - private ClusterMetadataManifest uploadManifest( - ClusterState clusterState, - List uploadedIndexMetadata, - String previousClusterUUID, - UploadedMetadataAttribute uploadedCoordinationMetadata, - UploadedMetadataAttribute uploadedSettingsMetadata, - UploadedMetadataAttribute uploadedTemplatesMetadata, - Map uploadedCustomMetadataMap, - boolean committed - ) throws IOException { - synchronized (this) { - final String manifestFileName = getManifestFileName( - clusterState.term(), - clusterState.version(), - committed, - MANIFEST_CURRENT_CODEC_VERSION - ); - final ClusterMetadataManifest manifest = new ClusterMetadataManifest( - clusterState.term(), - clusterState.getVersion(), - clusterState.metadata().clusterUUID(), - clusterState.stateUUID(), - Version.CURRENT, - nodeId, - committed, - MANIFEST_CURRENT_CODEC_VERSION, - null, - uploadedIndexMetadata, - previousClusterUUID, - clusterState.metadata().clusterUUIDCommitted(), - uploadedCoordinationMetadata, - uploadedSettingsMetadata, - uploadedTemplatesMetadata, - uploadedCustomMetadataMap - ); - writeMetadataManifest(clusterState.getClusterName().value(), clusterState.metadata().clusterUUID(), manifest, manifestFileName); - return manifest; - } + private void setSlowWriteLoggingThreshold(TimeValue slowWriteLoggingThreshold) { + this.slowWriteLoggingThreshold = slowWriteLoggingThreshold; } - private void writeMetadataManifest(String clusterName, String clusterUUID, ClusterMetadataManifest uploadManifest, String fileName) - throws IOException { - AtomicReference result = new AtomicReference(); - AtomicReference exceptionReference = new AtomicReference(); + // Package private for unit test + Optional getRemoteRoutingTableService() { + return this.remoteRoutingTableService; + } - final BlobContainer metadataManifestContainer = manifestContainer(clusterName, clusterUUID); + ThreadPool getThreadpool() { + return threadpool; + } - // latch to wait until upload is not finished - CountDownLatch latch = new CountDownLatch(1); + BlobStoreRepository getBlobStoreRepository() { + return blobStoreRepository; + } - LatchedActionListener completionListener = new LatchedActionListener<>(ActionListener.wrap(resp -> { - logger.trace(String.format(Locale.ROOT, "Manifest file uploaded successfully.")); - }, ex -> { exceptionReference.set(ex); }), latch); + BlobStore getBlobStore() { + return blobStoreRepository.blobStore(); + } - getClusterMetadataManifestBlobStoreFormat(fileName).writeAsyncWithUrgentPriority( - uploadManifest, - metadataManifestContainer, - fileName, - blobStoreRepository.getCompressor(), - completionListener, - FORMAT_PARAMS + /** + * Fetch latest ClusterState from remote, including global metadata, index metadata and cluster state version + * + * @param clusterUUID uuid of cluster state to refer to in remote + * @param clusterName name of the cluster + * @return {@link IndexMetadata} + */ + public ClusterState getLatestClusterState(String clusterName, String clusterUUID, boolean includeEphemeral) throws IOException { + Optional clusterMetadataManifest = remoteManifestManager.getLatestClusterMetadataManifest( + clusterName, + clusterUUID ); - - try { - if (latch.await(getMetadataManifestUploadTimeout().millis(), TimeUnit.MILLISECONDS) == false) { - RemoteStateTransferException ex = new RemoteStateTransferException( - String.format(Locale.ROOT, "Timed out waiting for transfer of manifest file to complete") - ); - throw ex; - } - } catch (InterruptedException ex) { - RemoteStateTransferException exception = new RemoteStateTransferException( - String.format(Locale.ROOT, "Timed out waiting for transfer of manifest file to complete - %s"), - ex + if (clusterMetadataManifest.isEmpty()) { + throw new IllegalStateException( + String.format(Locale.ROOT, "Latest cluster metadata manifest is not present for the provided clusterUUID: %s", clusterUUID) ); - Thread.currentThread().interrupt(); - throw exception; } - if (exceptionReference.get() != null) { - throw new RemoteStateTransferException(exceptionReference.get().getMessage(), exceptionReference.get()); - } - logger.debug( - "Metadata manifest file [{}] written during [{}] phase. ", - fileName, - uploadManifest.isCommitted() ? "commit" : "publish" - ); - } - - private BlobContainer indexMetadataContainer(String clusterName, String clusterUUID, String indexUUID) { - // 123456789012_test-cluster/cluster-state/dsgYj10Nkso7/index/ftqsCnn9TgOX - return blobStoreRepository.blobStore() - .blobContainer(getCusterMetadataBasePath(clusterName, clusterUUID).add(INDEX_PATH_TOKEN).add(indexUUID)); - } - private BlobContainer globalMetadataContainer(String clusterName, String clusterUUID) { - // 123456789012_test-cluster/cluster-state/dsgYj10Nkso7/global-metadata/ - return blobStoreRepository.blobStore() - .blobContainer(getCusterMetadataBasePath(clusterName, clusterUUID).add(GLOBAL_METADATA_PATH_TOKEN)); + return getClusterStateForManifest(clusterName, clusterMetadataManifest.get(), nodeId, includeEphemeral); } - private BlobContainer manifestContainer(String clusterName, String clusterUUID) { - // 123456789012_test-cluster/cluster-state/dsgYj10Nkso7/manifest - return blobStoreRepository.blobStore().blobContainer(getManifestFolderPath(clusterName, clusterUUID)); - } - - private BlobPath getCusterMetadataBasePath(String clusterName, String clusterUUID) { - return blobStoreRepository.basePath().add(encodeString(clusterName)).add(CLUSTER_STATE_PATH_TOKEN).add(clusterUUID); - } + private ClusterState readClusterStateInParallel( + ClusterState previousState, + ClusterMetadataManifest manifest, + String clusterUUID, + String localNodeId, + List indicesToRead, + Map customToRead, + boolean readCoordinationMetadata, + boolean readSettingsMetadata, + boolean readTransientSettingsMetadata, + boolean readTemplatesMetadata, + boolean readDiscoveryNodes, + boolean readClusterBlocks, + List indicesRoutingToRead, + boolean readHashesOfConsistentSettings, + Map clusterStateCustomToRead + ) throws IOException { + int totalReadTasks = + indicesToRead.size() + customToRead.size() + indicesRoutingToRead.size() + (readCoordinationMetadata ? 1 : 0) + (readSettingsMetadata ? 1 : 0) + ( + readTemplatesMetadata ? 1 : 0) + (readDiscoveryNodes ? 1 : 0) + (readClusterBlocks ? 1 : 0) + (readTransientSettingsMetadata ? 1 : 0) + (readHashesOfConsistentSettings ? 1 : 0) + + clusterStateCustomToRead.size(); + CountDownLatch latch = new CountDownLatch(totalReadTasks); + List> asyncMetadataReadActions = new ArrayList<>(); + List readResults = Collections.synchronizedList(new ArrayList<>()); + List readIndexRoutingTableResults = Collections.synchronizedList(new ArrayList<>()); + List exceptionList = Collections.synchronizedList(new ArrayList<>(totalReadTasks)); + + LatchedActionListener listener = new LatchedActionListener<>( + ActionListener.wrap( + response -> { + logger.debug("Successfully read cluster state component from remote"); + readResults.add(response); + }, + ex -> { + logger.error("Failed to read cluster state from remote", ex); + exceptionList.add(ex); + } + ), + latch + ); - private BlobContainer clusterUUIDContainer(String clusterName) { - return blobStoreRepository.blobStore() - .blobContainer( - blobStoreRepository.basePath() - .add(Base64.getUrlEncoder().withoutPadding().encodeToString(clusterName.getBytes(StandardCharsets.UTF_8))) - .add(CLUSTER_STATE_PATH_TOKEN) + for (UploadedIndexMetadata indexMetadata : indicesToRead) { + asyncMetadataReadActions.add( + remoteIndexMetadataManager.getAsyncIndexMetadataReadAction( + clusterUUID, + indexMetadata.getUploadedFilename(), + listener + ) ); - } - - private void setSlowWriteLoggingThreshold(TimeValue slowWriteLoggingThreshold) { - this.slowWriteLoggingThreshold = slowWriteLoggingThreshold; - } - - private void setIndexMetadataUploadTimeout(TimeValue newIndexMetadataUploadTimeout) { - this.indexMetadataUploadTimeout = newIndexMetadataUploadTimeout; - } - - private void setGlobalMetadataUploadTimeout(TimeValue newGlobalMetadataUploadTimeout) { - this.globalMetadataUploadTimeout = newGlobalMetadataUploadTimeout; - } - - private void setMetadataManifestUploadTimeout(TimeValue newMetadataManifestUploadTimeout) { - this.metadataManifestUploadTimeout = newMetadataManifestUploadTimeout; - } - - private Map getUpdatedCustoms(ClusterState currentState, ClusterState previousState) { - if (Metadata.isCustomMetadataEqual(previousState.metadata(), currentState.metadata())) { - return new HashMap<>(); - } - Map updatedCustom = new HashMap<>(); - Set currentCustoms = new HashSet<>(currentState.metadata().customs().keySet()); - for (Map.Entry cursor : previousState.metadata().customs().entrySet()) { - if (cursor.getValue().context().contains(Metadata.XContentContext.GATEWAY)) { - if (currentCustoms.contains(cursor.getKey()) - && !cursor.getValue().equals(currentState.metadata().custom(cursor.getKey()))) { - // If the custom metadata is updated, we need to upload the new version. - updatedCustom.put(cursor.getKey(), currentState.metadata().custom(cursor.getKey())); - } - currentCustoms.remove(cursor.getKey()); - } - } - for (String custom : currentCustoms) { - Metadata.Custom cursor = currentState.metadata().custom(custom); - if (cursor.context().contains(Metadata.XContentContext.GATEWAY)) { - updatedCustom.put(custom, cursor); - } } - return updatedCustom; - } - public TimeValue getIndexMetadataUploadTimeout() { - return this.indexMetadataUploadTimeout; - } - - public TimeValue getGlobalMetadataUploadTimeout() { - return this.globalMetadataUploadTimeout; - } - - public TimeValue getMetadataManifestUploadTimeout() { - return this.metadataManifestUploadTimeout; - } + LatchedActionListener routingTableLatchedActionListener = new LatchedActionListener<>( + ActionListener.wrap( + response -> { + logger.debug("Successfully read cluster state component from remote"); + readIndexRoutingTableResults.add(response); + }, + ex -> { + logger.error("Failed to read cluster state from remote", ex); + exceptionList.add(ex); + } + ), + latch + ); - // Package private for unit test - Optional getRemoteRoutingTableService() { - return this.remoteRoutingTableService; - } + for (UploadedIndexMetadata indexRouting : indicesRoutingToRead) { + asyncMetadataReadActions.add( + remoteRoutingTableService.get().getAsyncIndexMetadataReadAction( + indexRouting.getUploadedFilename(), + new Index(indexRouting.getIndexName(), indexRouting.getIndexUUID()), + routingTableLatchedActionListener + ) + ); + } - static String getManifestFileName(long term, long version, boolean committed, int codecVersion) { - // 123456789012_test-cluster/cluster-state/dsgYj10Nkso7/manifest/manifest______C/P____ - return String.join( - DELIMITER, - MANIFEST_PATH_TOKEN, - RemoteStoreUtils.invertLong(term), - RemoteStoreUtils.invertLong(version), - (committed ? "C" : "P"), // C for committed and P for published - RemoteStoreUtils.invertLong(System.currentTimeMillis()), - String.valueOf(codecVersion) // Keep the codec version at last place only, during read we reads last place to - // determine codec version. - ); - } + for (Map.Entry entry : customToRead.entrySet()) { + asyncMetadataReadActions.add( + remoteGlobalMetadataManager.getAsyncMetadataReadAction( + clusterUUID, + CUSTOM_METADATA, + entry.getKey(), + entry.getValue().getUploadedFilename(), + listener + ) + ); + } - static String indexMetadataFileName(IndexMetadata indexMetadata) { - // 123456789012_test-cluster/cluster-state/dsgYj10Nkso7/index//metadata______ - return String.join( - DELIMITER, - METADATA_FILE_PREFIX, - RemoteStoreUtils.invertLong(indexMetadata.getVersion()), - RemoteStoreUtils.invertLong(System.currentTimeMillis()), - String.valueOf(INDEX_METADATA_CURRENT_CODEC_VERSION) // Keep the codec version at last place only, during read we reads last - // place to determine codec version. - ); - } + if (readCoordinationMetadata) { + asyncMetadataReadActions.add( + remoteGlobalMetadataManager.getAsyncMetadataReadAction( + clusterUUID, + COORDINATION_METADATA, + COORDINATION_METADATA, + manifest.getCoordinationMetadata().getUploadedFilename(), + listener + ) + ); + } - private static String globalMetadataFileName(Metadata metadata) { - // 123456789012_test-cluster/cluster-state/dsgYj10Nkso7/global-metadata/metadata______ - return String.join( - DELIMITER, - METADATA_FILE_PREFIX, - RemoteStoreUtils.invertLong(metadata.version()), - RemoteStoreUtils.invertLong(System.currentTimeMillis()), - String.valueOf(GLOBAL_METADATA_CURRENT_CODEC_VERSION) - ); - } + if (readSettingsMetadata) { + asyncMetadataReadActions.add( + remoteGlobalMetadataManager.getAsyncMetadataReadAction( + clusterUUID, + SETTING_METADATA, + SETTING_METADATA, + manifest.getSettingsMetadata().getUploadedFilename(), + listener + ) + ); + } - private static String metadataAttributeFileName(String componentPrefix, Long metadataVersion) { - // 123456789012_test-cluster/cluster-state/dsgYj10Nkso7/global-metadata/______ - return String.join( - DELIMITER, - componentPrefix, - RemoteStoreUtils.invertLong(metadataVersion), - RemoteStoreUtils.invertLong(System.currentTimeMillis()), - String.valueOf(GLOBAL_METADATA_CURRENT_CODEC_VERSION) - ); - } + if (readTransientSettingsMetadata) { + asyncMetadataReadActions.add( + remoteGlobalMetadataManager.getAsyncMetadataReadAction( + clusterUUID, + TRANSIENT_SETTING_METADATA, + TRANSIENT_SETTING_METADATA, + manifest.getTransientSettingsMetadata().getUploadedFilename(), + listener + ) + ); + } - private BlobPath getManifestFolderPath(String clusterName, String clusterUUID) { - return getCusterMetadataBasePath(clusterName, clusterUUID).add(MANIFEST_PATH_TOKEN); - } + if (readTemplatesMetadata) { + asyncMetadataReadActions.add( + remoteGlobalMetadataManager.getAsyncMetadataReadAction( + clusterUUID, + TEMPLATES_METADATA, + TEMPLATES_METADATA, + manifest.getTemplatesMetadata().getUploadedFilename(), + listener + ) + ); + } - /** - * Fetch latest index metadata from remote cluster state - * - * @param clusterUUID uuid of cluster state to refer to in remote - * @param clusterName name of the cluster - * @param clusterMetadataManifest manifest file of cluster - * @return {@code Map} latest IndexUUID to IndexMetadata map - */ - private Map getIndexMetadataMap( - String clusterName, - String clusterUUID, - ClusterMetadataManifest clusterMetadataManifest - ) { - assert Objects.equals(clusterUUID, clusterMetadataManifest.getClusterUUID()) - : "Corrupt ClusterMetadataManifest found. Cluster UUID mismatch."; - Map remoteIndexMetadata = new HashMap<>(); - for (UploadedIndexMetadata uploadedIndexMetadata : clusterMetadataManifest.getIndices()) { - IndexMetadata indexMetadata = getIndexMetadata(clusterName, clusterUUID, uploadedIndexMetadata); - remoteIndexMetadata.put(uploadedIndexMetadata.getIndexUUID(), indexMetadata); - } - return remoteIndexMetadata; - } + if (readDiscoveryNodes) { + asyncMetadataReadActions.add( + remoteClusterStateAttributesManager.getAsyncMetadataReadAction( + clusterUUID, + DISCOVERY_NODES, + DISCOVERY_NODES, + manifest.getDiscoveryNodesMetadata().getUploadedFilename(), + listener + ) + ); + } - /** - * Fetch index metadata from remote cluster state - * - * @param clusterUUID uuid of cluster state to refer to in remote - * @param clusterName name of the cluster - * @param uploadedIndexMetadata {@link UploadedIndexMetadata} contains details about remote location of index metadata - * @return {@link IndexMetadata} - */ - private IndexMetadata getIndexMetadata(String clusterName, String clusterUUID, UploadedIndexMetadata uploadedIndexMetadata) { - BlobContainer blobContainer = indexMetadataContainer(clusterName, clusterUUID, uploadedIndexMetadata.getIndexUUID()); - try { - String[] splitPath = uploadedIndexMetadata.getUploadedFilename().split("/"); - return INDEX_METADATA_FORMAT.read( - blobContainer, - splitPath[splitPath.length - 1], - blobStoreRepository.getNamedXContentRegistry() + if (readClusterBlocks) { + asyncMetadataReadActions.add( + remoteClusterStateAttributesManager.getAsyncMetadataReadAction( + clusterUUID, + CLUSTER_BLOCKS, + CLUSTER_BLOCKS, + manifest.getClusterBlocksMetadata().getUploadedFilename(), + listener + ) ); - } catch (IOException e) { - throw new IllegalStateException( - String.format(Locale.ROOT, "Error while downloading IndexMetadata - %s", uploadedIndexMetadata.getUploadedFilename()), - e + } + + if (readHashesOfConsistentSettings) { + asyncMetadataReadActions.add( + remoteGlobalMetadataManager.getAsyncMetadataReadAction( + clusterUUID, + HASHES_OF_CONSISTENT_SETTINGS, + HASHES_OF_CONSISTENT_SETTINGS, + manifest.getHashesOfConsistentSettings().getUploadedFilename(), + listener + ) ); } - } - /** - * Fetch latest ClusterState from remote, including global metadata, index metadata and cluster state version - * - * @param clusterUUID uuid of cluster state to refer to in remote - * @param clusterName name of the cluster - * @return {@link IndexMetadata} - */ - public ClusterState getLatestClusterState(String clusterName, String clusterUUID) { - Optional clusterMetadataManifest = getLatestClusterMetadataManifest(clusterName, clusterUUID); - if (clusterMetadataManifest.isEmpty()) { - throw new IllegalStateException( - String.format(Locale.ROOT, "Latest cluster metadata manifest is not present for the provided clusterUUID: %s", clusterUUID) + for (Map.Entry entry : clusterStateCustomToRead.entrySet()) { + asyncMetadataReadActions.add( + remoteClusterStateAttributesManager.getAsyncMetadataReadAction( + clusterUUID, + CLUSTER_STATE_CUSTOM, + entry.getKey(), + entry.getValue().getUploadedFilename(), + listener + ) ); } - // Fetch Global Metadata - Metadata globalMetadata = getGlobalMetadata(clusterName, clusterUUID, clusterMetadataManifest.get()); + for (CheckedRunnable asyncMetadataReadAction : asyncMetadataReadActions) { + asyncMetadataReadAction.run(); + } - // Fetch Index Metadata - Map indices = getIndexMetadataMap(clusterName, clusterUUID, clusterMetadataManifest.get()); + try { + if (latch.await(this.remoteStateReadTimeout.getMillis(), TimeUnit.MILLISECONDS) == false) { + RemoteStateTransferException exception = new RemoteStateTransferException( + "Timed out waiting to read cluster state from remote within timeout " + this.remoteStateReadTimeout + ); + exceptionList.forEach(exception::addSuppressed); + throw exception; + } + } catch (InterruptedException e) { + exceptionList.forEach(e::addSuppressed); + RemoteStateTransferException ex = new RemoteStateTransferException("Interrupted while waiting to read cluster state from metadata"); + Thread.currentThread().interrupt(); + throw ex; + } + if (!exceptionList.isEmpty()) { + RemoteStateTransferException exception = new RemoteStateTransferException("Exception during reading cluster state from remote"); + exceptionList.forEach(exception::addSuppressed); + throw exception; + } + + ClusterState.Builder clusterStateBuilder = ClusterState.builder(previousState); + AtomicReference discoveryNodesBuilder = new AtomicReference<>(DiscoveryNodes.builder()); + Metadata.Builder metadataBuilder = Metadata.builder(previousState.metadata()); + metadataBuilder.version(manifest.getMetadataVersion()); + metadataBuilder.clusterUUID(manifest.getClusterUUID()); + metadataBuilder.clusterUUIDCommitted(manifest.isClusterUUIDCommitted()); Map indexMetadataMap = new HashMap<>(); - indices.values().forEach(indexMetadata -> { indexMetadataMap.put(indexMetadata.getIndex().getName(), indexMetadata); }); + Map indicesRouting = new HashMap<>(previousState.routingTable().getIndicesRouting()); + + readResults.forEach(remoteReadResult -> { + switch (remoteReadResult.getComponent()) { + case INDEX_PATH_TOKEN: + IndexMetadata indexMetadata = (IndexMetadata) remoteReadResult.getObj(); + indexMetadataMap.put(indexMetadata.getIndex().getName(), indexMetadata); + break; + case CUSTOM_METADATA: + metadataBuilder.putCustom(remoteReadResult.getComponentName(), (Metadata.Custom) remoteReadResult.getObj()); + break; + case COORDINATION_METADATA: + metadataBuilder.coordinationMetadata((CoordinationMetadata) remoteReadResult.getObj()); + break; + case SETTING_METADATA: + metadataBuilder.persistentSettings((Settings) remoteReadResult.getObj()); + break; + case TRANSIENT_SETTING_METADATA: + metadataBuilder.transientSettings((Settings) remoteReadResult.getObj()); + break; + case TEMPLATES_METADATA: + metadataBuilder.templates((TemplatesMetadata) remoteReadResult.getObj()); + break; + case HASHES_OF_CONSISTENT_SETTINGS: + metadataBuilder.hashesOfConsistentSettings((DiffableStringMap) remoteReadResult.getObj()); + break; + case CLUSTER_STATE_ATTRIBUTE: + if (remoteReadResult.getComponentName().equals(DISCOVERY_NODES)) { + discoveryNodesBuilder.set(DiscoveryNodes.builder((DiscoveryNodes) remoteReadResult.getObj())); + } else if (remoteReadResult.getComponentName().equals(CLUSTER_BLOCKS)) { + clusterStateBuilder.blocks((ClusterBlocks) remoteReadResult.getObj()); + } else if (remoteReadResult.getComponentName().startsWith(CLUSTER_STATE_CUSTOM)) { + // component name for mat is "cluster-state-custom--custom_name" + String custom = remoteReadResult.getComponentName().split(CUSTOM_DELIMITER)[1]; + clusterStateBuilder.putCustom(custom, (ClusterState.Custom) remoteReadResult.getObj()); + } + break; + default: + throw new IllegalStateException("Unknown component: " + remoteReadResult.getComponent()); + } + }); + + readIndexRoutingTableResults.forEach(indexRoutingTable -> { + indicesRouting.put(indexRoutingTable.getIndex().getName(), indexRoutingTable); + }); - return ClusterState.builder(ClusterState.EMPTY_STATE) - .version(clusterMetadataManifest.get().getStateVersion()) - .metadata(Metadata.builder(globalMetadata).indices(indexMetadataMap).build()) + metadataBuilder.indices(indexMetadataMap); + if (readDiscoveryNodes) { + clusterStateBuilder.nodes(discoveryNodesBuilder.get().localNodeId(localNodeId)); + } + return clusterStateBuilder.metadata(metadataBuilder) + .version(manifest.getStateVersion()) + .stateUUID(manifest.getStateUUID()) + .routingTable(new RoutingTable(manifest.getRoutingTableVersion(), indicesRouting)) .build(); } - private Metadata getGlobalMetadata(String clusterName, String clusterUUID, ClusterMetadataManifest clusterMetadataManifest) { - String globalMetadataFileName = clusterMetadataManifest.getGlobalMetadataFileName(); - try { - // Fetch Global metadata - if (globalMetadataFileName != null) { - String[] splitPath = globalMetadataFileName.split("/"); - return GLOBAL_METADATA_FORMAT.read( - globalMetadataContainer(clusterName, clusterUUID), - splitPath[splitPath.length - 1], - blobStoreRepository.getNamedXContentRegistry() - ); - } else if (clusterMetadataManifest.hasMetadataAttributesFiles()) { - CoordinationMetadata coordinationMetadata = getCoordinationMetadata( - clusterName, - clusterUUID, - clusterMetadataManifest.getCoordinationMetadata().getUploadedFilename() - ); - Settings settingsMetadata = getSettingsMetadata( - clusterName, - clusterUUID, - clusterMetadataManifest.getSettingsMetadata().getUploadedFilename() - ); - TemplatesMetadata templatesMetadata = getTemplatesMetadata( - clusterName, - clusterUUID, - clusterMetadataManifest.getTemplatesMetadata().getUploadedFilename() - ); - Metadata.Builder builder = new Metadata.Builder(); - builder.coordinationMetadata(coordinationMetadata); - builder.persistentSettings(settingsMetadata); - builder.templates(templatesMetadata); - clusterMetadataManifest.getCustomMetadataMap() - .forEach( - (key, value) -> builder.putCustom( - key, - getCustomsMetadata(clusterName, clusterUUID, value.getUploadedFilename(), key) - ) - ); - return builder.build(); - } else { - return Metadata.EMPTY_METADATA; - } - } catch (IOException e) { - throw new IllegalStateException( - String.format(Locale.ROOT, "Error while downloading Global Metadata - %s", globalMetadataFileName), - e + public ClusterState getClusterStateForManifest(String clusterName, ClusterMetadataManifest manifest, String localNodeId, boolean includeEphemeral) + throws IOException { + if (manifest.onOrAfterCodecVersion(CODEC_V2)) { + return readClusterStateInParallel( + ClusterState.builder(new ClusterName(clusterName)).build(), + manifest, + manifest.getClusterUUID(), + localNodeId, + manifest.getIndices(), + manifest.getCustomMetadataMap(), + manifest.getCoordinationMetadata() != null, + manifest.getSettingsMetadata() != null, + manifest.getTransientSettingsMetadata() != null, + manifest.getTemplatesMetadata() != null, + includeEphemeral && manifest.getDiscoveryNodesMetadata() != null, + includeEphemeral && manifest.getClusterBlocksMetadata() != null, + includeEphemeral ? manifest.getIndicesRouting() : emptyList(), + includeEphemeral && manifest.getHashesOfConsistentSettings() != null, + includeEphemeral ? manifest.getClusterStateCustomMap() : emptyMap() ); + } else { + ClusterState clusterState = readClusterStateInParallel( + ClusterState.builder(new ClusterName(clusterName)).build(), + manifest, + manifest.getClusterUUID(), + localNodeId, + manifest.getIndices(), + // for manifest codec V1, we don't have the following objects to read, so not passing anything + emptyMap(), + false, + false, + false, + false, + false, + false, + emptyList(), + false, + emptyMap() + ); + Metadata.Builder mb = Metadata.builder(remoteGlobalMetadataManager.getGlobalMetadata(manifest.getClusterUUID(), manifest)); + mb.indices(clusterState.metadata().indices()); + return ClusterState.builder(clusterState).metadata(mb).build(); } + } - private CoordinationMetadata getCoordinationMetadata(String clusterName, String clusterUUID, String coordinationMetadataFileName) { - try { - // Fetch Coordination metadata - if (coordinationMetadataFileName != null) { - String[] splitPath = coordinationMetadataFileName.split("/"); - return COORDINATION_METADATA_FORMAT.read( - globalMetadataContainer(clusterName, clusterUUID), - splitPath[splitPath.length - 1], - blobStoreRepository.getNamedXContentRegistry() - ); - } else { - return CoordinationMetadata.EMPTY_METADATA; + public ClusterState getClusterStateUsingDiff(String clusterName, ClusterMetadataManifest manifest, ClusterState previousState, String localNodeId) + throws IOException { + assert manifest.getDiffManifest() != null; + ClusterStateDiffManifest diff = manifest.getDiffManifest(); + List updatedIndices = diff.getIndicesUpdated().stream().map(idx -> { + Optional uploadedIndexMetadataOptional = manifest.getIndices().stream().filter(idx2 -> idx2.getIndexName().equals(idx)) + .findFirst(); + assert uploadedIndexMetadataOptional.isPresent() == true; + return uploadedIndexMetadataOptional.get(); + }).collect(Collectors.toList()); + + List updatedIndexRouting = remoteRoutingTableService.get().getUpdatedIndexRoutingTableMetadata(diff.getIndicesRoutingUpdated(), + manifest.getIndicesRouting()); + + Map updatedCustomMetadata = new HashMap<>(); + if (diff.getCustomMetadataUpdated() != null) { + for (String customType : diff.getCustomMetadataUpdated()) { + updatedCustomMetadata.put(customType, manifest.getCustomMetadataMap().get(customType)); } - } catch (IOException e) { - throw new IllegalStateException( - String.format(Locale.ROOT, "Error while downloading Coordination Metadata - %s", coordinationMetadataFileName), - e - ); } - } - - private Settings getSettingsMetadata(String clusterName, String clusterUUID, String settingsMetadataFileName) { - try { - // Fetch Settings metadata - if (settingsMetadataFileName != null) { - String[] splitPath = settingsMetadataFileName.split("/"); - return SETTINGS_METADATA_FORMAT.read( - globalMetadataContainer(clusterName, clusterUUID), - splitPath[splitPath.length - 1], - blobStoreRepository.getNamedXContentRegistry() - ); - } else { - return Settings.EMPTY; + Map updatedClusterStateCustom = new HashMap<>(); + if (diff.getClusterStateCustomUpdated() != null) { + for (String customType : diff.getClusterStateCustomUpdated()) { + updatedClusterStateCustom.put(customType, manifest.getClusterStateCustomMap().get(customType)); + } + } + ClusterState updatedClusterState = readClusterStateInParallel( + previousState, + manifest, + manifest.getClusterUUID(), + localNodeId, + updatedIndices, + updatedCustomMetadata, + diff.isCoordinationMetadataUpdated(), + diff.isSettingsMetadataUpdated(), + diff.isTransientSettingsMetadataUpdated(), + diff.isTemplatesMetadataUpdated(), + diff.isDiscoveryNodesUpdated(), + diff.isClusterBlocksUpdated(), + updatedIndexRouting, + diff.isHashesOfConsistentSettingsUpdated(), + updatedClusterStateCustom + ); + ClusterState.Builder clusterStateBuilder = ClusterState.builder(updatedClusterState); + Metadata.Builder metadataBuilder = Metadata.builder(updatedClusterState.metadata()); + // remove the deleted indices from the metadata + for (String index : diff.getIndicesDeleted()) { + metadataBuilder.remove(index); + } + // remove the deleted metadata customs from the metadata + if (diff.getCustomMetadataDeleted() != null) { + for (String customType : diff.getCustomMetadataDeleted()) { + metadataBuilder.removeCustom(customType); } - } catch (IOException e) { - throw new IllegalStateException( - String.format(Locale.ROOT, "Error while downloading Settings Metadata - %s", settingsMetadataFileName), - e - ); } - } - private TemplatesMetadata getTemplatesMetadata(String clusterName, String clusterUUID, String templatesMetadataFileName) { - try { - // Fetch Templates metadata - if (templatesMetadataFileName != null) { - String[] splitPath = templatesMetadataFileName.split("/"); - return TEMPLATES_METADATA_FORMAT.read( - globalMetadataContainer(clusterName, clusterUUID), - splitPath[splitPath.length - 1], - blobStoreRepository.getNamedXContentRegistry() - ); - } else { - return TemplatesMetadata.EMPTY_METADATA; + // remove the deleted cluster state customs from the metadata + if (diff.getClusterStateCustomDeleted() != null) { + for (String customType : diff.getClusterStateCustomDeleted()) { + clusterStateBuilder.removeCustom(customType); } - } catch (IOException e) { - throw new IllegalStateException( - String.format(Locale.ROOT, "Error while downloading Templates Metadata - %s", templatesMetadataFileName), - e - ); } - } - private Metadata.Custom getCustomsMetadata(String clusterName, String clusterUUID, String customMetadataFileName, String custom) { - requireNonNull(customMetadataFileName); - try { - // Fetch Custom metadata - String[] splitPath = customMetadataFileName.split("/"); - ChecksumBlobStoreFormat customChecksumBlobStoreFormat = new ChecksumBlobStoreFormat<>( - "custom", - METADATA_NAME_FORMAT, - (parser -> Metadata.Custom.fromXContent(parser, custom)) - ); - return customChecksumBlobStoreFormat.read( - globalMetadataContainer(clusterName, clusterUUID), - splitPath[splitPath.length - 1], - blobStoreRepository.getNamedXContentRegistry() - ); - } catch (IOException e) { - throw new IllegalStateException( - String.format(Locale.ROOT, "Error while downloading Custom Metadata - %s", customMetadataFileName), - e - ); + HashMap indexRoutingTables = new HashMap<>(updatedClusterState.getRoutingTable().getIndicesRouting()); + + for (String indexName : diff.getIndicesRoutingDeleted()) { + indexRoutingTables.remove(indexName); } - } - /** - * Fetch latest ClusterMetadataManifest from remote state store - * - * @param clusterUUID uuid of cluster state to refer to in remote - * @param clusterName name of the cluster - * @return ClusterMetadataManifest - */ - public Optional getLatestClusterMetadataManifest(String clusterName, String clusterUUID) { - Optional latestManifestFileName = getLatestManifestFileName(clusterName, clusterUUID); - return latestManifestFileName.map(s -> fetchRemoteClusterMetadataManifest(clusterName, clusterUUID, s)); + RoutingTable routingTable = new RoutingTable(manifest.getRoutingTableVersion(), indexRoutingTables); + + return clusterStateBuilder. + stateUUID(manifest.getStateUUID()). + version(manifest.getStateVersion()). + metadata(metadataBuilder). + routingTable(routingTable). + build(); + } /** @@ -1236,7 +1363,10 @@ public Optional getLatestClusterMetadataManifest(String public String getLastKnownUUIDFromRemote(String clusterName) { try { Set clusterUUIDs = getAllClusterUUIDs(clusterName); - Map latestManifests = getLatestManifestForAllClusterUUIDs(clusterName, clusterUUIDs); + Map latestManifests = remoteManifestManager.getLatestManifestForAllClusterUUIDs( + clusterName, + clusterUUIDs + ); List validChain = createClusterChain(latestManifests, clusterName); if (validChain.isEmpty()) { return ClusterState.UNKNOWN_UUID; @@ -1250,8 +1380,24 @@ public String getLastKnownUUIDFromRemote(String clusterName) { } } - private Set getAllClusterUUIDs(String clusterName) throws IOException { - Map clusterUUIDMetadata = clusterUUIDContainer(clusterName).children(); + public void setRemoteClusterStateEnabled(TimeValue remoteStateReadTimeout) { + this.remoteStateReadTimeout = remoteStateReadTimeout; + } + + + public RemoteClusterStateCleanupManager getCleanupManager() { + return remoteClusterStateCleanupManager; + } + + private BlobStoreTransferService getBlobStoreTransferService() { + if (blobStoreTransferService == null) { + blobStoreTransferService = new BlobStoreTransferService(getBlobStore(), threadpool); + } + return blobStoreTransferService; + } + + Set getAllClusterUUIDs(String clusterName) throws IOException { + Map clusterUUIDMetadata = clusterUUIDContainer(blobStoreRepository, clusterName).children(); if (clusterUUIDMetadata == null) { return Collections.emptySet(); } @@ -1296,7 +1442,7 @@ private List createClusterChain(final Map 1) { logger.info("Top level cluster UUIDs: {}", topLevelClusterUUIDs); @@ -1352,7 +1498,7 @@ private Map trimClusterUUIDs( if (!ClusterState.UNKNOWN_UUID.equals(currentManifest.getPreviousClusterUUID())) { ClusterMetadataManifest previousManifest = trimmedUUIDs.get(currentManifest.getPreviousClusterUUID()); if (isMetadataEqual(currentManifest, previousManifest, clusterName) - && isGlobalMetadataEqual(currentManifest, previousManifest, clusterName)) { + && remoteGlobalMetadataManager.isGlobalMetadataEqual(currentManifest, previousManifest, clusterName)) { trimmedUUIDs.remove(clusterUUID); } } @@ -1367,14 +1513,22 @@ private boolean isMetadataEqual(ClusterMetadataManifest first, ClusterMetadataMa } final Map secondIndices = second.getIndices() .stream() - .collect(Collectors.toMap(md -> md.getIndexName(), Function.identity())); + .collect(Collectors.toMap(UploadedIndexMetadata::getIndexName, Function.identity())); for (UploadedIndexMetadata uploadedIndexMetadata : first.getIndices()) { - final IndexMetadata firstIndexMetadata = getIndexMetadata(clusterName, first.getClusterUUID(), uploadedIndexMetadata); + final IndexMetadata firstIndexMetadata = remoteIndexMetadataManager.getIndexMetadata( + uploadedIndexMetadata, + first.getClusterUUID(), + first.getCodecVersion() + ); final UploadedIndexMetadata secondUploadedIndexMetadata = secondIndices.get(uploadedIndexMetadata.getIndexName()); if (secondUploadedIndexMetadata == null) { return false; } - final IndexMetadata secondIndexMetadata = getIndexMetadata(clusterName, second.getClusterUUID(), secondUploadedIndexMetadata); + final IndexMetadata secondIndexMetadata = remoteIndexMetadataManager.getIndexMetadata( + secondUploadedIndexMetadata, + second.getClusterUUID(), + second.getCodecVersion() + ); if (firstIndexMetadata.equals(secondIndexMetadata) == false) { return false; } @@ -1382,107 +1536,10 @@ private boolean isMetadataEqual(ClusterMetadataManifest first, ClusterMetadataMa return true; } - private boolean isGlobalMetadataEqual(ClusterMetadataManifest first, ClusterMetadataManifest second, String clusterName) { - Metadata secondGlobalMetadata = getGlobalMetadata(clusterName, second.getClusterUUID(), second); - Metadata firstGlobalMetadata = getGlobalMetadata(clusterName, first.getClusterUUID(), first); - return Metadata.isGlobalResourcesMetadataEquals(firstGlobalMetadata, secondGlobalMetadata); - } - private boolean isValidClusterUUID(ClusterMetadataManifest manifest) { return manifest.isClusterUUIDCommitted(); } - /** - * Fetch ClusterMetadataManifest files from remote state store in order - * - * @param clusterUUID uuid of cluster state to refer to in remote - * @param clusterName name of the cluster - * @param limit max no of files to fetch - * @return all manifest file names - */ - private List getManifestFileNames(String clusterName, String clusterUUID, int limit) throws IllegalStateException { - try { - - /* - {@link BlobContainer#listBlobsByPrefixInSortedOrder} will list the latest manifest file first - as the manifest file name generated via {@link RemoteClusterStateService#getManifestFileName} ensures - when sorted in LEXICOGRAPHIC order the latest uploaded manifest file comes on top. - */ - return manifestContainer(clusterName, clusterUUID).listBlobsByPrefixInSortedOrder( - MANIFEST_FILE_PREFIX + DELIMITER, - limit, - BlobContainer.BlobNameSortOrder.LEXICOGRAPHIC - ); - } catch (IOException e) { - throw new IllegalStateException("Error while fetching latest manifest file for remote cluster state", e); - } - } - - /** - * Fetch latest ClusterMetadataManifest file from remote state store - * - * @param clusterUUID uuid of cluster state to refer to in remote - * @param clusterName name of the cluster - * @return latest ClusterMetadataManifest filename - */ - private Optional getLatestManifestFileName(String clusterName, String clusterUUID) throws IllegalStateException { - List manifestFilesMetadata = getManifestFileNames(clusterName, clusterUUID, 1); - if (manifestFilesMetadata != null && !manifestFilesMetadata.isEmpty()) { - return Optional.of(manifestFilesMetadata.get(0).name()); - } - logger.info("No manifest file present in remote store for cluster name: {}, cluster UUID: {}", clusterName, clusterUUID); - return Optional.empty(); - } - - /** - * Fetch ClusterMetadataManifest from remote state store - * - * @param clusterUUID uuid of cluster state to refer to in remote - * @param clusterName name of the cluster - * @return ClusterMetadataManifest - */ - private ClusterMetadataManifest fetchRemoteClusterMetadataManifest(String clusterName, String clusterUUID, String filename) - throws IllegalStateException { - try { - return getClusterMetadataManifestBlobStoreFormat(filename).read( - manifestContainer(clusterName, clusterUUID), - filename, - blobStoreRepository.getNamedXContentRegistry() - ); - } catch (IOException e) { - throw new IllegalStateException(String.format(Locale.ROOT, "Error while downloading cluster metadata - %s", filename), e); - } - } - - private ChecksumBlobStoreFormat getClusterMetadataManifestBlobStoreFormat(String fileName) { - long codecVersion = getManifestCodecVersion(fileName); - if (codecVersion == MANIFEST_CURRENT_CODEC_VERSION) { - return CLUSTER_METADATA_MANIFEST_FORMAT; - } else if (codecVersion == ClusterMetadataManifest.CODEC_V1) { - return CLUSTER_METADATA_MANIFEST_FORMAT_V1; - } else if (codecVersion == ClusterMetadataManifest.CODEC_V0) { - return CLUSTER_METADATA_MANIFEST_FORMAT_V0; - } - - throw new IllegalArgumentException("Cluster metadata manifest file is corrupted, don't have valid codec version"); - } - - private int getManifestCodecVersion(String fileName) { - String[] splitName = fileName.split(DELIMITER); - if (splitName.length == SPLITED_MANIFEST_FILE_LENGTH) { - return Integer.parseInt(splitName[splitName.length - 1]); // Last value would be codec version. - } else if (splitName.length < SPLITED_MANIFEST_FILE_LENGTH) { // Where codec is not part of file name, i.e. default codec version 0 - // is used. - return ClusterMetadataManifest.CODEC_V0; - } else { - throw new IllegalArgumentException("Manifest file name is corrupted"); - } - } - - public static String encodeString(String content) { - return Base64.getUrlEncoder().withoutPadding().encodeToString(content.getBytes(StandardCharsets.UTF_8)); - } - public void writeMetadataFailed() { getStats().stateFailed(); } @@ -1501,265 +1558,7 @@ public RemoteStateTransferException(String errorDesc, Throwable cause) { } } - /** - * Purges all remote cluster state against provided cluster UUIDs - * - * @param clusterName name of the cluster - * @param clusterUUIDs clusteUUIDs for which the remote state needs to be purged - */ - void deleteStaleUUIDsClusterMetadata(String clusterName, List clusterUUIDs) { - clusterUUIDs.forEach(clusterUUID -> { - getBlobStoreTransferService().deleteAsync( - ThreadPool.Names.REMOTE_PURGE, - getCusterMetadataBasePath(clusterName, clusterUUID), - new ActionListener<>() { - @Override - public void onResponse(Void unused) { - logger.info("Deleted all remote cluster metadata for cluster UUID - {}", clusterUUID); - } - - @Override - public void onFailure(Exception e) { - logger.error( - new ParameterizedMessage( - "Exception occurred while deleting all remote cluster metadata for cluster UUID {}", - clusterUUID - ), - e - ); - remoteStateStats.cleanUpAttemptFailed(); - } - } - ); - }); - } - - /** - * Deletes older than last {@code versionsToRetain} manifests. Also cleans up unreferenced IndexMetadata associated with older manifests - * - * @param clusterName name of the cluster - * @param clusterUUID uuid of cluster state to refer to in remote - * @param manifestsToRetain no of latest manifest files to keep in remote - */ - // package private for testing - void deleteStaleClusterMetadata(String clusterName, String clusterUUID, int manifestsToRetain) { - if (deleteStaleMetadataRunning.compareAndSet(false, true) == false) { - logger.info("Delete stale cluster metadata task is already in progress."); - return; - } - try { - getBlobStoreTransferService().listAllInSortedOrderAsync( - ThreadPool.Names.REMOTE_PURGE, - getManifestFolderPath(clusterName, clusterUUID), - "manifest", - Integer.MAX_VALUE, - new ActionListener<>() { - @Override - public void onResponse(List blobMetadata) { - if (blobMetadata.size() > manifestsToRetain) { - deleteClusterMetadata( - clusterName, - clusterUUID, - blobMetadata.subList(0, manifestsToRetain - 1), - blobMetadata.subList(manifestsToRetain - 1, blobMetadata.size()) - ); - } - deleteStaleMetadataRunning.set(false); - } - - @Override - public void onFailure(Exception e) { - logger.error( - new ParameterizedMessage( - "Exception occurred while deleting Remote Cluster Metadata for clusterUUIDs {}", - clusterUUID - ) - ); - deleteStaleMetadataRunning.set(false); - } - } - ); - } catch (Exception e) { - deleteStaleMetadataRunning.set(false); - throw e; - } - } - - private void deleteClusterMetadata( - String clusterName, - String clusterUUID, - List activeManifestBlobMetadata, - List staleManifestBlobMetadata - ) { - try { - Set filesToKeep = new HashSet<>(); - Set staleManifestPaths = new HashSet<>(); - Set staleIndexMetadataPaths = new HashSet<>(); - Set staleGlobalMetadataPaths = new HashSet<>(); - activeManifestBlobMetadata.forEach(blobMetadata -> { - ClusterMetadataManifest clusterMetadataManifest = fetchRemoteClusterMetadataManifest( - clusterName, - clusterUUID, - blobMetadata.name() - ); - clusterMetadataManifest.getIndices() - .forEach(uploadedIndexMetadata -> filesToKeep.add(uploadedIndexMetadata.getUploadedFilename())); - if (clusterMetadataManifest.getGlobalMetadataFileName() != null) { - filesToKeep.add(clusterMetadataManifest.getGlobalMetadataFileName()); - } else { - filesToKeep.add(clusterMetadataManifest.getCoordinationMetadata().getUploadedFilename()); - filesToKeep.add(clusterMetadataManifest.getTemplatesMetadata().getUploadedFilename()); - filesToKeep.add(clusterMetadataManifest.getSettingsMetadata().getUploadedFilename()); - clusterMetadataManifest.getCustomMetadataMap() - .forEach((key, value) -> { filesToKeep.add(value.getUploadedFilename()); }); - } - }); - staleManifestBlobMetadata.forEach(blobMetadata -> { - ClusterMetadataManifest clusterMetadataManifest = fetchRemoteClusterMetadataManifest( - clusterName, - clusterUUID, - blobMetadata.name() - ); - staleManifestPaths.add(new BlobPath().add(MANIFEST_PATH_TOKEN).buildAsString() + blobMetadata.name()); - if (clusterMetadataManifest.getGlobalMetadataFileName() != null) { - if (filesToKeep.contains(clusterMetadataManifest.getGlobalMetadataFileName()) == false) { - String[] globalMetadataSplitPath = clusterMetadataManifest.getGlobalMetadataFileName().split("/"); - staleGlobalMetadataPaths.add( - new BlobPath().add(GLOBAL_METADATA_PATH_TOKEN).buildAsString() + GLOBAL_METADATA_FORMAT.blobName( - globalMetadataSplitPath[globalMetadataSplitPath.length - 1] - ) - ); - } - } else { - if (filesToKeep.contains(clusterMetadataManifest.getCoordinationMetadata().getUploadedFilename()) == false) { - String[] coordinationMetadataSplitPath = clusterMetadataManifest.getCoordinationMetadata() - .getUploadedFilename() - .split("/"); - staleGlobalMetadataPaths.add( - new BlobPath().add(GLOBAL_METADATA_PATH_TOKEN).buildAsString() + GLOBAL_METADATA_FORMAT.blobName( - coordinationMetadataSplitPath[coordinationMetadataSplitPath.length - 1] - ) - ); - } - if (filesToKeep.contains(clusterMetadataManifest.getTemplatesMetadata().getUploadedFilename()) == false) { - String[] templatesMetadataSplitPath = clusterMetadataManifest.getTemplatesMetadata() - .getUploadedFilename() - .split("/"); - staleGlobalMetadataPaths.add( - new BlobPath().add(GLOBAL_METADATA_PATH_TOKEN).buildAsString() + GLOBAL_METADATA_FORMAT.blobName( - templatesMetadataSplitPath[templatesMetadataSplitPath.length - 1] - ) - ); - } - if (filesToKeep.contains(clusterMetadataManifest.getSettingsMetadata().getUploadedFilename()) == false) { - String[] settingsMetadataSplitPath = clusterMetadataManifest.getSettingsMetadata().getUploadedFilename().split("/"); - staleGlobalMetadataPaths.add( - new BlobPath().add(GLOBAL_METADATA_PATH_TOKEN).buildAsString() + GLOBAL_METADATA_FORMAT.blobName( - settingsMetadataSplitPath[settingsMetadataSplitPath.length - 1] - ) - ); - } - clusterMetadataManifest.getCustomMetadataMap().forEach((key, value) -> { - if (filesToKeep.contains(value.getUploadedFilename()) == false) { - String[] customMetadataSplitPath = value.getUploadedFilename().split("/"); - staleGlobalMetadataPaths.add( - new BlobPath().add(GLOBAL_METADATA_PATH_TOKEN).buildAsString() + GLOBAL_METADATA_FORMAT.blobName( - customMetadataSplitPath[customMetadataSplitPath.length - 1] - ) - ); - } - }); - } - - clusterMetadataManifest.getIndices().forEach(uploadedIndexMetadata -> { - if (filesToKeep.contains(uploadedIndexMetadata.getUploadedFilename()) == false) { - staleIndexMetadataPaths.add( - new BlobPath().add(INDEX_PATH_TOKEN).add(uploadedIndexMetadata.getIndexUUID()).buildAsString() - + INDEX_METADATA_FORMAT.blobName(uploadedIndexMetadata.getUploadedFilename()) - ); - } - }); - }); - - if (staleManifestPaths.isEmpty()) { - logger.debug("No stale Remote Cluster Metadata files found"); - return; - } - - deleteStalePaths(clusterName, clusterUUID, new ArrayList<>(staleGlobalMetadataPaths)); - deleteStalePaths(clusterName, clusterUUID, new ArrayList<>(staleIndexMetadataPaths)); - deleteStalePaths(clusterName, clusterUUID, new ArrayList<>(staleManifestPaths)); - } catch (IllegalStateException e) { - logger.error("Error while fetching Remote Cluster Metadata manifests", e); - } catch (IOException e) { - logger.error("Error while deleting stale Remote Cluster Metadata files", e); - remoteStateStats.cleanUpAttemptFailed(); - } catch (Exception e) { - logger.error("Unexpected error while deleting stale Remote Cluster Metadata files", e); - remoteStateStats.cleanUpAttemptFailed(); - } - } - - private void deleteStalePaths(String clusterName, String clusterUUID, List stalePaths) throws IOException { - logger.debug(String.format(Locale.ROOT, "Deleting stale files from remote - %s", stalePaths)); - getBlobStoreTransferService().deleteBlobs(getCusterMetadataBasePath(clusterName, clusterUUID), stalePaths); - } - - /** - * Purges all remote cluster state against provided cluster UUIDs - * - * @param clusterState current state of the cluster - * @param committedManifest last committed ClusterMetadataManifest - */ - public void deleteStaleClusterUUIDs(ClusterState clusterState, ClusterMetadataManifest committedManifest) { - threadpool.executor(ThreadPool.Names.REMOTE_PURGE).execute(() -> { - String clusterName = clusterState.getClusterName().value(); - logger.debug("Deleting stale cluster UUIDs data from remote [{}]", clusterName); - Set allClustersUUIDsInRemote; - try { - allClustersUUIDsInRemote = new HashSet<>(getAllClusterUUIDs(clusterState.getClusterName().value())); - } catch (IOException e) { - logger.info(String.format(Locale.ROOT, "Error while fetching all cluster UUIDs for [%s]", clusterName)); - return; - } - // Retain last 2 cluster uuids data - allClustersUUIDsInRemote.remove(committedManifest.getClusterUUID()); - allClustersUUIDsInRemote.remove(committedManifest.getPreviousClusterUUID()); - deleteStaleUUIDsClusterMetadata(clusterName, new ArrayList<>(allClustersUUIDsInRemote)); - }); - } - public RemotePersistenceStats getStats() { return remoteStateStats; } - - private static class UploadedMetadataResults { - List uploadedIndexMetadata; - Map uploadedCustomMetadataMap; - UploadedMetadataAttribute uploadedCoordinationMetadata; - UploadedMetadataAttribute uploadedSettingsMetadata; - UploadedMetadataAttribute uploadedTemplatesMetadata; - - public UploadedMetadataResults( - List uploadedIndexMetadata, - Map uploadedCustomMetadataMap, - UploadedMetadataAttribute uploadedCoordinationMetadata, - UploadedMetadataAttribute uploadedSettingsMetadata, - UploadedMetadataAttribute uploadedTemplatesMetadata - ) { - this.uploadedIndexMetadata = uploadedIndexMetadata; - this.uploadedCustomMetadataMap = uploadedCustomMetadataMap; - this.uploadedCoordinationMetadata = uploadedCoordinationMetadata; - this.uploadedSettingsMetadata = uploadedSettingsMetadata; - this.uploadedTemplatesMetadata = uploadedTemplatesMetadata; - } - - public UploadedMetadataResults() { - this.uploadedIndexMetadata = new ArrayList<>(); - this.uploadedCustomMetadataMap = new HashMap<>(); - this.uploadedCoordinationMetadata = null; - this.uploadedSettingsMetadata = null; - this.uploadedTemplatesMetadata = null; - } - } } diff --git a/server/src/main/java/org/opensearch/gateway/remote/RemoteClusterStateUtils.java b/server/src/main/java/org/opensearch/gateway/remote/RemoteClusterStateUtils.java index 500d1af0211e8..fbf355ca8a69c 100644 --- a/server/src/main/java/org/opensearch/gateway/remote/RemoteClusterStateUtils.java +++ b/server/src/main/java/org/opensearch/gateway/remote/RemoteClusterStateUtils.java @@ -8,16 +8,127 @@ package org.opensearch.gateway.remote; +import java.util.Locale; +import org.opensearch.cluster.metadata.Metadata; +import org.opensearch.common.blobstore.BlobContainer; +import org.opensearch.common.blobstore.BlobPath; +import org.opensearch.core.xcontent.ToXContent; +import org.opensearch.repositories.blobstore.BlobStoreRepository; + import java.nio.charset.StandardCharsets; +import java.util.ArrayList; import java.util.Base64; +import java.util.HashMap; +import java.util.List; +import java.util.Map; /** * Utility class for Remote Cluster State */ public class RemoteClusterStateUtils { + public static final String METADATA_NAME_FORMAT = "%s.dat"; + public static final String METADATA_NAME_PLAIN_FORMAT = "%s"; + public static final String METADATA_FILE_PREFIX = "metadata"; + public static final String CLUSTER_STATE_PATH_TOKEN = "cluster-state"; + public static final String GLOBAL_METADATA_PATH_TOKEN = "global-metadata"; + public static final String CLUSTER_STATE_EPHEMERAL_PATH_TOKEN = "ephemeral"; + public static final int GLOBAL_METADATA_CURRENT_CODEC_VERSION = 1; + public static final String DELIMITER = "__"; public static final String PATH_DELIMITER = "/"; + // ToXContent Params with gateway mode. + // We are using gateway context mode to persist all custom metadata. + public static final ToXContent.Params FORMAT_PARAMS = new ToXContent.MapParams(Map.of(Metadata.CONTEXT_MODE_PARAM, Metadata.CONTEXT_MODE_GATEWAY)); + + public static BlobPath getCusterMetadataBasePath(BlobStoreRepository blobStoreRepository, String clusterName, String clusterUUID) { + return blobStoreRepository.basePath().add(encodeString(clusterName)).add(CLUSTER_STATE_PATH_TOKEN).add(clusterUUID); + } + public static String encodeString(String content) { return Base64.getUrlEncoder().withoutPadding().encodeToString(content.getBytes(StandardCharsets.UTF_8)); } + + public static String getFormattedFileName(String fileName, int codecVersion) { + if (codecVersion < ClusterMetadataManifest.CODEC_V3) { + return String.format(Locale.ROOT, METADATA_NAME_FORMAT, fileName); + } + return fileName; + } + + static BlobContainer clusterUUIDContainer(BlobStoreRepository blobStoreRepository, String clusterName) { + return blobStoreRepository.blobStore() + .blobContainer( + blobStoreRepository.basePath() + .add(Base64.getUrlEncoder().withoutPadding().encodeToString(clusterName.getBytes(StandardCharsets.UTF_8))) + .add(CLUSTER_STATE_PATH_TOKEN) + ); + } + + /** + * Exception for Remote state transfer. + */ + public static class RemoteStateTransferException extends RuntimeException { + + public RemoteStateTransferException(String errorDesc) { + super(errorDesc); + } + + public RemoteStateTransferException(String errorDesc, Throwable cause) { + super(errorDesc, cause); + } + } + + public static class UploadedMetadataResults { + List uploadedIndexMetadata; + Map uploadedCustomMetadataMap; + Map uploadedClusterStateCustomMetadataMap; + ClusterMetadataManifest.UploadedMetadataAttribute uploadedCoordinationMetadata; + ClusterMetadataManifest.UploadedMetadataAttribute uploadedSettingsMetadata; + ClusterMetadataManifest.UploadedMetadataAttribute uploadedTransientSettingsMetadata; + ClusterMetadataManifest.UploadedMetadataAttribute uploadedTemplatesMetadata; + ClusterMetadataManifest.UploadedMetadataAttribute uploadedDiscoveryNodes; + ClusterMetadataManifest.UploadedMetadataAttribute uploadedClusterBlocks; + List uploadedIndicesRoutingMetadata; + ClusterMetadataManifest.UploadedMetadataAttribute uploadedHashesOfConsistentSettings; + + public UploadedMetadataResults( + List uploadedIndexMetadata, + Map uploadedCustomMetadataMap, + ClusterMetadataManifest.UploadedMetadataAttribute uploadedCoordinationMetadata, + ClusterMetadataManifest.UploadedMetadataAttribute uploadedSettingsMetadata, + ClusterMetadataManifest.UploadedMetadataAttribute uploadedTransientSettingsMetadata, + ClusterMetadataManifest.UploadedMetadataAttribute uploadedTemplatesMetadata, + ClusterMetadataManifest.UploadedMetadataAttribute uploadedDiscoveryNodes, + ClusterMetadataManifest.UploadedMetadataAttribute uploadedClusterBlocks, + List uploadedIndicesRoutingMetadata, + ClusterMetadataManifest.UploadedMetadataAttribute uploadedHashesOfConsistentSettings, + Map uploadedClusterStateCustomMap + ) { + this.uploadedIndexMetadata = uploadedIndexMetadata; + this.uploadedCustomMetadataMap = uploadedCustomMetadataMap; + this.uploadedCoordinationMetadata = uploadedCoordinationMetadata; + this.uploadedSettingsMetadata = uploadedSettingsMetadata; + this.uploadedTransientSettingsMetadata = uploadedTransientSettingsMetadata; + this.uploadedTemplatesMetadata = uploadedTemplatesMetadata; + this.uploadedDiscoveryNodes = uploadedDiscoveryNodes; + this.uploadedClusterBlocks = uploadedClusterBlocks; + this.uploadedIndicesRoutingMetadata = uploadedIndicesRoutingMetadata; + this.uploadedHashesOfConsistentSettings = uploadedHashesOfConsistentSettings; + this.uploadedClusterStateCustomMetadataMap = uploadedClusterStateCustomMap; + } + + public UploadedMetadataResults() { + this.uploadedIndexMetadata = new ArrayList<>(); + this.uploadedCustomMetadataMap = new HashMap<>(); + this.uploadedCoordinationMetadata = null; + this.uploadedSettingsMetadata = null; + this.uploadedTransientSettingsMetadata = null; + this.uploadedTemplatesMetadata = null; + this.uploadedDiscoveryNodes = null; + this.uploadedClusterBlocks = null; + this.uploadedIndicesRoutingMetadata = new ArrayList<>(); + this.uploadedHashesOfConsistentSettings = null; + this.uploadedClusterStateCustomMetadataMap = new HashMap<>(); + } + } } diff --git a/server/src/main/java/org/opensearch/gateway/remote/RemoteGlobalMetadataManager.java b/server/src/main/java/org/opensearch/gateway/remote/RemoteGlobalMetadataManager.java new file mode 100644 index 0000000000000..731cc9094b06d --- /dev/null +++ b/server/src/main/java/org/opensearch/gateway/remote/RemoteGlobalMetadataManager.java @@ -0,0 +1,301 @@ +/* + * 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.gateway.remote; + +import static java.util.Objects.requireNonNull; +import static org.opensearch.gateway.remote.RemoteClusterStateUtils.RemoteStateTransferException; +import static org.opensearch.gateway.remote.model.RemoteHashesOfConsistentSettings.HASHES_OF_CONSISTENT_SETTINGS; +import static org.opensearch.gateway.remote.model.RemoteTransientSettingsMetadata.TRANSIENT_SETTING_METADATA; + +import java.io.IOException; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Locale; +import java.util.Map; +import java.util.Set; +import org.opensearch.action.LatchedActionListener; +import org.opensearch.cluster.ClusterState; +import org.opensearch.cluster.coordination.CoordinationMetadata; +import org.opensearch.cluster.metadata.DiffableStringMap; +import org.opensearch.cluster.metadata.Metadata; +import org.opensearch.cluster.metadata.Metadata.Custom; +import org.opensearch.cluster.metadata.TemplatesMetadata; +import org.opensearch.common.CheckedRunnable; +import org.opensearch.common.settings.ClusterSettings; +import org.opensearch.common.settings.Setting; +import org.opensearch.common.settings.Settings; +import org.opensearch.common.unit.TimeValue; +import org.opensearch.core.action.ActionListener; +import org.opensearch.core.compress.Compressor; +import org.opensearch.core.xcontent.NamedXContentRegistry; +import org.opensearch.core.xcontent.ToXContent; +import org.opensearch.common.remote.AbstractRemoteWritableBlobEntity; +import org.opensearch.gateway.remote.model.RemoteClusterStateBlobStore; +import org.opensearch.gateway.remote.model.RemoteCoordinationMetadata; +import org.opensearch.gateway.remote.model.RemoteCustomMetadata; +import org.opensearch.gateway.remote.model.RemoteGlobalMetadata; +import org.opensearch.gateway.remote.model.RemoteHashesOfConsistentSettings; +import org.opensearch.gateway.remote.model.RemotePersistentSettingsMetadata; +import org.opensearch.gateway.remote.model.RemoteReadResult; +import org.opensearch.gateway.remote.model.RemoteTemplatesMetadata; +import org.opensearch.gateway.remote.model.RemoteTransientSettingsMetadata; +import org.opensearch.common.remote.RemoteWritableEntityStore; + +public class RemoteGlobalMetadataManager { + + public static final TimeValue GLOBAL_METADATA_UPLOAD_TIMEOUT_DEFAULT = TimeValue.timeValueMillis(20000); + + public static final Setting GLOBAL_METADATA_UPLOAD_TIMEOUT_SETTING = Setting.timeSetting( + "cluster.remote_store.state.global_metadata.upload_timeout", + GLOBAL_METADATA_UPLOAD_TIMEOUT_DEFAULT, + Setting.Property.Dynamic, + Setting.Property.NodeScope + ); + + public static final int GLOBAL_METADATA_CURRENT_CODEC_VERSION = 1; + + private volatile TimeValue globalMetadataUploadTimeout; + private final RemoteWritableEntityStore globalMetadataBlobStore; + private final RemoteClusterStateBlobStore coordinationMetadataBlobStore; + private final RemoteClusterStateBlobStore transientSettingsBlobStore; + private final RemoteClusterStateBlobStore persistentSettingsBlobStore; + private final RemoteClusterStateBlobStore templatesMetadataBlobStore; + private final RemoteClusterStateBlobStore customMetadataBlobStore; + private final RemoteClusterStateBlobStore hashesOfConsistentSettingsBlobStore; + private final Compressor compressor; + private final NamedXContentRegistry namedXContentRegistry; + + RemoteGlobalMetadataManager(ClusterSettings clusterSettings, + RemoteWritableEntityStore globalMetadataBlobStore, + RemoteClusterStateBlobStore coordinationMetadataBlobStore, + RemoteClusterStateBlobStore transientSettingsBlobStore, + RemoteClusterStateBlobStore persistentSettingsBlobStore, + RemoteClusterStateBlobStore templatesMetadataBlobStore, + RemoteClusterStateBlobStore customMetadataBlobStore, + RemoteClusterStateBlobStore hashesOfConsistentSettingsBlobStore, + Compressor compressor, + NamedXContentRegistry namedXContentRegistry) { + this.globalMetadataUploadTimeout = clusterSettings.get(GLOBAL_METADATA_UPLOAD_TIMEOUT_SETTING); + this.compressor = compressor; + this.globalMetadataBlobStore = globalMetadataBlobStore; + this.namedXContentRegistry = namedXContentRegistry; + this.coordinationMetadataBlobStore = coordinationMetadataBlobStore; + this.transientSettingsBlobStore = transientSettingsBlobStore; + this.persistentSettingsBlobStore = persistentSettingsBlobStore; + this.templatesMetadataBlobStore = templatesMetadataBlobStore; + this.customMetadataBlobStore = customMetadataBlobStore; + this.hashesOfConsistentSettingsBlobStore = hashesOfConsistentSettingsBlobStore; + clusterSettings.addSettingsUpdateConsumer(GLOBAL_METADATA_UPLOAD_TIMEOUT_SETTING, this::setGlobalMetadataUploadTimeout); + } + + /** + * Allows async upload of Metadata components to remote + */ + CheckedRunnable getAsyncMetadataWriteAction( + Object objectToUpload, + long metadataVersion, + String clusterUUID, + LatchedActionListener latchedActionListener, + String customType + ) { + if (objectToUpload instanceof CoordinationMetadata) { + RemoteCoordinationMetadata remoteCoordinationMetadata = new RemoteCoordinationMetadata((CoordinationMetadata) objectToUpload, metadataVersion, + clusterUUID, + compressor, namedXContentRegistry); + return () -> coordinationMetadataBlobStore.writeAsync(remoteCoordinationMetadata, + getActionListener(remoteCoordinationMetadata, latchedActionListener)); + } else if (objectToUpload instanceof Settings) { + if (customType != null && customType.equals(TRANSIENT_SETTING_METADATA)) { + RemoteTransientSettingsMetadata remoteTransientSettingsMetadata = new RemoteTransientSettingsMetadata((Settings) objectToUpload, + metadataVersion, clusterUUID, compressor, namedXContentRegistry); + return () -> transientSettingsBlobStore.writeAsync(remoteTransientSettingsMetadata, + getActionListener(remoteTransientSettingsMetadata, latchedActionListener)); + } + RemotePersistentSettingsMetadata remotePersistentSettingsMetadata = new RemotePersistentSettingsMetadata((Settings) objectToUpload, metadataVersion, + clusterUUID, + compressor, namedXContentRegistry); + return () -> persistentSettingsBlobStore.writeAsync(remotePersistentSettingsMetadata, + getActionListener(remotePersistentSettingsMetadata, latchedActionListener)); + } else if (objectToUpload instanceof TemplatesMetadata) { + RemoteTemplatesMetadata remoteTemplatesMetadata = new RemoteTemplatesMetadata((TemplatesMetadata) objectToUpload, metadataVersion, clusterUUID, + compressor, namedXContentRegistry); + return () -> templatesMetadataBlobStore.writeAsync(remoteTemplatesMetadata, getActionListener(remoteTemplatesMetadata, latchedActionListener)); + } else if (objectToUpload instanceof DiffableStringMap) { + RemoteHashesOfConsistentSettings remoteObject = new RemoteHashesOfConsistentSettings( + (DiffableStringMap) objectToUpload, + metadataVersion, + clusterUUID, + compressor, + namedXContentRegistry + ); + return () -> hashesOfConsistentSettingsBlobStore.writeAsync(remoteObject, getActionListener(remoteObject, latchedActionListener)); + } else if (objectToUpload instanceof Custom) { + RemoteCustomMetadata remoteCustomMetadata = new RemoteCustomMetadata((Custom) objectToUpload, customType, metadataVersion, clusterUUID, + compressor, namedXContentRegistry); + return () -> customMetadataBlobStore.writeAsync(remoteCustomMetadata, getActionListener(remoteCustomMetadata, latchedActionListener)); + } + throw new RemoteStateTransferException("Remote object cannot be created for " + objectToUpload.getClass()); + } + + private ActionListener getActionListener(AbstractRemoteWritableBlobEntity remoteBlobStoreObject, + LatchedActionListener latchedActionListener) { + return ActionListener.wrap( + resp -> latchedActionListener.onResponse( + remoteBlobStoreObject.getUploadedMetadata() + ), + ex -> latchedActionListener.onFailure(new RemoteStateTransferException("Upload failed", ex)) + ); + } + + CheckedRunnable getAsyncMetadataReadAction( + String clusterUUID, + String component, + String componentName, + String uploadFilename, + LatchedActionListener listener + ) { + ActionListener actionListener = ActionListener.wrap( + response -> listener.onResponse(new RemoteReadResult((ToXContent) response, component, componentName)), listener::onFailure); + if (component.equals(RemoteCoordinationMetadata.COORDINATION_METADATA)) { + RemoteCoordinationMetadata remoteBlobStoreObject = new RemoteCoordinationMetadata(uploadFilename, clusterUUID, compressor, namedXContentRegistry); + return () -> coordinationMetadataBlobStore.readAsync(remoteBlobStoreObject, actionListener); + } else if (component.equals(RemoteTemplatesMetadata.TEMPLATES_METADATA)) { + RemoteTemplatesMetadata remoteBlobStoreObject = new RemoteTemplatesMetadata(uploadFilename, clusterUUID, compressor, namedXContentRegistry); + return () -> templatesMetadataBlobStore.readAsync(remoteBlobStoreObject, actionListener); + } else if (component.equals(RemotePersistentSettingsMetadata.SETTING_METADATA)) { + RemotePersistentSettingsMetadata remoteBlobStoreObject = new RemotePersistentSettingsMetadata(uploadFilename, clusterUUID, compressor, namedXContentRegistry); + return () -> persistentSettingsBlobStore.readAsync(remoteBlobStoreObject, actionListener); + } else if (component.equals(RemoteTransientSettingsMetadata.TRANSIENT_SETTING_METADATA)) { + RemoteTransientSettingsMetadata remoteBlobStoreObject = new RemoteTransientSettingsMetadata(uploadFilename, clusterUUID, + compressor, namedXContentRegistry); + return () -> transientSettingsBlobStore.readAsync(remoteBlobStoreObject, actionListener); + } else if (component.equals(RemoteCustomMetadata.CUSTOM_METADATA)) { + RemoteCustomMetadata remoteBlobStoreObject = new RemoteCustomMetadata(uploadFilename, componentName, clusterUUID, compressor, namedXContentRegistry); + return () -> customMetadataBlobStore.readAsync(remoteBlobStoreObject, actionListener); + } else if (component.equals(HASHES_OF_CONSISTENT_SETTINGS)) { + RemoteHashesOfConsistentSettings remoteHashesOfConsistentSettings = new RemoteHashesOfConsistentSettings(uploadFilename, clusterUUID, compressor, namedXContentRegistry); + return () -> hashesOfConsistentSettingsBlobStore.readAsync(remoteHashesOfConsistentSettings, actionListener); + } else { + throw new RemoteStateTransferException("Unknown component " + componentName); + } + } + + Metadata getGlobalMetadata(String clusterUUID, ClusterMetadataManifest clusterMetadataManifest) { + String globalMetadataFileName = clusterMetadataManifest.getGlobalMetadataFileName(); + try { + // Fetch Global metadata + if (globalMetadataFileName != null) { + RemoteGlobalMetadata remoteGlobalMetadata = new RemoteGlobalMetadata(globalMetadataFileName, clusterUUID, + compressor, namedXContentRegistry); + return globalMetadataBlobStore.read(remoteGlobalMetadata); + } else if (clusterMetadataManifest.hasMetadataAttributesFiles()) { + Metadata.Builder builder = new Metadata.Builder(); + if (clusterMetadataManifest.getCoordinationMetadata().getUploadedFilename() != null) { + RemoteCoordinationMetadata remoteCoordinationMetadata = new RemoteCoordinationMetadata( + clusterMetadataManifest.getCoordinationMetadata().getUploadedFilename(), + clusterUUID, + compressor, + namedXContentRegistry + ); + builder.coordinationMetadata(coordinationMetadataBlobStore.read(remoteCoordinationMetadata)); + } + if (clusterMetadataManifest.getTemplatesMetadata().getUploadedFilename() != null) { + RemoteTemplatesMetadata remoteTemplatesMetadata = new RemoteTemplatesMetadata( + clusterMetadataManifest.getTemplatesMetadata().getUploadedFilename(), + clusterUUID, + compressor, + namedXContentRegistry + ); + builder.templates(templatesMetadataBlobStore.read(remoteTemplatesMetadata)); + } + if (clusterMetadataManifest.getSettingsMetadata().getUploadedFilename() != null) { + RemotePersistentSettingsMetadata remotePersistentSettingsMetadata = new RemotePersistentSettingsMetadata( + clusterMetadataManifest.getSettingsMetadata().getUploadedFilename(), + clusterUUID, + compressor, + namedXContentRegistry + ); + builder.persistentSettings(persistentSettingsBlobStore.read(remotePersistentSettingsMetadata)); + } + builder.clusterUUID(clusterMetadataManifest.getClusterUUID()); + builder.clusterUUIDCommitted(clusterMetadataManifest.isClusterUUIDCommitted()); + builder.version(clusterMetadataManifest.getMetadataVersion()); + clusterMetadataManifest.getCustomMetadataMap() + .forEach( + (key, value) -> { + try { + builder.putCustom( + key, + customMetadataBlobStore.read(new RemoteCustomMetadata( + value.getUploadedFilename(), + key, + clusterUUID, + compressor, + namedXContentRegistry + )) + ); + } catch (IOException e) { + throw new IllegalStateException( + String.format(Locale.ROOT, "Error while downloading Custom Metadata - %s", value.getUploadedFilename()), + e + ); + } + } + ); + return builder.build(); + } else { + return Metadata.EMPTY_METADATA; + } + } catch (IOException e) { + throw new IllegalStateException( + String.format(Locale.ROOT, "Error while downloading Global Metadata - %s", globalMetadataFileName), + e + ); + } + } + + Map getUpdatedCustoms(ClusterState currentState, ClusterState previousState) { + if (Metadata.isCustomMetadataEqual(previousState.metadata(), currentState.metadata())) { + return new HashMap<>(); + } + Map updatedCustom = new HashMap<>(); + Set currentCustoms = new HashSet<>(currentState.metadata().customs().keySet()); + for (Map.Entry cursor : previousState.metadata().customs().entrySet()) { + if (cursor.getValue().context().contains(Metadata.XContentContext.GATEWAY)) { + if (currentCustoms.contains(cursor.getKey()) + && !cursor.getValue().equals(currentState.metadata().custom(cursor.getKey()))) { + // If the custom metadata is updated, we need to upload the new version. + updatedCustom.put(cursor.getKey(), currentState.metadata().custom(cursor.getKey())); + } + currentCustoms.remove(cursor.getKey()); + } + } + for (String custom : currentCustoms) { + Metadata.Custom cursor = currentState.metadata().custom(custom); + if (cursor.context().contains(Metadata.XContentContext.GATEWAY)) { + updatedCustom.put(custom, cursor); + } + } + return updatedCustom; + } + + boolean isGlobalMetadataEqual(ClusterMetadataManifest first, ClusterMetadataManifest second, String clusterName) { + Metadata secondGlobalMetadata = getGlobalMetadata(second.getClusterUUID(), second); + Metadata firstGlobalMetadata = getGlobalMetadata(first.getClusterUUID(), first); + return Metadata.isGlobalResourcesMetadataEquals(firstGlobalMetadata, secondGlobalMetadata); + } + + private void setGlobalMetadataUploadTimeout(TimeValue newGlobalMetadataUploadTimeout) { + this.globalMetadataUploadTimeout = newGlobalMetadataUploadTimeout; + } + + public TimeValue getGlobalMetadataUploadTimeout() { + return this.globalMetadataUploadTimeout; + } +} diff --git a/server/src/main/java/org/opensearch/gateway/remote/RemoteIndexMetadataManager.java b/server/src/main/java/org/opensearch/gateway/remote/RemoteIndexMetadataManager.java new file mode 100644 index 0000000000000..5a18c075acc7e --- /dev/null +++ b/server/src/main/java/org/opensearch/gateway/remote/RemoteIndexMetadataManager.java @@ -0,0 +1,113 @@ +/* + * 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.gateway.remote; + +import static org.opensearch.gateway.remote.RemoteClusterStateUtils.RemoteStateTransferException; +import static org.opensearch.gateway.remote.model.RemoteIndexMetadata.INDEX_PATH_TOKEN; + +import java.io.IOException; +import java.util.Locale; +import org.opensearch.action.LatchedActionListener; +import org.opensearch.cluster.metadata.IndexMetadata; +import org.opensearch.common.CheckedRunnable; +import org.opensearch.common.settings.ClusterSettings; +import org.opensearch.common.settings.Setting; +import org.opensearch.common.unit.TimeValue; +import org.opensearch.core.action.ActionListener; +import org.opensearch.core.compress.Compressor; +import org.opensearch.core.xcontent.NamedXContentRegistry; +import org.opensearch.gateway.remote.model.RemoteIndexMetadata; +import org.opensearch.gateway.remote.model.RemoteReadResult; +import org.opensearch.common.remote.RemoteWritableEntityStore; + +public class RemoteIndexMetadataManager { + + public static final TimeValue INDEX_METADATA_UPLOAD_TIMEOUT_DEFAULT = TimeValue.timeValueMillis(20000); + + public static final Setting INDEX_METADATA_UPLOAD_TIMEOUT_SETTING = Setting.timeSetting( + "cluster.remote_store.state.index_metadata.upload_timeout", + INDEX_METADATA_UPLOAD_TIMEOUT_DEFAULT, + Setting.Property.Dynamic, + Setting.Property.NodeScope + ); + + private final RemoteWritableEntityStore indexMetadataBlobStore; + private final Compressor compressor; + private final NamedXContentRegistry namedXContentRegistry; + + private volatile TimeValue indexMetadataUploadTimeout; + + public RemoteIndexMetadataManager(RemoteWritableEntityStore indexMetadataBlobStore, ClusterSettings clusterSettings, Compressor compressor, NamedXContentRegistry namedXContentRegistry) { + this.indexMetadataBlobStore = indexMetadataBlobStore; + this.compressor = compressor; + this.namedXContentRegistry = namedXContentRegistry; + this.indexMetadataUploadTimeout = clusterSettings.get(INDEX_METADATA_UPLOAD_TIMEOUT_SETTING); + clusterSettings.addSettingsUpdateConsumer(INDEX_METADATA_UPLOAD_TIMEOUT_SETTING, this::setIndexMetadataUploadTimeout); + } + + /** + * Allows async Upload of IndexMetadata to remote + * + * @param indexMetadata {@link IndexMetadata} to upload + * @param latchedActionListener listener to respond back on after upload finishes + */ + CheckedRunnable getIndexMetadataAsyncAction(IndexMetadata indexMetadata, String clusterUUID, + LatchedActionListener latchedActionListener) { + RemoteIndexMetadata remoteIndexMetadata = new RemoteIndexMetadata(indexMetadata, clusterUUID, compressor, namedXContentRegistry); + ActionListener completionListener = ActionListener.wrap( + resp -> latchedActionListener.onResponse( + remoteIndexMetadata.getUploadedMetadata() + ), + ex -> latchedActionListener.onFailure(new RemoteStateTransferException(indexMetadata.getIndex().getName(), ex)) + ); + return () -> indexMetadataBlobStore.writeAsync(remoteIndexMetadata, completionListener); + } + + CheckedRunnable getAsyncIndexMetadataReadAction( + String clusterUUID, + String uploadedFilename, + LatchedActionListener latchedActionListener + ) { + RemoteIndexMetadata remoteIndexMetadata = new RemoteIndexMetadata(uploadedFilename, clusterUUID, compressor, namedXContentRegistry); + ActionListener actionListener = ActionListener.wrap( + response -> latchedActionListener.onResponse(new RemoteReadResult(response, INDEX_PATH_TOKEN, response.getIndexName())), + latchedActionListener::onFailure); + return () -> indexMetadataBlobStore.readAsync(remoteIndexMetadata, actionListener); + } + + /** + * Fetch index metadata from remote cluster state + * + * @param uploadedIndexMetadata {@link ClusterMetadataManifest.UploadedIndexMetadata} contains details about remote location of index metadata + * @return {@link IndexMetadata} + */ + IndexMetadata getIndexMetadata( + ClusterMetadataManifest.UploadedIndexMetadata uploadedIndexMetadata, String clusterUUID, int manifestCodecVersion + ) { + RemoteIndexMetadata remoteIndexMetadata = new RemoteIndexMetadata(RemoteClusterStateUtils.getFormattedFileName( + uploadedIndexMetadata.getUploadedFilename(), manifestCodecVersion), clusterUUID, compressor, namedXContentRegistry); + try { + return indexMetadataBlobStore.read(remoteIndexMetadata); + } catch (IOException e) { + throw new IllegalStateException( + String.format(Locale.ROOT, "Error while downloading IndexMetadata - %s", uploadedIndexMetadata.getUploadedFilename()), + e + ); + } + } + + public TimeValue getIndexMetadataUploadTimeout() { + return this.indexMetadataUploadTimeout; + } + + private void setIndexMetadataUploadTimeout(TimeValue newIndexMetadataUploadTimeout) { + this.indexMetadataUploadTimeout = newIndexMetadataUploadTimeout; + } + +} diff --git a/server/src/main/java/org/opensearch/gateway/remote/RemoteManifestManager.java b/server/src/main/java/org/opensearch/gateway/remote/RemoteManifestManager.java new file mode 100644 index 0000000000000..1379de54f26bf --- /dev/null +++ b/server/src/main/java/org/opensearch/gateway/remote/RemoteManifestManager.java @@ -0,0 +1,301 @@ +/* + * 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.gateway.remote; + +import static org.opensearch.gateway.remote.RemoteClusterStateUtils.DELIMITER; +import static org.opensearch.gateway.remote.RemoteClusterStateUtils.RemoteStateTransferException; +import static org.opensearch.gateway.remote.RemoteClusterStateUtils.getCusterMetadataBasePath; + +import java.io.IOException; +import java.util.HashMap; +import java.util.List; +import java.util.Locale; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReference; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.opensearch.Version; +import org.opensearch.action.LatchedActionListener; +import org.opensearch.cluster.ClusterState; +import org.opensearch.common.blobstore.BlobContainer; +import org.opensearch.common.blobstore.BlobMetadata; +import org.opensearch.common.blobstore.BlobPath; +import org.opensearch.common.collect.Tuple; +import org.opensearch.common.settings.ClusterSettings; +import org.opensearch.common.settings.Setting; +import org.opensearch.common.unit.TimeValue; +import org.opensearch.core.action.ActionListener; +import org.opensearch.core.compress.Compressor; +import org.opensearch.core.xcontent.NamedXContentRegistry; +import org.opensearch.gateway.remote.model.RemoteClusterStateBlobStore; +import org.opensearch.gateway.remote.model.RemoteClusterMetadataManifest; +import org.opensearch.gateway.remote.model.RemoteUploadDetails; +import org.opensearch.index.remote.RemoteStoreUtils; +import org.opensearch.repositories.blobstore.BlobStoreRepository; + +public class RemoteManifestManager { + + public static final TimeValue METADATA_MANIFEST_UPLOAD_TIMEOUT_DEFAULT = TimeValue.timeValueMillis(20000); + + public static final Setting METADATA_MANIFEST_UPLOAD_TIMEOUT_SETTING = Setting.timeSetting( + "cluster.remote_store.state.metadata_manifest.upload_timeout", + METADATA_MANIFEST_UPLOAD_TIMEOUT_DEFAULT, + Setting.Property.Dynamic, + Setting.Property.NodeScope + ); + private static final Logger logger = LogManager.getLogger(RemoteManifestManager.class); + + private volatile TimeValue metadataManifestUploadTimeout; + private final String nodeId; + private final RemoteClusterStateBlobStore manifestBlobStore; + private final Compressor compressor; + private final NamedXContentRegistry namedXContentRegistry; + //todo remove blobStorerepo from here + private final BlobStoreRepository blobStoreRepository; + + RemoteManifestManager(RemoteClusterStateBlobStore manifestBlobStore, ClusterSettings clusterSettings, String nodeId, Compressor compressor, NamedXContentRegistry namedXContentRegistry, BlobStoreRepository blobStoreRepository) { + this.metadataManifestUploadTimeout = clusterSettings.get(METADATA_MANIFEST_UPLOAD_TIMEOUT_SETTING); + this.nodeId = nodeId; + this.manifestBlobStore = manifestBlobStore; + clusterSettings.addSettingsUpdateConsumer(METADATA_MANIFEST_UPLOAD_TIMEOUT_SETTING, this::setMetadataManifestUploadTimeout); + this.compressor = compressor; + this.namedXContentRegistry = namedXContentRegistry; + this.blobStoreRepository = blobStoreRepository; + } + + RemoteUploadDetails uploadManifest( + ClusterState clusterState, + RemoteClusterStateUtils.UploadedMetadataResults uploadedMetadataResult, + String previousClusterUUID, + ClusterStateDiffManifest clusterDiffManifest, + boolean committed + ) { + synchronized (this) { + ClusterMetadataManifest.Builder manifestBuilder = ClusterMetadataManifest.builder(); + manifestBuilder.clusterTerm(clusterState.term()) + .stateVersion(clusterState.getVersion()) + .clusterUUID(clusterState.metadata().clusterUUID()) + .stateUUID(clusterState.stateUUID()) + .opensearchVersion(Version.CURRENT) + .nodeId(nodeId) + .committed(committed) + .codecVersion(RemoteClusterMetadataManifest.MANIFEST_CURRENT_CODEC_VERSION) + .indices(uploadedMetadataResult.uploadedIndexMetadata) + .previousClusterUUID(previousClusterUUID) + .clusterUUIDCommitted(clusterState.metadata().clusterUUIDCommitted()) + .coordinationMetadata(uploadedMetadataResult.uploadedCoordinationMetadata) + .settingMetadata(uploadedMetadataResult.uploadedSettingsMetadata) + .templatesMetadata(uploadedMetadataResult.uploadedTemplatesMetadata) + .customMetadataMap(uploadedMetadataResult.uploadedCustomMetadataMap) + .discoveryNodesMetadata(uploadedMetadataResult.uploadedDiscoveryNodes) + .clusterBlocksMetadata(uploadedMetadataResult.uploadedClusterBlocks) + .diffManifest(clusterDiffManifest) + .routingTableVersion(clusterState.getRoutingTable().version()) + .indicesRouting(uploadedMetadataResult.uploadedIndicesRoutingMetadata) + .metadataVersion(clusterState.metadata().version()) + .transientSettingsMetadata(uploadedMetadataResult.uploadedTransientSettingsMetadata) + .clusterStateCustomMetadataMap(uploadedMetadataResult.uploadedClusterStateCustomMetadataMap) + .hashesOfConsistentSettings(uploadedMetadataResult.uploadedHashesOfConsistentSettings); + final ClusterMetadataManifest manifest = manifestBuilder.build(); + String manifestFileName = writeMetadataManifest(clusterState.getClusterName().value(), clusterState.metadata().clusterUUID(), manifest); + return new RemoteUploadDetails(manifest, manifestFileName); + } + } + + private String writeMetadataManifest(String clusterName, String clusterUUID, ClusterMetadataManifest uploadManifest) { + AtomicReference result = new AtomicReference(); + AtomicReference exceptionReference = new AtomicReference(); + + // latch to wait until upload is not finished + CountDownLatch latch = new CountDownLatch(1); + + LatchedActionListener completionListener = new LatchedActionListener<>(ActionListener.wrap(resp -> { + logger.trace(String.format(Locale.ROOT, "Manifest file uploaded successfully.")); + }, ex -> { exceptionReference.set(ex); }), latch); + + RemoteClusterMetadataManifest remoteClusterMetadataManifest = new RemoteClusterMetadataManifest(uploadManifest, clusterUUID, compressor, namedXContentRegistry); + manifestBlobStore.writeAsync(remoteClusterMetadataManifest, completionListener); + + try { + if (latch.await(getMetadataManifestUploadTimeout().millis(), TimeUnit.MILLISECONDS) == false) { + RemoteStateTransferException ex = new RemoteStateTransferException( + String.format(Locale.ROOT, "Timed out waiting for transfer of manifest file to complete") + ); + throw ex; + } + } catch (InterruptedException ex) { + RemoteStateTransferException exception = new RemoteStateTransferException( + String.format(Locale.ROOT, "Timed out waiting for transfer of manifest file to complete - %s"), + ex + ); + Thread.currentThread().interrupt(); + throw exception; + } + if (exceptionReference.get() != null) { + throw new RemoteStateTransferException(exceptionReference.get().getMessage(), exceptionReference.get()); + } + logger.debug( + "Metadata manifest file [{}] written during [{}] phase. ", + remoteClusterMetadataManifest.getBlobFileName(), + uploadManifest.isCommitted() ? "commit" : "publish" + ); + return remoteClusterMetadataManifest.getUploadedMetadata().getUploadedFilename(); + } + + /** + * Fetch latest ClusterMetadataManifest from remote state store + * + * @param clusterUUID uuid of cluster state to refer to in remote + * @param clusterName name of the cluster + * @return ClusterMetadataManifest + */ + public Optional getLatestClusterMetadataManifest(String clusterName, String clusterUUID) { + Optional latestManifestFileName = getLatestManifestFileName(clusterName, clusterUUID); + return latestManifestFileName.map(s -> fetchRemoteClusterMetadataManifest(clusterName, clusterUUID, s)); + } + + /** + * Fetch the cluster metadata manifest using term and version + * @param clusterName uuid of cluster state to refer to in remote + * @param clusterUUID name of the cluster + * @param term election term of the cluster + * @param version cluster state version number + * @return ClusterMetadataManifest + */ + public Optional getClusterMetadataManifestByTermVersion(String clusterName, String clusterUUID, long term, long version) { + Optional manifestFileName = getManifestFileNameByTermVersion(clusterName, clusterUUID, term, version); + return manifestFileName.map(s -> fetchRemoteClusterMetadataManifest(clusterName, clusterUUID, s)); + } + + public ClusterMetadataManifest getRemoteClusterMetadataManifestByFileName(String clusterUUID, String filename) + throws IllegalStateException { + try { + RemoteClusterMetadataManifest remoteClusterMetadataManifest = new RemoteClusterMetadataManifest(filename, clusterUUID, compressor, namedXContentRegistry); + return manifestBlobStore.read(remoteClusterMetadataManifest); + } catch (IOException e) { + throw new IllegalStateException(String.format(Locale.ROOT, "Error while downloading cluster metadata - %s", filename), e); + } + } + + /** + * Fetch ClusterMetadataManifest from remote state store + * + * @param clusterUUID uuid of cluster state to refer to in remote + * @param clusterName name of the cluster + * @return ClusterMetadataManifest + */ + ClusterMetadataManifest fetchRemoteClusterMetadataManifest(String clusterName, String clusterUUID, String filename) + throws IllegalStateException { + try { + String fullBlobName = getManifestFolderPath(clusterName, clusterUUID).buildAsString() + filename; + RemoteClusterMetadataManifest remoteClusterMetadataManifest = new RemoteClusterMetadataManifest(fullBlobName, clusterUUID, compressor, namedXContentRegistry); + return manifestBlobStore.read(remoteClusterMetadataManifest); + } catch (IOException e) { + throw new IllegalStateException(String.format(Locale.ROOT, "Error while downloading cluster metadata - %s", filename), e); + } + } + + Map getLatestManifestForAllClusterUUIDs(String clusterName, Set clusterUUIDs) { + Map manifestsByClusterUUID = new HashMap<>(); + for (String clusterUUID : clusterUUIDs) { + try { + Optional manifest = getLatestClusterMetadataManifest(clusterName, clusterUUID); + manifest.ifPresent(clusterMetadataManifest -> manifestsByClusterUUID.put(clusterUUID, clusterMetadataManifest)); + } catch (Exception e) { + throw new IllegalStateException( + String.format(Locale.ROOT, "Exception in fetching manifest for clusterUUID: %s", clusterUUID), + e + ); + } + } + return manifestsByClusterUUID; + } + + private BlobContainer manifestContainer(String clusterName, String clusterUUID) { + // 123456789012_test-cluster/cluster-state/dsgYj10Nkso7/manifest + return blobStoreRepository.blobStore().blobContainer(getManifestFolderPath(clusterName, clusterUUID)); + } + + BlobPath getManifestFolderPath(String clusterName, String clusterUUID) { + return getCusterMetadataBasePath(blobStoreRepository, clusterName, clusterUUID).add(RemoteClusterMetadataManifest.MANIFEST); + } + + public TimeValue getMetadataManifestUploadTimeout() { + return this.metadataManifestUploadTimeout; + } + + private void setMetadataManifestUploadTimeout(TimeValue newMetadataManifestUploadTimeout) { + this.metadataManifestUploadTimeout = newMetadataManifestUploadTimeout; + } + + /** + * Fetch ClusterMetadataManifest files from remote state store in order + * + * @param clusterUUID uuid of cluster state to refer to in remote + * @param clusterName name of the cluster + * @param limit max no of files to fetch + * @return all manifest file names + */ + private List getManifestFileNames(String clusterName, String clusterUUID, String filePrefix, int limit) throws IllegalStateException { + try { + + /* + {@link BlobContainer#listBlobsByPrefixInSortedOrder} will list the latest manifest file first + as the manifest file name generated via {@link RemoteClusterStateService#getManifestFileName} ensures + when sorted in LEXICOGRAPHIC order the latest uploaded manifest file comes on top. + */ + return manifestContainer(clusterName, clusterUUID).listBlobsByPrefixInSortedOrder( + filePrefix, + limit, + BlobContainer.BlobNameSortOrder.LEXICOGRAPHIC + ); + } catch (IOException e) { + throw new IllegalStateException("Error while fetching latest manifest file for remote cluster state", e); + } + } + + static String getManifestFilePrefixForTermVersion(long term, long version) { + return String.join( + DELIMITER, + RemoteClusterMetadataManifest.MANIFEST, + RemoteStoreUtils.invertLong(term), + RemoteStoreUtils.invertLong(version) + ) + DELIMITER; + } + + /** + * Fetch latest ClusterMetadataManifest file from remote state store + * + * @param clusterUUID uuid of cluster state to refer to in remote + * @param clusterName name of the cluster + * @return latest ClusterMetadataManifest filename + */ + private Optional getLatestManifestFileName(String clusterName, String clusterUUID) throws IllegalStateException { + List manifestFilesMetadata = getManifestFileNames(clusterName, clusterUUID, RemoteClusterMetadataManifest.MANIFEST + DELIMITER, 1); + if (manifestFilesMetadata != null && !manifestFilesMetadata.isEmpty()) { + return Optional.of(manifestFilesMetadata.get(0).name()); + } + logger.info("No manifest file present in remote store for cluster name: {}, cluster UUID: {}", clusterName, clusterUUID); + return Optional.empty(); + } + + private Optional getManifestFileNameByTermVersion(String clusterName, String clusterUUID, long term, long version) throws IllegalStateException { + final String filePrefix = getManifestFilePrefixForTermVersion(term, version); + List manifestFilesMetadata = getManifestFileNames(clusterName, clusterUUID, filePrefix, 1); + if (manifestFilesMetadata != null && !manifestFilesMetadata.isEmpty()) { + return Optional.of(manifestFilesMetadata.get(0).name()); + } + logger.info("No manifest file present in remote store for cluster name: {}, cluster UUID: {}, term: {}, version: {}", clusterName, clusterUUID, term, version); + return Optional.empty(); + } +} diff --git a/server/src/main/java/org/opensearch/gateway/remote/model/RemoteClusterBlocks.java b/server/src/main/java/org/opensearch/gateway/remote/model/RemoteClusterBlocks.java new file mode 100644 index 0000000000000..34741da435b44 --- /dev/null +++ b/server/src/main/java/org/opensearch/gateway/remote/model/RemoteClusterBlocks.java @@ -0,0 +1,85 @@ +/* + * 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.gateway.remote.model; + +import static org.opensearch.gateway.remote.RemoteClusterStateAttributesManager.CLUSTER_STATE_ATTRIBUTES_CURRENT_CODEC_VERSION; +import static org.opensearch.gateway.remote.RemoteClusterStateUtils.DELIMITER; + +import java.io.IOException; +import java.io.InputStream; +import java.util.List; +import org.opensearch.cluster.block.ClusterBlocks; +import org.opensearch.common.io.Streams; +import org.opensearch.common.remote.AbstractRemoteWritableBlobEntity; +import org.opensearch.common.remote.BlobPathParameters; +import org.opensearch.core.compress.Compressor; +import org.opensearch.core.xcontent.NamedXContentRegistry; +import org.opensearch.gateway.remote.ClusterMetadataManifest.UploadedMetadata; +import org.opensearch.gateway.remote.ClusterMetadataManifest.UploadedMetadataAttribute; +import org.opensearch.index.remote.RemoteStoreUtils; +import org.opensearch.repositories.blobstore.ChecksumWritableBlobStoreFormat; + +/** + * Wrapper class for uploading/downloading {@link ClusterBlocks} to/from remote blob store + */ +public class RemoteClusterBlocks extends AbstractRemoteWritableBlobEntity { + + public static final String CLUSTER_BLOCKS = "blocks"; + public static final ChecksumWritableBlobStoreFormat CLUSTER_BLOCKS_FORMAT = new ChecksumWritableBlobStoreFormat<>("blocks", ClusterBlocks::readFrom); + + private ClusterBlocks clusterBlocks; + private long stateVersion; + + public RemoteClusterBlocks(final ClusterBlocks clusterBlocks, long stateVersion, String clusterUUID, + final Compressor compressor, final NamedXContentRegistry namedXContentRegistry) { + super(clusterUUID, compressor, namedXContentRegistry); + this.clusterBlocks = clusterBlocks; + this.stateVersion = stateVersion; + } + + public RemoteClusterBlocks(final String blobName, final String clusterUUID, final Compressor compressor, final NamedXContentRegistry namedXContentRegistry) { + super(clusterUUID, compressor, namedXContentRegistry); + this.blobName = blobName; + } + + @Override + public BlobPathParameters getBlobPathParameters() { + return new BlobPathParameters(List.of("transient"), CLUSTER_BLOCKS); + } + + @Override + public String generateBlobFileName() { + // 123456789012_test-cluster/cluster-state/dsgYj10Nkso7/transient/______ + String blobFileName = String.join( + DELIMITER, + getBlobPathParameters().getFilePrefix(), + RemoteStoreUtils.invertLong(stateVersion), + RemoteStoreUtils.invertLong(System.currentTimeMillis()), + String.valueOf(CLUSTER_STATE_ATTRIBUTES_CURRENT_CODEC_VERSION) + ); + this.blobFileName = blobFileName; + return blobFileName; + } + + @Override + public UploadedMetadata getUploadedMetadata() { + assert blobName != null; + return new UploadedMetadataAttribute(CLUSTER_BLOCKS, blobName); + } + + @Override + public InputStream serialize() throws IOException { + return CLUSTER_BLOCKS_FORMAT.serialize(clusterBlocks, generateBlobFileName(), getCompressor()).streamInput(); + } + + @Override + public ClusterBlocks deserialize(final InputStream inputStream) throws IOException { + return CLUSTER_BLOCKS_FORMAT.deserialize(blobName, ClusterBlocks::readFrom, Streams.readFully(inputStream)); + } +} 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 new file mode 100644 index 0000000000000..d8365bae538f8 --- /dev/null +++ b/server/src/main/java/org/opensearch/gateway/remote/model/RemoteClusterMetadataManifest.java @@ -0,0 +1,138 @@ +/* + * 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.gateway.remote.model; + +import static org.opensearch.gateway.remote.RemoteClusterStateUtils.DELIMITER; + +import java.io.IOException; +import java.io.InputStream; +import java.util.List; +import org.opensearch.common.io.Streams; +import org.opensearch.common.remote.AbstractRemoteWritableBlobEntity; +import org.opensearch.common.remote.BlobPathParameters; +import org.opensearch.core.compress.Compressor; +import org.opensearch.core.xcontent.NamedXContentRegistry; +import org.opensearch.gateway.remote.ClusterMetadataManifest; +import org.opensearch.gateway.remote.ClusterMetadataManifest.UploadedMetadata; +import org.opensearch.gateway.remote.ClusterMetadataManifest.UploadedMetadataAttribute; +import org.opensearch.gateway.remote.RemoteClusterStateUtils; +import org.opensearch.index.remote.RemoteStoreUtils; +import org.opensearch.repositories.blobstore.ChecksumBlobStoreFormat; + +/** + * Wrapper class for uploading/downloading {@link ClusterMetadataManifest} to/from remote blob store + */ +public class RemoteClusterMetadataManifest extends AbstractRemoteWritableBlobEntity { + + public static final String MANIFEST = "manifest"; + public static final int SPLITTED_MANIFEST_FILE_LENGTH = 6; + + public static final String METADATA_MANIFEST_NAME_FORMAT = "%s"; + public static final int MANIFEST_CURRENT_CODEC_VERSION = ClusterMetadataManifest.CODEC_V3; + + /** + * Manifest format compatible with older codec v0, where codec version was missing. + */ + public static final ChecksumBlobStoreFormat CLUSTER_METADATA_MANIFEST_FORMAT_V0 = + new ChecksumBlobStoreFormat<>("cluster-metadata-manifest", METADATA_MANIFEST_NAME_FORMAT, ClusterMetadataManifest::fromXContentV0); + /** + * Manifest format compatible with older codec v1, where global metadata was missing. + */ + public static final ChecksumBlobStoreFormat CLUSTER_METADATA_MANIFEST_FORMAT_V1 = + new ChecksumBlobStoreFormat<>("cluster-metadata-manifest", METADATA_MANIFEST_NAME_FORMAT, ClusterMetadataManifest::fromXContentV1); + + /** + * Manifest format compatible with codec v2, where we introduced codec versions/global metadata. + */ + public static final ChecksumBlobStoreFormat CLUSTER_METADATA_MANIFEST_FORMAT = new ChecksumBlobStoreFormat<>( + "cluster-metadata-manifest", + METADATA_MANIFEST_NAME_FORMAT, + ClusterMetadataManifest::fromXContent + ); + + private ClusterMetadataManifest clusterMetadataManifest; + + public RemoteClusterMetadataManifest(final ClusterMetadataManifest clusterMetadataManifest, final String clusterUUID, + final Compressor compressor, final NamedXContentRegistry namedXContentRegistry) { + super(clusterUUID, compressor, namedXContentRegistry); + this.clusterMetadataManifest = clusterMetadataManifest; + } + + public RemoteClusterMetadataManifest(final String blobName, final String clusterUUID, + final Compressor compressor, final NamedXContentRegistry namedXContentRegistry) { + super(clusterUUID, compressor, namedXContentRegistry); + this.blobName = blobName; + } + + @Override + public BlobPathParameters getBlobPathParameters() { + return new BlobPathParameters(List.of(MANIFEST), MANIFEST); + } + + @Override + public String generateBlobFileName() { + // 123456789012_test-cluster/cluster-state/dsgYj10Nkso7/manifest/manifest______C/P____ + // + String blobFileName = String.join( + DELIMITER, + MANIFEST, + RemoteStoreUtils.invertLong(clusterMetadataManifest.getClusterTerm()), + RemoteStoreUtils.invertLong(clusterMetadataManifest.getStateVersion()), + (clusterMetadataManifest.isCommitted() ? "C" : "P"), // C for committed and P for published + RemoteStoreUtils.invertLong(System.currentTimeMillis()), + String.valueOf(clusterMetadataManifest.getCodecVersion()) // Keep the codec version at last place only, during read we reads last place to + // determine codec version. + ); + this.blobFileName = blobFileName; + return blobFileName; + } + + @Override + public UploadedMetadata getUploadedMetadata() { + assert blobName != null; + return new UploadedMetadataAttribute(MANIFEST, blobName); + } + + @Override + public InputStream serialize() throws IOException { + return CLUSTER_METADATA_MANIFEST_FORMAT.serialize(clusterMetadataManifest, generateBlobFileName(), getCompressor(), + RemoteClusterStateUtils.FORMAT_PARAMS).streamInput(); + } + + @Override + public ClusterMetadataManifest deserialize(final InputStream inputStream) throws IOException { + ChecksumBlobStoreFormat blobStoreFormat = getClusterMetadataManifestBlobStoreFormat(); + return blobStoreFormat.deserialize(blobName, getNamedXContentRegistry(), Streams.readFully(inputStream)); + } + + private int getManifestCodecVersion() { + assert blobName != null; + String[] splitName = blobName.split(DELIMITER); + if (splitName.length == SPLITTED_MANIFEST_FILE_LENGTH) { + return Integer.parseInt(splitName[splitName.length - 1]); // Last value would be codec version. + } else if (splitName.length < SPLITTED_MANIFEST_FILE_LENGTH) { // Where codec is not part of file name, i.e. default codec version 0 + // is used. + return ClusterMetadataManifest.CODEC_V0; + } else { + throw new IllegalArgumentException("Manifest file name is corrupted"); + } + } + + private ChecksumBlobStoreFormat getClusterMetadataManifestBlobStoreFormat() { + long codecVersion = getManifestCodecVersion(); + if (codecVersion == MANIFEST_CURRENT_CODEC_VERSION) { + return CLUSTER_METADATA_MANIFEST_FORMAT; + } else if (codecVersion == ClusterMetadataManifest.CODEC_V1) { + return CLUSTER_METADATA_MANIFEST_FORMAT_V1; + } else if (codecVersion == ClusterMetadataManifest.CODEC_V0) { + return CLUSTER_METADATA_MANIFEST_FORMAT_V0; + } + throw new IllegalArgumentException("Cluster metadata manifest file is corrupted, don't have valid codec version"); + } +} diff --git a/server/src/main/java/org/opensearch/gateway/remote/model/RemoteClusterStateBlobStore.java b/server/src/main/java/org/opensearch/gateway/remote/model/RemoteClusterStateBlobStore.java index 1aeecc4e70382..aadd4f353853c 100644 --- a/server/src/main/java/org/opensearch/gateway/remote/model/RemoteClusterStateBlobStore.java +++ b/server/src/main/java/org/opensearch/gateway/remote/model/RemoteClusterStateBlobStore.java @@ -9,6 +9,7 @@ package org.opensearch.gateway.remote.model; import org.opensearch.common.blobstore.BlobPath; +import org.opensearch.common.blobstore.stream.write.WritePriority; import org.opensearch.common.remote.AbstractRemoteWritableBlobEntity; import org.opensearch.common.remote.RemoteWritableEntityStore; import org.opensearch.common.remote.RemoteWriteableEntity; @@ -54,15 +55,14 @@ public void writeAsync(final U entity, final ActionListener listener) { try (InputStream inputStream = entity.serialize()) { BlobPath blobPath = getBlobPathForUpload(entity); entity.setFullBlobName(blobPath); - // TODO uncomment below logic after merging PR https://github.com/opensearch-project/OpenSearch/pull/13836 - // transferService.uploadBlob(inputStream, getBlobPathForUpload(entity), entity.getBlobFileName(), WritePriority.URGENT, - // listener); + transferService.uploadBlobAsync(inputStream, getBlobPathForUpload(entity), entity.getBlobFileName(), WritePriority.URGENT, listener); } } catch (Exception e) { listener.onFailure(e); } } + @Override public T read(final U entity) throws IOException { // TODO Add timing logs and tracing assert entity.getFullBlobName() != null; diff --git a/server/src/main/java/org/opensearch/gateway/remote/model/RemoteClusterStateCustoms.java b/server/src/main/java/org/opensearch/gateway/remote/model/RemoteClusterStateCustoms.java new file mode 100644 index 0000000000000..b5af4969b7454 --- /dev/null +++ b/server/src/main/java/org/opensearch/gateway/remote/model/RemoteClusterStateCustoms.java @@ -0,0 +1,121 @@ +/* + * 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.gateway.remote.model; + +import org.opensearch.cluster.ClusterState; +import org.opensearch.cluster.ClusterState.Custom; +import org.opensearch.cluster.metadata.Metadata; +import org.opensearch.common.io.Streams; +import org.opensearch.common.io.stream.BytesStreamOutput; +import org.opensearch.common.remote.AbstractRemoteWritableBlobEntity; +import org.opensearch.common.remote.BlobPathParameters; +import org.opensearch.core.common.bytes.BytesReference; +import org.opensearch.core.common.io.stream.BytesStreamInput; +import org.opensearch.core.common.io.stream.NamedWriteableAwareStreamInput; +import org.opensearch.core.common.io.stream.NamedWriteableRegistry; +import org.opensearch.core.compress.Compressor; +import org.opensearch.core.xcontent.NamedXContentRegistry; +import org.opensearch.gateway.remote.ClusterMetadataManifest; +import org.opensearch.gateway.remote.RemoteClusterStateUtils; +import org.opensearch.index.remote.RemoteStoreUtils; +import org.opensearch.repositories.blobstore.ChecksumBlobStoreFormat; + +import java.io.IOException; +import java.io.InputStream; +import java.util.List; + +import static org.opensearch.cluster.ClusterState.FeatureAware.shouldSerialize; +import static org.opensearch.core.common.bytes.BytesReference.toBytes; +import static org.opensearch.gateway.remote.RemoteClusterStateAttributesManager.CLUSTER_STATE_ATTRIBUTES_CURRENT_CODEC_VERSION; +import static org.opensearch.gateway.remote.RemoteClusterStateUtils.CLUSTER_STATE_EPHEMERAL_PATH_TOKEN; +import static org.opensearch.gateway.remote.RemoteClusterStateUtils.DELIMITER; +import static org.opensearch.gateway.remote.RemoteClusterStateUtils.METADATA_NAME_FORMAT; +import static org.opensearch.gateway.remote.model.RemoteCustomMetadata.CUSTOM_DELIMITER; + +public class RemoteClusterStateCustoms extends AbstractRemoteWritableBlobEntity { + public static final String CLUSTER_STATE_CUSTOM = "cluster-state-custom"; + + private long stateVersion; + private String customType; + private ClusterState.Custom custom; + private final NamedWriteableRegistry namedWriteableRegistry; + + public RemoteClusterStateCustoms( + final ClusterState.Custom custom, + final String customType, + final long stateVersion, + final String clusterUUID, + final Compressor compressor, + final NamedXContentRegistry namedXContentRegistry, + final NamedWriteableRegistry namedWriteableRegistry + ) { + super(clusterUUID, compressor, namedXContentRegistry); + this.stateVersion = stateVersion; + this.customType = customType; + this.custom = custom; + this.namedWriteableRegistry = namedWriteableRegistry; + } + + public RemoteClusterStateCustoms( + final String blobName, + final String customType, + final String clusterUUID, + final Compressor compressor, + final NamedXContentRegistry namedXContentRegistry, + final NamedWriteableRegistry namedWriteableRegistry + ) { + super(clusterUUID, compressor, namedXContentRegistry); + this.blobName = blobName; + this.customType = customType; + this.namedWriteableRegistry = namedWriteableRegistry; + } + + @Override + public BlobPathParameters getBlobPathParameters() { + return new BlobPathParameters(List.of(CLUSTER_STATE_EPHEMERAL_PATH_TOKEN), CLUSTER_STATE_CUSTOM); + } + + @Override + public String generateBlobFileName() { + // 123456789012_test-cluster/cluster-state/dsgYj10Nkso7/ephemeral/______ + String blobFileName = String.join( + DELIMITER, + getBlobPathParameters().getFilePrefix(), + RemoteStoreUtils.invertLong(stateVersion), + RemoteStoreUtils.invertLong(System.currentTimeMillis()), + String.valueOf(CLUSTER_STATE_ATTRIBUTES_CURRENT_CODEC_VERSION) + ); + this.blobFileName = blobFileName; + return blobFileName; + } + + @Override + public ClusterMetadataManifest.UploadedMetadata getUploadedMetadata() { + assert blobName != null; + return new ClusterMetadataManifest.UploadedMetadataAttribute(String.join(CUSTOM_DELIMITER, CLUSTER_STATE_CUSTOM, customType), blobName); + } + + @Override + public InputStream serialize() throws IOException { + BytesStreamOutput outputStream = new BytesStreamOutput(); + if (shouldSerialize(outputStream, custom)) { + outputStream.writeNamedWriteable(custom); + } + return outputStream.bytes().streamInput(); + } + + @Override + public ClusterState.Custom deserialize(final InputStream inputStream) throws IOException { + NamedWriteableAwareStreamInput in = new NamedWriteableAwareStreamInput( + new BytesStreamInput(toBytes(Streams.readFully(inputStream))), + this.namedWriteableRegistry + ); + return in.readNamedWriteable(Custom.class); + } +} diff --git a/server/src/main/java/org/opensearch/gateway/remote/model/RemoteCoordinationMetadata.java b/server/src/main/java/org/opensearch/gateway/remote/model/RemoteCoordinationMetadata.java new file mode 100644 index 0000000000000..bdc8e1cf8ec30 --- /dev/null +++ b/server/src/main/java/org/opensearch/gateway/remote/model/RemoteCoordinationMetadata.java @@ -0,0 +1,90 @@ +/* + * 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.gateway.remote.model; + +import static org.opensearch.gateway.remote.RemoteClusterStateUtils.DELIMITER; +import static org.opensearch.gateway.remote.RemoteClusterStateUtils.METADATA_NAME_FORMAT; +import static org.opensearch.gateway.remote.RemoteGlobalMetadataManager.GLOBAL_METADATA_CURRENT_CODEC_VERSION; + +import java.io.IOException; +import java.io.InputStream; +import java.util.List; +import org.opensearch.cluster.coordination.CoordinationMetadata; +import org.opensearch.common.io.Streams; +import org.opensearch.common.remote.AbstractRemoteWritableBlobEntity; +import org.opensearch.common.remote.BlobPathParameters; +import org.opensearch.core.compress.Compressor; +import org.opensearch.core.xcontent.NamedXContentRegistry; +import org.opensearch.gateway.remote.ClusterMetadataManifest.UploadedMetadata; +import org.opensearch.gateway.remote.ClusterMetadataManifest.UploadedMetadataAttribute; +import org.opensearch.gateway.remote.RemoteClusterStateUtils; +import org.opensearch.index.remote.RemoteStoreUtils; +import org.opensearch.repositories.blobstore.ChecksumBlobStoreFormat; + +/** + * Wrapper class for uploading/downloading {@link CoordinationMetadata} to/from remote blob store + */ +public class RemoteCoordinationMetadata extends AbstractRemoteWritableBlobEntity { + + public static final String COORDINATION_METADATA = "coordination"; + public static final ChecksumBlobStoreFormat COORDINATION_METADATA_FORMAT = new ChecksumBlobStoreFormat<>( + "coordination", + METADATA_NAME_FORMAT, + CoordinationMetadata::fromXContent + ); + + private CoordinationMetadata coordinationMetadata; + private long metadataVersion; + + public RemoteCoordinationMetadata(final CoordinationMetadata coordinationMetadata, final long metadataVersion, final String clusterUUID, final Compressor compressor, final NamedXContentRegistry namedXContentRegistry) { + super(clusterUUID, compressor, namedXContentRegistry); + this.coordinationMetadata = coordinationMetadata; + this.metadataVersion = metadataVersion; + } + + public RemoteCoordinationMetadata(final String blobName, final String clusterUUID, final Compressor compressor, final NamedXContentRegistry namedXContentRegistry) { + super(clusterUUID, compressor, namedXContentRegistry); + this.blobName = blobName; + } + + @Override + public BlobPathParameters getBlobPathParameters() { + return new BlobPathParameters(List.of("global-metadata"), COORDINATION_METADATA); + } + + @Override + public String generateBlobFileName() { + // 123456789012_test-cluster/cluster-state/dsgYj10Nkso7/global-metadata/______ + String blobFileName = String.join( + DELIMITER, + getBlobPathParameters().getFilePrefix(), + RemoteStoreUtils.invertLong(metadataVersion), + RemoteStoreUtils.invertLong(System.currentTimeMillis()), + String.valueOf(GLOBAL_METADATA_CURRENT_CODEC_VERSION) + ); + this.blobFileName = blobFileName; + return blobFileName; + } + + @Override + public InputStream serialize() throws IOException { + return COORDINATION_METADATA_FORMAT.serialize(coordinationMetadata, generateBlobFileName(), getCompressor(), RemoteClusterStateUtils.FORMAT_PARAMS).streamInput(); + } + + @Override + public CoordinationMetadata deserialize(final InputStream inputStream) throws IOException { + return COORDINATION_METADATA_FORMAT.deserialize(blobName, getNamedXContentRegistry(), Streams.readFully(inputStream)); + } + + @Override + public UploadedMetadata getUploadedMetadata() { + assert blobName != null; + return new UploadedMetadataAttribute(COORDINATION_METADATA, blobName); + } +} diff --git a/server/src/main/java/org/opensearch/gateway/remote/model/RemoteCustomMetadata.java b/server/src/main/java/org/opensearch/gateway/remote/model/RemoteCustomMetadata.java new file mode 100644 index 0000000000000..e228c80e179fe --- /dev/null +++ b/server/src/main/java/org/opensearch/gateway/remote/model/RemoteCustomMetadata.java @@ -0,0 +1,106 @@ +/* + * 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.gateway.remote.model; + +import static org.opensearch.gateway.remote.RemoteClusterStateUtils.DELIMITER; +import static org.opensearch.gateway.remote.RemoteClusterStateUtils.GLOBAL_METADATA_PATH_TOKEN; +import static org.opensearch.gateway.remote.RemoteClusterStateUtils.METADATA_NAME_FORMAT; +import static org.opensearch.gateway.remote.RemoteGlobalMetadataManager.GLOBAL_METADATA_CURRENT_CODEC_VERSION; + +import java.io.IOException; +import java.io.InputStream; +import java.util.List; +import org.opensearch.cluster.metadata.Metadata; +import org.opensearch.cluster.metadata.Metadata.Custom; +import org.opensearch.common.io.Streams; +import org.opensearch.common.remote.AbstractRemoteWritableBlobEntity; +import org.opensearch.common.remote.BlobPathParameters; +import org.opensearch.core.compress.Compressor; +import org.opensearch.core.xcontent.NamedXContentRegistry; +import org.opensearch.gateway.remote.ClusterMetadataManifest.UploadedMetadata; +import org.opensearch.gateway.remote.ClusterMetadataManifest.UploadedMetadataAttribute; +import org.opensearch.gateway.remote.RemoteClusterStateUtils; +import org.opensearch.index.remote.RemoteStoreUtils; +import org.opensearch.repositories.blobstore.ChecksumBlobStoreFormat; + +/** + * Wrapper class for uploading/downloading {@link Custom} to/from remote blob store + */ +public class RemoteCustomMetadata extends AbstractRemoteWritableBlobEntity { + + public static final String CUSTOM_METADATA = "custom"; + public static final String CUSTOM_DELIMITER = "--"; + public final ChecksumBlobStoreFormat customBlobStoreFormat; + + private Custom custom; + private final String customType; + private long metadataVersion; + + public RemoteCustomMetadata(final Custom custom, final String customType, final long metadataVersion, final String clusterUUID, + Compressor compressor, NamedXContentRegistry namedXContentRegistry) { + super(clusterUUID, compressor, namedXContentRegistry); + this.custom = custom; + this.customType = customType; + this.metadataVersion = metadataVersion; + this.customBlobStoreFormat = new ChecksumBlobStoreFormat<>( + "custom", + METADATA_NAME_FORMAT, + (parser -> Metadata.Custom.fromXContent(parser, customType)) + ); + } + + public RemoteCustomMetadata(final String blobName, final String customType, final String clusterUUID, + final Compressor compressor, final NamedXContentRegistry namedXContentRegistry) { + super(clusterUUID, compressor, namedXContentRegistry); + this.blobName = blobName; + this.customType = customType; + this.customBlobStoreFormat = new ChecksumBlobStoreFormat<>( + "custom", + METADATA_NAME_FORMAT, + (parser -> Metadata.Custom.fromXContent(parser, customType)) + ); + } + + @Override + public BlobPathParameters getBlobPathParameters() { + String prefix = String.join(CUSTOM_DELIMITER, CUSTOM_METADATA, customType); + return new BlobPathParameters(List.of(GLOBAL_METADATA_PATH_TOKEN), prefix); + } + + @Override + public String generateBlobFileName() { + // 123456789012_test-cluster/cluster-state/dsgYj10Nkso7/global-metadata/______ + // + String blobFileName = String.join( + DELIMITER, + getBlobPathParameters().getFilePrefix(), + RemoteStoreUtils.invertLong(metadataVersion), + RemoteStoreUtils.invertLong(System.currentTimeMillis()), + String.valueOf(GLOBAL_METADATA_CURRENT_CODEC_VERSION) + ); + this.blobFileName = blobFileName; + return blobFileName; + } + + @Override + public InputStream serialize() throws IOException { + return customBlobStoreFormat.serialize(custom, generateBlobFileName(), getCompressor(), RemoteClusterStateUtils.FORMAT_PARAMS).streamInput(); + } + + @Override + public Custom deserialize(final InputStream inputStream) throws IOException { + return customBlobStoreFormat.deserialize(blobName, getNamedXContentRegistry(), Streams.readFully(inputStream)); + } + + @Override + public UploadedMetadata getUploadedMetadata() { + assert blobName != null; + return new UploadedMetadataAttribute(String.join(CUSTOM_DELIMITER, CUSTOM_METADATA, customType), blobName); + } +} diff --git a/server/src/main/java/org/opensearch/gateway/remote/model/RemoteDiscoveryNodes.java b/server/src/main/java/org/opensearch/gateway/remote/model/RemoteDiscoveryNodes.java new file mode 100644 index 0000000000000..d5f2bcac3f2a9 --- /dev/null +++ b/server/src/main/java/org/opensearch/gateway/remote/model/RemoteDiscoveryNodes.java @@ -0,0 +1,93 @@ +/* + * 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.gateway.remote.model; + +import static org.opensearch.core.common.bytes.BytesReference.toBytes; +import static org.opensearch.gateway.remote.RemoteClusterStateAttributesManager.CLUSTER_STATE_ATTRIBUTES_CURRENT_CODEC_VERSION; +import static org.opensearch.gateway.remote.RemoteClusterStateUtils.CLUSTER_STATE_EPHEMERAL_PATH_TOKEN; +import static org.opensearch.gateway.remote.RemoteClusterStateUtils.DELIMITER; +import static org.opensearch.gateway.remote.RemoteClusterStateUtils.METADATA_NAME_FORMAT; + +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.InputStream; +import java.util.List; +import org.opensearch.cluster.node.DiscoveryNodes; +import org.opensearch.common.io.Streams; +import org.opensearch.common.io.stream.BytesStreamOutput; +import org.opensearch.common.remote.AbstractRemoteWritableBlobEntity; +import org.opensearch.common.remote.BlobPathParameters; +import org.opensearch.core.common.bytes.BytesReference; +import org.opensearch.core.common.io.stream.BytesStreamInput; +import org.opensearch.core.compress.Compressor; +import org.opensearch.core.xcontent.NamedXContentRegistry; +import org.opensearch.gateway.remote.ClusterMetadataManifest.UploadedMetadata; +import org.opensearch.gateway.remote.ClusterMetadataManifest.UploadedMetadataAttribute; +import org.opensearch.gateway.remote.RemoteClusterStateUtils; +import org.opensearch.index.remote.RemoteStoreUtils; +import org.opensearch.repositories.blobstore.ChecksumBlobStoreFormat; + +/** + * Wrapper class for uploading/downloading {@link DiscoveryNodes} to/from remote blob store + */ +public class RemoteDiscoveryNodes extends AbstractRemoteWritableBlobEntity { + + public static final String DISCOVERY_NODES = "nodes"; + + private DiscoveryNodes discoveryNodes; + private long stateVersion; + + public RemoteDiscoveryNodes(final DiscoveryNodes discoveryNodes, final long stateVersion, final String clusterUUID, final Compressor compressor, final NamedXContentRegistry namedXContentRegistry) { + super(clusterUUID, compressor, namedXContentRegistry); + this.discoveryNodes = discoveryNodes; + this.stateVersion = stateVersion; + } + + public RemoteDiscoveryNodes(final String blobName, final String clusterUUID, final Compressor compressor, final NamedXContentRegistry namedXContentRegistry) { + super(clusterUUID, compressor, namedXContentRegistry); + this.blobName = blobName; + } + + @Override + public BlobPathParameters getBlobPathParameters() { + return new BlobPathParameters(List.of(CLUSTER_STATE_EPHEMERAL_PATH_TOKEN), DISCOVERY_NODES); + } + + @Override + public String generateBlobFileName() { + // 123456789012_test-cluster/cluster-state/dsgYj10Nkso7/ephemeral/______ + String blobFileName = String.join( + DELIMITER, + getBlobPathParameters().getFilePrefix(), + RemoteStoreUtils.invertLong(stateVersion), + RemoteStoreUtils.invertLong(System.currentTimeMillis()), + String.valueOf(CLUSTER_STATE_ATTRIBUTES_CURRENT_CODEC_VERSION) + ); + this.blobFileName = blobFileName; + return blobFileName; + } + + @Override + public UploadedMetadata getUploadedMetadata() { + assert blobName != null; + return new UploadedMetadataAttribute(DISCOVERY_NODES, blobName); + } + + @Override + public InputStream serialize() throws IOException { + BytesStreamOutput outputStream = new BytesStreamOutput(); + discoveryNodes.writeTo(outputStream); + return outputStream.bytes().streamInput(); + } + + @Override + public DiscoveryNodes deserialize(final InputStream inputStream) throws IOException { + return DiscoveryNodes.readFrom(new BytesStreamInput(toBytes(Streams.readFully(inputStream))), null); + } +} diff --git a/server/src/main/java/org/opensearch/gateway/remote/model/RemoteGlobalMetadata.java b/server/src/main/java/org/opensearch/gateway/remote/model/RemoteGlobalMetadata.java new file mode 100644 index 0000000000000..9149a542383c1 --- /dev/null +++ b/server/src/main/java/org/opensearch/gateway/remote/model/RemoteGlobalMetadata.java @@ -0,0 +1,65 @@ +/* + * 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.gateway.remote.model; + +import static org.opensearch.gateway.remote.RemoteClusterStateUtils.METADATA_NAME_FORMAT; + +import java.io.IOException; +import java.io.InputStream; +import org.opensearch.cluster.metadata.Metadata; +import org.opensearch.common.io.Streams; +import org.opensearch.common.remote.AbstractRemoteWritableBlobEntity; +import org.opensearch.common.remote.BlobPathParameters; +import org.opensearch.core.compress.Compressor; +import org.opensearch.core.xcontent.NamedXContentRegistry; +import org.opensearch.gateway.remote.ClusterMetadataManifest.UploadedMetadata; +import org.opensearch.repositories.blobstore.ChecksumBlobStoreFormat; + +public class RemoteGlobalMetadata extends AbstractRemoteWritableBlobEntity { + + public static final ChecksumBlobStoreFormat GLOBAL_METADATA_FORMAT = new ChecksumBlobStoreFormat<>( + "metadata", + METADATA_NAME_FORMAT, + Metadata::fromXContent + ); + + private Metadata metadata; + private final String blobName; + + public RemoteGlobalMetadata(final String blobName, final String clusterUUID, final Compressor compressor, + final NamedXContentRegistry namedXContentRegistry) { + super(clusterUUID, compressor, namedXContentRegistry); + this.blobName = blobName; + } + + @Override + public BlobPathParameters getBlobPathParameters() { + throw new UnsupportedOperationException(); + } + + @Override + public String generateBlobFileName() { + throw new UnsupportedOperationException(); + } + + @Override + public UploadedMetadata getUploadedMetadata() { + throw new UnsupportedOperationException(); + } + + @Override + public InputStream serialize() throws IOException { + throw new UnsupportedOperationException(); + } + + @Override + public Metadata deserialize(final InputStream inputStream) throws IOException { + return GLOBAL_METADATA_FORMAT.deserialize(blobName, getNamedXContentRegistry(), Streams.readFully(inputStream)); + } +} diff --git a/server/src/main/java/org/opensearch/gateway/remote/model/RemoteHashesOfConsistentSettings.java b/server/src/main/java/org/opensearch/gateway/remote/model/RemoteHashesOfConsistentSettings.java new file mode 100644 index 0000000000000..f8ef85616a578 --- /dev/null +++ b/server/src/main/java/org/opensearch/gateway/remote/model/RemoteHashesOfConsistentSettings.java @@ -0,0 +1,86 @@ +/* + * 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.gateway.remote.model; + +import static org.opensearch.core.common.bytes.BytesReference.toBytes; +import static org.opensearch.gateway.remote.RemoteClusterStateAttributesManager.CLUSTER_STATE_ATTRIBUTES_CURRENT_CODEC_VERSION; +import static org.opensearch.gateway.remote.RemoteClusterStateUtils.DELIMITER; +import static org.opensearch.gateway.remote.RemoteClusterStateUtils.GLOBAL_METADATA_PATH_TOKEN; +import static org.opensearch.gateway.remote.RemoteClusterStateUtils.METADATA_NAME_FORMAT; + +import java.io.IOException; +import java.io.InputStream; +import java.util.List; +import org.opensearch.cluster.metadata.DiffableStringMap; +import org.opensearch.common.io.Streams; +import org.opensearch.common.io.stream.BytesStreamOutput; +import org.opensearch.common.remote.AbstractRemoteWritableBlobEntity; +import org.opensearch.common.remote.BlobPathParameters; +import org.opensearch.core.common.io.stream.BytesStreamInput; +import org.opensearch.core.common.io.stream.StreamInput; +import org.opensearch.core.compress.Compressor; +import org.opensearch.core.xcontent.NamedXContentRegistry; +import org.opensearch.gateway.remote.ClusterMetadataManifest; +import org.opensearch.gateway.remote.RemoteClusterStateUtils; +import org.opensearch.index.remote.RemoteStoreUtils; +import org.opensearch.repositories.blobstore.ChecksumBlobStoreFormat; + +public class RemoteHashesOfConsistentSettings extends AbstractRemoteWritableBlobEntity { + public static final String HASHES_OF_CONSISTENT_SETTINGS = "hashes-of-consistent-settings"; + + private DiffableStringMap hashesOfConsistentSettings; + private long metadataVersion; + public RemoteHashesOfConsistentSettings(final DiffableStringMap hashesOfConsistentSettings, final long metadataVersion, final String clusterUUID, final Compressor compressor, final NamedXContentRegistry namedXContentRegistry) { + super(clusterUUID, compressor, namedXContentRegistry); + this.metadataVersion = metadataVersion; + this.hashesOfConsistentSettings = hashesOfConsistentSettings; + } + + public RemoteHashesOfConsistentSettings(final String blobName, final String clusterUUID, final Compressor compressor, final NamedXContentRegistry namedXContentRegistry) { + super(clusterUUID, compressor, namedXContentRegistry); + this.blobName = blobName; + } + + @Override + public BlobPathParameters getBlobPathParameters() { + return new BlobPathParameters(List.of(GLOBAL_METADATA_PATH_TOKEN), HASHES_OF_CONSISTENT_SETTINGS); + } + + @Override + public String generateBlobFileName() { + String blobFileName = String.join( + DELIMITER, + getBlobPathParameters().getFilePrefix(), + RemoteStoreUtils.invertLong(metadataVersion), + RemoteStoreUtils.invertLong(System.currentTimeMillis()), + String.valueOf(CLUSTER_STATE_ATTRIBUTES_CURRENT_CODEC_VERSION) + ); + this.blobFileName = blobFileName; + return blobFileName; + } + + @Override + public ClusterMetadataManifest.UploadedMetadata getUploadedMetadata() { + assert blobName != null; + return new ClusterMetadataManifest.UploadedMetadataAttribute(HASHES_OF_CONSISTENT_SETTINGS, blobName); + } + + @Override + public InputStream serialize() throws IOException { + BytesStreamOutput bytesStreamOutput = new BytesStreamOutput(); + hashesOfConsistentSettings.writeTo(bytesStreamOutput); + return bytesStreamOutput.bytes().streamInput(); + } + + @Override + public DiffableStringMap deserialize(final InputStream inputStream) throws IOException { + StreamInput in = new BytesStreamInput(toBytes(Streams.readFully(inputStream))); + return DiffableStringMap.readFrom(in); + } +} diff --git a/server/src/main/java/org/opensearch/gateway/remote/model/RemoteIndexMetadata.java b/server/src/main/java/org/opensearch/gateway/remote/model/RemoteIndexMetadata.java new file mode 100644 index 0000000000000..d77831aaaea6f --- /dev/null +++ b/server/src/main/java/org/opensearch/gateway/remote/model/RemoteIndexMetadata.java @@ -0,0 +1,92 @@ +/* + * 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.gateway.remote.model; + + +import static org.opensearch.gateway.remote.RemoteClusterStateUtils.METADATA_NAME_PLAIN_FORMAT; + +import java.io.IOException; +import java.io.InputStream; +import java.util.List; +import org.opensearch.cluster.metadata.IndexMetadata; +import org.opensearch.common.io.Streams; +import org.opensearch.common.remote.AbstractRemoteWritableBlobEntity; +import org.opensearch.common.remote.BlobPathParameters; +import org.opensearch.core.compress.Compressor; +import org.opensearch.core.xcontent.NamedXContentRegistry; +import org.opensearch.gateway.remote.ClusterMetadataManifest.UploadedIndexMetadata; +import org.opensearch.gateway.remote.ClusterMetadataManifest.UploadedMetadata; +import org.opensearch.gateway.remote.RemoteClusterStateUtils; +import org.opensearch.index.remote.RemoteStoreUtils; +import org.opensearch.repositories.blobstore.ChecksumBlobStoreFormat; + +/** + * Wrapper class for uploading/downloading {@link IndexMetadata} to/from remote blob store + */ +public class RemoteIndexMetadata extends AbstractRemoteWritableBlobEntity { + + public static final int INDEX_METADATA_CURRENT_CODEC_VERSION = 1; + + public static final ChecksumBlobStoreFormat INDEX_METADATA_FORMAT = new ChecksumBlobStoreFormat<>( + "index-metadata", + METADATA_NAME_PLAIN_FORMAT, + IndexMetadata::fromXContent + ); + public static final String INDEX_PATH_TOKEN = "index"; + + private IndexMetadata indexMetadata; + + public RemoteIndexMetadata(final IndexMetadata indexMetadata, final String clusterUUID, final Compressor compressor, final NamedXContentRegistry namedXContentRegistry) { + super(clusterUUID, compressor, namedXContentRegistry); + this.indexMetadata = indexMetadata; + } + + public RemoteIndexMetadata(final String blobName, final String clusterUUID, final Compressor compressor, final NamedXContentRegistry namedXContentRegistry) { + super(clusterUUID, compressor, namedXContentRegistry); + this.blobName = blobName; + } + + @Override + public BlobPathParameters getBlobPathParameters() { + return new BlobPathParameters(List.of(INDEX_PATH_TOKEN, indexMetadata.getIndexUUID()), "metadata"); + } + + @Override + public String generateBlobFileName() { + String blobFileName = String.join( + RemoteClusterStateUtils.DELIMITER, + getBlobPathParameters().getFilePrefix(), + RemoteStoreUtils.invertLong(indexMetadata.getVersion()), + RemoteStoreUtils.invertLong(System.currentTimeMillis()), + String.valueOf(INDEX_METADATA_CURRENT_CODEC_VERSION) // Keep the codec version at last place only, during reads we read last + // place to determine codec version. + ); + this.blobFileName = blobFileName; + return blobFileName; + } + + @Override + public UploadedMetadata getUploadedMetadata() { + assert blobName != null; + return new UploadedIndexMetadata(indexMetadata.getIndexName(), indexMetadata.getIndexUUID(), blobName); + } + + @Override + public InputStream serialize() throws IOException { + return INDEX_METADATA_FORMAT.serialize(indexMetadata, generateBlobFileName(), getCompressor(), RemoteClusterStateUtils.FORMAT_PARAMS) + .streamInput(); + } + + @Override + public IndexMetadata deserialize(final InputStream inputStream) throws IOException { + // Blob name parameter is redundant + return INDEX_METADATA_FORMAT.deserialize(blobName, getNamedXContentRegistry(), Streams.readFully(inputStream)); + } + +} diff --git a/server/src/main/java/org/opensearch/gateway/remote/model/RemotePersistentSettingsMetadata.java b/server/src/main/java/org/opensearch/gateway/remote/model/RemotePersistentSettingsMetadata.java new file mode 100644 index 0000000000000..dd96afc457fa8 --- /dev/null +++ b/server/src/main/java/org/opensearch/gateway/remote/model/RemotePersistentSettingsMetadata.java @@ -0,0 +1,91 @@ +/* + * 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.gateway.remote.model; + +import static org.opensearch.gateway.remote.RemoteClusterStateUtils.DELIMITER; +import static org.opensearch.gateway.remote.RemoteClusterStateUtils.METADATA_NAME_FORMAT; +import static org.opensearch.gateway.remote.RemoteGlobalMetadataManager.GLOBAL_METADATA_CURRENT_CODEC_VERSION; + +import java.io.IOException; +import java.io.InputStream; +import java.util.List; +import org.opensearch.common.io.Streams; +import org.opensearch.common.remote.AbstractRemoteWritableBlobEntity; +import org.opensearch.common.remote.BlobPathParameters; +import org.opensearch.common.settings.Settings; +import org.opensearch.core.compress.Compressor; +import org.opensearch.core.xcontent.NamedXContentRegistry; +import org.opensearch.gateway.remote.ClusterMetadataManifest.UploadedMetadata; +import org.opensearch.gateway.remote.ClusterMetadataManifest.UploadedMetadataAttribute; +import org.opensearch.gateway.remote.RemoteClusterStateUtils; +import org.opensearch.index.remote.RemoteStoreUtils; +import org.opensearch.repositories.blobstore.ChecksumBlobStoreFormat; + +/** + * Wrapper class for uploading/downloading persistent {@link Settings} to/from remote blob store + */ +public class RemotePersistentSettingsMetadata extends AbstractRemoteWritableBlobEntity { + + public static final String SETTING_METADATA = "settings"; + + public static final ChecksumBlobStoreFormat SETTINGS_METADATA_FORMAT = new ChecksumBlobStoreFormat<>( + "settings", + METADATA_NAME_FORMAT, + Settings::fromXContent + ); + + private Settings persistentSettings; + private long metadataVersion; + + public RemotePersistentSettingsMetadata(final Settings settings, final long metadataVersion, final String clusterUUID, final Compressor compressor, final NamedXContentRegistry namedXContentRegistry) { + super(clusterUUID, compressor, namedXContentRegistry); + this.persistentSettings = settings; + this.metadataVersion = metadataVersion; + } + + public RemotePersistentSettingsMetadata(final String blobName, final String clusterUUID, final Compressor compressor, final NamedXContentRegistry namedXContentRegistry) { + super(clusterUUID, compressor, namedXContentRegistry); + this.blobName = blobName; + } + + @Override + public BlobPathParameters getBlobPathParameters() { + return new BlobPathParameters(List.of("global-metadata"), SETTING_METADATA); + } + + @Override + public String generateBlobFileName() { + String blobFileName = String.join( + DELIMITER, + getBlobPathParameters().getFilePrefix(), + RemoteStoreUtils.invertLong(metadataVersion), + RemoteStoreUtils.invertLong(System.currentTimeMillis()), + String.valueOf(GLOBAL_METADATA_CURRENT_CODEC_VERSION) + ); + this.blobFileName = blobFileName; + return blobFileName; + } + + @Override + public InputStream serialize() throws IOException { + return SETTINGS_METADATA_FORMAT.serialize(persistentSettings, generateBlobFileName(), getCompressor(), RemoteClusterStateUtils.FORMAT_PARAMS) + .streamInput(); + } + + @Override + public Settings deserialize(final InputStream inputStream) throws IOException { + return SETTINGS_METADATA_FORMAT.deserialize(blobName, getNamedXContentRegistry(), Streams.readFully(inputStream)); + } + + @Override + public UploadedMetadata getUploadedMetadata() { + assert blobName != null; + return new UploadedMetadataAttribute(SETTING_METADATA, blobName); + } +} diff --git a/server/src/main/java/org/opensearch/gateway/remote/model/RemoteReadResult.java b/server/src/main/java/org/opensearch/gateway/remote/model/RemoteReadResult.java new file mode 100644 index 0000000000000..adee09eaeffef --- /dev/null +++ b/server/src/main/java/org/opensearch/gateway/remote/model/RemoteReadResult.java @@ -0,0 +1,39 @@ +/* + * 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.gateway.remote.model; + +import org.opensearch.core.xcontent.ToXContent; + +/** + * Container class for entity read from remote store + */ +public class RemoteReadResult { + + ToXContent obj; + String component; + String componentName; + + public RemoteReadResult(ToXContent obj, String component, String componentName) { + this.obj = obj; + this.component = component; + this.componentName = componentName; + } + + public ToXContent getObj() { + return obj; + } + + public String getComponent() { + return component; + } + + public String getComponentName() { + return componentName; + } +} diff --git a/server/src/main/java/org/opensearch/gateway/remote/model/RemoteTemplatesMetadata.java b/server/src/main/java/org/opensearch/gateway/remote/model/RemoteTemplatesMetadata.java new file mode 100644 index 0000000000000..c558e822191b1 --- /dev/null +++ b/server/src/main/java/org/opensearch/gateway/remote/model/RemoteTemplatesMetadata.java @@ -0,0 +1,92 @@ +/* + * 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.gateway.remote.model; + +import static org.opensearch.gateway.remote.RemoteClusterStateUtils.DELIMITER; +import static org.opensearch.gateway.remote.RemoteClusterStateUtils.METADATA_NAME_FORMAT; +import static org.opensearch.gateway.remote.RemoteGlobalMetadataManager.GLOBAL_METADATA_CURRENT_CODEC_VERSION; + +import java.io.IOException; +import java.io.InputStream; +import java.util.List; +import org.opensearch.cluster.metadata.TemplatesMetadata; +import org.opensearch.common.io.Streams; +import org.opensearch.common.remote.AbstractRemoteWritableBlobEntity; +import org.opensearch.common.remote.BlobPathParameters; +import org.opensearch.core.compress.Compressor; +import org.opensearch.core.xcontent.NamedXContentRegistry; +import org.opensearch.gateway.remote.ClusterMetadataManifest.UploadedMetadata; +import org.opensearch.gateway.remote.ClusterMetadataManifest.UploadedMetadataAttribute; +import org.opensearch.gateway.remote.RemoteClusterStateUtils; +import org.opensearch.index.remote.RemoteStoreUtils; +import org.opensearch.repositories.blobstore.ChecksumBlobStoreFormat; + +/** + * Wrapper class for uploading/downloading {@link TemplatesMetadata} to/from remote blob store + */ +public class RemoteTemplatesMetadata extends AbstractRemoteWritableBlobEntity { + + public static final String TEMPLATES_METADATA = "templates"; + + public static final ChecksumBlobStoreFormat TEMPLATES_METADATA_FORMAT = new ChecksumBlobStoreFormat<>( + "templates", + METADATA_NAME_FORMAT, + TemplatesMetadata::fromXContent + ); + private TemplatesMetadata templatesMetadata; + private long metadataVersion; + + public RemoteTemplatesMetadata(final TemplatesMetadata templatesMetadata, final long metadataVersion, final String clusterUUID, final Compressor compressor, final NamedXContentRegistry namedXContentRegistry) { + super(clusterUUID, compressor, namedXContentRegistry); + this.templatesMetadata = templatesMetadata; + this.metadataVersion = metadataVersion; + } + + public RemoteTemplatesMetadata(final String blobName, final String clusterUUID, final Compressor compressor, final NamedXContentRegistry namedXContentRegistry) { + super(clusterUUID, compressor, namedXContentRegistry); + this.blobName = blobName; + } + + @Override + public BlobPathParameters getBlobPathParameters() { + return new BlobPathParameters(List.of("global-metadata"), TEMPLATES_METADATA); + } + + @Override + public String generateBlobFileName() { + // 123456789012_test-cluster/cluster-state/dsgYj10Nkso7/global-metadata/______ + // + String blobFileName = String.join( + DELIMITER, + getBlobPathParameters().getFilePrefix(), + RemoteStoreUtils.invertLong(metadataVersion), + RemoteStoreUtils.invertLong(System.currentTimeMillis()), + String.valueOf(GLOBAL_METADATA_CURRENT_CODEC_VERSION) + ); + this.blobFileName = blobFileName; + return blobFileName; + } + + @Override + public InputStream serialize() throws IOException { + return TEMPLATES_METADATA_FORMAT.serialize(templatesMetadata, generateBlobFileName(), getCompressor(), RemoteClusterStateUtils.FORMAT_PARAMS) + .streamInput(); + } + + @Override + public TemplatesMetadata deserialize(final InputStream inputStream) throws IOException { + return TEMPLATES_METADATA_FORMAT.deserialize(blobName, getNamedXContentRegistry(), Streams.readFully(inputStream)); + } + + @Override + public UploadedMetadata getUploadedMetadata() { + assert blobName != null; + return new UploadedMetadataAttribute(TEMPLATES_METADATA, blobName); + } +} diff --git a/server/src/main/java/org/opensearch/gateway/remote/model/RemoteTransientSettingsMetadata.java b/server/src/main/java/org/opensearch/gateway/remote/model/RemoteTransientSettingsMetadata.java new file mode 100644 index 0000000000000..957a1b3977aa4 --- /dev/null +++ b/server/src/main/java/org/opensearch/gateway/remote/model/RemoteTransientSettingsMetadata.java @@ -0,0 +1,91 @@ +/* + * 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.gateway.remote.model; + +import static org.opensearch.gateway.remote.RemoteClusterStateUtils.DELIMITER; +import static org.opensearch.gateway.remote.RemoteClusterStateUtils.METADATA_NAME_FORMAT; +import static org.opensearch.gateway.remote.RemoteGlobalMetadataManager.GLOBAL_METADATA_CURRENT_CODEC_VERSION; + +import java.io.IOException; +import java.io.InputStream; +import java.util.List; +import org.opensearch.common.io.Streams; +import org.opensearch.common.remote.AbstractRemoteWritableBlobEntity; +import org.opensearch.common.remote.BlobPathParameters; +import org.opensearch.common.settings.Settings; +import org.opensearch.core.compress.Compressor; +import org.opensearch.core.xcontent.NamedXContentRegistry; +import org.opensearch.gateway.remote.ClusterMetadataManifest.UploadedMetadata; +import org.opensearch.gateway.remote.ClusterMetadataManifest.UploadedMetadataAttribute; +import org.opensearch.gateway.remote.RemoteClusterStateUtils; +import org.opensearch.index.remote.RemoteStoreUtils; +import org.opensearch.repositories.blobstore.ChecksumBlobStoreFormat; + +/** + * Wrapper class for uploading/downloading transient {@link Settings} to/from remote blob store + */ +public class RemoteTransientSettingsMetadata extends AbstractRemoteWritableBlobEntity { + + public static final String TRANSIENT_SETTING_METADATA = "transient-settings"; + + public static final ChecksumBlobStoreFormat SETTINGS_METADATA_FORMAT = new ChecksumBlobStoreFormat<>( + "transient-settings", + METADATA_NAME_FORMAT, + Settings::fromXContent + ); + + private Settings transientSettings; + private long metadataVersion; + + public RemoteTransientSettingsMetadata(final Settings transientSettings, final long metadataVersion, final String clusterUUID, final Compressor compressor, final NamedXContentRegistry namedXContentRegistry) { + super(clusterUUID, compressor, namedXContentRegistry); + this.transientSettings = transientSettings; + this.metadataVersion = metadataVersion; + } + + public RemoteTransientSettingsMetadata(final String blobName, final String clusterUUID, final Compressor compressor, final NamedXContentRegistry namedXContentRegistry) { + super(clusterUUID, compressor, namedXContentRegistry); + this.blobName = blobName; + } + + @Override + public BlobPathParameters getBlobPathParameters() { + return new BlobPathParameters(List.of("global-metadata"), TRANSIENT_SETTING_METADATA); + } + + @Override + public String generateBlobFileName() { + String blobFileName = String.join( + DELIMITER, + getBlobPathParameters().getFilePrefix(), + RemoteStoreUtils.invertLong(metadataVersion), + RemoteStoreUtils.invertLong(System.currentTimeMillis()), + String.valueOf(GLOBAL_METADATA_CURRENT_CODEC_VERSION) + ); + this.blobFileName = blobFileName; + return blobFileName; + } + + @Override + public InputStream serialize() throws IOException { + return SETTINGS_METADATA_FORMAT.serialize(transientSettings, generateBlobFileName(), getCompressor(), RemoteClusterStateUtils.FORMAT_PARAMS) + .streamInput(); + } + + @Override + public Settings deserialize(final InputStream inputStream) throws IOException { + return SETTINGS_METADATA_FORMAT.deserialize(blobName, getNamedXContentRegistry(), Streams.readFully(inputStream)); + } + + @Override + public UploadedMetadata getUploadedMetadata() { + assert blobName != null; + return new UploadedMetadataAttribute(TRANSIENT_SETTING_METADATA, blobName); + } +} diff --git a/server/src/main/java/org/opensearch/gateway/remote/model/RemoteUploadDetails.java b/server/src/main/java/org/opensearch/gateway/remote/model/RemoteUploadDetails.java new file mode 100644 index 0000000000000..100280c454e43 --- /dev/null +++ b/server/src/main/java/org/opensearch/gateway/remote/model/RemoteUploadDetails.java @@ -0,0 +1,30 @@ +/* + * 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.gateway.remote.model; + +import org.opensearch.gateway.remote.ClusterMetadataManifest; + +public class RemoteUploadDetails { + + private final ClusterMetadataManifest clusterMetadataManifest; + private final String manifestFileName; + + public RemoteUploadDetails(final ClusterMetadataManifest manifest, final String manifestFileName) { + this.clusterMetadataManifest = manifest; + this.manifestFileName = manifestFileName; + } + + public ClusterMetadataManifest getClusterMetadataManifest() { + return clusterMetadataManifest; + } + + public String getManifestFileName() { + return manifestFileName; + } +} diff --git a/server/src/main/java/org/opensearch/gateway/remote/routingtable/IndexRoutingTableHeader.java b/server/src/main/java/org/opensearch/gateway/remote/routingtable/IndexRoutingTableHeader.java new file mode 100644 index 0000000000000..bc99fea9b5c09 --- /dev/null +++ b/server/src/main/java/org/opensearch/gateway/remote/routingtable/IndexRoutingTableHeader.java @@ -0,0 +1,79 @@ +/* + * 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.gateway.remote.routingtable; + +import org.apache.lucene.codecs.CodecUtil; +import org.apache.lucene.index.CorruptIndexException; +import org.apache.lucene.index.IndexFormatTooNewException; +import org.apache.lucene.index.IndexFormatTooOldException; +import org.apache.lucene.store.InputStreamDataInput; +import org.apache.lucene.store.OutputStreamDataOutput; +import org.opensearch.core.common.io.stream.StreamInput; +import org.opensearch.core.common.io.stream.StreamOutput; + +import java.io.EOFException; +import java.io.IOException; + +/** + * The stored header information for the individual index routing table + */ +public class IndexRoutingTableHeader { + + public static final String INDEX_ROUTING_HEADER_CODEC = "index_routing_header_codec"; + public static final int INITIAL_VERSION = 1; + public static final int CURRENT_VERSION = INITIAL_VERSION; + private final String indexName; + + public IndexRoutingTableHeader(String indexName) { + this.indexName = indexName; + } + + /** + * Reads the contents on the byte array into the corresponding {@link IndexRoutingTableHeader} + * + * @param in + * @return IndexRoutingTableHeader + * @throws IOException + */ + public static IndexRoutingTableHeader read(StreamInput in) throws IOException { + try { + readHeaderVersion(in); + final String name = in.readString(); + return new IndexRoutingTableHeader(name); + } catch (EOFException e) { + throw new IOException("index routing header truncated", e); + } + } + + static int readHeaderVersion(final StreamInput in) throws IOException { + final int version; + try { + version = CodecUtil.checkHeader(new InputStreamDataInput(in), INDEX_ROUTING_HEADER_CODEC, INITIAL_VERSION, CURRENT_VERSION); + } catch (CorruptIndexException | IndexFormatTooOldException | IndexFormatTooNewException e) { + throw new IOException("index routing table header corrupted", e); + } + return version; + } + + /** + * Returns the bytes reference for the {@link IndexRoutingTableHeader} + * + * @throws IOException + */ + public void write(StreamOutput out) throws IOException { + CodecUtil.writeHeader(new OutputStreamDataOutput(out), INDEX_ROUTING_HEADER_CODEC, CURRENT_VERSION); + out.writeString(indexName); + out.flush(); + } + + public String getIndexName() { + return indexName; + } + +} diff --git a/server/src/main/java/org/opensearch/gateway/remote/routingtable/IndexRoutingTableInputStream.java b/server/src/main/java/org/opensearch/gateway/remote/routingtable/IndexRoutingTableInputStream.java new file mode 100644 index 0000000000000..d4e2594bf153c --- /dev/null +++ b/server/src/main/java/org/opensearch/gateway/remote/routingtable/IndexRoutingTableInputStream.java @@ -0,0 +1,173 @@ +/* + * 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.gateway.remote.routingtable; + +import org.opensearch.cluster.routing.IndexRoutingTable; +import org.opensearch.cluster.routing.IndexShardRoutingTable; +import org.opensearch.common.io.stream.BufferedChecksumStreamOutput; +import org.opensearch.common.io.stream.BytesStreamOutput; +import org.opensearch.core.common.bytes.BytesReference; + +import java.io.IOException; +import java.io.InputStream; +import java.util.Iterator; + +public class IndexRoutingTableInputStream extends InputStream { + + /** + * The buffer where data is stored. + */ + protected byte[] buf; + + /** + * The number of valid bytes in the buffer. + */ + protected int count; + + /** + * The buffer left over from the last fill + */ + protected byte[] leftOverBuf; + + /** + * The mark position + */ + protected int markPos = -1; + + /** + * The read limit + */ + protected int markLimit; + + /** + * The position + */ + protected int pos; + + private static final int BUFFER_SIZE = 8192; + + private final IndexRoutingTableHeader indexRoutingTableHeader; + private final Iterator shardIter; + private final BytesStreamOutput bytesStreamOutput; + private final BufferedChecksumStreamOutput out; + + public IndexRoutingTableInputStream(IndexRoutingTable indexRoutingTable) throws IOException { + this(indexRoutingTable, BUFFER_SIZE); + } + + public IndexRoutingTableInputStream(IndexRoutingTable indexRoutingTable, int size) throws IOException { + this.buf = new byte[size]; + this.shardIter = indexRoutingTable.iterator(); + this.indexRoutingTableHeader = new IndexRoutingTableHeader(indexRoutingTable.getIndex().getName()); + this.bytesStreamOutput = new BytesStreamOutput(); + this.out = new BufferedChecksumStreamOutput(bytesStreamOutput); + + initialFill(indexRoutingTable.shards().size()); + } + + @Override + public int read() throws IOException { + if (pos >= count) { + maybeResizeAndFill(); + if (pos >= count) return -1; + } + return buf[pos++] & 0xff; + } + + private void initialFill(int shardCount) throws IOException { + indexRoutingTableHeader.write(out); + out.writeVInt(shardCount); + + System.arraycopy(bytesStreamOutput.bytes().toBytesRef().bytes, 0, buf, 0, bytesStreamOutput.bytes().length()); + count = bytesStreamOutput.bytes().length(); + bytesStreamOutput.reset(); + fill(buf); + } + + private void fill(byte[] buf) throws IOException { + if (leftOverBuf != null) { + if (leftOverBuf.length > buf.length - count) { + // leftOverBuf has more content than length of buf, so we need to copy only based on buf length and keep the remaining in + // leftOverBuf. + System.arraycopy(leftOverBuf, 0, buf, count, buf.length - count); + byte[] tempLeftOverBuffer = new byte[leftOverBuf.length - (buf.length - count)]; + System.arraycopy(leftOverBuf, buf.length - count, tempLeftOverBuffer, 0, leftOverBuf.length - (buf.length - count)); + leftOverBuf = tempLeftOverBuffer; + count = buf.length - count; + } else { + System.arraycopy(leftOverBuf, 0, buf, count, leftOverBuf.length); + count += leftOverBuf.length; + leftOverBuf = null; + } + } + + if (count < buf.length && shardIter.hasNext()) { + IndexShardRoutingTable next = shardIter.next(); + IndexShardRoutingTable.Builder.writeTo(next, out); + // Add checksum for the file after all shards are done + if (!shardIter.hasNext()) { + out.writeLong(out.getChecksum()); + } + out.flush(); + BytesReference bytesRef = bytesStreamOutput.bytes(); + bytesStreamOutput.reset(); + + if (bytesRef.length() < buf.length - count) { + System.arraycopy(bytesRef.toBytesRef().bytes, 0, buf, count, bytesRef.length()); + count += bytesRef.length(); + leftOverBuf = null; + } else { + System.arraycopy(bytesRef.toBytesRef().bytes, 0, buf, count, buf.length - count); + leftOverBuf = new byte[bytesRef.length() - (buf.length - count)]; + System.arraycopy(bytesRef.toBytesRef().bytes, buf.length - count, leftOverBuf, 0, bytesRef.length() - (buf.length - count)); + count = buf.length; + } + } + } + + private void maybeResizeAndFill() throws IOException { + byte[] buffer = buf; + if (markPos == -1) pos = 0; /* no mark: throw away the buffer */ + else if (pos >= buffer.length) { /* no room left in buffer */ + if (markPos > 0) { /* can throw away early part of the buffer */ + int sz = pos - markPos; + System.arraycopy(buffer, markPos, buffer, 0, sz); + pos = sz; + markPos = 0; + } else if (buffer.length >= markLimit) { + markPos = -1; /* buffer got too big, invalidate mark */ + pos = 0; /* drop buffer contents */ + } else { /* grow buffer */ + int nsz = markLimit + 1; + byte[] nbuf = new byte[nsz]; + System.arraycopy(buffer, 0, nbuf, 0, pos); + buffer = nbuf; + } + } + count = pos; + fill(buffer); + } + + @Override + public void mark(int readlimit) { + markLimit = readlimit; + markPos = pos; + } + + @Override + public boolean markSupported() { + return true; + } + + @Override + public void reset() throws IOException { + if (markPos < 0) throw new IOException("Resetting to invalid mark"); + pos = markPos; + } +} diff --git a/server/src/main/java/org/opensearch/gateway/remote/routingtable/IndexRoutingTableInputStreamReader.java b/server/src/main/java/org/opensearch/gateway/remote/routingtable/IndexRoutingTableInputStreamReader.java new file mode 100644 index 0000000000000..e2b4f5da5f6e9 --- /dev/null +++ b/server/src/main/java/org/opensearch/gateway/remote/routingtable/IndexRoutingTableInputStreamReader.java @@ -0,0 +1,70 @@ +/* + * 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.gateway.remote.routingtable; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.opensearch.cluster.routing.IndexRoutingTable; +import org.opensearch.cluster.routing.IndexShardRoutingTable; +import org.opensearch.common.io.stream.BufferedChecksumStreamInput; +import org.opensearch.core.common.io.stream.InputStreamStreamInput; +import org.opensearch.core.common.io.stream.StreamInput; +import org.opensearch.core.index.Index; + +import java.io.EOFException; +import java.io.IOException; +import java.io.InputStream; + +public class IndexRoutingTableInputStreamReader { + + private final StreamInput streamInput; + + private static final Logger logger = LogManager.getLogger(IndexRoutingTableInputStreamReader.class); + + public IndexRoutingTableInputStreamReader(InputStream inputStream) throws IOException { + streamInput = new InputStreamStreamInput(inputStream); + } + + public IndexRoutingTable readIndexRoutingTable(Index index) throws IOException { + try { + try (BufferedChecksumStreamInput in = new BufferedChecksumStreamInput(streamInput, "assertion")) { + // Read the Table Header first and confirm the index + IndexRoutingTableHeader indexRoutingTableHeader = IndexRoutingTableHeader.read(in); + assert indexRoutingTableHeader.getIndexName().equals(index.getName()); + + int numberOfShardRouting = in.readVInt(); + logger.debug("Number of Index Routing Table {}", numberOfShardRouting); + IndexRoutingTable.Builder indicesRoutingTable = IndexRoutingTable.builder(index); + for (int idx = 0; idx < numberOfShardRouting; idx++) { + IndexShardRoutingTable indexShardRoutingTable = IndexShardRoutingTable.Builder.readFrom(in); + logger.debug("Index Shard Routing Table reading {}", indexShardRoutingTable); + indicesRoutingTable.addIndexShard(indexShardRoutingTable); + + } + verifyCheckSum(in); + return indicesRoutingTable.build(); + } + } catch (EOFException e) { + throw new IOException("Indices Routing table is corrupted", e); + } + } + + private void verifyCheckSum(BufferedChecksumStreamInput in) throws IOException { + long expectedChecksum = in.getChecksum(); + long readChecksum = in.readLong(); + if (readChecksum != expectedChecksum) { + throw new IOException( + "checksum verification failed - expected: 0x" + + Long.toHexString(expectedChecksum) + + ", got: 0x" + + Long.toHexString(readChecksum) + ); + } + } +} diff --git a/server/src/main/java/org/opensearch/index/recovery/RemoteStoreRestoreService.java b/server/src/main/java/org/opensearch/index/recovery/RemoteStoreRestoreService.java index fe90f24b0f544..e441e150e2acd 100644 --- a/server/src/main/java/org/opensearch/index/recovery/RemoteStoreRestoreService.java +++ b/server/src/main/java/org/opensearch/index/recovery/RemoteStoreRestoreService.java @@ -152,7 +152,7 @@ public RemoteRestoreResult restore( throw new IllegalArgumentException("clusterUUID to restore from should be different from current cluster UUID"); } logger.info("Restoring cluster state from remote store from cluster UUID : [{}]", restoreClusterUUID); - remoteState = remoteClusterStateService.getLatestClusterState(currentState.getClusterName().value(), restoreClusterUUID); + remoteState = remoteClusterStateService.getLatestClusterState(currentState.getClusterName().value(), restoreClusterUUID, false); remoteState.getMetadata().getIndices().values().forEach(indexMetadata -> { indexMetadataMap.put(indexMetadata.getIndex().getName(), new Tuple<>(true, indexMetadata)); }); diff --git a/server/src/main/java/org/opensearch/index/remote/RemoteIndexPathUploader.java b/server/src/main/java/org/opensearch/index/remote/RemoteIndexPathUploader.java index d736a82d57a7c..ffdc4729d7f21 100644 --- a/server/src/main/java/org/opensearch/index/remote/RemoteIndexPathUploader.java +++ b/server/src/main/java/org/opensearch/index/remote/RemoteIndexPathUploader.java @@ -23,7 +23,7 @@ import org.opensearch.core.action.ActionListener; import org.opensearch.core.index.Index; import org.opensearch.gateway.remote.IndexMetadataUploadListener; -import org.opensearch.gateway.remote.RemoteClusterStateService.RemoteStateTransferException; +import static org.opensearch.gateway.remote.RemoteClusterStateUtils.RemoteStateTransferException; import org.opensearch.index.remote.RemoteStoreEnums.PathType; import org.opensearch.node.Node; import org.opensearch.node.remotestore.RemoteStoreNodeAttribute; @@ -45,7 +45,7 @@ import java.util.function.Supplier; import java.util.stream.Collectors; -import static org.opensearch.gateway.remote.RemoteClusterStateService.INDEX_METADATA_UPLOAD_TIMEOUT_SETTING; +import static org.opensearch.gateway.remote.RemoteIndexMetadataManager.INDEX_METADATA_UPLOAD_TIMEOUT_SETTING; import static org.opensearch.index.remote.RemoteIndexPath.COMBINED_PATH; import static org.opensearch.index.remote.RemoteIndexPath.SEGMENT_PATH; import static org.opensearch.index.remote.RemoteIndexPath.TRANSLOG_PATH; diff --git a/server/src/main/java/org/opensearch/index/remote/RemoteStoreEnums.java b/server/src/main/java/org/opensearch/index/remote/RemoteStoreEnums.java index c1ac74724e405..78361079c9176 100644 --- a/server/src/main/java/org/opensearch/index/remote/RemoteStoreEnums.java +++ b/server/src/main/java/org/opensearch/index/remote/RemoteStoreEnums.java @@ -111,13 +111,20 @@ boolean requiresHashAlgorithm() { @Override public BlobPath generatePath(PathInput pathInput, PathHashAlgorithm hashAlgorithm) { assert Objects.nonNull(hashAlgorithm) : "hashAlgorithm is expected to be non-null"; - return BlobPath.cleanPath() + BlobPath path = BlobPath.cleanPath() .add(hashAlgorithm.hash(pathInput)) .add(pathInput.basePath()) - .add(pathInput.indexUUID()) - .add(pathInput.shardId()) - .add(pathInput.dataCategory().getName()) - .add(pathInput.dataType().getName()); + .add(pathInput.indexUUID()); + if (pathInput.shardId() != null) { + path.add(pathInput.shardId()); + } + if(pathInput.dataCategory() != null){ + path.add(pathInput.dataCategory().getName()); + } + if(pathInput.dataType() != null ) { + path.add(pathInput.dataType().getName()); + } + return path; } @Override @@ -188,11 +195,11 @@ public static PathType fromCode(int code) { public BlobPath path(PathInput pathInput, PathHashAlgorithm hashAlgorithm) { DataCategory dataCategory = pathInput.dataCategory(); DataType dataType = pathInput.dataType(); - assert dataCategory.isSupportedDataType(dataType) : "category:" - + dataCategory - + " type:" - + dataType - + " are not supported together"; +// assert dataCategory.isSupportedDataType(dataType) : "category:" +// + dataCategory +// + " type:" +// + dataType +// + " are not supported together"; return generatePath(pathInput, hashAlgorithm); } @@ -227,8 +234,7 @@ public enum PathHashAlgorithm { FNV_1A_BASE64(0) { @Override String hash(PathInput pathInput) { - String input = pathInput.indexUUID() + pathInput.shardId() + pathInput.dataCategory().getName() + pathInput.dataType() - .getName(); + String input = pathInput.indexUUID() + pathInput.shardId() + (pathInput.dataCategory() != null ? pathInput.dataCategory().getName(): "" )+ (pathInput.dataType()!= null ? pathInput.dataType().getName(): ""); long hash = FNV1a.hash64(input); return longToUrlBase64(hash); } diff --git a/server/src/main/java/org/opensearch/index/remote/RemoteStorePathStrategy.java b/server/src/main/java/org/opensearch/index/remote/RemoteStorePathStrategy.java index c58f6c3faac84..b5f28e4b6fdee 100644 --- a/server/src/main/java/org/opensearch/index/remote/RemoteStorePathStrategy.java +++ b/server/src/main/java/org/opensearch/index/remote/RemoteStorePathStrategy.java @@ -87,9 +87,12 @@ public static class PathInput { public PathInput(BlobPath basePath, String indexUUID, String shardId, DataCategory dataCategory, DataType dataType) { this.basePath = Objects.requireNonNull(basePath); this.indexUUID = Objects.requireNonNull(indexUUID); - this.shardId = Objects.requireNonNull(shardId); - this.dataCategory = Objects.requireNonNull(dataCategory); - this.dataType = Objects.requireNonNull(dataType); +// this.shardId = Objects.requireNonNull(shardId); +// this.dataCategory = Objects.requireNonNull(dataCategory); +// this.dataType = Objects.requireNonNull(dataType); + this.shardId =(shardId); + this.dataCategory = (dataCategory); + this.dataType = (dataType); } BlobPath basePath() { diff --git a/server/src/main/java/org/opensearch/index/translog/BaseTranslogReader.java b/server/src/main/java/org/opensearch/index/translog/BaseTranslogReader.java index d6fa2a2e53de3..9088eb6b20fb8 100644 --- a/server/src/main/java/org/opensearch/index/translog/BaseTranslogReader.java +++ b/server/src/main/java/org/opensearch/index/translog/BaseTranslogReader.java @@ -32,6 +32,7 @@ package org.opensearch.index.translog; +import org.opensearch.common.io.stream.BufferedChecksumStreamInput; import org.opensearch.core.common.io.stream.ByteBufferStreamInput; import org.opensearch.index.seqno.SequenceNumbers; diff --git a/server/src/main/java/org/opensearch/index/translog/Translog.java b/server/src/main/java/org/opensearch/index/translog/Translog.java index 842e9c77d2350..18b10cb886996 100644 --- a/server/src/main/java/org/opensearch/index/translog/Translog.java +++ b/server/src/main/java/org/opensearch/index/translog/Translog.java @@ -38,6 +38,8 @@ import org.opensearch.common.Nullable; import org.opensearch.common.UUIDs; import org.opensearch.common.annotation.PublicApi; +import org.opensearch.common.io.stream.BufferedChecksumStreamInput; +import org.opensearch.common.io.stream.BufferedChecksumStreamOutput; import org.opensearch.common.io.stream.ReleasableBytesStreamOutput; import org.opensearch.common.lease.Releasable; import org.opensearch.common.lease.Releasables; diff --git a/server/src/main/java/org/opensearch/index/translog/TranslogHeader.java b/server/src/main/java/org/opensearch/index/translog/TranslogHeader.java index 7b5be9505f27a..c622214a3a69d 100644 --- a/server/src/main/java/org/opensearch/index/translog/TranslogHeader.java +++ b/server/src/main/java/org/opensearch/index/translog/TranslogHeader.java @@ -40,6 +40,8 @@ import org.apache.lucene.store.OutputStreamDataOutput; import org.apache.lucene.util.BytesRef; import org.opensearch.common.io.Channels; +import org.opensearch.common.io.stream.BufferedChecksumStreamInput; +import org.opensearch.common.io.stream.BufferedChecksumStreamOutput; import org.opensearch.core.common.io.stream.InputStreamStreamInput; import org.opensearch.core.common.io.stream.OutputStreamStreamOutput; import org.opensearch.core.common.io.stream.StreamInput; diff --git a/server/src/main/java/org/opensearch/index/translog/TranslogSnapshot.java b/server/src/main/java/org/opensearch/index/translog/TranslogSnapshot.java index 89718156cbbe8..a6e322bf58fff 100644 --- a/server/src/main/java/org/opensearch/index/translog/TranslogSnapshot.java +++ b/server/src/main/java/org/opensearch/index/translog/TranslogSnapshot.java @@ -32,6 +32,7 @@ package org.opensearch.index.translog; import org.opensearch.common.io.Channels; +import org.opensearch.common.io.stream.BufferedChecksumStreamInput; import org.opensearch.index.seqno.SequenceNumbers; import java.io.EOFException; diff --git a/server/src/main/java/org/opensearch/index/translog/TranslogWriter.java b/server/src/main/java/org/opensearch/index/translog/TranslogWriter.java index 86f7567f3333d..da3c7a8dee219 100644 --- a/server/src/main/java/org/opensearch/index/translog/TranslogWriter.java +++ b/server/src/main/java/org/opensearch/index/translog/TranslogWriter.java @@ -42,6 +42,7 @@ import org.opensearch.common.collect.Tuple; import org.opensearch.common.io.Channels; import org.opensearch.common.io.DiskIoBufferPool; +import org.opensearch.common.io.stream.BufferedChecksumStreamInput; import org.opensearch.common.io.stream.ReleasableBytesStreamOutput; import org.opensearch.common.lease.Releasables; import org.opensearch.common.util.BigArrays; diff --git a/server/src/main/java/org/opensearch/index/translog/transfer/BlobStoreTransferService.java b/server/src/main/java/org/opensearch/index/translog/transfer/BlobStoreTransferService.java index 704f6419da60a..86f16b6d6d04a 100644 --- a/server/src/main/java/org/opensearch/index/translog/transfer/BlobStoreTransferService.java +++ b/server/src/main/java/org/opensearch/index/translog/transfer/BlobStoreTransferService.java @@ -8,9 +8,17 @@ package org.opensearch.index.translog.transfer; +import java.io.ByteArrayOutputStream; +import java.util.Arrays; +import java.util.Base64; +import java.util.HashMap; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; +import org.apache.lucene.store.ByteBuffersDataInput; +import org.apache.lucene.store.ByteBuffersIndexInput; +import org.apache.lucene.store.IndexInput; +import org.apache.lucene.util.BytesRef; import org.opensearch.action.ActionRunnable; import org.opensearch.common.annotation.ExperimentalApi; import org.opensearch.common.blobstore.AsyncMultiStreamBlobContainer; @@ -19,22 +27,23 @@ import org.opensearch.common.blobstore.BlobPath; import org.opensearch.common.blobstore.BlobStore; import org.opensearch.common.blobstore.InputStreamWithMetadata; -import org.opensearch.common.blobstore.stream.write.WriteContext; import org.opensearch.common.blobstore.stream.write.WritePriority; import org.opensearch.common.blobstore.transfer.RemoteTransferContainer; import org.opensearch.common.blobstore.transfer.stream.OffsetRangeFileInputStream; +import org.opensearch.common.blobstore.transfer.stream.OffsetRangeIndexInputStream; +import org.opensearch.common.io.Streams; +import org.opensearch.common.lucene.store.ByteArrayIndexInput; import org.opensearch.core.action.ActionListener; +import org.opensearch.core.common.bytes.BytesReference; +import org.opensearch.index.store.exception.ChecksumCombinationException; import org.opensearch.index.translog.ChannelFactory; import org.opensearch.index.translog.transfer.FileSnapshot.TransferFileSnapshot; import org.opensearch.threadpool.ThreadPool; -import java.io.ByteArrayOutputStream; import java.io.IOException; import java.io.InputStream; import java.nio.channels.FileChannel; import java.nio.file.StandardOpenOption; -import java.util.Base64; -import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Objects; @@ -42,6 +51,7 @@ import static org.opensearch.common.blobstore.BlobContainer.BlobNameSortOrder.LEXICOGRAPHIC; import static org.opensearch.index.translog.transfer.TranslogTransferManager.CHECKPOINT_FILE_DATA_KEY; +import static org.opensearch.common.blobstore.transfer.RemoteTransferContainer.checksumOfChecksum; /** * Service that handles remote transfer of translog and checkpoint files @@ -108,6 +118,47 @@ public void uploadBlobs( } + @Override + public void uploadBlobAsync(InputStream inputStream, Iterable remotePath, String fileName, WritePriority writePriority, + ActionListener listener) throws IOException { + assert remotePath instanceof BlobPath; + BlobPath blobPath = (BlobPath) remotePath; + final BlobContainer blobContainer = blobStore.blobContainer(blobPath); + if (blobContainer instanceof AsyncMultiStreamBlobContainer == false) { + blobContainer.writeBlob(fileName, inputStream, inputStream.available(), false); + listener.onResponse(null); + return; + } + final String resourceDescription = "BlobStoreTransferService.uploadBlob(blob=\"" + fileName + "\")"; + try (IndexInput input = inputStream.available() > 0 + ? new ByteBuffersIndexInput(new ByteBuffersDataInput(Arrays.asList(BytesReference.toByteBuffers(Streams.readFully(inputStream)))), + resourceDescription) + : new ByteArrayIndexInput(resourceDescription, BytesRef.EMPTY_BYTES)) { + long expectedChecksum; + try { + expectedChecksum = checksumOfChecksum(input.clone(), 8); + } catch (Exception e) { + throw new ChecksumCombinationException( + "Potentially corrupted file: Checksum combination failed while combining stored checksum " + + "and calculated checksum of stored checksum", + resourceDescription, + e + ); + } + + asyncBlobUpload(fileName, + fileName, + inputStream.available(), + blobPath, + writePriority, + (size, position) -> new OffsetRangeIndexInputStream(input, size, position), + expectedChecksum, + listener, + null + ); + } + } + // Builds a metadata map containing the Base64-encoded checkpoint file data associated with a translog file. static Map buildTransferFileMetadata(InputStream metadataInputStream) throws IOException { Map metadata = new HashMap<>(); @@ -150,37 +201,22 @@ private void uploadBlob( try (FileChannel channel = channelFactory.open(fileSnapshot.getPath(), StandardOpenOption.READ)) { contentLength = channel.size(); } - boolean remoteIntegrityEnabled = false; - BlobContainer blobContainer = blobStore.blobContainer(blobPath); - if (blobContainer instanceof AsyncMultiStreamBlobContainer) { - remoteIntegrityEnabled = ((AsyncMultiStreamBlobContainer) blobContainer).remoteIntegrityCheckSupported(); - } - RemoteTransferContainer remoteTransferContainer = new RemoteTransferContainer( - fileSnapshot.getName(), + ActionListener completionListener = ActionListener.wrap(resp -> listener.onResponse(fileSnapshot), ex -> { + logger.error(() -> new ParameterizedMessage("Failed to upload blob {}", fileSnapshot.getName()), ex); + listener.onFailure(new FileTransferException(fileSnapshot, ex)); + }); + + Objects.requireNonNull(fileSnapshot.getChecksum()); + asyncBlobUpload(fileSnapshot.getName(), fileSnapshot.getName(), contentLength, - true, + blobPath, writePriority, (size, position) -> new OffsetRangeFileInputStream(fileSnapshot.getPath(), size, position), - Objects.requireNonNull(fileSnapshot.getChecksum()), - remoteIntegrityEnabled, + fileSnapshot.getChecksum(), + completionListener, metadata ); - ActionListener completionListener = ActionListener.wrap(resp -> listener.onResponse(fileSnapshot), ex -> { - logger.error(() -> new ParameterizedMessage("Failed to upload blob {}", fileSnapshot.getName()), ex); - listener.onFailure(new FileTransferException(fileSnapshot, ex)); - }); - - completionListener = ActionListener.runBefore(completionListener, () -> { - try { - remoteTransferContainer.close(); - } catch (Exception e) { - logger.warn("Error occurred while closing streams", e); - } - }); - - WriteContext writeContext = remoteTransferContainer.createWriteContext(); - ((AsyncMultiStreamBlobContainer) blobStore.blobContainer(blobPath)).asyncBlobUpload(writeContext, completionListener); } catch (Exception e) { logger.error(() -> new ParameterizedMessage("Failed to upload blob {}", fileSnapshot.getName()), e); @@ -195,6 +231,27 @@ private void uploadBlob( } + private void asyncBlobUpload(String fileName, String remoteFileName, long contentLength, BlobPath blobPath, WritePriority writePriority, + RemoteTransferContainer.OffsetRangeInputStreamSupplier inputStreamSupplier, Long expectedChecksum, ActionListener completionListener, + Map metadata) throws IOException { + BlobContainer blobContainer = blobStore.blobContainer(blobPath); + assert blobContainer instanceof AsyncMultiStreamBlobContainer; + boolean remoteIntegrityEnabled = ((AsyncMultiStreamBlobContainer) blobContainer).remoteIntegrityCheckSupported(); + try (RemoteTransferContainer remoteTransferContainer = new RemoteTransferContainer( + fileName, + remoteFileName, + contentLength, + true, + writePriority, + inputStreamSupplier, + expectedChecksum, + remoteIntegrityEnabled, + metadata + )) { + ((AsyncMultiStreamBlobContainer) blobContainer).asyncBlobUpload(remoteTransferContainer.createWriteContext(), completionListener); + } + } + @Override public InputStream downloadBlob(Iterable path, String fileName) throws IOException { return blobStore.blobContainer((BlobPath) path).readBlob(fileName); @@ -273,7 +330,9 @@ public void listAllInSortedOrderAsync( int limit, ActionListener> listener ) { - threadPool.executor(threadpoolName).execute(() -> { listAllInSortedOrder(path, filenamePrefix, limit, listener); }); + threadPool.executor(threadpoolName).execute(() -> { + listAllInSortedOrder(path, filenamePrefix, limit, listener); + }); } } diff --git a/server/src/main/java/org/opensearch/index/translog/transfer/TransferService.java b/server/src/main/java/org/opensearch/index/translog/transfer/TransferService.java index 0ff983739438b..fceb488f3b604 100644 --- a/server/src/main/java/org/opensearch/index/translog/transfer/TransferService.java +++ b/server/src/main/java/org/opensearch/index/translog/transfer/TransferService.java @@ -65,6 +65,7 @@ void uploadBlobs( * @throws IOException the exception while transferring the data */ void uploadBlob(final TransferFileSnapshot fileSnapshot, Iterable remotePath, WritePriority writePriority) throws IOException; + void uploadBlobAsync(InputStream inputStream, Iterable remotePath, String blobName, WritePriority writePriority, ActionListener listener) throws IOException; void deleteBlobs(Iterable path, List fileNames) throws IOException; diff --git a/server/src/main/java/org/opensearch/node/Node.java b/server/src/main/java/org/opensearch/node/Node.java index 76109ba10624a..6d5e47404a5b3 100644 --- a/server/src/main/java/org/opensearch/node/Node.java +++ b/server/src/main/java/org/opensearch/node/Node.java @@ -138,6 +138,7 @@ import org.opensearch.gateway.MetaStateService; import org.opensearch.gateway.PersistedClusterStateService; import org.opensearch.gateway.ShardsBatchGatewayAllocator; +import org.opensearch.gateway.remote.RemoteClusterStateCleanupManager; import org.opensearch.gateway.remote.RemoteClusterStateService; import org.opensearch.http.HttpServerTransport; import org.opensearch.identity.IdentityService; @@ -752,6 +753,7 @@ protected Node( threadPool::relativeTimeInMillis ); final RemoteClusterStateService remoteClusterStateService; + final RemoteClusterStateCleanupManager remoteClusterStateCleanupManager; final RemoteIndexPathUploader remoteIndexPathUploader; if (isRemoteStoreClusterStateEnabled(settings)) { remoteIndexPathUploader = new RemoteIndexPathUploader( @@ -764,14 +766,17 @@ protected Node( nodeEnvironment.nodeId(), repositoriesServiceReference::get, settings, - clusterService.getClusterSettings(), + clusterService, threadPool::preciseRelativeTimeInNanos, threadPool, - List.of(remoteIndexPathUploader) + List.of(remoteIndexPathUploader), + namedWriteableRegistry ); + remoteClusterStateCleanupManager = remoteClusterStateService.getCleanupManager(); } else { remoteClusterStateService = null; remoteIndexPathUploader = null; + remoteClusterStateCleanupManager = null; } // collect engine factory providers from plugins @@ -1195,7 +1200,8 @@ protected Node( rerouteService, fsHealthService, persistedStateRegistry, - remoteStoreNodeService + remoteStoreNodeService, + remoteClusterStateService ); final SearchPipelineService searchPipelineService = new SearchPipelineService( clusterService, @@ -1376,6 +1382,7 @@ protected Node( b.bind(MetricsRegistry.class).toInstance(metricsRegistry); b.bind(RemoteClusterStateService.class).toProvider(() -> remoteClusterStateService); b.bind(RemoteIndexPathUploader.class).toProvider(() -> remoteIndexPathUploader); + b.bind(RemoteClusterStateCleanupManager.class).toProvider(() -> remoteClusterStateCleanupManager); b.bind(PersistedStateRegistry.class).toInstance(persistedStateRegistry); b.bind(SegmentReplicationStatsTracker.class).toInstance(segmentReplicationStatsTracker); b.bind(SearchRequestOperationsCompositeListenerFactory.class).toInstance(searchRequestOperationsCompositeListenerFactory); diff --git a/server/src/main/java/org/opensearch/node/remotestore/RemoteStoreNodeAttribute.java b/server/src/main/java/org/opensearch/node/remotestore/RemoteStoreNodeAttribute.java index a0f745a4270c4..8a7a27f6833b9 100644 --- a/server/src/main/java/org/opensearch/node/remotestore/RemoteStoreNodeAttribute.java +++ b/server/src/main/java/org/opensearch/node/remotestore/RemoteStoreNodeAttribute.java @@ -201,8 +201,9 @@ private static boolean isRemoteRoutingTableAttributePresent(Settings settings) { .isEmpty() == false; } - public static boolean isRemoteRoutingTableEnabled(Settings settings) { - return FeatureFlags.isEnabled(REMOTE_PUBLICATION_EXPERIMENTAL) && isRemoteRoutingTableAttributePresent(settings); + public static boolean isRemotePublicationEnabled(Settings settings) { + return FeatureFlags.isEnabled(REMOTE_PUBLICATION_EXPERIMENTAL) && isRemoteRoutingTableAttributePresent(settings) + && isRemoteStoreClusterStateEnabled(settings); } public RepositoriesMetadata getRepositoriesMetadata() { diff --git a/server/src/main/java/org/opensearch/repositories/IndexId.java b/server/src/main/java/org/opensearch/repositories/IndexId.java index 87a0063e8c21b..2f97368d42038 100644 --- a/server/src/main/java/org/opensearch/repositories/IndexId.java +++ b/server/src/main/java/org/opensearch/repositories/IndexId.java @@ -36,6 +36,7 @@ import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; import org.opensearch.core.common.io.stream.Writeable; +import org.opensearch.core.xcontent.XContentParser; import org.opensearch.core.index.Index; import org.opensearch.core.xcontent.ToXContent; import org.opensearch.core.xcontent.ToXContentObject; @@ -44,6 +45,8 @@ import java.io.IOException; import java.util.Objects; +import static org.opensearch.core.xcontent.XContentParserUtils.ensureExpectedToken; + /** * Represents a single snapshotted index in the repository. * @@ -133,4 +136,25 @@ public XContentBuilder toXContent(XContentBuilder builder, ToXContent.Params par builder.endObject(); return builder; } + + public static IndexId fromXContent(XContentParser parser) throws IOException { + ensureExpectedToken(XContentParser.Token.START_OBJECT, parser.currentToken(), parser); + String name = null; + String id = null; + while (parser.nextToken() != XContentParser.Token.END_OBJECT) { + String currentFieldName = parser.currentName(); + parser.nextToken(); + switch (currentFieldName) { + case NAME: + name = parser.text(); + break; + case ID: + id = parser.text(); + break; + default: + throw new IllegalArgumentException("unknown field [" + currentFieldName + "]"); + } + } + return new IndexId(name, id); + } } diff --git a/server/src/main/java/org/opensearch/repositories/blobstore/ChecksumBlobStoreFormat.java b/server/src/main/java/org/opensearch/repositories/blobstore/ChecksumBlobStoreFormat.java index e567e1b626c5a..53b9b3300ffe3 100644 --- a/server/src/main/java/org/opensearch/repositories/blobstore/ChecksumBlobStoreFormat.java +++ b/server/src/main/java/org/opensearch/repositories/blobstore/ChecksumBlobStoreFormat.java @@ -31,6 +31,13 @@ package org.opensearch.repositories.blobstore; +import static org.opensearch.common.blobstore.transfer.RemoteTransferContainer.checksumOfChecksum; + +import java.io.IOException; +import java.util.Arrays; +import java.util.HashMap; +import java.util.Locale; +import java.util.Map; import org.apache.lucene.codecs.CodecUtil; import org.apache.lucene.index.CorruptIndexException; import org.apache.lucene.index.IndexFormatTooNewException; @@ -61,14 +68,6 @@ import org.opensearch.index.store.exception.ChecksumCombinationException; import org.opensearch.snapshots.SnapshotInfo; -import java.io.IOException; -import java.util.Arrays; -import java.util.HashMap; -import java.util.Locale; -import java.util.Map; - -import static org.opensearch.common.blobstore.transfer.RemoteTransferContainer.checksumOfChecksum; - /** * Snapshot metadata file format used in v2.0 and above * @@ -114,8 +113,7 @@ public ChecksumBlobStoreFormat(String codec, String blobNameFormat, CheckedFunct * @param name name to be translated into * @return parsed blob object */ - public T read(BlobContainer blobContainer, String name, NamedXContentRegistry namedXContentRegistry) throws IOException { - String blobName = blobName(name); + public T read(BlobContainer blobContainer, String name, NamedXContentRegistry namedXContentRegistry) throws IOException {String blobName = blobName(name); return deserialize(blobName, namedXContentRegistry, Streams.readFully(blobContainer.readBlob(blobName))); } diff --git a/server/src/main/java/org/opensearch/repositories/blobstore/ChecksumWritableBlobStoreFormat.java b/server/src/main/java/org/opensearch/repositories/blobstore/ChecksumWritableBlobStoreFormat.java new file mode 100644 index 0000000000000..96dee617bd082 --- /dev/null +++ b/server/src/main/java/org/opensearch/repositories/blobstore/ChecksumWritableBlobStoreFormat.java @@ -0,0 +1,117 @@ +/* + * 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.repositories.blobstore; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.Arrays; +import java.util.Objects; +import org.apache.lucene.codecs.CodecUtil; +import org.apache.lucene.index.CorruptIndexException; +import org.apache.lucene.index.IndexFormatTooNewException; +import org.apache.lucene.index.IndexFormatTooOldException; +import org.apache.lucene.store.ByteBuffersDataInput; +import org.apache.lucene.store.ByteBuffersIndexInput; +import org.apache.lucene.store.IndexInput; +import org.apache.lucene.store.OutputStreamIndexOutput; +import org.apache.lucene.util.BytesRef; +import org.opensearch.Version; +import org.opensearch.common.CheckedFunction; +import org.opensearch.common.io.stream.BytesStreamOutput; +import org.opensearch.common.lucene.store.ByteArrayIndexInput; +import org.opensearch.common.lucene.store.IndexOutputOutputStream; +import org.opensearch.core.common.bytes.BytesReference; +import org.opensearch.core.common.io.stream.InputStreamStreamInput; +import org.opensearch.core.common.io.stream.OutputStreamStreamOutput; +import org.opensearch.core.common.io.stream.StreamInput; +import org.opensearch.core.common.io.stream.StreamOutput; +import org.opensearch.core.common.io.stream.Writeable; +import org.opensearch.core.compress.Compressor; +import org.opensearch.core.compress.CompressorRegistry; +import org.opensearch.core.compress.NotXContentException; +import org.opensearch.core.xcontent.XContentParser; +import org.opensearch.gateway.CorruptStateException; + +public class ChecksumWritableBlobStoreFormat { + + public static final int VERSION = 1; + + private static final int BUFFER_SIZE = 4096; + + private final String codec; + private final CheckedFunction reader; + + public ChecksumWritableBlobStoreFormat(String codec, CheckedFunction reader) { + this.codec = codec; + this.reader = reader; + } + + public BytesReference serialize( + final T obj, + final String blobName, + final Compressor compressor + ) throws IOException { + try (BytesStreamOutput outputStream = new BytesStreamOutput()) { + try ( + OutputStreamIndexOutput indexOutput = new OutputStreamIndexOutput( + "ChecksumBlobStoreFormat.writeBlob(blob=\"" + blobName + "\")", + blobName, + outputStream, + BUFFER_SIZE + ) + ) { + CodecUtil.writeHeader(indexOutput, codec, VERSION); + + try (OutputStream indexOutputOutputStream = new IndexOutputOutputStream(indexOutput) { + @Override + public void close() throws IOException { + // this is important since some of the XContentBuilders write bytes on close. + // in order to write the footer we need to prevent closing the actual index input. + } + }; + StreamOutput stream = new OutputStreamStreamOutput(compressor.threadLocalOutputStream(indexOutputOutputStream)); + ) { + // TODO The stream version should be configurable + stream.setVersion(Version.CURRENT); + obj.writeTo(stream); + } + CodecUtil.writeFooter(indexOutput); + } + return outputStream.bytes(); + } + } + + public T deserialize(String blobName, CheckedFunction reader, BytesReference bytes) throws IOException { + final String resourceDesc = "ChecksumBlobStoreFormat.readBlob(blob=\"" + blobName + "\")"; + try { + final IndexInput indexInput = bytes.length() > 0 + ? new ByteBuffersIndexInput(new ByteBuffersDataInput(Arrays.asList(BytesReference.toByteBuffers(bytes))), resourceDesc) + : new ByteArrayIndexInput(resourceDesc, BytesRef.EMPTY_BYTES); + CodecUtil.checksumEntireFile(indexInput); + CodecUtil.checkHeader(indexInput, codec, VERSION, VERSION); + long filePointer = indexInput.getFilePointer(); + long contentSize = indexInput.length() - CodecUtil.footerLength() - filePointer; + BytesReference bytesReference = bytes.slice((int) filePointer, (int) contentSize); + Compressor compressor; + try { + compressor = CompressorRegistry.compressor(bytesReference); + } catch (NotXContentException e) { + compressor = CompressorRegistry.none(); + } + try (StreamInput in = new InputStreamStreamInput(compressor.threadLocalInputStream(bytesReference.streamInput()))) { + return reader.apply(in); + } + } catch (CorruptIndexException | IndexFormatTooOldException | IndexFormatTooNewException ex) { + // we trick this into a dedicated exception with the original stacktrace + throw new CorruptStateException(ex); + } + } + +} diff --git a/server/src/main/java/org/opensearch/snapshots/SnapshotId.java b/server/src/main/java/org/opensearch/snapshots/SnapshotId.java index 4eeb956a0cb19..31b18f14c854b 100644 --- a/server/src/main/java/org/opensearch/snapshots/SnapshotId.java +++ b/server/src/main/java/org/opensearch/snapshots/SnapshotId.java @@ -38,6 +38,8 @@ import org.opensearch.core.common.io.stream.Writeable; import org.opensearch.core.xcontent.ToXContentObject; import org.opensearch.core.xcontent.XContentBuilder; +import org.opensearch.core.xcontent.XContentParser; +import org.opensearch.core.xcontent.XContentParserUtils; import java.io.IOException; import java.util.Objects; @@ -145,4 +147,23 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws builder.endObject(); return builder; } + + public static SnapshotId fromXContent(XContentParser parser) throws IOException { + XContentParserUtils.ensureExpectedToken(XContentParser.Token.START_OBJECT, parser.currentToken(), parser); + String name = null; + String uuid = null; + while (parser.nextToken() != XContentParser.Token.END_OBJECT) { + XContentParserUtils.ensureExpectedToken(XContentParser.Token.FIELD_NAME, parser.currentToken(), parser); + final String currentFieldName = parser.currentName(); + parser.nextToken(); + if (NAME.equals(currentFieldName)) { + name = parser.text(); + } else if (UUID.equals(currentFieldName)) { + uuid = parser.text(); + } else { + throw new IllegalArgumentException("unknown field [" + currentFieldName + "]"); + } + } + return new SnapshotId(name, uuid); + } } diff --git a/server/src/test/java/org/opensearch/cluster/block/ClusterBlockTests.java b/server/src/test/java/org/opensearch/cluster/block/ClusterBlockTests.java index 04e04bd96a7d3..7800873a5591c 100644 --- a/server/src/test/java/org/opensearch/cluster/block/ClusterBlockTests.java +++ b/server/src/test/java/org/opensearch/cluster/block/ClusterBlockTests.java @@ -33,18 +33,31 @@ package org.opensearch.cluster.block; import org.opensearch.Version; +import org.opensearch.cluster.metadata.Metadata; import org.opensearch.common.UUIDs; import org.opensearch.common.io.stream.BytesStreamOutput; +import org.opensearch.common.xcontent.json.JsonXContent; +import org.opensearch.core.common.bytes.BytesReference; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.rest.RestStatus; +import org.opensearch.core.xcontent.MediaType; +import org.opensearch.core.xcontent.MediaTypeRegistry; +import org.opensearch.core.xcontent.ToXContent; +import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.test.OpenSearchTestCase; +import org.opensearch.test.XContentTestUtils; +import java.io.IOException; import java.util.Arrays; import java.util.Collections; import java.util.List; +import java.util.Locale; import java.util.Map; +import static java.util.Collections.singletonMap; import static java.util.EnumSet.copyOf; +import static org.opensearch.cluster.metadata.Metadata.CONTEXT_MODE_API; +import static org.opensearch.cluster.metadata.Metadata.CONTEXT_MODE_GATEWAY; import static org.opensearch.test.VersionUtils.randomVersion; import static org.hamcrest.CoreMatchers.endsWith; import static org.hamcrest.CoreMatchers.equalTo; @@ -136,7 +149,111 @@ public void testGetIndexBlockWithId() { assertThat(builder.build().getIndexBlockWithId("index", randomValueOtherThan(blockId, OpenSearchTestCase::randomInt)), nullValue()); } - private ClusterBlock randomClusterBlock() { + public void testToXContent_APIMode() throws IOException { + ClusterBlock clusterBlock = randomClusterBlock(); + XContentBuilder builder = JsonXContent.contentBuilder().prettyPrint(); + builder.startObject(); + clusterBlock.toXContent(builder, ToXContent.EMPTY_PARAMS); + builder.endObject(); + + String expectedString = "{\n" + getExpectedXContentFragment(clusterBlock, " ", false) + "\n}"; + + assertEquals(expectedString, builder.toString()); + } + + public void testToXContent_GatewayMode() throws IOException { + ClusterBlock clusterBlock = randomClusterBlock(); + XContentBuilder builder = JsonXContent.contentBuilder().prettyPrint(); + builder.startObject(); + clusterBlock.toXContent(builder, new ToXContent.MapParams(singletonMap(Metadata.CONTEXT_MODE_PARAM, CONTEXT_MODE_GATEWAY))); + builder.endObject(); + + String expectedString = "{\n" + getExpectedXContentFragment(clusterBlock, " ", true) + "\n}"; + + assertEquals(expectedString, builder.toString()); + } + + public void testFromXContent() throws IOException { + doFromXContentTestWithRandomFields(false); + } + + public void testFromXContentWithRandomFields() throws IOException { + doFromXContentTestWithRandomFields(true); + } + + private void doFromXContentTestWithRandomFields(boolean addRandomFields) throws IOException { + ClusterBlock clusterBlock = randomClusterBlock(); + boolean humanReadable = randomBoolean(); + final MediaType mediaType = MediaTypeRegistry.JSON; + BytesReference originalBytes = toShuffledXContent(clusterBlock, mediaType, ToXContent.EMPTY_PARAMS, humanReadable); + + if (addRandomFields) { + String unsupportedField = "unsupported_field"; + BytesReference mutated = BytesReference.bytes( + XContentTestUtils.insertIntoXContent( + mediaType.xContent(), + originalBytes, + Collections.singletonList(Integer.toString(clusterBlock.id())), + () -> unsupportedField, + () -> randomAlphaOfLengthBetween(3, 10) + ) + ); + IllegalArgumentException e = expectThrows( + IllegalArgumentException.class, + () -> ClusterBlock.fromXContent(createParser(mediaType.xContent(), mutated), clusterBlock.id()) + ); + assertEquals(e.getMessage(), "unknown field [" + unsupportedField + "]"); + } else { + ClusterBlock parsedBlock = ClusterBlock.fromXContent(createParser(mediaType.xContent(), originalBytes), clusterBlock.id()); + assertEquals(clusterBlock, parsedBlock); + assertEquals(clusterBlock.description(), parsedBlock.description()); + assertEquals(clusterBlock.retryable(), parsedBlock.retryable()); + assertEquals(clusterBlock.disableStatePersistence(), parsedBlock.disableStatePersistence()); + assertArrayEquals(clusterBlock.levels().toArray(), parsedBlock.levels().toArray()); + } + } + + static String getExpectedXContentFragment(ClusterBlock clusterBlock, String indent, boolean gatewayMode) { + return indent + + "\"" + + clusterBlock.id() + + "\" : {\n" + + (clusterBlock.uuid() != null ? indent + " \"uuid\" : \"" + clusterBlock.uuid() + "\",\n" : "") + + indent + + " \"description\" : \"" + + clusterBlock.description() + + "\",\n" + + indent + + " \"retryable\" : " + + clusterBlock.retryable() + + ",\n" + + (clusterBlock.disableStatePersistence() + ? indent + " \"disable_state_persistence\" : " + clusterBlock.disableStatePersistence() + ",\n" + : "") + + String.format( + Locale.ROOT, + indent + " \"levels\" : [%s]", + clusterBlock.levels().isEmpty() + ? " " + : "\n" + + String.join( + ",\n", + clusterBlock.levels() + .stream() + .map(level -> indent + " \"" + level.name().toLowerCase(Locale.ROOT) + "\"") + .toArray(String[]::new) + ) + + "\n " + + indent + ) + + (gatewayMode ? ",\n" + + indent + " \"status\" : \"" + clusterBlock.status() + "\",\n" + + indent + " \"allow_release_resources\" : " + clusterBlock.isAllowReleaseResources() + "\n" : "\n" ) + + indent + + "}"; + } + + static ClusterBlock randomClusterBlock() { final String uuid = randomBoolean() ? UUIDs.randomBase64UUID() : null; final List levels = Arrays.asList(ClusterBlockLevel.values()); return new ClusterBlock( diff --git a/server/src/test/java/org/opensearch/cluster/block/ClusterBlocksTests.java b/server/src/test/java/org/opensearch/cluster/block/ClusterBlocksTests.java new file mode 100644 index 0000000000000..c1d4401760be3 --- /dev/null +++ b/server/src/test/java/org/opensearch/cluster/block/ClusterBlocksTests.java @@ -0,0 +1,151 @@ +/* + * 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.block; + +import org.opensearch.common.xcontent.json.JsonXContent; +import org.opensearch.core.common.bytes.BytesReference; +import org.opensearch.core.xcontent.MediaType; +import org.opensearch.core.xcontent.MediaTypeRegistry; +import org.opensearch.core.xcontent.ToXContent; +import org.opensearch.core.xcontent.XContentBuilder; +import org.opensearch.core.xcontent.XContentParser; +import org.opensearch.test.OpenSearchTestCase; +import org.opensearch.test.XContentTestUtils; + +import java.io.IOException; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Locale; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; + +import static org.opensearch.cluster.block.ClusterBlockTests.getExpectedXContentFragment; +import static org.opensearch.cluster.block.ClusterBlockTests.randomClusterBlock; + +public class ClusterBlocksTests extends OpenSearchTestCase { + public void testToXContent() throws IOException { + ClusterBlocks clusterBlocks = randomClusterBlocks(); + XContentBuilder builder = JsonXContent.contentBuilder().prettyPrint(); + builder.startObject(); + clusterBlocks.toXContent(builder, ToXContent.EMPTY_PARAMS); + builder.endObject(); + + String expectedXContent = "{\n" + + " \"blocks\" : {\n" + + String.format( + Locale.ROOT, + "%s", + clusterBlocks.global().isEmpty() + ? "" + : " \"global\" : {\n" + + clusterBlocks.global() + .stream() + .map(clusterBlock -> getExpectedXContentFragment(clusterBlock, " ", false)) + .collect(Collectors.joining(",\n")) + + "\n }" + + (!clusterBlocks.indices().isEmpty() ? "," : "") + + "\n" + ) + + String.format( + Locale.ROOT, + "%s", + clusterBlocks.indices().isEmpty() + ? "" + : " \"indices\" : {\n" + + clusterBlocks.indices() + .entrySet() + .stream() + .map( + entry -> " \"" + + entry.getKey() + + "\" : {" + + (entry.getValue().isEmpty() + ? " }" + : "\n" + + entry.getValue() + .stream() + .map(clusterBlock -> getExpectedXContentFragment(clusterBlock, " ", false)) + .collect(Collectors.joining(",\n")) + + "\n }") + ) + .collect(Collectors.joining(",\n")) + + "\n }\n" + ) + + " }\n" + + "}"; + + assertEquals(expectedXContent, builder.toString()); + } + + public void testFromXContent() throws IOException { + doFromXContentTestWithRandomFields(false); + } + + public void testFromXContentWithRandomFields() throws IOException { + doFromXContentTestWithRandomFields(true); + } + + private void doFromXContentTestWithRandomFields(boolean addRandomFields) throws IOException { + ClusterBlocks clusterBlocks = randomClusterBlocks(); + boolean humanReadable = randomBoolean(); + final MediaType mediaType = MediaTypeRegistry.JSON; + BytesReference originalBytes = toShuffledXContent(clusterBlocks, mediaType, ToXContent.EMPTY_PARAMS, humanReadable); + + if (addRandomFields) { + String unsupportedField = "unsupported_field"; + BytesReference mutated = BytesReference.bytes( + XContentTestUtils.insertIntoXContent( + mediaType.xContent(), + originalBytes, + Collections.singletonList("blocks"), + () -> unsupportedField, + () -> randomAlphaOfLengthBetween(3, 10) + ) + ); + IllegalArgumentException exception = expectThrows( + IllegalArgumentException.class, + () -> ClusterBlocks.fromXContent(createParser(mediaType.xContent(), mutated)) + ); + assertEquals("unknown field [" + unsupportedField + "]", exception.getMessage()); + } else { + try (XContentParser parser = createParser(JsonXContent.jsonXContent, originalBytes)) { + ClusterBlocks parsedClusterBlocks = ClusterBlocks.fromXContent(parser); + assertEquals(clusterBlocks.global().size(), parsedClusterBlocks.global().size()); + assertEquals(clusterBlocks.indices().size(), parsedClusterBlocks.indices().size()); + clusterBlocks.global().forEach(clusterBlock -> assertTrue(parsedClusterBlocks.global().contains(clusterBlock))); + clusterBlocks.indices().forEach((key, value) -> { + assertTrue(parsedClusterBlocks.indices().containsKey(key)); + value.forEach(clusterBlock -> assertTrue(parsedClusterBlocks.indices().get(key).contains(clusterBlock))); + }); + } + } + } + + private ClusterBlocks randomClusterBlocks() { + int randomGlobalBlocks = randomIntBetween(0, 10); + Set globalBlocks = new HashSet<>(); + for (int i = 0; i < randomGlobalBlocks; i++) { + globalBlocks.add(randomClusterBlock()); + } + + int randomIndices = randomIntBetween(0, 10); + Map> indexBlocks = new HashMap<>(); + for (int i = 0; i < randomIndices; i++) { + int randomIndexBlocks = randomIntBetween(0, 10); + Set blocks = new HashSet<>(); + for (int j = 0; j < randomIndexBlocks; j++) { + blocks.add(randomClusterBlock()); + } + indexBlocks.put("index-" + i, blocks); + } + return new ClusterBlocks(globalBlocks, indexBlocks); + } +} diff --git a/server/src/test/java/org/opensearch/cluster/coordination/CoordinationStateTests.java b/server/src/test/java/org/opensearch/cluster/coordination/CoordinationStateTests.java index bd71aecf89101..4ade19b3d7219 100644 --- a/server/src/test/java/org/opensearch/cluster/coordination/CoordinationStateTests.java +++ b/server/src/test/java/org/opensearch/cluster/coordination/CoordinationStateTests.java @@ -48,6 +48,7 @@ import org.opensearch.gateway.GatewayMetaState.RemotePersistedState; import org.opensearch.gateway.remote.ClusterMetadataManifest; import org.opensearch.gateway.remote.RemoteClusterStateService; +import org.opensearch.gateway.remote.model.RemoteUploadDetails; import org.opensearch.repositories.fs.FsRepository; import org.opensearch.test.EqualsHashCodeTestUtils; import org.opensearch.test.OpenSearchTestCase; @@ -944,7 +945,8 @@ public void testHandlePrePublishAndCommitWhenRemoteStateEnabled() throws IOExcep .previousClusterUUID(randomAlphaOfLength(10)) .clusterUUIDCommitted(true) .build(); - Mockito.when(remoteClusterStateService.writeFullMetadata(clusterState, previousClusterUUID)).thenReturn(manifest); + Mockito.when(remoteClusterStateService.writeFullMetadata(clusterState, previousClusterUUID)) + .thenReturn(new RemoteUploadDetails(manifest, "path/to/manifest")); final PersistedStateRegistry persistedStateRegistry = persistedStateRegistry(); persistedStateRegistry.addPersistedState(PersistedStateType.LOCAL, ps1); diff --git a/server/src/test/java/org/opensearch/cluster/coordination/NodeJoinTests.java b/server/src/test/java/org/opensearch/cluster/coordination/NodeJoinTests.java index 10d5dceb74f55..32cb153c32029 100644 --- a/server/src/test/java/org/opensearch/cluster/coordination/NodeJoinTests.java +++ b/server/src/test/java/org/opensearch/cluster/coordination/NodeJoinTests.java @@ -273,7 +273,8 @@ protected void onSendRequest( ElectionStrategy.DEFAULT_INSTANCE, nodeHealthService, persistedStateRegistry, - Mockito.mock(RemoteStoreNodeService.class) + Mockito.mock(RemoteStoreNodeService.class), + null ); transportService.start(); transportService.acceptIncomingRequests(); diff --git a/server/src/test/java/org/opensearch/cluster/coordination/PublicationTransportHandlerTests.java b/server/src/test/java/org/opensearch/cluster/coordination/PublicationTransportHandlerTests.java index 6d94054afdea2..a9950d82bfac8 100644 --- a/server/src/test/java/org/opensearch/cluster/coordination/PublicationTransportHandlerTests.java +++ b/server/src/test/java/org/opensearch/cluster/coordination/PublicationTransportHandlerTests.java @@ -76,7 +76,8 @@ public void testDiffSerializationFailure() { transportService, writableRegistry(), pu -> null, - (pu, l) -> {} + (pu, l) -> {}, + null ); transportService.start(); transportService.acceptIncomingRequests(); @@ -111,7 +112,7 @@ public void writeTo(StreamOutput out) throws IOException { OpenSearchException e = expectThrows( OpenSearchException.class, - () -> handler.newPublicationContext(new ClusterChangedEvent("test", unserializableClusterState, clusterState)) + () -> handler.newPublicationContext(new ClusterChangedEvent("test", unserializableClusterState, clusterState), false, null) ); assertNotNull(e.getCause()); assertThat(e.getCause(), instanceOf(IOException.class)); diff --git a/server/src/test/java/org/opensearch/cluster/metadata/DiffableStringMapTests.java b/server/src/test/java/org/opensearch/cluster/metadata/DiffableStringMapTests.java index fcf6ad97c0f00..8fc1953e69663 100644 --- a/server/src/test/java/org/opensearch/cluster/metadata/DiffableStringMapTests.java +++ b/server/src/test/java/org/opensearch/cluster/metadata/DiffableStringMapTests.java @@ -34,11 +34,21 @@ import org.opensearch.cluster.Diff; import org.opensearch.common.io.stream.BytesStreamOutput; +import org.opensearch.common.xcontent.XContentFactory; +import org.opensearch.common.xcontent.json.JsonXContent; +import org.opensearch.core.common.bytes.BytesReference; +import org.opensearch.core.xcontent.MediaType; +import org.opensearch.core.xcontent.MediaTypeRegistry; +import org.opensearch.core.xcontent.ToXContent; +import org.opensearch.core.xcontent.XContentBuilder; +import org.opensearch.core.xcontent.XContentParser; +import org.opensearch.search.suggest.phrase.Correction; import org.opensearch.test.OpenSearchTestCase; import java.io.IOException; import java.util.HashMap; import java.util.Map; +import java.util.stream.Collectors; import static org.hamcrest.Matchers.containsInAnyOrder; import static org.hamcrest.Matchers.equalTo; @@ -113,4 +123,45 @@ public void testSerialization() throws IOException { DiffableStringMap deserialized = DiffableStringMap.readFrom(bso.bytes().streamInput()); assertThat(deserialized, equalTo(dsm)); } + + public void testToXContent() throws IOException { + Map m = new HashMap<>(); + if (frequently()) { + m.put("foo", "bar"); + m.put("baz", "eggplant"); + m.put("potato", "canon"); + } + DiffableStringMap dsm = new DiffableStringMap(m); + XContentBuilder builder = JsonXContent.contentBuilder().prettyPrint(); + dsm.toXContent(builder, null); + String expectedString = "{\n" + + " \"inner_map\" : {\n" + + m.entrySet().stream().map(entry -> " \"" + entry.getKey() + "\" : \"" + entry.getValue() + "\"").collect(Collectors.joining(",\n")) + "\n" + + " }\n" + + "}"; + assertEquals(expectedString, builder.toString()); + } + + public void testFromXContent() throws IOException { + Map m = new HashMap<>(); + if (frequently()) { + m.put("foo", "bar"); + m.put("baz", "eggplant"); + m.put("potato", "canon"); + } + DiffableStringMap dsm = new DiffableStringMap(m); + boolean humanReadable = randomBoolean(); + final MediaType mediaType = MediaTypeRegistry.JSON; + BytesReference originalBytes = toShuffledXContent( + dsm, + mediaType, + null, + humanReadable + ); + + try (XContentParser parser = createParser(mediaType.xContent(), originalBytes)) { + DiffableStringMap parsed = DiffableStringMap.fromXContent(parser); + assertEquals(dsm, parsed); + } + } } diff --git a/server/src/test/java/org/opensearch/cluster/node/DiscoveryNodeTests.java b/server/src/test/java/org/opensearch/cluster/node/DiscoveryNodeTests.java index c8a6fc76ce820..b3a4c7e34125e 100644 --- a/server/src/test/java/org/opensearch/cluster/node/DiscoveryNodeTests.java +++ b/server/src/test/java/org/opensearch/cluster/node/DiscoveryNodeTests.java @@ -33,15 +33,20 @@ package org.opensearch.cluster.node; import org.opensearch.Version; +import org.opensearch.cluster.metadata.Metadata; import org.opensearch.common.io.stream.BytesStreamOutput; import org.opensearch.common.settings.Setting; import org.opensearch.common.settings.Settings; +import org.opensearch.common.xcontent.json.JsonXContent; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.transport.TransportAddress; +import org.opensearch.core.xcontent.ToXContent; +import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.node.remotestore.RemoteStoreNodeAttribute; import org.opensearch.test.NodeRoles; import org.opensearch.test.OpenSearchTestCase; +import java.io.IOException; import java.net.InetAddress; import java.util.Collections; import java.util.HashMap; @@ -53,6 +58,9 @@ import static java.util.Collections.emptyMap; import static java.util.Collections.emptySet; +import static java.util.Collections.singletonMap; +import static org.opensearch.cluster.metadata.Metadata.CONTEXT_MODE_API; +import static org.opensearch.cluster.metadata.Metadata.CONTEXT_MODE_GATEWAY; import static org.opensearch.test.NodeRoles.nonRemoteClusterClientNode; import static org.opensearch.test.NodeRoles.nonSearchNode; import static org.opensearch.test.NodeRoles.remoteClusterClientNode; @@ -249,4 +257,70 @@ public void testDiscoveryNodeIsSearchNode() { final DiscoveryNode node = DiscoveryNode.createLocal(settingWithSearchRole, buildNewFakeTransportAddress(), "node"); assertThat(node.isSearchNode(), equalTo(true)); } + + public void testToXContentInAPIMode() throws IOException { + final DiscoveryNode node = DiscoveryNode.createLocal( + Settings.EMPTY, + new TransportAddress(TransportAddress.META_ADDRESS, 9200), + "node_1" + ); + XContentBuilder builder = JsonXContent.contentBuilder().prettyPrint(); + builder.startObject(); + node.toXContent(builder, new ToXContent.MapParams(singletonMap(Metadata.CONTEXT_MODE_PARAM, CONTEXT_MODE_API))); + builder.endObject(); + + String expectedNodeAPUXContent = "{\n" + + " \"node_1\" : {\n" + + " \"name\" : \"" + + node.getName() + + "\",\n" + + " \"ephemeral_id\" : \"" + + node.getEphemeralId() + + "\",\n" + + " \"transport_address\" : \"0.0.0.0:9200\",\n" + + " \"attributes\" : { }\n" + + " }\n" + + "}"; + + assertEquals(expectedNodeAPUXContent, builder.toString()); + } + + public void testToXContentInGatewayMode() throws IOException { + final DiscoveryNode node = DiscoveryNode.createLocal( + Settings.EMPTY, + new TransportAddress(TransportAddress.META_ADDRESS, 9200), + "node_1" + ); + XContentBuilder builder = JsonXContent.contentBuilder().prettyPrint(); + builder.startObject(); + node.toXContent(builder, new ToXContent.MapParams(singletonMap(Metadata.CONTEXT_MODE_PARAM, CONTEXT_MODE_GATEWAY))); + builder.endObject(); + + String expectedNodeAPUXContent = "{\n" + + " \"node_1\" : {\n" + + " \"name\" : \"" + + node.getName() + + "\",\n" + + " \"ephemeral_id\" : \"" + + node.getEphemeralId() + + "\",\n" + + " \"transport_address\" : \"0.0.0.0:9200\",\n" + + " \"attributes\" : { },\n" + + " \"host_name\" : \"0.0.0.0\",\n" + + " \"host_address\" : \"0.0.0.0\",\n" + + " \"version\" : \"" + + node.getVersion() + + "\",\n" + + " \"roles\" : [\n" + + " \"cluster_manager\",\n" + + " \"data\",\n" + + " \"ingest\",\n" + + " \"remote_cluster_client\"\n" + + " ]\n" + + " }\n" + + "}"; + + assertEquals(expectedNodeAPUXContent, builder.toString()); + + } } diff --git a/server/src/test/java/org/opensearch/cluster/node/DiscoveryNodesTests.java b/server/src/test/java/org/opensearch/cluster/node/DiscoveryNodesTests.java index d2450859dfcd4..6a22601fefa93 100644 --- a/server/src/test/java/org/opensearch/cluster/node/DiscoveryNodesTests.java +++ b/server/src/test/java/org/opensearch/cluster/node/DiscoveryNodesTests.java @@ -36,10 +36,20 @@ import org.opensearch.LegacyESVersion; import org.opensearch.Version; +import org.opensearch.cluster.metadata.Metadata; import org.opensearch.common.settings.Setting; +import org.opensearch.common.xcontent.json.JsonXContent; +import org.opensearch.core.common.bytes.BytesReference; import org.opensearch.core.common.transport.TransportAddress; +import org.opensearch.core.xcontent.MediaType; +import org.opensearch.core.xcontent.MediaTypeRegistry; +import org.opensearch.core.xcontent.ToXContent; +import org.opensearch.core.xcontent.XContentBuilder; +import org.opensearch.core.xcontent.XContentParser; import org.opensearch.test.OpenSearchTestCase; +import org.opensearch.test.XContentTestUtils; +import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; @@ -47,6 +57,7 @@ import java.util.HashMap; import java.util.HashSet; import java.util.List; +import java.util.Locale; import java.util.Map; import java.util.Objects; import java.util.Set; @@ -54,6 +65,9 @@ import java.util.stream.Collectors; import java.util.stream.StreamSupport; +import static java.util.Collections.singletonMap; +import static org.opensearch.cluster.metadata.Metadata.CONTEXT_MODE_API; +import static org.opensearch.cluster.metadata.Metadata.CONTEXT_MODE_GATEWAY; import static org.hamcrest.CoreMatchers.containsString; import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.Matchers.arrayContainingInAnyOrder; @@ -497,6 +511,164 @@ public void testMaxMinNodeVersion() { assertEquals(LegacyESVersion.fromString("5.1.0"), build.getMinNodeVersion()); } + public void testToXContentInAPIMode() throws IOException { + DiscoveryNodes nodes = buildDiscoveryNodes(); + + String expectedNodeAPUXContent = "%1$s\"node_%2$d\" : {\n" + + "%1$s \"name\" : \"name_%2$d\",\n" + + "%1$s \"ephemeral_id\" : \"%3$s\",\n" + + "%1$s \"transport_address\" : \"%4$s\",\n" + + "%1$s \"attributes\" : {%5$s}\n" + + "%1$s}"; + + logger.info(nodes); + + verifyToXContentInContextMode( + CONTEXT_MODE_API, + nodes, + "{\n" + " \"nodes\" : {\n" + nodes.getNodes().entrySet().stream().map(entry -> { + int id = Integer.parseInt(entry.getKey().split("_")[1]); + return String.format( + Locale.ROOT, + expectedNodeAPUXContent, + " ", + id, + entry.getValue().getEphemeralId(), + entry.getValue().getAddress().toString(), + entry.getValue().getAttributes().isEmpty() + ? " " + : "\n" + " \"custom\" : \"" + entry.getValue().getAttributes().get("custom") + "\"\n " + ); + }).collect(Collectors.joining(",\n")) + "\n" + " }\n" + "}" + ); + } + + public void testToXContentInGatewayMode() throws IOException { + DiscoveryNodes nodes = buildDiscoveryNodes(); + String expectedXContent = getExpectedXContentInGatewayMode(nodes); + + verifyToXContentInContextMode(CONTEXT_MODE_GATEWAY, nodes, expectedXContent); + } + + private String getExpectedXContentInGatewayMode(DiscoveryNodes nodes) { + /* + * Following formatting creates a string like following: + * "node_1" : { + * "name" : "name_1", + * "ephemeral_id" : "3Q3xRwYKScWqBgVCrWmNCQ", + * "transport_address" : "0.0.0.0:2", + * "attributes" : { + * "custom" : "PKU" + * }, + * "host_name" : "0.0.0.0", + * "host_address" : "0.0.0.0", + * "version" : "3.0.0", + * "roles" : [ + * "custom_role", + * "ingest", + * "remote_cluster_client", + * "search" + * ] + * } + * */ + String expectedNodeAPUXContent = "%1$s\"node_%2$d\" : {\n" + + "%1$s \"name\" : \"name_%2$d\",\n" + + "%1$s \"ephemeral_id\" : \"%3$s\",\n" + + "%1$s \"transport_address\" : \"%4$s\",\n" + + "%1$s \"attributes\" : {%5$s},\n" + + "%1$s \"host_name\" : \"0.0.0.0\",\n" + + "%1$s \"host_address\" : \"0.0.0.0\",\n" + + "%1$s \"version\" : \"%6$s\",\n" + + "%1$s \"roles\" : [%7$s]\n" + + "%1$s}"; + + return "{\n" + " \"nodes\" : {\n" + nodes.getNodes().entrySet().stream().map(entry -> { + int id = Integer.parseInt(entry.getKey().split("_")[1]); + DiscoveryNode node = entry.getValue(); + String indent = " "; + return String.format( + Locale.ROOT, + expectedNodeAPUXContent, + indent, + id, + node.getEphemeralId(), + entry.getValue().getAddress().toString(), + node.getAttributes().isEmpty() + ? " " + : "\n" + indent + " \"custom\" : \"" + node.getAttributes().get("custom") + "\"\n " + indent, + node.getVersion(), + node.getRoles().isEmpty() + ? " " + : "\n" + + indent + + " \"" + + node.getRoles().stream().map(DiscoveryNodeRole::roleName).collect(Collectors.joining("\",\n" + indent + " \"")) + + "\"\n " + + indent + ); + }).collect(Collectors.joining(",\n")) + + "\n" + + " },\n" + + " \"cluster_manager\" : \"" + + nodes.getClusterManagerNodeId() + + "\"\n" + + "}"; + } + + public void verifyToXContentInContextMode(String context, DiscoveryNodes nodes, String expected) throws IOException { + XContentBuilder builder = JsonXContent.contentBuilder().prettyPrint(); + builder.startObject(); + nodes.toXContent(builder, new ToXContent.MapParams(singletonMap(Metadata.CONTEXT_MODE_PARAM, context))); + builder.endObject(); + + assertEquals(expected, builder.toString()); + } + + public void testFromXContent() throws IOException { + doFromXContentTestWithRandomFields(false); + } + + public void testFromXContentWithRandomFields() throws IOException { + doFromXContentTestWithRandomFields(true); + } + + private void doFromXContentTestWithRandomFields(boolean addRandomFields) throws IOException { + DiscoveryNodes nodes = buildDiscoveryNodes(); + boolean humanReadable = randomBoolean(); + final MediaType mediaType = MediaTypeRegistry.JSON; + BytesReference originalBytes = toShuffledXContent( + nodes, + mediaType, + new ToXContent.MapParams(singletonMap(Metadata.CONTEXT_MODE_PARAM, CONTEXT_MODE_GATEWAY)), + humanReadable + ); + + if (addRandomFields) { + String unsupportedField = "unsupported_field"; + BytesReference mutated = BytesReference.bytes( + XContentTestUtils.insertIntoXContent( + mediaType.xContent(), + originalBytes, + Collections.singletonList(""), + () -> unsupportedField, + () -> randomAlphaOfLengthBetween(3, 10) + ) + ); + IllegalArgumentException iae = expectThrows( + IllegalArgumentException.class, + () -> DiscoveryNodes.fromXContent(createParser(mediaType.xContent(), mutated)) + ); + assertEquals(iae.getMessage(), "unexpected value field " + unsupportedField); + } else { + try (XContentParser parser = createParser(mediaType.xContent(), originalBytes)) { + DiscoveryNodes parsedNodes = DiscoveryNodes.fromXContent(parser); + assertEquals(nodes.getSize(), parsedNodes.getSize()); + nodes.forEach(node -> node.equals(parsedNodes.get(node.getId()))); + assertEquals(nodes.getClusterManagerNodeId(), parsedNodes.getClusterManagerNodeId()); + } + } + } + private DiscoveryNode buildDiscoveryNodeFromExisting(DiscoveryNode existing, Version newVersion) { return new DiscoveryNode( existing.getName(), 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 9a9cbfa153259..131d1a4759ba2 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 @@ -8,30 +8,47 @@ package org.opensearch.cluster.routing.remote; +import org.junit.After; +import org.junit.Before; +import org.opensearch.Version; +import org.opensearch.cluster.ClusterName; +import org.opensearch.cluster.ClusterState; +import org.opensearch.cluster.metadata.IndexMetadata; +import org.opensearch.cluster.routing.IndexRoutingTable; +import org.opensearch.cluster.routing.RoutingTable; +import org.opensearch.cluster.routing.remote.RemoteRoutingTableService; +import org.opensearch.common.settings.ClusterSettings; import org.opensearch.common.settings.Settings; import org.opensearch.common.util.FeatureFlags; +import org.opensearch.core.index.Index; import org.opensearch.repositories.FilterRepository; import org.opensearch.repositories.RepositoriesService; import org.opensearch.repositories.RepositoryMissingException; import org.opensearch.repositories.blobstore.BlobStoreRepository; import org.opensearch.test.OpenSearchTestCase; -import org.junit.After; -import org.junit.Before; - +import org.opensearch.threadpool.TestThreadPool; +import org.opensearch.threadpool.ThreadPool; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; import java.util.function.Supplier; +import static org.mockito.Mockito.*; import static org.opensearch.common.util.FeatureFlags.REMOTE_PUBLICATION_EXPERIMENTAL; +import static org.opensearch.common.util.FeatureFlags.REMOTE_PUBLICATION_EXPERIMENTAL_SETTING; import static org.opensearch.node.remotestore.RemoteStoreNodeAttribute.REMOTE_STORE_ROUTING_TABLE_REPOSITORY_NAME_ATTRIBUTE_KEY; -import static org.mockito.Mockito.doThrow; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; public class RemoteRoutingTableServiceTests extends OpenSearchTestCase { private RemoteRoutingTableService remoteRoutingTableService; + private ClusterSettings clusterSettings; private Supplier repositoriesServiceSupplier; private RepositoriesService repositoriesService; private BlobStoreRepository blobStoreRepository; + private final ThreadPool threadPool = new TestThreadPool(getClass().getName()); @Before public void setup() { @@ -40,33 +57,45 @@ public void setup() { when(repositoriesServiceSupplier.get()).thenReturn(repositoriesService); Settings settings = Settings.builder() + .put(RemoteRoutingTableService.REMOTE_ROUTING_TABLE_ENABLED_SETTING.getKey(), true) .put("node.attr." + REMOTE_STORE_ROUTING_TABLE_REPOSITORY_NAME_ATTRIBUTE_KEY, "routing_repository") .build(); + clusterSettings = new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); + blobStoreRepository = mock(BlobStoreRepository.class); when(repositoriesService.repository("routing_repository")).thenReturn(blobStoreRepository); Settings nodeSettings = Settings.builder().put(REMOTE_PUBLICATION_EXPERIMENTAL, "true").build(); FeatureFlags.initializeFeatureFlags(nodeSettings); - remoteRoutingTableService = new RemoteRoutingTableService(repositoriesServiceSupplier, settings); + remoteRoutingTableService = new RemoteRoutingTableService( + repositoriesServiceSupplier, + settings, + threadPool + ); } @After public void teardown() throws Exception { super.tearDown(); remoteRoutingTableService.close(); + threadPool.shutdown(); } + public void testFailInitializationWhenRemoteRoutingDisabled() { final Settings settings = Settings.builder().build(); - assertThrows(AssertionError.class, () -> new RemoteRoutingTableService(repositoriesServiceSupplier, settings)); + assertThrows( + AssertionError.class, + () -> new RemoteRoutingTableService( + repositoriesServiceSupplier, + settings, + new ThreadPool(settings) + ) + ); } - public void testFailStartWhenRepositoryNotSet() { - doThrow(new RepositoryMissingException("repository missing")).when(repositoriesService).repository("routing_repository"); - assertThrows(RepositoryMissingException.class, () -> remoteRoutingTableService.start()); - } public void testFailStartWhenNotBlobRepository() { final FilterRepository filterRepository = mock(FilterRepository.class); @@ -74,4 +103,86 @@ public void testFailStartWhenNotBlobRepository() { assertThrows(AssertionError.class, () -> remoteRoutingTableService.start()); } + public void testGetChangedIndicesRouting() { + String indexName = randomAlphaOfLength(randomIntBetween(1, 50)); + final Index index = new Index(indexName, "uuid"); + final IndexMetadata indexMetadata = new IndexMetadata.Builder(indexName).settings( + Settings.builder() + .put(IndexMetadata.SETTING_VERSION_CREATED, Version.CURRENT) + .put(IndexMetadata.SETTING_INDEX_UUID, "uuid") + .build() + ).numberOfShards(1).numberOfReplicas(1).build(); + + RoutingTable routingTable = RoutingTable.builder().addAsNew(indexMetadata).build(); + ClusterState state = ClusterState.builder(ClusterName.DEFAULT).routingTable(routingTable).build(); + + assertEquals(0, RemoteRoutingTableService.getIndicesRoutingMapDiff(state.getRoutingTable(), state.getRoutingTable()).getUpserts().size()); + + //Reversing order to check for equality without order. + IndexRoutingTable indexRouting = routingTable.getIndicesRouting().get(indexName); + IndexRoutingTable indexRoutingTable = IndexRoutingTable.builder(index).addShard(indexRouting.getShards().get(0).replicaShards().get(0)) + .addShard(indexRouting.getShards().get(0).primaryShard()).build(); + ClusterState newState = ClusterState.builder(ClusterName.DEFAULT).routingTable(RoutingTable.builder().add(indexRoutingTable).build()).build(); + assertEquals(0, RemoteRoutingTableService.getIndicesRoutingMapDiff(state.getRoutingTable(), newState.getRoutingTable()).getUpserts().size()); + } + + public void testIndicesRoutingDiffWhenIndexDeleted() { + + ClusterState state = createIndices(randomIntBetween(1,100)); + RoutingTable routingTable = state.routingTable(); + + List allIndices = new ArrayList<>(); + routingTable.getIndicesRouting().forEach((k,v) -> allIndices.add(k)); + + String indexNameToDelete = allIndices.get(randomIntBetween(0, allIndices.size()-1)); + RoutingTable updatedRoutingTable = RoutingTable.builder(routingTable).remove(indexNameToDelete).build(); + + assertEquals(1, RemoteRoutingTableService.getIndicesRoutingMapDiff(state.getRoutingTable(), updatedRoutingTable).getDeletes().size()); + assertEquals(indexNameToDelete, RemoteRoutingTableService.getIndicesRoutingMapDiff(state.getRoutingTable(), updatedRoutingTable).getDeletes().get(0)); + } + + public void testIndicesRoutingDiffWhenIndexDeletedAndAdded() { + + ClusterState state = createIndices(randomIntBetween(1,100)); + RoutingTable routingTable = state.routingTable(); + + List allIndices = new ArrayList<>(); + routingTable.getIndicesRouting().forEach((k,v) -> allIndices.add(k)); + + String indexNameToDelete = allIndices.get(randomIntBetween(0, allIndices.size()-1)); + RoutingTable.Builder updatedRoutingTableBuilder = RoutingTable.builder(routingTable).remove(indexNameToDelete); + + String indexName = randomAlphaOfLength(randomIntBetween(1, 50)); + final IndexMetadata indexMetadata = new IndexMetadata.Builder(indexName).settings( + Settings.builder() + .put(IndexMetadata.SETTING_VERSION_CREATED, Version.CURRENT) + .put(IndexMetadata.SETTING_INDEX_UUID, "uuid") + .build() + ).numberOfShards(1).numberOfReplicas(1).build(); + + RoutingTable updatedRoutingTable = updatedRoutingTableBuilder.addAsNew(indexMetadata).build(); + + assertEquals(1, RemoteRoutingTableService.getIndicesRoutingMapDiff(state.getRoutingTable(), updatedRoutingTable).getDeletes().size()); + assertEquals(indexNameToDelete, RemoteRoutingTableService.getIndicesRoutingMapDiff(state.getRoutingTable(), updatedRoutingTable).getDeletes().get(0)); + + assertEquals(1, RemoteRoutingTableService.getIndicesRoutingMapDiff(state.getRoutingTable(), updatedRoutingTable).getUpserts().size()); + assertTrue(RemoteRoutingTableService.getIndicesRoutingMapDiff(state.getRoutingTable(), updatedRoutingTable).getUpserts().containsKey(indexName)); + } + + private ClusterState createIndices(int numberOfIndices) { + RoutingTable.Builder routingTableBuilder = RoutingTable.builder(); + for(int i=0; i< numberOfIndices; i++) { + String indexName = randomAlphaOfLength(randomIntBetween(1, 50)); + final Index index = new Index(indexName, "uuid"); + final IndexMetadata indexMetadata = new IndexMetadata.Builder(indexName).settings( + Settings.builder() + .put(IndexMetadata.SETTING_VERSION_CREATED, Version.CURRENT) + .put(IndexMetadata.SETTING_INDEX_UUID, "uuid") + .build() + ).numberOfShards(1).numberOfReplicas(1).build(); + + routingTableBuilder.addAsNew(indexMetadata); + } + return ClusterState.builder(ClusterName.DEFAULT).routingTable(routingTableBuilder.build()).build(); + } } diff --git a/server/src/test/java/org/opensearch/discovery/DiscoveryModuleTests.java b/server/src/test/java/org/opensearch/discovery/DiscoveryModuleTests.java index b33ebf8333b36..0b951c1927c71 100644 --- a/server/src/test/java/org/opensearch/discovery/DiscoveryModuleTests.java +++ b/server/src/test/java/org/opensearch/discovery/DiscoveryModuleTests.java @@ -128,7 +128,8 @@ private DiscoveryModule newModule(Settings settings, List plugi mock(RerouteService.class), null, new PersistedStateRegistry(), - remoteStoreNodeService + remoteStoreNodeService, + null ); } diff --git a/server/src/test/java/org/opensearch/gateway/GatewayMetaStatePersistedStateTests.java b/server/src/test/java/org/opensearch/gateway/GatewayMetaStatePersistedStateTests.java index 3ba98c44f8d3e..5ee819618daa4 100644 --- a/server/src/test/java/org/opensearch/gateway/GatewayMetaStatePersistedStateTests.java +++ b/server/src/test/java/org/opensearch/gateway/GatewayMetaStatePersistedStateTests.java @@ -69,6 +69,7 @@ import org.opensearch.gateway.remote.ClusterMetadataManifest; import org.opensearch.gateway.remote.RemoteClusterStateService; import org.opensearch.gateway.remote.RemotePersistenceStats; +import org.opensearch.gateway.remote.model.RemoteUploadDetails; import org.opensearch.index.recovery.RemoteStoreRestoreService; import org.opensearch.index.recovery.RemoteStoreRestoreService.RemoteRestoreResult; import org.opensearch.index.remote.RemoteIndexPathUploader; @@ -462,9 +463,7 @@ public void testDataOnlyNodePersistence() throws Exception { }); when(transportService.getThreadPool()).thenReturn(threadPool); ClusterService clusterService = mock(ClusterService.class); - when(clusterService.getClusterSettings()).thenReturn( - new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS) - ); + when(clusterService.getClusterSettings()).thenReturn(new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS)); final PersistedClusterStateService persistedClusterStateService = new PersistedClusterStateService( nodeEnvironment, xContentRegistry(), @@ -487,10 +486,11 @@ public void testDataOnlyNodePersistence() throws Exception { nodeEnvironment.nodeId(), repositoriesServiceSupplier, settings, - clusterSettings, + clusterService, () -> 0L, threadPool, - List.of(new RemoteIndexPathUploader(threadPool, settings, repositoriesServiceSupplier, clusterSettings)) + List.of(new RemoteIndexPathUploader(threadPool, settings, repositoriesServiceSupplier, clusterSettings)), + null ); } else { return null; @@ -725,9 +725,10 @@ public void testRemotePersistedState() throws IOException { final RemoteClusterStateService remoteClusterStateService = Mockito.mock(RemoteClusterStateService.class); final ClusterMetadataManifest manifest = ClusterMetadataManifest.builder().clusterTerm(1L).stateVersion(5L).build(); final String previousClusterUUID = "prev-cluster-uuid"; - Mockito.when(remoteClusterStateService.writeFullMetadata(Mockito.any(), Mockito.any())).thenReturn(manifest); + Mockito.when(remoteClusterStateService.writeFullMetadata(Mockito.any(), Mockito.any())).thenReturn(new RemoteUploadDetails(manifest, "path/to/manifest")); - Mockito.when(remoteClusterStateService.writeIncrementalMetadata(Mockito.any(), Mockito.any(), Mockito.any())).thenReturn(manifest); + Mockito.when(remoteClusterStateService.writeIncrementalMetadata(Mockito.any(), Mockito.any(), Mockito.any())) + .thenReturn(new RemoteUploadDetails(manifest, "path/to/manifest")); CoordinationState.PersistedState remotePersistedState = new RemotePersistedState(remoteClusterStateService, previousClusterUUID); assertThat(remotePersistedState.getLastAcceptedState(), nullValue()); @@ -781,9 +782,10 @@ public void testRemotePersistedStateNotCommitted() throws IOException { .build(); Mockito.when(remoteClusterStateService.getLatestClusterMetadataManifest(Mockito.any(), Mockito.any())) .thenReturn(Optional.of(manifest)); - Mockito.when(remoteClusterStateService.writeFullMetadata(Mockito.any(), Mockito.any())).thenReturn(manifest); + Mockito.when(remoteClusterStateService.writeFullMetadata(Mockito.any(), Mockito.any())).thenReturn(new RemoteUploadDetails(manifest, "path/to/manifest")); - Mockito.when(remoteClusterStateService.writeIncrementalMetadata(Mockito.any(), Mockito.any(), Mockito.any())).thenReturn(manifest); + Mockito.when(remoteClusterStateService.writeIncrementalMetadata(Mockito.any(), Mockito.any(), Mockito.any())) + .thenReturn(new RemoteUploadDetails(manifest, "path/to/manifest")); CoordinationState.PersistedState remotePersistedState = new RemotePersistedState( remoteClusterStateService, ClusterState.UNKNOWN_UUID 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 0b3cd49140939..2466c3fd2b869 100644 --- a/server/src/test/java/org/opensearch/gateway/remote/ClusterMetadataManifestTests.java +++ b/server/src/test/java/org/opensearch/gateway/remote/ClusterMetadataManifestTests.java @@ -20,6 +20,10 @@ import org.opensearch.core.xcontent.XContentParser; import org.opensearch.gateway.remote.ClusterMetadataManifest.UploadedIndexMetadata; import org.opensearch.gateway.remote.ClusterMetadataManifest.UploadedMetadataAttribute; +import org.opensearch.gateway.remote.model.RemoteCoordinationMetadata; +import org.opensearch.gateway.remote.model.RemoteCustomMetadata; +import org.opensearch.gateway.remote.model.RemotePersistentSettingsMetadata; +import org.opensearch.gateway.remote.model.RemoteTemplatesMetadata; import org.opensearch.test.EqualsHashCodeTestUtils; import org.opensearch.test.OpenSearchTestCase; @@ -33,6 +37,7 @@ import static org.opensearch.gateway.remote.ClusterMetadataManifest.CODEC_V0; import static org.opensearch.gateway.remote.ClusterMetadataManifest.CODEC_V1; +import static org.opensearch.gateway.remote.ClusterMetadataManifest.CODEC_V2; public class ClusterMetadataManifestTests extends OpenSearchTestCase { @@ -91,40 +96,40 @@ public void testClusterMetadataManifestXContentV1() throws IOException { public void testClusterMetadataManifestXContent() throws IOException { UploadedIndexMetadata uploadedIndexMetadata = new UploadedIndexMetadata("test-index", "test-uuid", "/test/upload/path"); - ClusterMetadataManifest originalManifest = new ClusterMetadataManifest( - 1L, - 1L, - "test-cluster-uuid", - "test-state-uuid", - Version.CURRENT, - "test-node-id", - false, - ClusterMetadataManifest.CODEC_V2, - null, - Collections.singletonList(uploadedIndexMetadata), - "prev-cluster-uuid", - true, - new UploadedMetadataAttribute(RemoteClusterStateService.COORDINATION_METADATA, "coordination-file"), - new UploadedMetadataAttribute(RemoteClusterStateService.SETTING_METADATA, "setting-file"), - new UploadedMetadataAttribute(RemoteClusterStateService.TEMPLATES_METADATA, "templates-file"), - Collections.unmodifiableList( + ClusterMetadataManifest.Builder manifestBuilder = ClusterMetadataManifest.builder() + .clusterTerm(1L) + .stateVersion(1L) + .clusterUUID("test-cluster-uuid") + .stateUUID("test-state-uuid") + .opensearchVersion(Version.CURRENT) + .nodeId("test-node-id") + .committed(false) + .codecVersion(CODEC_V2) + .indices(Collections.singletonList(uploadedIndexMetadata)) + .previousClusterUUID("prev-cluster-uuid") + .clusterUUIDCommitted(true) + .coordinationMetadata(new UploadedMetadataAttribute(RemoteCoordinationMetadata.COORDINATION_METADATA, "coordination-file")) + .settingMetadata(new UploadedMetadataAttribute(RemotePersistentSettingsMetadata.SETTING_METADATA, "setting-file")) + .templatesMetadata(new UploadedMetadataAttribute(RemoteTemplatesMetadata.TEMPLATES_METADATA, "templates-file")) + .customMetadataMap(Collections.unmodifiableList( Arrays.asList( new UploadedMetadataAttribute( - RemoteClusterStateService.CUSTOM_METADATA + RemoteClusterStateService.CUSTOM_DELIMITER + RepositoriesMetadata.TYPE, + RemoteCustomMetadata.CUSTOM_METADATA + RemoteCustomMetadata.CUSTOM_DELIMITER + + RepositoriesMetadata.TYPE, "custom--repositories-file" ), new UploadedMetadataAttribute( - RemoteClusterStateService.CUSTOM_METADATA + RemoteClusterStateService.CUSTOM_DELIMITER + IndexGraveyard.TYPE, + RemoteCustomMetadata.CUSTOM_METADATA + RemoteCustomMetadata.CUSTOM_DELIMITER + IndexGraveyard.TYPE, "custom--index_graveyard-file" ), new UploadedMetadataAttribute( - RemoteClusterStateService.CUSTOM_METADATA + RemoteClusterStateService.CUSTOM_DELIMITER + RemoteCustomMetadata.CUSTOM_METADATA + RemoteCustomMetadata.CUSTOM_DELIMITER + WeightedRoutingMetadata.TYPE, "custom--weighted_routing_netadata-file" ) ) - ).stream().collect(Collectors.toMap(UploadedMetadataAttribute::getAttributeName, Function.identity())) - ); + ).stream().collect(Collectors.toMap(UploadedMetadataAttribute::getAttributeName, Function.identity()))); + ClusterMetadataManifest originalManifest = manifestBuilder.build(); final XContentBuilder builder = JsonXContent.contentBuilder(); builder.startObject(); originalManifest.toXContent(builder, ToXContent.EMPTY_PARAMS); @@ -137,40 +142,40 @@ public void testClusterMetadataManifestXContent() throws IOException { } public void testClusterMetadataManifestSerializationEqualsHashCode() { - ClusterMetadataManifest initialManifest = new ClusterMetadataManifest( - 1337L, - 7L, - "HrYF3kP5SmSPWtKlWhnNSA", - "6By9p9G0Rv2MmFYJcPAOgA", - Version.CURRENT, - "B10RX1f5RJenMQvYccCgSQ", - true, - 2, - null, - randomUploadedIndexMetadataList(), - "yfObdx8KSMKKrXf8UyHhM", - true, - new UploadedMetadataAttribute(RemoteClusterStateService.COORDINATION_METADATA, "coordination-file"), - new UploadedMetadataAttribute(RemoteClusterStateService.SETTING_METADATA, "setting-file"), - new UploadedMetadataAttribute(RemoteClusterStateService.TEMPLATES_METADATA, "templates-file"), - Collections.unmodifiableList( + ClusterMetadataManifest.Builder manifestBuilder = ClusterMetadataManifest.builder(); + manifestBuilder.clusterTerm(1337L) + .stateVersion(7L) + .clusterUUID("HrYF3kP5SmSPWtKlWhnNSA") + .stateUUID("6By9p9G0Rv2MmFYJcPAOgA") + .opensearchVersion(Version.CURRENT) + .nodeId("B10RX1f5RJenMQvYccCgSQ") + .committed(true) + .codecVersion(2) + .indices(randomUploadedIndexMetadataList()) + .previousClusterUUID("yfObdx8KSMKKrXf8UyHhM") + .clusterUUIDCommitted(true) + .coordinationMetadata(new UploadedMetadataAttribute(RemoteCoordinationMetadata.COORDINATION_METADATA, "coordination-file")) + .settingMetadata(new UploadedMetadataAttribute(RemotePersistentSettingsMetadata.SETTING_METADATA, "setting-file")) + .templatesMetadata(new UploadedMetadataAttribute(RemoteTemplatesMetadata.TEMPLATES_METADATA, "templates-file")) + .customMetadataMap(Collections.unmodifiableList( Arrays.asList( new UploadedMetadataAttribute( - RemoteClusterStateService.CUSTOM_METADATA + RemoteClusterStateService.CUSTOM_DELIMITER + RepositoriesMetadata.TYPE, + RemoteCustomMetadata.CUSTOM_METADATA + RemoteCustomMetadata.CUSTOM_DELIMITER + + RepositoriesMetadata.TYPE, "custom--repositories-file" ), new UploadedMetadataAttribute( - RemoteClusterStateService.CUSTOM_METADATA + RemoteClusterStateService.CUSTOM_DELIMITER + IndexGraveyard.TYPE, + RemoteCustomMetadata.CUSTOM_METADATA + RemoteCustomMetadata.CUSTOM_DELIMITER + IndexGraveyard.TYPE, "custom--index_graveyard-file" ), new UploadedMetadataAttribute( - RemoteClusterStateService.CUSTOM_METADATA + RemoteClusterStateService.CUSTOM_DELIMITER + RemoteCustomMetadata.CUSTOM_METADATA + RemoteCustomMetadata.CUSTOM_DELIMITER + WeightedRoutingMetadata.TYPE, "custom--weighted_routing_netadata-file" ) ) - ).stream().collect(Collectors.toMap(UploadedMetadataAttribute::getAttributeName, Function.identity())) - ); + ).stream().collect(Collectors.toMap(UploadedMetadataAttribute::getAttributeName, Function.identity()))); + ClusterMetadataManifest initialManifest = manifestBuilder.build(); { // Mutate Cluster Term EqualsHashCodeTestUtils.checkEqualsAndHashCode( initialManifest, @@ -309,6 +314,35 @@ public void testClusterMetadataManifestSerializationEqualsHashCode() { } } + public void testClusterMetadataManifestXContentV2() throws IOException { + UploadedIndexMetadata uploadedIndexMetadata = new UploadedIndexMetadata("test-index", "test-uuid", "/test/upload/path"); + ClusterMetadataManifest.Builder manifestBuilder = ClusterMetadataManifest.builder(); + manifestBuilder.clusterTerm(1L) + .stateVersion(1L) + .clusterUUID("test-cluster-uuid") + .stateUUID("test-state-uuid") + .opensearchVersion(Version.CURRENT) + .nodeId("test-node-id") + .committed(false) + .codecVersion(CODEC_V2) + .indices(Collections.singletonList(uploadedIndexMetadata)) + .previousClusterUUID("prev-cluster-uuid") + .clusterUUIDCommitted(true) + .routingTableVersion(1L) + .indicesRouting(Collections.singletonList(uploadedIndexMetadata)); + + ClusterMetadataManifest originalManifest = manifestBuilder.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); + } + } + private List randomUploadedIndexMetadataList() { final int size = randomIntBetween(1, 10); final List uploadedIndexMetadataList = new ArrayList<>(size); diff --git a/server/src/test/java/org/opensearch/gateway/remote/RemoteClusterStateCleanupManagerTests.java b/server/src/test/java/org/opensearch/gateway/remote/RemoteClusterStateCleanupManagerTests.java new file mode 100644 index 0000000000000..e602748e45346 --- /dev/null +++ b/server/src/test/java/org/opensearch/gateway/remote/RemoteClusterStateCleanupManagerTests.java @@ -0,0 +1,434 @@ +/* + * 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.gateway.remote; + +import org.opensearch.cluster.ClusterName; +import org.opensearch.cluster.ClusterState; +import org.opensearch.cluster.metadata.Metadata; +import org.opensearch.cluster.node.DiscoveryNodes; +import org.opensearch.cluster.service.ClusterApplierService; +import org.opensearch.cluster.service.ClusterService; +import org.opensearch.common.blobstore.BlobContainer; +import org.opensearch.common.blobstore.BlobMetadata; +import org.opensearch.common.blobstore.BlobPath; +import org.opensearch.common.blobstore.BlobStore; +import org.opensearch.common.blobstore.support.PlainBlobMetadata; +import org.opensearch.common.settings.ClusterSettings; +import org.opensearch.common.settings.Settings; +import org.opensearch.common.util.concurrent.AbstractAsyncTask; +import org.opensearch.core.action.ActionListener; +import org.opensearch.repositories.RepositoriesService; +import org.opensearch.repositories.blobstore.BlobStoreRepository; +import org.opensearch.repositories.fs.FsRepository; +import org.opensearch.test.OpenSearchTestCase; +import org.opensearch.test.VersionUtils; +import org.opensearch.threadpool.TestThreadPool; +import org.opensearch.threadpool.ThreadPool; +import org.junit.After; +import org.junit.Before; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashSet; +import java.util.List; +import java.util.Locale; +import java.util.Set; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.Supplier; + +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.doNothing; +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.UploadedIndexMetadata; +import static org.opensearch.gateway.remote.ClusterMetadataManifest.UploadedMetadataAttribute; +import static org.opensearch.gateway.remote.RemoteClusterStateCleanupManager.AsyncStaleFileDeletion; +import static org.opensearch.gateway.remote.RemoteClusterStateCleanupManager.CLUSTER_STATE_CLEANUP_INTERVAL_DEFAULT; +import static org.opensearch.gateway.remote.RemoteClusterStateCleanupManager.REMOTE_CLUSTER_STATE_CLEANUP_INTERVAL_SETTING; +import static org.opensearch.gateway.remote.RemoteClusterStateCleanupManager.RETAINED_MANIFESTS; +import static org.opensearch.gateway.remote.RemoteClusterStateCleanupManager.SKIP_CLEANUP_STATE_CHANGES; +import static org.opensearch.gateway.remote.RemoteClusterStateServiceTests.generateClusterStateWithOneIndex; +import static org.opensearch.gateway.remote.RemoteClusterStateServiceTests.nodesWithLocalNodeClusterManager; +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.GLOBAL_METADATA_PATH_TOKEN; +import static org.opensearch.gateway.remote.RemoteClusterStateUtils.encodeString; +import static org.opensearch.gateway.remote.RemoteClusterStateUtils.getCusterMetadataBasePath; +import static org.opensearch.gateway.remote.model.RemoteClusterMetadataManifest.MANIFEST; +import static org.opensearch.gateway.remote.model.RemoteCoordinationMetadata.COORDINATION_METADATA; +import static org.opensearch.gateway.remote.model.RemotePersistentSettingsMetadata.SETTING_METADATA; +import static org.opensearch.gateway.remote.model.RemoteTemplatesMetadata.TEMPLATES_METADATA; +import static org.opensearch.gateway.remote.model.RemoteIndexMetadata.INDEX_PATH_TOKEN; +import static org.opensearch.node.remotestore.RemoteStoreNodeAttribute.REMOTE_STORE_CLUSTER_STATE_REPOSITORY_NAME_ATTRIBUTE_KEY; +import static org.opensearch.node.remotestore.RemoteStoreNodeAttribute.REMOTE_STORE_REPOSITORY_SETTINGS_ATTRIBUTE_KEY_PREFIX; +import static org.opensearch.node.remotestore.RemoteStoreNodeAttribute.REMOTE_STORE_REPOSITORY_TYPE_ATTRIBUTE_KEY_FORMAT; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyInt; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.doThrow; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +public class RemoteClusterStateCleanupManagerTests extends OpenSearchTestCase { + private RemoteClusterStateCleanupManager remoteClusterStateCleanupManager; + private Supplier repositoriesServiceSupplier; + private RepositoriesService repositoriesService; + private BlobStoreRepository blobStoreRepository; + private BlobStore blobStore; + private ClusterSettings clusterSettings; + private ClusterApplierService clusterApplierService; + private ClusterState clusterState; + private Metadata metadata; + private RemoteClusterStateService remoteClusterStateService; + private final ThreadPool threadPool = new TestThreadPool(getClass().getName()); + + @Before + public void setup() { + repositoriesServiceSupplier = mock(Supplier.class); + repositoriesService = mock(RepositoriesService.class); + when(repositoriesServiceSupplier.get()).thenReturn(repositoriesService); + + String stateRepoTypeAttributeKey = String.format( + Locale.getDefault(), + "node.attr." + REMOTE_STORE_REPOSITORY_TYPE_ATTRIBUTE_KEY_FORMAT, + "remote_store_repository" + ); + String stateRepoSettingsAttributeKeyPrefix = String.format( + Locale.getDefault(), + "node.attr." + REMOTE_STORE_REPOSITORY_SETTINGS_ATTRIBUTE_KEY_PREFIX, + "remote_store_repository" + ); + + Settings settings = Settings.builder() + .put("node.attr." + REMOTE_STORE_CLUSTER_STATE_REPOSITORY_NAME_ATTRIBUTE_KEY, "remote_store_repository") + .put(stateRepoTypeAttributeKey, FsRepository.TYPE) + .put(stateRepoSettingsAttributeKeyPrefix + "location", "randomRepoPath") + .put(RemoteClusterStateService.REMOTE_CLUSTER_STATE_ENABLED_SETTING.getKey(), true) + .build(); + + clusterSettings = new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); + clusterApplierService = mock(ClusterApplierService.class); + clusterState = mock(ClusterState.class); + metadata = mock(Metadata.class); + ClusterService clusterService = mock(ClusterService.class); + when(clusterService.getClusterSettings()).thenReturn(clusterSettings); + when(clusterState.getClusterName()).thenReturn(new ClusterName("test")); + when(metadata.clusterUUID()).thenReturn("testUUID"); + when(clusterState.metadata()).thenReturn(metadata); + when(clusterApplierService.state()).thenReturn(clusterState); + when(clusterService.getClusterApplierService()).thenReturn(clusterApplierService); + + blobStoreRepository = mock(BlobStoreRepository.class); + blobStore = mock(BlobStore.class); + when(blobStoreRepository.blobStore()).thenReturn(blobStore); + when(repositoriesService.repository("remote_store_repository")).thenReturn(blobStoreRepository); + + remoteClusterStateService = mock(RemoteClusterStateService.class); + when(remoteClusterStateService.getStats()).thenReturn(new RemotePersistenceStats()); + when(remoteClusterStateService.getThreadpool()).thenReturn(threadPool); + when(remoteClusterStateService.getBlobStore()).thenReturn(blobStore); + remoteClusterStateCleanupManager = new RemoteClusterStateCleanupManager(remoteClusterStateService, clusterService); + } + + @After + public void teardown() throws Exception { + super.tearDown(); + remoteClusterStateCleanupManager.close(); + threadPool.shutdown(); + } + + public void testDeleteClusterMetadata() throws IOException { + String clusterUUID = "clusterUUID"; + String clusterName = "test-cluster"; + List inactiveBlobs = Arrays.asList( + new PlainBlobMetadata("manifest1.dat", 1L), + new PlainBlobMetadata("manifest2.dat", 1L), + new PlainBlobMetadata("manifest3.dat", 1L) + ); + List activeBlobs = Arrays.asList( + new PlainBlobMetadata("manifest4.dat", 1L), + new PlainBlobMetadata("manifest5.dat", 1L) + ); + UploadedIndexMetadata index1Metadata = new UploadedIndexMetadata("index1", "indexUUID1", "index_metadata1"); + UploadedIndexMetadata index2Metadata = new UploadedIndexMetadata("index2", "indexUUID2", "index_metadata2"); + UploadedIndexMetadata index1UpdatedMetadata = new UploadedIndexMetadata("index1", "indexUUID1", "index_metadata1_updated"); + UploadedMetadataAttribute coordinationMetadata = new UploadedMetadataAttribute(COORDINATION_METADATA, "coordination_metadata"); + UploadedMetadataAttribute templateMetadata = new UploadedMetadataAttribute(TEMPLATES_METADATA, "template_metadata"); + UploadedMetadataAttribute settingMetadata = new UploadedMetadataAttribute(SETTING_METADATA, "settings_metadata"); + UploadedMetadataAttribute coordinationMetadataUpdated = new UploadedMetadataAttribute(COORDINATION_METADATA, "coordination_metadata_updated"); + UploadedMetadataAttribute templateMetadataUpdated = new UploadedMetadataAttribute(TEMPLATES_METADATA, "template_metadata_updated"); + UploadedMetadataAttribute settingMetadataUpdated = new UploadedMetadataAttribute(SETTING_METADATA, "settings_metadata_updated"); + ClusterMetadataManifest manifest1 = ClusterMetadataManifest.builder() + .indices(List.of(index1Metadata)) + .globalMetadataFileName("global_metadata") + .clusterTerm(1L) + .stateVersion(1L) + .codecVersion(CODEC_V1) + .stateUUID(randomAlphaOfLength(10)) + .clusterUUID(clusterUUID) + .nodeId("nodeA") + .opensearchVersion(VersionUtils.randomOpenSearchVersion(random())) + .previousClusterUUID(ClusterState.UNKNOWN_UUID) + .committed(true) + .build(); + ClusterMetadataManifest manifest2 = ClusterMetadataManifest.builder(manifest1) + .indices(List.of(index1Metadata, index2Metadata)) + .codecVersion(CODEC_V2) + .globalMetadataFileName(null) + .coordinationMetadata(coordinationMetadata) + .templatesMetadata(templateMetadata) + .settingMetadata(settingMetadata) + .build(); + ClusterMetadataManifest manifest3 = ClusterMetadataManifest.builder(manifest2) + .indices(List.of(index1UpdatedMetadata, index2Metadata)) + .settingMetadata(settingMetadataUpdated) + .build(); + + // active manifest have reference to index1Updated, index2, settingsUpdated, coordinationUpdated, templates, templatesUpdated + ClusterMetadataManifest manifest4 = ClusterMetadataManifest.builder(manifest3) + .coordinationMetadata(coordinationMetadataUpdated) + .build(); + ClusterMetadataManifest manifest5 = ClusterMetadataManifest.builder(manifest4) + .templatesMetadata(templateMetadataUpdated) + .build(); + + when(remoteClusterStateService.getRemoteManifestManager().fetchRemoteClusterMetadataManifest(eq(clusterName), eq(clusterUUID), any())) + .thenReturn(manifest4, manifest5, manifest1, manifest2, manifest3); + BlobContainer container = mock(BlobContainer.class); + when(blobStore.blobContainer(any())).thenReturn(container); + doNothing().when(container).deleteBlobsIgnoringIfNotExists(any()); + + remoteClusterStateCleanupManager.deleteClusterMetadata(clusterName, clusterUUID, activeBlobs, inactiveBlobs); + verify(container).deleteBlobsIgnoringIfNotExists( + List.of(new BlobPath().add(GLOBAL_METADATA_PATH_TOKEN).buildAsString() + coordinationMetadata.getUploadedFilename() + ".dat", + new BlobPath().add(GLOBAL_METADATA_PATH_TOKEN).buildAsString() + settingMetadata.getUploadedFilename() + ".dat", + new BlobPath().add(GLOBAL_METADATA_PATH_TOKEN).buildAsString() + "global_metadata.dat") + ); + verify(container).deleteBlobsIgnoringIfNotExists( + List.of(new BlobPath().add(INDEX_PATH_TOKEN).add(index1Metadata.getIndexUUID()).buildAsString() + index1Metadata.getUploadedFilePath() + ".dat") + ); + Set staleManifest = new HashSet<>(); + inactiveBlobs.forEach(blob -> staleManifest.add(new BlobPath().add(MANIFEST).buildAsString() + blob.name())); + verify(container).deleteBlobsIgnoringIfNotExists(new ArrayList<>(staleManifest)); + } + + public void testDeleteStaleClusterUUIDs() throws IOException { + final ClusterState clusterState = generateClusterStateWithOneIndex().nodes(nodesWithLocalNodeClusterManager()).build(); + ClusterMetadataManifest clusterMetadataManifest = ClusterMetadataManifest.builder() + .indices(List.of()) + .clusterTerm(1L) + .stateVersion(1L) + .stateUUID(randomAlphaOfLength(10)) + .clusterUUID("cluster-uuid1") + .nodeId("nodeA") + .opensearchVersion(VersionUtils.randomOpenSearchVersion(random())) + .previousClusterUUID(ClusterState.UNKNOWN_UUID) + .committed(true) + .build(); + + BlobPath blobPath = new BlobPath().add("random-path"); + BlobContainer uuidContainerContainer = mock(BlobContainer.class); + BlobContainer manifest2Container = mock(BlobContainer.class); + BlobContainer manifest3Container = mock(BlobContainer.class); + when(blobStore.blobContainer(any())).then(invocation -> { + BlobPath blobPath1 = invocation.getArgument(0); + if (blobPath1.buildAsString().endsWith("cluster-state/")) { + return uuidContainerContainer; + } else if (blobPath1.buildAsString().contains("cluster-state/cluster-uuid2/")) { + return manifest2Container; + } else if (blobPath1.buildAsString().contains("cluster-state/cluster-uuid3/")) { + return manifest3Container; + } else { + throw new IllegalArgumentException("Unexpected blob path " + blobPath1); + } + }); + when( + manifest2Container.listBlobsByPrefixInSortedOrder( + MANIFEST + DELIMITER, + Integer.MAX_VALUE, + BlobContainer.BlobNameSortOrder.LEXICOGRAPHIC + ) + ).thenReturn(List.of(new PlainBlobMetadata("mainfest2", 1L))); + when( + manifest3Container.listBlobsByPrefixInSortedOrder( + MANIFEST + DELIMITER, + Integer.MAX_VALUE, + BlobContainer.BlobNameSortOrder.LEXICOGRAPHIC + ) + ).thenReturn(List.of(new PlainBlobMetadata("mainfest3", 1L))); + Set uuids = new HashSet<>(Arrays.asList("cluster-uuid1", "cluster-uuid2", "cluster-uuid3")); + when(remoteClusterStateService.getAllClusterUUIDs(any())).thenReturn(uuids); + when(getCusterMetadataBasePath(any(), any(), any())).then( + invocationOnMock -> blobPath.add(encodeString(invocationOnMock.getArgument(0))) + .add(CLUSTER_STATE_PATH_TOKEN) + .add((String) invocationOnMock.getArgument(1)) + ); + remoteClusterStateCleanupManager.start(); + remoteClusterStateCleanupManager.deleteStaleClusterUUIDs(clusterState, clusterMetadataManifest); + try { + assertBusy(() -> { + verify(manifest2Container, times(1)).delete(); + verify(manifest3Container, times(1)).delete(); + }); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + public void testRemoteStateCleanupFailureStats() throws IOException { + BlobContainer blobContainer = mock(BlobContainer.class); + doThrow(IOException.class).when(blobContainer).delete(); + when(blobStore.blobContainer(any())).thenReturn(blobContainer); + BlobPath blobPath = new BlobPath().add("random-path"); + when((blobStoreRepository.basePath())).thenReturn(blobPath); + remoteClusterStateCleanupManager.start(); + remoteClusterStateCleanupManager.deleteStaleUUIDsClusterMetadata("cluster1", Arrays.asList("cluster-uuid1")); + try { + assertBusy(() -> { + // wait for stats to get updated + assertTrue(remoteClusterStateCleanupManager.getStats() != null); + assertEquals(0, remoteClusterStateCleanupManager.getStats().getSuccessCount()); + assertEquals(1, remoteClusterStateCleanupManager.getStats().getCleanupAttemptFailedCount()); + }); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + public void testSingleConcurrentExecutionOfStaleManifestCleanup() throws Exception { + BlobContainer blobContainer = mock(BlobContainer.class); + when(blobStore.blobContainer(any())).thenReturn(blobContainer); + + CountDownLatch latch = new CountDownLatch(1); + AtomicInteger callCount = new AtomicInteger(0); + doAnswer(invocation -> { + callCount.incrementAndGet(); + if (latch.await(5000, TimeUnit.SECONDS) == false) { + throw new Exception("Timed out waiting for delete task queuing to complete"); + } + return null; + }).when(blobContainer) + .listBlobsByPrefixInSortedOrder( + any(String.class), + any(int.class), + any(BlobContainer.BlobNameSortOrder.class), + any(ActionListener.class) + ); + + remoteClusterStateCleanupManager.start(); + remoteClusterStateCleanupManager.deleteStaleClusterMetadata("cluster-name", "cluster-uuid", RETAINED_MANIFESTS); + remoteClusterStateCleanupManager.deleteStaleClusterMetadata("cluster-name", "cluster-uuid", RETAINED_MANIFESTS); + + latch.countDown(); + assertBusy(() -> assertEquals(1, callCount.get())); + } + + public void testRemoteClusterStateCleanupSetting() { + remoteClusterStateCleanupManager.start(); + // verify default value + assertEquals(CLUSTER_STATE_CLEANUP_INTERVAL_DEFAULT, remoteClusterStateCleanupManager.getStaleFileCleanupInterval()); + + // verify update interval + int cleanupInterval = randomIntBetween(1, 10); + Settings newSettings = Settings.builder().put("cluster.remote_store.state.cleanup_interval", cleanupInterval + "s").build(); + clusterSettings.applySettings(newSettings); + assertEquals(cleanupInterval, remoteClusterStateCleanupManager.getStaleFileCleanupInterval().seconds()); + } + + public void testRemoteCleanupTaskScheduled() { + AbstractAsyncTask cleanupTask = remoteClusterStateCleanupManager.getStaleFileDeletionTask(); + assertNull(cleanupTask); + // now the task should be initialized + remoteClusterStateCleanupManager.start(); + assertNotNull(remoteClusterStateCleanupManager.getStaleFileDeletionTask()); + assertTrue(remoteClusterStateCleanupManager.getStaleFileDeletionTask().mustReschedule()); + assertEquals( + clusterSettings.get(REMOTE_CLUSTER_STATE_CLEANUP_INTERVAL_SETTING), + remoteClusterStateCleanupManager.getStaleFileDeletionTask().getInterval() + ); + assertTrue(remoteClusterStateCleanupManager.getStaleFileDeletionTask().isScheduled()); + assertFalse(remoteClusterStateCleanupManager.getStaleFileDeletionTask().isClosed()); + } + + public void testRemoteCleanupSkipsOnOnlyElectedClusterManager() { + DiscoveryNodes nodes = mock(DiscoveryNodes.class); + when(nodes.isLocalNodeElectedClusterManager()).thenReturn(false); + when(clusterState.nodes()).thenReturn(nodes); + RemoteClusterStateCleanupManager spyManager = spy(remoteClusterStateCleanupManager); + AtomicInteger callCount = new AtomicInteger(0); + doAnswer(invocation -> callCount.incrementAndGet()).when(spyManager).deleteStaleClusterMetadata(any(), any(), anyInt()); + spyManager.cleanUpStaleFiles(); + assertEquals(0, callCount.get()); + + when(nodes.isLocalNodeElectedClusterManager()).thenReturn(true); + when(clusterState.version()).thenReturn(randomLongBetween(11, 20)); + spyManager.cleanUpStaleFiles(); + assertEquals(1, callCount.get()); + } + + public void testRemoteCleanupSkipsIfVersionIncrementLessThanThreshold() { + DiscoveryNodes nodes = mock(DiscoveryNodes.class); + long version = randomLongBetween(1, SKIP_CLEANUP_STATE_CHANGES); + when(clusterApplierService.state()).thenReturn(clusterState); + when(nodes.isLocalNodeElectedClusterManager()).thenReturn(true); + when(clusterState.nodes()).thenReturn(nodes); + when(clusterState.version()).thenReturn(version); + + RemoteClusterStateCleanupManager spyManager = spy(remoteClusterStateCleanupManager); + AtomicInteger callCount = new AtomicInteger(0); + doAnswer(invocation -> callCount.incrementAndGet()).when(spyManager).deleteStaleClusterMetadata(any(), any(), anyInt()); + + remoteClusterStateCleanupManager.cleanUpStaleFiles(); + assertEquals(0, callCount.get()); + } + + public void testRemoteCleanupCallsDeleteIfVersionIncrementGreaterThanThreshold() { + DiscoveryNodes nodes = mock(DiscoveryNodes.class); + long version = randomLongBetween(SKIP_CLEANUP_STATE_CHANGES + 1, SKIP_CLEANUP_STATE_CHANGES + 10); + when(clusterApplierService.state()).thenReturn(clusterState); + when(nodes.isLocalNodeElectedClusterManager()).thenReturn(true); + when(clusterState.nodes()).thenReturn(nodes); + when(clusterState.version()).thenReturn(version); + + RemoteClusterStateCleanupManager spyManager = spy(remoteClusterStateCleanupManager); + AtomicInteger callCount = new AtomicInteger(0); + doAnswer(invocation -> callCount.incrementAndGet()).when(spyManager).deleteStaleClusterMetadata(any(), any(), anyInt()); + + // using spied cleanup manager so that stubbed deleteStaleClusterMetadata is called + spyManager.cleanUpStaleFiles(); + assertEquals(1, callCount.get()); + } + + public void testRemoteCleanupSchedulesEvenAfterFailure() { + remoteClusterStateCleanupManager.start(); + RemoteClusterStateCleanupManager spyManager = spy(remoteClusterStateCleanupManager); + AtomicInteger callCount = new AtomicInteger(0); + doAnswer(invocationOnMock -> { + callCount.incrementAndGet(); + throw new RuntimeException("Test exception"); + }).when(spyManager).cleanUpStaleFiles(); + AsyncStaleFileDeletion task = new AsyncStaleFileDeletion(spyManager); + assertTrue(task.isScheduled()); + task.run(); + // Task is still scheduled after the failure + assertTrue(task.isScheduled()); + assertEquals(1, callCount.get()); + + task.run(); + // Task is still scheduled after the failure + assertTrue(task.isScheduled()); + assertEquals(2, callCount.get()); + } +} 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 aa5996d734d27..0e2c3efae428d 100644 --- a/server/src/test/java/org/opensearch/gateway/remote/RemoteClusterStateServiceTests.java +++ b/server/src/test/java/org/opensearch/gateway/remote/RemoteClusterStateServiceTests.java @@ -17,8 +17,11 @@ import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.cluster.metadata.IndexTemplateMetadata; import org.opensearch.cluster.metadata.Metadata; +import org.opensearch.cluster.metadata.Metadata.Custom; import org.opensearch.cluster.metadata.TemplatesMetadata; import org.opensearch.cluster.node.DiscoveryNodes; +import org.opensearch.cluster.service.ClusterService; +import org.opensearch.cluster.routing.remote.RemoteRoutingTableService; import org.opensearch.common.blobstore.AsyncMultiStreamBlobContainer; import org.opensearch.common.blobstore.BlobContainer; import org.opensearch.common.blobstore.BlobMetadata; @@ -42,13 +45,21 @@ import org.opensearch.core.xcontent.NamedXContentRegistry; import org.opensearch.gateway.remote.ClusterMetadataManifest.UploadedIndexMetadata; import org.opensearch.gateway.remote.ClusterMetadataManifest.UploadedMetadataAttribute; +import org.opensearch.gateway.remote.model.RemoteClusterMetadataManifest; +import org.opensearch.gateway.remote.model.RemoteCoordinationMetadata; +import org.opensearch.gateway.remote.model.RemoteCustomMetadata; +import org.opensearch.gateway.remote.model.RemoteGlobalMetadata; +import org.opensearch.gateway.remote.model.RemoteIndexMetadata; +import org.opensearch.gateway.remote.model.RemotePersistentSettingsMetadata; +import org.opensearch.gateway.remote.model.RemoteTemplatesMetadata; +import org.opensearch.gateway.remote.model.RemoteUploadDetails; import org.opensearch.index.remote.RemoteIndexPathUploader; -import org.opensearch.index.remote.RemoteStoreUtils; import org.opensearch.indices.IndicesModule; import org.opensearch.repositories.FilterRepository; import org.opensearch.repositories.RepositoriesService; import org.opensearch.repositories.RepositoryMissingException; import org.opensearch.repositories.blobstore.BlobStoreRepository; +import org.opensearch.repositories.blobstore.ChecksumBlobStoreFormat; import org.opensearch.repositories.fs.FsRepository; import org.opensearch.test.OpenSearchTestCase; import org.opensearch.test.TestCustomMetadata; @@ -73,9 +84,6 @@ import java.util.Optional; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicInteger; import java.util.function.BiConsumer; import java.util.function.Function; import java.util.function.Supplier; @@ -88,16 +96,14 @@ import static java.util.stream.Collectors.toList; import static org.opensearch.common.util.FeatureFlags.REMOTE_PUBLICATION_EXPERIMENTAL; -import static org.opensearch.gateway.remote.RemoteClusterStateService.COORDINATION_METADATA; -import static org.opensearch.gateway.remote.RemoteClusterStateService.DELIMITER; -import static org.opensearch.gateway.remote.RemoteClusterStateService.FORMAT_PARAMS; -import static org.opensearch.gateway.remote.RemoteClusterStateService.INDEX_METADATA_CURRENT_CODEC_VERSION; -import static org.opensearch.gateway.remote.RemoteClusterStateService.MANIFEST_CURRENT_CODEC_VERSION; -import static org.opensearch.gateway.remote.RemoteClusterStateService.MANIFEST_FILE_PREFIX; -import static org.opensearch.gateway.remote.RemoteClusterStateService.METADATA_FILE_PREFIX; -import static org.opensearch.gateway.remote.RemoteClusterStateService.RETAINED_MANIFESTS; -import static org.opensearch.gateway.remote.RemoteClusterStateService.SETTING_METADATA; -import static org.opensearch.gateway.remote.RemoteClusterStateService.TEMPLATES_METADATA; +import static org.opensearch.gateway.remote.RemoteClusterStateUtils.DELIMITER; +import static org.opensearch.gateway.remote.RemoteClusterStateUtils.FORMAT_PARAMS; +import static org.opensearch.gateway.remote.RemoteClusterStateUtils.METADATA_NAME_FORMAT; +import static org.opensearch.gateway.remote.RemoteClusterStateUtils.RemoteStateTransferException; +import static org.opensearch.gateway.remote.model.RemoteCoordinationMetadata.COORDINATION_METADATA; +import static org.opensearch.gateway.remote.model.RemotePersistentSettingsMetadata.SETTING_METADATA; +import static org.opensearch.gateway.remote.model.RemoteTemplatesMetadata.TEMPLATES_METADATA; +import static org.opensearch.gateway.remote.model.RemoteClusterMetadataManifest.MANIFEST_CURRENT_CODEC_VERSION; import static org.opensearch.node.remotestore.RemoteStoreNodeAttribute.REMOTE_STORE_CLUSTER_STATE_REPOSITORY_NAME_ATTRIBUTE_KEY; import static org.opensearch.node.remotestore.RemoteStoreNodeAttribute.REMOTE_STORE_REPOSITORY_SETTINGS_ATTRIBUTE_KEY_PREFIX; import static org.opensearch.node.remotestore.RemoteStoreNodeAttribute.REMOTE_STORE_REPOSITORY_TYPE_ATTRIBUTE_KEY_FORMAT; @@ -112,18 +118,18 @@ import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.times; -import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; public class RemoteClusterStateServiceTests extends OpenSearchTestCase { private RemoteClusterStateService remoteClusterStateService; + private ClusterService clusterService; private ClusterSettings clusterSettings; private Supplier repositoriesServiceSupplier; private RepositoriesService repositoriesService; private BlobStoreRepository blobStoreRepository; private BlobStore blobStore; + private Settings settings; private final ThreadPool threadPool = new TestThreadPool(getClass().getName()); @Before @@ -143,7 +149,7 @@ public void setup() { "remote_store_repository" ); - Settings settings = Settings.builder() + settings = Settings.builder() .put("node.attr." + REMOTE_STORE_CLUSTER_STATE_REPOSITORY_NAME_ATTRIBUTE_KEY, "remote_store_repository") .put(stateRepoTypeAttributeKey, FsRepository.TYPE) .put(stateRepoSettingsAttributeKeyPrefix + "location", "randomRepoPath") @@ -151,6 +157,8 @@ public void setup() { .build(); clusterSettings = new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); + clusterService = mock(ClusterService.class); + when(clusterService.getClusterSettings()).thenReturn(clusterSettings); NamedXContentRegistry xContentRegistry = new NamedXContentRegistry( Stream.of( NetworkModule.getNamedXContents().stream(), @@ -168,10 +176,11 @@ public void setup() { "test-node-id", repositoriesServiceSupplier, settings, - clusterSettings, + clusterService, () -> 0L, threadPool, - List.of(new RemoteIndexPathUploader(threadPool, settings, repositoriesServiceSupplier, clusterSettings)) + List.of(new RemoteIndexPathUploader(threadPool, settings, repositoriesServiceSupplier, clusterSettings)), + null ); } @@ -184,23 +193,25 @@ public void teardown() throws Exception { public void testFailWriteFullMetadataNonClusterManagerNode() throws IOException { final ClusterState clusterState = generateClusterStateWithOneIndex().build(); - final ClusterMetadataManifest manifest = remoteClusterStateService.writeFullMetadata(clusterState, randomAlphaOfLength(10)); + final ClusterMetadataManifest manifest = remoteClusterStateService.writeFullMetadata(clusterState, randomAlphaOfLength(10)) + .getClusterMetadataManifest(); Assert.assertThat(manifest, nullValue()); } public void testFailInitializationWhenRemoteStateDisabled() { final Settings settings = Settings.builder().build(); - ClusterSettings clusterSettings = new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); + when(clusterService.getClusterSettings()).thenReturn(new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS)); assertThrows( AssertionError.class, () -> new RemoteClusterStateService( "test-node-id", repositoriesServiceSupplier, settings, - clusterSettings, + clusterService, () -> 0L, threadPool, - List.of(new RemoteIndexPathUploader(threadPool, settings, repositoriesServiceSupplier, clusterSettings)) + List.of(new RemoteIndexPathUploader(threadPool, settings, repositoriesServiceSupplier, clusterSettings)), + null ) ); } @@ -220,7 +231,8 @@ public void testWriteFullMetadataSuccess() throws IOException { final ClusterState clusterState = generateClusterStateWithOneIndex().nodes(nodesWithLocalNodeClusterManager()).build(); mockBlobStoreObjects(); remoteClusterStateService.start(); - final ClusterMetadataManifest manifest = remoteClusterStateService.writeFullMetadata(clusterState, "prev-cluster-uuid"); + final ClusterMetadataManifest manifest = remoteClusterStateService.writeFullMetadata(clusterState, "prev-cluster-uuid") + .getClusterMetadataManifest(); final UploadedIndexMetadata uploadedIndexMetadata = new UploadedIndexMetadata("test-index", "index-uuid", "metadata-filename"); List indices = List.of(uploadedIndexMetadata); @@ -264,7 +276,8 @@ public void testWriteFullMetadataInParallelSuccess() throws IOException { }).when(container).asyncBlobUpload(writeContextArgumentCaptor.capture(), actionListenerArgumentCaptor.capture()); remoteClusterStateService.start(); - final ClusterMetadataManifest manifest = remoteClusterStateService.writeFullMetadata(clusterState, "prev-cluster-uuid"); + final ClusterMetadataManifest manifest = remoteClusterStateService.writeFullMetadata(clusterState, "prev-cluster-uuid") + .getClusterMetadataManifest(); final UploadedIndexMetadata uploadedIndexMetadata = new UploadedIndexMetadata("test-index", "index-uuid", "metadata-filename"); List indices = List.of(uploadedIndexMetadata); @@ -301,7 +314,7 @@ public void testWriteFullMetadataInParallelSuccess() throws IOException { .provideStream(0) .getInputStream() .readAllBytes(); - IndexMetadata writtenIndexMetadata = RemoteClusterStateService.INDEX_METADATA_FORMAT.deserialize( + IndexMetadata writtenIndexMetadata = RemoteIndexMetadata.INDEX_METADATA_FORMAT.deserialize( capturedWriteContext.get("metadata").getFileName(), blobStoreRepository.getNamedXContentRegistry(), new BytesArray(writtenBytes) @@ -340,7 +353,7 @@ public void run() { remoteClusterStateService.start(); assertThrows( - RemoteClusterStateService.RemoteStateTransferException.class, + RemoteStateTransferException.class, () -> remoteClusterStateService.writeFullMetadata(clusterState, randomAlphaOfLength(10)) ); } @@ -373,7 +386,7 @@ public void testTimeoutWhileWritingManifestFile() throws IOException { try { remoteClusterStateService.writeFullMetadata(clusterState, randomAlphaOfLength(10)); } catch (Exception e) { - assertTrue(e instanceof RemoteClusterStateService.RemoteStateTransferException); + assertTrue(e instanceof RemoteStateTransferException); assertTrue(e.getMessage().contains("Timed out waiting for transfer of following metadata to complete")); } } @@ -394,7 +407,7 @@ public void testWriteFullMetadataInParallelFailureForIndexMetadata() throws IOEx remoteClusterStateService.start(); assertThrows( - RemoteClusterStateService.RemoteStateTransferException.class, + RemoteStateTransferException.class, () -> remoteClusterStateService.writeFullMetadata(clusterState, randomAlphaOfLength(10)) ); assertEquals(0, remoteClusterStateService.getStats().getSuccessCount()); @@ -403,7 +416,8 @@ public void testWriteFullMetadataInParallelFailureForIndexMetadata() throws IOEx public void testFailWriteIncrementalMetadataNonClusterManagerNode() throws IOException { final ClusterState clusterState = generateClusterStateWithOneIndex().build(); remoteClusterStateService.start(); - final ClusterMetadataManifest manifest = remoteClusterStateService.writeIncrementalMetadata(clusterState, clusterState, null); + final ClusterMetadataManifest manifest = remoteClusterStateService.writeIncrementalMetadata(clusterState, clusterState, null) + .getClusterMetadataManifest(); Assert.assertThat(manifest, nullValue()); assertEquals(0, remoteClusterStateService.getStats().getSuccessCount()); } @@ -435,7 +449,7 @@ public void testWriteIncrementalMetadataSuccess() throws IOException { previousClusterState, clusterState, previousManifest - ); + ).getClusterMetadataManifest(); final UploadedIndexMetadata uploadedIndexMetadata = new UploadedIndexMetadata("test-index", "index-uuid", "metadata-filename"); final List indices = List.of(uploadedIndexMetadata); @@ -510,7 +524,7 @@ private void verifyCodecMigrationManifest(int previousCodec) throws IOException previousClusterState, newClusterState, previousManifest - ); + ).getClusterMetadataManifest(); // global metadata is updated assertThat(manifestAfterUpdate.hasMetadataAttributesFiles(), is(true)); @@ -547,7 +561,7 @@ private void verifyWriteIncrementalGlobalMetadataFromOlderCodecSuccess(ClusterMe previousClusterState, clusterState, previousManifest - ); + ).getClusterMetadataManifest(); final ClusterMetadataManifest expectedManifest = ClusterMetadataManifest.builder() .codecVersion(2) @@ -738,7 +752,8 @@ public void testCustomMetadataDeletedUpdatedAndAdded() throws IOException { // Initial cluster state with index. final ClusterState initialClusterState = generateClusterStateWithOneIndex().nodes(nodesWithLocalNodeClusterManager()).build(); remoteClusterStateService.start(); - final ClusterMetadataManifest initialManifest = remoteClusterStateService.writeFullMetadata(initialClusterState, "_na_"); + final ClusterMetadataManifest initialManifest = remoteClusterStateService.writeFullMetadata(initialClusterState, "_na_") + .getClusterMetadataManifest(); ClusterState clusterState1 = ClusterState.builder(initialClusterState) .metadata( @@ -753,7 +768,7 @@ public void testCustomMetadataDeletedUpdatedAndAdded() throws IOException { initialClusterState, clusterState1, initialManifest - ); + ).getClusterMetadataManifest(); // remove custom1 from the cluster state, update custom2, custom3 is at it is, added custom4 ClusterState clusterState2 = ClusterState.builder(initialClusterState) .metadata( @@ -763,7 +778,8 @@ public void testCustomMetadataDeletedUpdatedAndAdded() throws IOException { .putCustom("custom4", new CustomMetadata1("mock_custom_metadata4")) ) .build(); - ClusterMetadataManifest manifest2 = remoteClusterStateService.writeIncrementalMetadata(clusterState1, clusterState2, manifest1); + ClusterMetadataManifest manifest2 = remoteClusterStateService.writeIncrementalMetadata(clusterState1, clusterState2, manifest1) + .getClusterMetadataManifest(); // custom1 is removed assertFalse(manifest2.getCustomMetadataMap().containsKey("custom1")); // custom2 is updated @@ -813,7 +829,8 @@ public void testIndexMetadataDeletedUpdatedAndAdded() throws IOException { // Initial cluster state with index. final ClusterState initialClusterState = generateClusterStateWithOneIndex().nodes(nodesWithLocalNodeClusterManager()).build(); remoteClusterStateService.start(); - final ClusterMetadataManifest initialManifest = remoteClusterStateService.writeFullMetadata(initialClusterState, "_na_"); + final ClusterMetadataManifest initialManifest = remoteClusterStateService.writeFullMetadata(initialClusterState, "_na_") + .getClusterMetadataManifest(); String initialIndex = "test-index"; Index index1 = new Index("test-index-1", "index-uuid-1"); Index index2 = new Index("test-index-2", "index-uuid-2"); @@ -846,7 +863,7 @@ public void testIndexMetadataDeletedUpdatedAndAdded() throws IOException { initialClusterState, clusterState1, initialManifest - ); + ).getClusterMetadataManifest(); // verify that initial index is removed, and new index are added assertEquals(1, initialManifest.getIndices().size()); assertEquals(2, manifest1.getIndices().size()); @@ -857,7 +874,8 @@ public void testIndexMetadataDeletedUpdatedAndAdded() throws IOException { ClusterState clusterState2 = ClusterState.builder(clusterState1) .metadata(Metadata.builder(clusterState1.getMetadata()).put(indexMetadata1, true).build()) .build(); - ClusterMetadataManifest manifest2 = remoteClusterStateService.writeIncrementalMetadata(clusterState1, clusterState2, manifest1); + ClusterMetadataManifest manifest2 = remoteClusterStateService.writeIncrementalMetadata(clusterState1, clusterState2, manifest1) + .getClusterMetadataManifest(); // index1 is updated assertEquals(2, manifest2.getIndices().size()); assertEquals( @@ -890,7 +908,8 @@ private void verifyMetadataAttributeOnlyUpdated( // Initial cluster state with index. final ClusterState initialClusterState = generateClusterStateWithOneIndex().nodes(nodesWithLocalNodeClusterManager()).build(); remoteClusterStateService.start(); - final ClusterMetadataManifest initialManifest = remoteClusterStateService.writeFullMetadata(initialClusterState, "_na_"); + final ClusterMetadataManifest initialManifest = remoteClusterStateService.writeFullMetadata(initialClusterState, "_na_") + .getClusterMetadataManifest(); ClusterState newClusterState = clusterStateUpdater.apply(initialClusterState); @@ -901,32 +920,15 @@ private void verifyMetadataAttributeOnlyUpdated( initialClusterState, newClusterState, initialManifest - ); + ).getClusterMetadataManifest(); } else { - manifestAfterMetadataUpdate = remoteClusterStateService.writeFullMetadata(newClusterState, initialClusterState.stateUUID()); + manifestAfterMetadataUpdate = remoteClusterStateService.writeFullMetadata(newClusterState, initialClusterState.stateUUID()) + .getClusterMetadataManifest(); } assertions.accept(initialManifest, manifestAfterMetadataUpdate); } - public void testReadLatestMetadataManifestFailedIOException() throws IOException { - final ClusterState clusterState = generateClusterStateWithOneIndex().nodes(nodesWithLocalNodeClusterManager()).build(); - - BlobContainer blobContainer = mockBlobStoreObjects(); - when(blobContainer.listBlobsByPrefixInSortedOrder("manifest" + DELIMITER, 1, BlobContainer.BlobNameSortOrder.LEXICOGRAPHIC)) - .thenThrow(IOException.class); - - remoteClusterStateService.start(); - Exception e = assertThrows( - IllegalStateException.class, - () -> remoteClusterStateService.getLatestClusterMetadataManifest( - clusterState.getClusterName().value(), - clusterState.metadata().clusterUUID() - ) - ); - assertEquals(e.getMessage(), "Error while fetching latest manifest file for remote cluster state"); - } - public void testReadLatestMetadataManifestFailedNoManifestFileInRemote() throws IOException { final ClusterState clusterState = generateClusterStateWithOneIndex().nodes(nodesWithLocalNodeClusterManager()).build(); @@ -981,7 +983,7 @@ public void testReadLatestMetadataManifestSuccessButNoIndexMetadata() throws IOE remoteClusterStateService.start(); assertEquals( - remoteClusterStateService.getLatestClusterState(clusterState.getClusterName().value(), clusterState.metadata().clusterUUID()) + remoteClusterStateService.getLatestClusterState(clusterState.getClusterName().value(), clusterState.metadata().clusterUUID(), false) .getMetadata() .getIndices() .size(), @@ -1013,7 +1015,8 @@ public void testReadLatestMetadataManifestSuccessButIndexMetadataFetchIOExceptio IllegalStateException.class, () -> remoteClusterStateService.getLatestClusterState( clusterState.getClusterName().value(), - clusterState.metadata().clusterUUID() + clusterState.metadata().clusterUUID(), + false ).getMetadata().getIndices() ); assertEquals(e.getMessage(), "Error while downloading IndexMetadata - " + uploadedIndexMetadata.getUploadedFilename()); @@ -1067,10 +1070,10 @@ public void testReadGlobalMetadata() throws IOException { .stateUUID("state-uuid") .clusterUUID("cluster-uuid") .codecVersion(MANIFEST_CURRENT_CODEC_VERSION) - .coordinationMetadata(new UploadedMetadataAttribute(COORDINATION_METADATA, "mock-coordination-file")) - .settingMetadata(new UploadedMetadataAttribute(SETTING_METADATA, "mock-setting-file")) - .templatesMetadata(new UploadedMetadataAttribute(TEMPLATES_METADATA, "mock-templates-file")) - .put(IndexGraveyard.TYPE, new UploadedMetadataAttribute(IndexGraveyard.TYPE, "mock-custom-" +IndexGraveyard.TYPE+ "-file")) + .coordinationMetadata(new ClusterMetadataManifest.UploadedMetadataAttribute(COORDINATION_METADATA, "mock-coordination-file")) + .settingMetadata(new ClusterMetadataManifest.UploadedMetadataAttribute(SETTING_METADATA, "mock-setting-file")) + .templatesMetadata(new ClusterMetadataManifest.UploadedMetadataAttribute(TEMPLATES_METADATA, "mock-templates-file")) + .put(IndexGraveyard.TYPE, new ClusterMetadataManifest.UploadedMetadataAttribute(IndexGraveyard.TYPE, "mock-custom-" +IndexGraveyard.TYPE+ "-file")) .nodeId("nodeA") .opensearchVersion(VersionUtils.randomOpenSearchVersion(random())) .previousClusterUUID("prev-cluster-uuid") @@ -1081,7 +1084,8 @@ public void testReadGlobalMetadata() throws IOException { ClusterState newClusterState = remoteClusterStateService.getLatestClusterState( clusterState.getClusterName().value(), - clusterState.metadata().clusterUUID() + clusterState.metadata().clusterUUID(), + false ); assertTrue(Metadata.isGlobalStateEquals(newClusterState.getMetadata(), expactedMetadata)); @@ -1117,7 +1121,7 @@ public void testReadGlobalMetadataIOException() throws IOException { BlobContainer blobContainer = mockBlobStoreObjects(); mockBlobContainerForGlobalMetadata(blobContainer, expectedManifest, expactedMetadata); - when(blobContainer.readBlob(RemoteClusterStateService.GLOBAL_METADATA_FORMAT.blobName(globalIndexMetadataName))).thenThrow( + when(blobContainer.readBlob(RemoteGlobalMetadata.GLOBAL_METADATA_FORMAT.blobName(globalIndexMetadataName))).thenThrow( FileNotFoundException.class ); @@ -1126,7 +1130,8 @@ public void testReadGlobalMetadataIOException() throws IOException { IllegalStateException.class, () -> remoteClusterStateService.getLatestClusterState( clusterState.getClusterName().value(), - clusterState.metadata().clusterUUID() + clusterState.metadata().clusterUUID(), + false ) ); assertEquals(e.getMessage(), "Error while downloading Global Metadata - " + globalIndexMetadataName); @@ -1165,7 +1170,8 @@ public void testReadLatestIndexMetadataSuccess() throws IOException { Map indexMetadataMap = remoteClusterStateService.getLatestClusterState( clusterState.getClusterName().value(), - clusterState.metadata().clusterUUID() + clusterState.metadata().clusterUUID(), + false ).getMetadata().getIndices(); assertEquals(indexMetadataMap.size(), 1); @@ -1182,7 +1188,8 @@ public void testMarkLastStateAsCommittedSuccess() throws IOException { List indices = List.of(uploadedIndexMetadata); final ClusterMetadataManifest previousManifest = ClusterMetadataManifest.builder().indices(indices).build(); - final ClusterMetadataManifest manifest = remoteClusterStateService.markLastStateAsCommitted(clusterState, previousManifest); + final RemoteUploadDetails manifestDetails = remoteClusterStateService.markLastStateAsCommitted(clusterState, previousManifest); + ClusterMetadataManifest manifest = manifestDetails.getClusterMetadataManifest(); final ClusterMetadataManifest expectedManifest = ClusterMetadataManifest.builder() .indices(indices) @@ -1283,77 +1290,12 @@ public void testGetValidPreviousClusterUUIDWhenLastUUIDUncommitted() throws IOEx assertThat(previousClusterUUID, equalTo("cluster-uuid2")); } - public void testDeleteStaleClusterUUIDs() throws IOException { - final ClusterState clusterState = generateClusterStateWithOneIndex().nodes(nodesWithLocalNodeClusterManager()).build(); - ClusterMetadataManifest clusterMetadataManifest = ClusterMetadataManifest.builder() - .indices(List.of()) - .clusterTerm(1L) - .stateVersion(1L) - .stateUUID(randomAlphaOfLength(10)) - .clusterUUID("cluster-uuid1") - .nodeId("nodeA") - .opensearchVersion(VersionUtils.randomOpenSearchVersion(random())) - .previousClusterUUID(ClusterState.UNKNOWN_UUID) - .committed(true) - .build(); - - BlobPath blobPath = new BlobPath().add("random-path"); - when((blobStoreRepository.basePath())).thenReturn(blobPath); - BlobContainer uuidContainerContainer = mock(BlobContainer.class); - BlobContainer manifest2Container = mock(BlobContainer.class); - BlobContainer manifest3Container = mock(BlobContainer.class); - when(blobStore.blobContainer(any())).then(invocation -> { - BlobPath blobPath1 = invocation.getArgument(0); - if (blobPath1.buildAsString().endsWith("cluster-state/")) { - return uuidContainerContainer; - } else if (blobPath1.buildAsString().contains("cluster-state/cluster-uuid2/")) { - return manifest2Container; - } else if (blobPath1.buildAsString().contains("cluster-state/cluster-uuid3/")) { - return manifest3Container; - } else { - throw new IllegalArgumentException("Unexpected blob path " + blobPath1); - } - }); - Map blobMetadataMap = Map.of( - "cluster-uuid1", - mock(BlobContainer.class), - "cluster-uuid2", - mock(BlobContainer.class), - "cluster-uuid3", - mock(BlobContainer.class) - ); - when(uuidContainerContainer.children()).thenReturn(blobMetadataMap); - when( - manifest2Container.listBlobsByPrefixInSortedOrder( - MANIFEST_FILE_PREFIX + DELIMITER, - Integer.MAX_VALUE, - BlobContainer.BlobNameSortOrder.LEXICOGRAPHIC - ) - ).thenReturn(List.of(new PlainBlobMetadata("mainfest2", 1L))); - when( - manifest3Container.listBlobsByPrefixInSortedOrder( - MANIFEST_FILE_PREFIX + DELIMITER, - Integer.MAX_VALUE, - BlobContainer.BlobNameSortOrder.LEXICOGRAPHIC - ) - ).thenReturn(List.of(new PlainBlobMetadata("mainfest3", 1L))); - remoteClusterStateService.start(); - remoteClusterStateService.deleteStaleClusterUUIDs(clusterState, clusterMetadataManifest); - try { - assertBusy(() -> { - verify(manifest2Container, times(1)).delete(); - verify(manifest3Container, times(1)).delete(); - }); - } catch (Exception e) { - throw new RuntimeException(e); - } - } - public void testRemoteStateStats() throws IOException { final ClusterState clusterState = generateClusterStateWithOneIndex().nodes(nodesWithLocalNodeClusterManager()).build(); mockBlobStoreObjects(); remoteClusterStateService.start(); - final ClusterMetadataManifest manifest = remoteClusterStateService.writeFullMetadata(clusterState, "prev-cluster-uuid"); + final ClusterMetadataManifest manifest = remoteClusterStateService.writeFullMetadata(clusterState, "prev-cluster-uuid") + .getClusterMetadataManifest(); assertTrue(remoteClusterStateService.getStats() != null); assertEquals(1, remoteClusterStateService.getStats().getSuccessCount()); @@ -1361,150 +1303,13 @@ public void testRemoteStateStats() throws IOException { assertEquals(0, remoteClusterStateService.getStats().getFailedCount()); } - public void testRemoteStateCleanupFailureStats() throws IOException { - BlobContainer blobContainer = mock(BlobContainer.class); - doThrow(IOException.class).when(blobContainer).delete(); - when(blobStore.blobContainer(any())).thenReturn(blobContainer); - BlobPath blobPath = new BlobPath().add("random-path"); - when((blobStoreRepository.basePath())).thenReturn(blobPath); - remoteClusterStateService.start(); - remoteClusterStateService.deleteStaleUUIDsClusterMetadata("cluster1", Arrays.asList("cluster-uuid1")); - try { - assertBusy(() -> { - // wait for stats to get updated - assertTrue(remoteClusterStateService.getStats() != null); - assertEquals(0, remoteClusterStateService.getStats().getSuccessCount()); - assertEquals(1, remoteClusterStateService.getStats().getCleanupAttemptFailedCount()); - }); - } catch (Exception e) { - throw new RuntimeException(e); - } - } - - public void testFileNames() { - final Index index = new Index("test-index", "index-uuid"); - final Settings idxSettings = Settings.builder() - .put(IndexMetadata.SETTING_VERSION_CREATED, Version.CURRENT) - .put(IndexMetadata.SETTING_INDEX_UUID, index.getUUID()) - .build(); - final IndexMetadata indexMetadata = new IndexMetadata.Builder(index.getName()).settings(idxSettings) - .numberOfShards(1) - .numberOfReplicas(0) - .build(); - - String indexMetadataFileName = RemoteClusterStateService.indexMetadataFileName(indexMetadata); - String[] splittedIndexMetadataFileName = indexMetadataFileName.split(DELIMITER); - assertThat(indexMetadataFileName.split(DELIMITER).length, is(4)); - assertThat(splittedIndexMetadataFileName[0], is(METADATA_FILE_PREFIX)); - assertThat(splittedIndexMetadataFileName[1], is(RemoteStoreUtils.invertLong(indexMetadata.getVersion()))); - assertThat(splittedIndexMetadataFileName[3], is(String.valueOf(INDEX_METADATA_CURRENT_CODEC_VERSION))); - - verifyManifestFileNameWithCodec(MANIFEST_CURRENT_CODEC_VERSION); - verifyManifestFileNameWithCodec(ClusterMetadataManifest.CODEC_V1); - verifyManifestFileNameWithCodec(ClusterMetadataManifest.CODEC_V0); - } - - private void verifyManifestFileNameWithCodec(int codecVersion) { - int term = randomIntBetween(5, 10); - int version = randomIntBetween(5, 10); - String manifestFileName = RemoteClusterStateService.getManifestFileName(term, version, true, codecVersion); - assertThat(manifestFileName.split(DELIMITER).length, is(6)); - String[] splittedName = manifestFileName.split(DELIMITER); - assertThat(splittedName[0], is(MANIFEST_FILE_PREFIX)); - assertThat(splittedName[1], is(RemoteStoreUtils.invertLong(term))); - assertThat(splittedName[2], is(RemoteStoreUtils.invertLong(version))); - assertThat(splittedName[3], is("C")); - assertThat(splittedName[5], is(String.valueOf(codecVersion))); - - manifestFileName = RemoteClusterStateService.getManifestFileName(term, version, false, codecVersion); - splittedName = manifestFileName.split(DELIMITER); - assertThat(splittedName[3], is("P")); - } - - public void testSingleConcurrentExecutionOfStaleManifestCleanup() throws Exception { - BlobContainer blobContainer = mock(BlobContainer.class); - BlobPath blobPath = new BlobPath().add("random-path"); - when((blobStoreRepository.basePath())).thenReturn(blobPath); - when(blobStore.blobContainer(any())).thenReturn(blobContainer); - - CountDownLatch latch = new CountDownLatch(1); - AtomicInteger callCount = new AtomicInteger(0); - doAnswer(invocation -> { - callCount.incrementAndGet(); - if (latch.await(5000, TimeUnit.SECONDS) == false) { - throw new Exception("Timed out waiting for delete task queuing to complete"); - } - return null; - }).when(blobContainer) - .listBlobsByPrefixInSortedOrder( - any(String.class), - any(int.class), - any(BlobContainer.BlobNameSortOrder.class), - any(ActionListener.class) - ); - - remoteClusterStateService.start(); - remoteClusterStateService.deleteStaleClusterMetadata("cluster-name", "cluster-uuid", RETAINED_MANIFESTS); - remoteClusterStateService.deleteStaleClusterMetadata("cluster-name", "cluster-uuid", RETAINED_MANIFESTS); - - latch.countDown(); - assertBusy(() -> assertEquals(1, callCount.get())); - } - - public void testIndexMetadataUploadWaitTimeSetting() { - // verify default value - assertEquals( - RemoteClusterStateService.INDEX_METADATA_UPLOAD_TIMEOUT_DEFAULT, - remoteClusterStateService.getIndexMetadataUploadTimeout() - ); - - // verify update index metadata upload timeout - int indexMetadataUploadTimeout = randomIntBetween(1, 10); - Settings newSettings = Settings.builder() - .put("cluster.remote_store.state.index_metadata.upload_timeout", indexMetadataUploadTimeout + "s") - .build(); - clusterSettings.applySettings(newSettings); - assertEquals(indexMetadataUploadTimeout, remoteClusterStateService.getIndexMetadataUploadTimeout().seconds()); - } - - public void testMetadataManifestUploadWaitTimeSetting() { - // verify default value - assertEquals( - RemoteClusterStateService.METADATA_MANIFEST_UPLOAD_TIMEOUT_DEFAULT, - remoteClusterStateService.getMetadataManifestUploadTimeout() - ); - - // verify update metadata manifest upload timeout - int metadataManifestUploadTimeout = randomIntBetween(1, 10); - Settings newSettings = Settings.builder() - .put("cluster.remote_store.state.metadata_manifest.upload_timeout", metadataManifestUploadTimeout + "s") - .build(); - clusterSettings.applySettings(newSettings); - assertEquals(metadataManifestUploadTimeout, remoteClusterStateService.getMetadataManifestUploadTimeout().seconds()); - } - - public void testGlobalMetadataUploadWaitTimeSetting() { - // verify default value - assertEquals( - RemoteClusterStateService.GLOBAL_METADATA_UPLOAD_TIMEOUT_DEFAULT, - remoteClusterStateService.getGlobalMetadataUploadTimeout() - ); - - // verify update global metadata upload timeout - int globalMetadataUploadTimeout = randomIntBetween(1, 10); - Settings newSettings = Settings.builder() - .put("cluster.remote_store.state.global_metadata.upload_timeout", globalMetadataUploadTimeout + "s") - .build(); - clusterSettings.applySettings(newSettings); - assertEquals(globalMetadataUploadTimeout, remoteClusterStateService.getGlobalMetadataUploadTimeout().seconds()); - } - public void testRemoteRoutingTableNotInitializedWhenDisabled() { - assertFalse(remoteClusterStateService.getRemoteRoutingTableService().isPresent()); + assertNull(remoteClusterStateService.getRemoteRoutingTableService()); } public void testRemoteRoutingTableInitializedWhenEnabled() { Settings newSettings = Settings.builder() + .put(RemoteRoutingTableService.REMOTE_ROUTING_TABLE_ENABLED_SETTING.getKey(), true) .put("node.attr." + REMOTE_STORE_ROUTING_TABLE_REPOSITORY_NAME_ATTRIBUTE_KEY, "routing_repository") .put("node.attr." + REMOTE_STORE_CLUSTER_STATE_REPOSITORY_NAME_ATTRIBUTE_KEY, "remote_store_repository") .put(RemoteClusterStateService.REMOTE_CLUSTER_STATE_ENABLED_SETTING.getKey(), true) @@ -1518,12 +1323,13 @@ public void testRemoteRoutingTableInitializedWhenEnabled() { "test-node-id", repositoriesServiceSupplier, newSettings, - clusterSettings, + clusterService, () -> 0L, threadPool, - List.of(new RemoteIndexPathUploader(threadPool, newSettings, repositoriesServiceSupplier, clusterSettings)) + List.of(new RemoteIndexPathUploader(threadPool, settings, repositoriesServiceSupplier, clusterSettings)), + null ); - assertTrue(remoteClusterStateService.getRemoteRoutingTableService().isPresent()); + assertNotNull(remoteClusterStateService.getRemoteRoutingTableService()); } private void mockObjectsForGettingPreviousClusterUUID(Map clusterUUIDsPointers) throws IOException { @@ -1772,7 +1578,7 @@ private void mockBlobContainer( when(blobContainer.listBlobsByPrefixInSortedOrder("manifest" + DELIMITER, 1, BlobContainer.BlobNameSortOrder.LEXICOGRAPHIC)) .thenReturn(Arrays.asList(blobMetadata)); - BytesReference bytes = RemoteClusterStateService.CLUSTER_METADATA_MANIFEST_FORMAT.serialize( + BytesReference bytes = RemoteClusterMetadataManifest.CLUSTER_METADATA_MANIFEST_FORMAT.serialize( clusterMetadataManifest, manifestFileName, blobStoreRepository.getCompressor(), @@ -1788,7 +1594,7 @@ private void mockBlobContainer( } String fileName = uploadedIndexMetadata.getUploadedFilename(); when(blobContainer.readBlob(fileName + ".dat")).thenAnswer((invocationOnMock) -> { - BytesReference bytesIndexMetadata = RemoteClusterStateService.INDEX_METADATA_FORMAT.serialize( + BytesReference bytesIndexMetadata = RemoteIndexMetadata.INDEX_METADATA_FORMAT.serialize( indexMetadata, fileName, blobStoreRepository.getCompressor(), @@ -1810,15 +1616,10 @@ private void mockBlobContainerForGlobalMetadata( int codecVersion = clusterMetadataManifest.getCodecVersion(); String mockManifestFileName = "manifest__1__2__C__456__" + codecVersion; BlobMetadata blobMetadata = new PlainBlobMetadata(mockManifestFileName, 1); - when( - blobContainer.listBlobsByPrefixInSortedOrder( - "manifest" + RemoteClusterStateService.DELIMITER, - 1, - BlobContainer.BlobNameSortOrder.LEXICOGRAPHIC - ) - ).thenReturn(Arrays.asList(blobMetadata)); + when(blobContainer.listBlobsByPrefixInSortedOrder("manifest" + DELIMITER, 1, BlobContainer.BlobNameSortOrder.LEXICOGRAPHIC)) + .thenReturn(Arrays.asList(blobMetadata)); - BytesReference bytes = RemoteClusterStateService.CLUSTER_METADATA_MANIFEST_FORMAT.serialize( + BytesReference bytes = RemoteClusterMetadataManifest.CLUSTER_METADATA_MANIFEST_FORMAT.serialize( clusterMetadataManifest, mockManifestFileName, blobStoreRepository.getCompressor(), @@ -1827,22 +1628,21 @@ private void mockBlobContainerForGlobalMetadata( when(blobContainer.readBlob(mockManifestFileName)).thenReturn(new ByteArrayInputStream(bytes.streamInput().readAllBytes())); if (codecVersion >= ClusterMetadataManifest.CODEC_V2) { String coordinationFileName = getFileNameFromPath(clusterMetadataManifest.getCoordinationMetadata().getUploadedFilename()); - when(blobContainer.readBlob(RemoteClusterStateService.COORDINATION_METADATA_FORMAT.blobName(coordinationFileName))).thenAnswer( - (invocationOnMock) -> { - BytesReference bytesReference = RemoteClusterStateService.COORDINATION_METADATA_FORMAT.serialize( + when(blobContainer.readBlob(RemoteCoordinationMetadata.COORDINATION_METADATA_FORMAT.blobName(coordinationFileName))) + .thenAnswer((invocationOnMock) -> { + BytesReference bytesReference = RemoteCoordinationMetadata.COORDINATION_METADATA_FORMAT.serialize( metadata.coordinationMetadata(), coordinationFileName, blobStoreRepository.getCompressor(), FORMAT_PARAMS ); return new ByteArrayInputStream(bytesReference.streamInput().readAllBytes()); - } - ); + }); String settingsFileName = getFileNameFromPath(clusterMetadataManifest.getSettingsMetadata().getUploadedFilename()); - when(blobContainer.readBlob(RemoteClusterStateService.SETTINGS_METADATA_FORMAT.blobName(settingsFileName))).thenAnswer( + when(blobContainer.readBlob(RemotePersistentSettingsMetadata.SETTINGS_METADATA_FORMAT.blobName(settingsFileName))).thenAnswer( (invocationOnMock) -> { - BytesReference bytesReference = RemoteClusterStateService.SETTINGS_METADATA_FORMAT.serialize( + BytesReference bytesReference = RemotePersistentSettingsMetadata.SETTINGS_METADATA_FORMAT.serialize( metadata.persistentSettings(), settingsFileName, blobStoreRepository.getCompressor(), @@ -1853,9 +1653,9 @@ private void mockBlobContainerForGlobalMetadata( ); String templatesFileName = getFileNameFromPath(clusterMetadataManifest.getTemplatesMetadata().getUploadedFilename()); - when(blobContainer.readBlob(RemoteClusterStateService.TEMPLATES_METADATA_FORMAT.blobName(templatesFileName))).thenAnswer( + when(blobContainer.readBlob(RemoteTemplatesMetadata.TEMPLATES_METADATA_FORMAT.blobName(templatesFileName))).thenAnswer( (invocationOnMock) -> { - BytesReference bytesReference = RemoteClusterStateService.TEMPLATES_METADATA_FORMAT.serialize( + BytesReference bytesReference = RemoteTemplatesMetadata.TEMPLATES_METADATA_FORMAT.serialize( metadata.templatesMetadata(), templatesFileName, blobStoreRepository.getCompressor(), @@ -1870,26 +1670,27 @@ private void mockBlobContainerForGlobalMetadata( .stream() .collect(Collectors.toMap(Map.Entry::getKey, entry -> getFileNameFromPath(entry.getValue().getUploadedFilename()))); - for (Map.Entry entry : customFileMap.entrySet()) { - String custom = entry.getKey(); - String fileName = entry.getValue(); - when(blobContainer.readBlob(RemoteClusterStateService.CUSTOM_METADATA_FORMAT.blobName(fileName))).thenAnswer( - (invocation) -> { - BytesReference bytesReference = RemoteClusterStateService.CUSTOM_METADATA_FORMAT.serialize( - metadata.custom(custom), - fileName, - blobStoreRepository.getCompressor(), - FORMAT_PARAMS - ); - return new ByteArrayInputStream(bytesReference.streamInput().readAllBytes()); - } - ); - } + // TODO uncomment +// for (Map.Entry entry : customFileMap.entrySet()) { +// String custom = entry.getKey(); +// String fileName = entry.getValue(); +// when(blobContainer.readBlob(RemoteCustomMetadata.CUSTOM_METADATA_FORMAT.blobName(fileName))).thenAnswer( +// (invocation) -> { +// BytesReference bytesReference = RemoteCustomMetadata.CUSTOM_METADATA_FORMAT.serialize( +// metadata.custom(custom), +// fileName, +// blobStoreRepository.getCompressor(), +// FORMAT_PARAMS +// ); +// return new ByteArrayInputStream(bytesReference.streamInput().readAllBytes()); +// } +// ); +// } } else if (codecVersion == ClusterMetadataManifest.CODEC_V1) { String[] splitPath = clusterMetadataManifest.getGlobalMetadataFileName().split("/"); - when(blobContainer.readBlob(RemoteClusterStateService.GLOBAL_METADATA_FORMAT.blobName(splitPath[splitPath.length - 1]))) + when(blobContainer.readBlob(RemoteGlobalMetadata.GLOBAL_METADATA_FORMAT.blobName(splitPath[splitPath.length - 1]))) .thenAnswer((invocationOnMock) -> { - BytesReference bytesGlobalMetadata = RemoteClusterStateService.GLOBAL_METADATA_FORMAT.serialize( + BytesReference bytesGlobalMetadata = RemoteGlobalMetadata.GLOBAL_METADATA_FORMAT.serialize( metadata, "global-metadata-file", blobStoreRepository.getCompressor(), @@ -1905,7 +1706,11 @@ private String getFileNameFromPath(String filePath) { return splitPath[splitPath.length - 1]; } - private static ClusterState.Builder generateClusterStateWithGlobalMetadata() { + private static RemoteUploadDetails getRemoteUploadDetails(ClusterMetadataManifest manifest) { + return new RemoteUploadDetails(manifest, "path/to/manifest"); + } + + static ClusterState.Builder generateClusterStateWithGlobalMetadata() { final Settings clusterSettings = Settings.builder().put("cluster.blocks.read_only", true).build(); final CoordinationMetadata coordinationMetadata = CoordinationMetadata.builder().term(1L).build(); @@ -1921,7 +1726,7 @@ private static ClusterState.Builder generateClusterStateWithGlobalMetadata() { ); } - private static ClusterState.Builder generateClusterStateWithOneIndex() { + static ClusterState.Builder generateClusterStateWithOneIndex() { final Index index = new Index("test-index", "index-uuid"); final Settings idxSettings = Settings.builder() .put(IndexMetadata.SETTING_VERSION_CREATED, Version.CURRENT) @@ -1951,7 +1756,7 @@ private static ClusterState.Builder generateClusterStateWithOneIndex() { ); } - private static DiscoveryNodes nodesWithLocalNodeClusterManager() { + static DiscoveryNodes nodesWithLocalNodeClusterManager() { return DiscoveryNodes.builder().clusterManagerNodeId("cluster-manager-id").localNodeId("cluster-manager-id").build(); } diff --git a/server/src/test/java/org/opensearch/gateway/remote/RemoteGlobalMetadataManagerTests.java b/server/src/test/java/org/opensearch/gateway/remote/RemoteGlobalMetadataManagerTests.java new file mode 100644 index 0000000000000..9ab8b7f340458 --- /dev/null +++ b/server/src/test/java/org/opensearch/gateway/remote/RemoteGlobalMetadataManagerTests.java @@ -0,0 +1,107 @@ +/* + * 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.gateway.remote; + +import static java.util.stream.Collectors.toList; +import static org.mockito.Mockito.mock; + +import java.util.function.Function; +import java.util.stream.Stream; +import org.junit.After; +import org.junit.Before; +import org.opensearch.cluster.ClusterModule; +import org.opensearch.cluster.coordination.CoordinationMetadata; +import org.opensearch.cluster.metadata.DiffableStringMap; +import org.opensearch.cluster.metadata.Metadata; +import org.opensearch.cluster.metadata.Metadata.Custom; +import org.opensearch.cluster.metadata.TemplatesMetadata; +import org.opensearch.common.network.NetworkModule; +import org.opensearch.common.settings.ClusterSettings; +import org.opensearch.common.settings.Settings; +import org.opensearch.core.compress.NoneCompressor; +import org.opensearch.core.xcontent.NamedXContentRegistry; +import org.opensearch.gateway.remote.model.RemoteCoordinationMetadata; +import org.opensearch.gateway.remote.model.RemoteCustomMetadata; +import org.opensearch.gateway.remote.model.RemoteGlobalMetadata; +import org.opensearch.gateway.remote.model.RemoteHashesOfConsistentSettings; +import org.opensearch.gateway.remote.model.RemotePersistentSettingsMetadata; +import org.opensearch.gateway.remote.model.RemoteTemplatesMetadata; +import org.opensearch.gateway.remote.model.RemoteTransientSettingsMetadata; +import org.opensearch.gateway.remote.model.RemoteClusterStateBlobStore; +import org.opensearch.index.translog.transfer.BlobStoreTransferService; +import org.opensearch.indices.IndicesModule; +import org.opensearch.repositories.blobstore.BlobStoreRepository; +import org.opensearch.test.OpenSearchTestCase; +import org.opensearch.threadpool.TestThreadPool; +import org.opensearch.threadpool.ThreadPool; + +public class RemoteGlobalMetadataManagerTests extends OpenSearchTestCase { + + private static final String CLUSTER_NAME = "test-cluster"; + private RemoteGlobalMetadataManager remoteGlobalMetadataManager; + private ClusterSettings clusterSettings; + private BlobStoreRepository blobStoreRepository; + private final ThreadPool threadPool = new TestThreadPool(getClass().getName()); + + @Before + public void setup() { + clusterSettings = new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); + blobStoreRepository = mock(BlobStoreRepository.class); + BlobStoreTransferService blobStoreTransferService = mock(BlobStoreTransferService.class); + RemoteClusterStateBlobStore globalMetadataBlobStore = new RemoteClusterStateBlobStore<>( + blobStoreTransferService, blobStoreRepository, CLUSTER_NAME, threadPool, ThreadPool.Names.GENERIC); + RemoteClusterStateBlobStore coordinationMetadataBlobStore = new RemoteClusterStateBlobStore<>( + blobStoreTransferService, blobStoreRepository, CLUSTER_NAME, threadPool, ThreadPool.Names.GENERIC); + RemoteClusterStateBlobStore transientSettingsBlobStore = new RemoteClusterStateBlobStore<>( + blobStoreTransferService, blobStoreRepository, CLUSTER_NAME, threadPool, ThreadPool.Names.GENERIC); + RemoteClusterStateBlobStore persistentSettingsBlobStore = new RemoteClusterStateBlobStore<>( + blobStoreTransferService, blobStoreRepository, CLUSTER_NAME, threadPool, ThreadPool.Names.GENERIC); + RemoteClusterStateBlobStore templatesMetadataBlobStore = new RemoteClusterStateBlobStore<>( + blobStoreTransferService, blobStoreRepository, CLUSTER_NAME, threadPool, ThreadPool.Names.GENERIC); + RemoteClusterStateBlobStore customMetadataBlobStore = new RemoteClusterStateBlobStore<>(blobStoreTransferService, + blobStoreRepository, CLUSTER_NAME, threadPool, ThreadPool.Names.GENERIC); + RemoteClusterStateBlobStore hashesOfConsistentSettingsBlobStore = + new RemoteClusterStateBlobStore<>( + blobStoreTransferService, + blobStoreRepository, CLUSTER_NAME, threadPool, ThreadPool.Names.GENERIC); + NamedXContentRegistry xContentRegistry = new NamedXContentRegistry( + Stream.of( + NetworkModule.getNamedXContents().stream(), + IndicesModule.getNamedXContents().stream(), + ClusterModule.getNamedXWriteables().stream() + ).flatMap(Function.identity()).collect(toList()) + ); + remoteGlobalMetadataManager = new RemoteGlobalMetadataManager(clusterSettings, globalMetadataBlobStore, coordinationMetadataBlobStore, + transientSettingsBlobStore, + persistentSettingsBlobStore, templatesMetadataBlobStore, customMetadataBlobStore, + hashesOfConsistentSettingsBlobStore, + new NoneCompressor(), xContentRegistry); + } + + @After + public void tearDown() throws Exception { + super.tearDown(); + } + + public void testGlobalMetadataUploadWaitTimeSetting() { + // verify default value + assertEquals( + RemoteGlobalMetadataManager.GLOBAL_METADATA_UPLOAD_TIMEOUT_DEFAULT, + remoteGlobalMetadataManager.getGlobalMetadataUploadTimeout() + ); + + // verify update global metadata upload timeout + int globalMetadataUploadTimeout = randomIntBetween(1, 10); + Settings newSettings = Settings.builder() + .put("cluster.remote_store.state.global_metadata.upload_timeout", globalMetadataUploadTimeout + "s") + .build(); + clusterSettings.applySettings(newSettings); + assertEquals(globalMetadataUploadTimeout, remoteGlobalMetadataManager.getGlobalMetadataUploadTimeout().seconds()); + } +} diff --git a/server/src/test/java/org/opensearch/gateway/remote/RemoteIndexMetadataManagerTests.java b/server/src/test/java/org/opensearch/gateway/remote/RemoteIndexMetadataManagerTests.java new file mode 100644 index 0000000000000..e3ba051f3b1da --- /dev/null +++ b/server/src/test/java/org/opensearch/gateway/remote/RemoteIndexMetadataManagerTests.java @@ -0,0 +1,76 @@ +/* + * 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.gateway.remote; + +import static java.util.stream.Collectors.toList; +import static org.mockito.Mockito.mock; + +import java.util.function.Function; +import java.util.stream.Stream; +import org.junit.After; +import org.junit.Before; +import org.opensearch.cluster.ClusterModule; +import org.opensearch.cluster.metadata.IndexMetadata; +import org.opensearch.common.network.NetworkModule; +import org.opensearch.common.settings.ClusterSettings; +import org.opensearch.common.settings.Settings; +import org.opensearch.core.compress.NoneCompressor; +import org.opensearch.core.xcontent.NamedXContentRegistry; +import org.opensearch.gateway.remote.model.RemoteIndexMetadata; +import org.opensearch.gateway.remote.model.RemoteClusterStateBlobStore; +import org.opensearch.index.translog.transfer.BlobStoreTransferService; +import org.opensearch.indices.IndicesModule; +import org.opensearch.repositories.blobstore.BlobStoreRepository; +import org.opensearch.test.OpenSearchTestCase; +import org.opensearch.threadpool.TestThreadPool; +import org.opensearch.threadpool.ThreadPool; + +public class RemoteIndexMetadataManagerTests extends OpenSearchTestCase { + private RemoteIndexMetadataManager remoteIndexMetadataManager; + private BlobStoreRepository blobStoreRepository; + private ClusterSettings clusterSettings; + private BlobStoreTransferService blobStoreTransferService; + + @Before + public void setup() { + clusterSettings = new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); + blobStoreRepository = mock(BlobStoreRepository.class); + blobStoreTransferService = mock(BlobStoreTransferService.class); + RemoteClusterStateBlobStore blobStore = new RemoteClusterStateBlobStore<>(blobStoreTransferService, blobStoreRepository, "cluster-name", new TestThreadPool("test"), ThreadPool.Names.GENERIC); + NamedXContentRegistry xContentRegistry = new NamedXContentRegistry( + Stream.of( + NetworkModule.getNamedXContents().stream(), + IndicesModule.getNamedXContents().stream(), + ClusterModule.getNamedXWriteables().stream() + ).flatMap(Function.identity()).collect(toList()) + ); + remoteIndexMetadataManager = new RemoteIndexMetadataManager(blobStore, clusterSettings, new NoneCompressor(), xContentRegistry); + } + + @After + public void tearDown() throws Exception { + super.tearDown(); + } + + public void testIndexMetadataUploadWaitTimeSetting() { + // verify default value + assertEquals( + RemoteIndexMetadataManager.INDEX_METADATA_UPLOAD_TIMEOUT_DEFAULT, + remoteIndexMetadataManager.getIndexMetadataUploadTimeout() + ); + + // verify update index metadata upload timeout + int indexMetadataUploadTimeout = randomIntBetween(1, 10); + Settings newSettings = Settings.builder() + .put("cluster.remote_store.state.index_metadata.upload_timeout", indexMetadataUploadTimeout + "s") + .build(); + clusterSettings.applySettings(newSettings); + assertEquals(indexMetadataUploadTimeout, remoteIndexMetadataManager.getIndexMetadataUploadTimeout().seconds()); + } +} diff --git a/server/src/test/java/org/opensearch/gateway/remote/RemoteManifestManagerTests.java b/server/src/test/java/org/opensearch/gateway/remote/RemoteManifestManagerTests.java new file mode 100644 index 0000000000000..79ec8c4b13ff7 --- /dev/null +++ b/server/src/test/java/org/opensearch/gateway/remote/RemoteManifestManagerTests.java @@ -0,0 +1,117 @@ +/* + * 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.gateway.remote; + +import static java.util.stream.Collectors.toList; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; +import static org.opensearch.gateway.remote.RemoteClusterStateServiceTests.generateClusterStateWithOneIndex; +import static org.opensearch.gateway.remote.RemoteClusterStateServiceTests.nodesWithLocalNodeClusterManager; +import static org.opensearch.gateway.remote.RemoteClusterStateUtils.DELIMITER; + +import java.io.IOException; +import java.util.function.Function; +import java.util.stream.Stream; +import org.junit.After; +import org.junit.Before; +import org.opensearch.cluster.ClusterModule; +import org.opensearch.cluster.ClusterState; +import org.opensearch.common.blobstore.BlobContainer; +import org.opensearch.common.blobstore.BlobPath; +import org.opensearch.common.blobstore.BlobStore; +import org.opensearch.common.network.NetworkModule; +import org.opensearch.common.settings.ClusterSettings; +import org.opensearch.common.settings.Settings; +import org.opensearch.core.compress.NoneCompressor; +import org.opensearch.core.xcontent.NamedXContentRegistry; +import org.opensearch.gateway.remote.model.RemoteClusterMetadataManifest; +import org.opensearch.gateway.remote.model.RemoteClusterStateBlobStore; +import org.opensearch.index.translog.transfer.BlobStoreTransferService; +import org.opensearch.indices.IndicesModule; +import org.opensearch.repositories.blobstore.BlobStoreRepository; +import org.opensearch.test.OpenSearchTestCase; +import org.opensearch.threadpool.TestThreadPool; +import org.opensearch.threadpool.ThreadPool; + +public class RemoteManifestManagerTests extends OpenSearchTestCase { + private RemoteManifestManager remoteManifestManager; + private ClusterSettings clusterSettings; + private BlobStoreRepository blobStoreRepository; + private BlobStore blobStore; + private BlobStoreTransferService blobStoreTransferService; + + @Before + public void setup() { + clusterSettings = new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); + blobStoreRepository = mock(BlobStoreRepository.class); + NamedXContentRegistry xContentRegistry = new NamedXContentRegistry( + Stream.of( + NetworkModule.getNamedXContents().stream(), + IndicesModule.getNamedXContents().stream(), + ClusterModule.getNamedXWriteables().stream() + ).flatMap(Function.identity()).collect(toList()) + ); + blobStoreTransferService = mock(BlobStoreTransferService.class); + RemoteClusterStateBlobStore manifestBlobStore = new RemoteClusterStateBlobStore<>(blobStoreTransferService, blobStoreRepository, "test-cluster-name", new TestThreadPool("test"), ThreadPool.Names.GENERIC); + remoteManifestManager = new RemoteManifestManager(manifestBlobStore, clusterSettings, "test-node-id", new NoneCompressor(), xContentRegistry, blobStoreRepository); + blobStore = mock(BlobStore.class); + when(blobStoreRepository.blobStore()).thenReturn(blobStore); + } + + @After + public void tearDown() throws Exception { + super.tearDown(); + } + + public void testMetadataManifestUploadWaitTimeSetting() { + // verify default value + assertEquals( + RemoteManifestManager.METADATA_MANIFEST_UPLOAD_TIMEOUT_DEFAULT, + remoteManifestManager.getMetadataManifestUploadTimeout() + ); + + // verify update metadata manifest upload timeout + int metadataManifestUploadTimeout = randomIntBetween(1, 10); + Settings newSettings = Settings.builder() + .put("cluster.remote_store.state.metadata_manifest.upload_timeout", metadataManifestUploadTimeout + "s") + .build(); + clusterSettings.applySettings(newSettings); + assertEquals(metadataManifestUploadTimeout, remoteManifestManager.getMetadataManifestUploadTimeout().seconds()); + } + + public void testReadLatestMetadataManifestFailedIOException() throws IOException { + final ClusterState clusterState = generateClusterStateWithOneIndex().nodes(nodesWithLocalNodeClusterManager()).build(); + + BlobContainer blobContainer = mockBlobStoreObjects(); + when(blobContainer.listBlobsByPrefixInSortedOrder("manifest" + DELIMITER, 1, BlobContainer.BlobNameSortOrder.LEXICOGRAPHIC)) + .thenThrow(IOException.class); + + Exception e = assertThrows( + IllegalStateException.class, + () -> remoteManifestManager.getLatestClusterMetadataManifest( + clusterState.getClusterName().value(), + clusterState.metadata().clusterUUID() + ) + ); + assertEquals(e.getMessage(), "Error while fetching latest manifest file for remote cluster state"); + } + + private BlobContainer mockBlobStoreObjects() { + final BlobPath blobPath = mock(BlobPath.class); + when((blobStoreRepository.basePath())).thenReturn(blobPath); + when(blobPath.add(anyString())).thenReturn(blobPath); + when(blobPath.buildAsString()).thenReturn("/blob/path/"); + final BlobContainer blobContainer = mock(BlobContainer.class); + when(blobContainer.path()).thenReturn(blobPath); + when(blobStore.blobContainer(any())).thenReturn(blobContainer); + return blobContainer; + } +} diff --git a/server/src/test/java/org/opensearch/gateway/remote/model/RemoteClusterBlocksTest.java b/server/src/test/java/org/opensearch/gateway/remote/model/RemoteClusterBlocksTest.java new file mode 100644 index 0000000000000..280a53e0dc2c4 --- /dev/null +++ b/server/src/test/java/org/opensearch/gateway/remote/model/RemoteClusterBlocksTest.java @@ -0,0 +1,170 @@ +/* + * 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.gateway.remote.model; + +import static java.util.stream.Collectors.toList; +import static org.hamcrest.Matchers.greaterThan; +import static org.hamcrest.Matchers.is; +import static org.hamcrest.Matchers.lessThanOrEqualTo; +import static org.hamcrest.Matchers.nullValue; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; +import static org.opensearch.gateway.remote.RemoteClusterStateAttributesManager.CLUSTER_STATE_ATTRIBUTES_CURRENT_CODEC_VERSION; +import static org.opensearch.gateway.remote.model.RemoteClusterBlocks.CLUSTER_BLOCKS; + +import java.io.IOException; +import java.io.InputStream; +import java.util.EnumSet; +import java.util.List; +import java.util.function.Function; +import java.util.stream.Stream; +import org.junit.After; +import org.junit.Before; +import org.opensearch.cluster.ClusterModule; +import org.opensearch.cluster.block.ClusterBlock; +import org.opensearch.cluster.block.ClusterBlockLevel; +import org.opensearch.cluster.block.ClusterBlocks; +import org.opensearch.common.blobstore.BlobPath; +import org.opensearch.common.compress.DeflateCompressor; +import org.opensearch.common.network.NetworkModule; +import org.opensearch.common.remote.BlobPathParameters; +import org.opensearch.common.settings.ClusterSettings; +import org.opensearch.common.settings.Settings; +import org.opensearch.core.compress.Compressor; +import org.opensearch.core.compress.NoneCompressor; +import org.opensearch.core.rest.RestStatus; +import org.opensearch.core.xcontent.NamedXContentRegistry; +import org.opensearch.gateway.remote.ClusterMetadataManifest.UploadedMetadata; +import org.opensearch.gateway.remote.RemoteClusterStateUtils; +import org.opensearch.index.remote.RemoteStoreUtils; +import org.opensearch.index.translog.transfer.BlobStoreTransferService; +import org.opensearch.indices.IndicesModule; +import org.opensearch.repositories.blobstore.BlobStoreRepository; +import org.opensearch.test.OpenSearchTestCase; +import org.opensearch.threadpool.TestThreadPool; +import org.opensearch.threadpool.ThreadPool; + +public class RemoteClusterBlocksTest extends OpenSearchTestCase { + + private static final String TEST_BLOB_NAME = "/test-path/test-blob-name"; + private static final String TEST_BLOB_PATH = "test-path"; + private static final String TEST_BLOB_FILE_NAME = "test-blob-name"; + private static final long VERSION = 5L; + private static final String PATH_PARAM_TRANSIENT = "transient"; + + private String clusterUUID; + private BlobStoreTransferService blobStoreTransferService; + private BlobStoreRepository blobStoreRepository; + private String clusterName; + private ClusterSettings clusterSettings; + private Compressor compressor; + private NamedXContentRegistry namedXContentRegistry; + private final ThreadPool threadPool = new TestThreadPool(getClass().getName()); + + @Before + public void setup() { + clusterSettings = new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); + this.clusterUUID = "test-cluster-uuid"; + this.blobStoreTransferService = mock(BlobStoreTransferService.class); + this.blobStoreRepository = mock(BlobStoreRepository.class); + BlobPath blobPath = new BlobPath().add("/path"); + when(blobStoreRepository.basePath()).thenReturn(blobPath); + when(blobStoreRepository.getCompressor()).thenReturn(new DeflateCompressor()); + this.clusterName = "test-cluster-name"; + compressor = new NoneCompressor(); + namedXContentRegistry = new NamedXContentRegistry( + Stream.of( + NetworkModule.getNamedXContents().stream(), + IndicesModule.getNamedXContents().stream(), + ClusterModule.getNamedXWriteables().stream() + ).flatMap(Function.identity()).collect(toList()) + ); + } + + @After + public void tearDown() throws Exception { + super.tearDown(); + threadPool.shutdown(); + } + + public void testClusterUUID() { + ClusterBlocks clusterBlocks = getClusterBlocks(); + RemoteClusterBlocks remoteObjectForUpload = new RemoteClusterBlocks(clusterBlocks, VERSION, clusterUUID, compressor, namedXContentRegistry); + assertThat(remoteObjectForUpload.clusterUUID(), is(clusterUUID)); + + RemoteClusterBlocks remoteObjectForDownload = new RemoteClusterBlocks(TEST_BLOB_NAME, clusterUUID, compressor, namedXContentRegistry); + assertThat(remoteObjectForDownload.clusterUUID(), is(clusterUUID)); + } + + public void testFullBlobName() { + ClusterBlocks clusterBlocks = getClusterBlocks(); + RemoteClusterBlocks remoteObjectForUpload = new RemoteClusterBlocks(clusterBlocks, VERSION, clusterUUID, compressor, namedXContentRegistry); + assertThat(remoteObjectForUpload.getFullBlobName(), nullValue()); + + RemoteClusterBlocks remoteObjectForDownload = new RemoteClusterBlocks(TEST_BLOB_NAME, clusterUUID, compressor, namedXContentRegistry); + assertThat(remoteObjectForDownload.getFullBlobName(), is(TEST_BLOB_NAME)); + } + + public void testBlobFileName() { + ClusterBlocks clusterBlocks = getClusterBlocks(); + RemoteClusterBlocks remoteObjectForUpload = new RemoteClusterBlocks(clusterBlocks, VERSION, clusterUUID, compressor, namedXContentRegistry); + assertThat(remoteObjectForUpload.getBlobFileName(), nullValue()); + + RemoteClusterBlocks remoteObjectForDownload = new RemoteClusterBlocks(TEST_BLOB_NAME, clusterUUID, compressor, namedXContentRegistry); + assertThat(remoteObjectForDownload.getBlobFileName(), is(TEST_BLOB_FILE_NAME)); + } + + public void testBlobPathParameters() { + ClusterBlocks clusterBlocks = getClusterBlocks(); + RemoteClusterBlocks remoteObjectForUpload = new RemoteClusterBlocks(clusterBlocks, VERSION, clusterUUID, compressor, namedXContentRegistry); + BlobPathParameters params = remoteObjectForUpload.getBlobPathParameters(); + assertThat(params.getPathTokens(), is(List.of(PATH_PARAM_TRANSIENT))); + assertThat(params.getFilePrefix(), is(CLUSTER_BLOCKS)); + } + + public void testGenerateBlobFileName() { + ClusterBlocks clusterBlocks = getClusterBlocks(); + RemoteClusterBlocks remoteObjectForUpload = new RemoteClusterBlocks(clusterBlocks, VERSION, clusterUUID, compressor, namedXContentRegistry); + String blobFileName = remoteObjectForUpload.generateBlobFileName(); + String[] nameTokens = blobFileName.split(RemoteClusterStateUtils.DELIMITER); + assertThat(nameTokens[0], is(CLUSTER_BLOCKS)); + assertThat(RemoteStoreUtils.invertLong(nameTokens[1]), is(VERSION)); + assertThat(RemoteStoreUtils.invertLong(nameTokens[2]), lessThanOrEqualTo(System.currentTimeMillis())); + assertThat(nameTokens[3], is(String.valueOf(CLUSTER_STATE_ATTRIBUTES_CURRENT_CODEC_VERSION))); + } + + public void testGetUploadedMetadata() throws IOException { + ClusterBlocks clusterBlocks = getClusterBlocks(); + RemoteClusterBlocks remoteObjectForUpload = new RemoteClusterBlocks(clusterBlocks, VERSION, clusterUUID, compressor, namedXContentRegistry); + assertThrows(AssertionError.class, remoteObjectForUpload::getUploadedMetadata); + + try (InputStream inputStream = remoteObjectForUpload.serialize()) { + remoteObjectForUpload.setFullBlobName(new BlobPath().add(TEST_BLOB_PATH)); + UploadedMetadata uploadedMetadata = remoteObjectForUpload.getUploadedMetadata(); + assertThat(uploadedMetadata.getComponent(), is(CLUSTER_BLOCKS)); + assertThat(uploadedMetadata.getUploadedFilename(), is(remoteObjectForUpload.getFullBlobName())); + } + } + + public void testSerDe() throws IOException { + ClusterBlocks clusterBlocks = getClusterBlocks(); + RemoteClusterBlocks remoteObjectForUpload = new RemoteClusterBlocks(clusterBlocks, VERSION, clusterUUID, compressor, namedXContentRegistry); + try (InputStream inputStream = remoteObjectForUpload.serialize()) { + assertThat(inputStream.available(), greaterThan(0)); + ClusterBlocks readClusterBlocks = remoteObjectForUpload.deserialize(inputStream); + assertThat(readClusterBlocks.global().size(), is(1)); + assertThat(readClusterBlocks.global().iterator().next(), is(clusterBlocks.global().iterator().next())); + } + } + + private ClusterBlocks getClusterBlocks() { + return new ClusterBlocks.Builder().addGlobalBlock(new ClusterBlock(12345, "test block", true, false, true, RestStatus.CREATED, EnumSet.of( + ClusterBlockLevel.WRITE))).build(); + } +} diff --git a/server/src/test/java/org/opensearch/gateway/remote/model/RemoteClusterMetadataManifestTests.java b/server/src/test/java/org/opensearch/gateway/remote/model/RemoteClusterMetadataManifestTests.java new file mode 100644 index 0000000000000..7c5c8aa105b44 --- /dev/null +++ b/server/src/test/java/org/opensearch/gateway/remote/model/RemoteClusterMetadataManifestTests.java @@ -0,0 +1,194 @@ +/* + * 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.gateway.remote.model; + +import static java.util.stream.Collectors.toList; +import static org.hamcrest.Matchers.greaterThan; +import static org.hamcrest.Matchers.is; +import static org.hamcrest.Matchers.lessThanOrEqualTo; +import static org.hamcrest.Matchers.nullValue; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; +import static org.opensearch.gateway.remote.model.RemoteClusterMetadataManifest.MANIFEST_CURRENT_CODEC_VERSION; +import static org.opensearch.gateway.remote.model.RemoteClusterMetadataManifest.MANIFEST; + +import java.io.ByteArrayInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.util.Collections; +import java.util.List; +import java.util.function.Function; +import java.util.stream.Stream; +import org.junit.After; +import org.junit.Before; +import org.opensearch.Version; +import org.opensearch.cluster.ClusterModule; +import org.opensearch.common.blobstore.BlobPath; +import org.opensearch.common.compress.DeflateCompressor; +import org.opensearch.common.network.NetworkModule; +import org.opensearch.common.remote.BlobPathParameters; +import org.opensearch.common.settings.ClusterSettings; +import org.opensearch.common.settings.Settings; +import org.opensearch.core.compress.Compressor; +import org.opensearch.core.compress.NoneCompressor; +import org.opensearch.core.xcontent.NamedXContentRegistry; +import org.opensearch.gateway.remote.ClusterMetadataManifest; +import org.opensearch.gateway.remote.ClusterMetadataManifest.UploadedMetadata; +import org.opensearch.gateway.remote.ClusterMetadataManifest.UploadedMetadataAttribute; +import org.opensearch.gateway.remote.ClusterStateDiffManifest; +import org.opensearch.gateway.remote.RemoteClusterStateUtils; +import org.opensearch.index.remote.RemoteStoreUtils; +import org.opensearch.index.translog.transfer.BlobStoreTransferService; +import org.opensearch.indices.IndicesModule; +import org.opensearch.repositories.blobstore.BlobStoreRepository; +import org.opensearch.test.OpenSearchTestCase; +import org.opensearch.threadpool.TestThreadPool; +import org.opensearch.threadpool.ThreadPool; + +public class RemoteClusterMetadataManifestTests extends OpenSearchTestCase { + + private static final String TEST_BLOB_NAME = "/test-path/test-blob-name"; + private static final String TEST_BLOB_PATH = "test-path"; + private static final String TEST_BLOB_FILE_NAME = "test-blob-name"; + + private static final long TERM = 2L; + private static final long VERSION = 5L; + + private String clusterUUID; + private BlobStoreTransferService blobStoreTransferService; + private BlobStoreRepository blobStoreRepository; + private String clusterName; + private ClusterSettings clusterSettings; + private Compressor compressor; + private NamedXContentRegistry namedXContentRegistry; + private final ThreadPool threadPool = new TestThreadPool(getClass().getName()); + + @Before + public void setup() { + clusterSettings = new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); + this.clusterUUID = "test-cluster-uuid"; + this.blobStoreTransferService = mock(BlobStoreTransferService.class); + this.blobStoreRepository = mock(BlobStoreRepository.class); + BlobPath blobPath = new BlobPath().add("/path"); + when(blobStoreRepository.basePath()).thenReturn(blobPath); + when(blobStoreRepository.getCompressor()).thenReturn(new DeflateCompressor()); + compressor = new NoneCompressor(); + namedXContentRegistry = new NamedXContentRegistry( + Stream.of( + NetworkModule.getNamedXContents().stream(), + IndicesModule.getNamedXContents().stream(), + ClusterModule.getNamedXWriteables().stream() + ).flatMap(Function.identity()).collect(toList()) + ); + this.clusterName = "test-cluster-name"; + } + + @After + public void tearDown() throws Exception { + super.tearDown(); + threadPool.shutdown(); + } + + public void testClusterUUID() { + ClusterMetadataManifest manifest = getClusterMetadataManifest(); + RemoteClusterMetadataManifest remoteObjectForUpload = new RemoteClusterMetadataManifest(manifest, clusterUUID, compressor, namedXContentRegistry); + assertThat(remoteObjectForUpload.clusterUUID(), is(clusterUUID)); + + RemoteClusterMetadataManifest remoteObjectForDownload = new RemoteClusterMetadataManifest(TEST_BLOB_NAME, clusterUUID, compressor, + namedXContentRegistry); + assertThat(remoteObjectForDownload.clusterUUID(), is(clusterUUID)); + } + + public void testFullBlobName() { + ClusterMetadataManifest manifest = getClusterMetadataManifest(); + RemoteClusterMetadataManifest remoteObjectForUpload = new RemoteClusterMetadataManifest(manifest, clusterUUID, compressor, namedXContentRegistry); + assertThat(remoteObjectForUpload.getFullBlobName(), nullValue()); + + RemoteClusterMetadataManifest remoteObjectForDownload = new RemoteClusterMetadataManifest(TEST_BLOB_NAME, clusterUUID, compressor, + namedXContentRegistry); + assertThat(remoteObjectForDownload.getFullBlobName(), is(TEST_BLOB_NAME)); + } + + public void testBlobFileName() { + ClusterMetadataManifest manifest = getClusterMetadataManifest(); + RemoteClusterMetadataManifest remoteObjectForUpload = new RemoteClusterMetadataManifest(manifest, clusterUUID, compressor, namedXContentRegistry); + assertThat(remoteObjectForUpload.getBlobFileName(), nullValue()); + + RemoteClusterMetadataManifest remoteObjectForDownload = new RemoteClusterMetadataManifest(TEST_BLOB_NAME, clusterUUID, compressor, + namedXContentRegistry); + assertThat(remoteObjectForDownload.getBlobFileName(), is(TEST_BLOB_FILE_NAME)); + } + + public void testBlobPathTokens() { + String uploadedFile = "user/local/opensearch/manifest"; + RemoteClusterMetadataManifest remoteObjectForDownload = new RemoteClusterMetadataManifest(uploadedFile, clusterUUID, compressor, namedXContentRegistry); + assertThat(remoteObjectForDownload.getBlobPathTokens(), is(new String[]{"user", "local", "opensearch", "manifest"})); + } + + public void testBlobPathParameters() { + ClusterMetadataManifest manifest = getClusterMetadataManifest(); + RemoteClusterMetadataManifest remoteObjectForUpload = new RemoteClusterMetadataManifest(manifest, clusterUUID, compressor, namedXContentRegistry); + BlobPathParameters params = remoteObjectForUpload.getBlobPathParameters(); + assertThat(params.getPathTokens(), is(List.of(MANIFEST))); + assertThat(params.getFilePrefix(), is(RemoteClusterMetadataManifest.MANIFEST)); + } + + public void testGenerateBlobFileName() { + ClusterMetadataManifest manifest = getClusterMetadataManifest(); + RemoteClusterMetadataManifest remoteObjectForUpload = new RemoteClusterMetadataManifest(manifest, clusterUUID, compressor, namedXContentRegistry); + String blobFileName = remoteObjectForUpload.generateBlobFileName(); + String[] nameTokens = blobFileName.split(RemoteClusterStateUtils.DELIMITER); + assertThat(nameTokens[0], is(RemoteClusterMetadataManifest.MANIFEST)); + assertThat(RemoteStoreUtils.invertLong(nameTokens[1]), is(TERM)); + assertThat(RemoteStoreUtils.invertLong(nameTokens[2]), is(VERSION)); + assertThat(nameTokens[3], is("C")); + assertThat(RemoteStoreUtils.invertLong(nameTokens[4]), lessThanOrEqualTo(System.currentTimeMillis())); + assertThat(nameTokens[5], is(String.valueOf(MANIFEST_CURRENT_CODEC_VERSION))); + } + + public void testGetUploadedMetadata() throws IOException { + ClusterMetadataManifest manifest = getClusterMetadataManifest(); + RemoteClusterMetadataManifest remoteObjectForUpload = new RemoteClusterMetadataManifest(manifest, clusterUUID, compressor, namedXContentRegistry); + assertThrows(AssertionError.class, remoteObjectForUpload::getUploadedMetadata); + + try (InputStream inputStream = remoteObjectForUpload.serialize()) { + remoteObjectForUpload.setFullBlobName(new BlobPath().add(TEST_BLOB_PATH)); + UploadedMetadata uploadedMetadata = remoteObjectForUpload.getUploadedMetadata(); + assertThat(uploadedMetadata.getComponent(), is(MANIFEST)); + assertThat(uploadedMetadata.getUploadedFilename(), is(remoteObjectForUpload.getFullBlobName())); + } + } + + public void testSerDe() throws IOException { + ClusterMetadataManifest manifest = getClusterMetadataManifest(); + RemoteClusterMetadataManifest remoteObjectForUpload = new RemoteClusterMetadataManifest(manifest, clusterUUID, compressor, namedXContentRegistry); + try (InputStream inputStream = remoteObjectForUpload.serialize()) { + remoteObjectForUpload.setFullBlobName(BlobPath.cleanPath()); + assertThat(inputStream.available(), greaterThan(0)); + ClusterMetadataManifest readManifest = remoteObjectForUpload.deserialize(inputStream); + assertThat(readManifest, is(manifest)); + } + + String blobName = "/usr/local/manifest__1__2__3__4__5__6"; + RemoteClusterMetadataManifest invalidRemoteObject = new RemoteClusterMetadataManifest(blobName, clusterUUID, compressor, namedXContentRegistry); + assertThrows(IllegalArgumentException.class, () -> invalidRemoteObject.deserialize(new ByteArrayInputStream(new byte[0]))); + } + + private ClusterMetadataManifest getClusterMetadataManifest() { + return ClusterMetadataManifest.builder().opensearchVersion(Version.CURRENT).codecVersion(MANIFEST_CURRENT_CODEC_VERSION).nodeId("test-node") + .clusterUUID("test-uuid").previousClusterUUID("_NA_").stateUUID("state-uuid").clusterTerm(TERM).stateVersion(VERSION).committed(true) + .coordinationMetadata(new UploadedMetadataAttribute("test-attr", "uploaded-file")).diffManifest( + ClusterStateDiffManifest.builder().fromStateUUID("from-uuid").toStateUUID("to-uuid").indicesUpdated(Collections.emptyList()) + .indicesDeleted(Collections.emptyList()) + .customMetadataUpdated(Collections.emptyList()).customMetadataDeleted(Collections.emptyList()) + .indicesRoutingUpdated(Collections.emptyList()).indicesRoutingDeleted(Collections.emptyList()) + .clusterStateCustomUpdated(Collections.emptyList()).clusterStateCustomDeleted(Collections.emptyList()).build()) + .build(); + } +} diff --git a/server/src/test/java/org/opensearch/gateway/remote/model/RemoteCoordinationMetadataTests.java b/server/src/test/java/org/opensearch/gateway/remote/model/RemoteCoordinationMetadataTests.java new file mode 100644 index 0000000000000..97866d5fa16db --- /dev/null +++ b/server/src/test/java/org/opensearch/gateway/remote/model/RemoteCoordinationMetadataTests.java @@ -0,0 +1,178 @@ +/* + * 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.gateway.remote.model; + +import static java.util.stream.Collectors.toList; +import static org.hamcrest.Matchers.greaterThan; +import static org.hamcrest.Matchers.is; +import static org.hamcrest.Matchers.lessThanOrEqualTo; +import static org.hamcrest.Matchers.nullValue; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; +import static org.opensearch.gateway.remote.RemoteClusterStateUtils.GLOBAL_METADATA_CURRENT_CODEC_VERSION; + +import java.io.IOException; +import java.io.InputStream; +import java.util.List; +import java.util.Set; +import java.util.function.Function; +import java.util.stream.Stream; +import org.junit.After; +import org.junit.Before; +import org.opensearch.cluster.ClusterModule; +import org.opensearch.cluster.coordination.CoordinationMetadata; +import org.opensearch.cluster.coordination.CoordinationMetadata.VotingConfigExclusion; +import org.opensearch.cluster.coordination.CoordinationMetadata.VotingConfiguration; +import org.opensearch.common.blobstore.BlobPath; +import org.opensearch.common.compress.DeflateCompressor; +import org.opensearch.common.network.NetworkModule; +import org.opensearch.common.remote.BlobPathParameters; +import org.opensearch.common.settings.ClusterSettings; +import org.opensearch.common.settings.Settings; +import org.opensearch.core.compress.Compressor; +import org.opensearch.core.compress.NoneCompressor; +import org.opensearch.core.xcontent.NamedXContentRegistry; +import org.opensearch.gateway.remote.ClusterMetadataManifest.UploadedMetadata; +import org.opensearch.gateway.remote.RemoteClusterStateUtils; +import org.opensearch.index.remote.RemoteStoreUtils; +import org.opensearch.index.translog.transfer.BlobStoreTransferService; +import org.opensearch.indices.IndicesModule; +import org.opensearch.repositories.blobstore.BlobStoreRepository; +import org.opensearch.test.OpenSearchTestCase; +import org.opensearch.threadpool.TestThreadPool; +import org.opensearch.threadpool.ThreadPool; + +public class RemoteCoordinationMetadataTests extends OpenSearchTestCase { + private static final String TEST_BLOB_NAME = "/test-path/test-blob-name"; + private static final String TEST_BLOB_PATH = "test-path"; + private static final String TEST_BLOB_FILE_NAME = "test-blob-name"; + private static final long TERM = 3L; + private static final long METADATA_VERSION = 3L; + private String clusterUUID; + private BlobStoreTransferService blobStoreTransferService; + private BlobStoreRepository blobStoreRepository; + private String clusterName; + private ClusterSettings clusterSettings; + private Compressor compressor; + private NamedXContentRegistry namedXContentRegistry; + private final ThreadPool threadPool = new TestThreadPool(getClass().getName()); + + @Before + public void setup() { + clusterSettings = new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); + this.clusterUUID = "test-cluster-uuid"; + this.blobStoreTransferService = mock(BlobStoreTransferService.class); + this.blobStoreRepository = mock(BlobStoreRepository.class); + BlobPath blobPath = new BlobPath().add("/path"); + when(blobStoreRepository.basePath()).thenReturn(blobPath); + when(blobStoreRepository.getCompressor()).thenReturn(new DeflateCompressor()); + compressor = new NoneCompressor(); + namedXContentRegistry = new NamedXContentRegistry( + Stream.of( + NetworkModule.getNamedXContents().stream(), + IndicesModule.getNamedXContents().stream(), + ClusterModule.getNamedXWriteables().stream() + ).flatMap(Function.identity()).collect(toList()) + ); + this.clusterName = "test-cluster-name"; + } + + @After + public void tearDown() throws Exception { + super.tearDown(); + threadPool.shutdown(); + } + + public void testClusterUUID() { + CoordinationMetadata coordinationMetadata = getCoordinationMetadata(); + RemoteCoordinationMetadata remoteObjectForUpload = new RemoteCoordinationMetadata(coordinationMetadata, METADATA_VERSION, clusterUUID, compressor, namedXContentRegistry); + assertThat(remoteObjectForUpload.clusterUUID(), is(clusterUUID)); + + RemoteCoordinationMetadata remoteObjectForDownload = new RemoteCoordinationMetadata(TEST_BLOB_NAME, clusterUUID, compressor, + namedXContentRegistry); + assertThat(remoteObjectForDownload.clusterUUID(), is(clusterUUID)); + } + + public void testFullBlobName() { + CoordinationMetadata coordinationMetadata = getCoordinationMetadata(); + RemoteCoordinationMetadata remoteObjectForUpload = new RemoteCoordinationMetadata(coordinationMetadata, METADATA_VERSION, clusterUUID, compressor, namedXContentRegistry); + assertThat(remoteObjectForUpload.getFullBlobName(), nullValue()); + + RemoteCoordinationMetadata remoteObjectForDownload = new RemoteCoordinationMetadata(TEST_BLOB_NAME, clusterUUID, compressor, + namedXContentRegistry); + assertThat(remoteObjectForDownload.getFullBlobName(), is(TEST_BLOB_NAME)); + } + + public void testBlobFileName() { + CoordinationMetadata coordinationMetadata = getCoordinationMetadata(); + RemoteCoordinationMetadata remoteObjectForUpload = new RemoteCoordinationMetadata(coordinationMetadata, METADATA_VERSION, clusterUUID, compressor, namedXContentRegistry); + assertThat(remoteObjectForUpload.getBlobFileName(), nullValue()); + + RemoteCoordinationMetadata remoteObjectForDownload = new RemoteCoordinationMetadata(TEST_BLOB_NAME, clusterUUID, compressor, + namedXContentRegistry); + assertThat(remoteObjectForDownload.getBlobFileName(), is(TEST_BLOB_FILE_NAME)); + } + + public void testBlobPathTokens() { + String uploadedFile = "user/local/opensearch/coordinationMetadata"; + RemoteCoordinationMetadata remoteObjectForDownload = new RemoteCoordinationMetadata(uploadedFile, clusterUUID, compressor, namedXContentRegistry); + assertThat(remoteObjectForDownload.getBlobPathTokens(), is(new String[]{"user", "local", "opensearch", "coordinationMetadata"})); + } + + public void testBlobPathParameters() { + CoordinationMetadata coordinationMetadata = getCoordinationMetadata(); + RemoteCoordinationMetadata remoteObjectForUpload = new RemoteCoordinationMetadata(coordinationMetadata, METADATA_VERSION, clusterUUID, compressor, namedXContentRegistry); + BlobPathParameters params = remoteObjectForUpload.getBlobPathParameters(); + assertThat(params.getPathTokens(), is(List.of(RemoteClusterStateUtils.GLOBAL_METADATA_PATH_TOKEN))); + assertThat(params.getFilePrefix(), is(RemoteCoordinationMetadata.COORDINATION_METADATA)); + } + + public void testGenerateBlobFileName() { + CoordinationMetadata coordinationMetadata = getCoordinationMetadata(); + RemoteCoordinationMetadata remoteObjectForUpload = new RemoteCoordinationMetadata(coordinationMetadata, METADATA_VERSION, clusterUUID, compressor, namedXContentRegistry); + String blobFileName = remoteObjectForUpload.generateBlobFileName(); + String[] nameTokens = blobFileName.split(RemoteClusterStateUtils.DELIMITER); + assertThat(nameTokens[0], is(RemoteCoordinationMetadata.COORDINATION_METADATA)); + assertThat(RemoteStoreUtils.invertLong(nameTokens[1]), is(METADATA_VERSION)); + assertThat(RemoteStoreUtils.invertLong(nameTokens[2]), lessThanOrEqualTo(System.currentTimeMillis())); + assertThat(nameTokens[3], is(String.valueOf(GLOBAL_METADATA_CURRENT_CODEC_VERSION))); + + } + + public void testGetUploadedMetadata() throws IOException { + CoordinationMetadata coordinationMetadata = getCoordinationMetadata(); + RemoteCoordinationMetadata remoteObjectForUpload = new RemoteCoordinationMetadata(coordinationMetadata, METADATA_VERSION, clusterUUID, compressor, namedXContentRegistry); + assertThrows(AssertionError.class, remoteObjectForUpload::getUploadedMetadata); + + try (InputStream inputStream = remoteObjectForUpload.serialize()) { + remoteObjectForUpload.setFullBlobName(new BlobPath().add(TEST_BLOB_PATH)); + UploadedMetadata uploadedMetadata = remoteObjectForUpload.getUploadedMetadata(); + assertThat(uploadedMetadata.getComponent(), is(RemoteCoordinationMetadata.COORDINATION_METADATA)); + assertThat(uploadedMetadata.getUploadedFilename(), is(remoteObjectForUpload.getFullBlobName())); + } + } + + public void testSerDe() throws IOException { + CoordinationMetadata coordinationMetadata = getCoordinationMetadata(); + RemoteCoordinationMetadata remoteObjectForUpload = new RemoteCoordinationMetadata(coordinationMetadata, METADATA_VERSION, clusterUUID, compressor, namedXContentRegistry); + try (InputStream inputStream = remoteObjectForUpload.serialize()) { + remoteObjectForUpload.setFullBlobName(BlobPath.cleanPath()); + assertThat(inputStream.available(), greaterThan(0)); + CoordinationMetadata readcoordinationMetadata = remoteObjectForUpload.deserialize(inputStream); + assertThat(readcoordinationMetadata, is(coordinationMetadata)); + } + } + + private CoordinationMetadata getCoordinationMetadata() { + return CoordinationMetadata.builder().term(TERM).lastAcceptedConfiguration(new VotingConfiguration(Set.of("node1"))) + .lastCommittedConfiguration(new VotingConfiguration(Set.of("node1"))) + .addVotingConfigExclusion(new VotingConfigExclusion("node2", " node-2")) + .build(); + } +} diff --git a/server/src/test/java/org/opensearch/gateway/remote/model/RemoteCustomMetadataTests.java b/server/src/test/java/org/opensearch/gateway/remote/model/RemoteCustomMetadataTests.java new file mode 100644 index 0000000000000..b68409175305d --- /dev/null +++ b/server/src/test/java/org/opensearch/gateway/remote/model/RemoteCustomMetadataTests.java @@ -0,0 +1,181 @@ +/* + * 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.gateway.remote.model; + +import static java.util.stream.Collectors.toList; +import static org.hamcrest.Matchers.greaterThan; +import static org.hamcrest.Matchers.is; +import static org.hamcrest.Matchers.lessThanOrEqualTo; +import static org.hamcrest.Matchers.nullValue; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; +import static org.opensearch.gateway.remote.RemoteClusterStateUtils.GLOBAL_METADATA_CURRENT_CODEC_VERSION; +import static org.opensearch.gateway.remote.model.RemoteCustomMetadata.CUSTOM_DELIMITER; +import static org.opensearch.gateway.remote.model.RemoteCustomMetadata.CUSTOM_METADATA; + +import java.io.IOException; +import java.io.InputStream; +import java.util.List; +import java.util.function.Function; +import java.util.stream.Stream; +import org.junit.After; +import org.junit.Before; +import org.opensearch.cluster.ClusterModule; +import org.opensearch.cluster.metadata.IndexGraveyard; +import org.opensearch.cluster.metadata.Metadata.Custom; +import org.opensearch.common.blobstore.BlobPath; +import org.opensearch.common.compress.DeflateCompressor; +import org.opensearch.common.network.NetworkModule; +import org.opensearch.common.remote.BlobPathParameters; +import org.opensearch.common.settings.ClusterSettings; +import org.opensearch.common.settings.Settings; +import org.opensearch.core.compress.Compressor; +import org.opensearch.core.compress.NoneCompressor; +import org.opensearch.core.index.Index; +import org.opensearch.core.xcontent.NamedXContentRegistry; +import org.opensearch.gateway.remote.ClusterMetadataManifest.UploadedMetadata; +import org.opensearch.gateway.remote.RemoteClusterStateUtils; +import org.opensearch.index.remote.RemoteStoreUtils; +import org.opensearch.index.translog.transfer.BlobStoreTransferService; +import org.opensearch.indices.IndicesModule; +import org.opensearch.repositories.blobstore.BlobStoreRepository; +import org.opensearch.test.OpenSearchTestCase; +import org.opensearch.threadpool.TestThreadPool; +import org.opensearch.threadpool.ThreadPool; + +public class RemoteCustomMetadataTests extends OpenSearchTestCase { + private static final String TEST_BLOB_NAME = "/test-path/test-blob-name"; + private static final String TEST_BLOB_PATH = "test-path"; + private static final String TEST_BLOB_FILE_NAME = "test-blob-name"; + private static final String CUSTOM_TYPE = "test-custom"; + private static final long METADATA_VERSION = 3L; + private String clusterUUID; + private BlobStoreTransferService blobStoreTransferService; + private BlobStoreRepository blobStoreRepository; + private String clusterName; + private ClusterSettings clusterSettings; + private Compressor compressor; + private NamedXContentRegistry namedXContentRegistry; + private final ThreadPool threadPool = new TestThreadPool(getClass().getName()); + + @Before + public void setup() { + clusterSettings = new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); + this.clusterUUID = "test-cluster-uuid"; + this.blobStoreTransferService = mock(BlobStoreTransferService.class); + this.blobStoreRepository = mock(BlobStoreRepository.class); + BlobPath blobPath = new BlobPath().add("/path"); + when(blobStoreRepository.basePath()).thenReturn(blobPath); + when(blobStoreRepository.getCompressor()).thenReturn(new DeflateCompressor()); + compressor = new NoneCompressor(); + namedXContentRegistry = new NamedXContentRegistry( + Stream.of( + NetworkModule.getNamedXContents().stream(), + IndicesModule.getNamedXContents().stream(), + ClusterModule.getNamedXWriteables().stream() + ).flatMap(Function.identity()).collect(toList()) + ); +// namedXContentRegistry = new NamedXContentRegistry(List.of(new Entry(Metadata.Custom.class, new ParseField(CUSTOM_TYPE), p->TestCustomMetadata.fromXContent(CustomMetadata1::new, p)))); + this.clusterName = "test-cluster-name"; + } + + @After + public void tearDown() throws Exception { + super.tearDown(); + threadPool.shutdown(); + } + + public void testClusterUUID() { + Custom customMetadata = getCustomMetadata(); + RemoteCustomMetadata remoteObjectForUpload = new RemoteCustomMetadata(customMetadata, "test-custom", METADATA_VERSION, clusterUUID, compressor, namedXContentRegistry); + assertThat(remoteObjectForUpload.clusterUUID(), is(clusterUUID)); + + RemoteCustomMetadata remoteObjectForDownload = new RemoteCustomMetadata(TEST_BLOB_NAME, "test-custom", clusterUUID, compressor, + namedXContentRegistry); + assertThat(remoteObjectForDownload.clusterUUID(), is(clusterUUID)); + } + + public void testFullBlobName() { + Custom customMetadata = getCustomMetadata(); + RemoteCustomMetadata remoteObjectForUpload = new RemoteCustomMetadata(customMetadata, "test-custom", METADATA_VERSION, clusterUUID, compressor, namedXContentRegistry); + assertThat(remoteObjectForUpload.getFullBlobName(), nullValue()); + + RemoteCustomMetadata remoteObjectForDownload = new RemoteCustomMetadata(TEST_BLOB_NAME, "test-custom", clusterUUID, compressor, + namedXContentRegistry); + assertThat(remoteObjectForDownload.getFullBlobName(), is(TEST_BLOB_NAME)); + } + + public void testBlobFileName() { + Custom customMetadata = getCustomMetadata(); + RemoteCustomMetadata remoteObjectForUpload = new RemoteCustomMetadata(customMetadata, "test-custom", METADATA_VERSION, clusterUUID, compressor, namedXContentRegistry); + assertThat(remoteObjectForUpload.getBlobFileName(), nullValue()); + + RemoteCustomMetadata remoteObjectForDownload = new RemoteCustomMetadata(TEST_BLOB_NAME, "test-custom", clusterUUID, compressor, + namedXContentRegistry); + assertThat(remoteObjectForDownload.getBlobFileName(), is(TEST_BLOB_FILE_NAME)); + } + + public void testBlobPathTokens() { + String uploadedFile = "user/local/opensearch/customMetadata"; + RemoteCustomMetadata remoteObjectForDownload = new RemoteCustomMetadata(uploadedFile, "test-custom", clusterUUID, compressor, namedXContentRegistry); + assertThat(remoteObjectForDownload.getBlobPathTokens(), is(new String[]{"user", "local", "opensearch", "customMetadata"})); + } + + public void testBlobPathParameters() { + Custom customMetadata = getCustomMetadata(); + RemoteCustomMetadata remoteObjectForUpload = new RemoteCustomMetadata(customMetadata, "test-custom", METADATA_VERSION, clusterUUID, compressor, namedXContentRegistry); + BlobPathParameters params = remoteObjectForUpload.getBlobPathParameters(); + assertThat(params.getPathTokens(), is(List.of(RemoteClusterStateUtils.GLOBAL_METADATA_PATH_TOKEN))); + String expectedPrefix = CUSTOM_METADATA + CUSTOM_DELIMITER + CUSTOM_TYPE; + assertThat(params.getFilePrefix(), is(expectedPrefix)); + } + + public void testGenerateBlobFileName() { + Custom customMetadata = getCustomMetadata(); + RemoteCustomMetadata remoteObjectForUpload = new RemoteCustomMetadata(customMetadata, "test-custom", METADATA_VERSION, clusterUUID, compressor, namedXContentRegistry); + String blobFileName = remoteObjectForUpload.generateBlobFileName(); + String[] nameTokens = blobFileName.split(RemoteClusterStateUtils.DELIMITER); + String expectedPrefix = CUSTOM_METADATA + CUSTOM_DELIMITER + CUSTOM_TYPE; + assertThat(nameTokens[0], is(expectedPrefix)); + assertThat(RemoteStoreUtils.invertLong(nameTokens[1]), is(METADATA_VERSION)); + assertThat(RemoteStoreUtils.invertLong(nameTokens[2]), lessThanOrEqualTo(System.currentTimeMillis())); + assertThat(nameTokens[3], is(String.valueOf(GLOBAL_METADATA_CURRENT_CODEC_VERSION))); + + } + + public void testGetUploadedMetadata() throws IOException { + Custom customMetadata = getCustomMetadata(); + RemoteCustomMetadata remoteObjectForUpload = new RemoteCustomMetadata(customMetadata, "test-custom", METADATA_VERSION, clusterUUID, compressor, namedXContentRegistry); + assertThrows(AssertionError.class, remoteObjectForUpload::getUploadedMetadata); + + try (InputStream inputStream = remoteObjectForUpload.serialize()) { + remoteObjectForUpload.setFullBlobName(new BlobPath().add(TEST_BLOB_PATH)); + UploadedMetadata uploadedMetadata = remoteObjectForUpload.getUploadedMetadata(); + String expectedPrefix = CUSTOM_METADATA + CUSTOM_DELIMITER + CUSTOM_TYPE; + assertThat(uploadedMetadata.getComponent(), is(expectedPrefix)); + assertThat(uploadedMetadata.getUploadedFilename(), is(remoteObjectForUpload.getFullBlobName())); + } + } + + public void testSerDe() throws IOException { + Custom customMetadata = getCustomMetadata(); + RemoteCustomMetadata remoteObjectForUpload = new RemoteCustomMetadata(customMetadata, IndexGraveyard.TYPE, METADATA_VERSION, clusterUUID, compressor, namedXContentRegistry); + try (InputStream inputStream = remoteObjectForUpload.serialize()) { + remoteObjectForUpload.setFullBlobName(BlobPath.cleanPath()); + assertThat(inputStream.available(), greaterThan(0)); + Custom readCustomMetadata = remoteObjectForUpload.deserialize(inputStream); + assertThat(readCustomMetadata, is(customMetadata)); + } + } + + private Custom getCustomMetadata() { + return IndexGraveyard.builder().addTombstone(new Index("test-index", "3q2423")).build(); + } + +} diff --git a/server/src/test/java/org/opensearch/gateway/remote/model/RemoteGlobalMetadataTests.java b/server/src/test/java/org/opensearch/gateway/remote/model/RemoteGlobalMetadataTests.java new file mode 100644 index 0000000000000..3f5a1d1885d25 --- /dev/null +++ b/server/src/test/java/org/opensearch/gateway/remote/model/RemoteGlobalMetadataTests.java @@ -0,0 +1,162 @@ +/* + * 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.gateway.remote.model; + +import static java.util.stream.Collectors.toList; +import static org.hamcrest.Matchers.greaterThan; +import static org.hamcrest.Matchers.is; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; +import static org.opensearch.gateway.remote.model.RemoteGlobalMetadata.GLOBAL_METADATA_FORMAT; + +import java.io.IOException; +import java.io.InputStream; +import java.util.Arrays; +import java.util.Set; +import java.util.function.Function; +import java.util.stream.Stream; +import org.junit.After; +import org.junit.Before; +import org.opensearch.cluster.ClusterModule; +import org.opensearch.cluster.coordination.CoordinationMetadata; +import org.opensearch.cluster.coordination.CoordinationMetadata.VotingConfigExclusion; +import org.opensearch.cluster.coordination.CoordinationMetadata.VotingConfiguration; +import org.opensearch.cluster.metadata.IndexTemplateMetadata; +import org.opensearch.cluster.metadata.Metadata; +import org.opensearch.cluster.metadata.TemplatesMetadata; +import org.opensearch.common.blobstore.BlobPath; +import org.opensearch.common.compress.DeflateCompressor; +import org.opensearch.common.network.NetworkModule; +import org.opensearch.common.settings.ClusterSettings; +import org.opensearch.common.settings.Settings; +import org.opensearch.core.compress.Compressor; +import org.opensearch.core.compress.NoneCompressor; +import org.opensearch.core.xcontent.NamedXContentRegistry; +import org.opensearch.gateway.remote.RemoteClusterStateUtils; +import org.opensearch.index.translog.transfer.BlobStoreTransferService; +import org.opensearch.indices.IndicesModule; +import org.opensearch.repositories.blobstore.BlobStoreRepository; +import org.opensearch.test.OpenSearchTestCase; +import org.opensearch.threadpool.TestThreadPool; +import org.opensearch.threadpool.ThreadPool; + +public class RemoteGlobalMetadataTests extends OpenSearchTestCase { + + private static final String TEST_BLOB_NAME = "/test-path/test-blob-name"; + private static final String TEST_BLOB_FILE_NAME = "test-blob-name"; + private static final long TERM = 3L; + private String clusterUUID; + private BlobStoreTransferService blobStoreTransferService; + private BlobStoreRepository blobStoreRepository; + private String clusterName; + private ClusterSettings clusterSettings; + private Compressor compressor; + private NamedXContentRegistry namedXContentRegistry; + private final ThreadPool threadPool = new TestThreadPool(getClass().getName()); + + @Before + public void setup() { + clusterSettings = new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); + this.clusterUUID = "test-cluster-uuid"; + this.blobStoreTransferService = mock(BlobStoreTransferService.class); + this.blobStoreRepository = mock(BlobStoreRepository.class); + BlobPath blobPath = new BlobPath().add("/path"); + when(blobStoreRepository.basePath()).thenReturn(blobPath); + when(blobStoreRepository.getCompressor()).thenReturn(new DeflateCompressor()); + compressor = new NoneCompressor(); + namedXContentRegistry = new NamedXContentRegistry( + Stream.of( + NetworkModule.getNamedXContents().stream(), + IndicesModule.getNamedXContents().stream(), + ClusterModule.getNamedXWriteables().stream() + ).flatMap(Function.identity()).collect(toList()) + ); + this.clusterName = "test-cluster-name"; + } + + @After + public void tearDown() throws Exception { + super.tearDown(); + threadPool.shutdown(); + } + + public void testClusterUUID() { + RemoteGlobalMetadata remoteObjectForDownload = new RemoteGlobalMetadata(TEST_BLOB_NAME, clusterUUID, compressor, + namedXContentRegistry); + assertThat(remoteObjectForDownload.clusterUUID(), is(clusterUUID)); + } + + public void testFullBlobName() { + RemoteGlobalMetadata remoteObjectForDownload = new RemoteGlobalMetadata(TEST_BLOB_NAME, clusterUUID, compressor, + namedXContentRegistry); + assertThat(remoteObjectForDownload.getFullBlobName(), is(TEST_BLOB_NAME)); + } + + public void testBlobFileName() { + RemoteGlobalMetadata remoteObjectForDownload = new RemoteGlobalMetadata(TEST_BLOB_NAME, clusterUUID, compressor, + namedXContentRegistry); + assertThat(remoteObjectForDownload.getBlobFileName(), is(TEST_BLOB_FILE_NAME)); + } + + public void testBlobPathTokens() { + String uploadedFile = "user/local/opensearch/globalMetadata"; + RemoteGlobalMetadata remoteObjectForDownload = new RemoteGlobalMetadata(uploadedFile, clusterUUID, compressor, namedXContentRegistry); + assertThat(remoteObjectForDownload.getBlobPathTokens(), is(new String[]{"user", "local", "opensearch", "globalMetadata"})); + } + + public void testBlobPathParameters() { + RemoteGlobalMetadata remoteObjectForDownload = new RemoteGlobalMetadata(TEST_BLOB_NAME, clusterUUID, compressor, namedXContentRegistry); + assertThrows(UnsupportedOperationException.class, remoteObjectForDownload::getBlobPathParameters); + } + + public void testGenerateBlobFileName() { + RemoteGlobalMetadata remoteObjectForDownload = new RemoteGlobalMetadata(TEST_BLOB_NAME, clusterUUID, compressor, namedXContentRegistry); + assertThrows(UnsupportedOperationException.class, remoteObjectForDownload::generateBlobFileName); + } + + public void testGetUploadedMetadata() { + RemoteGlobalMetadata remoteObjectForDownload = new RemoteGlobalMetadata(TEST_BLOB_NAME, clusterUUID, compressor, namedXContentRegistry); + assertThrows(UnsupportedOperationException.class, remoteObjectForDownload::getUploadedMetadata); + } + + public void testSerDe() throws IOException { + Metadata globalMetadata = getGlobalMetadata(); + try (InputStream inputStream = GLOBAL_METADATA_FORMAT.serialize(globalMetadata, TEST_BLOB_FILE_NAME, compressor, RemoteClusterStateUtils.FORMAT_PARAMS) + .streamInput()) { + RemoteGlobalMetadata remoteObjectForDownload = new RemoteGlobalMetadata(TEST_BLOB_NAME, clusterUUID, compressor, namedXContentRegistry); + assertThat(inputStream.available(), greaterThan(0)); + Metadata readglobalMetadata = remoteObjectForDownload.deserialize(inputStream); + assertTrue(Metadata.isGlobalStateEquals(readglobalMetadata, globalMetadata)); + } + } + + private Metadata getGlobalMetadata() { + return Metadata.builder() + .templates( + TemplatesMetadata.builder() + .put( + IndexTemplateMetadata.builder("template" + randomAlphaOfLength(3)) + .patterns(Arrays.asList("bar-*", "foo-*")) + .settings( + Settings.builder() + .put("index.random_index_setting_" + randomAlphaOfLength(3), randomAlphaOfLength(5)) + .build() + ) + .build() + ) + .build() + ) + .coordinationMetadata(CoordinationMetadata.builder().term(TERM).lastAcceptedConfiguration(new VotingConfiguration(Set.of("node1"))) + .lastCommittedConfiguration(new VotingConfiguration(Set.of("node1"))) + .addVotingConfigExclusion(new VotingConfigExclusion("node2", " node-2")) + .build()) + .build(); + + } +} diff --git a/server/src/test/java/org/opensearch/gateway/remote/model/RemoteIndexMetadataTests.java b/server/src/test/java/org/opensearch/gateway/remote/model/RemoteIndexMetadataTests.java new file mode 100644 index 0000000000000..9c16c093b99b5 --- /dev/null +++ b/server/src/test/java/org/opensearch/gateway/remote/model/RemoteIndexMetadataTests.java @@ -0,0 +1,175 @@ +/* + * 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.gateway.remote.model; + +import static java.util.stream.Collectors.toList; +import static org.hamcrest.Matchers.greaterThan; +import static org.hamcrest.Matchers.is; +import static org.hamcrest.Matchers.lessThanOrEqualTo; +import static org.hamcrest.Matchers.nullValue; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; +import static org.opensearch.gateway.remote.ClusterMetadataManifest.UploadedIndexMetadata.COMPONENT_PREFIX; +import static org.opensearch.gateway.remote.model.RemoteIndexMetadata.INDEX_METADATA_CURRENT_CODEC_VERSION; +import static org.opensearch.gateway.remote.model.RemoteIndexMetadata.INDEX_PATH_TOKEN; + +import java.io.IOException; +import java.io.InputStream; +import java.util.List; +import java.util.function.Function; +import java.util.stream.Stream; +import org.junit.After; +import org.junit.Before; +import org.opensearch.Version; +import org.opensearch.cluster.ClusterModule; +import org.opensearch.cluster.metadata.IndexMetadata; +import org.opensearch.common.blobstore.BlobPath; +import org.opensearch.common.compress.DeflateCompressor; +import org.opensearch.common.network.NetworkModule; +import org.opensearch.common.settings.ClusterSettings; +import org.opensearch.common.settings.Settings; +import org.opensearch.core.compress.Compressor; +import org.opensearch.core.compress.NoneCompressor; +import org.opensearch.core.index.Index; +import org.opensearch.core.xcontent.NamedXContentRegistry; +import org.opensearch.gateway.remote.ClusterMetadataManifest.UploadedMetadata; +import org.opensearch.gateway.remote.RemoteClusterStateUtils; +import org.opensearch.common.remote.BlobPathParameters; +import org.opensearch.index.remote.RemoteStoreUtils; +import org.opensearch.index.translog.transfer.BlobStoreTransferService; +import org.opensearch.indices.IndicesModule; +import org.opensearch.repositories.blobstore.BlobStoreRepository; +import org.opensearch.test.OpenSearchTestCase; +import org.opensearch.threadpool.TestThreadPool; +import org.opensearch.threadpool.ThreadPool; + +public class RemoteIndexMetadataTests extends OpenSearchTestCase { + + private static final String TEST_BLOB_NAME = "/test-path/test-blob-name"; + private static final String TEST_BLOB_PATH = "test-path"; + private static final String TEST_BLOB_FILE_NAME = "test-blob-name"; + private static final long VERSION = 5L; + + private String clusterUUID; + private BlobStoreTransferService blobStoreTransferService; + private BlobStoreRepository blobStoreRepository; + private String clusterName; + private ClusterSettings clusterSettings; + private Compressor compressor; + private NamedXContentRegistry namedXContentRegistry; + private final ThreadPool threadPool = new TestThreadPool(getClass().getName()); + + @Before + public void setup() { + clusterSettings = new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); + this.clusterUUID = "test-cluster-uuid"; + this.blobStoreTransferService = mock(BlobStoreTransferService.class); + this.blobStoreRepository = mock(BlobStoreRepository.class); + BlobPath blobPath = new BlobPath().add("/path"); + when(blobStoreRepository.basePath()).thenReturn(blobPath); + when(blobStoreRepository.getCompressor()).thenReturn(new DeflateCompressor()); + this.clusterName = "test-cluster-name"; + compressor = new NoneCompressor(); + namedXContentRegistry = new NamedXContentRegistry( + Stream.of( + NetworkModule.getNamedXContents().stream(), + IndicesModule.getNamedXContents().stream(), + ClusterModule.getNamedXWriteables().stream() + ).flatMap(Function.identity()).collect(toList()) + ); + } + + @After + public void tearDown() throws Exception { + super.tearDown(); + threadPool.shutdown(); + } + + public void testClusterUUID() { + IndexMetadata indexMetadata = getIndexMetadata(); + RemoteIndexMetadata remoteObjectForUpload = new RemoteIndexMetadata(indexMetadata, clusterUUID, compressor, namedXContentRegistry); + assertThat(remoteObjectForUpload.clusterUUID(), is(clusterUUID)); + + RemoteIndexMetadata remoteObjectForDownload = new RemoteIndexMetadata(TEST_BLOB_NAME, clusterUUID, compressor, namedXContentRegistry); + assertThat(remoteObjectForDownload.clusterUUID(), is(clusterUUID)); + } + + public void testFullBlobName() { + IndexMetadata indexMetadata = getIndexMetadata(); + RemoteIndexMetadata remoteObjectForUpload = new RemoteIndexMetadata(indexMetadata, clusterUUID, compressor, namedXContentRegistry); + assertThat(remoteObjectForUpload.getFullBlobName(), nullValue()); + + RemoteIndexMetadata remoteObjectForDownload = new RemoteIndexMetadata(TEST_BLOB_NAME, clusterUUID, compressor, namedXContentRegistry); + assertThat(remoteObjectForDownload.getFullBlobName(), is(TEST_BLOB_NAME)); + } + + public void testBlobFileName() { + IndexMetadata indexMetadata = getIndexMetadata(); + RemoteIndexMetadata remoteObjectForUpload = new RemoteIndexMetadata(indexMetadata, clusterUUID, compressor, namedXContentRegistry); + assertThat(remoteObjectForUpload.getBlobFileName(), nullValue()); + + RemoteIndexMetadata remoteObjectForDownload = new RemoteIndexMetadata(TEST_BLOB_NAME, clusterUUID, compressor, namedXContentRegistry); + assertThat(remoteObjectForDownload.getBlobFileName(), is(TEST_BLOB_FILE_NAME)); + } + + public void testBlobPathParameters() { + IndexMetadata indexMetadata = getIndexMetadata(); + RemoteIndexMetadata remoteObjectForUpload = new RemoteIndexMetadata(indexMetadata, clusterUUID, compressor, namedXContentRegistry); + BlobPathParameters params = remoteObjectForUpload.getBlobPathParameters(); + assertThat(params.getPathTokens(), is(List.of(INDEX_PATH_TOKEN, indexMetadata.getIndexUUID()))); + assertThat(params.getFilePrefix(), is("metadata")); + } + + public void testGenerateBlobFileName() { + IndexMetadata indexMetadata = getIndexMetadata(); + RemoteIndexMetadata remoteObjectForUpload = new RemoteIndexMetadata(indexMetadata, clusterUUID, compressor, namedXContentRegistry); + String blobFileName = remoteObjectForUpload.generateBlobFileName(); + String[] nameTokens = blobFileName.split(RemoteClusterStateUtils.DELIMITER); + assertThat(nameTokens[0], is("metadata")); + assertThat(RemoteStoreUtils.invertLong(nameTokens[1]), is(VERSION)); + assertThat(RemoteStoreUtils.invertLong(nameTokens[2]), lessThanOrEqualTo(System.currentTimeMillis())); + assertThat(nameTokens[3], is(String.valueOf(INDEX_METADATA_CURRENT_CODEC_VERSION))); + } + + public void testGetUploadedMetadata() throws IOException { + IndexMetadata indexMetadata = getIndexMetadata(); + RemoteIndexMetadata remoteObjectForUpload = new RemoteIndexMetadata(indexMetadata, clusterUUID, compressor, namedXContentRegistry); + assertThrows(AssertionError.class, remoteObjectForUpload::getUploadedMetadata); + + try (InputStream inputStream = remoteObjectForUpload.serialize()){ + remoteObjectForUpload.setFullBlobName(new BlobPath().add(TEST_BLOB_PATH)); + UploadedMetadata uploadedMetadata = remoteObjectForUpload.getUploadedMetadata(); + assertThat(uploadedMetadata.getComponent(), is(COMPONENT_PREFIX + "test-index")); + assertThat(uploadedMetadata.getUploadedFilename(), is(remoteObjectForUpload.getFullBlobName())); + } + } + + public void testSerDe() throws IOException { + IndexMetadata indexMetadata = getIndexMetadata(); + RemoteIndexMetadata remoteObjectForUpload = new RemoteIndexMetadata(indexMetadata, clusterUUID, compressor, namedXContentRegistry); + try (InputStream inputStream = remoteObjectForUpload.serialize()) { + assertThat(inputStream.available(), greaterThan(0)); + IndexMetadata readIndexMetadata = remoteObjectForUpload.deserialize(inputStream); + assertThat(readIndexMetadata, is(indexMetadata)); + } + } + + private IndexMetadata getIndexMetadata() { + final Index index = new Index("test-index", "index-uuid"); + final Settings idxSettings = Settings.builder() + .put(IndexMetadata.SETTING_VERSION_CREATED, Version.CURRENT) + .put(IndexMetadata.SETTING_INDEX_UUID, index.getUUID()) + .build(); + return new IndexMetadata.Builder(index.getName()).settings(idxSettings) + .version(VERSION) + .numberOfShards(1) + .numberOfReplicas(0) + .build(); + } +} diff --git a/server/src/test/java/org/opensearch/gateway/remote/model/RemotePersistentSettingsMetadataTests.java b/server/src/test/java/org/opensearch/gateway/remote/model/RemotePersistentSettingsMetadataTests.java new file mode 100644 index 0000000000000..477027ab68ce8 --- /dev/null +++ b/server/src/test/java/org/opensearch/gateway/remote/model/RemotePersistentSettingsMetadataTests.java @@ -0,0 +1,172 @@ +/* + * 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.gateway.remote.model; + +import static java.util.stream.Collectors.toList; +import static org.hamcrest.Matchers.greaterThan; +import static org.hamcrest.Matchers.is; +import static org.hamcrest.Matchers.lessThanOrEqualTo; +import static org.hamcrest.Matchers.nullValue; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; +import static org.opensearch.gateway.remote.RemoteClusterStateUtils.GLOBAL_METADATA_CURRENT_CODEC_VERSION; + +import java.io.IOException; +import java.io.InputStream; +import java.util.List; +import java.util.function.Function; +import java.util.stream.Stream; +import org.junit.After; +import org.junit.Before; +import org.opensearch.cluster.ClusterModule; +import org.opensearch.common.blobstore.BlobPath; +import org.opensearch.common.compress.DeflateCompressor; +import org.opensearch.common.network.NetworkModule; +import org.opensearch.common.remote.BlobPathParameters; +import org.opensearch.common.settings.ClusterSettings; +import org.opensearch.common.settings.Settings; +import org.opensearch.core.compress.Compressor; +import org.opensearch.core.compress.NoneCompressor; +import org.opensearch.core.xcontent.NamedXContentRegistry; +import org.opensearch.gateway.remote.ClusterMetadataManifest.UploadedMetadata; +import org.opensearch.gateway.remote.RemoteClusterStateUtils; +import org.opensearch.index.remote.RemoteStoreUtils; +import org.opensearch.index.translog.transfer.BlobStoreTransferService; +import org.opensearch.indices.IndicesModule; +import org.opensearch.repositories.blobstore.BlobStoreRepository; +import org.opensearch.test.OpenSearchTestCase; +import org.opensearch.threadpool.TestThreadPool; +import org.opensearch.threadpool.ThreadPool; + +public class RemotePersistentSettingsMetadataTests extends OpenSearchTestCase { + private static final String TEST_BLOB_NAME = "/test-path/test-blob-name"; + private static final String TEST_BLOB_PATH = "test-path"; + private static final String TEST_BLOB_FILE_NAME = "test-blob-name"; + private static final long METADATA_VERSION = 3L; + private String clusterUUID; + private BlobStoreTransferService blobStoreTransferService; + private BlobStoreRepository blobStoreRepository; + private String clusterName; + private ClusterSettings clusterSettings; + private Compressor compressor; + private NamedXContentRegistry namedXContentRegistry; + private final ThreadPool threadPool = new TestThreadPool(getClass().getName()); + + @Before + public void setup() { + clusterSettings = new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); + this.clusterUUID = "test-cluster-uuid"; + this.blobStoreTransferService = mock(BlobStoreTransferService.class); + this.blobStoreRepository = mock(BlobStoreRepository.class); + BlobPath blobPath = new BlobPath().add("/path"); + when(blobStoreRepository.basePath()).thenReturn(blobPath); + when(blobStoreRepository.getCompressor()).thenReturn(new DeflateCompressor()); + compressor = new NoneCompressor(); + namedXContentRegistry = new NamedXContentRegistry( + Stream.of( + NetworkModule.getNamedXContents().stream(), + IndicesModule.getNamedXContents().stream(), + ClusterModule.getNamedXWriteables().stream() + ).flatMap(Function.identity()).collect(toList()) + ); + this.clusterName = "test-cluster-name"; + } + + @After + public void tearDown() throws Exception { + super.tearDown(); + threadPool.shutdown(); + } + + public void testClusterUUID() { + Settings settings = getSettings(); + RemotePersistentSettingsMetadata remoteObjectForUpload = new RemotePersistentSettingsMetadata(settings, METADATA_VERSION, clusterUUID, compressor, namedXContentRegistry); + assertThat(remoteObjectForUpload.clusterUUID(), is(clusterUUID)); + + RemotePersistentSettingsMetadata remoteObjectForDownload = new RemotePersistentSettingsMetadata(TEST_BLOB_NAME, clusterUUID, compressor, + namedXContentRegistry); + assertThat(remoteObjectForDownload.clusterUUID(), is(clusterUUID)); + } + + public void testFullBlobName() { + Settings settings = getSettings(); + RemotePersistentSettingsMetadata remoteObjectForUpload = new RemotePersistentSettingsMetadata(settings, METADATA_VERSION, clusterUUID, compressor, namedXContentRegistry); + assertThat(remoteObjectForUpload.getFullBlobName(), nullValue()); + + RemotePersistentSettingsMetadata remoteObjectForDownload = new RemotePersistentSettingsMetadata(TEST_BLOB_NAME, clusterUUID, compressor, + namedXContentRegistry); + assertThat(remoteObjectForDownload.getFullBlobName(), is(TEST_BLOB_NAME)); + } + + public void testBlobFileName() { + Settings settings = getSettings(); + RemotePersistentSettingsMetadata remoteObjectForUpload = new RemotePersistentSettingsMetadata(settings, METADATA_VERSION, clusterUUID, compressor, namedXContentRegistry); + assertThat(remoteObjectForUpload.getBlobFileName(), nullValue()); + + RemotePersistentSettingsMetadata remoteObjectForDownload = new RemotePersistentSettingsMetadata(TEST_BLOB_NAME, clusterUUID, compressor, + namedXContentRegistry); + assertThat(remoteObjectForDownload.getBlobFileName(), is(TEST_BLOB_FILE_NAME)); + } + + public void testBlobPathTokens() { + String uploadedFile = "user/local/opensearch/settings"; + RemotePersistentSettingsMetadata remoteObjectForDownload = new RemotePersistentSettingsMetadata(uploadedFile, clusterUUID, compressor, namedXContentRegistry); + assertThat(remoteObjectForDownload.getBlobPathTokens(), is(new String[]{"user", "local", "opensearch", "settings"})); + } + + public void testBlobPathParameters() { + Settings settings = getSettings(); + RemotePersistentSettingsMetadata remoteObjectForUpload = new RemotePersistentSettingsMetadata(settings, METADATA_VERSION, clusterUUID, compressor, namedXContentRegistry); + BlobPathParameters params = remoteObjectForUpload.getBlobPathParameters(); + assertThat(params.getPathTokens(), is(List.of(RemoteClusterStateUtils.GLOBAL_METADATA_PATH_TOKEN))); + assertThat(params.getFilePrefix(), is(RemotePersistentSettingsMetadata.SETTING_METADATA)); + } + + public void testGenerateBlobFileName() { + Settings settings = getSettings(); + RemotePersistentSettingsMetadata remoteObjectForUpload = new RemotePersistentSettingsMetadata(settings, METADATA_VERSION, clusterUUID, compressor, namedXContentRegistry); + String blobFileName = remoteObjectForUpload.generateBlobFileName(); + String[] nameTokens = blobFileName.split(RemoteClusterStateUtils.DELIMITER); + assertThat(nameTokens[0], is(RemotePersistentSettingsMetadata.SETTING_METADATA)); + assertThat(RemoteStoreUtils.invertLong(nameTokens[1]), is(METADATA_VERSION)); + assertThat(RemoteStoreUtils.invertLong(nameTokens[2]), lessThanOrEqualTo(System.currentTimeMillis())); + assertThat(nameTokens[3], is(String.valueOf(GLOBAL_METADATA_CURRENT_CODEC_VERSION))); + + } + + public void testGetUploadedMetadata() throws IOException { + Settings settings = getSettings(); + RemotePersistentSettingsMetadata remoteObjectForUpload = new RemotePersistentSettingsMetadata(settings, METADATA_VERSION, clusterUUID, compressor, namedXContentRegistry); + assertThrows(AssertionError.class, remoteObjectForUpload::getUploadedMetadata); + + try (InputStream inputStream = remoteObjectForUpload.serialize()) { + remoteObjectForUpload.setFullBlobName(new BlobPath().add(TEST_BLOB_PATH)); + UploadedMetadata uploadedMetadata = remoteObjectForUpload.getUploadedMetadata(); + assertThat(uploadedMetadata.getComponent(), is(RemotePersistentSettingsMetadata.SETTING_METADATA)); + assertThat(uploadedMetadata.getUploadedFilename(), is(remoteObjectForUpload.getFullBlobName())); + } + } + + public void testSerDe() throws IOException { + Settings settings = getSettings(); + RemotePersistentSettingsMetadata remoteObjectForUpload = new RemotePersistentSettingsMetadata(settings, METADATA_VERSION, clusterUUID, compressor, namedXContentRegistry); + try (InputStream inputStream = remoteObjectForUpload.serialize()) { + remoteObjectForUpload.setFullBlobName(BlobPath.cleanPath()); + assertThat(inputStream.available(), greaterThan(0)); + Settings readsettings = remoteObjectForUpload.deserialize(inputStream); + assertThat(readsettings, is(settings)); + } + } + + private Settings getSettings() { + return Settings.builder() + .put("random_index_setting_" + randomAlphaOfLength(3), randomAlphaOfLength(5)) + .build(); + } +} diff --git a/server/src/test/java/org/opensearch/gateway/remote/model/RemoteTemplatesMetadataTests.java b/server/src/test/java/org/opensearch/gateway/remote/model/RemoteTemplatesMetadataTests.java new file mode 100644 index 0000000000000..6ad99d5f9e8c3 --- /dev/null +++ b/server/src/test/java/org/opensearch/gateway/remote/model/RemoteTemplatesMetadataTests.java @@ -0,0 +1,184 @@ +/* + * 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.gateway.remote.model; + +import static java.util.stream.Collectors.toList; +import static org.hamcrest.Matchers.greaterThan; +import static org.hamcrest.Matchers.is; +import static org.hamcrest.Matchers.lessThanOrEqualTo; +import static org.hamcrest.Matchers.nullValue; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; +import static org.opensearch.gateway.remote.RemoteGlobalMetadataManager.GLOBAL_METADATA_CURRENT_CODEC_VERSION; + +import java.io.IOException; +import java.io.InputStream; +import java.util.Arrays; +import java.util.List; +import java.util.function.Function; +import java.util.stream.Stream; +import org.junit.After; +import org.junit.Before; +import org.opensearch.cluster.ClusterModule; +import org.opensearch.cluster.metadata.IndexTemplateMetadata; +import org.opensearch.cluster.metadata.TemplatesMetadata; +import org.opensearch.common.blobstore.BlobPath; +import org.opensearch.common.compress.DeflateCompressor; +import org.opensearch.common.network.NetworkModule; +import org.opensearch.common.remote.BlobPathParameters; +import org.opensearch.common.settings.ClusterSettings; +import org.opensearch.common.settings.Settings; +import org.opensearch.core.compress.Compressor; +import org.opensearch.core.compress.NoneCompressor; +import org.opensearch.core.xcontent.NamedXContentRegistry; +import org.opensearch.gateway.remote.ClusterMetadataManifest.UploadedMetadata; +import org.opensearch.gateway.remote.RemoteClusterStateUtils; +import org.opensearch.index.remote.RemoteStoreUtils; +import org.opensearch.index.translog.transfer.BlobStoreTransferService; +import org.opensearch.indices.IndicesModule; +import org.opensearch.repositories.blobstore.BlobStoreRepository; +import org.opensearch.test.OpenSearchTestCase; +import org.opensearch.threadpool.TestThreadPool; +import org.opensearch.threadpool.ThreadPool; + +public class RemoteTemplatesMetadataTests extends OpenSearchTestCase { + private static final String TEST_BLOB_NAME = "/test-path/test-blob-name"; + private static final String TEST_BLOB_PATH = "test-path"; + private static final String TEST_BLOB_FILE_NAME = "test-blob-name"; + private static final long METADATA_VERSION = 3L; + private String clusterUUID; + private BlobStoreTransferService blobStoreTransferService; + private BlobStoreRepository blobStoreRepository; + private String clusterName; + private ClusterSettings clusterSettings; + private Compressor compressor; + private NamedXContentRegistry namedXContentRegistry; + private final ThreadPool threadPool = new TestThreadPool(getClass().getName()); + + @Before + public void setup() { + clusterSettings = new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); + this.clusterUUID = "test-cluster-uuid"; + this.blobStoreTransferService = mock(BlobStoreTransferService.class); + this.blobStoreRepository = mock(BlobStoreRepository.class); + BlobPath blobPath = new BlobPath().add("/path"); + when(blobStoreRepository.basePath()).thenReturn(blobPath); + when(blobStoreRepository.getCompressor()).thenReturn(new DeflateCompressor()); + compressor = new NoneCompressor(); + namedXContentRegistry = new NamedXContentRegistry( + Stream.of( + NetworkModule.getNamedXContents().stream(), + IndicesModule.getNamedXContents().stream(), + ClusterModule.getNamedXWriteables().stream() + ).flatMap(Function.identity()).collect(toList()) + ); + this.clusterName = "test-cluster-name"; + } + + @After + public void tearDown() throws Exception { + super.tearDown(); + threadPool.shutdown(); + } + + public void testClusterUUID() { + TemplatesMetadata settings = getTemplatesMetadata(); + RemoteTemplatesMetadata remoteObjectForUpload = new RemoteTemplatesMetadata(settings, METADATA_VERSION, clusterUUID, compressor, namedXContentRegistry); + assertThat(remoteObjectForUpload.clusterUUID(), is(clusterUUID)); + + RemoteTemplatesMetadata remoteObjectForDownload = new RemoteTemplatesMetadata(TEST_BLOB_NAME, clusterUUID, compressor, + namedXContentRegistry); + assertThat(remoteObjectForDownload.clusterUUID(), is(clusterUUID)); + } + + public void testFullBlobName() { + TemplatesMetadata settings = getTemplatesMetadata(); + RemoteTemplatesMetadata remoteObjectForUpload = new RemoteTemplatesMetadata(settings, METADATA_VERSION, clusterUUID, compressor, namedXContentRegistry); + assertThat(remoteObjectForUpload.getFullBlobName(), nullValue()); + + RemoteTemplatesMetadata remoteObjectForDownload = new RemoteTemplatesMetadata(TEST_BLOB_NAME, clusterUUID, compressor, + namedXContentRegistry); + assertThat(remoteObjectForDownload.getFullBlobName(), is(TEST_BLOB_NAME)); + } + + public void testBlobFileName() { + TemplatesMetadata settings = getTemplatesMetadata(); + RemoteTemplatesMetadata remoteObjectForUpload = new RemoteTemplatesMetadata(settings, METADATA_VERSION, clusterUUID, compressor, namedXContentRegistry); + assertThat(remoteObjectForUpload.getBlobFileName(), nullValue()); + + RemoteTemplatesMetadata remoteObjectForDownload = new RemoteTemplatesMetadata(TEST_BLOB_NAME, clusterUUID, compressor, + namedXContentRegistry); + assertThat(remoteObjectForDownload.getBlobFileName(), is(TEST_BLOB_FILE_NAME)); + } + + public void testBlobPathTokens() { + String uploadedFile = "user/local/opensearch/settings"; + RemoteTemplatesMetadata remoteObjectForDownload = new RemoteTemplatesMetadata(uploadedFile, clusterUUID, compressor, namedXContentRegistry); + assertThat(remoteObjectForDownload.getBlobPathTokens(), is(new String[]{"user", "local", "opensearch", "settings"})); + } + + public void testBlobPathParameters() { + TemplatesMetadata settings = getTemplatesMetadata(); + RemoteTemplatesMetadata remoteObjectForUpload = new RemoteTemplatesMetadata(settings, METADATA_VERSION, clusterUUID, compressor, namedXContentRegistry); + BlobPathParameters params = remoteObjectForUpload.getBlobPathParameters(); + assertThat(params.getPathTokens(), is(List.of(RemoteClusterStateUtils.GLOBAL_METADATA_PATH_TOKEN))); + assertThat(params.getFilePrefix(), is(RemoteTemplatesMetadata.TEMPLATES_METADATA)); + } + + public void testGenerateBlobFileName() { + TemplatesMetadata settings = getTemplatesMetadata(); + RemoteTemplatesMetadata remoteObjectForUpload = new RemoteTemplatesMetadata(settings, METADATA_VERSION, clusterUUID, compressor, namedXContentRegistry); + String blobFileName = remoteObjectForUpload.generateBlobFileName(); + String[] nameTokens = blobFileName.split(RemoteClusterStateUtils.DELIMITER); + assertThat(nameTokens[0], is(RemoteTemplatesMetadata.TEMPLATES_METADATA)); + assertThat(RemoteStoreUtils.invertLong(nameTokens[1]), is(METADATA_VERSION)); + assertThat(RemoteStoreUtils.invertLong(nameTokens[2]), lessThanOrEqualTo(System.currentTimeMillis())); + assertThat(nameTokens[3], is(String.valueOf(GLOBAL_METADATA_CURRENT_CODEC_VERSION))); + + } + + public void testGetUploadedMetadata() throws IOException { + TemplatesMetadata settings = getTemplatesMetadata(); + RemoteTemplatesMetadata remoteObjectForUpload = new RemoteTemplatesMetadata(settings, METADATA_VERSION, clusterUUID, compressor, namedXContentRegistry); + assertThrows(AssertionError.class, remoteObjectForUpload::getUploadedMetadata); + + try (InputStream inputStream = remoteObjectForUpload.serialize()) { + remoteObjectForUpload.setFullBlobName(new BlobPath().add(TEST_BLOB_PATH)); + UploadedMetadata uploadedMetadata = remoteObjectForUpload.getUploadedMetadata(); + assertThat(uploadedMetadata.getComponent(), is(RemoteTemplatesMetadata.TEMPLATES_METADATA)); + assertThat(uploadedMetadata.getUploadedFilename(), is(remoteObjectForUpload.getFullBlobName())); + } + } + + public void testSerDe() throws IOException { + TemplatesMetadata settings = getTemplatesMetadata(); + RemoteTemplatesMetadata remoteObjectForUpload = new RemoteTemplatesMetadata(settings, METADATA_VERSION, clusterUUID, compressor, namedXContentRegistry); + try (InputStream inputStream = remoteObjectForUpload.serialize()) { + remoteObjectForUpload.setFullBlobName(BlobPath.cleanPath()); + assertThat(inputStream.available(), greaterThan(0)); + TemplatesMetadata readsettings = remoteObjectForUpload.deserialize(inputStream); + assertThat(readsettings, is(settings)); + } + } + + private TemplatesMetadata getTemplatesMetadata() { + return TemplatesMetadata.builder() + .put( + IndexTemplateMetadata.builder("template" + randomAlphaOfLength(3)) + .patterns(Arrays.asList("bar-*", "foo-*")) + .settings( + Settings.builder() + .put("index.random_index_setting_" + randomAlphaOfLength(3), randomAlphaOfLength(5)) + .build() + ) + .build() + ) + .build(); + } +} diff --git a/server/src/test/java/org/opensearch/gateway/remote/routingtable/IndexRoutingTableHeaderTests.java b/server/src/test/java/org/opensearch/gateway/remote/routingtable/IndexRoutingTableHeaderTests.java new file mode 100644 index 0000000000000..0f42508615a26 --- /dev/null +++ b/server/src/test/java/org/opensearch/gateway/remote/routingtable/IndexRoutingTableHeaderTests.java @@ -0,0 +1,29 @@ +/* + * 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.gateway.remote.routingtable; + +import org.opensearch.common.io.stream.BytesStreamOutput; +import org.opensearch.core.common.io.stream.BytesStreamInput; +import org.opensearch.test.OpenSearchTestCase; + +import java.io.IOException; + +public class IndexRoutingTableHeaderTests extends OpenSearchTestCase { + + public void testIndexRoutingTableHeader() throws IOException { + IndexRoutingTableHeader header = new IndexRoutingTableHeader("dummyIndex"); + BytesStreamOutput out = new BytesStreamOutput(); + header.write(out); + + BytesStreamInput in = new BytesStreamInput(out.bytes().toBytesRef().bytes); + IndexRoutingTableHeader headerRead = IndexRoutingTableHeader.read(in); + assertEquals("dummyIndex", headerRead.getIndexName()); + } + +} diff --git a/server/src/test/java/org/opensearch/gateway/remote/routingtable/IndexRoutingTableInputStreamTests.java b/server/src/test/java/org/opensearch/gateway/remote/routingtable/IndexRoutingTableInputStreamTests.java new file mode 100644 index 0000000000000..a6e2e4bda00e8 --- /dev/null +++ b/server/src/test/java/org/opensearch/gateway/remote/routingtable/IndexRoutingTableInputStreamTests.java @@ -0,0 +1,73 @@ +/* + * 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.gateway.remote.routingtable; + +import org.opensearch.Version; +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.ShardRoutingState; +import org.opensearch.test.OpenSearchTestCase; + +import java.io.IOException; +import java.io.InputStream; +import java.util.concurrent.atomic.AtomicInteger; + +public class IndexRoutingTableInputStreamTests extends OpenSearchTestCase { + + public void testRoutingTableInputStream() { + Metadata metadata = Metadata.builder() + .put(IndexMetadata.builder("test").settings(settings(Version.CURRENT)).numberOfShards(1).numberOfReplicas(1)) + .build(); + + RoutingTable initialRoutingTable = RoutingTable.builder().addAsNew(metadata.index("test")).build(); + + initialRoutingTable.getIndicesRouting().values().forEach(indexShardRoutingTables -> { + try { + InputStream indexRoutingStream = new IndexRoutingTableInputStream(indexShardRoutingTables); + + IndexRoutingTableInputStreamReader reader = new IndexRoutingTableInputStreamReader(indexRoutingStream); + IndexRoutingTable indexRoutingTable = reader.readIndexRoutingTable(metadata.index("test").getIndex()); + + assertEquals(1, indexRoutingTable.getShards().size()); + assertEquals(indexRoutingTable.getIndex(), metadata.index("test").getIndex()); + assertEquals(indexRoutingTable.shardsWithState(ShardRoutingState.UNASSIGNED).size(), 2); + } catch (IOException e) { + throw new RuntimeException(e); + } + }); + } + + public void testRoutingTableInputStreamWithInvalidIndex() { + Metadata metadata = Metadata.builder() + .put(IndexMetadata.builder("test").settings(settings(Version.CURRENT)).numberOfShards(1).numberOfReplicas(1)) + .put(IndexMetadata.builder("invalid-index").settings(settings(Version.CURRENT)).numberOfShards(1).numberOfReplicas(1)) + .build(); + + RoutingTable initialRoutingTable = RoutingTable.builder().addAsNew(metadata.index("test")).build(); + AtomicInteger assertionError = new AtomicInteger(); + initialRoutingTable.getIndicesRouting().values().forEach(indexShardRoutingTables -> { + try { + InputStream indexRoutingStream = new IndexRoutingTableInputStream(indexShardRoutingTables); + + IndexRoutingTableInputStreamReader reader = new IndexRoutingTableInputStreamReader(indexRoutingStream); + reader.readIndexRoutingTable(metadata.index("invalid-index").getIndex()); + + } catch (AssertionError e) { + assertionError.getAndIncrement(); + } catch (IOException e) { + throw new RuntimeException(e); + } + }); + + assertEquals(1, assertionError.get()); + } + +} diff --git a/server/src/test/java/org/opensearch/index/remote/RemoteIndexPathUploaderTests.java b/server/src/test/java/org/opensearch/index/remote/RemoteIndexPathUploaderTests.java index e539b382a5f3b..214d4baed2a02 100644 --- a/server/src/test/java/org/opensearch/index/remote/RemoteIndexPathUploaderTests.java +++ b/server/src/test/java/org/opensearch/index/remote/RemoteIndexPathUploaderTests.java @@ -46,6 +46,7 @@ import org.mockito.Mockito; +import static org.opensearch.gateway.remote.RemoteIndexMetadataManager.INDEX_METADATA_UPLOAD_TIMEOUT_SETTING; import static org.opensearch.index.remote.RemoteStoreEnums.PathType.FIXED; import static org.opensearch.index.remote.RemoteStoreEnums.PathType.HASHED_INFIX; import static org.opensearch.index.remote.RemoteStoreEnums.PathType.HASHED_PREFIX; @@ -276,7 +277,7 @@ public void testInterceptWithLatchAwaitTimeout() throws IOException { Settings settings = Settings.builder() .put(this.settings) - .put(RemoteClusterStateService.INDEX_METADATA_UPLOAD_TIMEOUT_SETTING.getKey(), TimeValue.ZERO) + .put(INDEX_METADATA_UPLOAD_TIMEOUT_SETTING.getKey(), TimeValue.ZERO) .build(); clusterSettings.applySettings(settings); SetOnce exceptionSetOnce = new SetOnce<>(); @@ -306,7 +307,7 @@ public void testInterceptWithInterruptedExceptionDuringLatchAwait() throws Excep remoteIndexPathUploader.start(); Settings settings = Settings.builder() .put(this.settings) - .put(RemoteClusterStateService.INDEX_METADATA_UPLOAD_TIMEOUT_SETTING.getKey(), TimeValue.timeValueSeconds(1)) + .put(INDEX_METADATA_UPLOAD_TIMEOUT_SETTING.getKey(), TimeValue.timeValueSeconds(1)) .build(); clusterSettings.applySettings(settings); SetOnce exceptionSetOnce = new SetOnce<>(); diff --git a/server/src/test/java/org/opensearch/index/translog/transfer/BlobStoreTransferServiceTests.java b/server/src/test/java/org/opensearch/index/translog/transfer/BlobStoreTransferServiceTests.java index 3ed90c0837663..691c1c49f0874 100644 --- a/server/src/test/java/org/opensearch/index/translog/transfer/BlobStoreTransferServiceTests.java +++ b/server/src/test/java/org/opensearch/index/translog/transfer/BlobStoreTransferServiceTests.java @@ -8,19 +8,54 @@ package org.opensearch.index.translog.transfer; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.when; + +import java.io.IOException; +import java.io.InputStream; +import java.io.Serializable; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.StandardOpenOption; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import org.opensearch.Version; +import static org.opensearch.index.translog.transfer.TranslogTransferManager.CHECKPOINT_FILE_DATA_KEY; + +import java.io.ByteArrayInputStream; +import java.io.InputStream; +import java.nio.charset.StandardCharsets; +import java.util.Base64; +import java.util.Map; import org.opensearch.action.LatchedActionListener; +import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.cluster.metadata.RepositoryMetadata; import org.opensearch.cluster.service.ClusterService; +import org.opensearch.common.blobstore.AsyncMultiStreamBlobContainer; +import org.opensearch.common.blobstore.BlobContainer; +import org.opensearch.common.blobstore.BlobPath; +import org.opensearch.common.blobstore.BlobStore; +import org.opensearch.common.blobstore.fs.FsBlobContainer; +import org.opensearch.common.blobstore.fs.FsBlobStore; +import org.opensearch.common.blobstore.stream.read.ReadContext; +import org.opensearch.common.blobstore.stream.write.WriteContext; import org.opensearch.common.blobstore.stream.write.WritePriority; import org.opensearch.common.settings.ClusterSettings; import org.opensearch.common.settings.Settings; import org.opensearch.core.action.ActionListener; +import org.opensearch.core.compress.NoneCompressor; +import org.opensearch.core.index.Index; import org.opensearch.env.Environment; import org.opensearch.env.TestEnvironment; +import org.opensearch.gateway.remote.RemoteClusterStateUtils; import org.opensearch.indices.recovery.RecoverySettings; import org.opensearch.repositories.Repository; import org.opensearch.repositories.blobstore.BlobStoreRepository; import org.opensearch.repositories.blobstore.BlobStoreTestUtil; +import org.opensearch.repositories.blobstore.ChecksumBlobStoreFormat; import org.opensearch.repositories.fs.FsRepository; import org.opensearch.test.OpenSearchTestCase; import org.opensearch.threadpool.TestThreadPool; @@ -110,6 +145,64 @@ public void onFailure(Exception e) { assertTrue(succeeded.get()); } + public void testUploadBlobFromInputStreamSyncFSRepo() throws IOException, InterruptedException{ + TransferService transferService = new BlobStoreTransferService(repository.blobStore(), threadPool); + uploadBlobFromInputStream(transferService); + } + + public void testUploadBlobFromInputStreamAsyncFSRepo() throws IOException, InterruptedException{ + BlobStore blobStore = createTestBlobStore(); + MockAsyncFsContainer mockAsyncFsContainer = new MockAsyncFsContainer( + (FsBlobStore) blobStore, + BlobPath.cleanPath(), + null + ); + FsBlobStore fsBlobStore = mock(FsBlobStore.class); + when(fsBlobStore.blobContainer(any())).thenReturn(mockAsyncFsContainer); + + TransferService transferService = new BlobStoreTransferService(fsBlobStore, threadPool); + uploadBlobFromInputStream(transferService); + } + + private IndexMetadata getIndexMetadata() { + final Index index = new Index("test-index", "index-uuid"); + final Settings idxSettings = Settings.builder() + .put(IndexMetadata.SETTING_VERSION_CREATED, Version.CURRENT) + .put(IndexMetadata.SETTING_INDEX_UUID, index.getUUID()) + .build(); + return new IndexMetadata.Builder(index.getName()).settings(idxSettings) + .version(5L) + .numberOfShards(1) + .numberOfReplicas(0) + .build(); + } + + private void uploadBlobFromInputStream(TransferService transferService) throws IOException, InterruptedException { + TestClass testObject = new TestClass("field1", "value1"); + AtomicBoolean succeeded = new AtomicBoolean(false); + ChecksumBlobStoreFormat blobStoreFormat = new ChecksumBlobStoreFormat<>("coordination", "%s", IndexMetadata::fromXContent); + IndexMetadata indexMetadata = getIndexMetadata(); + try (InputStream inputStream = blobStoreFormat.serialize(indexMetadata, "index-metadata", new NoneCompressor(), RemoteClusterStateUtils.FORMAT_PARAMS).streamInput()) { + CountDownLatch latch = new CountDownLatch(1); + ActionListener listener = new LatchedActionListener<>(new ActionListener<>() { + @Override + public void onResponse(TestClass testObject) { + assert succeeded.compareAndSet(false, true); + assert testObject.name.equals("field1"); + } + + @Override + public void onFailure(Exception e) { + throw new AssertionError("Failed to perform uploadBlobAsync", e); + } + }, latch); + ActionListener completionListener = ActionListener.wrap(resp -> listener.onResponse(testObject), ex -> listener.onFailure(ex)); + transferService.uploadBlobAsync(inputStream, repository.basePath(), "test-object", WritePriority.URGENT, completionListener); + assertTrue(latch.await(1000, TimeUnit.MILLISECONDS)); + assertTrue(succeeded.get()); + } + } + @Override public void tearDown() throws Exception { super.tearDown(); @@ -141,6 +234,10 @@ protected void assertSnapshotOrGenericThread() { return repository; } + private BlobStore createTestBlobStore() throws IOException { + return new FsBlobStore(randomIntBetween(1, 8) * 1024, createTempDir(), false); + } + /** Create a {@link Environment} with random path.home and path.repo **/ private Environment createEnvironment() { Path home = createTempDir(); @@ -184,4 +281,49 @@ public void testBuildTransferFileMetadata_SmallInputStreamOptimization() throws assertEquals(expectedBase64String, metadata.get(CHECKPOINT_FILE_DATA_KEY)); } + private static class TestClass implements Serializable { + private TestClass(String name, String value) { + this.name = name; + this.value = value; + } + + private final String name; + private final String value; + + @Override + public String toString() { + return "TestClass{ name: " + name + ", value: " + value + " }"; + } + } + + private static class MockAsyncFsContainer extends FsBlobContainer implements AsyncMultiStreamBlobContainer { + + private BlobContainer delegate; + + public MockAsyncFsContainer(FsBlobStore blobStore, BlobPath blobPath, Path path) { + super(blobStore, blobPath, path); + delegate = blobStore.blobContainer(BlobPath.cleanPath()); + } + + @Override + public void asyncBlobUpload(WriteContext writeContext, ActionListener completionListener) throws IOException { + InputStream inputStream = writeContext.getStreamProvider(Integer.MAX_VALUE).provideStream(0).getInputStream(); + delegate.writeBlob(writeContext.getFileName(), inputStream, writeContext.getFileSize(), true); + completionListener.onResponse(null); + } + + @Override + public void readBlobAsync(String blobName, ActionListener listener) { + throw new RuntimeException("read not supported"); + } + + @Override + public boolean remoteIntegrityCheckSupported() { + return false; + } + + public BlobContainer getDelegate() { + return delegate; + } + } } diff --git a/server/src/test/java/org/opensearch/repositories/blobstore/ChecksumWritableBlobStoreFormatTests.java b/server/src/test/java/org/opensearch/repositories/blobstore/ChecksumWritableBlobStoreFormatTests.java new file mode 100644 index 0000000000000..68dda7bdc6dde --- /dev/null +++ b/server/src/test/java/org/opensearch/repositories/blobstore/ChecksumWritableBlobStoreFormatTests.java @@ -0,0 +1,48 @@ +/* + * 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.repositories.blobstore; + +import static org.hamcrest.Matchers.is; + +import java.io.IOException; +import org.junit.Before; +import org.opensearch.Version; +import org.opensearch.cluster.metadata.IndexMetadata; +import org.opensearch.common.settings.Settings; +import org.opensearch.core.common.bytes.BytesReference; +import org.opensearch.core.compress.CompressorRegistry; +import org.opensearch.core.index.Index; +import org.opensearch.test.OpenSearchTestCase; + +public class ChecksumWritableBlobStoreFormatTests extends OpenSearchTestCase { + private static final String TEST_BLOB_FILE_NAME = "test-blob-name"; + private static final long VERSION = 5L; + + private final ChecksumWritableBlobStoreFormat clusterBlocksFormat = new ChecksumWritableBlobStoreFormat<>("index-metadata", IndexMetadata::readFrom); + + public void testSerDe() throws IOException { + IndexMetadata indexMetadata = getIndexMetadata(); + BytesReference bytesReference = clusterBlocksFormat.serialize(indexMetadata, TEST_BLOB_FILE_NAME, CompressorRegistry.none()); + IndexMetadata readIndexMetadata = clusterBlocksFormat.deserialize(TEST_BLOB_FILE_NAME, IndexMetadata::readFrom, bytesReference); + assertThat(readIndexMetadata, is(indexMetadata)); + } + + private IndexMetadata getIndexMetadata() { + final Index index = new Index("test-index", "index-uuid"); + final Settings idxSettings = Settings.builder() + .put(IndexMetadata.SETTING_VERSION_CREATED, Version.CURRENT) + .put(IndexMetadata.SETTING_INDEX_UUID, index.getUUID()) + .build(); + return new IndexMetadata.Builder(index.getName()).settings(idxSettings) + .version(VERSION) + .numberOfShards(1) + .numberOfReplicas(0) + .build(); + } +} diff --git a/server/src/test/java/org/opensearch/snapshots/BlobStoreFormatTests.java b/server/src/test/java/org/opensearch/snapshots/BlobStoreFormatTests.java index c5f36fcc01983..8c64f9a3170b2 100644 --- a/server/src/test/java/org/opensearch/snapshots/BlobStoreFormatTests.java +++ b/server/src/test/java/org/opensearch/snapshots/BlobStoreFormatTests.java @@ -55,7 +55,7 @@ import org.opensearch.core.xcontent.ToXContentFragment; import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.core.xcontent.XContentParser; -import org.opensearch.index.translog.BufferedChecksumStreamOutput; +import org.opensearch.common.io.stream.BufferedChecksumStreamOutput; import org.opensearch.repositories.blobstore.ChecksumBlobStoreFormat; import org.opensearch.test.OpenSearchTestCase; diff --git a/server/src/test/java/org/opensearch/snapshots/SnapshotResiliencyTests.java b/server/src/test/java/org/opensearch/snapshots/SnapshotResiliencyTests.java index 5b39880930984..c199d3f25edb3 100644 --- a/server/src/test/java/org/opensearch/snapshots/SnapshotResiliencyTests.java +++ b/server/src/test/java/org/opensearch/snapshots/SnapshotResiliencyTests.java @@ -2559,7 +2559,8 @@ public void start(ClusterState initialState) { ElectionStrategy.DEFAULT_INSTANCE, () -> new StatusInfo(HEALTHY, "healthy-info"), persistedStateRegistry, - remoteStoreNodeService + remoteStoreNodeService, + null ); clusterManagerService.setClusterStatePublisher(coordinator); coordinator.start(); diff --git a/server/src/test/java/org/opensearch/snapshots/SnapshotsInProgressSerializationTests.java b/server/src/test/java/org/opensearch/snapshots/SnapshotsInProgressSerializationTests.java index 8fd1f44286094..b19d84815ded0 100644 --- a/server/src/test/java/org/opensearch/snapshots/SnapshotsInProgressSerializationTests.java +++ b/server/src/test/java/org/opensearch/snapshots/SnapshotsInProgressSerializationTests.java @@ -36,17 +36,27 @@ import org.opensearch.cluster.ClusterModule; import org.opensearch.cluster.ClusterState.Custom; import org.opensearch.cluster.Diff; +import org.opensearch.cluster.SnapshotDeletionsInProgress; import org.opensearch.cluster.SnapshotsInProgress; import org.opensearch.cluster.SnapshotsInProgress.Entry; import org.opensearch.cluster.SnapshotsInProgress.ShardState; import org.opensearch.cluster.SnapshotsInProgress.State; +import org.opensearch.cluster.metadata.Metadata; +import org.opensearch.cluster.node.DiscoveryNodes; import org.opensearch.common.UUIDs; import org.opensearch.common.io.stream.BytesStreamOutput; +import org.opensearch.common.xcontent.json.JsonXContent; +import org.opensearch.core.common.bytes.BytesReference; import org.opensearch.core.common.io.stream.NamedWriteableRegistry; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.Writeable; import org.opensearch.core.index.Index; import org.opensearch.core.index.shard.ShardId; +import org.opensearch.core.xcontent.MediaType; +import org.opensearch.core.xcontent.MediaTypeRegistry; +import org.opensearch.core.xcontent.ToXContent; +import org.opensearch.core.xcontent.XContentBuilder; +import org.opensearch.core.xcontent.XContentParser; import org.opensearch.repositories.IndexId; import org.opensearch.test.AbstractDiffableWireSerializationTestCase; import org.opensearch.test.VersionUtils; @@ -60,6 +70,9 @@ import java.util.Map; import java.util.stream.Collectors; +import static java.lang.Math.abs; +import static java.util.Collections.singletonMap; +import static org.opensearch.cluster.metadata.Metadata.CONTEXT_MODE_GATEWAY; import static org.opensearch.test.VersionUtils.randomVersion; public class SnapshotsInProgressSerializationTests extends AbstractDiffableWireSerializationTestCase { @@ -84,7 +97,7 @@ private Entry randomSnapshot() { for (int i = 0; i < numberOfIndices; i++) { indices.add(new IndexId(randomAlphaOfLength(10), randomAlphaOfLength(10))); } - long startTime = randomLong(); + long startTime = abs(randomLong()); long repositoryStateId = randomLong(); Map builder = new HashMap<>(); final List esIndices = indices.stream() @@ -183,6 +196,41 @@ protected Custom mutateInstance(Custom instance) { return SnapshotsInProgress.of(entries); } + public void testToXContent() throws IOException { + SnapshotsInProgress sip = SnapshotsInProgress.of(List.of(randomSnapshot(), randomSnapshot())); + boolean humanReadable = false; + XContentBuilder builder = JsonXContent.contentBuilder().prettyPrint(); + final MediaType mediaType = MediaTypeRegistry.JSON; + BytesReference originalBytes = toShuffledXContent( + sip, + mediaType, + new ToXContent.MapParams(singletonMap(Metadata.CONTEXT_MODE_PARAM, CONTEXT_MODE_GATEWAY)), + humanReadable + ); + try (XContentParser parser = createParser(mediaType.xContent(), originalBytes)) { + SnapshotsInProgress parsed = SnapshotsInProgress.fromXContent(parser); + assertEquals(sip, parsed); + } + } + + public void testToXContent_deletion() throws IOException { + SnapshotDeletionsInProgress.Entry entry = new SnapshotDeletionsInProgress.Entry(List.of(new SnapshotId("name1", "uuid1")), "repo", 10000000L, 10000L, SnapshotDeletionsInProgress.State.WAITING); + SnapshotDeletionsInProgress sdip = SnapshotDeletionsInProgress.of(List.of(entry)); + boolean humanReadable = false; + XContentBuilder builder = JsonXContent.contentBuilder().prettyPrint(); + final MediaType mediaType = MediaTypeRegistry.JSON; + BytesReference originalBytes = toShuffledXContent( + sdip, + mediaType, + new ToXContent.MapParams(singletonMap(Metadata.CONTEXT_MODE_PARAM, CONTEXT_MODE_GATEWAY)), + humanReadable + ); + try (XContentParser parser = createParser(mediaType.xContent(), originalBytes)) { + SnapshotDeletionsInProgress parsed = SnapshotDeletionsInProgress.fromXContent(parser); + assertEquals(sdip, parsed); + } + } + public void testSerDeRemoteStoreIndexShallowCopy() throws IOException { SnapshotsInProgress.Entry entry = new SnapshotsInProgress.Entry( new Snapshot(randomName("repo"), new SnapshotId(randomName("snap"), UUIDs.randomBase64UUID())), @@ -191,7 +239,7 @@ public void testSerDeRemoteStoreIndexShallowCopy() throws IOException { SnapshotsInProgressSerializationTests.randomState(Map.of()), Collections.emptyList(), Collections.emptyList(), - Math.abs(randomLong()), + abs(randomLong()), randomIntBetween(0, 1000), Map.of(), null, diff --git a/test/framework/src/main/java/org/opensearch/cluster/coordination/AbstractCoordinatorTestCase.java b/test/framework/src/main/java/org/opensearch/cluster/coordination/AbstractCoordinatorTestCase.java index 3ae737bf63923..65058b41d4bb3 100644 --- a/test/framework/src/main/java/org/opensearch/cluster/coordination/AbstractCoordinatorTestCase.java +++ b/test/framework/src/main/java/org/opensearch/cluster/coordination/AbstractCoordinatorTestCase.java @@ -1183,7 +1183,8 @@ protected Optional getDisruptableMockTransport(Transpo getElectionStrategy(), nodeHealthService, persistedStateRegistry, - remoteStoreNodeService + remoteStoreNodeService, + null ); clusterManagerService.setClusterStatePublisher(coordinator); final GatewayService gatewayService = new GatewayService(