diff --git a/CHANGELOG.md b/CHANGELOG.md index f17741c232b1a..93a98de2c5af8 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -27,6 +27,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Add allowlist setting for ingest-geoip and ingest-useragent ([#15325](https://github.com/opensearch-project/OpenSearch/pull/15325)) - Adding access to noSubMatches and noOverlappingMatches in Hyphenation ([#13895](https://github.com/opensearch-project/OpenSearch/pull/13895)) - Add support for index level max slice count setting for concurrent segment search ([#15336](https://github.com/opensearch-project/OpenSearch/pull/15336)) +- Support cancellation for cat shards and node stats API.([#13966](https://github.com/opensearch-project/OpenSearch/pull/13966)) - [Streaming Indexing] Introduce bulk HTTP API streaming flavor ([#15381](https://github.com/opensearch-project/OpenSearch/pull/15381)) - Add support for centralize snapshot creation with pinned timestamp ([#15124](https://github.com/opensearch-project/OpenSearch/pull/15124)) - Add concurrent search support for Derived Fields ([#15326](https://github.com/opensearch-project/OpenSearch/pull/15326)) @@ -43,6 +44,9 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - [Workload Management] Add query group level failure tracking ([#15227](https://github.com/opensearch-project/OpenSearch/pull/15527)) - Add support to upload snapshot shard blobs with hashed prefix ([#15426](https://github.com/opensearch-project/OpenSearch/pull/15426)) - [Remote Publication] Add remote download stats ([#15291](https://github.com/opensearch-project/OpenSearch/pull/15291))) +- Add support for comma-separated list of index names to be used with Snapshot Status API ([#15409](https://github.com/opensearch-project/OpenSearch/pull/15409)) +- Add prefix support to hashed prefix & infix path types on remote store ([#15557](https://github.com/opensearch-project/OpenSearch/pull/15557)) +- [Remote Publication] Added checksum validation for cluster state behind a cluster setting ([#15218](https://github.com/opensearch-project/OpenSearch/pull/15218)) ### Dependencies - Bump `netty` from 4.1.111.Final to 4.1.112.Final ([#15081](https://github.com/opensearch-project/OpenSearch/pull/15081)) diff --git a/client/rest-high-level/src/test/java/org/opensearch/client/SnapshotRequestConvertersTests.java b/client/rest-high-level/src/test/java/org/opensearch/client/SnapshotRequestConvertersTests.java index af178ad2a5d47..e9ca03262bc4f 100644 --- a/client/rest-high-level/src/test/java/org/opensearch/client/SnapshotRequestConvertersTests.java +++ b/client/rest-high-level/src/test/java/org/opensearch/client/SnapshotRequestConvertersTests.java @@ -230,6 +230,7 @@ public void testSnapshotsStatus() { Map expectedParams = new HashMap<>(); String repository = RequestConvertersTests.randomIndicesNames(1, 1)[0]; String[] snapshots = RequestConvertersTests.randomIndicesNames(1, 5); + String[] indices = RequestConvertersTests.randomIndicesNames(1, 5); StringBuilder snapshotNames = new StringBuilder(snapshots[0]); for (int idx = 1; idx < snapshots.length; idx++) { snapshotNames.append(",").append(snapshots[idx]); @@ -237,7 +238,7 @@ public void testSnapshotsStatus() { boolean ignoreUnavailable = randomBoolean(); String endpoint = "/_snapshot/" + repository + "/" + snapshotNames.toString() + "/_status"; - SnapshotsStatusRequest snapshotsStatusRequest = new SnapshotsStatusRequest(repository, snapshots); + SnapshotsStatusRequest snapshotsStatusRequest = new SnapshotsStatusRequest(repository, snapshots, indices); RequestConvertersTests.setRandomClusterManagerTimeout(snapshotsStatusRequest, expectedParams); snapshotsStatusRequest.ignoreUnavailable(ignoreUnavailable); expectedParams.put("ignore_unavailable", Boolean.toString(ignoreUnavailable)); diff --git a/libs/core/src/main/java/org/opensearch/core/common/io/stream/BufferedChecksumStreamOutput.java b/libs/core/src/main/java/org/opensearch/core/common/io/stream/BufferedChecksumStreamOutput.java index 422f956c0cd47..18bd53dc5d77c 100644 --- a/libs/core/src/main/java/org/opensearch/core/common/io/stream/BufferedChecksumStreamOutput.java +++ b/libs/core/src/main/java/org/opensearch/core/common/io/stream/BufferedChecksumStreamOutput.java @@ -33,9 +33,18 @@ package org.opensearch.core.common.io.stream; import org.apache.lucene.store.BufferedChecksum; +import org.opensearch.common.Nullable; import org.opensearch.common.annotation.PublicApi; import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.TreeMap; +import java.util.stream.Collectors; import java.util.zip.CRC32; import java.util.zip.Checksum; @@ -90,4 +99,75 @@ public void reset() throws IOException { public void resetDigest() { digest.reset(); } + + @Override + public void writeMap(@Nullable Map map) throws IOException { + Map newMap = new TreeMap<>(map); + writeGenericValue(newMap); + } + + @Override + public void writeMap(Map map, final Writeable.Writer keyWriter, final Writeable.Writer valueWriter) + throws IOException { + writeVInt(map.size()); + map.keySet().stream().sorted().forEachOrdered(key -> { + try { + keyWriter.write(this, key); + valueWriter.write(this, map.get(key)); + } catch (IOException e) { + throw new RuntimeException("Failed to write map values.", e); + } + }); + } + + public void writeMapValues(Map map, final Writeable.Writer valueWriter) throws IOException { + writeVInt(map.size()); + map.keySet().stream().sorted().forEachOrdered(key -> { + try { + valueWriter.write(this, map.get(key)); + } catch (IOException e) { + throw new RuntimeException("Failed to write map values.", e); + } + }); + } + + @Override + public void writeStringArray(String[] array) throws IOException { + String[] copyArray = Arrays.copyOf(array, array.length); + Arrays.sort(copyArray); + super.writeStringArray(copyArray); + } + + @Override + public void writeVLongArray(long[] values) throws IOException { + long[] copyValues = Arrays.copyOf(values, values.length); + Arrays.sort(copyValues); + super.writeVLongArray(copyValues); + } + + @Override + public void writeCollection(final Collection collection) throws IOException { + List sortedList = collection.stream().sorted().collect(Collectors.toList()); + super.writeCollection(sortedList, (o, v) -> v.writeTo(o)); + } + + @Override + public void writeStringCollection(final Collection collection) throws IOException { + List listCollection = new ArrayList<>(collection); + Collections.sort(listCollection); + writeCollection(listCollection, StreamOutput::writeString); + } + + @Override + public void writeOptionalStringCollection(final Collection collection) throws IOException { + if (collection != null) { + List listCollection = new ArrayList<>(collection); + Collections.sort(listCollection); + writeBoolean(true); + writeCollection(listCollection, StreamOutput::writeString); + } else { + writeBoolean(false); + } + } + } diff --git a/libs/core/src/main/java/org/opensearch/core/common/io/stream/StreamOutput.java b/libs/core/src/main/java/org/opensearch/core/common/io/stream/StreamOutput.java index b7599265aece3..cac8ddc8f94e3 100644 --- a/libs/core/src/main/java/org/opensearch/core/common/io/stream/StreamOutput.java +++ b/libs/core/src/main/java/org/opensearch/core/common/io/stream/StreamOutput.java @@ -633,7 +633,7 @@ public final void writeMapOfLists(final Map> map, final Writer * @param keyWriter The key writer * @param valueWriter The value writer */ - public final void writeMap(final Map map, final Writer keyWriter, final Writer valueWriter) throws IOException { + public void writeMap(final Map map, final Writer keyWriter, final Writer valueWriter) throws IOException { writeVInt(map.size()); for (final Map.Entry entry : map.entrySet()) { keyWriter.write(this, entry.getKey()); @@ -969,9 +969,13 @@ public void writeOptionalArray(@Nullable T[] array) throws } public void writeOptionalWriteable(@Nullable Writeable writeable) throws IOException { + writeOptionalWriteable((out, writable) -> writable.writeTo(out), writeable); + } + + public void writeOptionalWriteable(final Writer writer, @Nullable T writeable) throws IOException { if (writeable != null) { writeBoolean(true); - writeable.writeTo(this); + writer.write(this, writeable); } else { writeBoolean(false); } diff --git a/libs/core/src/main/java/org/opensearch/core/common/io/stream/VerifiableWriteable.java b/libs/core/src/main/java/org/opensearch/core/common/io/stream/VerifiableWriteable.java new file mode 100644 index 0000000000000..6683606e27195 --- /dev/null +++ b/libs/core/src/main/java/org/opensearch/core/common/io/stream/VerifiableWriteable.java @@ -0,0 +1,20 @@ +/* + * 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.core.common.io.stream; + +import java.io.IOException; + +/** + * Provides a method for serialization which will give ordered stream, creating same byte array on every invocation. + * This should be invoked with a stream that provides ordered serialization. + */ +public interface VerifiableWriteable extends Writeable { + + void writeVerifiableTo(BufferedChecksumStreamOutput out) throws IOException; +} diff --git a/qa/repository-multi-version/src/test/java/org/opensearch/upgrades/MultiVersionRepositoryAccessIT.java b/qa/repository-multi-version/src/test/java/org/opensearch/upgrades/MultiVersionRepositoryAccessIT.java index 7c27f2ea71942..5c6dcdd142995 100644 --- a/qa/repository-multi-version/src/test/java/org/opensearch/upgrades/MultiVersionRepositoryAccessIT.java +++ b/qa/repository-multi-version/src/test/java/org/opensearch/upgrades/MultiVersionRepositoryAccessIT.java @@ -32,6 +32,7 @@ package org.opensearch.upgrades; +import com.sun.jna.StringArray; import org.opensearch.action.admin.cluster.repositories.put.PutRepositoryRequest; import org.opensearch.action.admin.cluster.snapshots.delete.DeleteSnapshotRequest; import org.opensearch.action.admin.cluster.snapshots.status.SnapshotStatus; @@ -44,6 +45,7 @@ import org.opensearch.client.RestClient; import org.opensearch.client.RestHighLevelClient; import org.opensearch.common.settings.Settings; +import org.opensearch.core.common.Strings; import org.opensearch.core.xcontent.DeprecationHandler; import org.opensearch.core.xcontent.XContentParser; import org.opensearch.common.xcontent.json.JsonXContent; @@ -141,14 +143,14 @@ public void testCreateAndRestoreSnapshot() throws IOException { case STEP2_NEW_CLUSTER: case STEP4_NEW_CLUSTER: assertSnapshotStatusSuccessful(client, repoName, - snapshots.stream().map(sn -> (String) sn.get("snapshot")).toArray(String[]::new)); + snapshots.stream().map(sn -> (String) sn.get("snapshot")).toArray(String[]::new), Strings.EMPTY_ARRAY); break; case STEP1_OLD_CLUSTER: - assertSnapshotStatusSuccessful(client, repoName, "snapshot-" + TEST_STEP); + assertSnapshotStatusSuccessful(client, repoName, new String[] {"snapshot-" + TEST_STEP}, Strings.EMPTY_ARRAY); break; case STEP3_OLD_CLUSTER: assertSnapshotStatusSuccessful( - client, repoName, "snapshot-" + TEST_STEP, "snapshot-" + TestStep.STEP3_OLD_CLUSTER); + client, repoName, new String[] {"snapshot-" + TEST_STEP, "snapshot-" + TestStep.STEP3_OLD_CLUSTER}, Strings.EMPTY_ARRAY); break; } if (TEST_STEP == TestStep.STEP3_OLD_CLUSTER) { @@ -186,10 +188,10 @@ public void testReadOnlyRepo() throws IOException { break; } if (TEST_STEP == TestStep.STEP1_OLD_CLUSTER || TEST_STEP == TestStep.STEP3_OLD_CLUSTER) { - assertSnapshotStatusSuccessful(client, repoName, "snapshot-" + TestStep.STEP1_OLD_CLUSTER); + assertSnapshotStatusSuccessful(client, repoName, new String[] {"snapshot-" + TestStep.STEP1_OLD_CLUSTER}, Strings.EMPTY_ARRAY); } else { assertSnapshotStatusSuccessful(client, repoName, - "snapshot-" + TestStep.STEP1_OLD_CLUSTER, "snapshot-" + TestStep.STEP2_NEW_CLUSTER); + new String[] {"snapshot-" + TestStep.STEP1_OLD_CLUSTER, "snapshot-" + TestStep.STEP2_NEW_CLUSTER}, Strings.EMPTY_ARRAY); } if (TEST_STEP == TestStep.STEP3_OLD_CLUSTER) { ensureSnapshotRestoreWorks(repoName, "snapshot-" + TestStep.STEP1_OLD_CLUSTER, shards); @@ -214,7 +216,7 @@ public void testUpgradeMovesRepoToNewMetaVersion() throws IOException { // Every step creates one snapshot assertThat(snapshots, hasSize(TEST_STEP.ordinal() + 1)); assertSnapshotStatusSuccessful(client, repoName, - snapshots.stream().map(sn -> (String) sn.get("snapshot")).toArray(String[]::new)); + snapshots.stream().map(sn -> (String) sn.get("snapshot")).toArray(String[]::new), Strings.EMPTY_ARRAY); if (TEST_STEP == TestStep.STEP1_OLD_CLUSTER) { ensureSnapshotRestoreWorks(repoName, "snapshot-" + TestStep.STEP1_OLD_CLUSTER, shards); } else { @@ -239,9 +241,9 @@ public void testUpgradeMovesRepoToNewMetaVersion() throws IOException { } private static void assertSnapshotStatusSuccessful(RestHighLevelClient client, String repoName, - String... snapshots) throws IOException { + String[] snapshots, String[] indices) throws IOException { final SnapshotsStatusResponse statusResponse = client.snapshot() - .status(new SnapshotsStatusRequest(repoName, snapshots), RequestOptions.DEFAULT); + .status(new SnapshotsStatusRequest(repoName, snapshots, indices), RequestOptions.DEFAULT); for (SnapshotStatus status : statusResponse.getSnapshots()) { assertThat(status.getShardsStats().getFailedShards(), is(0)); } diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/snapshot.status.json b/rest-api-spec/src/main/resources/rest-api-spec/api/snapshot.status.json index 1ac6042941013..354d3c35d2bda 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/snapshot.status.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/snapshot.status.json @@ -40,6 +40,26 @@ "description":"A comma-separated list of snapshot names" } } + }, + { + "path":"/_snapshot/{repository}/{snapshot}/{index}/_status", + "methods":[ + "GET" + ], + "parts":{ + "repository":{ + "type":"string", + "description":"A repository name" + }, + "snapshot":{ + "type":"string", + "description":"A snapshot name" + }, + "index":{ + "type": "list", + "description":"A comma-separated list of index names" + } + } } ] }, @@ -58,7 +78,7 @@ }, "ignore_unavailable":{ "type":"boolean", - "description":"Whether to ignore unavailable snapshots, defaults to false which means a SnapshotMissingException is thrown" + "description":"Whether to ignore unavailable snapshots and indices, defaults to false which means a SnapshotMissingException or IndexNotFoundException is thrown" } } } diff --git a/server/src/internalClusterTest/java/org/opensearch/action/admin/cluster/shards/TransportCatShardsActionIT.java b/server/src/internalClusterTest/java/org/opensearch/action/admin/cluster/shards/TransportCatShardsActionIT.java new file mode 100644 index 0000000000000..b86521dedf739 --- /dev/null +++ b/server/src/internalClusterTest/java/org/opensearch/action/admin/cluster/shards/TransportCatShardsActionIT.java @@ -0,0 +1,129 @@ +/* + * 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.action.admin.cluster.shards; + +import org.opensearch.action.admin.cluster.state.ClusterStateResponse; +import org.opensearch.action.admin.indices.stats.IndicesStatsResponse; +import org.opensearch.cluster.metadata.IndexMetadata; +import org.opensearch.cluster.routing.ShardRouting; +import org.opensearch.common.settings.Settings; +import org.opensearch.common.unit.TimeValue; +import org.opensearch.core.action.ActionListener; +import org.opensearch.core.common.Strings; +import org.opensearch.core.tasks.TaskCancelledException; +import org.opensearch.test.InternalTestCluster; +import org.opensearch.test.OpenSearchIntegTestCase; + +import java.io.IOException; +import java.util.List; +import java.util.concurrent.CountDownLatch; + +import static org.opensearch.cluster.routing.UnassignedInfo.INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING; +import static org.opensearch.common.unit.TimeValue.timeValueMillis; +import static org.opensearch.search.SearchService.NO_TIMEOUT; + +@OpenSearchIntegTestCase.ClusterScope(numDataNodes = 0, scope = OpenSearchIntegTestCase.Scope.TEST) +public class TransportCatShardsActionIT extends OpenSearchIntegTestCase { + + public void testCatShardsWithSuccessResponse() throws InterruptedException { + internalCluster().startClusterManagerOnlyNodes(1); + List nodes = internalCluster().startDataOnlyNodes(3); + createIndex( + "test", + Settings.builder() + .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 2) + .put(INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING.getKey(), "60m") + .build() + ); + ensureGreen("test"); + + final CatShardsRequest shardsRequest = new CatShardsRequest(); + shardsRequest.setCancelAfterTimeInterval(NO_TIMEOUT); + shardsRequest.setIndices(Strings.EMPTY_ARRAY); + CountDownLatch latch = new CountDownLatch(1); + client().execute(CatShardsAction.INSTANCE, shardsRequest, new ActionListener() { + @Override + public void onResponse(CatShardsResponse catShardsResponse) { + ClusterStateResponse clusterStateResponse = catShardsResponse.getClusterStateResponse(); + IndicesStatsResponse indicesStatsResponse = catShardsResponse.getIndicesStatsResponse(); + for (ShardRouting shard : clusterStateResponse.getState().routingTable().allShards()) { + assertEquals("test", shard.getIndexName()); + assertNotNull(indicesStatsResponse.asMap().get(shard)); + } + latch.countDown(); + } + + @Override + public void onFailure(Exception e) { + fail(); + latch.countDown(); + } + }); + latch.await(); + } + + public void testCatShardsWithTimeoutException() throws IOException, AssertionError, InterruptedException { + List masterNodes = internalCluster().startClusterManagerOnlyNodes(1); + List nodes = internalCluster().startDataOnlyNodes(3); + createIndex( + "test", + Settings.builder() + .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 2) + .put(INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING.getKey(), "60m") + .build() + ); + ensureGreen("test"); + + Settings clusterManagerDataPathSettings = internalCluster().dataPathSettings(masterNodes.get(0)); + // Dropping master node to delay in cluster state call. + internalCluster().stopRandomNode(InternalTestCluster.nameFilter(masterNodes.get(0))); + + CountDownLatch latch = new CountDownLatch(2); + new Thread(() -> { + try { + // Ensures the cancellation timeout expires. + Thread.sleep(2000); + // Starting master node to proceed in cluster state call. + internalCluster().startClusterManagerOnlyNode( + Settings.builder().put("node.name", masterNodes.get(0)).put(clusterManagerDataPathSettings).build() + ); + latch.countDown(); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + }).start(); + + final CatShardsRequest shardsRequest = new CatShardsRequest(); + TimeValue timeoutInterval = timeValueMillis(1000); + shardsRequest.setCancelAfterTimeInterval(timeoutInterval); + shardsRequest.clusterManagerNodeTimeout(timeValueMillis(2500)); + shardsRequest.setIndices(Strings.EMPTY_ARRAY); + client().execute(CatShardsAction.INSTANCE, shardsRequest, new ActionListener() { + @Override + public void onResponse(CatShardsResponse catShardsResponse) { + // onResponse should not be called. + latch.countDown(); + throw new AssertionError( + "The cat shards action is expected to fail with a TaskCancelledException, but it received a successful response instead." + ); + } + + @Override + public void onFailure(Exception e) { + assertSame(e.getClass(), TaskCancelledException.class); + assertEquals(e.getMessage(), "Cancellation timeout of " + timeoutInterval + " is expired"); + latch.countDown(); + } + }); + latch.await(); + } + +} diff --git a/server/src/internalClusterTest/java/org/opensearch/gateway/remote/RemoteRoutingTableServiceIT.java b/server/src/internalClusterTest/java/org/opensearch/gateway/remote/RemoteRoutingTableServiceIT.java index 9b80e4f907070..3c0c9a0611439 100644 --- a/server/src/internalClusterTest/java/org/opensearch/gateway/remote/RemoteRoutingTableServiceIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/gateway/remote/RemoteRoutingTableServiceIT.java @@ -67,6 +67,7 @@ protected Settings nodeSettings(int nodeOrdinal) { ) .put("node.attr." + REMOTE_STORE_ROUTING_TABLE_REPOSITORY_NAME_ATTRIBUTE_KEY, REMOTE_ROUTING_TABLE_REPO) .put(REMOTE_PUBLICATION_EXPERIMENTAL, true) + .put(RemoteClusterStateService.REMOTE_CLUSTER_STATE_CHECKSUM_VALIDATION_ENABLED_SETTING.getKey(), true) .build(); } diff --git a/server/src/internalClusterTest/java/org/opensearch/gateway/remote/RemoteStatePublicationIT.java b/server/src/internalClusterTest/java/org/opensearch/gateway/remote/RemoteStatePublicationIT.java index f237a81942e1a..3a3e023db6446 100644 --- a/server/src/internalClusterTest/java/org/opensearch/gateway/remote/RemoteStatePublicationIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/gateway/remote/RemoteStatePublicationIT.java @@ -90,6 +90,7 @@ protected Settings nodeSettings(int nodeOrdinal) { .put("node.attr." + REMOTE_STORE_ROUTING_TABLE_REPOSITORY_NAME_ATTRIBUTE_KEY, routingTableRepoName) .put(routingTableRepoTypeAttributeKey, ReloadableFsRepository.TYPE) .put(routingTableRepoSettingsAttributeKeyPrefix + "location", segmentRepoPath) + .put(RemoteClusterStateService.REMOTE_CLUSTER_STATE_CHECKSUM_VALIDATION_ENABLED_SETTING.getKey(), true) .build(); } diff --git a/server/src/internalClusterTest/java/org/opensearch/index/mapper/StarTreeMapperIT.java b/server/src/internalClusterTest/java/org/opensearch/index/mapper/StarTreeMapperIT.java index 52c6c6801a3c2..8cfb710679137 100644 --- a/server/src/internalClusterTest/java/org/opensearch/index/mapper/StarTreeMapperIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/index/mapper/StarTreeMapperIT.java @@ -8,19 +8,28 @@ package org.opensearch.index.mapper; +import org.opensearch.action.admin.cluster.settings.ClusterUpdateSettingsRequest; +import org.opensearch.action.index.IndexResponse; +import org.opensearch.action.search.SearchResponse; import org.opensearch.action.support.master.AcknowledgedResponse; import org.opensearch.common.Rounding; import org.opensearch.common.settings.Settings; import org.opensearch.common.util.FeatureFlags; +import org.opensearch.core.common.unit.ByteSizeUnit; +import org.opensearch.core.common.unit.ByteSizeValue; import org.opensearch.core.index.Index; +import org.opensearch.core.rest.RestStatus; import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.index.IndexService; +import org.opensearch.index.IndexSettings; import org.opensearch.index.compositeindex.CompositeIndexSettings; import org.opensearch.index.compositeindex.datacube.DateDimension; import org.opensearch.index.compositeindex.datacube.MetricStat; import org.opensearch.index.compositeindex.datacube.startree.StarTreeFieldConfiguration; import org.opensearch.index.compositeindex.datacube.startree.StarTreeIndexSettings; +import org.opensearch.index.query.QueryBuilders; import org.opensearch.indices.IndicesService; +import org.opensearch.search.SearchHit; import org.opensearch.test.OpenSearchIntegTestCase; import org.junit.After; import org.junit.Before; @@ -39,6 +48,10 @@ */ public class StarTreeMapperIT extends OpenSearchIntegTestCase { private static final String TEST_INDEX = "test"; + Settings settings = Settings.builder() + .put(StarTreeIndexSettings.IS_COMPOSITE_INDEX_SETTING.getKey(), true) + .put(IndexSettings.INDEX_TRANSLOG_FLUSH_THRESHOLD_SIZE_SETTING.getKey(), new ByteSizeValue(512, ByteSizeUnit.MB)) + .build(); private static XContentBuilder createMinimalTestMapping(boolean invalidDim, boolean invalidMetric, boolean keywordDim) { try { @@ -116,6 +129,12 @@ private static XContentBuilder createMaxDimTestMapping() { .startObject() .field("name", "dim2") .endObject() + .startObject() + .field("name", "dim3") + .endObject() + .startObject() + .field("name", "dim4") + .endObject() .endArray() .endObject() .endObject() @@ -132,6 +151,10 @@ private static XContentBuilder createMaxDimTestMapping() { .field("type", "integer") .field("doc_values", true) .endObject() + .startObject("dim4") + .field("type", "integer") + .field("doc_values", true) + .endObject() .endObject() .endObject(); } catch (IOException e) { @@ -223,6 +246,56 @@ private static XContentBuilder createUpdateTestMapping(boolean changeDim, boolea } } + private XContentBuilder getMappingWithDuplicateFields(boolean isDuplicateDim, boolean isDuplicateMetric) { + XContentBuilder mapping = null; + try { + mapping = jsonBuilder().startObject() + .startObject("composite") + .startObject("startree-1") + .field("type", "star_tree") + .startObject("config") + .startArray("ordered_dimensions") + .startObject() + .field("name", "timestamp") + .endObject() + .startObject() + .field("name", "numeric_dv") + .endObject() + .startObject() + .field("name", isDuplicateDim ? "numeric_dv" : "numeric_dv1") // Duplicate dimension + .endObject() + .endArray() + .startArray("metrics") + .startObject() + .field("name", "numeric_dv") + .endObject() + .startObject() + .field("name", isDuplicateMetric ? "numeric_dv" : "numeric_dv1") // Duplicate metric + .endObject() + .endArray() + .endObject() + .endObject() + .endObject() + .startObject("properties") + .startObject("timestamp") + .field("type", "date") + .endObject() + .startObject("numeric_dv") + .field("type", "integer") + .field("doc_values", true) + .endObject() + .startObject("numeric_dv1") + .field("type", "integer") + .field("doc_values", true) + .endObject() + .endObject() + .endObject(); + } catch (IOException e) { + fail("Failed to create mapping: " + e.getMessage()); + } + return mapping; + } + private static String getDim(boolean hasDocValues, boolean isKeyword) { if (hasDocValues) { return "numeric"; @@ -244,7 +317,7 @@ public final void setupNodeSettings() { } public void testValidCompositeIndex() { - prepareCreate(TEST_INDEX).setMapping(createMinimalTestMapping(false, false, false)).get(); + prepareCreate(TEST_INDEX).setSettings(settings).setMapping(createMinimalTestMapping(false, false, false)).get(); Iterable dataNodeInstances = internalCluster().getDataNodeInstances(IndicesService.class); for (IndicesService service : dataNodeInstances) { final Index index = resolveIndex("test"); @@ -285,8 +358,91 @@ public void testValidCompositeIndex() { } } + public void testCompositeIndexWithIndexNotSpecified() { + Settings settings = Settings.builder() + .put(IndexSettings.INDEX_TRANSLOG_FLUSH_THRESHOLD_SIZE_SETTING.getKey(), new ByteSizeValue(512, ByteSizeUnit.MB)) + .build(); + MapperParsingException ex = expectThrows( + MapperParsingException.class, + () -> prepareCreate(TEST_INDEX).setSettings(settings).setMapping(createMinimalTestMapping(false, false, false)).get() + ); + assertEquals( + "Failed to parse mapping [_doc]: Set 'index.composite_index' as true as part of index settings to use star tree index", + ex.getMessage() + ); + } + + public void testCompositeIndexWithHigherTranslogFlushSize() { + Settings settings = Settings.builder() + .put(StarTreeIndexSettings.IS_COMPOSITE_INDEX_SETTING.getKey(), true) + .put(IndexSettings.INDEX_TRANSLOG_FLUSH_THRESHOLD_SIZE_SETTING.getKey(), new ByteSizeValue(513, ByteSizeUnit.MB)) + .build(); + IllegalArgumentException ex = expectThrows( + IllegalArgumentException.class, + () -> prepareCreate(TEST_INDEX).setSettings(settings).setMapping(createMinimalTestMapping(false, false, false)).get() + ); + assertEquals("You can configure 'index.translog.flush_threshold_size' with upto '512mb' for composite index", ex.getMessage()); + } + + public void testCompositeIndexWithArraysInCompositeField() throws IOException { + prepareCreate(TEST_INDEX).setSettings(settings).setMapping(createMinimalTestMapping(false, false, false)).get(); + // Attempt to index a document with an array field + XContentBuilder doc = jsonBuilder().startObject() + .field("timestamp", "2023-06-01T12:00:00Z") + .startArray("numeric_dv") + .value(10) + .value(20) + .value(30) + .endArray() + .endObject(); + + // Index the document and refresh + MapperParsingException ex = expectThrows( + MapperParsingException.class, + () -> client().prepareIndex(TEST_INDEX).setSource(doc).get() + ); + assertEquals( + "object mapping for [_doc] with array for [numeric_dv] cannot be accepted as field is also part of composite index mapping which does not accept arrays", + ex.getMessage() + ); + } + + public void testCompositeIndexWithArraysInNonCompositeField() throws IOException { + prepareCreate(TEST_INDEX).setSettings(settings).setMapping(createMinimalTestMapping(false, false, false)).get(); + // Attempt to index a document with an array field + XContentBuilder doc = jsonBuilder().startObject() + .field("timestamp", "2023-06-01T12:00:00Z") + .startArray("numeric") + .value(10) + .value(20) + .value(30) + .endArray() + .endObject(); + + // Index the document and refresh + IndexResponse indexResponse = client().prepareIndex(TEST_INDEX).setSource(doc).get(); + + assertEquals(RestStatus.CREATED, indexResponse.status()); + + client().admin().indices().prepareRefresh(TEST_INDEX).get(); + // Verify the document was indexed + SearchResponse searchResponse = client().prepareSearch(TEST_INDEX).setQuery(QueryBuilders.matchAllQuery()).get(); + + assertEquals(1, searchResponse.getHits().getTotalHits().value); + + // Verify the values in the indexed document + SearchHit hit = searchResponse.getHits().getAt(0); + assertEquals("2023-06-01T12:00:00Z", hit.getSourceAsMap().get("timestamp")); + + List values = (List) hit.getSourceAsMap().get("numeric"); + assertEquals(3, values.size()); + assertTrue(values.contains(10)); + assertTrue(values.contains(20)); + assertTrue(values.contains(30)); + } + public void testUpdateIndexWithAdditionOfStarTree() { - prepareCreate(TEST_INDEX).setMapping(createMinimalTestMapping(false, false, false)).get(); + prepareCreate(TEST_INDEX).setSettings(settings).setMapping(createMinimalTestMapping(false, false, false)).get(); IllegalArgumentException ex = expectThrows( IllegalArgumentException.class, @@ -296,7 +452,7 @@ public void testUpdateIndexWithAdditionOfStarTree() { } public void testUpdateIndexWithNewerStarTree() { - prepareCreate(TEST_INDEX).setMapping(createTestMappingWithoutStarTree(false, false, false)).get(); + prepareCreate(TEST_INDEX).setSettings(settings).setMapping(createTestMappingWithoutStarTree(false, false, false)).get(); IllegalArgumentException ex = expectThrows( IllegalArgumentException.class, @@ -309,7 +465,7 @@ public void testUpdateIndexWithNewerStarTree() { } public void testUpdateIndexWhenMappingIsDifferent() { - prepareCreate(TEST_INDEX).setMapping(createMinimalTestMapping(false, false, false)).get(); + prepareCreate(TEST_INDEX).setSettings(settings).setMapping(createMinimalTestMapping(false, false, false)).get(); // update some field in the mapping IllegalArgumentException ex = expectThrows( @@ -320,7 +476,7 @@ public void testUpdateIndexWhenMappingIsDifferent() { } public void testUpdateIndexWhenMappingIsSame() { - prepareCreate(TEST_INDEX).setMapping(createMinimalTestMapping(false, false, false)).get(); + prepareCreate(TEST_INDEX).setSettings(settings).setMapping(createMinimalTestMapping(false, false, false)).get(); // update some field in the mapping AcknowledgedResponse putMappingResponse = client().admin() @@ -368,7 +524,7 @@ public void testUpdateIndexWhenMappingIsSame() { public void testInvalidDimCompositeIndex() { IllegalArgumentException ex = expectThrows( IllegalArgumentException.class, - () -> prepareCreate(TEST_INDEX).setMapping(createMinimalTestMapping(true, false, false)).get() + () -> prepareCreate(TEST_INDEX).setSettings(settings).setMapping(createMinimalTestMapping(true, false, false)).get() ); assertEquals( "Aggregations not supported for the dimension field [numeric] with field type [integer] as part of star tree field", @@ -379,8 +535,14 @@ public void testInvalidDimCompositeIndex() { public void testMaxDimsCompositeIndex() { MapperParsingException ex = expectThrows( MapperParsingException.class, - () -> prepareCreate(TEST_INDEX).setMapping(createMaxDimTestMapping()) - .setSettings(Settings.builder().put(StarTreeIndexSettings.STAR_TREE_MAX_DIMENSIONS_SETTING.getKey(), 2)) + () -> prepareCreate(TEST_INDEX).setSettings(settings) + .setMapping(createMaxDimTestMapping()) + .setSettings( + Settings.builder() + .put(StarTreeIndexSettings.STAR_TREE_MAX_DIMENSIONS_SETTING.getKey(), 2) + .put(StarTreeIndexSettings.IS_COMPOSITE_INDEX_SETTING.getKey(), true) + .put(IndexSettings.INDEX_TRANSLOG_FLUSH_THRESHOLD_SIZE_SETTING.getKey(), new ByteSizeValue(512, ByteSizeUnit.MB)) + ) .get() ); assertEquals( @@ -389,11 +551,35 @@ public void testMaxDimsCompositeIndex() { ); } + public void testMaxMetricsCompositeIndex() { + MapperParsingException ex = expectThrows( + MapperParsingException.class, + () -> prepareCreate(TEST_INDEX).setSettings(settings) + .setMapping(createMaxDimTestMapping()) + .setSettings( + Settings.builder() + .put(StarTreeIndexSettings.STAR_TREE_MAX_BASE_METRICS_SETTING.getKey(), 4) + .put(StarTreeIndexSettings.IS_COMPOSITE_INDEX_SETTING.getKey(), true) + .put(IndexSettings.INDEX_TRANSLOG_FLUSH_THRESHOLD_SIZE_SETTING.getKey(), new ByteSizeValue(512, ByteSizeUnit.MB)) + ) + .get() + ); + assertEquals( + "Failed to parse mapping [_doc]: There cannot be more than [4] base metrics for star tree field [startree-1]", + ex.getMessage() + ); + } + public void testMaxCalendarIntervalsCompositeIndex() { MapperParsingException ex = expectThrows( MapperParsingException.class, () -> prepareCreate(TEST_INDEX).setMapping(createMaxDimTestMapping()) - .setSettings(Settings.builder().put(StarTreeIndexSettings.STAR_TREE_MAX_DATE_INTERVALS_SETTING.getKey(), 1)) + .setSettings( + Settings.builder() + .put(StarTreeIndexSettings.STAR_TREE_MAX_DATE_INTERVALS_SETTING.getKey(), 1) + .put(StarTreeIndexSettings.IS_COMPOSITE_INDEX_SETTING.getKey(), true) + .put(IndexSettings.INDEX_TRANSLOG_FLUSH_THRESHOLD_SIZE_SETTING.getKey(), new ByteSizeValue(512, ByteSizeUnit.MB)) + ) .get() ); assertEquals( @@ -405,7 +591,7 @@ public void testMaxCalendarIntervalsCompositeIndex() { public void testUnsupportedDim() { MapperParsingException ex = expectThrows( MapperParsingException.class, - () -> prepareCreate(TEST_INDEX).setMapping(createMinimalTestMapping(false, false, true)).get() + () -> prepareCreate(TEST_INDEX).setSettings(settings).setMapping(createMinimalTestMapping(false, false, true)).get() ); assertEquals( "Failed to parse mapping [_doc]: unsupported field type associated with dimension [keyword] as part of star tree field [startree-1]", @@ -416,7 +602,7 @@ public void testUnsupportedDim() { public void testInvalidMetric() { IllegalArgumentException ex = expectThrows( IllegalArgumentException.class, - () -> prepareCreate(TEST_INDEX).setMapping(createMinimalTestMapping(false, true, false)).get() + () -> prepareCreate(TEST_INDEX).setSettings(settings).setMapping(createMinimalTestMapping(false, true, false)).get() ); assertEquals( "Aggregations not supported for the metrics field [numeric] with field type [integer] as part of star tree field", @@ -424,6 +610,145 @@ public void testInvalidMetric() { ); } + public void testDuplicateDimensions() { + XContentBuilder finalMapping = getMappingWithDuplicateFields(true, false); + MapperParsingException ex = expectThrows( + MapperParsingException.class, + () -> prepareCreate(TEST_INDEX).setSettings(settings).setMapping(finalMapping).setSettings(settings).get() + ); + assertEquals( + "Failed to parse mapping [_doc]: Duplicate dimension [numeric_dv] present as part star tree index field [startree-1]", + ex.getMessage() + ); + } + + public void testDuplicateMetrics() { + XContentBuilder finalMapping = getMappingWithDuplicateFields(false, true); + MapperParsingException ex = expectThrows( + MapperParsingException.class, + () -> prepareCreate(TEST_INDEX).setSettings(settings).setMapping(finalMapping).setSettings(settings).get() + ); + assertEquals( + "Failed to parse mapping [_doc]: Duplicate metrics [numeric_dv] present as part star tree index field [startree-1]", + ex.getMessage() + ); + } + + public void testValidTranslogFlushThresholdSize() { + Settings indexSettings = Settings.builder() + .put(settings) + .put(IndexSettings.INDEX_TRANSLOG_FLUSH_THRESHOLD_SIZE_SETTING.getKey(), new ByteSizeValue(256, ByteSizeUnit.MB)) + .build(); + + AcknowledgedResponse response = prepareCreate(TEST_INDEX).setSettings(indexSettings) + .setMapping(createMinimalTestMapping(false, false, false)) + .get(); + + assertTrue(response.isAcknowledged()); + } + + public void testInvalidTranslogFlushThresholdSize() { + Settings indexSettings = Settings.builder() + .put(settings) + .put(IndexSettings.INDEX_TRANSLOG_FLUSH_THRESHOLD_SIZE_SETTING.getKey(), new ByteSizeValue(1024, ByteSizeUnit.MB)) + .build(); + + IllegalArgumentException ex = expectThrows( + IllegalArgumentException.class, + () -> prepareCreate(TEST_INDEX).setSettings(indexSettings).setMapping(createMinimalTestMapping(false, false, false)).get() + ); + + assertTrue( + ex.getMessage().contains("You can configure 'index.translog.flush_threshold_size' with upto '512mb' for composite index") + ); + } + + public void testTranslogFlushThresholdSizeWithDefaultCompositeSettingLow() { + Settings updatedSettings = Settings.builder() + .put(CompositeIndexSettings.COMPOSITE_INDEX_MAX_TRANSLOG_FLUSH_THRESHOLD_SIZE_SETTING.getKey(), "130m") + .build(); + + ClusterUpdateSettingsRequest updateSettingsRequest = new ClusterUpdateSettingsRequest().transientSettings(updatedSettings); + + client().admin().cluster().updateSettings(updateSettingsRequest).actionGet(); + IllegalArgumentException ex = expectThrows( + IllegalArgumentException.class, + () -> prepareCreate(TEST_INDEX).setSettings(settings).setMapping(createMinimalTestMapping(false, false, false)).get() + ); + + assertEquals("You can configure 'index.translog.flush_threshold_size' with upto '130mb' for composite index", ex.getMessage()); + } + + public void testUpdateTranslogFlushThresholdSize() { + prepareCreate(TEST_INDEX).setSettings(settings).setMapping(createMinimalTestMapping(false, false, false)).get(); + + // Update to a valid value + AcknowledgedResponse validUpdateResponse = client().admin() + .indices() + .prepareUpdateSettings(TEST_INDEX) + .setSettings(Settings.builder().put(IndexSettings.INDEX_TRANSLOG_FLUSH_THRESHOLD_SIZE_SETTING.getKey(), "256mb")) + .get(); + assertTrue(validUpdateResponse.isAcknowledged()); + + // Try to update to an invalid value + IllegalArgumentException ex = expectThrows( + IllegalArgumentException.class, + () -> client().admin() + .indices() + .prepareUpdateSettings(TEST_INDEX) + .setSettings(Settings.builder().put(IndexSettings.INDEX_TRANSLOG_FLUSH_THRESHOLD_SIZE_SETTING.getKey(), "1024mb")) + .get() + ); + + assertTrue( + ex.getMessage().contains("You can configure 'index.translog.flush_threshold_size' with upto '512mb' for composite index") + ); + + // update cluster settings to higher value + Settings updatedSettings = Settings.builder() + .put(CompositeIndexSettings.COMPOSITE_INDEX_MAX_TRANSLOG_FLUSH_THRESHOLD_SIZE_SETTING.getKey(), "1030m") + .build(); + + ClusterUpdateSettingsRequest updateSettingsRequest = new ClusterUpdateSettingsRequest().transientSettings(updatedSettings); + + client().admin().cluster().updateSettings(updateSettingsRequest).actionGet(); + + // update index threshold flush to higher value + validUpdateResponse = client().admin() + .indices() + .prepareUpdateSettings(TEST_INDEX) + .setSettings(Settings.builder().put(IndexSettings.INDEX_TRANSLOG_FLUSH_THRESHOLD_SIZE_SETTING.getKey(), "1024mb")) + .get(); + assertTrue(validUpdateResponse.isAcknowledged()); + } + + public void testMinimumTranslogFlushThresholdSize() { + Settings indexSettings = Settings.builder() + .put(settings) + .put(IndexSettings.INDEX_TRANSLOG_FLUSH_THRESHOLD_SIZE_SETTING.getKey(), new ByteSizeValue(56, ByteSizeUnit.BYTES)) + .build(); + + AcknowledgedResponse response = prepareCreate(TEST_INDEX).setSettings(indexSettings) + .setMapping(createMinimalTestMapping(false, false, false)) + .get(); + + assertTrue(response.isAcknowledged()); + } + + public void testBelowMinimumTranslogFlushThresholdSize() { + Settings indexSettings = Settings.builder() + .put(settings) + .put(IndexSettings.INDEX_TRANSLOG_FLUSH_THRESHOLD_SIZE_SETTING.getKey(), new ByteSizeValue(55, ByteSizeUnit.BYTES)) + .build(); + + IllegalArgumentException ex = expectThrows( + IllegalArgumentException.class, + () -> prepareCreate(TEST_INDEX).setSettings(indexSettings).setMapping(createMinimalTestMapping(false, false, false)).get() + ); + + assertEquals("failed to parse value [55b] for setting [index.translog.flush_threshold_size], must be >= [56b]", ex.getMessage()); + } + @After public final void cleanupNodeSettings() { assertAcked( diff --git a/server/src/internalClusterTest/java/org/opensearch/remotemigration/RemotePrimaryLocalRecoveryIT.java b/server/src/internalClusterTest/java/org/opensearch/remotemigration/RemotePrimaryLocalRecoveryIT.java index 024fc68602a19..8ab093b8495cb 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotemigration/RemotePrimaryLocalRecoveryIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotemigration/RemotePrimaryLocalRecoveryIT.java @@ -19,6 +19,7 @@ import org.opensearch.core.util.FileSystemUtils; import org.opensearch.index.remote.RemoteSegmentStats; import org.opensearch.index.translog.RemoteTranslogStats; +import org.opensearch.indices.RemoteStoreSettings; import org.opensearch.test.InternalTestCluster; import org.opensearch.test.OpenSearchIntegTestCase; @@ -67,6 +68,7 @@ public void testLocalRecoveryRollingRestartAndNodeFailure() throws Exception { assertTrue(remoteSegmentStats.getUploadBytesSucceeded() > 0); } + String segmentsPathFixedPrefix = RemoteStoreSettings.CLUSTER_REMOTE_STORE_SEGMENTS_PATH_PREFIX.get(getNodeSettings()); assertBusy(() -> { String shardPath = getShardLevelBlobPath( client(), @@ -74,7 +76,8 @@ public void testLocalRecoveryRollingRestartAndNodeFailure() throws Exception { new BlobPath(), String.valueOf(shardRouting.getId()), SEGMENTS, - DATA + DATA, + segmentsPathFixedPrefix ).buildAsString(); Path segmentDataRepoPath = segmentRepoPath.resolve(shardPath); List segmentsNFilesInRepo = Arrays.stream(FileSystemUtils.files(segmentDataRepoPath)) diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteRestoreSnapshotIT.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteRestoreSnapshotIT.java index 56078a6ef8800..0acb578e2e7bf 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteRestoreSnapshotIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteRestoreSnapshotIT.java @@ -443,13 +443,15 @@ public void testRestoreInSameRemoteStoreEnabledIndex() throws IOException { void assertRemoteSegmentsAndTranslogUploaded(String idx) throws IOException { Client client = client(); - String path = getShardLevelBlobPath(client, idx, new BlobPath(), "0", TRANSLOG, METADATA).buildAsString(); + String translogPathFixedPrefix = RemoteStoreSettings.CLUSTER_REMOTE_STORE_TRANSLOG_PATH_PREFIX.get(getNodeSettings()); + String segmentsPathFixedPrefix = RemoteStoreSettings.CLUSTER_REMOTE_STORE_SEGMENTS_PATH_PREFIX.get(getNodeSettings()); + String path = getShardLevelBlobPath(client, idx, new BlobPath(), "0", TRANSLOG, METADATA, translogPathFixedPrefix).buildAsString(); Path remoteTranslogMetadataPath = Path.of(remoteRepoPath + "/" + path); - path = getShardLevelBlobPath(client, idx, new BlobPath(), "0", TRANSLOG, DATA).buildAsString(); + path = getShardLevelBlobPath(client, idx, new BlobPath(), "0", TRANSLOG, DATA, translogPathFixedPrefix).buildAsString(); Path remoteTranslogDataPath = Path.of(remoteRepoPath + "/" + path); - path = getShardLevelBlobPath(client, idx, new BlobPath(), "0", SEGMENTS, METADATA).buildAsString(); + path = getShardLevelBlobPath(client, idx, new BlobPath(), "0", SEGMENTS, METADATA, segmentsPathFixedPrefix).buildAsString(); Path segmentMetadataPath = Path.of(remoteRepoPath + "/" + path); - path = getShardLevelBlobPath(client, idx, new BlobPath(), "0", SEGMENTS, DATA).buildAsString(); + path = getShardLevelBlobPath(client, idx, new BlobPath(), "0", SEGMENTS, DATA, segmentsPathFixedPrefix).buildAsString(); Path segmentDataPath = Path.of(remoteRepoPath + "/" + path); try ( diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreIT.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreIT.java index 9a8d3651160c0..692727357a88a 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreIT.java @@ -205,7 +205,16 @@ public void testStaleCommitDeletionWithInvokeFlush() throws Exception { createIndex(INDEX_NAME, remoteStoreIndexSettings(1, 10000l, -1)); int numberOfIterations = randomIntBetween(5, 15); indexData(numberOfIterations, true, INDEX_NAME); - String shardPath = getShardLevelBlobPath(client(), INDEX_NAME, BlobPath.cleanPath(), "0", SEGMENTS, METADATA).buildAsString(); + String segmentsPathFixedPrefix = RemoteStoreSettings.CLUSTER_REMOTE_STORE_SEGMENTS_PATH_PREFIX.get(getNodeSettings()); + String shardPath = getShardLevelBlobPath( + client(), + INDEX_NAME, + BlobPath.cleanPath(), + "0", + SEGMENTS, + METADATA, + segmentsPathFixedPrefix + ).buildAsString(); Path indexPath = Path.of(segmentRepoPath + "/" + shardPath); ; IndexShard indexShard = getIndexShard(dataNode, INDEX_NAME); @@ -236,7 +245,16 @@ public void testStaleCommitDeletionWithoutInvokeFlush() throws Exception { createIndex(INDEX_NAME, remoteStoreIndexSettings(1, 10000l, -1)); int numberOfIterations = randomIntBetween(5, 15); indexData(numberOfIterations, false, INDEX_NAME); - String shardPath = getShardLevelBlobPath(client(), INDEX_NAME, BlobPath.cleanPath(), "0", SEGMENTS, METADATA).buildAsString(); + String segmentsPathFixedPrefix = RemoteStoreSettings.CLUSTER_REMOTE_STORE_SEGMENTS_PATH_PREFIX.get(getNodeSettings()); + String shardPath = getShardLevelBlobPath( + client(), + INDEX_NAME, + BlobPath.cleanPath(), + "0", + SEGMENTS, + METADATA, + segmentsPathFixedPrefix + ).buildAsString(); Path indexPath = Path.of(segmentRepoPath + "/" + shardPath); int actualFileCount = getFileCount(indexPath); // We also allow (numberOfIterations + 1) as index creation also triggers refresh. @@ -247,11 +265,19 @@ public void testStaleCommitDeletionWithMinSegmentFiles_3() throws Exception { Settings.Builder settings = Settings.builder() .put(RemoteStoreSettings.CLUSTER_REMOTE_INDEX_SEGMENT_METADATA_RETENTION_MAX_COUNT_SETTING.getKey(), "3"); internalCluster().startNode(settings); - + String segmentsPathFixedPrefix = RemoteStoreSettings.CLUSTER_REMOTE_STORE_SEGMENTS_PATH_PREFIX.get(getNodeSettings()); createIndex(INDEX_NAME, remoteStoreIndexSettings(1, 10000l, -1)); int numberOfIterations = randomIntBetween(5, 15); indexData(numberOfIterations, true, INDEX_NAME); - String shardPath = getShardLevelBlobPath(client(), INDEX_NAME, BlobPath.cleanPath(), "0", SEGMENTS, METADATA).buildAsString(); + String shardPath = getShardLevelBlobPath( + client(), + INDEX_NAME, + BlobPath.cleanPath(), + "0", + SEGMENTS, + METADATA, + segmentsPathFixedPrefix + ).buildAsString(); Path indexPath = Path.of(segmentRepoPath + "/" + shardPath); int actualFileCount = getFileCount(indexPath); // We also allow (numberOfIterations + 1) as index creation also triggers refresh. @@ -271,7 +297,16 @@ public void testStaleCommitDeletionWithMinSegmentFiles_Disabled() throws Excepti createIndex(INDEX_NAME, remoteStoreIndexSettings(1, 10000l, -1)); int numberOfIterations = randomIntBetween(12, 18); indexData(numberOfIterations, true, INDEX_NAME); - String shardPath = getShardLevelBlobPath(client(), INDEX_NAME, BlobPath.cleanPath(), "0", SEGMENTS, METADATA).buildAsString(); + String segmentsPathFixedPrefix = RemoteStoreSettings.CLUSTER_REMOTE_STORE_SEGMENTS_PATH_PREFIX.get(getNodeSettings()); + String shardPath = getShardLevelBlobPath( + client(), + INDEX_NAME, + BlobPath.cleanPath(), + "0", + SEGMENTS, + METADATA, + segmentsPathFixedPrefix + ).buildAsString(); Path indexPath = Path.of(segmentRepoPath + "/" + shardPath); ; int actualFileCount = getFileCount(indexPath); @@ -604,8 +639,10 @@ public void testFallbackToNodeToNodeSegmentCopy() throws Exception { indexBulk(INDEX_NAME, 50); flushAndRefresh(INDEX_NAME); + String segmentsPathFixedPrefix = RemoteStoreSettings.CLUSTER_REMOTE_STORE_SEGMENTS_PATH_PREFIX.get(getNodeSettings()); // 3. Delete data from remote segment store - String shardPath = getShardLevelBlobPath(client(), INDEX_NAME, BlobPath.cleanPath(), "0", SEGMENTS, DATA).buildAsString(); + String shardPath = getShardLevelBlobPath(client(), INDEX_NAME, BlobPath.cleanPath(), "0", SEGMENTS, DATA, segmentsPathFixedPrefix) + .buildAsString(); Path segmentDataPath = Path.of(segmentRepoPath + "/" + shardPath); try (Stream files = Files.list(segmentDataPath)) { @@ -844,7 +881,16 @@ public void testLocalOnlyTranslogCleanupOnNodeRestart() throws Exception { .get() .getSetting(INDEX_NAME, IndexMetadata.SETTING_INDEX_UUID); - String shardPath = getShardLevelBlobPath(client(), INDEX_NAME, BlobPath.cleanPath(), "0", TRANSLOG, METADATA).buildAsString(); + String translogPathFixedPrefix = RemoteStoreSettings.CLUSTER_REMOTE_STORE_TRANSLOG_PATH_PREFIX.get(getNodeSettings()); + String shardPath = getShardLevelBlobPath( + client(), + INDEX_NAME, + BlobPath.cleanPath(), + "0", + TRANSLOG, + METADATA, + translogPathFixedPrefix + ).buildAsString(); Path translogMetaDataPath = Path.of(translogRepoPath + "/" + shardPath); try (Stream files = Files.list(translogMetaDataPath)) { diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreRefreshListenerIT.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreRefreshListenerIT.java index 7ae08bf968ade..a7292482100e1 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreRefreshListenerIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreRefreshListenerIT.java @@ -13,6 +13,7 @@ import org.opensearch.action.admin.indices.stats.IndicesStatsResponse; import org.opensearch.common.blobstore.BlobPath; import org.opensearch.common.settings.Settings; +import org.opensearch.indices.RemoteStoreSettings; import org.opensearch.test.OpenSearchIntegTestCase; import java.nio.file.Path; @@ -50,7 +51,10 @@ public void testRemoteRefreshRetryOnFailure() throws Exception { String indexName = response.getShards()[0].getShardRouting().index().getName(); String indexUuid = response.getShards()[0].getShardRouting().index().getUUID(); - String shardPath = getShardLevelBlobPath(client(), indexName, new BlobPath(), "0", SEGMENTS, DATA).buildAsString(); + + String segmentsPathFixedPrefix = RemoteStoreSettings.CLUSTER_REMOTE_STORE_SEGMENTS_PATH_PREFIX.get(getNodeSettings()); + String shardPath = getShardLevelBlobPath(client(), indexName, new BlobPath(), "0", SEGMENTS, DATA, segmentsPathFixedPrefix) + .buildAsString(); Path segmentDataRepoPath = location.resolve(shardPath); String segmentDataLocalPath = String.format(Locale.ROOT, "%s/indices/%s/0/index", response.getShards()[0].getDataPath(), indexUuid); diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/RestoreShallowSnapshotV2IT.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/RestoreShallowSnapshotV2IT.java new file mode 100644 index 0000000000000..c5a55f16cab2b --- /dev/null +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/RestoreShallowSnapshotV2IT.java @@ -0,0 +1,805 @@ +/* + * 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.remotestore; + +import org.opensearch.action.DocWriteResponse; +import org.opensearch.action.admin.cluster.remotestore.restore.RestoreRemoteStoreRequest; +import org.opensearch.action.admin.cluster.snapshots.restore.RestoreSnapshotResponse; +import org.opensearch.action.admin.indices.delete.DeleteIndexRequest; +import org.opensearch.action.admin.indices.recovery.RecoveryResponse; +import org.opensearch.action.delete.DeleteResponse; +import org.opensearch.action.support.PlainActionFuture; +import org.opensearch.client.Client; +import org.opensearch.client.Requests; +import org.opensearch.cluster.ClusterState; +import org.opensearch.cluster.metadata.IndexMetadata; +import org.opensearch.cluster.routing.RecoverySource; +import org.opensearch.common.Nullable; +import org.opensearch.common.blobstore.BlobPath; +import org.opensearch.common.io.PathUtils; +import org.opensearch.common.settings.Settings; +import org.opensearch.common.util.io.IOUtils; +import org.opensearch.core.index.Index; +import org.opensearch.core.rest.RestStatus; +import org.opensearch.index.IndexService; +import org.opensearch.index.IndexSettings; +import org.opensearch.index.remote.RemoteStoreEnums; +import org.opensearch.index.shard.IndexShard; +import org.opensearch.indices.IndicesService; +import org.opensearch.indices.RemoteStoreSettings; +import org.opensearch.indices.recovery.RecoveryState; +import org.opensearch.indices.replication.common.ReplicationType; +import org.opensearch.repositories.blobstore.BlobStoreRepository; +import org.opensearch.snapshots.AbstractSnapshotIntegTestCase; +import org.opensearch.snapshots.SnapshotInfo; +import org.opensearch.snapshots.SnapshotRestoreException; +import org.opensearch.snapshots.SnapshotState; +import org.opensearch.test.InternalTestCluster; +import org.opensearch.test.OpenSearchIntegTestCase; +import org.junit.After; +import org.junit.Before; + +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.concurrent.ExecutionException; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_REMOTE_STORE_ENABLED; +import static org.opensearch.index.remote.RemoteStoreEnums.DataCategory.SEGMENTS; +import static org.opensearch.index.remote.RemoteStoreEnums.DataCategory.TRANSLOG; +import static org.opensearch.index.remote.RemoteStoreEnums.DataType.DATA; +import static org.opensearch.index.remote.RemoteStoreEnums.DataType.METADATA; +import static org.opensearch.indices.RemoteStoreSettings.CLUSTER_REMOTE_STORE_PATH_TYPE_SETTING; +import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertAcked; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.greaterThan; +import static org.hamcrest.Matchers.greaterThanOrEqualTo; +import static org.hamcrest.Matchers.lessThanOrEqualTo; + +@OpenSearchIntegTestCase.ClusterScope(scope = OpenSearchIntegTestCase.Scope.TEST, numDataNodes = 0) +public class RestoreShallowSnapshotV2IT extends AbstractSnapshotIntegTestCase { + + private static final String BASE_REMOTE_REPO = "test-rs-repo" + TEST_REMOTE_STORE_REPO_SUFFIX; + private Path remoteRepoPath; + + @Before + public void setup() { + remoteRepoPath = randomRepoPath().toAbsolutePath(); + } + + @After + public void teardown() { + clusterAdmin().prepareCleanupRepository(BASE_REMOTE_REPO).get(); + } + + @Override + protected Settings nodeSettings(int nodeOrdinal) { + return Settings.builder() + .put(super.nodeSettings(nodeOrdinal)) + .put(remoteStoreClusterSettings(BASE_REMOTE_REPO, remoteRepoPath)) + .put(RemoteStoreSettings.CLUSTER_REMOTE_STORE_PINNED_TIMESTAMP_ENABLED.getKey(), true) + .build(); + } + + @Override + protected Settings.Builder getRepositorySettings(Path location, boolean shallowCopyEnabled) { + Settings.Builder settingsBuilder = randomRepositorySettings(); + settingsBuilder.put("location", location); + if (shallowCopyEnabled) { + settingsBuilder.put(BlobStoreRepository.REMOTE_STORE_INDEX_SHALLOW_COPY.getKey(), true) + .put(BlobStoreRepository.SHALLOW_SNAPSHOT_V2.getKey(), true); + } + return settingsBuilder; + } + + private Settings.Builder getIndexSettings(int numOfShards, int numOfReplicas) { + Settings.Builder settingsBuilder = Settings.builder() + .put(super.indexSettings()) + .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, numOfShards) + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, numOfReplicas) + .put(IndexSettings.INDEX_REFRESH_INTERVAL_SETTING.getKey(), "300s"); + return settingsBuilder; + } + + private void indexDocuments(Client client, String indexName, int numOfDocs) { + indexDocuments(client, indexName, 0, numOfDocs); + } + + protected void indexDocuments(Client client, String indexName, int fromId, int toId) { + for (int i = fromId; i < toId; i++) { + String id = Integer.toString(i); + client.prepareIndex(indexName).setId(id).setSource("text", "sometext").get(); + } + } + + private void assertDocsPresentInIndex(Client client, String indexName, int numOfDocs) { + for (int i = 0; i < numOfDocs; i++) { + String id = Integer.toString(i); + logger.info("checking for index " + indexName + " with docId" + id); + assertTrue("doc with id" + id + " is not present for index " + indexName, client.prepareGet(indexName, id).get().isExists()); + } + } + + public void testRestoreOperationsShallowCopyEnabled() throws Exception { + String clusterManagerNode = internalCluster().startClusterManagerOnlyNode(); + String primary = internalCluster().startDataOnlyNode(); + String indexName1 = "testindex1"; + String indexName2 = "testindex2"; + String snapshotRepoName = "test-restore-snapshot-repo"; + String snapshotName1 = "test-restore-snapshot1"; + String snapshotName2 = "test-restore-snapshot2"; + Path absolutePath1 = randomRepoPath().toAbsolutePath(); + logger.info("Snapshot Path [{}]", absolutePath1); + String restoredIndexName1 = indexName1 + "-restored"; + String restoredIndexName2 = indexName2 + "-restored"; + + createRepository(snapshotRepoName, "fs", getRepositorySettings(absolutePath1, true)); + + Client client = client(); + Settings indexSettings = getIndexSettings(1, 0).build(); + createIndex(indexName1, indexSettings); + + Settings indexSettings2 = getIndexSettings(1, 0).build(); + createIndex(indexName2, indexSettings2); + + final int numDocsInIndex1 = 5; + final int numDocsInIndex2 = 6; + indexDocuments(client, indexName1, numDocsInIndex1); + indexDocuments(client, indexName2, numDocsInIndex2); + ensureGreen(indexName1, indexName2); + + internalCluster().startDataOnlyNode(); + logger.info("--> snapshot"); + + SnapshotInfo snapshotInfo = createSnapshot(snapshotRepoName, snapshotName1, new ArrayList<>()); + assertThat(snapshotInfo.state(), equalTo(SnapshotState.SUCCESS)); + assertThat(snapshotInfo.successfulShards(), greaterThan(0)); + assertThat(snapshotInfo.successfulShards(), equalTo(snapshotInfo.totalShards())); + + updateRepository(snapshotRepoName, "fs", getRepositorySettings(absolutePath1, false)); + SnapshotInfo snapshotInfo2 = createSnapshot( + snapshotRepoName, + snapshotName2, + new ArrayList<>(Arrays.asList(indexName1, indexName2)) + ); + assertThat(snapshotInfo2.state(), equalTo(SnapshotState.SUCCESS)); + assertThat(snapshotInfo2.successfulShards(), greaterThan(0)); + assertThat(snapshotInfo2.successfulShards(), equalTo(snapshotInfo2.totalShards())); + + DeleteResponse deleteResponse = client().prepareDelete(indexName1, "0").execute().actionGet(); + assertEquals(deleteResponse.getResult(), DocWriteResponse.Result.DELETED); + indexDocuments(client, indexName1, numDocsInIndex1, numDocsInIndex1 + randomIntBetween(2, 5)); + ensureGreen(indexName1); + + RestoreSnapshotResponse restoreSnapshotResponse1 = client.admin() + .cluster() + .prepareRestoreSnapshot(snapshotRepoName, snapshotName1) + .setWaitForCompletion(false) + .setIndices(indexName1) + .setRenamePattern(indexName1) + .setRenameReplacement(restoredIndexName1) + .get(); + RestoreSnapshotResponse restoreSnapshotResponse2 = client.admin() + .cluster() + .prepareRestoreSnapshot(snapshotRepoName, snapshotName2) + .setWaitForCompletion(false) + .setIndices(indexName2) + .setRenamePattern(indexName2) + .setRenameReplacement(restoredIndexName2) + .get(); + assertEquals(restoreSnapshotResponse1.status(), RestStatus.ACCEPTED); + assertEquals(restoreSnapshotResponse2.status(), RestStatus.ACCEPTED); + ensureGreen(restoredIndexName1, restoredIndexName2); + assertDocsPresentInIndex(client, restoredIndexName1, numDocsInIndex1); + assertDocsPresentInIndex(client, restoredIndexName2, numDocsInIndex2); + + // deleting data for restoredIndexName1 and restoring from remote store. + internalCluster().stopRandomNode(InternalTestCluster.nameFilter(primary)); + ensureRed(restoredIndexName1); + // Re-initialize client to make sure we are not using client from stopped node. + client = client(clusterManagerNode); + assertAcked(client.admin().indices().prepareClose(restoredIndexName1)); + client.admin() + .cluster() + .restoreRemoteStore( + new RestoreRemoteStoreRequest().indices(restoredIndexName1).restoreAllShards(true), + PlainActionFuture.newFuture() + ); + ensureYellowAndNoInitializingShards(restoredIndexName1); + ensureGreen(restoredIndexName1); + assertDocsPresentInIndex(client(), restoredIndexName1, numDocsInIndex1); + // indexing some new docs and validating + indexDocuments(client, restoredIndexName1, numDocsInIndex1, numDocsInIndex1 + 2); + ensureGreen(restoredIndexName1); + assertDocsPresentInIndex(client, restoredIndexName1, numDocsInIndex1 + 2); + } + + public void testRemoteStoreCustomDataOnIndexCreationAndRestore() { + String clusterManagerNode = internalCluster().startClusterManagerOnlyNode(); + internalCluster().startDataOnlyNode(); + String indexName1 = "testindex1"; + String indexName2 = "testindex2"; + String snapshotRepoName = "test-restore-snapshot-repo"; + String snapshotName1 = "test-restore-snapshot1"; + Path absolutePath1 = randomRepoPath().toAbsolutePath(); + logger.info("Snapshot Path [{}]", absolutePath1); + String restoredIndexName1version1 = indexName1 + "-restored-1"; + String restoredIndexName1version2 = indexName1 + "-restored-2"; + + client(clusterManagerNode).admin() + .cluster() + .prepareUpdateSettings() + .setTransientSettings(Settings.builder().put(CLUSTER_REMOTE_STORE_PATH_TYPE_SETTING.getKey(), RemoteStoreEnums.PathType.FIXED)) + .get(); + createRepository(snapshotRepoName, "fs", getRepositorySettings(absolutePath1, true)); + Client client = client(); + Settings indexSettings = getIndexSettings(1, 0).build(); + createIndex(indexName1, indexSettings); + + indexDocuments(client, indexName1, randomIntBetween(5, 10)); + ensureGreen(indexName1); + validatePathType(indexName1, RemoteStoreEnums.PathType.FIXED); + + logger.info("--> snapshot"); + SnapshotInfo snapshotInfo = createSnapshot(snapshotRepoName, snapshotName1, new ArrayList<>(Arrays.asList(indexName1))); + assertEquals(SnapshotState.SUCCESS, snapshotInfo.state()); + assertTrue(snapshotInfo.successfulShards() > 0); + assertEquals(snapshotInfo.totalShards(), snapshotInfo.successfulShards()); + + RestoreSnapshotResponse restoreSnapshotResponse = client.admin() + .cluster() + .prepareRestoreSnapshot(snapshotRepoName, snapshotName1) + .setWaitForCompletion(false) + .setRenamePattern(indexName1) + .setRenameReplacement(restoredIndexName1version1) + .get(); + assertEquals(RestStatus.ACCEPTED, restoreSnapshotResponse.status()); + ensureGreen(restoredIndexName1version1); + validatePathType(restoredIndexName1version1, RemoteStoreEnums.PathType.FIXED); + + client(clusterManagerNode).admin() + .cluster() + .prepareUpdateSettings() + .setTransientSettings( + Settings.builder().put(CLUSTER_REMOTE_STORE_PATH_TYPE_SETTING.getKey(), RemoteStoreEnums.PathType.HASHED_PREFIX) + ) + .get(); + + restoreSnapshotResponse = client.admin() + .cluster() + .prepareRestoreSnapshot(snapshotRepoName, snapshotName1) + .setWaitForCompletion(false) + .setRenamePattern(indexName1) + .setRenameReplacement(restoredIndexName1version2) + .get(); + assertEquals(RestStatus.ACCEPTED, restoreSnapshotResponse.status()); + ensureGreen(restoredIndexName1version2); + validatePathType( + restoredIndexName1version2, + RemoteStoreEnums.PathType.HASHED_PREFIX, + RemoteStoreEnums.PathHashAlgorithm.FNV_1A_COMPOSITE_1 + ); + + // Create index with cluster setting cluster.remote_store.index.path.type as hashed_prefix. + indexSettings = getIndexSettings(1, 0).build(); + createIndex(indexName2, indexSettings); + ensureGreen(indexName2); + validatePathType(indexName2, RemoteStoreEnums.PathType.HASHED_PREFIX, RemoteStoreEnums.PathHashAlgorithm.FNV_1A_COMPOSITE_1); + + // Validating that custom data has not changed for indexes which were created before the cluster setting got updated + validatePathType(indexName1, RemoteStoreEnums.PathType.FIXED); + + // Create Snapshot of index 2 + String snapshotName2 = "test-restore-snapshot2"; + snapshotInfo = createSnapshot(snapshotRepoName, snapshotName2, new ArrayList<>(List.of(indexName2))); + assertEquals(SnapshotState.SUCCESS, snapshotInfo.state()); + assertTrue(snapshotInfo.successfulShards() > 0); + assertEquals(snapshotInfo.totalShards(), snapshotInfo.successfulShards()); + + // Update cluster settings to FIXED + client(clusterManagerNode).admin() + .cluster() + .prepareUpdateSettings() + .setTransientSettings(Settings.builder().put(CLUSTER_REMOTE_STORE_PATH_TYPE_SETTING.getKey(), RemoteStoreEnums.PathType.FIXED)) + .get(); + + // Close index 2 + assertAcked(client().admin().indices().prepareClose(indexName2)); + restoreSnapshotResponse = client.admin() + .cluster() + .prepareRestoreSnapshot(snapshotRepoName, snapshotName2) + .setWaitForCompletion(false) + .setIndices(indexName2) + .get(); + assertEquals(RestStatus.ACCEPTED, restoreSnapshotResponse.status()); + ensureGreen(indexName2); + + // Validating that custom data has not changed for testindex2 which was created before the cluster setting got updated + validatePathType(indexName2, RemoteStoreEnums.PathType.HASHED_PREFIX, RemoteStoreEnums.PathHashAlgorithm.FNV_1A_COMPOSITE_1); + } + + private void validatePathType(String index, RemoteStoreEnums.PathType pathType) { + validatePathType(index, pathType, null); + } + + private void validatePathType( + String index, + RemoteStoreEnums.PathType pathType, + @Nullable RemoteStoreEnums.PathHashAlgorithm pathHashAlgorithm + ) { + ClusterState state = client().admin().cluster().prepareState().execute().actionGet().getState(); + // Validate that the remote_store custom data is present in index metadata for the created index. + Map remoteCustomData = state.metadata().index(index).getCustomData(IndexMetadata.REMOTE_STORE_CUSTOM_KEY); + assertNotNull(remoteCustomData); + assertEquals(pathType.name(), remoteCustomData.get(RemoteStoreEnums.PathType.NAME)); + if (Objects.nonNull(pathHashAlgorithm)) { + assertEquals(pathHashAlgorithm.name(), remoteCustomData.get(RemoteStoreEnums.PathHashAlgorithm.NAME)); + } + } + + public void testRestoreInSameRemoteStoreEnabledIndex() throws IOException { + String clusterManagerNode = internalCluster().startClusterManagerOnlyNode(); + String primary = internalCluster().startDataOnlyNode(); + String indexName1 = "testindex1"; + String indexName2 = "testindex2"; + String snapshotRepoName = "test-restore-snapshot-repo"; + String snapshotName1 = "test-restore-snapshot1"; + String snapshotName2 = "test-restore-snapshot2"; + Path absolutePath1 = randomRepoPath().toAbsolutePath(); + logger.info("Snapshot Path [{}]", absolutePath1); + String restoredIndexName2 = indexName2 + "-restored"; + + boolean enableShallowCopy = randomBoolean(); + createRepository(snapshotRepoName, "fs", getRepositorySettings(absolutePath1, enableShallowCopy)); + + Client client = client(); + Settings indexSettings = getIndexSettings(1, 0).build(); + createIndex(indexName1, indexSettings); + + Settings indexSettings2 = getIndexSettings(1, 0).build(); + createIndex(indexName2, indexSettings2); + + final int numDocsInIndex1 = 5; + final int numDocsInIndex2 = 6; + indexDocuments(client, indexName1, numDocsInIndex1); + indexDocuments(client, indexName2, numDocsInIndex2); + ensureGreen(indexName1, indexName2); + + internalCluster().startDataOnlyNode(); + logger.info("--> snapshot"); + SnapshotInfo snapshotInfo1 = createSnapshot( + snapshotRepoName, + snapshotName1, + new ArrayList<>(Arrays.asList(indexName1, indexName2)) + ); + assertThat(snapshotInfo1.successfulShards(), greaterThan(0)); + assertThat(snapshotInfo1.successfulShards(), equalTo(snapshotInfo1.totalShards())); + assertThat(snapshotInfo1.state(), equalTo(SnapshotState.SUCCESS)); + + updateRepository(snapshotRepoName, "fs", getRepositorySettings(absolutePath1, false)); + SnapshotInfo snapshotInfo2 = createSnapshot( + snapshotRepoName, + snapshotName2, + new ArrayList<>(Arrays.asList(indexName1, indexName2)) + ); + assertThat(snapshotInfo2.successfulShards(), greaterThan(0)); + assertThat(snapshotInfo2.successfulShards(), equalTo(snapshotInfo2.totalShards())); + assertThat(snapshotInfo2.state(), equalTo(SnapshotState.SUCCESS)); + + DeleteResponse deleteResponse = client().prepareDelete(indexName1, "0").execute().actionGet(); + assertEquals(deleteResponse.getResult(), DocWriteResponse.Result.DELETED); + indexDocuments(client, indexName1, numDocsInIndex1, numDocsInIndex1 + randomIntBetween(2, 5)); + ensureGreen(indexName1); + + assertAcked(client().admin().indices().prepareClose(indexName1)); + + RestoreSnapshotResponse restoreSnapshotResponse1 = client.admin() + .cluster() + .prepareRestoreSnapshot(snapshotRepoName, snapshotName1) + .setWaitForCompletion(false) + .setIndices(indexName1) + .get(); + RestoreSnapshotResponse restoreSnapshotResponse2 = client.admin() + .cluster() + .prepareRestoreSnapshot(snapshotRepoName, snapshotName2) + .setWaitForCompletion(false) + .setIndices(indexName2) + .setRenamePattern(indexName2) + .setRenameReplacement(restoredIndexName2) + .get(); + assertEquals(restoreSnapshotResponse1.status(), RestStatus.ACCEPTED); + assertEquals(restoreSnapshotResponse2.status(), RestStatus.ACCEPTED); + ensureGreen(indexName1, restoredIndexName2); + + assertRemoteSegmentsAndTranslogUploaded(restoredIndexName2); + assertDocsPresentInIndex(client, indexName1, numDocsInIndex1); + assertDocsPresentInIndex(client, restoredIndexName2, numDocsInIndex2); + // indexing some new docs and validating + indexDocuments(client, indexName1, numDocsInIndex1, numDocsInIndex1 + 2); + ensureGreen(indexName1); + assertDocsPresentInIndex(client, indexName1, numDocsInIndex1 + 2); + + // deleting data for restoredIndexName1 and restoring from remote store. + internalCluster().stopRandomNode(InternalTestCluster.nameFilter(primary)); + ensureRed(indexName1); + // Re-initialize client to make sure we are not using client from stopped node. + client = client(clusterManagerNode); + assertAcked(client.admin().indices().prepareClose(indexName1)); + client.admin() + .cluster() + .restoreRemoteStore(new RestoreRemoteStoreRequest().indices(indexName1).restoreAllShards(true), PlainActionFuture.newFuture()); + ensureYellowAndNoInitializingShards(indexName1); + ensureGreen(indexName1); + assertDocsPresentInIndex(client(), indexName1, numDocsInIndex1); + // indexing some new docs and validating + indexDocuments(client, indexName1, numDocsInIndex1 + 2, numDocsInIndex1 + 4); + ensureGreen(indexName1); + assertDocsPresentInIndex(client, indexName1, numDocsInIndex1 + 4); + } + + void assertRemoteSegmentsAndTranslogUploaded(String idx) throws IOException { + Client client = client(); + String translogPathFixedPrefix = RemoteStoreSettings.CLUSTER_REMOTE_STORE_TRANSLOG_PATH_PREFIX.get(getNodeSettings()); + String segmentsPathFixedPrefix = RemoteStoreSettings.CLUSTER_REMOTE_STORE_SEGMENTS_PATH_PREFIX.get(getNodeSettings()); + String path = getShardLevelBlobPath(client, idx, new BlobPath(), "0", TRANSLOG, METADATA, translogPathFixedPrefix).buildAsString(); + Path remoteTranslogMetadataPath = Path.of(remoteRepoPath + "/" + path); + path = getShardLevelBlobPath(client, idx, new BlobPath(), "0", TRANSLOG, DATA, translogPathFixedPrefix).buildAsString(); + Path remoteTranslogDataPath = Path.of(remoteRepoPath + "/" + path); + path = getShardLevelBlobPath(client, idx, new BlobPath(), "0", SEGMENTS, METADATA, segmentsPathFixedPrefix).buildAsString(); + Path segmentMetadataPath = Path.of(remoteRepoPath + "/" + path); + path = getShardLevelBlobPath(client, idx, new BlobPath(), "0", SEGMENTS, DATA, segmentsPathFixedPrefix).buildAsString(); + Path segmentDataPath = Path.of(remoteRepoPath + "/" + path); + + try ( + Stream translogMetadata = Files.list(remoteTranslogMetadataPath); + Stream translogData = Files.list(remoteTranslogDataPath); + Stream segmentMetadata = Files.list(segmentMetadataPath); + Stream segmentData = Files.list(segmentDataPath); + + ) { + assertTrue(translogData.count() > 0); + assertTrue(translogMetadata.count() > 0); + assertTrue(segmentMetadata.count() > 0); + assertTrue(segmentData.count() > 0); + } + + } + + public void testRemoteRestoreIndexRestoredFromSnapshot() throws IOException, ExecutionException, InterruptedException { + internalCluster().startClusterManagerOnlyNode(); + internalCluster().startDataOnlyNodes(2); + + String indexName1 = "testindex1"; + String snapshotRepoName = "test-restore-snapshot-repo"; + String snapshotName1 = "test-restore-snapshot1"; + Path absolutePath1 = randomRepoPath().toAbsolutePath(); + logger.info("Snapshot Path [{}]", absolutePath1); + + createRepository(snapshotRepoName, "fs", getRepositorySettings(absolutePath1, true)); + + Settings indexSettings = getIndexSettings(1, 0).build(); + createIndex(indexName1, indexSettings); + + final int numDocsInIndex1 = randomIntBetween(20, 30); + indexDocuments(client(), indexName1, numDocsInIndex1); + flushAndRefresh(indexName1); + ensureGreen(indexName1); + + logger.info("--> snapshot"); + SnapshotInfo snapshotInfo1 = createSnapshot(snapshotRepoName, snapshotName1, new ArrayList<>(Arrays.asList(indexName1))); + assertThat(snapshotInfo1.successfulShards(), greaterThan(0)); + assertThat(snapshotInfo1.successfulShards(), equalTo(snapshotInfo1.totalShards())); + assertThat(snapshotInfo1.state(), equalTo(SnapshotState.SUCCESS)); + + assertAcked(client().admin().indices().delete(new DeleteIndexRequest(indexName1)).get()); + assertFalse(indexExists(indexName1)); + + RestoreSnapshotResponse restoreSnapshotResponse1 = client().admin() + .cluster() + .prepareRestoreSnapshot(snapshotRepoName, snapshotName1) + .setWaitForCompletion(false) + .setIndices(indexName1) + .get(); + + assertEquals(restoreSnapshotResponse1.status(), RestStatus.ACCEPTED); + ensureGreen(indexName1); + assertDocsPresentInIndex(client(), indexName1, numDocsInIndex1); + + assertRemoteSegmentsAndTranslogUploaded(indexName1); + + // Clear the local data before stopping the node. This will make sure that remote translog is empty. + IndexShard indexShard = getIndexShard(primaryNodeName(indexName1), indexName1); + try (Stream files = Files.list(indexShard.shardPath().resolveTranslog())) { + IOUtils.deleteFilesIgnoringExceptions(files.collect(Collectors.toList())); + } + internalCluster().stopRandomNode(InternalTestCluster.nameFilter(primaryNodeName(indexName1))); + + ensureRed(indexName1); + + client().admin() + .cluster() + .restoreRemoteStore(new RestoreRemoteStoreRequest().indices(indexName1).restoreAllShards(false), PlainActionFuture.newFuture()); + + ensureGreen(indexName1); + assertDocsPresentInIndex(client(), indexName1, numDocsInIndex1); + } + + private IndexShard getIndexShard(String node, String indexName) { + final Index index = resolveIndex(indexName); + IndicesService indicesService = internalCluster().getInstance(IndicesService.class, node); + IndexService indexService = indicesService.indexService(index); + assertNotNull(indexService); + final Optional shardId = indexService.shardIds().stream().findFirst(); + return shardId.map(indexService::getShard).orElse(null); + } + + public void testRestoreShallowSnapshotRepository() throws ExecutionException, InterruptedException { + String indexName1 = "testindex1"; + String snapshotRepoName = "test-restore-snapshot-repo"; + String remoteStoreRepoNameUpdated = "test-rs-repo-updated" + TEST_REMOTE_STORE_REPO_SUFFIX; + String snapshotName1 = "test-restore-snapshot1"; + Path absolutePath1 = randomRepoPath().toAbsolutePath(); + Path absolutePath2 = randomRepoPath().toAbsolutePath(); + String[] pathTokens = absolutePath1.toString().split("/"); + String basePath = pathTokens[pathTokens.length - 1]; + Arrays.copyOf(pathTokens, pathTokens.length - 1); + Path location = PathUtils.get(String.join("/", pathTokens)); + pathTokens = absolutePath2.toString().split("/"); + String basePath2 = pathTokens[pathTokens.length - 1]; + Arrays.copyOf(pathTokens, pathTokens.length - 1); + Path location2 = PathUtils.get(String.join("/", pathTokens)); + logger.info("Path 1 [{}]", absolutePath1); + logger.info("Path 2 [{}]", absolutePath2); + String restoredIndexName1 = indexName1 + "-restored"; + + createRepository(snapshotRepoName, "fs", getRepositorySettings(location, basePath, true)); + + Client client = client(); + Settings indexSettings = Settings.builder() + .put(super.indexSettings()) + .put(IndexSettings.INDEX_REFRESH_INTERVAL_SETTING.getKey(), "300s") + .put(IndexMetadata.SETTING_REPLICATION_TYPE, ReplicationType.SEGMENT) + .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0) + .build(); + createIndex(indexName1, indexSettings); + + int numDocsInIndex1 = randomIntBetween(2, 5); + indexDocuments(client, indexName1, numDocsInIndex1); + + ensureGreen(indexName1); + + logger.info("--> snapshot"); + SnapshotInfo snapshotInfo1 = createSnapshot(snapshotRepoName, snapshotName1, new ArrayList<>(List.of(indexName1))); + assertThat(snapshotInfo1.successfulShards(), greaterThan(0)); + assertThat(snapshotInfo1.successfulShards(), equalTo(snapshotInfo1.totalShards())); + assertThat(snapshotInfo1.state(), equalTo(SnapshotState.SUCCESS)); + + client().admin().indices().close(Requests.closeIndexRequest(indexName1)).get(); + createRepository(remoteStoreRepoNameUpdated, "fs", remoteRepoPath); + RestoreSnapshotResponse restoreSnapshotResponse2 = client.admin() + .cluster() + .prepareRestoreSnapshot(snapshotRepoName, snapshotName1) + .setWaitForCompletion(true) + .setIndices(indexName1) + .setRenamePattern(indexName1) + .setRenameReplacement(restoredIndexName1) + .setSourceRemoteStoreRepository(remoteStoreRepoNameUpdated) + .get(); + + assertTrue(restoreSnapshotResponse2.getRestoreInfo().failedShards() == 0); + ensureGreen(restoredIndexName1); + assertDocsPresentInIndex(client, restoredIndexName1, numDocsInIndex1); + + // ensure recovery details are non-zero + RecoveryResponse recoveryResponse = client().admin().indices().prepareRecoveries(restoredIndexName1).execute().actionGet(); + assertEquals(1, recoveryResponse.getTotalShards()); + assertEquals(1, recoveryResponse.getSuccessfulShards()); + assertEquals(0, recoveryResponse.getFailedShards()); + assertEquals(1, recoveryResponse.shardRecoveryStates().size()); + assertTrue(recoveryResponse.shardRecoveryStates().containsKey(restoredIndexName1)); + assertEquals(1, recoveryResponse.shardRecoveryStates().get(restoredIndexName1).size()); + + RecoveryState recoveryState = recoveryResponse.shardRecoveryStates().get(restoredIndexName1).get(0); + assertEquals(RecoveryState.Stage.DONE, recoveryState.getStage()); + assertEquals(0, recoveryState.getShardId().getId()); + assertTrue(recoveryState.getPrimary()); + assertEquals(RecoverySource.Type.SNAPSHOT, recoveryState.getRecoverySource().getType()); + assertThat(recoveryState.getIndex().time(), greaterThanOrEqualTo(0L)); + + // ensure populated file details + assertTrue(recoveryState.getIndex().totalFileCount() > 0); + assertTrue(recoveryState.getIndex().totalRecoverFiles() > 0); + assertTrue(recoveryState.getIndex().recoveredFileCount() > 0); + assertThat(recoveryState.getIndex().recoveredFilesPercent(), greaterThanOrEqualTo(0.0f)); + assertThat(recoveryState.getIndex().recoveredFilesPercent(), lessThanOrEqualTo(100.0f)); + assertFalse(recoveryState.getIndex().fileDetails().isEmpty()); + + // ensure populated bytes details + assertTrue(recoveryState.getIndex().recoveredBytes() > 0L); + assertTrue(recoveryState.getIndex().totalBytes() > 0L); + assertTrue(recoveryState.getIndex().totalRecoverBytes() > 0L); + assertThat(recoveryState.getIndex().recoveredBytesPercent(), greaterThanOrEqualTo(0.0f)); + assertThat(recoveryState.getIndex().recoveredBytesPercent(), lessThanOrEqualTo(100.0f)); + + // indexing some new docs and validating + indexDocuments(client, restoredIndexName1, numDocsInIndex1, numDocsInIndex1 + 2); + ensureGreen(restoredIndexName1); + assertDocsPresentInIndex(client, restoredIndexName1, numDocsInIndex1 + 2); + } + + public void testRestoreShallowSnapshotIndexAfterSnapshot() throws ExecutionException, InterruptedException { + String indexName1 = "testindex1"; + String snapshotRepoName = "test-restore-snapshot-repo"; + String remoteStoreRepoNameUpdated = "test-rs-repo-updated" + TEST_REMOTE_STORE_REPO_SUFFIX; + String snapshotName1 = "test-restore-snapshot1"; + Path absolutePath1 = randomRepoPath().toAbsolutePath(); + Path absolutePath2 = randomRepoPath().toAbsolutePath(); + String[] pathTokens = absolutePath1.toString().split("/"); + String basePath = pathTokens[pathTokens.length - 1]; + Arrays.copyOf(pathTokens, pathTokens.length - 1); + Path location = PathUtils.get(String.join("/", pathTokens)); + pathTokens = absolutePath2.toString().split("/"); + String basePath2 = pathTokens[pathTokens.length - 1]; + Arrays.copyOf(pathTokens, pathTokens.length - 1); + Path location2 = PathUtils.get(String.join("/", pathTokens)); + logger.info("Path 1 [{}]", absolutePath1); + logger.info("Path 2 [{}]", absolutePath2); + String restoredIndexName1 = indexName1 + "-restored"; + + createRepository(snapshotRepoName, "fs", getRepositorySettings(location, basePath, true)); + + Client client = client(); + Settings indexSettings = Settings.builder() + .put(super.indexSettings()) + .put(IndexMetadata.SETTING_REPLICATION_TYPE, ReplicationType.SEGMENT) + .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0) + .build(); + createIndex(indexName1, indexSettings); + + int numDocsInIndex1 = randomIntBetween(2, 5); + indexDocuments(client, indexName1, numDocsInIndex1); + + ensureGreen(indexName1); + + logger.info("--> snapshot"); + SnapshotInfo snapshotInfo1 = createSnapshot(snapshotRepoName, snapshotName1, new ArrayList<>(List.of(indexName1))); + assertThat(snapshotInfo1.successfulShards(), greaterThan(0)); + assertThat(snapshotInfo1.successfulShards(), equalTo(snapshotInfo1.totalShards())); + assertThat(snapshotInfo1.state(), equalTo(SnapshotState.SUCCESS)); + + int extraNumDocsInIndex1 = randomIntBetween(20, 50); + indexDocuments(client, indexName1, extraNumDocsInIndex1); + refresh(indexName1); + + client().admin().indices().close(Requests.closeIndexRequest(indexName1)).get(); + createRepository(remoteStoreRepoNameUpdated, "fs", remoteRepoPath); + RestoreSnapshotResponse restoreSnapshotResponse2 = client.admin() + .cluster() + .prepareRestoreSnapshot(snapshotRepoName, snapshotName1) + .setWaitForCompletion(true) + .setIndices(indexName1) + .setRenamePattern(indexName1) + .setRenameReplacement(restoredIndexName1) + .setSourceRemoteStoreRepository(remoteStoreRepoNameUpdated) + .get(); + + assertTrue(restoreSnapshotResponse2.getRestoreInfo().failedShards() == 0); + ensureGreen(restoredIndexName1); + assertDocsPresentInIndex(client, restoredIndexName1, numDocsInIndex1); + + // indexing some new docs and validating + indexDocuments(client, restoredIndexName1, numDocsInIndex1, numDocsInIndex1 + 2); + ensureGreen(restoredIndexName1); + assertDocsPresentInIndex(client, restoredIndexName1, numDocsInIndex1 + 2); + } + + public void testInvalidRestoreRequestScenarios() throws Exception { + internalCluster().startClusterManagerOnlyNode(); + internalCluster().startDataOnlyNode(); + String index = "test-index"; + String snapshotRepo = "test-restore-snapshot-repo"; + String newRemoteStoreRepo = "test-new-rs-repo"; + String snapshotName1 = "test-restore-snapshot1"; + String snapshotName2 = "test-restore-snapshot2"; + Path absolutePath1 = randomRepoPath().toAbsolutePath(); + logger.info("Snapshot Path [{}]", absolutePath1); + String restoredIndex = index + "-restored"; + + createRepository(snapshotRepo, "fs", getRepositorySettings(absolutePath1, true)); + + Client client = client(); + Settings indexSettings = getIndexSettings(1, 0).build(); + createIndex(index, indexSettings); + + final int numDocsInIndex = 5; + indexDocuments(client, index, numDocsInIndex); + ensureGreen(index); + + internalCluster().startDataOnlyNode(); + logger.info("--> snapshot"); + + SnapshotInfo snapshotInfo = createSnapshot(snapshotRepo, snapshotName1, new ArrayList<>(List.of(index))); + assertThat(snapshotInfo.state(), equalTo(SnapshotState.SUCCESS)); + assertThat(snapshotInfo.successfulShards(), greaterThan(0)); + assertThat(snapshotInfo.successfulShards(), equalTo(snapshotInfo.totalShards())); + + updateRepository(snapshotRepo, "fs", getRepositorySettings(absolutePath1, false)); + SnapshotInfo snapshotInfo2 = createSnapshot(snapshotRepo, snapshotName2, new ArrayList<>(List.of(index))); + assertThat(snapshotInfo2.state(), equalTo(SnapshotState.SUCCESS)); + assertThat(snapshotInfo2.successfulShards(), greaterThan(0)); + assertThat(snapshotInfo2.successfulShards(), equalTo(snapshotInfo2.totalShards())); + + DeleteResponse deleteResponse = client().prepareDelete(index, "0").execute().actionGet(); + assertEquals(deleteResponse.getResult(), DocWriteResponse.Result.DELETED); + indexDocuments(client, index, numDocsInIndex, numDocsInIndex + randomIntBetween(2, 5)); + ensureGreen(index); + + // try index restore with remote store disabled + SnapshotRestoreException exception = expectThrows( + SnapshotRestoreException.class, + () -> client().admin() + .cluster() + .prepareRestoreSnapshot(snapshotRepo, snapshotName1) + .setWaitForCompletion(false) + .setIgnoreIndexSettings(SETTING_REMOTE_STORE_ENABLED) + .setIndices(index) + .setRenamePattern(index) + .setRenameReplacement(restoredIndex) + .get() + ); + assertTrue(exception.getMessage().contains("cannot remove setting [index.remote_store.enabled] on restore")); + + // try index restore with remote store repository modified + Settings remoteStoreIndexSettings = Settings.builder() + .put(IndexMetadata.SETTING_REMOTE_SEGMENT_STORE_REPOSITORY, newRemoteStoreRepo) + .build(); + + exception = expectThrows( + SnapshotRestoreException.class, + () -> client().admin() + .cluster() + .prepareRestoreSnapshot(snapshotRepo, snapshotName1) + .setWaitForCompletion(false) + .setIndexSettings(remoteStoreIndexSettings) + .setIndices(index) + .setRenamePattern(index) + .setRenameReplacement(restoredIndex) + .get() + ); + assertTrue(exception.getMessage().contains("cannot modify setting [index.remote_store.segment.repository]" + " on restore")); + + // try index restore with remote store repository and translog store repository disabled + exception = expectThrows( + SnapshotRestoreException.class, + () -> client().admin() + .cluster() + .prepareRestoreSnapshot(snapshotRepo, snapshotName1) + .setWaitForCompletion(false) + .setIgnoreIndexSettings( + IndexMetadata.SETTING_REMOTE_SEGMENT_STORE_REPOSITORY, + IndexMetadata.SETTING_REMOTE_TRANSLOG_STORE_REPOSITORY + ) + .setIndices(index) + .setRenamePattern(index) + .setRenameReplacement(restoredIndex) + .get() + ); + assertTrue(exception.getMessage().contains("cannot remove setting [index.remote_store.segment.repository]" + " on restore")); + } +} diff --git a/server/src/internalClusterTest/java/org/opensearch/snapshots/DeleteSnapshotIT.java b/server/src/internalClusterTest/java/org/opensearch/snapshots/DeleteSnapshotIT.java index 2331d52c3a1bc..26b30af4c2c50 100644 --- a/server/src/internalClusterTest/java/org/opensearch/snapshots/DeleteSnapshotIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/snapshots/DeleteSnapshotIT.java @@ -285,6 +285,7 @@ public void testDeleteMultipleShallowCopySnapshotsCase3() throws Exception { assert (getLockFilesInRemoteStore(remoteStoreEnabledIndexName, REMOTE_REPO_NAME).length == 0); } + @AwaitsFix(bugUrl = "https://github.com/opensearch-project/OpenSearch/issues/9208") public void testRemoteStoreCleanupForDeletedIndex() throws Exception { disableRepoConsistencyCheck("Remote store repository is being used in the test"); final Path remoteStoreRepoPath = randomRepoPath(); @@ -323,13 +324,15 @@ public void testRemoteStoreCleanupForDeletedIndex() throws Exception { final RepositoriesService repositoriesService = internalCluster().getCurrentClusterManagerNodeInstance(RepositoriesService.class); final BlobStoreRepository remoteStoreRepository = (BlobStoreRepository) repositoriesService.repository(REMOTE_REPO_NAME); + String segmentsPathFixedPrefix = RemoteStoreSettings.CLUSTER_REMOTE_STORE_SEGMENTS_PATH_PREFIX.get(getNodeSettings()); BlobPath shardLevelBlobPath = getShardLevelBlobPath( client(), remoteStoreEnabledIndexName, remoteStoreRepository.basePath(), "0", SEGMENTS, - LOCK_FILES + LOCK_FILES, + segmentsPathFixedPrefix ); BlobContainer blobContainer = remoteStoreRepository.blobStore().blobContainer(shardLevelBlobPath); String[] lockFiles; diff --git a/server/src/internalClusterTest/java/org/opensearch/snapshots/RemoteIndexSnapshotStatusApiIT.java b/server/src/internalClusterTest/java/org/opensearch/snapshots/RemoteIndexSnapshotStatusApiIT.java index 8e2580aba1745..e84de36df2fca 100644 --- a/server/src/internalClusterTest/java/org/opensearch/snapshots/RemoteIndexSnapshotStatusApiIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/snapshots/RemoteIndexSnapshotStatusApiIT.java @@ -32,20 +32,28 @@ package org.opensearch.snapshots; +import org.opensearch.action.admin.cluster.settings.ClusterUpdateSettingsRequest; import org.opensearch.action.admin.cluster.snapshots.create.CreateSnapshotResponse; import org.opensearch.action.admin.cluster.snapshots.status.SnapshotIndexShardStage; import org.opensearch.action.admin.cluster.snapshots.status.SnapshotIndexShardStatus; +import org.opensearch.action.admin.cluster.snapshots.status.SnapshotIndexStatus; import org.opensearch.action.admin.cluster.snapshots.status.SnapshotStatus; import org.opensearch.cluster.SnapshotsInProgress; import org.opensearch.common.action.ActionFuture; import org.opensearch.common.settings.Settings; +import org.opensearch.indices.RemoteStoreSettings; +import org.opensearch.repositories.blobstore.BlobStoreRepository; import org.opensearch.test.OpenSearchIntegTestCase; import org.opensearch.threadpool.ThreadPool; import org.junit.Before; import java.nio.file.Path; +import java.util.Map; +import java.util.concurrent.TimeUnit; import static org.opensearch.remotestore.RemoteStoreBaseIntegTestCase.remoteStoreClusterSettings; +import static org.opensearch.snapshots.SnapshotsService.MAX_SHARDS_ALLOWED_IN_STATUS_API; +import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertAcked; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.greaterThan; import static org.hamcrest.Matchers.is; @@ -192,6 +200,110 @@ public void testStatusAPICallInProgressShallowSnapshot() throws Exception { createSnapshotResponseActionFuture.actionGet(); } + public void testStatusAPICallForShallowV2Snapshot() throws Exception { + disableRepoConsistencyCheck("Remote store repository is being used for the test"); + Settings pinnedTimestampSettings = Settings.builder() + .put(RemoteStoreSettings.CLUSTER_REMOTE_STORE_PINNED_TIMESTAMP_ENABLED.getKey(), true) + .build(); + internalCluster().startClusterManagerOnlyNode(pinnedTimestampSettings); + internalCluster().startDataOnlyNodes(2, pinnedTimestampSettings); + + final String index1 = "remote-index-1"; + final String index2 = "remote-index-2"; + final String index3 = "remote-index-3"; + final String snapshotRepoName = "snapshot-repo-name"; + final String snapshot = "snapshot"; + + logger.info("Create repository for shallow V2 snapshots"); + Settings.Builder snapshotV2RepoSettings = snapshotRepoSettingsForShallowCopy().put( + BlobStoreRepository.SHALLOW_SNAPSHOT_V2.getKey(), + Boolean.TRUE + ); + createRepository(snapshotRepoName, "fs", snapshotV2RepoSettings); + + final Settings remoteStoreEnabledIndexSettings = getRemoteStoreBackedIndexSettings(); + createIndex(index1, remoteStoreEnabledIndexSettings); + createIndex(index2, remoteStoreEnabledIndexSettings); + createIndex(index3, remoteStoreEnabledIndexSettings); + ensureGreen(); + + logger.info("Indexing some data"); + for (int i = 0; i < 50; i++) { + index(index1, "_doc", Integer.toString(i), "foo", "bar" + i); + index(index2, "_doc", Integer.toString(i), "foo", "bar" + i); + index(index3, "_doc", Integer.toString(i), "foo", "bar" + i); + } + refresh(); + + SnapshotInfo snapshotInfo = createFullSnapshot(snapshotRepoName, snapshot); + assertTrue(snapshotInfo.getPinnedTimestamp() > 0); // to assert creation of a shallow v2 snapshot + + logger.info("Set MAX_SHARDS_ALLOWED_IN_STATUS_API to a low value"); + ClusterUpdateSettingsRequest updateSettingsRequest = new ClusterUpdateSettingsRequest(); + updateSettingsRequest.persistentSettings(Settings.builder().put(MAX_SHARDS_ALLOWED_IN_STATUS_API.getKey(), 2)); + assertAcked(client().admin().cluster().updateSettings(updateSettingsRequest).actionGet()); + + assertBusy(() -> { + // without index filter + // although no. of shards in snapshot (3) is greater than the max value allowed in a status api call, the request does not fail + SnapshotStatus snapshotStatusWithoutIndexFilter = client().admin() + .cluster() + .prepareSnapshotStatus(snapshotRepoName) + .setSnapshots(snapshot) + .execute() + .actionGet() + .getSnapshots() + .get(0); + + assertShallowV2SnapshotStatus(snapshotStatusWithoutIndexFilter, false); + + // with index filter + SnapshotStatus snapshotStatusWithIndexFilter = client().admin() + .cluster() + .prepareSnapshotStatus(snapshotRepoName) + .setSnapshots(snapshot) + .setIndices(index1, index2) + .execute() + .actionGet() + .getSnapshots() + .get(0); + + assertShallowV2SnapshotStatus(snapshotStatusWithIndexFilter, true); + + }, 1, TimeUnit.MINUTES); + + } + + private void assertShallowV2SnapshotStatus(SnapshotStatus snapshotStatus, boolean hasIndexFilter) { + if (hasIndexFilter) { + assertEquals(0, snapshotStatus.getStats().getTotalSize()); + } else { + // TODO: after adding primary store size at the snapshot level, total size here should be > 0 + } + // assert that total and incremental values of file count and size_in_bytes are 0 at index and shard levels + assertEquals(0, snapshotStatus.getStats().getTotalFileCount()); + assertEquals(0, snapshotStatus.getStats().getIncrementalSize()); + assertEquals(0, snapshotStatus.getStats().getIncrementalFileCount()); + + for (Map.Entry entry : snapshotStatus.getIndices().entrySet()) { + // index level + SnapshotIndexStatus snapshotIndexStatus = entry.getValue(); + assertEquals(0, snapshotIndexStatus.getStats().getTotalSize()); + assertEquals(0, snapshotIndexStatus.getStats().getTotalFileCount()); + assertEquals(0, snapshotIndexStatus.getStats().getIncrementalSize()); + assertEquals(0, snapshotIndexStatus.getStats().getIncrementalFileCount()); + + for (SnapshotIndexShardStatus snapshotIndexShardStatus : snapshotStatus.getShards()) { + // shard level + assertEquals(0, snapshotIndexShardStatus.getStats().getTotalSize()); + assertEquals(0, snapshotIndexShardStatus.getStats().getTotalFileCount()); + assertEquals(0, snapshotIndexShardStatus.getStats().getIncrementalSize()); + assertEquals(0, snapshotIndexShardStatus.getStats().getIncrementalFileCount()); + assertEquals(SnapshotIndexShardStage.DONE, snapshotIndexShardStatus.getStage()); + } + } + } + private static SnapshotIndexShardStatus stateFirstShard(SnapshotStatus snapshotStatus, String indexName) { return snapshotStatus.getIndices().get(indexName).getShards().get(0); } diff --git a/server/src/internalClusterTest/java/org/opensearch/snapshots/SnapshotStatusApisIT.java b/server/src/internalClusterTest/java/org/opensearch/snapshots/SnapshotStatusApisIT.java index 3c75b30580c30..5a043e69e9735 100644 --- a/server/src/internalClusterTest/java/org/opensearch/snapshots/SnapshotStatusApisIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/snapshots/SnapshotStatusApisIT.java @@ -33,11 +33,14 @@ package org.opensearch.snapshots; import org.opensearch.Version; +import org.opensearch.action.ActionRequestValidationException; +import org.opensearch.action.admin.cluster.settings.ClusterUpdateSettingsRequest; import org.opensearch.action.admin.cluster.snapshots.create.CreateSnapshotResponse; import org.opensearch.action.admin.cluster.snapshots.get.GetSnapshotsRequest; import org.opensearch.action.admin.cluster.snapshots.get.GetSnapshotsResponse; import org.opensearch.action.admin.cluster.snapshots.status.SnapshotIndexShardStage; import org.opensearch.action.admin.cluster.snapshots.status.SnapshotIndexShardStatus; +import org.opensearch.action.admin.cluster.snapshots.status.SnapshotIndexStatus; import org.opensearch.action.admin.cluster.snapshots.status.SnapshotStats; import org.opensearch.action.admin.cluster.snapshots.status.SnapshotStatus; import org.opensearch.action.admin.cluster.snapshots.status.SnapshotsStatusResponse; @@ -49,6 +52,8 @@ import org.opensearch.common.util.io.IOUtils; import org.opensearch.core.common.Strings; import org.opensearch.core.common.unit.ByteSizeUnit; +import org.opensearch.core.rest.RestStatus; +import org.opensearch.index.IndexNotFoundException; import org.opensearch.repositories.IndexId; import org.opensearch.repositories.blobstore.BlobStoreRepository; import org.opensearch.threadpool.ThreadPool; @@ -60,9 +65,12 @@ import java.util.Collections; import java.util.List; import java.util.Locale; +import java.util.Map; +import java.util.Set; import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; +import static org.opensearch.snapshots.SnapshotsService.MAX_SHARDS_ALLOWED_IN_STATUS_API; import static org.opensearch.test.OpenSearchIntegTestCase.resolvePath; import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertAcked; import static org.hamcrest.Matchers.equalTo; @@ -564,6 +572,194 @@ public void testGetSnapshotsRequest() throws Exception { waitForCompletion(repositoryName, inProgressSnapshot, TimeValue.timeValueSeconds(60)); } + public void testSnapshotStatusApiFailureForTooManyShardsAcrossSnapshots() throws Exception { + String repositoryName = "test-repo"; + String index1 = "test-idx-1"; + String index2 = "test-idx-2"; + String index3 = "test-idx-3"; + createRepository(repositoryName, "fs"); + + logger.info("Create indices"); + createIndex(index1, index2, index3); + ensureGreen(); + + logger.info("Indexing some data"); + for (int i = 0; i < 10; i++) { + index(index1, "_doc", Integer.toString(i), "foo", "bar" + i); + index(index2, "_doc", Integer.toString(i), "foo", "baz" + i); + index(index3, "_doc", Integer.toString(i), "foo", "baz" + i); + } + refresh(); + String snapshot1 = "test-snap-1"; + String snapshot2 = "test-snap-2"; + createSnapshot(repositoryName, snapshot1, List.of(index1, index2, index3)); + createSnapshot(repositoryName, snapshot2, List.of(index1, index2)); + + logger.info("Set MAX_SHARDS_ALLOWED_IN_STATUS_API to a low value"); + ClusterUpdateSettingsRequest updateSettingsRequest = new ClusterUpdateSettingsRequest(); + updateSettingsRequest.persistentSettings(Settings.builder().put(MAX_SHARDS_ALLOWED_IN_STATUS_API.getKey(), 2)); + assertAcked(client().admin().cluster().updateSettings(updateSettingsRequest).actionGet()); + + // across a single snapshot + assertBusy(() -> { + TooManyShardsInSnapshotsStatusException exception = expectThrows( + TooManyShardsInSnapshotsStatusException.class, + () -> client().admin().cluster().prepareSnapshotStatus(repositoryName).setSnapshots(snapshot1).execute().actionGet() + ); + assertEquals(exception.status(), RestStatus.REQUEST_ENTITY_TOO_LARGE); + assertTrue( + exception.getMessage().endsWith(" is more than the maximum allowed value of shard count [2] for snapshot status request") + ); + }, 1, TimeUnit.MINUTES); + + // across multiple snapshots + assertBusy(() -> { + TooManyShardsInSnapshotsStatusException exception = expectThrows( + TooManyShardsInSnapshotsStatusException.class, + () -> client().admin() + .cluster() + .prepareSnapshotStatus(repositoryName) + .setSnapshots(snapshot1, snapshot2) + .execute() + .actionGet() + ); + assertEquals(exception.status(), RestStatus.REQUEST_ENTITY_TOO_LARGE); + assertTrue( + exception.getMessage().endsWith(" is more than the maximum allowed value of shard count [2] for snapshot status request") + ); + }, 1, TimeUnit.MINUTES); + + logger.info("Reset MAX_SHARDS_ALLOWED_IN_STATUS_API to default value"); + updateSettingsRequest.persistentSettings(Settings.builder().putNull(MAX_SHARDS_ALLOWED_IN_STATUS_API.getKey())); + assertAcked(internalCluster().client().admin().cluster().updateSettings(updateSettingsRequest).actionGet()); + } + + public void testSnapshotStatusForIndexFilter() throws Exception { + String repositoryName = "test-repo"; + String index1 = "test-idx-1"; + String index2 = "test-idx-2"; + String index3 = "test-idx-3"; + createRepository(repositoryName, "fs"); + + logger.info("Create indices"); + createIndex(index1, index2, index3); + ensureGreen(); + + logger.info("Indexing some data"); + for (int i = 0; i < 10; i++) { + index(index1, "_doc", Integer.toString(i), "foo", "bar" + i); + index(index2, "_doc", Integer.toString(i), "foo", "baz" + i); + index(index3, "_doc", Integer.toString(i), "foo", "baz" + i); + } + refresh(); + String snapshot = "test-snap-1"; + createSnapshot(repositoryName, snapshot, List.of(index1, index2, index3)); + + assertBusy(() -> { + SnapshotStatus snapshotsStatus = client().admin() + .cluster() + .prepareSnapshotStatus(repositoryName) + .setSnapshots(snapshot) + .setIndices(index1, index2) + .get() + .getSnapshots() + .get(0); + Map snapshotIndexStatusMap = snapshotsStatus.getIndices(); + // Although the snapshot contains 3 indices, the response of status api call only contains results for 2 + assertEquals(snapshotIndexStatusMap.size(), 2); + assertEquals(snapshotIndexStatusMap.keySet(), Set.of(index1, index2)); + }, 1, TimeUnit.MINUTES); + } + + public void testSnapshotStatusFailuresWithIndexFilter() throws Exception { + String repositoryName = "test-repo"; + String index1 = "test-idx-1"; + String index2 = "test-idx-2"; + String index3 = "test-idx-3"; + createRepository(repositoryName, "fs"); + + logger.info("Create indices"); + createIndex(index1, index2, index3); + ensureGreen(); + + logger.info("Indexing some data"); + for (int i = 0; i < 10; i++) { + index(index1, "_doc", Integer.toString(i), "foo", "bar" + i); + index(index2, "_doc", Integer.toString(i), "foo", "baz" + i); + index(index3, "_doc", Integer.toString(i), "foo", "baz" + i); + } + refresh(); + String snapshot1 = "test-snap-1"; + String snapshot2 = "test-snap-2"; + createSnapshot(repositoryName, snapshot1, List.of(index1, index2, index3)); + createSnapshot(repositoryName, snapshot2, List.of(index1)); + + assertBusy(() -> { + // failure due to passing index filter for multiple snapshots + ActionRequestValidationException ex = expectThrows( + ActionRequestValidationException.class, + () -> client().admin() + .cluster() + .prepareSnapshotStatus(repositoryName) + .setSnapshots(snapshot1, snapshot2) + .setIndices(index1, index2, index3) + .execute() + .actionGet() + ); + String cause = "index list filter is supported only for a single snapshot"; + assertTrue(ex.getMessage().contains(cause)); + }, 1, TimeUnit.MINUTES); + + assertBusy(() -> { + // failure due to index not found in snapshot + IndexNotFoundException ex = expectThrows( + IndexNotFoundException.class, + () -> client().admin() + .cluster() + .prepareSnapshotStatus(repositoryName) + .setSnapshots(snapshot2) + .setIndices(index1, index2, index3) + .execute() + .actionGet() + ); + assertEquals(ex.status(), RestStatus.NOT_FOUND); + String cause = String.format( + Locale.ROOT, + "indices [%s] missing in snapshot [%s] of repository [%s]", + String.join(", ", List.of(index2, index3)), + snapshot2, + repositoryName + ); + assertEquals(cause, ex.getCause().getMessage()); + + }, 1, TimeUnit.MINUTES); + + assertBusy(() -> { + // failure due to too many shards requested + logger.info("Set MAX_SHARDS_ALLOWED_IN_STATUS_API to a low value"); + ClusterUpdateSettingsRequest updateSettingsRequest = new ClusterUpdateSettingsRequest(); + updateSettingsRequest.persistentSettings(Settings.builder().put(MAX_SHARDS_ALLOWED_IN_STATUS_API.getKey(), 2)); + assertAcked(client().admin().cluster().updateSettings(updateSettingsRequest).actionGet()); + + TooManyShardsInSnapshotsStatusException ex = expectThrows( + TooManyShardsInSnapshotsStatusException.class, + () -> client().admin() + .cluster() + .prepareSnapshotStatus(repositoryName) + .setSnapshots(snapshot1) + .setIndices(index1, index2, index3) + .execute() + .actionGet() + ); + assertEquals(ex.status(), RestStatus.REQUEST_ENTITY_TOO_LARGE); + assertTrue(ex.getMessage().endsWith(" is more than the maximum allowed value of shard count [2] for snapshot status request")); + + logger.info("Reset MAX_SHARDS_ALLOWED_IN_STATUS_API to default value"); + updateSettingsRequest.persistentSettings(Settings.builder().putNull(MAX_SHARDS_ALLOWED_IN_STATUS_API.getKey())); + assertAcked(internalCluster().client().admin().cluster().updateSettings(updateSettingsRequest).actionGet()); + }, 2, TimeUnit.MINUTES); + } + private static SnapshotIndexShardStatus stateFirstShard(SnapshotStatus snapshotStatus, String indexName) { return snapshotStatus.getIndices().get(indexName).getShards().get(0); } diff --git a/server/src/main/java/org/opensearch/OpenSearchServerException.java b/server/src/main/java/org/opensearch/OpenSearchServerException.java index b0ab62259d5c0..a1fc61834f69b 100644 --- a/server/src/main/java/org/opensearch/OpenSearchServerException.java +++ b/server/src/main/java/org/opensearch/OpenSearchServerException.java @@ -1210,6 +1210,14 @@ public static void registerExceptions() { V_2_17_0 ) ); + registerExceptionHandle( + new OpenSearchExceptionHandle( + org.opensearch.snapshots.TooManyShardsInSnapshotsStatusException.class, + org.opensearch.snapshots.TooManyShardsInSnapshotsStatusException::new, + 175, + V_2_17_0 + ) + ); registerExceptionHandle( new OpenSearchExceptionHandle( org.opensearch.cluster.block.IndexCreateBlockException.class, diff --git a/server/src/main/java/org/opensearch/action/ActionModule.java b/server/src/main/java/org/opensearch/action/ActionModule.java index c86e6580122d5..fbf90b97d1e8f 100644 --- a/server/src/main/java/org/opensearch/action/ActionModule.java +++ b/server/src/main/java/org/opensearch/action/ActionModule.java @@ -85,7 +85,9 @@ import org.opensearch.action.admin.cluster.reroute.TransportClusterRerouteAction; import org.opensearch.action.admin.cluster.settings.ClusterUpdateSettingsAction; import org.opensearch.action.admin.cluster.settings.TransportClusterUpdateSettingsAction; +import org.opensearch.action.admin.cluster.shards.CatShardsAction; import org.opensearch.action.admin.cluster.shards.ClusterSearchShardsAction; +import org.opensearch.action.admin.cluster.shards.TransportCatShardsAction; import org.opensearch.action.admin.cluster.shards.TransportClusterSearchShardsAction; import org.opensearch.action.admin.cluster.shards.routing.weighted.delete.ClusterDeleteWeightedRoutingAction; import org.opensearch.action.admin.cluster.shards.routing.weighted.delete.TransportDeleteWeightedRoutingAction; @@ -646,6 +648,7 @@ public void reg actions.register(ClusterGetWeightedRoutingAction.INSTANCE, TransportGetWeightedRoutingAction.class); actions.register(ClusterDeleteWeightedRoutingAction.INSTANCE, TransportDeleteWeightedRoutingAction.class); actions.register(IndicesStatsAction.INSTANCE, TransportIndicesStatsAction.class); + actions.register(CatShardsAction.INSTANCE, TransportCatShardsAction.class); actions.register(IndicesSegmentsAction.INSTANCE, TransportIndicesSegmentsAction.class); actions.register(IndicesShardStoresAction.INSTANCE, TransportIndicesShardStoresAction.class); actions.register(CreateIndexAction.INSTANCE, TransportCreateIndexAction.class); diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/allocation/ClusterAllocationExplanation.java b/server/src/main/java/org/opensearch/action/admin/cluster/allocation/ClusterAllocationExplanation.java index 3c8f07613561d..70a223d60069a 100644 --- a/server/src/main/java/org/opensearch/action/admin/cluster/allocation/ClusterAllocationExplanation.java +++ b/server/src/main/java/org/opensearch/action/admin/cluster/allocation/ClusterAllocationExplanation.java @@ -95,7 +95,7 @@ public ClusterAllocationExplanation(StreamInput in) throws IOException { @Override public void writeTo(StreamOutput out) throws IOException { shardRouting.writeTo(out); - out.writeOptionalWriteable(currentNode); + out.writeOptionalWriteable((stream, node) -> node.writeToWithAttribute(stream), currentNode); out.writeOptionalWriteable(relocationTargetNode); out.writeOptionalWriteable(clusterInfo); shardAllocationDecision.writeTo(out); diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/repositories/cleanup/TransportCleanupRepositoryAction.java b/server/src/main/java/org/opensearch/action/admin/cluster/repositories/cleanup/TransportCleanupRepositoryAction.java index 774bffa10da4f..0bf40d0af01dc 100644 --- a/server/src/main/java/org/opensearch/action/admin/cluster/repositories/cleanup/TransportCleanupRepositoryAction.java +++ b/server/src/main/java/org/opensearch/action/admin/cluster/repositories/cleanup/TransportCleanupRepositoryAction.java @@ -53,6 +53,7 @@ import org.opensearch.core.action.ActionListener; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.index.store.lockmanager.RemoteStoreLockManagerFactory; +import org.opensearch.indices.RemoteStoreSettings; import org.opensearch.repositories.RepositoriesService; import org.opensearch.repositories.Repository; import org.opensearch.repositories.RepositoryCleanupResult; @@ -109,7 +110,8 @@ public TransportCleanupRepositoryAction( SnapshotsService snapshotsService, ThreadPool threadPool, ActionFilters actionFilters, - IndexNameExpressionResolver indexNameExpressionResolver + IndexNameExpressionResolver indexNameExpressionResolver, + RemoteStoreSettings remoteStoreSettings ) { super( CleanupRepositoryAction.NAME, @@ -122,7 +124,10 @@ public TransportCleanupRepositoryAction( ); this.repositoriesService = repositoriesService; this.snapshotsService = snapshotsService; - this.remoteStoreLockManagerFactory = new RemoteStoreLockManagerFactory(() -> repositoriesService); + this.remoteStoreLockManagerFactory = new RemoteStoreLockManagerFactory( + () -> repositoriesService, + remoteStoreSettings.getSegmentsPathFixedPrefix() + ); // We add a state applier that will remove any dangling repository cleanup actions on cluster-manager failover. // This is safe to do since cleanups will increment the repository state id before executing any operations to prevent concurrent // operations from corrupting the repository. This is the same safety mechanism used by snapshot deletes. diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/shards/CatShardsAction.java b/server/src/main/java/org/opensearch/action/admin/cluster/shards/CatShardsAction.java new file mode 100644 index 0000000000000..381a950fe9a79 --- /dev/null +++ b/server/src/main/java/org/opensearch/action/admin/cluster/shards/CatShardsAction.java @@ -0,0 +1,25 @@ +/* + * 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.action.admin.cluster.shards; + +import org.opensearch.action.ActionType; + +/** + * Transport action for cat shards + * + * @opensearch.internal + */ +public class CatShardsAction extends ActionType { + public static final CatShardsAction INSTANCE = new CatShardsAction(); + public static final String NAME = "cluster:monitor/shards"; + + private CatShardsAction() { + super(NAME, CatShardsResponse::new); + } +} diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/shards/CatShardsRequest.java b/server/src/main/java/org/opensearch/action/admin/cluster/shards/CatShardsRequest.java new file mode 100644 index 0000000000000..49299777db8ae --- /dev/null +++ b/server/src/main/java/org/opensearch/action/admin/cluster/shards/CatShardsRequest.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.action.admin.cluster.shards; + +import org.opensearch.action.ActionRequestValidationException; +import org.opensearch.action.support.clustermanager.ClusterManagerNodeReadRequest; +import org.opensearch.common.unit.TimeValue; +import org.opensearch.core.common.io.stream.StreamInput; +import org.opensearch.core.tasks.TaskId; +import org.opensearch.rest.action.admin.cluster.ClusterAdminTask; + +import java.io.IOException; +import java.util.Map; + +/** + * A request of _cat/shards. + * + * @opensearch.api + */ +public class CatShardsRequest extends ClusterManagerNodeReadRequest { + + private String[] indices; + private TimeValue cancelAfterTimeInterval; + + public CatShardsRequest() {} + + public CatShardsRequest(StreamInput in) throws IOException { + super(in); + } + + @Override + public ActionRequestValidationException validate() { + return null; + } + + public void setIndices(String[] indices) { + this.indices = indices; + } + + public String[] getIndices() { + return this.indices; + } + + public void setCancelAfterTimeInterval(TimeValue timeout) { + this.cancelAfterTimeInterval = timeout; + } + + public TimeValue getCancelAfterTimeInterval() { + return this.cancelAfterTimeInterval; + } + + @Override + public ClusterAdminTask createTask(long id, String type, String action, TaskId parentTaskId, Map headers) { + return new ClusterAdminTask(id, type, action, parentTaskId, headers, this.cancelAfterTimeInterval); + } +} diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/shards/CatShardsResponse.java b/server/src/main/java/org/opensearch/action/admin/cluster/shards/CatShardsResponse.java new file mode 100644 index 0000000000000..3dd88a2cda037 --- /dev/null +++ b/server/src/main/java/org/opensearch/action/admin/cluster/shards/CatShardsResponse.java @@ -0,0 +1,57 @@ +/* + * 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.action.admin.cluster.shards; + +import org.opensearch.action.admin.cluster.state.ClusterStateResponse; +import org.opensearch.action.admin.indices.stats.IndicesStatsResponse; +import org.opensearch.core.action.ActionResponse; +import org.opensearch.core.common.io.stream.StreamInput; +import org.opensearch.core.common.io.stream.StreamOutput; + +import java.io.IOException; + +/** + * A response of a cat shards request. + * + * @opensearch.api + */ +public class CatShardsResponse extends ActionResponse { + + private ClusterStateResponse clusterStateResponse = null; + + private IndicesStatsResponse indicesStatsResponse = null; + + public CatShardsResponse() {} + + public CatShardsResponse(StreamInput in) throws IOException { + super(in); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + clusterStateResponse.writeTo(out); + indicesStatsResponse.writeTo(out); + } + + public void setClusterStateResponse(ClusterStateResponse clusterStateResponse) { + this.clusterStateResponse = clusterStateResponse; + } + + public ClusterStateResponse getClusterStateResponse() { + return this.clusterStateResponse; + } + + public void setIndicesStatsResponse(IndicesStatsResponse indicesStatsResponse) { + this.indicesStatsResponse = indicesStatsResponse; + } + + public IndicesStatsResponse getIndicesStatsResponse() { + return this.indicesStatsResponse; + } +} diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/shards/TransportCatShardsAction.java b/server/src/main/java/org/opensearch/action/admin/cluster/shards/TransportCatShardsAction.java new file mode 100644 index 0000000000000..224d3cbc5f10a --- /dev/null +++ b/server/src/main/java/org/opensearch/action/admin/cluster/shards/TransportCatShardsAction.java @@ -0,0 +1,110 @@ +/* + * 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.action.admin.cluster.shards; + +import org.opensearch.action.admin.cluster.state.ClusterStateRequest; +import org.opensearch.action.admin.cluster.state.ClusterStateResponse; +import org.opensearch.action.admin.indices.stats.IndicesStatsRequest; +import org.opensearch.action.admin.indices.stats.IndicesStatsResponse; +import org.opensearch.action.support.ActionFilters; +import org.opensearch.action.support.HandledTransportAction; +import org.opensearch.action.support.TimeoutTaskCancellationUtility; +import org.opensearch.client.node.NodeClient; +import org.opensearch.common.inject.Inject; +import org.opensearch.core.action.ActionListener; +import org.opensearch.core.action.NotifyOnceListener; +import org.opensearch.tasks.CancellableTask; +import org.opensearch.tasks.Task; +import org.opensearch.transport.TransportService; + +/** + * Perform cat shards action + * + * @opensearch.internal + */ +public class TransportCatShardsAction extends HandledTransportAction { + + private final NodeClient client; + + @Inject + public TransportCatShardsAction(NodeClient client, TransportService transportService, ActionFilters actionFilters) { + super(CatShardsAction.NAME, transportService, actionFilters, CatShardsRequest::new); + this.client = client; + } + + @Override + public void doExecute(Task parentTask, CatShardsRequest shardsRequest, ActionListener listener) { + final ClusterStateRequest clusterStateRequest = new ClusterStateRequest(); + clusterStateRequest.setShouldCancelOnTimeout(true); + clusterStateRequest.local(shardsRequest.local()); + clusterStateRequest.clusterManagerNodeTimeout(shardsRequest.clusterManagerNodeTimeout()); + clusterStateRequest.clear().nodes(true).routingTable(true).indices(shardsRequest.getIndices()); + assert parentTask instanceof CancellableTask; + clusterStateRequest.setParentTask(client.getLocalNodeId(), parentTask.getId()); + + ActionListener originalListener = new NotifyOnceListener() { + @Override + protected void innerOnResponse(CatShardsResponse catShardsResponse) { + listener.onResponse(catShardsResponse); + } + + @Override + protected void innerOnFailure(Exception e) { + listener.onFailure(e); + } + }; + ActionListener cancellableListener = TimeoutTaskCancellationUtility.wrapWithCancellationListener( + client, + (CancellableTask) parentTask, + ((CancellableTask) parentTask).getCancellationTimeout(), + originalListener, + e -> { + originalListener.onFailure(e); + } + ); + CatShardsResponse catShardsResponse = new CatShardsResponse(); + try { + client.admin().cluster().state(clusterStateRequest, new ActionListener() { + @Override + public void onResponse(ClusterStateResponse clusterStateResponse) { + catShardsResponse.setClusterStateResponse(clusterStateResponse); + IndicesStatsRequest indicesStatsRequest = new IndicesStatsRequest(); + indicesStatsRequest.setShouldCancelOnTimeout(true); + indicesStatsRequest.all(); + indicesStatsRequest.indices(shardsRequest.getIndices()); + indicesStatsRequest.setParentTask(client.getLocalNodeId(), parentTask.getId()); + try { + client.admin().indices().stats(indicesStatsRequest, new ActionListener() { + @Override + public void onResponse(IndicesStatsResponse indicesStatsResponse) { + catShardsResponse.setIndicesStatsResponse(indicesStatsResponse); + cancellableListener.onResponse(catShardsResponse); + } + + @Override + public void onFailure(Exception e) { + cancellableListener.onFailure(e); + } + }); + } catch (Exception e) { + cancellableListener.onFailure(e); + } + } + + @Override + public void onFailure(Exception e) { + cancellableListener.onFailure(e); + } + }); + } catch (Exception e) { + cancellableListener.onFailure(e); + } + + } +} diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/snapshots/restore/RestoreSnapshotRequest.java b/server/src/main/java/org/opensearch/action/admin/cluster/snapshots/restore/RestoreSnapshotRequest.java index 492ef86bb7843..409c48cabad35 100644 --- a/server/src/main/java/org/opensearch/action/admin/cluster/snapshots/restore/RestoreSnapshotRequest.java +++ b/server/src/main/java/org/opensearch/action/admin/cluster/snapshots/restore/RestoreSnapshotRequest.java @@ -121,6 +121,8 @@ private static StorageType fromString(String string) { private StorageType storageType = StorageType.LOCAL; @Nullable private String sourceRemoteStoreRepository = null; + @Nullable + private String sourceRemoteTranslogRepository = null; @Nullable // if any snapshot UUID will do private String snapshotUuid; @@ -159,6 +161,9 @@ public RestoreSnapshotRequest(StreamInput in) throws IOException { if (in.getVersion().onOrAfter(Version.V_2_10_0)) { sourceRemoteStoreRepository = in.readOptionalString(); } + if (in.getVersion().onOrAfter(Version.CURRENT)) { + sourceRemoteTranslogRepository = in.readOptionalString(); + } } @Override @@ -183,6 +188,9 @@ public void writeTo(StreamOutput out) throws IOException { if (out.getVersion().onOrAfter(Version.V_2_10_0)) { out.writeOptionalString(sourceRemoteStoreRepository); } + if (out.getVersion().onOrAfter(Version.CURRENT)) { + out.writeOptionalString(sourceRemoteTranslogRepository); + } } @Override @@ -545,6 +553,16 @@ public RestoreSnapshotRequest setSourceRemoteStoreRepository(String sourceRemote return this; } + /** + * Sets Source Remote Translog Repository for all the restored indices + * + * @param sourceRemoteTranslogRepository name of the remote translog repository that should be used for all restored indices. + */ + public RestoreSnapshotRequest setSourceRemoteTranslogRepository(String sourceRemoteTranslogRepository) { + this.sourceRemoteTranslogRepository = sourceRemoteTranslogRepository; + return this; + } + /** * Returns Source Remote Store Repository for all the restored indices * @@ -554,6 +572,15 @@ public String getSourceRemoteStoreRepository() { return sourceRemoteStoreRepository; } + /** + * Returns Source Remote Translog Repository for all the restored indices + * + * @return source Remote Translog Repository + */ + public String getSourceRemoteTranslogRepository() { + return sourceRemoteTranslogRepository; + } + /** * Parses restore definition * @@ -673,6 +700,9 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws if (sourceRemoteStoreRepository != null) { builder.field("source_remote_store_repository", sourceRemoteStoreRepository); } + if (sourceRemoteTranslogRepository != null) { + builder.field("source_remote_translog_repository", sourceRemoteTranslogRepository); + } builder.endObject(); return builder; } @@ -701,7 +731,8 @@ public boolean equals(Object o) { && Arrays.equals(ignoreIndexSettings, that.ignoreIndexSettings) && Objects.equals(snapshotUuid, that.snapshotUuid) && Objects.equals(storageType, that.storageType) - && Objects.equals(sourceRemoteStoreRepository, that.sourceRemoteStoreRepository); + && Objects.equals(sourceRemoteStoreRepository, that.sourceRemoteStoreRepository) + && Objects.equals(sourceRemoteTranslogRepository, that.sourceRemoteTranslogRepository); return equals; } @@ -721,7 +752,8 @@ public int hashCode() { indexSettings, snapshotUuid, storageType, - sourceRemoteStoreRepository + sourceRemoteStoreRepository, + sourceRemoteTranslogRepository ); result = 31 * result + Arrays.hashCode(indices); result = 31 * result + Arrays.hashCode(ignoreIndexSettings); diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/snapshots/status/SnapshotsStatusRequest.java b/server/src/main/java/org/opensearch/action/admin/cluster/snapshots/status/SnapshotsStatusRequest.java index 061e73f1094b5..6c3438fbe58f6 100644 --- a/server/src/main/java/org/opensearch/action/admin/cluster/snapshots/status/SnapshotsStatusRequest.java +++ b/server/src/main/java/org/opensearch/action/admin/cluster/snapshots/status/SnapshotsStatusRequest.java @@ -32,6 +32,7 @@ package org.opensearch.action.admin.cluster.snapshots.status; +import org.opensearch.Version; import org.opensearch.action.ActionRequestValidationException; import org.opensearch.action.support.clustermanager.ClusterManagerNodeRequest; import org.opensearch.common.annotation.PublicApi; @@ -54,6 +55,7 @@ public class SnapshotsStatusRequest extends ClusterManagerNodeRequesttrue to ignore unavailable snapshots, instead of throwing an exception. - * Defaults to false, which means unavailable snapshots cause an exception to be thrown. + * Returns the names of the indices. + * + * @return the names of indices + */ + public String[] indices() { + return this.indices; + } + + /** + * Sets the list of indices to be returned + * + * @return this request + */ + public SnapshotsStatusRequest indices(String[] indices) { + this.indices = indices; + return this; + } + + /** + * Set to true to ignore unavailable snapshots and indices, instead of throwing an exception. + * Defaults to false, which means unavailable snapshots and indices cause an exception to be thrown. * - * @param ignoreUnavailable whether to ignore unavailable snapshots + * @param ignoreUnavailable whether to ignore unavailable snapshots and indices * @return this request */ public SnapshotsStatusRequest ignoreUnavailable(boolean ignoreUnavailable) { @@ -158,9 +189,9 @@ public SnapshotsStatusRequest ignoreUnavailable(boolean ignoreUnavailable) { } /** - * Returns whether the request permits unavailable snapshots to be ignored. + * Returns whether the request permits unavailable snapshots and indices to be ignored. * - * @return true if the request will ignore unavailable snapshots, false if it will throw an exception on unavailable snapshots + * @return true if the request will ignore unavailable snapshots and indices, false if it will throw an exception on unavailable snapshots and indices */ public boolean ignoreUnavailable() { return ignoreUnavailable; diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/snapshots/status/SnapshotsStatusRequestBuilder.java b/server/src/main/java/org/opensearch/action/admin/cluster/snapshots/status/SnapshotsStatusRequestBuilder.java index 9377eca60e353..6f0ac278d01c4 100644 --- a/server/src/main/java/org/opensearch/action/admin/cluster/snapshots/status/SnapshotsStatusRequestBuilder.java +++ b/server/src/main/java/org/opensearch/action/admin/cluster/snapshots/status/SnapshotsStatusRequestBuilder.java @@ -96,10 +96,32 @@ public SnapshotsStatusRequestBuilder addSnapshots(String... snapshots) { } /** - * Set to true to ignore unavailable snapshots, instead of throwing an exception. - * Defaults to false, which means unavailable snapshots cause an exception to be thrown. + * Sets list of indices to return * - * @param ignoreUnavailable whether to ignore unavailable snapshots. + * @param indices list of indices + * @return this builder + */ + public SnapshotsStatusRequestBuilder setIndices(String... indices) { + request.indices(indices); + return this; + } + + /** + * Adds additional indices to the list of indices to return + * + * @param indices additional indices + * @return this builder + */ + public SnapshotsStatusRequestBuilder addIndices(String... indices) { + request.indices(ArrayUtils.concat(request.indices(), indices)); + return this; + } + + /** + * Set to true to ignore unavailable snapshots and indices, instead of throwing an exception. + * Defaults to false, which means unavailable snapshots and indices cause an exception to be thrown. + * + * @param ignoreUnavailable whether to ignore unavailable snapshots and indices. * @return this builder */ public SnapshotsStatusRequestBuilder setIgnoreUnavailable(boolean ignoreUnavailable) { diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/snapshots/status/TransportSnapshotsStatusAction.java b/server/src/main/java/org/opensearch/action/admin/cluster/snapshots/status/TransportSnapshotsStatusAction.java index 4fc2acb2caa51..f2a9b88f790c9 100644 --- a/server/src/main/java/org/opensearch/action/admin/cluster/snapshots/status/TransportSnapshotsStatusAction.java +++ b/server/src/main/java/org/opensearch/action/admin/cluster/snapshots/status/TransportSnapshotsStatusAction.java @@ -52,6 +52,7 @@ import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.util.CollectionUtils; import org.opensearch.core.index.shard.ShardId; +import org.opensearch.index.IndexNotFoundException; import org.opensearch.index.snapshots.IndexShardSnapshotStatus; import org.opensearch.repositories.IndexId; import org.opensearch.repositories.RepositoriesService; @@ -65,6 +66,7 @@ import org.opensearch.snapshots.SnapshotShardsService; import org.opensearch.snapshots.SnapshotState; import org.opensearch.snapshots.SnapshotsService; +import org.opensearch.snapshots.TooManyShardsInSnapshotsStatusException; import org.opensearch.threadpool.ThreadPool; import org.opensearch.transport.TransportService; @@ -81,6 +83,7 @@ import java.util.stream.Collectors; import static java.util.Collections.unmodifiableMap; +import static org.opensearch.snapshots.SnapshotsService.MAX_SHARDS_ALLOWED_IN_STATUS_API; /** * Transport action for accessing snapshot status @@ -95,6 +98,8 @@ public class TransportSnapshotsStatusAction extends TransportClusterManagerNodeA private final TransportNodesSnapshotsStatus transportNodesSnapshotsStatus; + private long maximumAllowedShardCount; + @Inject public TransportSnapshotsStatusAction( TransportService transportService, @@ -314,38 +319,34 @@ private void loadRepositoryData( String repositoryName, ActionListener listener ) { - final Set requestedSnapshotNames = Sets.newHashSet(request.snapshots()); + maximumAllowedShardCount = clusterService.getClusterSettings().get(MAX_SHARDS_ALLOWED_IN_STATUS_API); final StepListener repositoryDataListener = new StepListener<>(); repositoriesService.getRepositoryData(repositoryName, repositoryDataListener); repositoryDataListener.whenComplete(repositoryData -> { - final Map matchedSnapshotIds = repositoryData.getSnapshotIds() - .stream() - .filter(s -> requestedSnapshotNames.contains(s.getName())) - .collect(Collectors.toMap(SnapshotId::getName, Function.identity())); - for (final String snapshotName : request.snapshots()) { - if (currentSnapshotNames.contains(snapshotName)) { - // we've already found this snapshot in the current snapshot entries, so skip over - continue; - } - SnapshotId snapshotId = matchedSnapshotIds.get(snapshotName); - if (snapshotId == null) { - // neither in the current snapshot entries nor found in the repository - if (request.ignoreUnavailable()) { - // ignoring unavailable snapshots, so skip over - logger.debug( - "snapshot status request ignoring snapshot [{}], not found in repository [{}]", - snapshotName, - repositoryName - ); - continue; - } else { - throw new SnapshotMissingException(repositoryName, snapshotName); - } - } - SnapshotInfo snapshotInfo = snapshot(snapshotsInProgress, repositoryName, snapshotId); + Map snapshotsInfoMap = snapshotsInfo( + request, + repositoryName, + repositoryData, + snapshotsInProgress, + currentSnapshotNames + ); + for (Map.Entry entry : snapshotsInfoMap.entrySet()) { + SnapshotId snapshotId = entry.getKey(); + SnapshotInfo snapshotInfo = entry.getValue(); List shardStatusBuilder = new ArrayList<>(); if (snapshotInfo.state().completed()) { - Map shardStatuses = snapshotShards(repositoryName, repositoryData, snapshotInfo); + Map shardStatuses = snapshotShards( + request, + repositoryName, + repositoryData, + snapshotInfo + ); + boolean isShallowV2Snapshot = snapshotInfo.getPinnedTimestamp() > 0; + long initialSnapshotTotalSize = 0; + if (isShallowV2Snapshot && request.indices().length == 0) { + // TODO: add primary store size in bytes at the snapshot level + } + for (Map.Entry shardStatus : shardStatuses.entrySet()) { IndexShardSnapshotStatus.Copy lastSnapshotStatus = shardStatus.getValue().asCopy(); shardStatusBuilder.add(new SnapshotIndexShardStatus(shardStatus.getKey(), lastSnapshotStatus)); @@ -406,6 +407,68 @@ private SnapshotInfo snapshot(SnapshotsInProgress snapshotsInProgress, String re return repositoriesService.repository(repositoryName).getSnapshotInfo(snapshotId); } + /** + * Returns snapshot info for finished snapshots + * @param request snapshot status request + * @param repositoryName repository name + * @param repositoryData repository data + * @param snapshotsInProgress currently running snapshots + * @param currentSnapshotNames list of names of currently running snapshots + * @return map of snapshot id to snapshot info + */ + private Map snapshotsInfo( + SnapshotsStatusRequest request, + String repositoryName, + RepositoryData repositoryData, + SnapshotsInProgress snapshotsInProgress, + Set currentSnapshotNames + ) { + final Set requestedSnapshotNames = Sets.newHashSet(request.snapshots()); + final Map snapshotsInfoMap = new HashMap<>(); + final Map matchedSnapshotIds = repositoryData.getSnapshotIds() + .stream() + .filter(s -> requestedSnapshotNames.contains(s.getName())) + .collect(Collectors.toMap(SnapshotId::getName, Function.identity())); + int totalShardsAcrossSnapshots = 0; + for (final String snapshotName : request.snapshots()) { + if (currentSnapshotNames.contains(snapshotName)) { + // we've already found this snapshot in the current snapshot entries, so skip over + continue; + } + SnapshotId snapshotId = matchedSnapshotIds.get(snapshotName); + if (snapshotId == null) { + // neither in the current snapshot entries nor found in the repository + if (request.ignoreUnavailable()) { + // ignoring unavailable snapshots, so skip over + logger.debug( + "snapshot status request ignoring snapshot [{}], not found in repository [{}]", + snapshotName, + repositoryName + ); + continue; + } else { + throw new SnapshotMissingException(repositoryName, snapshotName); + } + } + SnapshotInfo snapshotInfo = snapshot(snapshotsInProgress, repositoryName, snapshotId); + boolean isV2Snapshot = snapshotInfo.getPinnedTimestamp() > 0; + if (isV2Snapshot == false && request.indices().length == 0) { + totalShardsAcrossSnapshots += snapshotInfo.totalShards(); + } + snapshotsInfoMap.put(snapshotId, snapshotInfo); + } + if (totalShardsAcrossSnapshots > maximumAllowedShardCount && request.indices().length == 0) { + String message = "Total shard count [" + + totalShardsAcrossSnapshots + + "] is more than the maximum allowed value of shard count [" + + maximumAllowedShardCount + + "] for snapshot status request"; + + throw new TooManyShardsInSnapshotsStatusException(repositoryName, message, request.snapshots()); + } + return unmodifiableMap(snapshotsInfoMap); + } + /** * Returns status of shards currently finished snapshots *

@@ -413,21 +476,65 @@ private SnapshotInfo snapshot(SnapshotsInProgress snapshotsInProgress, String re * {@link SnapshotShardsService#currentSnapshotShards(Snapshot)} because it * returns similar information but for already finished snapshots. *

- * + * @param request snapshot status request * @param repositoryName repository name * @param snapshotInfo snapshot info * @return map of shard id to snapshot status */ private Map snapshotShards( + final SnapshotsStatusRequest request, final String repositoryName, final RepositoryData repositoryData, final SnapshotInfo snapshotInfo ) throws IOException { + final Set requestedIndexNames = Sets.newHashSet(request.indices()); + String snapshotName = snapshotInfo.snapshotId().getName(); + Set indices = Sets.newHashSet(snapshotInfo.indices()); + if (requestedIndexNames.isEmpty() == false) { + Set finalIndices = indices; + List indicesNotFound = requestedIndexNames.stream() + .filter(i -> finalIndices.contains(i) == false) + .collect(Collectors.toList()); + if (indicesNotFound.isEmpty() == false) { + handleIndexNotFound(String.join(", ", indicesNotFound), request, snapshotName, repositoryName); + } + indices = requestedIndexNames; + } + final Repository repository = repositoriesService.repository(repositoryName); - final Map shardStatus = new HashMap<>(); - for (String index : snapshotInfo.indices()) { + boolean isV2Snapshot = snapshotInfo.getPinnedTimestamp() > 0; + int totalShardsAcrossIndices = 0; + final Map indexMetadataMap = new HashMap<>(); + + for (String index : indices) { IndexId indexId = repositoryData.resolveIndexId(index); IndexMetadata indexMetadata = repository.getSnapshotIndexMetaData(repositoryData, snapshotInfo.snapshotId(), indexId); + if (indexMetadata != null) { + if (requestedIndexNames.isEmpty() == false && isV2Snapshot == false) { + totalShardsAcrossIndices += indexMetadata.getNumberOfShards(); + } + indexMetadataMap.put(indexId, indexMetadata); + } else if (requestedIndexNames.isEmpty() == false) { + handleIndexNotFound(index, request, snapshotName, repositoryName); + } + } + + if (totalShardsAcrossIndices > maximumAllowedShardCount && requestedIndexNames.isEmpty() == false && isV2Snapshot == false) { + String message = "Total shard count [" + + totalShardsAcrossIndices + + "] across the requested indices [" + + requestedIndexNames.stream().collect(Collectors.joining(", ")) + + "] is more than the maximum allowed value of shard count [" + + maximumAllowedShardCount + + "] for snapshot status request"; + + throw new TooManyShardsInSnapshotsStatusException(repositoryName, message, snapshotName); + } + + final Map shardStatus = new HashMap<>(); + for (Map.Entry entry : indexMetadataMap.entrySet()) { + IndexId indexId = entry.getKey(); + IndexMetadata indexMetadata = entry.getValue(); if (indexMetadata != null) { int numberOfShards = indexMetadata.getNumberOfShards(); for (int i = 0; i < numberOfShards; i++) { @@ -447,7 +554,12 @@ private Map snapshotShards( // could not be taken due to partial being set to false. shardSnapshotStatus = IndexShardSnapshotStatus.newFailed("skipped"); } else { - shardSnapshotStatus = repository.getShardSnapshotStatus(snapshotInfo.snapshotId(), indexId, shardId); + // TODO: to be refactored later + if (isV2Snapshot) { + shardSnapshotStatus = IndexShardSnapshotStatus.newDone(0, 0, 0, 0, 0, 0, null); + } else { + shardSnapshotStatus = repository.getShardSnapshotStatus(snapshotInfo.snapshotId(), indexId, shardId); + } } shardStatus.put(shardId, shardSnapshotStatus); } @@ -457,6 +569,21 @@ private Map snapshotShards( return unmodifiableMap(shardStatus); } + private void handleIndexNotFound(String index, SnapshotsStatusRequest request, String snapshotName, String repositoryName) { + if (request.ignoreUnavailable()) { + // ignoring unavailable index + logger.debug( + "snapshot status request ignoring indices [{}], not found in snapshot[{}] in repository [{}]", + index, + snapshotName, + repositoryName + ); + } else { + String cause = "indices [" + index + "] missing in snapshot [" + snapshotName + "] of repository [" + repositoryName + "]"; + throw new IndexNotFoundException(index, new IllegalArgumentException(cause)); + } + } + private static SnapshotShardFailure findShardFailure(List shardFailures, ShardId shardId) { for (SnapshotShardFailure shardFailure : shardFailures) { if (shardId.getIndexName().equals(shardFailure.index()) && shardId.getId() == shardFailure.shardId()) { diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/state/ClusterStateRequest.java b/server/src/main/java/org/opensearch/action/admin/cluster/state/ClusterStateRequest.java index 90a52f7406d57..66bb64ca5580b 100644 --- a/server/src/main/java/org/opensearch/action/admin/cluster/state/ClusterStateRequest.java +++ b/server/src/main/java/org/opensearch/action/admin/cluster/state/ClusterStateRequest.java @@ -41,8 +41,12 @@ 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.tasks.TaskId; +import org.opensearch.rest.action.admin.cluster.ClusterAdminTask; +import org.opensearch.tasks.Task; import java.io.IOException; +import java.util.Map; /** * Transport request for obtaining cluster state @@ -211,4 +215,13 @@ public ClusterStateRequest waitForMetadataVersion(long waitForMetadataVersion) { this.waitForMetadataVersion = waitForMetadataVersion; return this; } + + @Override + public Task createTask(long id, String type, String action, TaskId parentTaskId, Map headers) { + if (this.getShouldCancelOnTimeout()) { + return new ClusterAdminTask(id, type, action, parentTaskId, headers); + } else { + return super.createTask(id, type, action, parentTaskId, headers); + } + } } diff --git a/server/src/main/java/org/opensearch/action/admin/indices/stats/IndicesStatsRequest.java b/server/src/main/java/org/opensearch/action/admin/indices/stats/IndicesStatsRequest.java index 2b64464a76899..c36e53098d166 100644 --- a/server/src/main/java/org/opensearch/action/admin/indices/stats/IndicesStatsRequest.java +++ b/server/src/main/java/org/opensearch/action/admin/indices/stats/IndicesStatsRequest.java @@ -36,8 +36,12 @@ 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.tasks.TaskId; +import org.opensearch.rest.action.admin.cluster.ClusterAdminTask; +import org.opensearch.tasks.Task; import java.io.IOException; +import java.util.Map; /** * A request to get indices level stats. Allow to enable different stats to be returned. @@ -103,6 +107,15 @@ public IndicesStatsRequest groups(String... groups) { return this; } + @Override + public Task createTask(long id, String type, String action, TaskId parentTaskId, Map headers) { + if (this.getShouldCancelOnTimeout()) { + return new ClusterAdminTask(id, type, action, parentTaskId, headers); + } else { + return super.createTask(id, type, action, parentTaskId, headers); + } + } + public String[] groups() { return this.flags.groups(); } diff --git a/server/src/main/java/org/opensearch/action/admin/indices/stats/IndicesStatsResponse.java b/server/src/main/java/org/opensearch/action/admin/indices/stats/IndicesStatsResponse.java index 6242081cd2371..900a886481fe6 100644 --- a/server/src/main/java/org/opensearch/action/admin/indices/stats/IndicesStatsResponse.java +++ b/server/src/main/java/org/opensearch/action/admin/indices/stats/IndicesStatsResponse.java @@ -69,7 +69,7 @@ public class IndicesStatsResponse extends BroadcastResponse { shards = in.readArray(ShardStats::new, (size) -> new ShardStats[size]); } - IndicesStatsResponse( + public IndicesStatsResponse( ShardStats[] shards, int totalShards, int successfulShards, diff --git a/server/src/main/java/org/opensearch/action/search/TransportSearchAction.java b/server/src/main/java/org/opensearch/action/search/TransportSearchAction.java index 88bf7ebea8e52..8c4927afa9a14 100644 --- a/server/src/main/java/org/opensearch/action/search/TransportSearchAction.java +++ b/server/src/main/java/org/opensearch/action/search/TransportSearchAction.java @@ -312,8 +312,9 @@ protected void doExecute(Task task, SearchRequest searchRequest, ActionListener< listener = TimeoutTaskCancellationUtility.wrapWithCancellationListener( client, (CancellableTask) task, - clusterService.getClusterSettings(), - listener + clusterService.getClusterSettings().get(SEARCH_CANCEL_AFTER_TIME_INTERVAL_SETTING), + listener, + e -> {} ); } executeRequest(task, searchRequest, this::searchAsyncAction, listener); diff --git a/server/src/main/java/org/opensearch/action/support/TimeoutTaskCancellationUtility.java b/server/src/main/java/org/opensearch/action/support/TimeoutTaskCancellationUtility.java index a317a45eab31f..d2e7121a592cb 100644 --- a/server/src/main/java/org/opensearch/action/support/TimeoutTaskCancellationUtility.java +++ b/server/src/main/java/org/opensearch/action/support/TimeoutTaskCancellationUtility.java @@ -12,11 +12,12 @@ import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; import org.opensearch.action.admin.cluster.node.tasks.cancel.CancelTasksRequest; +import org.opensearch.action.admin.cluster.node.tasks.cancel.CancelTasksResponse; import org.opensearch.client.OriginSettingClient; import org.opensearch.client.node.NodeClient; -import org.opensearch.common.settings.ClusterSettings; import org.opensearch.common.unit.TimeValue; import org.opensearch.core.action.ActionListener; +import org.opensearch.core.tasks.TaskCancelledException; import org.opensearch.core.tasks.TaskId; import org.opensearch.search.SearchService; import org.opensearch.tasks.CancellableTask; @@ -27,7 +28,6 @@ import java.util.concurrent.atomic.AtomicBoolean; import static org.opensearch.action.admin.cluster.node.tasks.get.GetTaskAction.TASKS_ORIGIN; -import static org.opensearch.action.search.TransportSearchAction.SEARCH_CANCEL_AFTER_TIME_INTERVAL_SETTING; /** * Utility to cancel a timeout task @@ -38,25 +38,25 @@ public class TimeoutTaskCancellationUtility { private static final Logger logger = LogManager.getLogger(TimeoutTaskCancellationUtility.class); + private static final AtomicBoolean executeResponseOrFailureOnce = new AtomicBoolean(true); + /** * Wraps a listener with a timeout listener {@link TimeoutRunnableListener} to schedule the task cancellation for provided tasks on * generic thread pool * @param client - {@link NodeClient} * @param taskToCancel - task to schedule cancellation for - * @param clusterSettings - {@link ClusterSettings} + * @param timeout - {@link TimeValue} * @param listener - original listener associated with the task * @return wrapped listener */ public static ActionListener wrapWithCancellationListener( NodeClient client, CancellableTask taskToCancel, - ClusterSettings clusterSettings, - ActionListener listener + TimeValue timeout, + ActionListener listener, + TimeoutHandler timeoutHandler ) { - final TimeValue globalTimeout = clusterSettings.get(SEARCH_CANCEL_AFTER_TIME_INTERVAL_SETTING); - final TimeValue timeoutInterval = (taskToCancel.getCancellationTimeout() == null) - ? globalTimeout - : taskToCancel.getCancellationTimeout(); + final TimeValue timeoutInterval = (taskToCancel.getCancellationTimeout() == null) ? timeout : taskToCancel.getCancellationTimeout(); // Note: -1 (or no timeout) will help to turn off cancellation. The combinations will be request level set at -1 or request level // set to null and cluster level set to -1. ActionListener listenerToReturn = listener; @@ -72,24 +72,30 @@ public static ActionListener wrapWithCancellationListener( // force the origin to execute the cancellation as a system user new OriginSettingClient(client, TASKS_ORIGIN).admin() .cluster() - .cancelTasks( - cancelTasksRequest, - ActionListener.wrap( - r -> logger.debug( + .cancelTasks(cancelTasksRequest, new ActionListener() { + @Override + public void onResponse(CancelTasksResponse cancelTasksResponse) { + logger.debug( "Scheduled cancel task with timeout: {} for original task: {} is successfully completed", timeoutInterval, cancelTasksRequest.getTaskId() - ), - e -> logger.error( + ); + // Notify the timeoutHandler that the task was canceled due to timeout + timeoutHandler.onTimeout(new TaskCancelledException(cancelTasksRequest.getReason())); + } + + @Override + public void onFailure(Exception e) { + logger.error( new ParameterizedMessage( "Scheduled cancel task with timeout: {} for original task: {} is failed", timeoutInterval, cancelTasksRequest.getTaskId() ), e - ) - ) - ); + ); + } + }); }); wrappedListener.cancellable = client.threadPool().schedule(wrappedListener, timeoutInterval, ThreadPool.Names.GENERIC); listenerToReturn = wrappedListener; @@ -100,6 +106,22 @@ public static ActionListener wrapWithCancellationListener( return listenerToReturn; } + /** + * A functional interface used to handle the timeout of a cancellable task. + * Implementations of this interface provide a callback method that is invoked + * when a task is cancelled due to a timeout. + */ + public interface TimeoutHandler { + + /** + * Called when a cancellable task is cancelled due to a timeout. + * + * @param e the exception that contains details about the task cancellation, + * including the reason for cancellation. + */ + void onTimeout(TaskCancelledException e); + } + /** * Timeout listener which executes the provided runnable after timeout is expired and if a response/failure is not yet received. * If either a response/failure is received before timeout then the scheduled task is cancelled and response/failure is sent back to @@ -144,6 +166,7 @@ public void run() { if (executeRunnable.compareAndSet(true, false)) { timeoutRunnable.run(); } // else do nothing since either response/failure is already sent to client + } catch (Exception ex) { // ignore the exception logger.error( diff --git a/server/src/main/java/org/opensearch/action/support/broadcast/BroadcastRequest.java b/server/src/main/java/org/opensearch/action/support/broadcast/BroadcastRequest.java index 8a27e032cec5e..784c43c938f2c 100644 --- a/server/src/main/java/org/opensearch/action/support/broadcast/BroadcastRequest.java +++ b/server/src/main/java/org/opensearch/action/support/broadcast/BroadcastRequest.java @@ -52,6 +52,8 @@ public class BroadcastRequest> extends protected String[] indices; private IndicesOptions indicesOptions = IndicesOptions.strictExpandOpenAndForbidClosed(); + protected boolean shouldCancelOnTimeout = false; + public BroadcastRequest(StreamInput in) throws IOException { super(in); indices = in.readStringArray(); @@ -125,4 +127,12 @@ public void writeTo(StreamOutput out) throws IOException { out.writeStringArrayNullable(indices); indicesOptions.writeIndicesOptions(out); } + + public void setShouldCancelOnTimeout(boolean shouldCancelOnTimeout) { + this.shouldCancelOnTimeout = shouldCancelOnTimeout; + } + + public boolean getShouldCancelOnTimeout() { + return this.shouldCancelOnTimeout; + } } diff --git a/server/src/main/java/org/opensearch/action/support/clustermanager/ClusterManagerNodeReadRequest.java b/server/src/main/java/org/opensearch/action/support/clustermanager/ClusterManagerNodeReadRequest.java index 6dcc6ed1b098e..c373107163082 100644 --- a/server/src/main/java/org/opensearch/action/support/clustermanager/ClusterManagerNodeReadRequest.java +++ b/server/src/main/java/org/opensearch/action/support/clustermanager/ClusterManagerNodeReadRequest.java @@ -47,6 +47,8 @@ public abstract class ClusterManagerNodeReadRequest> implements Diff { this.part = part; } + @Override + public String toString() { + return "CompleteDiff{" + "part=" + part + '}'; + } + /** * Creates simple diff without changes */ diff --git a/server/src/main/java/org/opensearch/cluster/ClusterState.java b/server/src/main/java/org/opensearch/cluster/ClusterState.java index 9e63f961d241d..1e4fd2dfffe0f 100644 --- a/server/src/main/java/org/opensearch/cluster/ClusterState.java +++ b/server/src/main/java/org/opensearch/cluster/ClusterState.java @@ -156,7 +156,7 @@ default boolean isPrivate() { } - private static final NamedDiffableValueSerializer CUSTOM_VALUE_SERIALIZER = new NamedDiffableValueSerializer<>(Custom.class); + public static final NamedDiffableValueSerializer CUSTOM_VALUE_SERIALIZER = new NamedDiffableValueSerializer<>(Custom.class); public static final String UNKNOWN_UUID = "_na_"; @@ -781,7 +781,7 @@ public void writeTo(StreamOutput out) throws IOException { out.writeString(stateUUID); metadata.writeTo(out); routingTable.writeTo(out); - nodes.writeTo(out); + nodes.writeToWithAttribute(out); blocks.writeTo(out); // filter out custom states not supported by the other node int numberOfCustoms = 0; @@ -839,6 +839,34 @@ private static class ClusterStateDiff implements Diff { minimumClusterManagerNodesOnPublishingClusterManager = after.minimumClusterManagerNodesOnPublishingClusterManager; } + @Override + public String toString() { + return new StringBuilder().append("ClusterStateDiff{toVersion=") + .append(toVersion) + .append(", fromUuid='") + .append(fromUuid) + .append('\'') + .append(", toUuid='") + .append(toUuid) + .append('\'') + .append(", clusterName=") + .append(clusterName) + .append(", routingTable=") + .append(routingTable) + .append(", nodes=") + .append(nodes) + .append(", metadata=") + .append(metadata) + .append(", blocks=") + .append(blocks) + .append(", customs=") + .append(customs) + .append(", minimumClusterManagerNodesOnPublishingClusterManager=") + .append(minimumClusterManagerNodesOnPublishingClusterManager) + .append("}") + .toString(); + } + ClusterStateDiff(StreamInput in, DiscoveryNode localNode) throws IOException { clusterName = new ClusterName(in); fromUuid = in.readString(); @@ -859,13 +887,23 @@ public void writeTo(StreamOutput out) throws IOException { out.writeString(toUuid); out.writeLong(toVersion); routingTable.writeTo(out); - nodes.writeTo(out); + nodesWriteToWithAttributes(nodes, out); metadata.writeTo(out); blocks.writeTo(out); customs.writeTo(out); out.writeVInt(minimumClusterManagerNodesOnPublishingClusterManager); } + private void nodesWriteToWithAttributes(Diff nodes, StreamOutput out) throws IOException { + DiscoveryNodes part = nodes.apply(null); + if (part != null) { + out.writeBoolean(true); + part.writeToWithAttribute(out); + } else { + out.writeBoolean(false); + } + } + @Override public ClusterState apply(ClusterState state) { Builder builder = new Builder(clusterName); diff --git a/server/src/main/java/org/opensearch/cluster/DiffableUtils.java b/server/src/main/java/org/opensearch/cluster/DiffableUtils.java index d21cd354bf659..4b5dcaa52cc50 100644 --- a/server/src/main/java/org/opensearch/cluster/DiffableUtils.java +++ b/server/src/main/java/org/opensearch/cluster/DiffableUtils.java @@ -271,6 +271,18 @@ public Map getUpserts() { return upserts; } + @Override + public String toString() { + return new StringBuilder().append("MapDiff{deletes=") + .append(deletes) + .append(", diffs=") + .append(diffs) + .append(", upserts=") + .append(upserts) + .append("}") + .toString(); + } + @Override public void writeTo(StreamOutput out) throws IOException { out.writeCollection(deletes, (o, v) -> keySerializer.writeKey(v, o)); 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..7c0a7a2a6b837 100644 --- a/server/src/main/java/org/opensearch/cluster/block/ClusterBlock.java +++ b/server/src/main/java/org/opensearch/cluster/block/ClusterBlock.java @@ -52,7 +52,7 @@ * @opensearch.api */ @PublicApi(since = "1.0.0") -public class ClusterBlock implements Writeable, ToXContentFragment { +public class ClusterBlock implements Writeable, ToXContentFragment, Comparable { private final int id; @Nullable @@ -217,7 +217,13 @@ public int hashCode() { return Objects.hash(id, uuid); } + @Override + public int compareTo(ClusterBlock block) { + return Integer.compare(block.id(), this.id()); + } + public boolean isAllowReleaseResources() { return allowReleaseResources; } + } 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..615ea18315cd1 100644 --- a/server/src/main/java/org/opensearch/cluster/block/ClusterBlocks.java +++ b/server/src/main/java/org/opensearch/cluster/block/ClusterBlocks.java @@ -39,8 +39,10 @@ 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.BufferedChecksumStreamOutput; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; +import org.opensearch.core.common.io.stream.VerifiableWriteable; import org.opensearch.core.rest.RestStatus; import java.io.IOException; @@ -62,7 +64,7 @@ * @opensearch.api */ @PublicApi(since = "1.0.0") -public class ClusterBlocks extends AbstractDiffable { +public class ClusterBlocks extends AbstractDiffable implements VerifiableWriteable { public static final ClusterBlocks EMPTY_CLUSTER_BLOCK = new ClusterBlocks(emptySet(), Map.of()); private final Set global; @@ -303,6 +305,11 @@ public void writeTo(StreamOutput out) throws IOException { out.writeMap(indicesBlocks, StreamOutput::writeString, (o, s) -> writeBlockSet(s, o)); } + @Override + public void writeVerifiableTo(BufferedChecksumStreamOutput out) throws IOException { + writeTo(out); + } + private static void writeBlockSet(Set blocks, StreamOutput out) throws IOException { out.writeCollection(blocks); } diff --git a/server/src/main/java/org/opensearch/cluster/coordination/CoordinationMetadata.java b/server/src/main/java/org/opensearch/cluster/coordination/CoordinationMetadata.java index 53398d6f3f98f..d7291a3689192 100644 --- a/server/src/main/java/org/opensearch/cluster/coordination/CoordinationMetadata.java +++ b/server/src/main/java/org/opensearch/cluster/coordination/CoordinationMetadata.java @@ -35,8 +35,10 @@ import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.util.set.Sets; import org.opensearch.core.ParseField; +import org.opensearch.core.common.io.stream.BufferedChecksumStreamOutput; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; +import org.opensearch.core.common.io.stream.VerifiableWriteable; import org.opensearch.core.common.io.stream.Writeable; import org.opensearch.core.xcontent.ConstructingObjectParser; import org.opensearch.core.xcontent.ToXContentFragment; @@ -59,7 +61,7 @@ * @opensearch.api */ @PublicApi(since = "1.0.0") -public class CoordinationMetadata implements Writeable, ToXContentFragment { +public class CoordinationMetadata implements VerifiableWriteable, ToXContentFragment { public static final CoordinationMetadata EMPTY_METADATA = builder().build(); @@ -149,6 +151,11 @@ public void writeTo(StreamOutput out) throws IOException { out.writeCollection(votingConfigExclusions); } + @Override + public void writeVerifiableTo(BufferedChecksumStreamOutput out) throws IOException { + writeTo(out); + } + @Override public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { return builder.field(TERM_PARSE_FIELD.getPreferredName(), term) @@ -272,7 +279,7 @@ public CoordinationMetadata build() { * @opensearch.api */ @PublicApi(since = "1.0.0") - public static class VotingConfigExclusion implements Writeable, ToXContentFragment { + public static class VotingConfigExclusion implements Writeable, ToXContentFragment, Comparable { public static final String MISSING_VALUE_MARKER = "_absent_"; private final String nodeId; private final String nodeName; @@ -361,6 +368,10 @@ public String toString() { return sb.toString(); } + @Override + public int compareTo(VotingConfigExclusion votingConfigExclusion) { + return votingConfigExclusion.getNodeId().compareTo(this.getNodeId()); + } } /** diff --git a/server/src/main/java/org/opensearch/cluster/coordination/Join.java b/server/src/main/java/org/opensearch/cluster/coordination/Join.java index 58fa85992ebc8..ce1a234998690 100644 --- a/server/src/main/java/org/opensearch/cluster/coordination/Join.java +++ b/server/src/main/java/org/opensearch/cluster/coordination/Join.java @@ -78,8 +78,8 @@ public Join(StreamInput in) throws IOException { @Override public void writeTo(StreamOutput out) throws IOException { - sourceNode.writeTo(out); - targetNode.writeTo(out); + sourceNode.writeToWithAttribute(out); + targetNode.writeToWithAttribute(out); out.writeLong(term); out.writeLong(lastAcceptedTerm); out.writeLong(lastAcceptedVersion); diff --git a/server/src/main/java/org/opensearch/cluster/coordination/JoinRequest.java b/server/src/main/java/org/opensearch/cluster/coordination/JoinRequest.java index 04f87d16ee400..1447838a41502 100644 --- a/server/src/main/java/org/opensearch/cluster/coordination/JoinRequest.java +++ b/server/src/main/java/org/opensearch/cluster/coordination/JoinRequest.java @@ -84,7 +84,7 @@ public JoinRequest(StreamInput in) throws IOException { @Override public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); - sourceNode.writeTo(out); + sourceNode.writeToWithAttribute(out); out.writeLong(minimumTerm); out.writeOptionalWriteable(optionalJoin.orElse(null)); } diff --git a/server/src/main/java/org/opensearch/cluster/coordination/StartJoinRequest.java b/server/src/main/java/org/opensearch/cluster/coordination/StartJoinRequest.java index de58eb721b28f..287418aaf378e 100644 --- a/server/src/main/java/org/opensearch/cluster/coordination/StartJoinRequest.java +++ b/server/src/main/java/org/opensearch/cluster/coordination/StartJoinRequest.java @@ -64,7 +64,7 @@ public StartJoinRequest(StreamInput input) throws IOException { @Override public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); - sourceNode.writeTo(out); + sourceNode.writeToWithAttribute(out); out.writeLong(term); } 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 405e9b3aabdb6..bb470ea9e4ab8 100644 --- a/server/src/main/java/org/opensearch/cluster/metadata/IndexMetadata.java +++ b/server/src/main/java/org/opensearch/cluster/metadata/IndexMetadata.java @@ -54,8 +54,10 @@ import org.opensearch.common.xcontent.XContentHelper; import org.opensearch.core.Assertions; import org.opensearch.core.common.Strings; +import org.opensearch.core.common.io.stream.BufferedChecksumStreamOutput; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; +import org.opensearch.core.common.io.stream.VerifiableWriteable; import org.opensearch.core.common.io.stream.Writeable; import org.opensearch.core.index.Index; import org.opensearch.core.index.shard.ShardId; @@ -88,6 +90,7 @@ import java.util.Map; import java.util.Objects; import java.util.Set; +import java.util.TreeSet; import java.util.function.Function; import static org.opensearch.cluster.metadata.Metadata.CONTEXT_MODE_PARAM; @@ -103,7 +106,7 @@ * @opensearch.api */ @PublicApi(since = "1.0.0") -public class IndexMetadata implements Diffable, ToXContentFragment { +public class IndexMetadata implements Diffable, ToXContentFragment, VerifiableWriteable { public static final ClusterBlock INDEX_READ_ONLY_BLOCK = new ClusterBlock( 5, @@ -1264,6 +1267,32 @@ public void writeTo(StreamOutput out) throws IOException { } } + @Override + public void writeVerifiableTo(BufferedChecksumStreamOutput out) throws IOException { + out.writeString(index.getName()); // uuid will come as part of settings + out.writeLong(version); + out.writeVLong(mappingVersion); + out.writeVLong(settingsVersion); + out.writeVLong(aliasesVersion); + out.writeInt(routingNumShards); + out.writeByte(state.id()); + writeSettingsToStream(settings, out); + out.writeVLongArray(primaryTerms); + out.writeMapValues(mappings, (stream, val) -> val.writeTo(stream)); + out.writeMapValues(aliases, (stream, val) -> val.writeTo(stream)); + out.writeMap(customData, StreamOutput::writeString, (stream, val) -> val.writeTo(stream)); + out.writeMap( + inSyncAllocationIds, + StreamOutput::writeVInt, + (stream, val) -> DiffableUtils.StringSetValueSerializer.getInstance().write(new TreeSet<>(val), stream) + ); + out.writeMapValues(rolloverInfos, (stream, val) -> val.writeTo(stream)); + out.writeBoolean(isSystem); + if (out.getVersion().onOrAfter(Version.V_2_17_0)) { + out.writeOptionalWriteable(context); + } + } + public boolean isSystem() { return isSystem; } diff --git a/server/src/main/java/org/opensearch/cluster/metadata/IndexTemplateMetadata.java b/server/src/main/java/org/opensearch/cluster/metadata/IndexTemplateMetadata.java index 3d532208bcfe2..b09acc54653c4 100644 --- a/server/src/main/java/org/opensearch/cluster/metadata/IndexTemplateMetadata.java +++ b/server/src/main/java/org/opensearch/cluster/metadata/IndexTemplateMetadata.java @@ -44,8 +44,10 @@ import org.opensearch.common.xcontent.XContentFactory; import org.opensearch.common.xcontent.XContentHelper; import org.opensearch.common.xcontent.json.JsonXContent; +import org.opensearch.core.common.io.stream.BufferedChecksumStreamOutput; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; +import org.opensearch.core.common.io.stream.VerifiableWriteable; import org.opensearch.core.xcontent.ToXContent; import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.core.xcontent.XContentParser; @@ -67,7 +69,7 @@ * @opensearch.api */ @PublicApi(since = "1.0.0") -public class IndexTemplateMetadata extends AbstractDiffable { +public class IndexTemplateMetadata extends AbstractDiffable implements VerifiableWriteable { private static final DeprecationLogger deprecationLogger = DeprecationLogger.getLogger(IndexTemplateMetadata.class); @@ -257,6 +259,17 @@ public void writeTo(StreamOutput out) throws IOException { out.writeOptionalVInt(version); } + @Override + public void writeVerifiableTo(BufferedChecksumStreamOutput out) throws IOException { + out.writeString(name); + out.writeInt(order); + out.writeStringCollection(patterns); + Settings.writeSettingsToStream(settings, out); + out.writeMap(mappings, StreamOutput::writeString, (stream, val) -> val.writeTo(stream)); + out.writeMapValues(aliases, (stream, val) -> val.writeTo(stream)); + out.writeOptionalVInt(version); + } + @Override public String toString() { try { 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 5ace5b7bc2460..600f408cc963b 100644 --- a/server/src/main/java/org/opensearch/cluster/metadata/Metadata.java +++ b/server/src/main/java/org/opensearch/cluster/metadata/Metadata.java @@ -253,7 +253,7 @@ static Custom fromXContent(XContentParser parser, String name) throws IOExceptio public static final String GLOBAL_STATE_FILE_PREFIX = "global-"; - private static final NamedDiffableValueSerializer CUSTOM_VALUE_SERIALIZER = new NamedDiffableValueSerializer<>(Custom.class); + public static final NamedDiffableValueSerializer CUSTOM_VALUE_SERIALIZER = new NamedDiffableValueSerializer<>(Custom.class); private final String clusterUUID; private final boolean clusterUUIDCommitted; diff --git a/server/src/main/java/org/opensearch/cluster/metadata/MetadataCreateIndexService.java b/server/src/main/java/org/opensearch/cluster/metadata/MetadataCreateIndexService.java index ad0ee0be59de2..8b08927bc146a 100644 --- a/server/src/main/java/org/opensearch/cluster/metadata/MetadataCreateIndexService.java +++ b/server/src/main/java/org/opensearch/cluster/metadata/MetadataCreateIndexService.java @@ -81,6 +81,7 @@ import org.opensearch.common.xcontent.XContentHelper; import org.opensearch.core.action.ActionListener; import org.opensearch.core.common.Strings; +import org.opensearch.core.common.unit.ByteSizeValue; import org.opensearch.core.index.Index; import org.opensearch.core.xcontent.NamedXContentRegistry; import org.opensearch.env.Environment; @@ -88,7 +89,9 @@ import org.opensearch.index.IndexNotFoundException; import org.opensearch.index.IndexService; import org.opensearch.index.IndexSettings; +import org.opensearch.index.compositeindex.CompositeIndexSettings; import org.opensearch.index.compositeindex.CompositeIndexValidator; +import org.opensearch.index.compositeindex.datacube.startree.StarTreeIndexSettings; import org.opensearch.index.mapper.DocumentMapper; import org.opensearch.index.mapper.MapperService; import org.opensearch.index.mapper.MapperService.MergeReason; @@ -153,6 +156,7 @@ import static org.opensearch.cluster.metadata.Metadata.DEFAULT_REPLICA_COUNT_SETTING; import static org.opensearch.cluster.metadata.MetadataIndexTemplateService.findContextTemplateName; import static org.opensearch.index.IndexModule.INDEX_STORE_TYPE_SETTING; +import static org.opensearch.index.IndexSettings.INDEX_TRANSLOG_FLUSH_THRESHOLD_SIZE_SETTING; import static org.opensearch.indices.IndicesService.CLUSTER_REPLICATION_TYPE_SETTING; import static org.opensearch.node.remotestore.RemoteStoreNodeAttribute.isRemoteDataAttributePresent; import static org.opensearch.node.remotestore.RemoteStoreNodeService.REMOTE_STORE_COMPATIBILITY_MODE_SETTING; @@ -1072,6 +1076,7 @@ static Settings aggregateIndexSettings( validateTranslogRetentionSettings(indexSettings); validateStoreTypeSettings(indexSettings); validateRefreshIntervalSettings(request.settings(), clusterSettings); + validateTranslogFlushIntervalSettingsForCompositeIndex(request.settings(), clusterSettings); validateTranslogDurabilitySettings(request.settings(), clusterSettings, settings); return indexSettings; } @@ -1740,6 +1745,71 @@ public static void validateTranslogRetentionSettings(Settings indexSettings) { } } + /** + * Validates {@code index.translog.flush_threshold_size} is equal or below the {@code indices.composite_index.translog.max_flush_threshold_size} + * for composite indices based on {{@code index.composite_index}} + * + * @param requestSettings settings passed in during index create/update request + * @param clusterSettings cluster setting + */ + public static void validateTranslogFlushIntervalSettingsForCompositeIndex(Settings requestSettings, ClusterSettings clusterSettings) { + if (StarTreeIndexSettings.IS_COMPOSITE_INDEX_SETTING.exists(requestSettings) == false + || requestSettings.get(StarTreeIndexSettings.IS_COMPOSITE_INDEX_SETTING.getKey()) == null) { + return; + } + ByteSizeValue translogFlushSize = INDEX_TRANSLOG_FLUSH_THRESHOLD_SIZE_SETTING.get(requestSettings); + ByteSizeValue compositeIndexMaxFlushSize = clusterSettings.get( + CompositeIndexSettings.COMPOSITE_INDEX_MAX_TRANSLOG_FLUSH_THRESHOLD_SIZE_SETTING + ); + if (translogFlushSize.compareTo(compositeIndexMaxFlushSize) > 0) { + throw new IllegalArgumentException( + String.format( + Locale.ROOT, + "You can configure '%s' with upto '%s' for composite index", + INDEX_TRANSLOG_FLUSH_THRESHOLD_SIZE_SETTING.getKey(), + compositeIndexMaxFlushSize + ) + ); + } + } + + /** + * Validates {@code index.translog.flush_threshold_size} is equal or below the {@code indices.composite_index.translog.max_flush_threshold_size} + * for composite indices based on {{@code index.composite_index}} + * This is used during update index settings flow + * + * @param requestSettings settings passed in during index update request + * @param clusterSettings cluster setting + * @param indexSettings index settings + */ + public static Optional validateTranslogFlushIntervalSettingsForCompositeIndex( + Settings requestSettings, + ClusterSettings clusterSettings, + Settings indexSettings + ) { + if (INDEX_TRANSLOG_FLUSH_THRESHOLD_SIZE_SETTING.exists(requestSettings) == false + || requestSettings.get(INDEX_TRANSLOG_FLUSH_THRESHOLD_SIZE_SETTING.getKey()) == null + || StarTreeIndexSettings.IS_COMPOSITE_INDEX_SETTING.exists(indexSettings) == false + || indexSettings.get(StarTreeIndexSettings.IS_COMPOSITE_INDEX_SETTING.getKey()) == null) { + return Optional.empty(); + } + ByteSizeValue translogFlushSize = INDEX_TRANSLOG_FLUSH_THRESHOLD_SIZE_SETTING.get(requestSettings); + ByteSizeValue compositeIndexMaxFlushSize = clusterSettings.get( + CompositeIndexSettings.COMPOSITE_INDEX_MAX_TRANSLOG_FLUSH_THRESHOLD_SIZE_SETTING + ); + if (translogFlushSize.compareTo(compositeIndexMaxFlushSize) > 0) { + return Optional.of( + String.format( + Locale.ROOT, + "You can configure '%s' with upto '%s' for composite index", + INDEX_TRANSLOG_FLUSH_THRESHOLD_SIZE_SETTING.getKey(), + compositeIndexMaxFlushSize + ) + ); + } + return Optional.empty(); + } + /** * Validates {@code index.refresh_interval} is equal or below the {@code cluster.minimum.index.refresh_interval}. * diff --git a/server/src/main/java/org/opensearch/cluster/metadata/MetadataIndexTemplateService.java b/server/src/main/java/org/opensearch/cluster/metadata/MetadataIndexTemplateService.java index a7b9eba6dbc05..e4afc798cc64d 100644 --- a/server/src/main/java/org/opensearch/cluster/metadata/MetadataIndexTemplateService.java +++ b/server/src/main/java/org/opensearch/cluster/metadata/MetadataIndexTemplateService.java @@ -102,6 +102,7 @@ import static org.opensearch.cluster.metadata.MetadataCreateDataStreamService.validateTimestampFieldMapping; import static org.opensearch.cluster.metadata.MetadataCreateIndexService.validateRefreshIntervalSettings; +import static org.opensearch.cluster.metadata.MetadataCreateIndexService.validateTranslogFlushIntervalSettingsForCompositeIndex; import static org.opensearch.common.util.concurrent.ThreadContext.ACTION_ORIGIN_TRANSIENT_NAME; import static org.opensearch.indices.cluster.IndicesClusterStateService.AllocatedIndices.IndexRemovalReason.NO_LONGER_ASSIGNED; @@ -1639,6 +1640,7 @@ private void validate(String name, @Nullable Settings settings, List ind // validate index refresh interval and translog durability settings validateRefreshIntervalSettings(settings, clusterService.getClusterSettings()); + validateTranslogFlushIntervalSettingsForCompositeIndex(settings, clusterService.getClusterSettings()); validateTranslogDurabilitySettingsInTemplate(settings, clusterService.getClusterSettings()); } diff --git a/server/src/main/java/org/opensearch/cluster/metadata/MetadataUpdateSettingsService.java b/server/src/main/java/org/opensearch/cluster/metadata/MetadataUpdateSettingsService.java index 6713b85fee25a..4e7e31bbb9222 100644 --- a/server/src/main/java/org/opensearch/cluster/metadata/MetadataUpdateSettingsService.java +++ b/server/src/main/java/org/opensearch/cluster/metadata/MetadataUpdateSettingsService.java @@ -82,6 +82,7 @@ import static org.opensearch.cluster.metadata.MetadataCreateIndexService.validateOverlap; import static org.opensearch.cluster.metadata.MetadataCreateIndexService.validateRefreshIntervalSettings; import static org.opensearch.cluster.metadata.MetadataCreateIndexService.validateTranslogDurabilitySettings; +import static org.opensearch.cluster.metadata.MetadataCreateIndexService.validateTranslogFlushIntervalSettingsForCompositeIndex; import static org.opensearch.cluster.metadata.MetadataIndexTemplateService.findComponentTemplate; import static org.opensearch.common.settings.AbstractScopedSettings.ARCHIVED_SETTINGS_PREFIX; import static org.opensearch.index.IndexSettings.same; @@ -221,6 +222,12 @@ public ClusterState execute(ClusterState currentState) { index.getName() ).ifPresent(validationErrors::add); } + validateTranslogFlushIntervalSettingsForCompositeIndex( + normalizedSettings, + clusterService.getClusterSettings(), + metadata.getSettings() + ).ifPresent(validationErrors::add); + } if (validationErrors.size() > 0) { diff --git a/server/src/main/java/org/opensearch/cluster/metadata/TemplatesMetadata.java b/server/src/main/java/org/opensearch/cluster/metadata/TemplatesMetadata.java index 6ecc471c5e0ae..c01cd8f373c72 100644 --- a/server/src/main/java/org/opensearch/cluster/metadata/TemplatesMetadata.java +++ b/server/src/main/java/org/opensearch/cluster/metadata/TemplatesMetadata.java @@ -10,7 +10,9 @@ import org.opensearch.cluster.AbstractDiffable; import org.opensearch.common.annotation.PublicApi; +import org.opensearch.core.common.io.stream.BufferedChecksumStreamOutput; import org.opensearch.core.common.io.stream.StreamOutput; +import org.opensearch.core.common.io.stream.VerifiableWriteable; import org.opensearch.core.xcontent.ToXContentFragment; import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.core.xcontent.XContentParser; @@ -27,7 +29,7 @@ * @opensearch.api */ @PublicApi(since = "2.15.0") -public class TemplatesMetadata extends AbstractDiffable implements ToXContentFragment { +public class TemplatesMetadata extends AbstractDiffable implements ToXContentFragment, VerifiableWriteable { public static TemplatesMetadata EMPTY_METADATA = builder().build(); private final Map templates; @@ -65,6 +67,11 @@ public void writeTo(StreamOutput out) throws IOException { } } + @Override + public void writeVerifiableTo(BufferedChecksumStreamOutput out) throws IOException { + out.writeMapValues(templates, (stream, value) -> value.writeVerifiableTo((BufferedChecksumStreamOutput) stream)); + } + @Override public boolean equals(Object o) { if (this == o) return true; @@ -80,6 +87,11 @@ public int hashCode() { return templates != null ? templates.hashCode() : 0; } + @Override + public String toString() { + return "TemplatesMetadata{" + "templates=" + templates + '}'; + } + /** * Builder for the templates metadata * @@ -133,16 +145,15 @@ public static TemplatesMetadata fromXContent(XContentParser parser) throws IOExc String currentFieldName = parser.currentName(); if (currentFieldName == null) { token = parser.nextToken(); - if (token == XContentParser.Token.START_OBJECT) { - // move to the field name - token = parser.nextToken(); - } - currentFieldName = parser.currentName(); } - if (currentFieldName != null) { - while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) { + if (token == XContentParser.Token.START_OBJECT) { + // move to the field name + token = parser.nextToken(); + } + if (parser.currentName() != null && token != XContentParser.Token.END_OBJECT) { + do { builder.put(IndexTemplateMetadata.Builder.fromXContent(parser, parser.currentName())); - } + } while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT); } return builder.build(); } 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 653f81830ed17..8ac978a72b21b 100644 --- a/server/src/main/java/org/opensearch/cluster/node/DiscoveryNode.java +++ b/server/src/main/java/org/opensearch/cluster/node/DiscoveryNode.java @@ -37,9 +37,10 @@ import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.settings.Setting; import org.opensearch.common.settings.Settings; +import org.opensearch.core.common.io.stream.BufferedChecksumStreamOutput; 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.common.io.stream.VerifiableWriteable; import org.opensearch.core.common.transport.TransportAddress; import org.opensearch.core.xcontent.ToXContentFragment; import org.opensearch.core.xcontent.XContentBuilder; @@ -71,7 +72,7 @@ * @opensearch.api */ @PublicApi(since = "1.0.0") -public class DiscoveryNode implements Writeable, ToXContentFragment { +public class DiscoveryNode implements VerifiableWriteable, ToXContentFragment { static final String COORDINATING_ONLY = "coordinating_only"; @@ -329,6 +330,7 @@ public DiscoveryNode(StreamInput in) throws IOException { for (int i = 0; i < size; i++) { this.attributes.put(in.readString(), in.readString()); } + int rolesSize = in.readVInt(); final Set roles = new HashSet<>(rolesSize); for (int i = 0; i < rolesSize; i++) { @@ -358,17 +360,51 @@ public DiscoveryNode(StreamInput in) throws IOException { @Override public void writeTo(StreamOutput out) throws IOException { + if (out.getVersion().onOrAfter(Version.V_3_0_0)) { + writeToUtil(out, false); + } else { + writeToUtil(out, true); + } + + } + + public void writeToWithAttribute(StreamOutput out) throws IOException { + writeToUtil(out, true); + } + + public void writeToUtil(StreamOutput out, boolean includeAllAttributes) throws IOException { + writeNodeDetails(out); + + if (includeAllAttributes) { + out.writeVInt(attributes.size()); + for (Map.Entry entry : attributes.entrySet()) { + out.writeString(entry.getKey()); + out.writeString(entry.getValue()); + } + } else { + out.writeVInt(0); + } + + writeRolesAndVersion(out); + } + + @Override + public void writeVerifiableTo(BufferedChecksumStreamOutput out) throws IOException { + writeNodeDetails(out); + out.writeMap(attributes, StreamOutput::writeString, StreamOutput::writeString); + writeRolesAndVersion(out); + } + + private void writeNodeDetails(StreamOutput out) throws IOException { out.writeString(nodeName); out.writeString(nodeId); out.writeString(ephemeralId); out.writeString(hostName); out.writeString(hostAddress); address.writeTo(out); - out.writeVInt(attributes.size()); - for (Map.Entry entry : attributes.entrySet()) { - out.writeString(entry.getKey()); - out.writeString(entry.getValue()); - } + } + + private void writeRolesAndVersion(StreamOutput out) throws IOException { out.writeVInt(roles.size()); for (final DiscoveryNodeRole role : roles) { final DiscoveryNodeRole compatibleRole = role.getCompatibilityRole(out.getVersion()); 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..52d830aafda38 100644 --- a/server/src/main/java/org/opensearch/cluster/node/DiscoveryNodes.java +++ b/server/src/main/java/org/opensearch/cluster/node/DiscoveryNodes.java @@ -41,8 +41,10 @@ import org.opensearch.common.regex.Regex; import org.opensearch.common.util.set.Sets; import org.opensearch.core.common.Strings; +import org.opensearch.core.common.io.stream.BufferedChecksumStreamOutput; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; +import org.opensearch.core.common.io.stream.VerifiableWriteable; import org.opensearch.core.common.transport.TransportAddress; import java.io.IOException; @@ -66,7 +68,7 @@ * @opensearch.api */ @PublicApi(since = "1.0.0") -public class DiscoveryNodes extends AbstractDiffable implements Iterable { +public class DiscoveryNodes extends AbstractDiffable implements Iterable, VerifiableWriteable { public static final DiscoveryNodes EMPTY_NODES = builder().build(); @@ -688,16 +690,67 @@ public String shortSummary() { @Override public void writeTo(StreamOutput out) throws IOException { + writeToUtil((output, value) -> value.writeTo(output), out); + } + + public void writeToWithAttribute(StreamOutput out) throws IOException { + writeToUtil((output, value) -> value.writeToWithAttribute(output), out); + } + + public void writeToUtil(final Writer writer, StreamOutput out) throws IOException { + writeClusterManager(out); + out.writeVInt(nodes.size()); + for (DiscoveryNode node : this) { + writer.write(out, node); + } + } + + @Override + public void writeVerifiableTo(BufferedChecksumStreamOutput out) throws IOException { + writeClusterManager(out); + out.writeMapValues(nodes, (stream, val) -> val.writeVerifiableTo((BufferedChecksumStreamOutput) stream)); + } + + private void writeClusterManager(StreamOutput out) throws IOException { if (clusterManagerNodeId == null) { out.writeBoolean(false); } else { out.writeBoolean(true); out.writeString(clusterManagerNodeId); } - out.writeVInt(nodes.size()); - for (DiscoveryNode node : this) { - node.writeTo(out); - } + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + DiscoveryNodes that = (DiscoveryNodes) o; + return Objects.equals(nodes, that.nodes) + && Objects.equals(dataNodes, that.dataNodes) + && Objects.equals(clusterManagerNodes, that.clusterManagerNodes) + && Objects.equals(ingestNodes, that.ingestNodes) + && Objects.equals(clusterManagerNodeId, that.clusterManagerNodeId) + && Objects.equals(localNodeId, that.localNodeId) + && Objects.equals(minNonClientNodeVersion, that.minNonClientNodeVersion) + && Objects.equals(maxNonClientNodeVersion, that.maxNonClientNodeVersion) + && Objects.equals(maxNodeVersion, that.maxNodeVersion) + && Objects.equals(minNodeVersion, that.minNodeVersion); + } + + @Override + public int hashCode() { + return Objects.hash( + nodes, + dataNodes, + clusterManagerNodes, + ingestNodes, + clusterManagerNodeId, + localNodeId, + minNonClientNodeVersion, + maxNonClientNodeVersion, + maxNodeVersion, + minNodeVersion + ); } public static DiscoveryNodes readFrom(StreamInput in, DiscoveryNode localNode) throws IOException { diff --git a/server/src/main/java/org/opensearch/cluster/routing/IndexRoutingTable.java b/server/src/main/java/org/opensearch/cluster/routing/IndexRoutingTable.java index 80c9d42cdb617..9cc3bb21e2d12 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/IndexRoutingTable.java +++ b/server/src/main/java/org/opensearch/cluster/routing/IndexRoutingTable.java @@ -45,8 +45,10 @@ import org.opensearch.cluster.routing.RecoverySource.SnapshotRecoverySource; import org.opensearch.common.Randomness; import org.opensearch.common.annotation.PublicApi; +import org.opensearch.core.common.io.stream.BufferedChecksumStreamOutput; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; +import org.opensearch.core.common.io.stream.VerifiableWriteable; import org.opensearch.core.index.Index; import org.opensearch.core.index.shard.ShardId; @@ -79,7 +81,10 @@ * @opensearch.api */ @PublicApi(since = "1.0.0") -public class IndexRoutingTable extends AbstractDiffable implements Iterable { +public class IndexRoutingTable extends AbstractDiffable + implements + Iterable, + VerifiableWriteable { private final Index index; private final ShardShuffler shuffler; @@ -353,6 +358,11 @@ public int hashCode() { return result; } + @Override + public String toString() { + return "IndexRoutingTable{" + "shards=" + shards + ", index=" + index + '}'; + } + public static IndexRoutingTable readFrom(StreamInput in) throws IOException { Index index = new Index(in); Builder builder = new Builder(index); @@ -378,6 +388,11 @@ public void writeTo(StreamOutput out) throws IOException { } } + public void writeVerifiableTo(BufferedChecksumStreamOutput out) throws IOException { + index.writeTo(out); + out.writeMapValues(shards, (stream, value) -> IndexShardRoutingTable.Builder.writeVerifiableTo(value, stream)); + } + public static Builder builder(Index index) { return new Builder(index); } 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 f782cfc1b060a..4cc3300676986 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/IndexShardRoutingTable.java +++ b/server/src/main/java/org/opensearch/cluster/routing/IndexShardRoutingTable.java @@ -34,6 +34,7 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; +import org.apache.logging.log4j.message.ParameterizedMessage; import org.opensearch.cluster.AbstractDiffable; import org.opensearch.cluster.Diff; import org.opensearch.cluster.node.DiscoveryNode; @@ -62,6 +63,7 @@ import java.util.Map; import java.util.Optional; import java.util.Set; +import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Predicate; import java.util.stream.Collectors; @@ -1169,6 +1171,27 @@ public static void writeToThin(IndexShardRoutingTable indexShard, StreamOutput o } } + public static void writeVerifiableTo(IndexShardRoutingTable indexShard, StreamOutput out) throws IOException { + out.writeVInt(indexShard.shardId.id()); + out.writeVInt(indexShard.shards.size()); + // Order allocated shards by allocationId + AtomicInteger assignedShardCount = new AtomicInteger(); + indexShard.shards.stream() + .filter(shardRouting -> shardRouting.allocationId() != null) + .sorted(Comparator.comparing(o -> o.allocationId().getId())) + .forEach(shardRouting -> { + try { + assignedShardCount.getAndIncrement(); + shardRouting.writeToThin(out); + } catch (IOException e) { + logger.error(() -> new ParameterizedMessage("Failed to write shard {}. Exception {}", indexShard, e)); + throw new RuntimeException("Failed to write IndexShardRoutingTable", e); + } + }); + // is primary assigned + out.writeBoolean(indexShard.primaryShard().allocationId() != null); + out.writeVInt(indexShard.shards.size() - assignedShardCount.get()); + } } @Override diff --git a/server/src/main/java/org/opensearch/cluster/routing/RecoverySource.java b/server/src/main/java/org/opensearch/cluster/routing/RecoverySource.java index 8d407c6aff5b6..43e195ed47553 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/RecoverySource.java +++ b/server/src/main/java/org/opensearch/cluster/routing/RecoverySource.java @@ -48,6 +48,8 @@ import java.io.IOException; import java.util.Objects; +import static org.opensearch.Version.CURRENT; + /** * Represents the recovery source of a shard. Available recovery types are: *

@@ -264,6 +266,9 @@ public static class SnapshotRecoverySource extends RecoverySource { private final boolean isSearchableSnapshot; private final boolean remoteStoreIndexShallowCopy; private final String sourceRemoteStoreRepository; + private final String sourceRemoteTranslogRepository; + + private final long pinnedTimestamp; public SnapshotRecoverySource(String restoreUUID, Snapshot snapshot, Version version, IndexId indexId) { this(restoreUUID, snapshot, version, indexId, false, false, null); @@ -277,6 +282,30 @@ public SnapshotRecoverySource( boolean isSearchableSnapshot, boolean remoteStoreIndexShallowCopy, @Nullable String sourceRemoteStoreRepository + ) { + this( + restoreUUID, + snapshot, + version, + indexId, + isSearchableSnapshot, + remoteStoreIndexShallowCopy, + sourceRemoteStoreRepository, + null, + 0L + ); + } + + public SnapshotRecoverySource( + String restoreUUID, + Snapshot snapshot, + Version version, + IndexId indexId, + boolean isSearchableSnapshot, + boolean remoteStoreIndexShallowCopy, + @Nullable String sourceRemoteStoreRepository, + @Nullable String sourceRemoteTranslogRepository, + long pinnedTimestamp ) { this.restoreUUID = restoreUUID; this.snapshot = Objects.requireNonNull(snapshot); @@ -285,6 +314,8 @@ public SnapshotRecoverySource( this.isSearchableSnapshot = isSearchableSnapshot; this.remoteStoreIndexShallowCopy = remoteStoreIndexShallowCopy; this.sourceRemoteStoreRepository = sourceRemoteStoreRepository; + this.sourceRemoteTranslogRepository = sourceRemoteTranslogRepository; + this.pinnedTimestamp = pinnedTimestamp; } SnapshotRecoverySource(StreamInput in) throws IOException { @@ -304,6 +335,13 @@ public SnapshotRecoverySource( remoteStoreIndexShallowCopy = false; sourceRemoteStoreRepository = null; } + if (in.getVersion().onOrAfter(CURRENT)) { + sourceRemoteTranslogRepository = in.readOptionalString(); + pinnedTimestamp = in.readLong(); + } else { + sourceRemoteTranslogRepository = null; + pinnedTimestamp = 0L; + } } public String restoreUUID() { @@ -336,10 +374,18 @@ public String sourceRemoteStoreRepository() { return sourceRemoteStoreRepository; } + public String sourceRemoteTranslogRepository() { + return sourceRemoteTranslogRepository; + } + public boolean remoteStoreIndexShallowCopy() { return remoteStoreIndexShallowCopy; } + public long pinnedTimestamp() { + return pinnedTimestamp; + } + @Override protected void writeAdditionalFields(StreamOutput out) throws IOException { out.writeString(restoreUUID); @@ -353,6 +399,10 @@ protected void writeAdditionalFields(StreamOutput out) throws IOException { out.writeBoolean(remoteStoreIndexShallowCopy); out.writeOptionalString(sourceRemoteStoreRepository); } + if (out.getVersion().onOrAfter(CURRENT)) { + out.writeOptionalString(sourceRemoteTranslogRepository); + out.writeLong(pinnedTimestamp); + } } @Override @@ -369,7 +419,8 @@ public void addAdditionalFields(XContentBuilder builder, ToXContent.Params param .field("restoreUUID", restoreUUID) .field("isSearchableSnapshot", isSearchableSnapshot) .field("remoteStoreIndexShallowCopy", remoteStoreIndexShallowCopy) - .field("sourceRemoteStoreRepository", sourceRemoteStoreRepository); + .field("sourceRemoteStoreRepository", sourceRemoteStoreRepository) + .field("sourceRemoteTranslogRepository", sourceRemoteTranslogRepository); } @Override @@ -394,8 +445,11 @@ public boolean equals(Object o) { && isSearchableSnapshot == that.isSearchableSnapshot && remoteStoreIndexShallowCopy == that.remoteStoreIndexShallowCopy && sourceRemoteStoreRepository != null - ? sourceRemoteStoreRepository.equals(that.sourceRemoteStoreRepository) - : that.sourceRemoteStoreRepository == null; + ? sourceRemoteStoreRepository.equals(that.sourceRemoteStoreRepository) + : that.sourceRemoteStoreRepository == null && sourceRemoteTranslogRepository != null + ? sourceRemoteTranslogRepository.equals(that.sourceRemoteTranslogRepository) + : that.sourceRemoteTranslogRepository == null && pinnedTimestamp == that.pinnedTimestamp; + } @Override @@ -407,10 +461,11 @@ public int hashCode() { version, isSearchableSnapshot, remoteStoreIndexShallowCopy, - sourceRemoteStoreRepository + sourceRemoteStoreRepository, + sourceRemoteTranslogRepository, + pinnedTimestamp ); } - } /** 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 544d4f9326fba..7128eb44bfb14 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/RoutingTable.java +++ b/server/src/main/java/org/opensearch/cluster/routing/RoutingTable.java @@ -42,8 +42,10 @@ import org.opensearch.common.Nullable; import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.util.iterable.Iterables; +import org.opensearch.core.common.io.stream.BufferedChecksumStreamOutput; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; +import org.opensearch.core.common.io.stream.VerifiableWriteable; import org.opensearch.core.index.Index; import org.opensearch.core.index.shard.ShardId; import org.opensearch.index.IndexNotFoundException; @@ -70,7 +72,7 @@ * @opensearch.api */ @PublicApi(since = "1.0.0") -public class RoutingTable implements Iterable, Diffable { +public class RoutingTable implements Iterable, Diffable, VerifiableWriteable { public static final RoutingTable EMPTY_ROUTING_TABLE = builder().build(); @@ -407,6 +409,12 @@ public void writeTo(StreamOutput out) throws IOException { } } + @Override + public void writeVerifiableTo(BufferedChecksumStreamOutput out) throws IOException { + out.writeLong(version); + out.writeMapValues(indicesRouting, (stream, value) -> value.writeVerifiableTo((BufferedChecksumStreamOutput) stream)); + } + private static class RoutingTableDiff implements Diff, StringKeyDiffProvider { private final long version; @@ -426,6 +434,11 @@ private static class RoutingTableDiff implements Diff, StringKeyDi indicesRouting = DiffableUtils.readJdkMapDiff(in, DiffableUtils.getStringKeySerializer(), DIFF_VALUE_READER); } + @Override + public String toString() { + return "RoutingTableDiff{" + "version=" + version + ", indicesRouting=" + indicesRouting + '}'; + } + @Override public RoutingTable apply(RoutingTable part) { return new RoutingTable(version, indicesRouting.apply(part.indicesRouting)); diff --git a/server/src/main/java/org/opensearch/cluster/routing/allocation/AbstractAllocationDecision.java b/server/src/main/java/org/opensearch/cluster/routing/allocation/AbstractAllocationDecision.java index 59a39b358cb70..614e9f49c8726 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/allocation/AbstractAllocationDecision.java +++ b/server/src/main/java/org/opensearch/cluster/routing/allocation/AbstractAllocationDecision.java @@ -107,7 +107,7 @@ public List getNodeDecisions() { @Override public void writeTo(StreamOutput out) throws IOException { - out.writeOptionalWriteable(targetNode); + out.writeOptionalWriteable((stream, node) -> node.writeToWithAttribute(stream), targetNode); if (nodeDecisions != null) { out.writeBoolean(true); out.writeList(nodeDecisions); diff --git a/server/src/main/java/org/opensearch/cluster/routing/allocation/NodeAllocationResult.java b/server/src/main/java/org/opensearch/cluster/routing/allocation/NodeAllocationResult.java index 4163a5fd4c16f..6b805ca91fa58 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/allocation/NodeAllocationResult.java +++ b/server/src/main/java/org/opensearch/cluster/routing/allocation/NodeAllocationResult.java @@ -104,7 +104,7 @@ public NodeAllocationResult(StreamInput in) throws IOException { @Override public void writeTo(StreamOutput out) throws IOException { - node.writeTo(out); + node.writeToWithAttribute(out); out.writeOptionalWriteable(shardStoreInfo); out.writeOptionalWriteable(canAllocateDecision); nodeDecision.writeTo(out); 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 31e47dc14db23..8e036209daff0 100644 --- a/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java +++ b/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java @@ -636,6 +636,7 @@ public void apply(Settings value, Settings current, Settings previous) { HandshakingTransportAddressConnector.PROBE_CONNECT_TIMEOUT_SETTING, HandshakingTransportAddressConnector.PROBE_HANDSHAKE_TIMEOUT_SETTING, SnapshotsService.MAX_CONCURRENT_SNAPSHOT_OPERATIONS_SETTING, + SnapshotsService.MAX_SHARDS_ALLOWED_IN_STATUS_API, FsHealthService.ENABLED_SETTING, FsHealthService.REFRESH_INTERVAL_SETTING, FsHealthService.SLOW_PATH_LOGGING_THRESHOLD_SETTING, @@ -739,6 +740,7 @@ public void apply(Settings value, Settings current, Settings previous) { IndicesService.CLUSTER_INDEX_RESTRICT_REPLICATION_TYPE_SETTING, RemoteRoutingTableBlobStore.REMOTE_ROUTING_TABLE_PATH_TYPE_SETTING, RemoteRoutingTableBlobStore.REMOTE_ROUTING_TABLE_PATH_HASH_ALGO_SETTING, + RemoteClusterStateService.REMOTE_CLUSTER_STATE_CHECKSUM_VALIDATION_ENABLED_SETTING, // Admission Control Settings AdmissionControlSettings.ADMISSION_CONTROL_TRANSPORT_LAYER_MODE, @@ -765,11 +767,14 @@ public void apply(Settings value, Settings current, Settings previous) { RemoteStoreSettings.CLUSTER_REMOTE_STORE_PINNED_TIMESTAMP_SCHEDULER_INTERVAL, RemoteStoreSettings.CLUSTER_REMOTE_STORE_PINNED_TIMESTAMP_LOOKBACK_INTERVAL, RemoteStoreSettings.CLUSTER_REMOTE_STORE_PINNED_TIMESTAMP_ENABLED, + RemoteStoreSettings.CLUSTER_REMOTE_STORE_SEGMENTS_PATH_PREFIX, + RemoteStoreSettings.CLUSTER_REMOTE_STORE_TRANSLOG_PATH_PREFIX, SearchService.CLUSTER_ALLOW_DERIVED_FIELD_SETTING, // Composite index settings CompositeIndexSettings.STAR_TREE_INDEX_ENABLED_SETTING, + CompositeIndexSettings.COMPOSITE_INDEX_MAX_TRANSLOG_FLUSH_THRESHOLD_SIZE_SETTING, SystemTemplatesService.SETTING_APPLICATION_BASED_CONFIGURATION_TEMPLATES_ENABLED, diff --git a/server/src/main/java/org/opensearch/common/settings/IndexScopedSettings.java b/server/src/main/java/org/opensearch/common/settings/IndexScopedSettings.java index 9cad80e8e3e82..00c73de1c1780 100644 --- a/server/src/main/java/org/opensearch/common/settings/IndexScopedSettings.java +++ b/server/src/main/java/org/opensearch/common/settings/IndexScopedSettings.java @@ -249,6 +249,8 @@ public final class IndexScopedSettings extends AbstractScopedSettings { StarTreeIndexSettings.DEFAULT_METRICS_LIST, StarTreeIndexSettings.DEFAULT_DATE_INTERVALS, StarTreeIndexSettings.STAR_TREE_MAX_DATE_INTERVALS_SETTING, + StarTreeIndexSettings.STAR_TREE_MAX_BASE_METRICS_SETTING, + StarTreeIndexSettings.IS_COMPOSITE_INDEX_SETTING, IndexSettings.INDEX_CONTEXT_CREATED_VERSION, IndexSettings.INDEX_CONTEXT_CURRENT_VERSION, 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 8025940b35997..405e5cd784196 100644 --- a/server/src/main/java/org/opensearch/gateway/remote/ClusterMetadataManifest.java +++ b/server/src/main/java/org/opensearch/gateway/remote/ClusterMetadataManifest.java @@ -45,7 +45,8 @@ public class ClusterMetadataManifest implements Writeable, ToXContentFragment { // also we introduce index routing-metadata, diff and other attributes as part of manifest // required for state publication public static final int CODEC_V3 = 3; // In Codec V3, we have introduced new diff field in diff-manifest's routing_table_diff - public static final int CODEC_V4 = 4; // In Codec V4, we have removed upserts and delete field for routing table in diff manifest + public static final int CODEC_V4 = 4; // In Codec V4, we have removed upserts and delete field for routing table in diff manifest and + // added checksum of cluster state. public static final int[] CODEC_VERSIONS = { CODEC_V0, CODEC_V1, CODEC_V2, CODEC_V3, CODEC_V4 }; private static final ParseField CLUSTER_TERM_FIELD = new ParseField("cluster_term"); @@ -75,6 +76,7 @@ public class ClusterMetadataManifest implements Writeable, ToXContentFragment { ); 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 CHECKSUM = new ParseField("checksum"); private static ClusterMetadataManifest.Builder manifestV0Builder(Object[] fields) { return ClusterMetadataManifest.builder() @@ -117,7 +119,7 @@ private static ClusterMetadataManifest.Builder manifestV3Builder(Object[] fields } private static ClusterMetadataManifest.Builder manifestV4Builder(Object[] fields) { - return manifestV3Builder(fields); + return manifestV3Builder(fields).checksum(checksum(fields)); } private static long term(Object[] fields) { @@ -222,6 +224,10 @@ private static ClusterStateDiffManifest diffManifest(Object[] fields) { return (ClusterStateDiffManifest) fields[23]; } + private static ClusterStateChecksum checksum(Object[] fields) { + return (ClusterStateChecksum) fields[24]; + } + private static final ConstructingObjectParser PARSER_V0 = new ConstructingObjectParser<>( "cluster_metadata_manifest", fields -> manifestV0Builder(fields).build() @@ -248,6 +254,7 @@ private static ClusterStateDiffManifest diffManifest(Object[] fields) { ); private static final ConstructingObjectParser CURRENT_PARSER = PARSER_V4; + public static final int MANIFEST_CURRENT_CODEC_VERSION = CODEC_V4; private static final Map VERSION_TO_CODEC_MAPPING; @@ -360,6 +367,13 @@ private static void declareParser(ConstructingObjectParser= CODEC_V4) { + parser.declareObject( + ConstructingObjectParser.optionalConstructorArg(), + (p, c) -> ClusterStateChecksum.fromXContent(p), + CHECKSUM + ); + } } private final int codecVersion; @@ -387,6 +401,7 @@ private static void declareParser(ConstructingObjectParser uploadedClusterStateCustomMap; private final ClusterStateDiffManifest diffManifest; + private ClusterStateChecksum clusterStateChecksum; public List getIndices() { return indices; @@ -495,6 +510,10 @@ public List getIndicesRouting() { return indicesRouting; } + public ClusterStateChecksum getClusterStateChecksum() { + return clusterStateChecksum; + } + public ClusterMetadataManifest( long clusterTerm, long version, @@ -520,7 +539,8 @@ public ClusterMetadataManifest( UploadedMetadataAttribute uploadedTransientSettingsMetadata, UploadedMetadataAttribute uploadedHashesOfConsistentSettings, Map uploadedClusterStateCustomMap, - ClusterStateDiffManifest diffManifest + ClusterStateDiffManifest diffManifest, + ClusterStateChecksum clusterStateChecksum ) { this.clusterTerm = clusterTerm; this.stateVersion = version; @@ -551,6 +571,7 @@ public ClusterMetadataManifest( this.uploadedClusterStateCustomMap = Collections.unmodifiableMap( uploadedClusterStateCustomMap != null ? uploadedClusterStateCustomMap : new HashMap<>() ); + this.clusterStateChecksum = clusterStateChecksum; } public ClusterMetadataManifest(StreamInput in) throws IOException { @@ -564,6 +585,7 @@ public ClusterMetadataManifest(StreamInput in) throws IOException { this.indices = Collections.unmodifiableList(in.readList(UploadedIndexMetadata::new)); this.previousClusterUUID = in.readString(); this.clusterUUIDCommitted = in.readBoolean(); + clusterStateChecksum = null; if (in.getVersion().onOrAfter(Version.V_2_15_0)) { this.codecVersion = in.readInt(); this.uploadedCoordinationMetadata = new UploadedMetadataAttribute(in); @@ -626,6 +648,9 @@ public ClusterMetadataManifest(StreamInput in) throws IOException { this.uploadedHashesOfConsistentSettings = null; this.uploadedClusterStateCustomMap = null; } + if (in.getVersion().onOrAfter(Version.V_2_17_0) && in.readBoolean()) { + clusterStateChecksum = new ClusterStateChecksum(in); + } } public static Builder builder() { @@ -723,6 +748,13 @@ 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_V4)) { + if (getClusterStateChecksum() != null) { + builder.startObject(CHECKSUM.getPreferredName()); + getClusterStateChecksum().toXContent(builder, params); + builder.endObject(); + } + } return builder; } @@ -782,6 +814,14 @@ public void writeTo(StreamOutput out) throws IOException { out.writeInt(codecVersion); out.writeString(globalMetadataFileName); } + if (out.getVersion().onOrAfter(Version.V_2_17_0)) { + if (clusterStateChecksum != null) { + out.writeBoolean(true); + clusterStateChecksum.writeTo(out); + } else { + out.writeBoolean(false); + } + } } @Override @@ -817,7 +857,8 @@ public boolean equals(Object o) { && Objects.equals(uploadedTransientSettingsMetadata, that.uploadedTransientSettingsMetadata) && Objects.equals(uploadedHashesOfConsistentSettings, that.uploadedHashesOfConsistentSettings) && Objects.equals(uploadedClusterStateCustomMap, that.uploadedClusterStateCustomMap) - && Objects.equals(diffManifest, that.diffManifest); + && Objects.equals(diffManifest, that.diffManifest) + && Objects.equals(clusterStateChecksum, that.clusterStateChecksum); } @Override @@ -847,7 +888,8 @@ public int hashCode() { uploadedTransientSettingsMetadata, uploadedHashesOfConsistentSettings, uploadedClusterStateCustomMap, - diffManifest + diffManifest, + clusterStateChecksum ); } @@ -912,6 +954,7 @@ public static class Builder { private UploadedMetadataAttribute hashesOfConsistentSettings; private Map clusterStateCustomMetadataMap; private ClusterStateDiffManifest diffManifest; + private ClusterStateChecksum checksum; public Builder indices(List indices) { this.indices = indices; @@ -1051,6 +1094,11 @@ public Builder diffManifest(ClusterStateDiffManifest diffManifest) { return this; } + public Builder checksum(ClusterStateChecksum checksum) { + this.checksum = checksum; + return this; + } + public Builder() { indices = new ArrayList<>(); customMetadataMap = new HashMap<>(); @@ -1083,6 +1131,7 @@ public Builder(ClusterMetadataManifest manifest) { this.diffManifest = manifest.diffManifest; this.hashesOfConsistentSettings = manifest.uploadedHashesOfConsistentSettings; this.clusterStateCustomMetadataMap = manifest.uploadedClusterStateCustomMap; + this.checksum = manifest.clusterStateChecksum; } public ClusterMetadataManifest build() { @@ -1111,7 +1160,8 @@ public ClusterMetadataManifest build() { transientSettingsMetadata, hashesOfConsistentSettings, clusterStateCustomMetadataMap, - diffManifest + diffManifest, + checksum ); } diff --git a/server/src/main/java/org/opensearch/gateway/remote/ClusterStateChecksum.java b/server/src/main/java/org/opensearch/gateway/remote/ClusterStateChecksum.java new file mode 100644 index 0000000000000..d6739c4572d1a --- /dev/null +++ b/server/src/main/java/org/opensearch/gateway/remote/ClusterStateChecksum.java @@ -0,0 +1,485 @@ +/* + * 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.opensearch.cluster.ClusterState; +import org.opensearch.cluster.metadata.DiffableStringMap; +import org.opensearch.common.io.stream.BytesStreamOutput; +import org.opensearch.common.settings.Settings; +import org.opensearch.core.common.io.stream.BufferedChecksumStreamOutput; +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.ToXContentFragment; +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.List; +import java.util.Objects; + +import com.jcraft.jzlib.JZlib; + +import static org.opensearch.core.xcontent.XContentParserUtils.ensureExpectedToken; + +/** + * Stores checksum for all components in cluster state. This will be used to ensure cluster state is same across all nodes in the cluster. + */ +public class ClusterStateChecksum implements ToXContentFragment, Writeable { + + static final String ROUTING_TABLE_CS = "routing_table"; + static final String NODES_CS = "discovery_nodes"; + static final String BLOCKS_CS = "blocks"; + static final String CUSTOMS_CS = "customs"; + static final String COORDINATION_MD_CS = "coordination_md"; + static final String SETTINGS_MD_CS = "settings_md"; + static final String TRANSIENT_SETTINGS_MD_CS = "transient_settings_md"; + static final String TEMPLATES_MD_CS = "templates_md"; + static final String CUSTOM_MD_CS = "customs_md"; + static final String HASHES_MD_CS = "hashes_md"; + static final String INDICES_CS = "indices_md"; + private static final String CLUSTER_STATE_CS = "cluster_state"; + private static final int CHECKSUM_SIZE = 8; + private static final Logger logger = LogManager.getLogger(ClusterStateChecksum.class); + + long routingTableChecksum; + long nodesChecksum; + long blocksChecksum; + long clusterStateCustomsChecksum; + long coordinationMetadataChecksum; + long settingMetadataChecksum; + long transientSettingsMetadataChecksum; + long templatesMetadataChecksum; + long customMetadataMapChecksum; + long hashesOfConsistentSettingsChecksum; + long indicesChecksum; + long clusterStateChecksum; + + public ClusterStateChecksum(ClusterState clusterState) { + try ( + BytesStreamOutput out = new BytesStreamOutput(); + BufferedChecksumStreamOutput checksumOut = new BufferedChecksumStreamOutput(out) + ) { + clusterState.routingTable().writeVerifiableTo(checksumOut); + routingTableChecksum = checksumOut.getChecksum(); + + checksumOut.reset(); + clusterState.nodes().writeVerifiableTo(checksumOut); + nodesChecksum = checksumOut.getChecksum(); + + checksumOut.reset(); + clusterState.coordinationMetadata().writeVerifiableTo(checksumOut); + coordinationMetadataChecksum = checksumOut.getChecksum(); + + // Settings create sortedMap by default, so no explicit sorting required here. + checksumOut.reset(); + Settings.writeSettingsToStream(clusterState.metadata().persistentSettings(), checksumOut); + settingMetadataChecksum = checksumOut.getChecksum(); + + checksumOut.reset(); + Settings.writeSettingsToStream(clusterState.metadata().transientSettings(), checksumOut); + transientSettingsMetadataChecksum = checksumOut.getChecksum(); + + checksumOut.reset(); + clusterState.metadata().templatesMetadata().writeVerifiableTo(checksumOut); + templatesMetadataChecksum = checksumOut.getChecksum(); + + checksumOut.reset(); + checksumOut.writeStringCollection(clusterState.metadata().customs().keySet()); + customMetadataMapChecksum = checksumOut.getChecksum(); + + checksumOut.reset(); + ((DiffableStringMap) clusterState.metadata().hashesOfConsistentSettings()).writeTo(checksumOut); + hashesOfConsistentSettingsChecksum = checksumOut.getChecksum(); + + checksumOut.reset(); + checksumOut.writeMapValues( + clusterState.metadata().indices(), + (stream, value) -> value.writeVerifiableTo((BufferedChecksumStreamOutput) stream) + ); + indicesChecksum = checksumOut.getChecksum(); + + checksumOut.reset(); + clusterState.blocks().writeVerifiableTo(checksumOut); + blocksChecksum = checksumOut.getChecksum(); + + checksumOut.reset(); + checksumOut.writeStringCollection(clusterState.customs().keySet()); + clusterStateCustomsChecksum = checksumOut.getChecksum(); + } catch (IOException e) { + logger.error("Failed to create checksum for cluster state.", e); + throw new RemoteStateTransferException("Failed to create checksum for cluster state.", e); + } + createClusterStateChecksum(); + } + + private void createClusterStateChecksum() { + clusterStateChecksum = JZlib.crc32_combine(routingTableChecksum, nodesChecksum, CHECKSUM_SIZE); + clusterStateChecksum = JZlib.crc32_combine(clusterStateChecksum, blocksChecksum, CHECKSUM_SIZE); + clusterStateChecksum = JZlib.crc32_combine(clusterStateChecksum, clusterStateCustomsChecksum, CHECKSUM_SIZE); + clusterStateChecksum = JZlib.crc32_combine(clusterStateChecksum, coordinationMetadataChecksum, CHECKSUM_SIZE); + clusterStateChecksum = JZlib.crc32_combine(clusterStateChecksum, settingMetadataChecksum, CHECKSUM_SIZE); + clusterStateChecksum = JZlib.crc32_combine(clusterStateChecksum, transientSettingsMetadataChecksum, CHECKSUM_SIZE); + clusterStateChecksum = JZlib.crc32_combine(clusterStateChecksum, templatesMetadataChecksum, CHECKSUM_SIZE); + clusterStateChecksum = JZlib.crc32_combine(clusterStateChecksum, customMetadataMapChecksum, CHECKSUM_SIZE); + clusterStateChecksum = JZlib.crc32_combine(clusterStateChecksum, hashesOfConsistentSettingsChecksum, CHECKSUM_SIZE); + clusterStateChecksum = JZlib.crc32_combine(clusterStateChecksum, indicesChecksum, CHECKSUM_SIZE); + } + + public static ClusterStateChecksum.Builder builder() { + return new ClusterStateChecksum.Builder(); + } + + public ClusterStateChecksum( + long routingTableChecksum, + long nodesChecksum, + long blocksChecksum, + long clusterStateCustomsChecksum, + long coordinationMetadataChecksum, + long settingMetadataChecksum, + long transientSettingsMetadataChecksum, + long templatesMetadataChecksum, + long customMetadataMapChecksum, + long hashesOfConsistentSettingsChecksum, + long indicesChecksum, + long clusterStateChecksum + ) { + this.routingTableChecksum = routingTableChecksum; + this.nodesChecksum = nodesChecksum; + this.blocksChecksum = blocksChecksum; + this.clusterStateCustomsChecksum = clusterStateCustomsChecksum; + this.coordinationMetadataChecksum = coordinationMetadataChecksum; + this.settingMetadataChecksum = settingMetadataChecksum; + this.transientSettingsMetadataChecksum = transientSettingsMetadataChecksum; + this.templatesMetadataChecksum = templatesMetadataChecksum; + this.customMetadataMapChecksum = customMetadataMapChecksum; + this.hashesOfConsistentSettingsChecksum = hashesOfConsistentSettingsChecksum; + this.indicesChecksum = indicesChecksum; + this.clusterStateChecksum = clusterStateChecksum; + } + + public ClusterStateChecksum(StreamInput in) throws IOException { + routingTableChecksum = in.readLong(); + nodesChecksum = in.readLong(); + blocksChecksum = in.readLong(); + clusterStateCustomsChecksum = in.readLong(); + coordinationMetadataChecksum = in.readLong(); + settingMetadataChecksum = in.readLong(); + transientSettingsMetadataChecksum = in.readLong(); + templatesMetadataChecksum = in.readLong(); + customMetadataMapChecksum = in.readLong(); + hashesOfConsistentSettingsChecksum = in.readLong(); + indicesChecksum = in.readLong(); + clusterStateChecksum = in.readLong(); + } + + public static ClusterStateChecksum fromXContent(XContentParser parser) throws IOException { + ClusterStateChecksum.Builder builder = new ClusterStateChecksum.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); + XContentParser.Token token; + String currentFieldName = parser.currentName(); + while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) { + if (parser.currentToken() == XContentParser.Token.FIELD_NAME) { + currentFieldName = parser.currentName(); + } else if (token.isValue()) { + switch (currentFieldName) { + case ROUTING_TABLE_CS: + builder.routingTableChecksum(parser.longValue()); + break; + case NODES_CS: + builder.nodesChecksum(parser.longValue()); + break; + case BLOCKS_CS: + builder.blocksChecksum(parser.longValue()); + break; + case CUSTOMS_CS: + builder.clusterStateCustomsChecksum(parser.longValue()); + break; + case COORDINATION_MD_CS: + builder.coordinationMetadataChecksum(parser.longValue()); + break; + case SETTINGS_MD_CS: + builder.settingMetadataChecksum(parser.longValue()); + break; + case TRANSIENT_SETTINGS_MD_CS: + builder.transientSettingsMetadataChecksum(parser.longValue()); + break; + case TEMPLATES_MD_CS: + builder.templatesMetadataChecksum(parser.longValue()); + break; + case CUSTOM_MD_CS: + builder.customMetadataMapChecksum(parser.longValue()); + break; + case HASHES_MD_CS: + builder.hashesOfConsistentSettingsChecksum(parser.longValue()); + break; + case INDICES_CS: + builder.indicesChecksum(parser.longValue()); + break; + case CLUSTER_STATE_CS: + builder.clusterStateChecksum(parser.longValue()); + break; + default: + throw new XContentParseException("Unexpected field [" + currentFieldName + "]"); + } + } else { + throw new XContentParseException("Unexpected token [" + token + "]"); + } + } + return builder.build(); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeLong(routingTableChecksum); + out.writeLong(nodesChecksum); + out.writeLong(blocksChecksum); + out.writeLong(clusterStateCustomsChecksum); + out.writeLong(coordinationMetadataChecksum); + out.writeLong(settingMetadataChecksum); + out.writeLong(transientSettingsMetadataChecksum); + out.writeLong(templatesMetadataChecksum); + out.writeLong(customMetadataMapChecksum); + out.writeLong(hashesOfConsistentSettingsChecksum); + out.writeLong(indicesChecksum); + out.writeLong(clusterStateChecksum); + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.field(ROUTING_TABLE_CS, routingTableChecksum); + builder.field(NODES_CS, nodesChecksum); + builder.field(BLOCKS_CS, blocksChecksum); + builder.field(CUSTOMS_CS, clusterStateCustomsChecksum); + builder.field(COORDINATION_MD_CS, coordinationMetadataChecksum); + builder.field(SETTINGS_MD_CS, settingMetadataChecksum); + builder.field(TRANSIENT_SETTINGS_MD_CS, transientSettingsMetadataChecksum); + builder.field(TEMPLATES_MD_CS, templatesMetadataChecksum); + builder.field(CUSTOM_MD_CS, customMetadataMapChecksum); + builder.field(HASHES_MD_CS, hashesOfConsistentSettingsChecksum); + builder.field(INDICES_CS, indicesChecksum); + builder.field(CLUSTER_STATE_CS, clusterStateChecksum); + return builder; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + ClusterStateChecksum that = (ClusterStateChecksum) o; + return routingTableChecksum == that.routingTableChecksum + && nodesChecksum == that.nodesChecksum + && blocksChecksum == that.blocksChecksum + && clusterStateCustomsChecksum == that.clusterStateCustomsChecksum + && coordinationMetadataChecksum == that.coordinationMetadataChecksum + && settingMetadataChecksum == that.settingMetadataChecksum + && transientSettingsMetadataChecksum == that.transientSettingsMetadataChecksum + && templatesMetadataChecksum == that.templatesMetadataChecksum + && customMetadataMapChecksum == that.customMetadataMapChecksum + && hashesOfConsistentSettingsChecksum == that.hashesOfConsistentSettingsChecksum + && indicesChecksum == that.indicesChecksum + && clusterStateChecksum == that.clusterStateChecksum; + } + + @Override + public int hashCode() { + return Objects.hash( + routingTableChecksum, + nodesChecksum, + blocksChecksum, + clusterStateCustomsChecksum, + coordinationMetadataChecksum, + settingMetadataChecksum, + transientSettingsMetadataChecksum, + templatesMetadataChecksum, + customMetadataMapChecksum, + hashesOfConsistentSettingsChecksum, + indicesChecksum, + clusterStateChecksum + ); + } + + @Override + public String toString() { + return "ClusterStateChecksum{" + + "routingTableChecksum=" + + routingTableChecksum + + ", nodesChecksum=" + + nodesChecksum + + ", blocksChecksum=" + + blocksChecksum + + ", clusterStateCustomsChecksum=" + + clusterStateCustomsChecksum + + ", coordinationMetadataChecksum=" + + coordinationMetadataChecksum + + ", settingMetadataChecksum=" + + settingMetadataChecksum + + ", transientSettingsMetadataChecksum=" + + transientSettingsMetadataChecksum + + ", templatesMetadataChecksum=" + + templatesMetadataChecksum + + ", customMetadataMapChecksum=" + + customMetadataMapChecksum + + ", hashesOfConsistentSettingsChecksum=" + + hashesOfConsistentSettingsChecksum + + ", indicesChecksum=" + + indicesChecksum + + ", clusterStateChecksum=" + + clusterStateChecksum + + '}'; + } + + public List getMismatchEntities(ClusterStateChecksum otherClusterStateChecksum) { + if (this.clusterStateChecksum == otherClusterStateChecksum.clusterStateChecksum) { + logger.debug("No mismatch in checksums."); + return List.of(); + } + List mismatches = new ArrayList<>(); + addIfMismatch(this.routingTableChecksum, otherClusterStateChecksum.routingTableChecksum, ROUTING_TABLE_CS, mismatches); + addIfMismatch(this.nodesChecksum, otherClusterStateChecksum.nodesChecksum, NODES_CS, mismatches); + addIfMismatch(this.blocksChecksum, otherClusterStateChecksum.blocksChecksum, BLOCKS_CS, mismatches); + addIfMismatch(this.clusterStateCustomsChecksum, otherClusterStateChecksum.clusterStateCustomsChecksum, CUSTOMS_CS, mismatches); + addIfMismatch( + this.coordinationMetadataChecksum, + otherClusterStateChecksum.coordinationMetadataChecksum, + COORDINATION_MD_CS, + mismatches + ); + addIfMismatch(this.settingMetadataChecksum, otherClusterStateChecksum.settingMetadataChecksum, SETTINGS_MD_CS, mismatches); + addIfMismatch( + this.transientSettingsMetadataChecksum, + otherClusterStateChecksum.transientSettingsMetadataChecksum, + TRANSIENT_SETTINGS_MD_CS, + mismatches + ); + addIfMismatch(this.templatesMetadataChecksum, otherClusterStateChecksum.templatesMetadataChecksum, TEMPLATES_MD_CS, mismatches); + addIfMismatch(this.customMetadataMapChecksum, otherClusterStateChecksum.customMetadataMapChecksum, CUSTOM_MD_CS, mismatches); + addIfMismatch( + this.hashesOfConsistentSettingsChecksum, + otherClusterStateChecksum.hashesOfConsistentSettingsChecksum, + HASHES_MD_CS, + mismatches + ); + addIfMismatch(this.indicesChecksum, otherClusterStateChecksum.indicesChecksum, INDICES_CS, mismatches); + + return mismatches; + } + + private void addIfMismatch(long checksum, long otherChecksum, String entityName, List mismatches) { + if (checksum != otherChecksum) { + mismatches.add(entityName); + } + } + + /** + * Builder for ClusterStateChecksum + */ + public static class Builder { + long routingTableChecksum; + long nodesChecksum; + long blocksChecksum; + long clusterStateCustomsChecksum; + long coordinationMetadataChecksum; + long settingMetadataChecksum; + long transientSettingsMetadataChecksum; + long templatesMetadataChecksum; + long customMetadataMapChecksum; + long hashesOfConsistentSettingsChecksum; + long indicesChecksum; + long clusterStateChecksum; + + public Builder routingTableChecksum(long routingTableChecksum) { + this.routingTableChecksum = routingTableChecksum; + return this; + } + + public Builder nodesChecksum(long nodesChecksum) { + this.nodesChecksum = nodesChecksum; + return this; + } + + public Builder blocksChecksum(long blocksChecksum) { + this.blocksChecksum = blocksChecksum; + return this; + } + + public Builder clusterStateCustomsChecksum(long clusterStateCustomsChecksum) { + this.clusterStateCustomsChecksum = clusterStateCustomsChecksum; + return this; + } + + public Builder coordinationMetadataChecksum(long coordinationMetadataChecksum) { + this.coordinationMetadataChecksum = coordinationMetadataChecksum; + return this; + } + + public Builder settingMetadataChecksum(long settingMetadataChecksum) { + this.settingMetadataChecksum = settingMetadataChecksum; + return this; + } + + public Builder transientSettingsMetadataChecksum(long transientSettingsMetadataChecksum) { + this.transientSettingsMetadataChecksum = transientSettingsMetadataChecksum; + return this; + } + + public Builder templatesMetadataChecksum(long templatesMetadataChecksum) { + this.templatesMetadataChecksum = templatesMetadataChecksum; + return this; + } + + public Builder customMetadataMapChecksum(long customMetadataMapChecksum) { + this.customMetadataMapChecksum = customMetadataMapChecksum; + return this; + } + + public Builder hashesOfConsistentSettingsChecksum(long hashesOfConsistentSettingsChecksum) { + this.hashesOfConsistentSettingsChecksum = hashesOfConsistentSettingsChecksum; + return this; + } + + public Builder indicesChecksum(long indicesChecksum) { + this.indicesChecksum = indicesChecksum; + return this; + } + + public Builder clusterStateChecksum(long clusterStateChecksum) { + this.clusterStateChecksum = clusterStateChecksum; + return this; + } + + public ClusterStateChecksum build() { + return new ClusterStateChecksum( + routingTableChecksum, + nodesChecksum, + blocksChecksum, + clusterStateCustomsChecksum, + coordinationMetadataChecksum, + settingMetadataChecksum, + transientSettingsMetadataChecksum, + templatesMetadataChecksum, + customMetadataMapChecksum, + hashesOfConsistentSettingsChecksum, + indicesChecksum, + clusterStateChecksum + ); + } + } + +} 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 098a36c7d42c4..88dda3953a5f9 100644 --- a/server/src/main/java/org/opensearch/gateway/remote/RemoteClusterStateService.java +++ b/server/src/main/java/org/opensearch/gateway/remote/RemoteClusterStateService.java @@ -90,6 +90,7 @@ import static java.util.Collections.emptyList; import static java.util.Collections.emptyMap; +import static org.opensearch.cluster.ClusterState.CUSTOM_VALUE_SERIALIZER; import static org.opensearch.common.util.FeatureFlags.REMOTE_PUBLICATION_EXPERIMENTAL; import static org.opensearch.gateway.PersistedClusterStateService.SLOW_WRITE_LOGGING_THRESHOLD; import static org.opensearch.gateway.remote.ClusterMetadataManifest.CODEC_V2; @@ -140,6 +141,13 @@ public class RemoteClusterStateService implements Closeable { Setting.Property.NodeScope ); + public static final Setting REMOTE_CLUSTER_STATE_CHECKSUM_VALIDATION_ENABLED_SETTING = Setting.boolSetting( + "cluster.remote_store.state.checksum_validation.enabled", + false, + Property.Dynamic, + Property.NodeScope + ); + private TimeValue remoteStateReadTimeout; private final String nodeId; private final Supplier repositoriesService; @@ -151,6 +159,7 @@ public class RemoteClusterStateService implements Closeable { private BlobStoreTransferService blobStoreTransferService; private RemoteRoutingTableService remoteRoutingTableService; private volatile TimeValue slowWriteLoggingThreshold; + private boolean checksumValidationEnabled; private final RemotePersistenceStats remoteStateStats; private RemoteClusterStateCleanupManager remoteClusterStateCleanupManager; @@ -197,6 +206,12 @@ public RemoteClusterStateService( clusterSettings.addSettingsUpdateConsumer(SLOW_WRITE_LOGGING_THRESHOLD, this::setSlowWriteLoggingThreshold); this.remoteStateReadTimeout = clusterSettings.get(REMOTE_STATE_READ_TIMEOUT_SETTING); clusterSettings.addSettingsUpdateConsumer(REMOTE_STATE_READ_TIMEOUT_SETTING, this::setRemoteStateReadTimeout); + this.checksumValidationEnabled = clusterSettings.get(REMOTE_CLUSTER_STATE_CHECKSUM_VALIDATION_ENABLED_SETTING); + clusterSettings.addSettingsUpdateConsumer( + REMOTE_CLUSTER_STATE_CHECKSUM_VALIDATION_ENABLED_SETTING, + this::setChecksumValidationEnabled + ); + this.remoteStateStats = new RemotePersistenceStats(); this.namedWriteableRegistry = namedWriteableRegistry; this.indexMetadataUploadListeners = indexMetadataUploadListeners; @@ -257,6 +272,7 @@ public RemoteClusterStateManifestInfo writeFullMetadata(ClusterState clusterStat uploadedMetadataResults, previousClusterUUID, clusterStateDiffManifest, + checksumValidationEnabled ? new ClusterStateChecksum(clusterState) : null, false, codecVersion ); @@ -456,6 +472,7 @@ public RemoteClusterStateManifestInfo writeIncrementalMetadata( uploadedMetadataResults, previousManifest.getPreviousClusterUUID(), clusterStateDiffManifest, + checksumValidationEnabled ? new ClusterStateChecksum(clusterState) : null, false, previousManifest.getCodecVersion() ); @@ -900,6 +917,7 @@ public RemoteClusterStateManifestInfo markLastStateAsCommitted(ClusterState clus uploadedMetadataResults, previousManifest.getPreviousClusterUUID(), previousManifest.getDiffManifest(), + checksumValidationEnabled ? previousManifest.getClusterStateChecksum() : null, true, previousManifest.getCodecVersion() ); @@ -985,6 +1003,10 @@ private void setSlowWriteLoggingThreshold(TimeValue slowWriteLoggingThreshold) { this.slowWriteLoggingThreshold = slowWriteLoggingThreshold; } + private void setChecksumValidationEnabled(Boolean checksumValidationEnabled) { + this.checksumValidationEnabled = checksumValidationEnabled; + } + // Package private for unit test RemoteRoutingTableService getRemoteRoutingTableService() { return this.remoteRoutingTableService; @@ -1353,6 +1375,10 @@ public ClusterState getClusterStateForManifest( false, includeEphemeral ); + + if (includeEphemeral && checksumValidationEnabled && manifest.getClusterStateChecksum() != null) { + validateClusterStateFromChecksum(manifest, clusterState, clusterName, localNodeId, true); + } } else { ClusterState state = readClusterStateInParallel( ClusterState.builder(new ClusterName(clusterName)).build(), @@ -1472,6 +1498,9 @@ public ClusterState getClusterStateUsingDiff(ClusterMetadataManifest manifest, C .routingTable(new RoutingTable(manifest.getRoutingTableVersion(), indexRoutingTables)) .build(); + if (checksumValidationEnabled && manifest.getClusterStateChecksum() != null) { + validateClusterStateFromChecksum(manifest, clusterState, previousState.getClusterName().value(), localNodeId, false); + } final long durationMillis = TimeValue.nsecToMSec(relativeTimeNanosSupplier.getAsLong() - startTimeNanos); remoteStateStats.stateDiffDownloadSucceeded(); remoteStateStats.stateDiffDownloadTook(durationMillis); @@ -1479,6 +1508,167 @@ public ClusterState getClusterStateUsingDiff(ClusterMetadataManifest manifest, C return clusterState; } + void validateClusterStateFromChecksum( + ClusterMetadataManifest manifest, + ClusterState clusterState, + String clusterName, + String localNodeId, + boolean isFullStateDownload + ) { + ClusterStateChecksum newClusterStateChecksum = new ClusterStateChecksum(clusterState); + List failedValidation = newClusterStateChecksum.getMismatchEntities(manifest.getClusterStateChecksum()); + if (!failedValidation.isEmpty()) { + logger.error( + () -> new ParameterizedMessage( + "Cluster state checksums do not match. Checksum from manifest {}, checksum from created cluster state {}. Entities failing validation {}", + manifest.getClusterStateChecksum(), + newClusterStateChecksum, + failedValidation + ) + ); + if (isFullStateDownload) { + throw new IllegalStateException( + "Cluster state checksums do not match during full state read. Validation failed for " + failedValidation + ); + } + // download full cluster state and match against state created for the failing entities + ClusterState fullClusterState = 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, + manifest.getDiscoveryNodesMetadata() != null, + manifest.getClusterBlocksMetadata() != null, + manifest.getIndicesRouting(), + manifest.getHashesOfConsistentSettings() != null, + manifest.getClusterStateCustomMap(), + false, + true + ); + for (String failedEntity : failedValidation) { + switch (failedEntity) { + case ClusterStateChecksum.ROUTING_TABLE_CS: + Diff routingTableDiff = fullClusterState.routingTable().diff(clusterState.routingTable()); + logger.error(() -> new ParameterizedMessage("Failing Diff in routing table {}", routingTableDiff)); + break; + case ClusterStateChecksum.NODES_CS: + logger.error( + () -> new ParameterizedMessage( + "Failing Diff in discovery nodes {}", + fullClusterState.nodes().diff(clusterState.nodes()) + ) + ); + break; + case ClusterStateChecksum.BLOCKS_CS: + logger.error( + () -> new ParameterizedMessage( + "Failing Diff in cluster blocks {}", + fullClusterState.blocks().diff(clusterState.blocks()) + ) + ); + break; + case ClusterStateChecksum.CUSTOMS_CS: + logger.error( + () -> new ParameterizedMessage( + "Failing Diff in cluster state customs {}", + DiffableUtils.diff( + clusterState.customs(), + fullClusterState.customs(), + DiffableUtils.getStringKeySerializer(), + CUSTOM_VALUE_SERIALIZER + ) + ) + ); + break; + case ClusterStateChecksum.COORDINATION_MD_CS: + logger.error( + () -> new ParameterizedMessage( + "Failing Diff in coordination md. current md {}, full state md {}", + clusterState.metadata().coordinationMetadata(), + fullClusterState.metadata().coordinationMetadata() + ) + ); + break; + case ClusterStateChecksum.TRANSIENT_SETTINGS_MD_CS: + logger.error( + () -> new ParameterizedMessage( + "Failing Diff in transient settings md. current md {}, full state md {}", + clusterState.metadata().transientSettings(), + fullClusterState.metadata().transientSettings() + ) + ); + + break; + case ClusterStateChecksum.SETTINGS_MD_CS: + logger.error( + () -> new ParameterizedMessage( + "Failing Diff in settings md. current md {}, full state md {}", + clusterState.metadata().settings(), + fullClusterState.metadata().settings() + ) + ); + + break; + case ClusterStateChecksum.HASHES_MD_CS: + logger.error( + () -> new ParameterizedMessage( + "Failing Diff in hashes md {}", + ((DiffableStringMap) fullClusterState.metadata().hashesOfConsistentSettings()).diff( + (DiffableStringMap) clusterState.metadata().hashesOfConsistentSettings() + ) + ) + ); + break; + case ClusterStateChecksum.TEMPLATES_MD_CS: + logger.error( + () -> new ParameterizedMessage( + "Failing Diff in templates md{}", + fullClusterState.metadata().templatesMetadata().diff(clusterState.metadata().templatesMetadata()) + ) + ); + break; + case ClusterStateChecksum.CUSTOM_MD_CS: + logger.error( + () -> new ParameterizedMessage( + "Failing Diff in customs md {}", + DiffableUtils.diff( + clusterState.metadata().customs(), + fullClusterState.metadata().customs(), + DiffableUtils.getStringKeySerializer(), + Metadata.CUSTOM_VALUE_SERIALIZER + ) + ) + ); + break; + case ClusterStateChecksum.INDICES_CS: + logger.error( + () -> new ParameterizedMessage( + "Failing Diff in index md {}", + DiffableUtils.diff( + clusterState.metadata().indices(), + fullClusterState.metadata().indices(), + DiffableUtils.getStringKeySerializer() + ) + ) + ); + break; + default: + logger.error(() -> new ParameterizedMessage("Unknown failed entity {}", failedEntity)); + break; + } + } + throw new IllegalStateException( + "Cluster state checksums do not match during diff read. Validation failed for " + failedValidation + ); + } + } + /** * Fetch the previous cluster UUIDs from remote state store and return the most recent valid cluster UUID * diff --git a/server/src/main/java/org/opensearch/gateway/remote/RemoteManifestManager.java b/server/src/main/java/org/opensearch/gateway/remote/RemoteManifestManager.java index 67b1b77447bed..47c847b5dc32a 100644 --- a/server/src/main/java/org/opensearch/gateway/remote/RemoteManifestManager.java +++ b/server/src/main/java/org/opensearch/gateway/remote/RemoteManifestManager.java @@ -10,6 +10,7 @@ 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.ClusterState; @@ -98,6 +99,7 @@ RemoteClusterStateManifestInfo uploadManifest( RemoteClusterStateUtils.UploadedMetadataResults uploadedMetadataResult, String previousClusterUUID, ClusterStateDiffManifest clusterDiffManifest, + ClusterStateChecksum clusterStateChecksum, boolean committed, int codecVersion ) { @@ -126,8 +128,10 @@ RemoteClusterStateManifestInfo uploadManifest( .metadataVersion(clusterState.metadata().version()) .transientSettingsMetadata(uploadedMetadataResult.uploadedTransientSettingsMetadata) .clusterStateCustomMetadataMap(uploadedMetadataResult.uploadedClusterStateCustomMetadataMap) - .hashesOfConsistentSettings(uploadedMetadataResult.uploadedHashesOfConsistentSettings); + .hashesOfConsistentSettings(uploadedMetadataResult.uploadedHashesOfConsistentSettings) + .checksum(clusterStateChecksum); final ClusterMetadataManifest manifest = manifestBuilder.build(); + logger.trace(() -> new ParameterizedMessage("[{}] uploading manifest", manifest)); String manifestFileName = writeMetadataManifest(clusterState.metadata().clusterUUID(), manifest); return new RemoteClusterStateManifestInfo(manifest, manifestFileName); } diff --git a/server/src/main/java/org/opensearch/gateway/remote/model/RemoteClusterMetadataManifest.java b/server/src/main/java/org/opensearch/gateway/remote/model/RemoteClusterMetadataManifest.java index 9460e240e1706..999beaa4e865d 100644 --- a/server/src/main/java/org/opensearch/gateway/remote/model/RemoteClusterMetadataManifest.java +++ b/server/src/main/java/org/opensearch/gateway/remote/model/RemoteClusterMetadataManifest.java @@ -35,6 +35,7 @@ public class RemoteClusterMetadataManifest extends AbstractClusterMetadataWritea public static final int SPLITTED_MANIFEST_FILE_LENGTH = 6; public static final String METADATA_MANIFEST_NAME_FORMAT = "%s"; + public static final String COMMITTED = "C"; public static final String PUBLISHED = "P"; 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 index 446207a767009..829036c6d122b 100644 --- a/server/src/main/java/org/opensearch/gateway/remote/model/RemoteDiscoveryNodes.java +++ b/server/src/main/java/org/opensearch/gateway/remote/model/RemoteDiscoveryNodes.java @@ -88,7 +88,12 @@ public UploadedMetadata getUploadedMetadata() { @Override public InputStream serialize() throws IOException { - return DISCOVERY_NODES_FORMAT.serialize(discoveryNodes, generateBlobFileName(), getCompressor()).streamInput(); + return DISCOVERY_NODES_FORMAT.serialize( + (out, discoveryNode) -> discoveryNode.writeToWithAttribute(out), + discoveryNodes, + generateBlobFileName(), + getCompressor() + ).streamInput(); } @Override diff --git a/server/src/main/java/org/opensearch/index/IndexSettings.java b/server/src/main/java/org/opensearch/index/IndexSettings.java index 12715b353c526..3e58b9df63378 100644 --- a/server/src/main/java/org/opensearch/index/IndexSettings.java +++ b/server/src/main/java/org/opensearch/index/IndexSettings.java @@ -48,6 +48,7 @@ import org.opensearch.core.common.unit.ByteSizeUnit; import org.opensearch.core.common.unit.ByteSizeValue; import org.opensearch.core.index.Index; +import org.opensearch.index.compositeindex.datacube.startree.StarTreeIndexSettings; import org.opensearch.index.remote.RemoteStoreEnums.PathType; import org.opensearch.index.remote.RemoteStorePathStrategy; import org.opensearch.index.remote.RemoteStoreUtils; @@ -889,6 +890,8 @@ private void setRetentionLeaseMillis(final TimeValue retentionLease) { */ private volatile double docIdFuzzySetFalsePositiveProbability; + private final boolean isCompositeIndex; + /** * Returns the default search fields for this index. */ @@ -1052,7 +1055,7 @@ public IndexSettings(final IndexMetadata indexMetadata, final Settings nodeSetti setEnableFuzzySetForDocId(scopedSettings.get(INDEX_DOC_ID_FUZZY_SET_ENABLED_SETTING)); setDocIdFuzzySetFalsePositiveProbability(scopedSettings.get(INDEX_DOC_ID_FUZZY_SET_FALSE_POSITIVE_PROBABILITY_SETTING)); - + isCompositeIndex = scopedSettings.get(StarTreeIndexSettings.IS_COMPOSITE_INDEX_SETTING); scopedSettings.addSettingsUpdateConsumer( TieredMergePolicyProvider.INDEX_COMPOUND_FORMAT_SETTING, tieredMergePolicyProvider::setNoCFSRatio @@ -1297,6 +1300,10 @@ public int getNumberOfReplicas() { return settings.getAsInt(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, null); } + public boolean isCompositeIndex() { + return isCompositeIndex; + } + /** * Returns true if segment replication is enabled on the index. * diff --git a/server/src/main/java/org/opensearch/index/compositeindex/CompositeIndexSettings.java b/server/src/main/java/org/opensearch/index/compositeindex/CompositeIndexSettings.java index 014dd22426a10..a29e642d30f05 100644 --- a/server/src/main/java/org/opensearch/index/compositeindex/CompositeIndexSettings.java +++ b/server/src/main/java/org/opensearch/index/compositeindex/CompositeIndexSettings.java @@ -13,6 +13,8 @@ import org.opensearch.common.settings.Setting; import org.opensearch.common.settings.Settings; import org.opensearch.common.util.FeatureFlags; +import org.opensearch.core.common.unit.ByteSizeUnit; +import org.opensearch.core.common.unit.ByteSizeValue; /** * Cluster level settings for composite indices @@ -37,12 +39,23 @@ public class CompositeIndexSettings { Setting.Property.Dynamic ); + /** + * This sets the max flush threshold size for composite index + */ + public static final Setting COMPOSITE_INDEX_MAX_TRANSLOG_FLUSH_THRESHOLD_SIZE_SETTING = Setting.byteSizeSetting( + "indices.composite_index.translog.max_flush_threshold_size", + new ByteSizeValue(512, ByteSizeUnit.MB), + new ByteSizeValue(128, ByteSizeUnit.MB), + new ByteSizeValue(Long.MAX_VALUE, ByteSizeUnit.BYTES), + Setting.Property.NodeScope, + Setting.Property.Dynamic + ); + private volatile boolean starTreeIndexCreationEnabled; public CompositeIndexSettings(Settings settings, ClusterSettings clusterSettings) { this.starTreeIndexCreationEnabled = STAR_TREE_INDEX_ENABLED_SETTING.get(settings); clusterSettings.addSettingsUpdateConsumer(STAR_TREE_INDEX_ENABLED_SETTING, this::starTreeIndexCreationEnabled); - } private void starTreeIndexCreationEnabled(boolean value) { diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/StarTreeIndexSettings.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/StarTreeIndexSettings.java index ce389a99b3626..e665831b83d93 100644 --- a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/StarTreeIndexSettings.java +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/StarTreeIndexSettings.java @@ -26,6 +26,7 @@ public class StarTreeIndexSettings { public static int STAR_TREE_MAX_DIMENSIONS_DEFAULT = 10; + public static int STAR_TREE_MAX_BASE_METRICS_DEFAULT = 100; /** * This setting determines the max number of star tree fields that can be part of composite index mapping. For each * star tree field, we will generate associated star tree index. @@ -52,6 +53,19 @@ public class StarTreeIndexSettings { Setting.Property.Final ); + /** + * This setting determines the max number of dimensions that can be part of star tree index field. Number of + * dimensions and associated cardinality has direct effect of star tree index size and query performance. + */ + public static final Setting STAR_TREE_MAX_BASE_METRICS_SETTING = Setting.intSetting( + "index.composite_index.star_tree.field.max_base_metrics", + STAR_TREE_MAX_BASE_METRICS_DEFAULT, + 4, + 100, + Setting.Property.IndexScope, + Setting.Property.Final + ); + /** * This setting determines the max number of date intervals that can be part of star tree date field. */ @@ -108,4 +122,11 @@ public static Rounding.DateTimeUnit getTimeUnit(String expression) { } return DateHistogramAggregationBuilder.DATE_FIELD_UNITS.get(expression); } + + public static final Setting IS_COMPOSITE_INDEX_SETTING = Setting.boolSetting( + "index.composite_index", + false, + Setting.Property.IndexScope, + Setting.Property.Final + ); } diff --git a/server/src/main/java/org/opensearch/index/mapper/DocumentParser.java b/server/src/main/java/org/opensearch/index/mapper/DocumentParser.java index b03026d560dbf..50ff816695156 100644 --- a/server/src/main/java/org/opensearch/index/mapper/DocumentParser.java +++ b/server/src/main/java/org/opensearch/index/mapper/DocumentParser.java @@ -661,6 +661,17 @@ private static void parseNonDynamicArray(ParseContext context, ObjectMapper mapp throws IOException { XContentParser parser = context.parser(); XContentParser.Token token; + // block array values for composite index fields + if (context.indexSettings().isCompositeIndex() && context.mapperService().isFieldPartOfCompositeIndex(arrayFieldName)) { + throw new MapperParsingException( + String.format( + Locale.ROOT, + "object mapping for [%s] with array for [%s] cannot be accepted as field is also part of composite index mapping which does not accept arrays", + mapper.name(), + arrayFieldName + ) + ); + } final String[] paths = splitAndValidatePath(lastFieldName); while ((token = parser.nextToken()) != XContentParser.Token.END_ARRAY) { if (token == XContentParser.Token.START_OBJECT) { diff --git a/server/src/main/java/org/opensearch/index/mapper/MapperService.java b/server/src/main/java/org/opensearch/index/mapper/MapperService.java index 400232ff164db..f75aa0d367ff7 100644 --- a/server/src/main/java/org/opensearch/index/mapper/MapperService.java +++ b/server/src/main/java/org/opensearch/index/mapper/MapperService.java @@ -227,6 +227,7 @@ public enum MergeReason { private final BooleanSupplier idFieldDataEnabled; private volatile Set compositeMappedFieldTypes; + private volatile Set fieldsPartOfCompositeMappings; public MapperService( IndexSettings indexSettings, @@ -547,9 +548,18 @@ private synchronized Map internalMerge(DocumentMapper ma // initialize composite fields post merge this.compositeMappedFieldTypes = getCompositeFieldTypesFromMapper(); + buildCompositeFieldLookup(); return results; } + private void buildCompositeFieldLookup() { + Set fieldsPartOfCompositeMappings = new HashSet<>(); + for (CompositeMappedFieldType fieldType : compositeMappedFieldTypes) { + fieldsPartOfCompositeMappings.addAll(fieldType.fields()); + } + this.fieldsPartOfCompositeMappings = fieldsPartOfCompositeMappings; + } + private boolean assertSerialization(DocumentMapper mapper) { // capture the source now, it may change due to concurrent parsing final CompressedXContent mappingSource = mapper.mappingSource(); @@ -677,6 +687,10 @@ private Set getCompositeFieldTypesFromMapper() { return compositeMappedFieldTypes; } + public boolean isFieldPartOfCompositeIndex(String field) { + return fieldsPartOfCompositeMappings.contains(field); + } + public ObjectMapper getObjectMapper(String name) { return this.mapper == null ? null : this.mapper.objectMappers().get(name); } diff --git a/server/src/main/java/org/opensearch/index/mapper/ObjectMapper.java b/server/src/main/java/org/opensearch/index/mapper/ObjectMapper.java index 533e6ca73d737..dd984373fc9df 100644 --- a/server/src/main/java/org/opensearch/index/mapper/ObjectMapper.java +++ b/server/src/main/java/org/opensearch/index/mapper/ObjectMapper.java @@ -440,6 +440,15 @@ protected static void parseCompositeField( + " feature flag in the JVM options" ); } + if (StarTreeIndexSettings.IS_COMPOSITE_INDEX_SETTING.get(parserContext.getSettings()) == false) { + throw new IllegalArgumentException( + String.format( + Locale.ROOT, + "Set '%s' as true as part of index settings to use star tree index", + StarTreeIndexSettings.IS_COMPOSITE_INDEX_SETTING.getKey() + ) + ); + } Iterator> iterator = compositeNode.entrySet().iterator(); if (compositeNode.size() > StarTreeIndexSettings.STAR_TREE_MAX_FIELDS_SETTING.get(parserContext.getSettings())) { throw new IllegalArgumentException( diff --git a/server/src/main/java/org/opensearch/index/mapper/StarTreeMapper.java b/server/src/main/java/org/opensearch/index/mapper/StarTreeMapper.java index e52d6a621e4e8..17c27ef149e54 100644 --- a/server/src/main/java/org/opensearch/index/mapper/StarTreeMapper.java +++ b/server/src/main/java/org/opensearch/index/mapper/StarTreeMapper.java @@ -22,6 +22,7 @@ import org.opensearch.search.lookup.SearchLookup; import java.util.ArrayList; +import java.util.HashSet; import java.util.LinkedHashSet; import java.util.LinkedList; import java.util.List; @@ -155,8 +156,20 @@ private List buildDimensions(String fieldName, Map ma String.format(Locale.ROOT, "Atleast two dimensions are required to build star tree index field [%s]", fieldName) ); } + Set dimensionFieldNames = new HashSet<>(); for (Object dim : dimList) { - dimensions.add(getDimension(fieldName, dim, context)); + Dimension dimension = getDimension(fieldName, dim, context); + if (dimensionFieldNames.add(dimension.getField()) == false) { + throw new IllegalArgumentException( + String.format( + Locale.ROOT, + "Duplicate dimension [%s] present as part star tree index field [%s]", + dimension.getField(), + fieldName + ) + ); + } + dimensions.add(dimension); } } else { throw new MapperParsingException( @@ -223,6 +236,7 @@ private List buildMetrics(String fieldName, Map map, Map } if (metricsFromInput instanceof List) { List metricsList = (List) metricsFromInput; + Set metricFieldNames = new HashSet<>(); for (Object metric : metricsList) { Map metricMap = (Map) metric; String name = (String) XContentMapValues.extractValue(CompositeDataCubeFieldType.NAME, metricMap); @@ -232,7 +246,18 @@ private List buildMetrics(String fieldName, Map map, Map } metricMap.remove(CompositeDataCubeFieldType.NAME); if (objbuilder == null || objbuilder.mappersBuilders == null) { - metrics.add(getMetric(name, metricMap, context)); + Metric metricFromParser = getMetric(name, metricMap, context); + if (metricFieldNames.add(metricFromParser.getField()) == false) { + throw new IllegalArgumentException( + String.format( + Locale.ROOT, + "Duplicate metrics [%s] present as part star tree index field [%s]", + metricFromParser.getField(), + fieldName + ) + ); + } + metrics.add(metricFromParser); } else { Optional meticBuilder = findMapperBuilderByName(name, this.objbuilder.mappersBuilders); if (meticBuilder.isEmpty()) { @@ -243,7 +268,18 @@ private List buildMetrics(String fieldName, Map map, Map String.format(Locale.ROOT, "non-numeric field type is associated with star tree metric [%s]", this.name) ); } - metrics.add(getMetric(name, metricMap, context)); + Metric metricFromParser = getMetric(name, metricMap, context); + if (metricFieldNames.add(metricFromParser.getField()) == false) { + throw new IllegalArgumentException( + String.format( + Locale.ROOT, + "Duplicate metrics [%s] present as part star tree index field [%s]", + metricFromParser.getField(), + fieldName + ) + ); + } + metrics.add(metricFromParser); DocumentMapperParser.checkNoRemainingFields( metricMap, context.indexVersionCreated(), @@ -254,6 +290,32 @@ private List buildMetrics(String fieldName, Map map, Map } else { throw new MapperParsingException(String.format(Locale.ROOT, "unable to parse metrics for star tree field [%s]", this.name)); } + int numBaseMetrics = 0; + for (Metric metric : metrics) { + for (MetricStat metricStat : metric.getMetrics()) { + if (metricStat.isDerivedMetric() == false) { + numBaseMetrics++; + } + } + } + if (numBaseMetrics > context.getSettings() + .getAsInt( + StarTreeIndexSettings.STAR_TREE_MAX_BASE_METRICS_SETTING.getKey(), + StarTreeIndexSettings.STAR_TREE_MAX_BASE_METRICS_DEFAULT + )) { + throw new IllegalArgumentException( + String.format( + Locale.ROOT, + "There cannot be more than [%s] base metrics for star tree field [%s]", + context.getSettings() + .getAsInt( + StarTreeIndexSettings.STAR_TREE_MAX_BASE_METRICS_SETTING.getKey(), + StarTreeIndexSettings.STAR_TREE_MAX_BASE_METRICS_DEFAULT + ), + fieldName + ) + ); + } Metric docCountMetric = new Metric(DocCountFieldMapper.NAME, List.of(MetricStat.DOC_COUNT)); metrics.add(docCountMetric); return metrics; diff --git a/server/src/main/java/org/opensearch/index/remote/RemoteIndexPath.java b/server/src/main/java/org/opensearch/index/remote/RemoteIndexPath.java index 899ff16c9d607..e686bb60140c0 100644 --- a/server/src/main/java/org/opensearch/index/remote/RemoteIndexPath.java +++ b/server/src/main/java/org/opensearch/index/remote/RemoteIndexPath.java @@ -20,6 +20,7 @@ import org.opensearch.index.remote.RemoteStoreEnums.PathType; import org.opensearch.index.remote.RemoteStorePathStrategy.PathInput; import org.opensearch.index.remote.RemoteStorePathStrategy.ShardDataPathInput; +import org.opensearch.indices.RemoteStoreSettings; import java.io.IOException; import java.util.Collections; @@ -68,6 +69,7 @@ public class RemoteIndexPath implements ToXContentFragment { private final Iterable basePath; private final PathType pathType; private final PathHashAlgorithm pathHashAlgorithm; + private final RemoteStoreSettings remoteStoreSettings; /** * This keeps the map of paths that would be present in the content of the index path file. For eg - It is possible @@ -82,7 +84,8 @@ public RemoteIndexPath( Iterable basePath, PathType pathType, PathHashAlgorithm pathHashAlgorithm, - Map> pathCreationMap + Map> pathCreationMap, + RemoteStoreSettings remoteStoreSettings ) { if (Objects.isNull(pathCreationMap) || Objects.isNull(pathType) @@ -119,6 +122,7 @@ public RemoteIndexPath( this.pathType = pathType; this.pathHashAlgorithm = pathHashAlgorithm; this.pathCreationMap = pathCreationMap; + this.remoteStoreSettings = remoteStoreSettings; } @Override @@ -148,6 +152,11 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws .shardId(Integer.toString(shardNo)) .dataCategory(dataCategory) .dataType(type) + .fixedPrefix( + dataCategory == TRANSLOG + ? remoteStoreSettings.getTranslogPathFixedPrefix() + : remoteStoreSettings.getSegmentsPathFixedPrefix() + ) .build(); builder.value(pathType.path(pathInput, pathHashAlgorithm).buildAsString()); } 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 d5617bdfd94a7..5878dff03acc2 100644 --- a/server/src/main/java/org/opensearch/index/remote/RemoteIndexPathUploader.java +++ b/server/src/main/java/org/opensearch/index/remote/RemoteIndexPathUploader.java @@ -25,6 +25,7 @@ import org.opensearch.gateway.remote.IndexMetadataUploadListener; import org.opensearch.gateway.remote.RemoteStateTransferException; import org.opensearch.index.remote.RemoteStoreEnums.PathType; +import org.opensearch.indices.RemoteStoreSettings; import org.opensearch.node.Node; import org.opensearch.node.remotestore.RemoteStoreNodeAttribute; import org.opensearch.repositories.RepositoriesService; @@ -79,6 +80,7 @@ public class RemoteIndexPathUploader extends IndexMetadataUploadListener { private final Settings settings; private final boolean isRemoteDataAttributePresent; private final boolean isTranslogSegmentRepoSame; + private final RemoteStoreSettings remoteStoreSettings; private final Supplier repositoriesService; private volatile TimeValue metadataUploadTimeout; @@ -89,7 +91,8 @@ public RemoteIndexPathUploader( ThreadPool threadPool, Settings settings, Supplier repositoriesService, - ClusterSettings clusterSettings + ClusterSettings clusterSettings, + RemoteStoreSettings remoteStoreSettings ) { super(threadPool, ThreadPool.Names.GENERIC); this.settings = Objects.requireNonNull(settings); @@ -100,6 +103,7 @@ public RemoteIndexPathUploader( Objects.requireNonNull(clusterSettings); metadataUploadTimeout = clusterSettings.get(GLOBAL_METADATA_UPLOAD_TIMEOUT_SETTING); clusterSettings.addSettingsUpdateConsumer(GLOBAL_METADATA_UPLOAD_TIMEOUT_SETTING, this::setMetadataUploadTimeout); + this.remoteStoreSettings = remoteStoreSettings; } @Override @@ -208,7 +212,8 @@ private void writePathToRemoteStore( basePath, pathType, hashAlgorithm, - pathCreationMap + pathCreationMap, + remoteStoreSettings ); String fileName = generateFileName(indexUUID, idxMD.getVersion(), remoteIndexPath.getVersion()); REMOTE_INDEX_PATH_FORMAT.writeAsyncWithUrgentPriority(remoteIndexPath, blobContainer, fileName, actionListener); 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 0af0d07e11597..c00e1a6a96abd 100644 --- a/server/src/main/java/org/opensearch/index/remote/RemoteStoreEnums.java +++ b/server/src/main/java/org/opensearch/index/remote/RemoteStoreEnums.java @@ -13,6 +13,7 @@ import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.blobstore.BlobPath; import org.opensearch.common.hash.FNV1a; +import org.opensearch.core.common.Strings; import org.opensearch.index.remote.RemoteStorePathStrategy.PathInput; import java.util.HashMap; @@ -107,7 +108,11 @@ boolean requiresHashAlgorithm() { @Override public BlobPath generatePath(PathInput pathInput, PathHashAlgorithm hashAlgorithm) { assert Objects.nonNull(hashAlgorithm) : "hashAlgorithm is expected to be non-null"; - return BlobPath.cleanPath().add(hashAlgorithm.hash(pathInput)).add(pathInput.basePath()).add(pathInput.fixedSubPath()); + String fixedPrefix = pathInput.fixedPrefix(); + return BlobPath.cleanPath() + .add(Strings.isNullOrEmpty(fixedPrefix) ? hashAlgorithm.hash(pathInput) : fixedPrefix + hashAlgorithm.hash(pathInput)) + .add(pathInput.basePath()) + .add(pathInput.fixedSubPath()); } @Override @@ -119,7 +124,10 @@ boolean requiresHashAlgorithm() { @Override public BlobPath generatePath(PathInput pathInput, PathHashAlgorithm hashAlgorithm) { assert Objects.nonNull(hashAlgorithm) : "hashAlgorithm is expected to be non-null"; - return pathInput.basePath().add(hashAlgorithm.hash(pathInput)).add(pathInput.fixedSubPath()); + String fixedPrefix = pathInput.fixedPrefix(); + return pathInput.basePath() + .add(Strings.isNullOrEmpty(fixedPrefix) ? hashAlgorithm.hash(pathInput) : fixedPrefix + hashAlgorithm.hash(pathInput)) + .add(pathInput.fixedSubPath()); } @Override 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 7bead3e75f12e..ee28f47664614 100644 --- a/server/src/main/java/org/opensearch/index/remote/RemoteStorePathStrategy.java +++ b/server/src/main/java/org/opensearch/index/remote/RemoteStorePathStrategy.java @@ -83,10 +83,12 @@ public BlobPath generatePath(PathInput pathInput) { public static class PathInput { private final BlobPath basePath; private final String indexUUID; + private final String fixedPrefix; public PathInput(Builder builder) { this.basePath = Objects.requireNonNull(builder.basePath); this.indexUUID = Objects.requireNonNull(builder.indexUUID); + this.fixedPrefix = Objects.isNull(builder.fixedPrefix) ? "" : builder.fixedPrefix; } BlobPath basePath() { @@ -97,6 +99,10 @@ String indexUUID() { return indexUUID; } + String fixedPrefix() { + return fixedPrefix; + } + BlobPath fixedSubPath() { return BlobPath.cleanPath().add(indexUUID); } @@ -126,6 +132,7 @@ public void assertIsValid() { public static class Builder> { private BlobPath basePath; private String indexUUID; + private String fixedPrefix; public T basePath(BlobPath basePath) { this.basePath = basePath; @@ -137,6 +144,11 @@ public T indexUUID(String indexUUID) { return self(); } + public T fixedPrefix(String fixedPrefix) { + this.fixedPrefix = fixedPrefix; + return self(); + } + protected T self() { return (T) this; } diff --git a/server/src/main/java/org/opensearch/index/shard/IndexShard.java b/server/src/main/java/org/opensearch/index/shard/IndexShard.java index 4b18df7a7829d..6e12e4ed3da1a 100644 --- a/server/src/main/java/org/opensearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/opensearch/index/shard/IndexShard.java @@ -150,6 +150,7 @@ import org.opensearch.index.recovery.RecoveryStats; import org.opensearch.index.refresh.RefreshStats; import org.opensearch.index.remote.RemoteSegmentStats; +import org.opensearch.index.remote.RemoteStorePathStrategy; import org.opensearch.index.remote.RemoteStoreStatsTrackerFactory; import org.opensearch.index.search.stats.SearchStats; import org.opensearch.index.search.stats.ShardSearchStats; @@ -2479,6 +2480,10 @@ private void loadGlobalCheckpointToReplicationTracker() throws IOException { * Operations from the translog will be replayed to bring lucene up to date. **/ public void openEngineAndRecoverFromTranslog() throws IOException { + openEngineAndRecoverFromTranslog(true); + } + + public void openEngineAndRecoverFromTranslog(boolean syncFromRemote) throws IOException { recoveryState.validateCurrentStage(RecoveryState.Stage.INDEX); maybeCheckIndex(); recoveryState.setStage(RecoveryState.Stage.TRANSLOG); @@ -2499,7 +2504,16 @@ public void openEngineAndRecoverFromTranslog() throws IOException { loadGlobalCheckpointToReplicationTracker(); } - innerOpenEngineAndTranslog(replicationTracker); + if (isSnapshotV2Restore()) { + translogConfig.setDownloadRemoteTranslogOnInit(false); + } + + innerOpenEngineAndTranslog(replicationTracker, syncFromRemote); + + if (isSnapshotV2Restore()) { + translogConfig.setDownloadRemoteTranslogOnInit(true); + } + getEngine().translogManager() .recoverFromTranslog(translogRecoveryRunner, getEngine().getProcessedLocalCheckpoint(), Long.MAX_VALUE); } @@ -2561,7 +2575,7 @@ private void innerOpenEngineAndTranslog(LongSupplier globalCheckpointSupplier, b if (shardRouting.primary()) { if (syncFromRemote) { syncRemoteTranslogAndUpdateGlobalCheckpoint(); - } else { + } else if (isSnapshotV2Restore() == false) { // we will enter this block when we do not want to recover from remote translog. // currently only during snapshot restore, we are coming into this block. // here, as while initiliazing remote translog we cannot skip downloading translog files, @@ -2607,6 +2621,11 @@ private void innerOpenEngineAndTranslog(LongSupplier globalCheckpointSupplier, b recoveryState.validateCurrentStage(RecoveryState.Stage.TRANSLOG); } + private boolean isSnapshotV2Restore() { + return routingEntry().recoverySource().getType() == RecoverySource.Type.SNAPSHOT + && ((SnapshotRecoverySource) routingEntry().recoverySource()).pinnedTimestamp() > 0; + } + private boolean assertSequenceNumbersInCommit() throws IOException { final Map userData = fetchUserData(); assert userData.containsKey(SequenceNumbers.LOCAL_CHECKPOINT_KEY) : "commit point doesn't contains a local checkpoint"; @@ -2892,7 +2911,26 @@ public void restoreFromSnapshotAndRemoteStore( assert recoveryState.getRecoverySource().getType() == RecoverySource.Type.SNAPSHOT : "invalid recovery type: " + recoveryState.getRecoverySource(); StoreRecovery storeRecovery = new StoreRecovery(shardId, logger); - storeRecovery.recoverFromSnapshotAndRemoteStore(this, repository, repositoriesService, listener, threadPool); + SnapshotRecoverySource recoverySource = (SnapshotRecoverySource) recoveryState().getRecoverySource(); + if (recoverySource.pinnedTimestamp() != 0) { + storeRecovery.recoverShallowSnapshotV2( + this, + repository, + repositoriesService, + listener, + remoteStoreSettings.getSegmentsPathFixedPrefix(), + threadPool + ); + } else { + storeRecovery.recoverFromSnapshotAndRemoteStore( + this, + repository, + repositoriesService, + listener, + remoteStoreSettings.getSegmentsPathFixedPrefix(), + threadPool + ); + } } catch (Exception e) { listener.onFailure(e); } @@ -5000,16 +5038,33 @@ public void syncTranslogFilesFromRemoteTranslog() throws IOException { TranslogFactory translogFactory = translogFactorySupplier.apply(indexSettings, shardRouting); assert translogFactory instanceof RemoteBlobStoreInternalTranslogFactory; Repository repository = ((RemoteBlobStoreInternalTranslogFactory) translogFactory).getRepository(); + syncTranslogFilesFromGivenRemoteTranslog( + repository, + shardId, + indexSettings.getRemoteStorePathStrategy(), + indexSettings().isTranslogMetadataEnabled(), + 0 + ); + } + + public void syncTranslogFilesFromGivenRemoteTranslog( + Repository repository, + ShardId shardId, + RemoteStorePathStrategy remoteStorePathStrategy, + boolean isTranslogMetadataEnabled, + long timestamp + ) throws IOException { RemoteFsTranslog.download( repository, shardId, getThreadPool(), shardPath().resolveTranslog(), - indexSettings.getRemoteStorePathStrategy(), + remoteStorePathStrategy, remoteStoreSettings, logger, shouldSeedRemoteStore(), - indexSettings().isTranslogMetadataEnabled() + isTranslogMetadataEnabled, + timestamp ); } @@ -5098,15 +5153,13 @@ public void syncSegmentsFromRemoteSegmentStore(boolean overrideLocal, final Runn * Downloads segments from given remote segment store for a specific commit. * @param overrideLocal flag to override local segment files with those in remote store * @param sourceRemoteDirectory RemoteSegmentDirectory Instance from which we need to sync segments - * @param primaryTerm Primary Term for shard at the time of commit operation for which we are syncing segments - * @param commitGeneration commit generation at the time of commit operation for which we are syncing segments * @throws IOException if exception occurs while reading segments from remote store */ public void syncSegmentsFromGivenRemoteSegmentStore( boolean overrideLocal, RemoteSegmentStoreDirectory sourceRemoteDirectory, - long primaryTerm, - long commitGeneration + RemoteSegmentMetadata remoteSegmentMetadata, + boolean pinnedTimestamp ) throws IOException { logger.trace("Downloading segments from given remote segment store"); RemoteSegmentStoreDirectory remoteDirectory = null; @@ -5142,12 +5195,29 @@ public void syncSegmentsFromGivenRemoteSegmentStore( overrideLocal, () -> {} ); - if (segmentsNFile != null) { + if (pinnedTimestamp) { + final SegmentInfos infosSnapshot = store.buildSegmentInfos( + remoteSegmentMetadata.getSegmentInfosBytes(), + remoteSegmentMetadata.getGeneration() + ); + long processedLocalCheckpoint = Long.parseLong(infosSnapshot.getUserData().get(LOCAL_CHECKPOINT_KEY)); + // delete any other commits, we want to start the engine only from a new commit made with the downloaded infos bytes. + // Extra segments will be wiped on engine open. + for (String file : List.of(store.directory().listAll())) { + if (file.startsWith(IndexFileNames.SEGMENTS)) { + store.deleteQuiet(file); + } + } + assert Arrays.stream(store.directory().listAll()).filter(f -> f.startsWith(IndexFileNames.SEGMENTS)).findAny().isEmpty() + : "There should not be any segments file in the dir"; + store.commitSegmentInfos(infosSnapshot, processedLocalCheckpoint, processedLocalCheckpoint); + } else if (segmentsNFile != null) { try ( ChecksumIndexInput indexInput = new BufferedChecksumIndexInput( storeDirectory.openInput(segmentsNFile, IOContext.DEFAULT) ) ) { + long commitGeneration = SegmentInfos.generationFromSegmentsFileName(segmentsNFile); SegmentInfos infosSnapshot = SegmentInfos.readCommit(store.directory(), indexInput, commitGeneration); long processedLocalCheckpoint = Long.parseLong(infosSnapshot.getUserData().get(LOCAL_CHECKPOINT_KEY)); if (remoteStore != null) { diff --git a/server/src/main/java/org/opensearch/index/shard/StoreRecovery.java b/server/src/main/java/org/opensearch/index/shard/StoreRecovery.java index 8d689e8769728..6933e4e161dd1 100644 --- a/server/src/main/java/org/opensearch/index/shard/StoreRecovery.java +++ b/server/src/main/java/org/opensearch/index/shard/StoreRecovery.java @@ -58,12 +58,15 @@ import org.opensearch.index.engine.Engine; import org.opensearch.index.engine.EngineException; import org.opensearch.index.mapper.MapperService; +import org.opensearch.index.remote.RemoteStorePathStrategy; +import org.opensearch.index.remote.RemoteStoreUtils; import org.opensearch.index.seqno.SequenceNumbers; import org.opensearch.index.snapshots.IndexShardRestoreFailedException; import org.opensearch.index.snapshots.blobstore.RemoteStoreShardShallowCopySnapshot; import org.opensearch.index.store.RemoteSegmentStoreDirectory; import org.opensearch.index.store.RemoteSegmentStoreDirectoryFactory; import org.opensearch.index.store.Store; +import org.opensearch.index.store.remote.metadata.RemoteSegmentMetadata; import org.opensearch.index.translog.Checkpoint; import org.opensearch.index.translog.Translog; import org.opensearch.index.translog.TranslogHeader; @@ -72,6 +75,7 @@ import org.opensearch.repositories.IndexId; import org.opensearch.repositories.RepositoriesService; import org.opensearch.repositories.Repository; +import org.opensearch.repositories.RepositoryData; import org.opensearch.threadpool.ThreadPool; import java.io.IOException; @@ -367,6 +371,7 @@ void recoverFromSnapshotAndRemoteStore( Repository repository, RepositoriesService repositoriesService, ActionListener listener, + String segmentsPathFixedPrefix, ThreadPool threadPool ) { try { @@ -397,7 +402,8 @@ void recoverFromSnapshotAndRemoteStore( RemoteSegmentStoreDirectoryFactory directoryFactory = new RemoteSegmentStoreDirectoryFactory( () -> repositoriesService, - threadPool + threadPool, + segmentsPathFixedPrefix ); RemoteSegmentStoreDirectory sourceRemoteDirectory = (RemoteSegmentStoreDirectory) directoryFactory.newDirectory( remoteStoreRepository, @@ -405,14 +411,14 @@ void recoverFromSnapshotAndRemoteStore( shardId, shallowCopyShardMetadata.getRemoteStorePathStrategy() ); - sourceRemoteDirectory.initializeToSpecificCommit( + RemoteSegmentMetadata remoteSegmentMetadata = sourceRemoteDirectory.initializeToSpecificCommit( primaryTerm, commitGeneration, recoverySource.snapshot().getSnapshotId().getUUID() ); - indexShard.syncSegmentsFromGivenRemoteSegmentStore(true, sourceRemoteDirectory, primaryTerm, commitGeneration); + indexShard.syncSegmentsFromGivenRemoteSegmentStore(true, sourceRemoteDirectory, remoteSegmentMetadata, false); final Store store = indexShard.store(); - if (indexShard.indexSettings.isRemoteTranslogStoreEnabled() == false) { + if (indexShard.indexSettings.isRemoteStoreEnabled() == false) { bootstrap(indexShard, store); } else { bootstrapForSnapshot(indexShard, store); @@ -441,6 +447,98 @@ void recoverFromSnapshotAndRemoteStore( } } + void recoverShallowSnapshotV2( + final IndexShard indexShard, + Repository repository, + RepositoriesService repositoriesService, + ActionListener listener, + String segmentsPathFixedPrefix, + ThreadPool threadPool + ) { + try { + if (canRecover(indexShard)) { + indexShard.preRecovery(); + RecoverySource.Type recoveryType = indexShard.recoveryState().getRecoverySource().getType(); + assert recoveryType == RecoverySource.Type.SNAPSHOT : "expected snapshot recovery type: " + recoveryType; + SnapshotRecoverySource recoverySource = (SnapshotRecoverySource) indexShard.recoveryState().getRecoverySource(); + indexShard.prepareForIndexRecovery(); + + assert recoverySource.pinnedTimestamp() != 0; + final StepListener repositoryDataListener = new StepListener<>(); + repository.getRepositoryData(repositoryDataListener); + repositoryDataListener.whenComplete(repositoryData -> { + IndexId indexId = repositoryData.resolveIndexId(recoverySource.index().getName()); + IndexMetadata prevIndexMetadata = repository.getSnapshotIndexMetaData( + repositoryData, + recoverySource.snapshot().getSnapshotId(), + indexId + ); + RemoteSegmentStoreDirectoryFactory directoryFactory = new RemoteSegmentStoreDirectoryFactory( + () -> repositoriesService, + threadPool, + segmentsPathFixedPrefix + ); + String remoteSegmentStoreRepository = ((SnapshotRecoverySource) indexShard.recoveryState().getRecoverySource()) + .sourceRemoteStoreRepository(); + if (remoteSegmentStoreRepository == null) { + remoteSegmentStoreRepository = IndexMetadata.INDEX_REMOTE_SEGMENT_STORE_REPOSITORY_SETTING.get( + prevIndexMetadata.getSettings() + ); + } + RemoteStorePathStrategy remoteStorePathStrategy = RemoteStoreUtils.determineRemoteStorePathStrategy(prevIndexMetadata); + RemoteSegmentStoreDirectory sourceRemoteDirectory = (RemoteSegmentStoreDirectory) directoryFactory.newDirectory( + remoteSegmentStoreRepository, + prevIndexMetadata.getIndexUUID(), + shardId, + remoteStorePathStrategy + ); + RemoteSegmentMetadata remoteSegmentMetadata = sourceRemoteDirectory.initializeToSpecificTimestamp( + recoverySource.pinnedTimestamp() + ); + + String remoteTranslogRepository = ((SnapshotRecoverySource) indexShard.recoveryState().getRecoverySource()) + .sourceRemoteTranslogRepository(); + if (remoteTranslogRepository == null) { + remoteTranslogRepository = IndexMetadata.INDEX_REMOTE_TRANSLOG_REPOSITORY_SETTING.get( + prevIndexMetadata.getSettings() + ); + } + + indexShard.syncSegmentsFromGivenRemoteSegmentStore(true, sourceRemoteDirectory, remoteSegmentMetadata, true); + indexShard.syncTranslogFilesFromGivenRemoteTranslog( + repositoriesService.repository(remoteTranslogRepository), + new ShardId(prevIndexMetadata.getIndex(), shardId.id()), + remoteStorePathStrategy, + RemoteStoreUtils.determineTranslogMetadataEnabled(prevIndexMetadata), + recoverySource.pinnedTimestamp() + ); + + assert indexShard.shardRouting.primary() : "only primary shards can recover from store"; + writeEmptyRetentionLeasesFile(indexShard); + indexShard.recoveryState().getIndex().setFileDetailsComplete(); + indexShard.openEngineAndRecoverFromTranslog(false); + indexShard.getEngine().fillSeqNoGaps(indexShard.getPendingPrimaryTerm()); + indexShard.finalizeRecovery(); + if (indexShard.isRemoteTranslogEnabled() && indexShard.shardRouting.primary()) { + indexShard.waitForRemoteStoreSync(); + } + indexShard.postRecovery("post recovery from remote_store"); + SegmentInfos committedSegmentInfos = indexShard.store().readLastCommittedSegmentsInfo(); + try { + indexShard.getEngine() + .translogManager() + .setMinSeqNoToKeep(Long.parseLong(committedSegmentInfos.getUserData().get(SequenceNumbers.MAX_SEQ_NO)) + 1); + } catch (IllegalArgumentException e) { + logger.warn("MinSeqNoToKeep is already past the maxSeqNo from commited segment infos"); + } + listener.onResponse(true); + }, listener::onFailure); + } + } catch (Exception e) { + listener.onFailure(e); + } + } + private boolean canRecover(IndexShard indexShard) { if (indexShard.state() == IndexShardState.CLOSED) { // got closed on us, just ignore this recovery diff --git a/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryFactory.java b/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryFactory.java index 3f6f4eeeef87b..ea7e2506deec3 100644 --- a/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryFactory.java +++ b/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryFactory.java @@ -40,11 +40,17 @@ @PublicApi(since = "2.3.0") public class RemoteSegmentStoreDirectoryFactory implements IndexStorePlugin.DirectoryFactory { private final Supplier repositoriesService; + private final String segmentsPathFixedPrefix; private final ThreadPool threadPool; - public RemoteSegmentStoreDirectoryFactory(Supplier repositoriesService, ThreadPool threadPool) { + public RemoteSegmentStoreDirectoryFactory( + Supplier repositoriesService, + ThreadPool threadPool, + String segmentsPathFixedPrefix + ) { this.repositoriesService = repositoriesService; + this.segmentsPathFixedPrefix = segmentsPathFixedPrefix; this.threadPool = threadPool; } @@ -71,6 +77,7 @@ public Directory newDirectory(String repositoryName, String indexUUID, ShardId s .shardId(shardIdStr) .dataCategory(SEGMENTS) .dataType(DATA) + .fixedPrefix(segmentsPathFixedPrefix) .build(); // Derive the path for data directory of SEGMENTS BlobPath dataPath = pathStrategy.generatePath(dataPathInput); @@ -87,6 +94,7 @@ public Directory newDirectory(String repositoryName, String indexUUID, ShardId s .shardId(shardIdStr) .dataCategory(SEGMENTS) .dataType(METADATA) + .fixedPrefix(segmentsPathFixedPrefix) .build(); // Derive the path for metadata directory of SEGMENTS BlobPath mdPath = pathStrategy.generatePath(mdPathInput); @@ -98,7 +106,8 @@ public Directory newDirectory(String repositoryName, String indexUUID, ShardId s repositoryName, indexUUID, shardIdStr, - pathStrategy + pathStrategy, + segmentsPathFixedPrefix ); return new RemoteSegmentStoreDirectory(dataDirectory, metadataDirectory, mdLockManager, threadPool, shardId); diff --git a/server/src/main/java/org/opensearch/index/store/lockmanager/RemoteStoreLockManagerFactory.java b/server/src/main/java/org/opensearch/index/store/lockmanager/RemoteStoreLockManagerFactory.java index 993c1bbdf033f..40b540d30adc7 100644 --- a/server/src/main/java/org/opensearch/index/store/lockmanager/RemoteStoreLockManagerFactory.java +++ b/server/src/main/java/org/opensearch/index/store/lockmanager/RemoteStoreLockManagerFactory.java @@ -31,9 +31,11 @@ @PublicApi(since = "2.8.0") public class RemoteStoreLockManagerFactory { private final Supplier repositoriesService; + private final String segmentsPathFixedPrefix; - public RemoteStoreLockManagerFactory(Supplier repositoriesService) { + public RemoteStoreLockManagerFactory(Supplier repositoriesService, String segmentsPathFixedPrefix) { this.repositoriesService = repositoriesService; + this.segmentsPathFixedPrefix = segmentsPathFixedPrefix; } public RemoteStoreLockManager newLockManager( @@ -42,7 +44,7 @@ public RemoteStoreLockManager newLockManager( String shardId, RemoteStorePathStrategy pathStrategy ) { - return newLockManager(repositoriesService.get(), repositoryName, indexUUID, shardId, pathStrategy); + return newLockManager(repositoriesService.get(), repositoryName, indexUUID, shardId, pathStrategy, segmentsPathFixedPrefix); } public static RemoteStoreMetadataLockManager newLockManager( @@ -50,7 +52,8 @@ public static RemoteStoreMetadataLockManager newLockManager( String repositoryName, String indexUUID, String shardId, - RemoteStorePathStrategy pathStrategy + RemoteStorePathStrategy pathStrategy, + String segmentsPathFixedPrefix ) { try (Repository repository = repositoriesService.repository(repositoryName)) { assert repository instanceof BlobStoreRepository : "repository should be instance of BlobStoreRepository"; @@ -62,6 +65,7 @@ public static RemoteStoreMetadataLockManager newLockManager( .shardId(shardId) .dataCategory(SEGMENTS) .dataType(LOCK_FILES) + .fixedPrefix(segmentsPathFixedPrefix) .build(); BlobPath lockDirectoryPath = pathStrategy.generatePath(lockFilesPathInput); BlobContainer lockDirectoryBlobContainer = ((BlobStoreRepository) repository).blobStore().blobContainer(lockDirectoryPath); diff --git a/server/src/main/java/org/opensearch/index/translog/RemoteFsTranslog.java b/server/src/main/java/org/opensearch/index/translog/RemoteFsTranslog.java index f6afc4c508720..16ebac0a44139 100644 --- a/server/src/main/java/org/opensearch/index/translog/RemoteFsTranslog.java +++ b/server/src/main/java/org/opensearch/index/translog/RemoteFsTranslog.java @@ -123,36 +123,6 @@ public RemoteFsTranslog( BooleanSupplier startedPrimarySupplier, RemoteTranslogTransferTracker remoteTranslogTransferTracker, RemoteStoreSettings remoteStoreSettings - ) throws IOException { - this( - config, - translogUUID, - deletionPolicy, - globalCheckpointSupplier, - primaryTermSupplier, - persistedSequenceNumberConsumer, - blobStoreRepository, - threadPool, - startedPrimarySupplier, - remoteTranslogTransferTracker, - remoteStoreSettings, - 0 - ); - } - - public RemoteFsTranslog( - TranslogConfig config, - String translogUUID, - TranslogDeletionPolicy deletionPolicy, - LongSupplier globalCheckpointSupplier, - LongSupplier primaryTermSupplier, - LongConsumer persistedSequenceNumberConsumer, - BlobStoreRepository blobStoreRepository, - ThreadPool threadPool, - BooleanSupplier startedPrimarySupplier, - RemoteTranslogTransferTracker remoteTranslogTransferTracker, - RemoteStoreSettings remoteStoreSettings, - long timestamp ) throws IOException { super(config, translogUUID, deletionPolicy, globalCheckpointSupplier, primaryTermSupplier, persistedSequenceNumberConsumer); logger = Loggers.getLogger(getClass(), shardId); @@ -173,7 +143,9 @@ public RemoteFsTranslog( isTranslogMetadataEnabled ); try { - download(translogTransferManager, location, logger, config.shouldSeedRemote(), timestamp); + if (config.downloadRemoteTranslogOnInit()) { + download(translogTransferManager, location, logger, config.shouldSeedRemote(), 0); + } Checkpoint checkpoint = readCheckpoint(location); logger.info("Downloaded data from remote translog till maxSeqNo = {}", checkpoint.maxSeqNo); this.readers.addAll(recoverFromFiles(checkpoint)); @@ -182,6 +154,9 @@ public RemoteFsTranslog( logger.error(errorMsg); throw new IllegalStateException(errorMsg); } + if (config.downloadRemoteTranslogOnInit() == false) { + translogTransferManager.populateFileTrackerWithLocalState(this.readers); + } boolean success = false; current = null; try { @@ -214,31 +189,6 @@ RemoteTranslogTransferTracker getRemoteTranslogTracker() { return remoteTranslogTransferTracker; } - public static void download( - Repository repository, - ShardId shardId, - ThreadPool threadPool, - Path location, - RemoteStorePathStrategy pathStrategy, - RemoteStoreSettings remoteStoreSettings, - Logger logger, - boolean seedRemote, - boolean isTranslogMetadataEnabled - ) throws IOException { - download( - repository, - shardId, - threadPool, - location, - pathStrategy, - remoteStoreSettings, - logger, - seedRemote, - isTranslogMetadataEnabled, - 0 - ); - } - public static void download( Repository repository, ShardId shardId, @@ -392,6 +342,7 @@ public static TranslogTransferManager buildTranslogTransferManager( .shardId(shardIdStr) .dataCategory(TRANSLOG) .dataType(DATA) + .fixedPrefix(remoteStoreSettings.getTranslogPathFixedPrefix()) .build(); BlobPath dataPath = pathStrategy.generatePath(dataPathInput); RemoteStorePathStrategy.ShardDataPathInput mdPathInput = RemoteStorePathStrategy.ShardDataPathInput.builder() @@ -400,6 +351,7 @@ public static TranslogTransferManager buildTranslogTransferManager( .shardId(shardIdStr) .dataCategory(TRANSLOG) .dataType(METADATA) + .fixedPrefix(remoteStoreSettings.getTranslogPathFixedPrefix()) .build(); BlobPath mdPath = pathStrategy.generatePath(mdPathInput); BlobStoreTransferService transferService = new BlobStoreTransferService(blobStoreRepository.blobStore(), threadPool); diff --git a/server/src/main/java/org/opensearch/index/translog/TranslogConfig.java b/server/src/main/java/org/opensearch/index/translog/TranslogConfig.java index f720f041b287c..52e20d9838fca 100644 --- a/server/src/main/java/org/opensearch/index/translog/TranslogConfig.java +++ b/server/src/main/java/org/opensearch/index/translog/TranslogConfig.java @@ -60,6 +60,7 @@ public final class TranslogConfig { private final ByteSizeValue bufferSize; private final String nodeId; private final boolean seedRemote; + private boolean downloadRemoteTranslogOnInit = true; /** * Creates a new TranslogConfig instance @@ -140,4 +141,12 @@ public String getNodeId() { public boolean shouldSeedRemote() { return seedRemote; } + + public boolean downloadRemoteTranslogOnInit() { + return downloadRemoteTranslogOnInit; + } + + public void setDownloadRemoteTranslogOnInit(boolean downloadRemoteTranslogOnInit) { + this.downloadRemoteTranslogOnInit = downloadRemoteTranslogOnInit; + } } diff --git a/server/src/main/java/org/opensearch/index/translog/transfer/TranslogTransferManager.java b/server/src/main/java/org/opensearch/index/translog/transfer/TranslogTransferManager.java index 527fb0123d800..56a9aa6447dec 100644 --- a/server/src/main/java/org/opensearch/index/translog/transfer/TranslogTransferManager.java +++ b/server/src/main/java/org/opensearch/index/translog/transfer/TranslogTransferManager.java @@ -28,6 +28,7 @@ import org.opensearch.index.remote.RemoteStoreUtils; import org.opensearch.index.remote.RemoteTranslogTransferTracker; import org.opensearch.index.translog.Translog; +import org.opensearch.index.translog.TranslogReader; import org.opensearch.index.translog.transfer.listener.TranslogTransferListener; import org.opensearch.indices.RemoteStoreSettings; import org.opensearch.threadpool.ThreadPool; @@ -712,4 +713,23 @@ public void onFailure(Exception e) { public int getMaxRemoteTranslogReadersSettings() { return this.remoteStoreSettings.getMaxRemoteTranslogReaders(); } + + public void populateFileTrackerWithLocalState(List readers) { + if (readers == null) { + return; + } + for (TranslogReader reader : readers) { + long generation = reader.getGeneration(); + String tlogFilename = Translog.getFilename(generation); + fileTransferTracker.add(tlogFilename, true); + if (isTranslogMetadataEnabled) { + String ckpFilename = Translog.getCommitCheckpointFileName(generation); + fileTransferTracker.add(ckpFilename, true); + } + } + } + + protected FileTransferTracker getFileTransferTracker() { + return fileTransferTracker; + } } diff --git a/server/src/main/java/org/opensearch/indices/RemoteStoreSettings.java b/server/src/main/java/org/opensearch/indices/RemoteStoreSettings.java index 00d9ffdac6086..4bd0caa709436 100644 --- a/server/src/main/java/org/opensearch/indices/RemoteStoreSettings.java +++ b/server/src/main/java/org/opensearch/indices/RemoteStoreSettings.java @@ -164,6 +164,26 @@ public class RemoteStoreSettings { Setting.Property.NodeScope ); + /** + * Controls the fixed prefix for the translog path on remote store. + */ + public static final Setting CLUSTER_REMOTE_STORE_TRANSLOG_PATH_PREFIX = Setting.simpleString( + "cluster.remote_store.translog.path.prefix", + "", + Property.NodeScope, + Property.Final + ); + + /** + * Controls the fixed prefix for the segments path on remote store. + */ + public static final Setting CLUSTER_REMOTE_STORE_SEGMENTS_PATH_PREFIX = Setting.simpleString( + "cluster.remote_store.segments.path.prefix", + "", + Property.NodeScope, + Property.Final + ); + private volatile TimeValue clusterRemoteTranslogBufferInterval; private volatile int minRemoteSegmentMetadataFiles; private volatile TimeValue clusterRemoteTranslogTransferTimeout; @@ -175,6 +195,8 @@ public class RemoteStoreSettings { private static volatile boolean isPinnedTimestampsEnabled; private static volatile TimeValue pinnedTimestampsSchedulerInterval; private static volatile TimeValue pinnedTimestampsLookbackInterval; + private final String translogPathFixedPrefix; + private final String segmentsPathFixedPrefix; public RemoteStoreSettings(Settings settings, ClusterSettings clusterSettings) { clusterRemoteTranslogBufferInterval = CLUSTER_REMOTE_TRANSLOG_BUFFER_INTERVAL_SETTING.get(settings); @@ -216,6 +238,9 @@ public RemoteStoreSettings(Settings settings, ClusterSettings clusterSettings) { pinnedTimestampsSchedulerInterval = CLUSTER_REMOTE_STORE_PINNED_TIMESTAMP_SCHEDULER_INTERVAL.get(settings); pinnedTimestampsLookbackInterval = CLUSTER_REMOTE_STORE_PINNED_TIMESTAMP_LOOKBACK_INTERVAL.get(settings); isPinnedTimestampsEnabled = CLUSTER_REMOTE_STORE_PINNED_TIMESTAMP_ENABLED.get(settings); + + translogPathFixedPrefix = CLUSTER_REMOTE_STORE_TRANSLOG_PATH_PREFIX.get(settings); + segmentsPathFixedPrefix = CLUSTER_REMOTE_STORE_SEGMENTS_PATH_PREFIX.get(settings); } public TimeValue getClusterRemoteTranslogBufferInterval() { @@ -300,4 +325,12 @@ public static void setPinnedTimestampsLookbackInterval(TimeValue pinnedTimestamp public static boolean isPinnedTimestampsEnabled() { return isPinnedTimestampsEnabled; } + + public String getTranslogPathFixedPrefix() { + return translogPathFixedPrefix; + } + + public String getSegmentsPathFixedPrefix() { + return segmentsPathFixedPrefix; + } } diff --git a/server/src/main/java/org/opensearch/indices/recovery/PeerRecoveryTargetService.java b/server/src/main/java/org/opensearch/indices/recovery/PeerRecoveryTargetService.java index 6279a8ec3646c..d746aaa2a0783 100644 --- a/server/src/main/java/org/opensearch/indices/recovery/PeerRecoveryTargetService.java +++ b/server/src/main/java/org/opensearch/indices/recovery/PeerRecoveryTargetService.java @@ -294,6 +294,7 @@ private void doRecovery(final long recoveryId, final StartRecoveryRequest preExi logger.debug("{} reestablishing recovery from {}", startRequest.shardId(), startRequest.sourceNode()); } } + transportService.sendRequest( startRequest.sourceNode(), actionName, diff --git a/server/src/main/java/org/opensearch/indices/recovery/StartRecoveryRequest.java b/server/src/main/java/org/opensearch/indices/recovery/StartRecoveryRequest.java index 1df0d3861f686..7309712314acd 100644 --- a/server/src/main/java/org/opensearch/indices/recovery/StartRecoveryRequest.java +++ b/server/src/main/java/org/opensearch/indices/recovery/StartRecoveryRequest.java @@ -144,8 +144,8 @@ public void writeTo(StreamOutput out) throws IOException { out.writeLong(recoveryId); shardId.writeTo(out); out.writeString(targetAllocationId); - sourceNode.writeTo(out); - targetNode.writeTo(out); + sourceNode.writeToWithAttribute(out); + targetNode.writeToWithAttribute(out); metadataSnapshot.writeTo(out); out.writeBoolean(primaryRelocation); out.writeLong(startingSeqNo); diff --git a/server/src/main/java/org/opensearch/node/Node.java b/server/src/main/java/org/opensearch/node/Node.java index 0d3ed222b9e76..3978186e55dd7 100644 --- a/server/src/main/java/org/opensearch/node/Node.java +++ b/server/src/main/java/org/opensearch/node/Node.java @@ -791,6 +791,7 @@ protected Node( clusterService.getClusterSettings(), threadPool::relativeTimeInMillis ); + final RemoteStoreSettings remoteStoreSettings = new RemoteStoreSettings(settings, settingsModule.getClusterSettings()); final RemoteClusterStateService remoteClusterStateService; final RemoteClusterStateCleanupManager remoteClusterStateCleanupManager; final RemoteIndexPathUploader remoteIndexPathUploader; @@ -799,7 +800,8 @@ protected Node( threadPool, settings, repositoriesServiceReference::get, - clusterService.getClusterSettings() + clusterService.getClusterSettings(), + remoteStoreSettings ); remoteClusterStateService = new RemoteClusterStateService( nodeEnvironment.nodeId(), @@ -872,12 +874,12 @@ protected Node( final RecoverySettings recoverySettings = new RecoverySettings(settings, settingsModule.getClusterSettings()); - final RemoteStoreSettings remoteStoreSettings = new RemoteStoreSettings(settings, settingsModule.getClusterSettings()); final CompositeIndexSettings compositeIndexSettings = new CompositeIndexSettings(settings, settingsModule.getClusterSettings()); final IndexStorePlugin.DirectoryFactory remoteDirectoryFactory = new RemoteSegmentStoreDirectoryFactory( repositoriesServiceReference::get, - threadPool + threadPool, + remoteStoreSettings.getSegmentsPathFixedPrefix() ); final TaskResourceTrackingService taskResourceTrackingService = new TaskResourceTrackingService( @@ -1210,7 +1212,8 @@ protected Node( repositoryService, transportService, actionModule.getActionFilters(), - remoteStorePinnedTimestampService + remoteStorePinnedTimestampService, + remoteStoreSettings ); SnapshotShardsService snapshotShardsService = new SnapshotShardsService( settings, @@ -1420,6 +1423,7 @@ protected Node( b.bind(SnapshotsInfoService.class).toInstance(snapshotsInfoService); b.bind(GatewayMetaState.class).toInstance(gatewayMetaState); b.bind(Discovery.class).toInstance(discoveryModule.getDiscovery()); + b.bind(RemoteStoreSettings.class).toInstance(remoteStoreSettings); { b.bind(PeerRecoverySourceService.class) .toInstance(new PeerRecoverySourceService(transportService, indicesService, recoverySettings)); diff --git a/server/src/main/java/org/opensearch/repositories/FilterRepository.java b/server/src/main/java/org/opensearch/repositories/FilterRepository.java index 114cd0260fcca..188d61e00f206 100644 --- a/server/src/main/java/org/opensearch/repositories/FilterRepository.java +++ b/server/src/main/java/org/opensearch/repositories/FilterRepository.java @@ -97,6 +97,27 @@ public void getRepositoryData(ActionListener listener) { in.getRepositoryData(listener); } + @Override + public void finalizeSnapshot( + ShardGenerations shardGenerations, + long repositoryStateId, + Metadata clusterMetadata, + SnapshotInfo snapshotInfo, + Version repositoryMetaVersion, + Function stateTransformer, + ActionListener listener + ) { + in.finalizeSnapshot( + shardGenerations, + repositoryStateId, + clusterMetadata, + snapshotInfo, + repositoryMetaVersion, + stateTransformer, + listener + ); + } + @Override public void finalizeSnapshot( ShardGenerations shardGenerations, diff --git a/server/src/main/java/org/opensearch/repositories/Repository.java b/server/src/main/java/org/opensearch/repositories/Repository.java index 637503d3f54df..4338c42e20b4b 100644 --- a/server/src/main/java/org/opensearch/repositories/Repository.java +++ b/server/src/main/java/org/opensearch/repositories/Repository.java @@ -139,6 +139,30 @@ default Repository create(RepositoryMetadata metadata, Function listener); + /** + * Finalizes snapshotting process + *

+ * This method is called on cluster-manager after all shards are snapshotted. + * + * @param shardGenerations updated shard generations + * @param repositoryStateId the unique id identifying the state of the repository when the snapshot began + * @param clusterMetadata cluster metadata + * @param snapshotInfo SnapshotInfo instance to write for this snapshot + * @param repositoryMetaVersion version of the updated repository metadata to write + * @param stateTransformer a function that filters the last cluster state update that the snapshot finalization will execute and + * is used to remove any state tracked for the in-progress snapshot from the cluster state + * @param listener listener to be invoked with the new {@link RepositoryData} after completing the snapshot + */ + void finalizeSnapshot( + ShardGenerations shardGenerations, + long repositoryStateId, + Metadata clusterMetadata, + SnapshotInfo snapshotInfo, + Version repositoryMetaVersion, + Function stateTransformer, + ActionListener listener + ); + /** * Finalizes snapshotting process *

@@ -394,6 +418,18 @@ default RemoteStoreShardShallowCopySnapshot getRemoteStoreShallowCopyShardMetada */ IndexShardSnapshotStatus getShardSnapshotStatus(SnapshotId snapshotId, IndexId indexId, ShardId shardId); + /** + * Retrieve shard snapshot status for the stored snapshot + * + * @param snapshotInfo snapshot info + * @param indexId the snapshotted index id for the shard to get status for + * @param shardId shard id + * @return snapshot status + */ + default IndexShardSnapshotStatus getShardSnapshotStatus(SnapshotInfo snapshotInfo, IndexId indexId, ShardId shardId) { + return getShardSnapshotStatus(snapshotInfo.snapshotId(), indexId, shardId); + } + /** * Update the repository with the incoming cluster state. This method is invoked from {@link RepositoriesService#applyClusterState} and * thus the same semantics as with {@link org.opensearch.cluster.ClusterStateApplier#applyClusterState} apply for the diff --git a/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreRepository.java b/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreRepository.java index d2df42fed6db5..d193ab3c14154 100644 --- a/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreRepository.java +++ b/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreRepository.java @@ -130,6 +130,7 @@ import org.opensearch.index.store.lockmanager.FileLockInfo; import org.opensearch.index.store.lockmanager.RemoteStoreLockManager; import org.opensearch.index.store.lockmanager.RemoteStoreLockManagerFactory; +import org.opensearch.indices.RemoteStoreSettings; import org.opensearch.indices.recovery.RecoverySettings; import org.opensearch.indices.recovery.RecoveryState; import org.opensearch.repositories.IndexId; @@ -420,6 +421,8 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp private final RecoverySettings recoverySettings; + private final RemoteStoreSettings remoteStoreSettings; + private final NamedXContentRegistry namedXContentRegistry; /** @@ -472,6 +475,7 @@ protected BlobStoreRepository( this.threadPool = clusterService.getClusterApplierService().threadPool(); this.clusterService = clusterService; this.recoverySettings = recoverySettings; + this.remoteStoreSettings = new RemoteStoreSettings(clusterService.getSettings(), clusterService.getClusterSettings()); } @Override @@ -827,7 +831,7 @@ boolean getPrefixModeVerification() { * maintains single lazy instance of {@link BlobContainer} */ protected BlobContainer blobContainer() { - assertSnapshotOrGenericThread(); + // assertSnapshotOrGenericThread(); BlobContainer blobContainer = this.blobContainer.get(); if (blobContainer == null) { @@ -1296,7 +1300,8 @@ protected void releaseRemoteStoreLockAndCleanup( // related issue: https://github.com/opensearch-project/OpenSearch/issues/8469 RemoteSegmentStoreDirectoryFactory remoteDirectoryFactory = new RemoteSegmentStoreDirectoryFactory( remoteStoreLockManagerFactory.getRepositoriesService(), - threadPool + threadPool, + remoteStoreSettings.getSegmentsPathFixedPrefix() ); remoteDirectoryCleanupAsync( remoteDirectoryFactory, @@ -2019,6 +2024,28 @@ private DeleteResult cleanUpStaleSnapshotShardPathsFile(List matchingSha return new DeleteResult(matchingShardPaths.size(), totalBytes); } + @Override + public void finalizeSnapshot( + final ShardGenerations shardGenerations, + final long repositoryStateId, + final Metadata clusterMetadata, + SnapshotInfo snapshotInfo, + Version repositoryMetaVersion, + Function stateTransformer, + final ActionListener listener + ) { + finalizeSnapshot( + shardGenerations, + repositoryStateId, + clusterMetadata, + snapshotInfo, + repositoryMetaVersion, + stateTransformer, + Priority.NORMAL, + listener + ); + } + @Override public void finalizeSnapshot( final ShardGenerations shardGenerations, @@ -3722,6 +3749,11 @@ public IndexShardSnapshotStatus getShardSnapshotStatus(SnapshotId snapshotId, In return snapshot.getIndexShardSnapshotStatus(); } + public IndexShardSnapshotStatus getShardSnapshotStatus(SnapshotInfo snapshotInfo, IndexId indexId, ShardId shardId) { + IndexShardSnapshot snapshot = loadShardSnapshot(shardContainer(indexId, shardId), snapshotInfo); + return snapshot.getIndexShardSnapshotStatus(); + } + @Override public void verify(String seed, DiscoveryNode localNode) { if (isSystemRepository == false) { @@ -3930,6 +3962,38 @@ public IndexShardSnapshot loadShardSnapshot(BlobContainer shardContainer, Snapsh } } + public IndexShardSnapshot loadShardSnapshot(BlobContainer shardContainer, SnapshotInfo snapshotInfo) { + try { + SnapshotId snapshotId = snapshotInfo.snapshotId(); + if (snapshotInfo.getPinnedTimestamp() != 0) { + return () -> IndexShardSnapshotStatus.newDone(0L, 0L, 0, 0, 0, 0, "1"); + } else if (snapshotInfo.isRemoteStoreIndexShallowCopyEnabled()) { + if (shardContainer.blobExists(REMOTE_STORE_SHARD_SHALLOW_COPY_SNAPSHOT_FORMAT.blobName(snapshotId.getUUID()))) { + return REMOTE_STORE_SHARD_SHALLOW_COPY_SNAPSHOT_FORMAT.read( + shardContainer, + snapshotId.getUUID(), + namedXContentRegistry + ); + } else { + throw new SnapshotMissingException(metadata.name(), snapshotId.getName()); + } + } else { + if (shardContainer.blobExists(INDEX_SHARD_SNAPSHOT_FORMAT.blobName(snapshotId.getUUID()))) { + return INDEX_SHARD_SNAPSHOT_FORMAT.read(shardContainer, snapshotId.getUUID(), namedXContentRegistry); + } else { + throw new SnapshotMissingException(metadata.name(), snapshotId.getName()); + } + } + } catch (IOException ex) { + throw new SnapshotException( + metadata.name(), + snapshotInfo.snapshotId(), + "failed to read shard snapshot file for [" + shardContainer.path() + ']', + ex + ); + } + } + /** * Loads all available snapshots in the repository using the given {@code generation} or falling back to trying to determine it from * the given list of blobs in the shard container. diff --git a/server/src/main/java/org/opensearch/repositories/blobstore/ChecksumWritableBlobStoreFormat.java b/server/src/main/java/org/opensearch/repositories/blobstore/ChecksumWritableBlobStoreFormat.java index 0add86ab88a16..88672995f4fd6 100644 --- a/server/src/main/java/org/opensearch/repositories/blobstore/ChecksumWritableBlobStoreFormat.java +++ b/server/src/main/java/org/opensearch/repositories/blobstore/ChecksumWritableBlobStoreFormat.java @@ -28,6 +28,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.common.io.stream.Writeable.Writer; import org.opensearch.core.compress.Compressor; import org.opensearch.core.compress.CompressorRegistry; import org.opensearch.gateway.CorruptStateException; @@ -56,6 +57,10 @@ public ChecksumWritableBlobStoreFormat(String codec, CheckedFunction unSerializedObj.writeTo(out), obj, blobName, compressor); + } + + public BytesReference serialize(final Writer writer, T obj, final String blobName, final Compressor compressor) throws IOException { try (BytesStreamOutput outputStream = new BytesStreamOutput()) { try ( OutputStreamIndexOutput indexOutput = new OutputStreamIndexOutput( @@ -76,7 +81,7 @@ public void close() throws IOException { }; StreamOutput stream = new OutputStreamStreamOutput(compressor.threadLocalOutputStream(indexOutputOutputStream));) { // TODO The stream version should be configurable stream.setVersion(Version.CURRENT); - obj.writeTo(stream); + writer.write(stream, obj); } CodecUtil.writeFooter(indexOutput); } diff --git a/server/src/main/java/org/opensearch/rest/action/admin/cluster/ClusterAdminTask.java b/server/src/main/java/org/opensearch/rest/action/admin/cluster/ClusterAdminTask.java new file mode 100644 index 0000000000000..5361b05ebecc3 --- /dev/null +++ b/server/src/main/java/org/opensearch/rest/action/admin/cluster/ClusterAdminTask.java @@ -0,0 +1,47 @@ +/* + * 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.rest.action.admin.cluster; + +import org.opensearch.common.annotation.PublicApi; +import org.opensearch.common.unit.TimeValue; +import org.opensearch.core.tasks.TaskId; +import org.opensearch.tasks.CancellableTask; + +import java.util.Map; + +import static org.opensearch.search.SearchService.NO_TIMEOUT; + +/** + * Task storing information about a currently running ClusterRequest. + * + * @opensearch.api + */ +@PublicApi(since = "2.17.0") +public class ClusterAdminTask extends CancellableTask { + + public ClusterAdminTask(long id, String type, String action, TaskId parentTaskId, Map headers) { + this(id, type, action, parentTaskId, headers, NO_TIMEOUT); + } + + public ClusterAdminTask( + long id, + String type, + String action, + TaskId parentTaskId, + Map headers, + TimeValue cancelAfterTimeInterval + ) { + super(id, type, action, null, parentTaskId, headers, cancelAfterTimeInterval); + } + + @Override + public boolean shouldCancelChildrenOnCancellation() { + return true; + } +} diff --git a/server/src/main/java/org/opensearch/rest/action/admin/cluster/RestSnapshotsStatusAction.java b/server/src/main/java/org/opensearch/rest/action/admin/cluster/RestSnapshotsStatusAction.java index 2b4b188ab0acd..502be16f2fa8e 100644 --- a/server/src/main/java/org/opensearch/rest/action/admin/cluster/RestSnapshotsStatusAction.java +++ b/server/src/main/java/org/opensearch/rest/action/admin/cluster/RestSnapshotsStatusAction.java @@ -61,6 +61,7 @@ public class RestSnapshotsStatusAction extends BaseRestHandler { public List routes() { return unmodifiableList( asList( + new Route(GET, "/_snapshot/{repository}/{snapshot}/{index}/_status"), new Route(GET, "/_snapshot/{repository}/{snapshot}/_status"), new Route(GET, "/_snapshot/{repository}/_status"), new Route(GET, "/_snapshot/_status") @@ -80,7 +81,8 @@ public RestChannelConsumer prepareRequest(final RestRequest request, final NodeC if (snapshots.length == 1 && "_all".equalsIgnoreCase(snapshots[0])) { snapshots = Strings.EMPTY_ARRAY; } - SnapshotsStatusRequest snapshotsStatusRequest = snapshotsStatusRequest(repository).snapshots(snapshots); + String[] indices = request.paramAsStringArray("index", Strings.EMPTY_ARRAY); + SnapshotsStatusRequest snapshotsStatusRequest = snapshotsStatusRequest(repository).snapshots(snapshots).indices(indices); snapshotsStatusRequest.ignoreUnavailable(request.paramAsBoolean("ignore_unavailable", snapshotsStatusRequest.ignoreUnavailable())); snapshotsStatusRequest.clusterManagerNodeTimeout( diff --git a/server/src/main/java/org/opensearch/rest/action/cat/RestShardsAction.java b/server/src/main/java/org/opensearch/rest/action/cat/RestShardsAction.java index 2113b8d59c531..a7ad5fe6c14a3 100644 --- a/server/src/main/java/org/opensearch/rest/action/cat/RestShardsAction.java +++ b/server/src/main/java/org/opensearch/rest/action/cat/RestShardsAction.java @@ -32,10 +32,11 @@ package org.opensearch.rest.action.cat; -import org.opensearch.action.admin.cluster.state.ClusterStateRequest; +import org.opensearch.action.admin.cluster.shards.CatShardsAction; +import org.opensearch.action.admin.cluster.shards.CatShardsRequest; +import org.opensearch.action.admin.cluster.shards.CatShardsResponse; import org.opensearch.action.admin.cluster.state.ClusterStateResponse; import org.opensearch.action.admin.indices.stats.CommonStats; -import org.opensearch.action.admin.indices.stats.IndicesStatsRequest; import org.opensearch.action.admin.indices.stats.IndicesStatsResponse; import org.opensearch.action.admin.indices.stats.ShardStats; import org.opensearch.client.node.NodeClient; @@ -61,7 +62,6 @@ import org.opensearch.index.warmer.WarmerStats; import org.opensearch.rest.RestRequest; import org.opensearch.rest.RestResponse; -import org.opensearch.rest.action.RestActionListener; import org.opensearch.rest.action.RestResponseListener; import org.opensearch.search.suggest.completion.CompletionStats; @@ -73,6 +73,7 @@ import static java.util.Arrays.asList; import static java.util.Collections.unmodifiableList; import static org.opensearch.rest.RestRequest.Method.GET; +import static org.opensearch.search.SearchService.NO_TIMEOUT; /** * _cat API action to get shard information @@ -107,25 +108,18 @@ protected void documentation(StringBuilder sb) { @Override public RestChannelConsumer doCatRequest(final RestRequest request, final NodeClient client) { final String[] indices = Strings.splitStringByCommaToArray(request.param("index")); - final ClusterStateRequest clusterStateRequest = new ClusterStateRequest(); - clusterStateRequest.local(request.paramAsBoolean("local", clusterStateRequest.local())); - clusterStateRequest.clusterManagerNodeTimeout( - request.paramAsTime("cluster_manager_timeout", clusterStateRequest.clusterManagerNodeTimeout()) - ); - parseDeprecatedMasterTimeoutParameter(clusterStateRequest, request, deprecationLogger, getName()); - clusterStateRequest.clear().nodes(true).routingTable(true).indices(indices); - return channel -> client.admin().cluster().state(clusterStateRequest, new RestActionListener(channel) { + final CatShardsRequest shardsRequest = new CatShardsRequest(); + shardsRequest.local(request.paramAsBoolean("local", shardsRequest.local())); + shardsRequest.clusterManagerNodeTimeout(request.paramAsTime("cluster_manager_timeout", shardsRequest.clusterManagerNodeTimeout())); + shardsRequest.setCancelAfterTimeInterval(request.paramAsTime("cancel_after_time_interval", NO_TIMEOUT)); + shardsRequest.setIndices(indices); + parseDeprecatedMasterTimeoutParameter(shardsRequest, request, deprecationLogger, getName()); + return channel -> client.execute(CatShardsAction.INSTANCE, shardsRequest, new RestResponseListener(channel) { @Override - public void processResponse(final ClusterStateResponse clusterStateResponse) { - IndicesStatsRequest indicesStatsRequest = new IndicesStatsRequest(); - indicesStatsRequest.all(); - indicesStatsRequest.indices(indices); - client.admin().indices().stats(indicesStatsRequest, new RestResponseListener(channel) { - @Override - public RestResponse buildResponse(IndicesStatsResponse indicesStatsResponse) throws Exception { - return RestTable.buildResponse(buildTable(request, clusterStateResponse, indicesStatsResponse), channel); - } - }); + public RestResponse buildResponse(CatShardsResponse catShardsResponse) throws Exception { + ClusterStateResponse clusterStateResponse = catShardsResponse.getClusterStateResponse(); + IndicesStatsResponse indicesStatsResponse = catShardsResponse.getIndicesStatsResponse(); + return RestTable.buildResponse(buildTable(request, clusterStateResponse, indicesStatsResponse), channel); } }); } diff --git a/server/src/main/java/org/opensearch/snapshots/InternalSnapshotsInfoService.java b/server/src/main/java/org/opensearch/snapshots/InternalSnapshotsInfoService.java index 797a58f3b0d9b..e300f845e6f58 100644 --- a/server/src/main/java/org/opensearch/snapshots/InternalSnapshotsInfoService.java +++ b/server/src/main/java/org/opensearch/snapshots/InternalSnapshotsInfoService.java @@ -238,14 +238,18 @@ protected void doRun() throws Exception { final Repository repository = repositories.repository(snapshotShard.snapshot.getRepository()); logger.debug("fetching snapshot shard size for {}", snapshotShard); - final long snapshotShardSize = repository.getShardSnapshotStatus( - snapshotShard.snapshot().getSnapshotId(), - snapshotShard.index(), - snapshotShard.shardId() - ).asCopy().getTotalSize(); + long snapshotShardSize; + if (snapshotShard.pinnedTimestamp > 0) { + snapshotShardSize = 0; + } else { + snapshotShardSize = repository.getShardSnapshotStatus( + snapshotShard.snapshot().getSnapshotId(), + snapshotShard.index(), + snapshotShard.shardId() + ).asCopy().getTotalSize(); + } logger.debug("snapshot shard size for {}: {} bytes", snapshotShard, snapshotShardSize); - boolean updated = false; synchronized (mutex) { removed = unknownSnapshotShards.remove(snapshotShard); @@ -354,7 +358,8 @@ private static Set listOfSnapshotShards(final ClusterState state) final SnapshotShard snapshotShard = new SnapshotShard( snapshotRecoverySource.snapshot(), snapshotRecoverySource.index(), - shardRouting.shardId() + shardRouting.shardId(), + snapshotRecoverySource.pinnedTimestamp() ); snapshotShards.add(snapshotShard); } @@ -374,10 +379,17 @@ public static class SnapshotShard { private final IndexId index; private final ShardId shardId; + private long pinnedTimestamp; + public SnapshotShard(Snapshot snapshot, IndexId index, ShardId shardId) { + this(snapshot, index, shardId, 0L); + } + + public SnapshotShard(Snapshot snapshot, IndexId index, ShardId shardId, long pinnedTimestamp) { this.snapshot = snapshot; this.index = index; this.shardId = shardId; + this.pinnedTimestamp = pinnedTimestamp; } public Snapshot snapshot() { diff --git a/server/src/main/java/org/opensearch/snapshots/RestoreService.java b/server/src/main/java/org/opensearch/snapshots/RestoreService.java index 174cad50b00e2..79a70d835f773 100644 --- a/server/src/main/java/org/opensearch/snapshots/RestoreService.java +++ b/server/src/main/java/org/opensearch/snapshots/RestoreService.java @@ -428,7 +428,9 @@ public ClusterState execute(ClusterState currentState) { snapshotIndexId, isSearchableSnapshot, isRemoteStoreShallowCopy, - request.getSourceRemoteStoreRepository() + request.getSourceRemoteStoreRepository(), + request.getSourceRemoteTranslogRepository(), + snapshotInfo.getPinnedTimestamp() ); final Version minIndexCompatibilityVersion; if (isSearchableSnapshot && isSearchableSnapshotsExtendedCompatibilityEnabled()) { @@ -551,7 +553,7 @@ public ClusterState execute(ClusterState currentState) { for (int shard = 0; shard < snapshotIndexMetadata.getNumberOfShards(); shard++) { if (isRemoteSnapshot) { IndexShardSnapshotStatus.Copy shardStatus = repository.getShardSnapshotStatus( - snapshotInfo.snapshotId(), + snapshotInfo, snapshotIndexId, new ShardId(metadata.index(index).getIndex(), shard) ).asCopy(); diff --git a/server/src/main/java/org/opensearch/snapshots/SnapshotInfo.java b/server/src/main/java/org/opensearch/snapshots/SnapshotInfo.java index 7558c4456109e..bc33928acf3da 100644 --- a/server/src/main/java/org/opensearch/snapshots/SnapshotInfo.java +++ b/server/src/main/java/org/opensearch/snapshots/SnapshotInfo.java @@ -354,6 +354,38 @@ public SnapshotInfo(SnapshotsInProgress.Entry entry) { ); } + public SnapshotInfo( + SnapshotId snapshotId, + List indices, + List dataStreams, + long startTime, + String reason, + long endTime, + int totalShards, + List shardFailures, + Boolean includeGlobalState, + Map userMetadata, + Boolean remoteStoreIndexShallowCopy + ) { + this( + snapshotId, + indices, + dataStreams, + snapshotState(reason, shardFailures), + reason, + Version.CURRENT, + startTime, + endTime, + totalShards, + totalShards - shardFailures.size(), + shardFailures, + includeGlobalState, + userMetadata, + remoteStoreIndexShallowCopy, + 0 + ); + } + public SnapshotInfo( SnapshotId snapshotId, List indices, diff --git a/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java b/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java index 5ecb991127fd5..8b595fb7135c1 100644 --- a/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java +++ b/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java @@ -92,6 +92,7 @@ import org.opensearch.core.index.Index; import org.opensearch.core.index.shard.ShardId; import org.opensearch.index.store.lockmanager.RemoteStoreLockManagerFactory; +import org.opensearch.indices.RemoteStoreSettings; import org.opensearch.node.remotestore.RemoteStorePinnedTimestampService; import org.opensearch.repositories.IndexId; import org.opensearch.repositories.RepositoriesService; @@ -204,7 +205,18 @@ public class SnapshotsService extends AbstractLifecycleComponent implements Clus Setting.Property.Dynamic ); - private static final String SNAPSHOT_PINNED_TIMESTAMP_DELIMITER = ":"; + private static final String SNAPSHOT_PINNED_TIMESTAMP_DELIMITER = "__"; + /** + * Setting to specify the maximum number of shards that can be included in the result for the snapshot status + * API call. Note that it does not apply to V2-shallow snapshots. + */ + public static final Setting MAX_SHARDS_ALLOWED_IN_STATUS_API = Setting.intSetting( + "snapshot.max_shards_allowed_in_status_api", + 200000, + 1, + Setting.Property.NodeScope, + Setting.Property.Dynamic + ); private volatile int maxConcurrentOperations; public SnapshotsService( @@ -214,12 +226,16 @@ public SnapshotsService( RepositoriesService repositoriesService, TransportService transportService, ActionFilters actionFilters, - @Nullable RemoteStorePinnedTimestampService remoteStorePinnedTimestampService + @Nullable RemoteStorePinnedTimestampService remoteStorePinnedTimestampService, + RemoteStoreSettings remoteStoreSettings ) { this.clusterService = clusterService; this.indexNameExpressionResolver = indexNameExpressionResolver; this.repositoriesService = repositoriesService; - this.remoteStoreLockManagerFactory = new RemoteStoreLockManagerFactory(() -> repositoriesService); + this.remoteStoreLockManagerFactory = new RemoteStoreLockManagerFactory( + () -> repositoriesService, + remoteStoreSettings.getSegmentsPathFixedPrefix() + ); this.threadPool = transportService.getThreadPool(); this.transportService = transportService; this.remoteStorePinnedTimestampService = remoteStorePinnedTimestampService; diff --git a/server/src/main/java/org/opensearch/snapshots/TooManyShardsInSnapshotsStatusException.java b/server/src/main/java/org/opensearch/snapshots/TooManyShardsInSnapshotsStatusException.java new file mode 100644 index 0000000000000..1689b3e4941ec --- /dev/null +++ b/server/src/main/java/org/opensearch/snapshots/TooManyShardsInSnapshotsStatusException.java @@ -0,0 +1,69 @@ +/* + * 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. + */ + +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +/* + * Modifications Copyright OpenSearch Contributors. See + * GitHub history for details. + */ + +package org.opensearch.snapshots; + +import org.opensearch.core.common.io.stream.StreamInput; +import org.opensearch.core.rest.RestStatus; + +import java.io.IOException; + +/** + * Thrown if the number of shards across the requested resources (snapshot(s) or the index/indices of a particular snapshot) + * breaches the limit of snapshot.max_shards_allowed_in_status_api cluster setting + * + * @opensearch.internal + */ +public class TooManyShardsInSnapshotsStatusException extends SnapshotException { + + public TooManyShardsInSnapshotsStatusException( + final String repositoryName, + final SnapshotId snapshotId, + final String message, + final Throwable cause + ) { + super(repositoryName, snapshotId, message, cause); + } + + public TooManyShardsInSnapshotsStatusException(final String repositoryName, final String message, String... snapshotName) { + super(repositoryName, String.join(", ", snapshotName), message); + } + + public TooManyShardsInSnapshotsStatusException(StreamInput in) throws IOException { + super(in); + } + + @Override + public RestStatus status() { + return RestStatus.REQUEST_ENTITY_TOO_LARGE; + } +} diff --git a/server/src/main/java/org/opensearch/transport/TransportService.java b/server/src/main/java/org/opensearch/transport/TransportService.java index d08b28730d417..fff6d82b23c7e 100644 --- a/server/src/main/java/org/opensearch/transport/TransportService.java +++ b/server/src/main/java/org/opensearch/transport/TransportService.java @@ -752,7 +752,7 @@ public HandshakeResponse(StreamInput in) throws IOException { @Override public void writeTo(StreamOutput out) throws IOException { - out.writeOptionalWriteable(discoveryNode); + out.writeOptionalWriteable((stream, node) -> node.writeToWithAttribute(stream), discoveryNode); clusterName.writeTo(out); out.writeVersion(version); } diff --git a/server/src/test/java/org/opensearch/ExceptionSerializationTests.java b/server/src/test/java/org/opensearch/ExceptionSerializationTests.java index 2e4a2d7bdd59c..eff312a36dbc6 100644 --- a/server/src/test/java/org/opensearch/ExceptionSerializationTests.java +++ b/server/src/test/java/org/opensearch/ExceptionSerializationTests.java @@ -119,6 +119,7 @@ import org.opensearch.snapshots.SnapshotId; import org.opensearch.snapshots.SnapshotInProgressException; import org.opensearch.snapshots.SnapshotInUseDeletionException; +import org.opensearch.snapshots.TooManyShardsInSnapshotsStatusException; import org.opensearch.test.OpenSearchTestCase; import org.opensearch.test.VersionUtils; import org.opensearch.transport.ActionNotFoundTransportException; @@ -898,6 +899,7 @@ public void testIds() { ids.put(172, ViewNotFoundException.class); ids.put(173, ViewAlreadyExistsException.class); ids.put(174, InvalidIndexContextException.class); + ids.put(175, TooManyShardsInSnapshotsStatusException.class); ids.put(10001, IndexCreateBlockException.class); Map, Integer> reverse = new HashMap<>(); diff --git a/server/src/test/java/org/opensearch/action/admin/indices/shards/CatShardsResponseTests.java b/server/src/test/java/org/opensearch/action/admin/indices/shards/CatShardsResponseTests.java new file mode 100644 index 0000000000000..d0b98f4d286ae --- /dev/null +++ b/server/src/test/java/org/opensearch/action/admin/indices/shards/CatShardsResponseTests.java @@ -0,0 +1,61 @@ +/* + * 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. + */ + +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +/* + * Modifications Copyright OpenSearch Contributors. See + * GitHub history for details. + */ + +package org.opensearch.action.admin.indices.shards; + +import org.opensearch.action.admin.cluster.shards.CatShardsResponse; +import org.opensearch.action.admin.cluster.state.ClusterStateResponse; +import org.opensearch.action.admin.indices.stats.IndicesStatsResponse; +import org.opensearch.cluster.ClusterName; +import org.opensearch.test.OpenSearchTestCase; + +import static org.junit.Assert.assertEquals; + +public class CatShardsResponseTests extends OpenSearchTestCase { + + private final CatShardsResponse catShardsResponse = new CatShardsResponse(); + + public void testGetAndSetClusterStateResponse() { + ClusterName clusterName = new ClusterName("1"); + ClusterStateResponse clusterStateResponse = new ClusterStateResponse(clusterName, null, false); + catShardsResponse.setClusterStateResponse(clusterStateResponse); + + assertEquals(clusterStateResponse, catShardsResponse.getClusterStateResponse()); + } + + public void testGetAndSetIndicesStatsResponse() { + final IndicesStatsResponse indicesStatsResponse = new IndicesStatsResponse(null, 0, 0, 0, null); + catShardsResponse.setIndicesStatsResponse(indicesStatsResponse); + + assertEquals(indicesStatsResponse, catShardsResponse.getIndicesStatsResponse()); + } +} 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..839e831d38b1b --- /dev/null +++ b/server/src/test/java/org/opensearch/cluster/block/ClusterBlocksTests.java @@ -0,0 +1,55 @@ +/* + * 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.io.stream.BytesStreamOutput; +import org.opensearch.core.common.io.stream.BufferedChecksumStreamOutput; +import org.opensearch.core.common.io.stream.StreamInput; +import org.opensearch.test.OpenSearchTestCase; + +import static org.opensearch.cluster.block.ClusterBlockTests.randomClusterBlock; + +public class ClusterBlocksTests extends OpenSearchTestCase { + + public void testWriteVerifiableTo() throws Exception { + ClusterBlock clusterBlock1 = randomClusterBlock(); + ClusterBlock clusterBlock2 = randomClusterBlock(); + ClusterBlock clusterBlock3 = randomClusterBlock(); + + ClusterBlocks clusterBlocks = ClusterBlocks.builder() + .addGlobalBlock(clusterBlock1) + .addGlobalBlock(clusterBlock2) + .addGlobalBlock(clusterBlock3) + .addIndexBlock("index-1", clusterBlock1) + .addIndexBlock("index-2", clusterBlock2) + .build(); + BytesStreamOutput out = new BytesStreamOutput(); + BufferedChecksumStreamOutput checksumOut = new BufferedChecksumStreamOutput(out); + clusterBlocks.writeVerifiableTo(checksumOut); + StreamInput in = out.bytes().streamInput(); + ClusterBlocks result = ClusterBlocks.readFrom(in); + + assertEquals(clusterBlocks.global().size(), result.global().size()); + assertEquals(clusterBlocks.global(), result.global()); + assertEquals(clusterBlocks.indices().size(), result.indices().size()); + assertEquals(clusterBlocks.indices(), result.indices()); + + ClusterBlocks clusterBlocks2 = ClusterBlocks.builder() + .addGlobalBlock(clusterBlock3) + .addGlobalBlock(clusterBlock1) + .addGlobalBlock(clusterBlock2) + .addIndexBlock("index-2", clusterBlock2) + .addIndexBlock("index-1", clusterBlock1) + .build(); + BytesStreamOutput out2 = new BytesStreamOutput(); + BufferedChecksumStreamOutput checksumOut2 = new BufferedChecksumStreamOutput(out2); + clusterBlocks2.writeVerifiableTo(checksumOut2); + assertEquals(checksumOut.getChecksum(), checksumOut2.getChecksum()); + } +} diff --git a/server/src/test/java/org/opensearch/cluster/coordination/CoordinationMetadataTests.java b/server/src/test/java/org/opensearch/cluster/coordination/CoordinationMetadataTests.java index 290479941aaa9..74c25246c2aa4 100644 --- a/server/src/test/java/org/opensearch/cluster/coordination/CoordinationMetadataTests.java +++ b/server/src/test/java/org/opensearch/cluster/coordination/CoordinationMetadataTests.java @@ -33,10 +33,13 @@ import org.opensearch.cluster.coordination.CoordinationMetadata.VotingConfigExclusion; import org.opensearch.cluster.coordination.CoordinationMetadata.VotingConfiguration; +import org.opensearch.common.io.stream.BytesStreamOutput; import org.opensearch.common.util.set.Sets; import org.opensearch.common.xcontent.json.JsonXContent; import org.opensearch.core.common.bytes.BytesReference; +import org.opensearch.core.common.io.stream.BufferedChecksumStreamOutput; import org.opensearch.core.common.io.stream.NamedWriteableRegistry; +import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.xcontent.ToXContent; import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.core.xcontent.XContentParser; @@ -46,8 +49,11 @@ import java.io.IOException; import java.util.Collections; +import java.util.Comparator; import java.util.HashSet; +import java.util.LinkedHashSet; import java.util.Set; +import java.util.stream.Collectors; import static org.hamcrest.Matchers.equalTo; @@ -231,4 +237,29 @@ public void testXContent() throws IOException { assertThat(originalMeta, equalTo(fromXContentMeta)); } } + + public void testWriteVerifiableTo() throws IOException { + VotingConfiguration votingConfiguration = randomVotingConfig(); + Set votingTombstones = randomVotingTombstones(); + CoordinationMetadata meta1 = new CoordinationMetadata(1, votingConfiguration, votingConfiguration, votingTombstones); + BytesStreamOutput out = new BytesStreamOutput(); + BufferedChecksumStreamOutput checksumOut = new BufferedChecksumStreamOutput(out); + meta1.writeVerifiableTo(checksumOut); + StreamInput in = out.bytes().streamInput(); + CoordinationMetadata result = new CoordinationMetadata(in); + + assertEquals(meta1, result); + + VotingConfiguration votingConfiguration2 = new VotingConfiguration( + (Set) votingConfiguration.getNodeIds().stream().sorted().collect(Collectors.toCollection(LinkedHashSet::new)) + ); + Set votingTombstones2 = votingTombstones.stream() + .sorted(Comparator.comparing(VotingConfigExclusion::getNodeId)) + .collect(Collectors.toCollection(LinkedHashSet::new)); + CoordinationMetadata meta2 = new CoordinationMetadata(1, votingConfiguration2, votingConfiguration2, votingTombstones2); + BytesStreamOutput out2 = new BytesStreamOutput(); + BufferedChecksumStreamOutput checksumOut2 = new BufferedChecksumStreamOutput(out2); + meta2.writeVerifiableTo(checksumOut2); + assertEquals(checksumOut.getChecksum(), checksumOut2.getChecksum()); + } } diff --git a/server/src/test/java/org/opensearch/cluster/metadata/IndexMetadataTests.java b/server/src/test/java/org/opensearch/cluster/metadata/IndexMetadataTests.java index 3b0a2fed52c2d..a92a5b41e8580 100644 --- a/server/src/test/java/org/opensearch/cluster/metadata/IndexMetadataTests.java +++ b/server/src/test/java/org/opensearch/cluster/metadata/IndexMetadataTests.java @@ -43,6 +43,7 @@ import org.opensearch.common.xcontent.json.JsonXContent; import org.opensearch.core.common.Strings; import org.opensearch.core.common.bytes.BytesReference; +import org.opensearch.core.common.io.stream.BufferedChecksumStreamOutput; import org.opensearch.core.common.io.stream.NamedWriteableAwareStreamInput; import org.opensearch.core.common.io.stream.NamedWriteableRegistry; import org.opensearch.core.common.io.stream.StreamInput; @@ -173,6 +174,86 @@ public void testIndexMetadataSerialization() throws IOException { } } + public void testWriteVerifiableTo() throws IOException { + int numberOfReplicas = randomIntBetween(0, 10); + final boolean system = randomBoolean(); + Map customMap = new HashMap<>(); + customMap.put(randomAlphaOfLength(5), randomAlphaOfLength(10)); + customMap.put(randomAlphaOfLength(10), randomAlphaOfLength(15)); + + RolloverInfo info1 = new RolloverInfo( + randomAlphaOfLength(5), + Arrays.asList( + new MaxAgeCondition(TimeValue.timeValueMillis(randomNonNegativeLong())), + new MaxSizeCondition(new ByteSizeValue(randomNonNegativeLong())), + new MaxDocsCondition(randomNonNegativeLong()) + ), + randomNonNegativeLong() + ); + RolloverInfo info2 = new RolloverInfo( + randomAlphaOfLength(5), + Arrays.asList( + new MaxAgeCondition(TimeValue.timeValueMillis(randomNonNegativeLong())), + new MaxSizeCondition(new ByteSizeValue(randomNonNegativeLong())), + new MaxDocsCondition(randomNonNegativeLong()) + ), + randomNonNegativeLong() + ); + + IndexMetadata metadata1 = IndexMetadata.builder("foo") + .settings( + Settings.builder() + .put("index.version.created", 1) + .put("index.number_of_shards", 4) + .put("index.number_of_replicas", numberOfReplicas) + .build() + ) + .creationDate(randomLong()) + .primaryTerm(0, 2) + .primaryTerm(1, 3) + .setRoutingNumShards(32) + .system(system) + .putCustom("my_custom", customMap) + .putCustom("my_custom2", customMap) + .putAlias(AliasMetadata.builder("alias-1").routing("routing-1").build()) + .putAlias(AliasMetadata.builder("alias-2").routing("routing-2").build()) + .putRolloverInfo(info1) + .putRolloverInfo(info2) + .putInSyncAllocationIds(0, Set.of("1", "2", "3")) + .build(); + + BytesStreamOutput out = new BytesStreamOutput(); + BufferedChecksumStreamOutput checksumOut = new BufferedChecksumStreamOutput(out); + metadata1.writeVerifiableTo(checksumOut); + + IndexMetadata metadata2 = IndexMetadata.builder(metadata1.getIndex().getName()) + .settings( + Settings.builder() + .put("index.number_of_replicas", numberOfReplicas) + .put("index.number_of_shards", 4) + .put("index.version.created", 1) + .build() + ) + .creationDate(metadata1.getCreationDate()) + .primaryTerm(1, 3) + .primaryTerm(0, 2) + .setRoutingNumShards(32) + .system(system) + .putCustom("my_custom2", customMap) + .putCustom("my_custom", customMap) + .putAlias(AliasMetadata.builder("alias-2").routing("routing-2").build()) + .putAlias(AliasMetadata.builder("alias-1").routing("routing-1").build()) + .putRolloverInfo(info2) + .putRolloverInfo(info1) + .putInSyncAllocationIds(0, Set.of("3", "1", "2")) + .build(); + + BytesStreamOutput out2 = new BytesStreamOutput(); + BufferedChecksumStreamOutput checksumOut2 = new BufferedChecksumStreamOutput(out2); + metadata2.writeVerifiableTo(checksumOut2); + assertEquals(checksumOut.getChecksum(), checksumOut2.getChecksum()); + } + public void testGetRoutingFactor() { Integer numShard = randomFrom(1, 2, 4, 8, 16); int routingFactor = IndexMetadata.getRoutingFactor(32, numShard); diff --git a/server/src/test/java/org/opensearch/cluster/metadata/IndexTemplateMetadataTests.java b/server/src/test/java/org/opensearch/cluster/metadata/IndexTemplateMetadataTests.java index 0ea2834cc3024..6b0d5bcd980d8 100644 --- a/server/src/test/java/org/opensearch/cluster/metadata/IndexTemplateMetadataTests.java +++ b/server/src/test/java/org/opensearch/cluster/metadata/IndexTemplateMetadataTests.java @@ -31,11 +31,14 @@ package org.opensearch.cluster.metadata; +import org.opensearch.common.io.stream.BytesStreamOutput; import org.opensearch.common.settings.Settings; import org.opensearch.common.xcontent.XContentHelper; import org.opensearch.common.xcontent.json.JsonXContent; import org.opensearch.core.common.bytes.BytesArray; import org.opensearch.core.common.bytes.BytesReference; +import org.opensearch.core.common.io.stream.BufferedChecksumStreamOutput; +import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.xcontent.DeprecationHandler; import org.opensearch.core.xcontent.MediaTypeRegistry; import org.opensearch.core.xcontent.NamedXContentRegistry; @@ -232,6 +235,50 @@ public void testFromToXContent() throws Exception { } } + public void testWriteVerifiableTo() throws Exception { + String templateName = randomUnicodeOfCodepointLengthBetween(1, 10); + IndexTemplateMetadata.Builder templateBuilder = IndexTemplateMetadata.builder(templateName); + templateBuilder.patterns(Arrays.asList("pattern-1", "pattern-2")); + int numAlias = between(2, 5); + for (int i = 0; i < numAlias; i++) { + AliasMetadata.Builder alias = AliasMetadata.builder(randomRealisticUnicodeOfLengthBetween(1, 100)); + alias.indexRouting(randomRealisticUnicodeOfLengthBetween(1, 100)); + alias.searchRouting(randomRealisticUnicodeOfLengthBetween(1, 100)); + templateBuilder.putAlias(alias); + } + templateBuilder.settings(Settings.builder().put("index.setting-1", randomLong())); + templateBuilder.settings(Settings.builder().put("index.setting-2", randomTimeValue())); + templateBuilder.order(randomInt()); + templateBuilder.version(between(0, 100)); + templateBuilder.putMapping("doc", "{\"doc\":{\"properties\":{\"type\":\"text\"}}}"); + + IndexTemplateMetadata template = templateBuilder.build(); + BytesStreamOutput out = new BytesStreamOutput(); + BufferedChecksumStreamOutput checksumOut = new BufferedChecksumStreamOutput(out); + template.writeVerifiableTo(checksumOut); + StreamInput in = out.bytes().streamInput(); + IndexTemplateMetadata result = IndexTemplateMetadata.readFrom(in); + assertEquals(result, template); + + IndexTemplateMetadata.Builder templateBuilder2 = IndexTemplateMetadata.builder(templateName); + templateBuilder2.patterns(Arrays.asList("pattern-2", "pattern-1")); + template.getAliases() + .entrySet() + .stream() + .sorted(Map.Entry.comparingByKey()) + .forEachOrdered(entry -> templateBuilder2.putAlias(entry.getValue())); + templateBuilder2.settings(template.settings()); + templateBuilder2.order(template.order()); + templateBuilder2.version(template.version()); + templateBuilder2.putMapping("doc", template.mappings()); + + IndexTemplateMetadata template2 = templateBuilder.build(); + BytesStreamOutput out2 = new BytesStreamOutput(); + BufferedChecksumStreamOutput checksumOut2 = new BufferedChecksumStreamOutput(out2); + template2.writeVerifiableTo(checksumOut2); + assertEquals(checksumOut.getChecksum(), checksumOut2.getChecksum()); + } + public void testDeprecationWarningsOnMultipleMappings() throws IOException { IndexTemplateMetadata.Builder builder = IndexTemplateMetadata.builder("my-template"); builder.patterns(Arrays.asList("a", "b")); diff --git a/server/src/test/java/org/opensearch/cluster/metadata/MetadataIndexTemplateServiceTests.java b/server/src/test/java/org/opensearch/cluster/metadata/MetadataIndexTemplateServiceTests.java index 8312267205d62..99e259c8170f3 100644 --- a/server/src/test/java/org/opensearch/cluster/metadata/MetadataIndexTemplateServiceTests.java +++ b/server/src/test/java/org/opensearch/cluster/metadata/MetadataIndexTemplateServiceTests.java @@ -59,6 +59,8 @@ import org.opensearch.env.Environment; import org.opensearch.index.IndexSettings; import org.opensearch.index.codec.CodecService; +import org.opensearch.index.compositeindex.CompositeIndexSettings; +import org.opensearch.index.compositeindex.datacube.startree.StarTreeIndexSettings; import org.opensearch.index.engine.EngineConfig; import org.opensearch.index.mapper.MapperParsingException; import org.opensearch.index.mapper.MapperService; @@ -2425,6 +2427,19 @@ public void testAsyncTranslogDurabilityBlocked() { assertThat(throwables.get(0), instanceOf(IllegalArgumentException.class)); } + public void testMaxTranslogFlushSizeWithCompositeIndex() { + Settings clusterSettings = Settings.builder() + .put(CompositeIndexSettings.COMPOSITE_INDEX_MAX_TRANSLOG_FLUSH_THRESHOLD_SIZE_SETTING.getKey(), "130m") + .build(); + PutRequest request = new PutRequest("test", "test_replicas"); + request.patterns(singletonList("test_shards_wait*")); + Settings.Builder settingsBuilder = builder().put(StarTreeIndexSettings.IS_COMPOSITE_INDEX_SETTING.getKey(), "true") + .put(IndexSettings.INDEX_TRANSLOG_FLUSH_THRESHOLD_SIZE_SETTING.getKey(), "131m"); + request.settings(settingsBuilder.build()); + List throwables = putTemplate(xContentRegistry(), request, clusterSettings); + assertThat(throwables.get(0), instanceOf(IllegalArgumentException.class)); + } + private static List putTemplate(NamedXContentRegistry xContentRegistry, PutRequest request) { return putTemplate(xContentRegistry, request, Settings.EMPTY); } diff --git a/server/src/test/java/org/opensearch/cluster/metadata/MetadataTests.java b/server/src/test/java/org/opensearch/cluster/metadata/MetadataTests.java index a434a713f330b..c6cde72ab60b1 100644 --- a/server/src/test/java/org/opensearch/cluster/metadata/MetadataTests.java +++ b/server/src/test/java/org/opensearch/cluster/metadata/MetadataTests.java @@ -1536,6 +1536,41 @@ public static Metadata randomMetadata() { return md.build(); } + public void testXContentWithTemplateMetadata() throws IOException { + final TemplatesMetadata templatesMetadata = getTemplatesMetadata(0); + verifyTemplatesMetadata(templatesMetadata); + final TemplatesMetadata templatesMetadata2 = getTemplatesMetadata(2); + verifyTemplatesMetadata(templatesMetadata2); + } + + private void verifyTemplatesMetadata(TemplatesMetadata templatesMetadata) throws IOException { + final Metadata originalMeta = Metadata.builder().templates(templatesMetadata).build(); + final XContentBuilder builder = JsonXContent.contentBuilder(); + builder.startObject(); + Metadata.FORMAT.toXContent(builder, originalMeta); + builder.endObject(); + try (XContentParser parser = createParser(JsonXContent.jsonXContent, BytesReference.bytes(builder))) { + final Metadata fromXContentMeta = Metadata.fromXContent(parser); + assertThat(fromXContentMeta.templates(), equalTo(originalMeta.templates())); + } + } + + private static TemplatesMetadata getTemplatesMetadata(int numberOfTemplates) { + TemplatesMetadata.Builder builder = TemplatesMetadata.builder(); + for (int i = 0; i < numberOfTemplates; i++) { + builder.put( + IndexTemplateMetadata.builder("template" + i) + .order(1234) + .patterns(Arrays.asList(randomAlphaOfLength(3) + "-*")) + .settings( + Settings.builder().put("index.random_index_setting_" + randomAlphaOfLength(3), randomAlphaOfLength(5)).build() + ) + .build() + ); + } + return builder.build(); + } + private static CreateIndexResult createIndices(int numIndices, int numBackingIndices, String dataStreamName) { // create some indices that do not back a data stream final List indices = new ArrayList<>(); diff --git a/server/src/test/java/org/opensearch/cluster/metadata/TranslogFlushIntervalSettingsTests.java b/server/src/test/java/org/opensearch/cluster/metadata/TranslogFlushIntervalSettingsTests.java new file mode 100644 index 0000000000000..f54b6cdf1b152 --- /dev/null +++ b/server/src/test/java/org/opensearch/cluster/metadata/TranslogFlushIntervalSettingsTests.java @@ -0,0 +1,146 @@ +/* + * 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.metadata; + +import org.opensearch.common.settings.ClusterSettings; +import org.opensearch.common.settings.Settings; +import org.opensearch.index.IndexSettings; +import org.opensearch.index.compositeindex.CompositeIndexSettings; +import org.opensearch.index.compositeindex.datacube.startree.StarTreeIndexSettings; +import org.opensearch.test.OpenSearchTestCase; + +import java.util.Optional; + +/** + * Tests for translog flush interval settings update with and without composite index + */ +public class TranslogFlushIntervalSettingsTests extends OpenSearchTestCase { + + Settings settings = Settings.builder() + .put(CompositeIndexSettings.COMPOSITE_INDEX_MAX_TRANSLOG_FLUSH_THRESHOLD_SIZE_SETTING.getKey(), "130mb") + .build(); + ClusterSettings clusterSettings = new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); + + public void testValidSettings() { + Settings requestSettings = Settings.builder() + .put(IndexSettings.INDEX_TRANSLOG_FLUSH_THRESHOLD_SIZE_SETTING.getKey(), "50mb") + .put(StarTreeIndexSettings.IS_COMPOSITE_INDEX_SETTING.getKey(), true) + .build(); + + // This should not throw an exception + MetadataCreateIndexService.validateTranslogFlushIntervalSettingsForCompositeIndex(requestSettings, clusterSettings); + } + + public void testDefaultTranslogFlushSetting() { + Settings requestSettings = Settings.builder().put(StarTreeIndexSettings.IS_COMPOSITE_INDEX_SETTING.getKey(), true).build(); + + // This should not throw an exception + IllegalArgumentException ex = expectThrows( + IllegalArgumentException.class, + () -> MetadataCreateIndexService.validateTranslogFlushIntervalSettingsForCompositeIndex(requestSettings, clusterSettings) + ); + assertEquals("You can configure 'index.translog.flush_threshold_size' with upto '130mb' for composite index", ex.getMessage()); + } + + public void testMissingCompositeIndexSetting() { + Settings requestSettings = Settings.builder() + .put(IndexSettings.INDEX_TRANSLOG_FLUSH_THRESHOLD_SIZE_SETTING.getKey(), "50mb") + .build(); + + // This should not throw an exception + MetadataCreateIndexService.validateTranslogFlushIntervalSettingsForCompositeIndex(requestSettings, clusterSettings); + } + + public void testNullTranslogFlushSetting() { + Settings requestSettings = Settings.builder() + .putNull(IndexSettings.INDEX_TRANSLOG_FLUSH_THRESHOLD_SIZE_SETTING.getKey()) + .put(StarTreeIndexSettings.IS_COMPOSITE_INDEX_SETTING.getKey(), true) + .build(); + + // This should not throw an exception + IllegalArgumentException ex = expectThrows( + IllegalArgumentException.class, + () -> MetadataCreateIndexService.validateTranslogFlushIntervalSettingsForCompositeIndex(requestSettings, clusterSettings) + ); + assertEquals("You can configure 'index.translog.flush_threshold_size' with upto '130mb' for composite index", ex.getMessage()); + } + + public void testExceedingMaxFlushSize() { + Settings requestSettings = Settings.builder() + .put(IndexSettings.INDEX_TRANSLOG_FLUSH_THRESHOLD_SIZE_SETTING.getKey(), "150mb") + .put(StarTreeIndexSettings.IS_COMPOSITE_INDEX_SETTING.getKey(), true) + .build(); + + IllegalArgumentException ex = expectThrows( + IllegalArgumentException.class, + () -> MetadataCreateIndexService.validateTranslogFlushIntervalSettingsForCompositeIndex(requestSettings, clusterSettings) + ); + assertEquals("You can configure 'index.translog.flush_threshold_size' with upto '130mb' for composite index", ex.getMessage()); + } + + public void testEqualToMaxFlushSize() { + Settings requestSettings = Settings.builder() + .put(IndexSettings.INDEX_TRANSLOG_FLUSH_THRESHOLD_SIZE_SETTING.getKey(), "100mb") + .put(StarTreeIndexSettings.IS_COMPOSITE_INDEX_SETTING.getKey(), true) + .build(); + + // This should not throw an exception + MetadataCreateIndexService.validateTranslogFlushIntervalSettingsForCompositeIndex(requestSettings, clusterSettings); + } + + public void testUpdateIndexThresholdFlushSize() { + Settings requestSettings = Settings.builder() + .put(IndexSettings.INDEX_TRANSLOG_FLUSH_THRESHOLD_SIZE_SETTING.getKey(), "100mb") + .build(); + + Settings indexSettings = Settings.builder().put(StarTreeIndexSettings.IS_COMPOSITE_INDEX_SETTING.getKey(), true).build(); + + // This should not throw an exception + assertTrue( + MetadataCreateIndexService.validateTranslogFlushIntervalSettingsForCompositeIndex( + requestSettings, + clusterSettings, + indexSettings + ).isEmpty() + ); + } + + public void testUpdateFlushSizeAboveThresholdWithCompositeIndex() { + Settings requestSettings = Settings.builder() + .put(IndexSettings.INDEX_TRANSLOG_FLUSH_THRESHOLD_SIZE_SETTING.getKey(), "131mb") + .build(); + + Settings indexSettings = Settings.builder().put(StarTreeIndexSettings.IS_COMPOSITE_INDEX_SETTING.getKey(), true).build(); + + Optional err = MetadataCreateIndexService.validateTranslogFlushIntervalSettingsForCompositeIndex( + requestSettings, + clusterSettings, + indexSettings + ); + assertTrue(err.isPresent()); + assertEquals("You can configure 'index.translog.flush_threshold_size' with upto '130mb' for composite index", err.get()); + } + + public void testUpdateFlushSizeAboveThresholdWithoutCompositeIndex() { + Settings requestSettings = Settings.builder() + .put(IndexSettings.INDEX_TRANSLOG_FLUSH_THRESHOLD_SIZE_SETTING.getKey(), "131mb") + .build(); + + Settings indexSettings = Settings.builder().build(); + + // This should not throw an exception + assertTrue( + MetadataCreateIndexService.validateTranslogFlushIntervalSettingsForCompositeIndex( + requestSettings, + clusterSettings, + indexSettings + ).isEmpty() + ); + } +} 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..525a53f3e6158 100644 --- a/server/src/test/java/org/opensearch/cluster/node/DiscoveryNodeTests.java +++ b/server/src/test/java/org/opensearch/cluster/node/DiscoveryNodeTests.java @@ -36,6 +36,7 @@ import org.opensearch.common.io.stream.BytesStreamOutput; import org.opensearch.common.settings.Setting; import org.opensearch.common.settings.Settings; +import org.opensearch.core.common.io.stream.BufferedChecksumStreamOutput; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.transport.TransportAddress; import org.opensearch.node.remotestore.RemoteStoreNodeAttribute; @@ -46,6 +47,7 @@ import java.util.Collections; import java.util.HashMap; import java.util.HashSet; +import java.util.LinkedHashSet; import java.util.Locale; import java.util.Map; import java.util.Set; @@ -128,6 +130,43 @@ public void testDiscoveryNodeSerializationKeepsHost() throws Exception { assertEquals(transportAddress.getPort(), serialized.getAddress().getPort()); } + public void testWriteVerifiableTo() throws Exception { + InetAddress inetAddress = InetAddress.getByAddress("name1", new byte[] { (byte) 192, (byte) 168, (byte) 0, (byte) 1 }); + TransportAddress transportAddress = new TransportAddress(inetAddress, randomIntBetween(0, 65535)); + final Set roles = new HashSet<>(randomSubsetOf(DiscoveryNodeRole.BUILT_IN_ROLES)); + Map attributes = new HashMap<>(); + attributes.put("att-1", "test-repo"); + attributes.put("att-2", "test-repo"); + DiscoveryNode node = new DiscoveryNode("name1", "id1", transportAddress, attributes, roles, Version.CURRENT); + + BytesStreamOutput out = new BytesStreamOutput(); + BufferedChecksumStreamOutput checksumOut = new BufferedChecksumStreamOutput(out); + node.writeVerifiableTo(checksumOut); + StreamInput in = out.bytes().streamInput(); + DiscoveryNode result = new DiscoveryNode(in); + assertEquals(result, node); + + Map attributes2 = new HashMap<>(); + attributes2.put("att-2", "test-repo"); + attributes2.put("att-1", "test-repo"); + + DiscoveryNode node2 = new DiscoveryNode( + node.getName(), + node.getId(), + node.getEphemeralId(), + node.getHostName(), + node.getHostAddress(), + transportAddress, + attributes2, + roles.stream().sorted().collect(Collectors.toCollection(LinkedHashSet::new)), + Version.CURRENT + ); + BytesStreamOutput out2 = new BytesStreamOutput(); + BufferedChecksumStreamOutput checksumOut2 = new BufferedChecksumStreamOutput(out2); + node2.writeVerifiableTo(checksumOut2); + assertEquals(checksumOut.getChecksum(), checksumOut2.getChecksum()); + } + public void testDiscoveryNodeRoleWithOldVersion() throws Exception { InetAddress inetAddress = InetAddress.getByAddress("name1", new byte[] { (byte) 192, (byte) 168, (byte) 0, (byte) 1 }); TransportAddress transportAddress = new TransportAddress(inetAddress, randomIntBetween(0, 65535)); 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..61b86856c9ebc 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,14 @@ import org.opensearch.LegacyESVersion; import org.opensearch.Version; +import org.opensearch.common.io.stream.BytesStreamOutput; import org.opensearch.common.settings.Setting; +import org.opensearch.core.common.io.stream.BufferedChecksumStreamOutput; +import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.transport.TransportAddress; import org.opensearch.test.OpenSearchTestCase; +import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; @@ -328,6 +332,28 @@ public void testDeprecatedMasterNodeFilter() { assertThat(discoveryNodes.resolveNodes("master:false", "_all"), arrayContainingInAnyOrder(allNodes)); } + public void testWriteVerifiableTo() throws IOException { + final DiscoveryNodes discoveryNodes = buildDiscoveryNodes(); + BytesStreamOutput out = new BytesStreamOutput(); + BufferedChecksumStreamOutput checksumOut = new BufferedChecksumStreamOutput(out); + discoveryNodes.writeVerifiableTo(checksumOut); + StreamInput in = out.bytes().streamInput(); + DiscoveryNodes result = DiscoveryNodes.readFrom(in, discoveryNodes.getLocalNode()); + assertEquals(result, discoveryNodes); + + final DiscoveryNodes.Builder discoveryNodesBuilder = DiscoveryNodes.builder() + .clusterManagerNodeId(discoveryNodes.getClusterManagerNodeId()); + discoveryNodes.getNodes() + .entrySet() + .stream() + .sorted(Map.Entry.comparingByKey()) + .forEachOrdered(entry -> discoveryNodesBuilder.add(entry.getValue())); + BytesStreamOutput out2 = new BytesStreamOutput(); + BufferedChecksumStreamOutput checksumOut2 = new BufferedChecksumStreamOutput(out2); + discoveryNodesBuilder.build().writeVerifiableTo(checksumOut2); + assertEquals(checksumOut.getChecksum(), checksumOut2.getChecksum()); + } + private static AtomicInteger idGenerator = new AtomicInteger(); private static List randomNodes(final int numNodes) { diff --git a/server/src/test/java/org/opensearch/cluster/routing/IndexShardRoutingTableTests.java b/server/src/test/java/org/opensearch/cluster/routing/IndexShardRoutingTableTests.java index 6bfe60980adf3..5f8dabdcd4e45 100644 --- a/server/src/test/java/org/opensearch/cluster/routing/IndexShardRoutingTableTests.java +++ b/server/src/test/java/org/opensearch/cluster/routing/IndexShardRoutingTableTests.java @@ -32,10 +32,13 @@ package org.opensearch.cluster.routing; +import org.opensearch.common.io.stream.BytesStreamOutput; +import org.opensearch.core.common.io.stream.BufferedChecksumStreamOutput; import org.opensearch.core.index.Index; import org.opensearch.core.index.shard.ShardId; import org.opensearch.test.OpenSearchTestCase; +import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; import java.util.List; @@ -77,6 +80,25 @@ public void testEquals() { assertEquals(table4, table5); } + public void testWriteVerifiableTo() throws IOException { + Index index = new Index("a", "b"); + ShardId shardId = new ShardId(index, 1); + ShardRouting shard1 = TestShardRouting.newShardRouting(shardId, "node-1", true, ShardRoutingState.STARTED); + ShardRouting shard2 = TestShardRouting.newShardRouting(shardId, "node-2", false, ShardRoutingState.STARTED); + ShardRouting shard3 = TestShardRouting.newShardRouting(shardId, null, false, ShardRoutingState.UNASSIGNED); + + IndexShardRoutingTable table1 = new IndexShardRoutingTable(shardId, Arrays.asList(shard1, shard2, shard3)); + BytesStreamOutput out = new BytesStreamOutput(); + BufferedChecksumStreamOutput checksumOut = new BufferedChecksumStreamOutput(out); + IndexShardRoutingTable.Builder.writeVerifiableTo(table1, checksumOut); + + IndexShardRoutingTable table2 = new IndexShardRoutingTable(shardId, Arrays.asList(shard3, shard1, shard2)); + BytesStreamOutput out2 = new BytesStreamOutput(); + BufferedChecksumStreamOutput checksumOut2 = new BufferedChecksumStreamOutput(out2); + IndexShardRoutingTable.Builder.writeVerifiableTo(table2, checksumOut2); + assertEquals(checksumOut.getChecksum(), checksumOut2.getChecksum()); + } + public void testShardsMatchingPredicate() { ShardId shardId = new ShardId(new Index("a", UUID.randomUUID().toString()), 0); ShardRouting primary = TestShardRouting.newShardRouting(shardId, "node-1", true, ShardRoutingState.STARTED); diff --git a/server/src/test/java/org/opensearch/cluster/routing/RoutingTableTests.java b/server/src/test/java/org/opensearch/cluster/routing/RoutingTableTests.java index 97283f561d6d4..d8a67c09e442c 100644 --- a/server/src/test/java/org/opensearch/cluster/routing/RoutingTableTests.java +++ b/server/src/test/java/org/opensearch/cluster/routing/RoutingTableTests.java @@ -714,4 +714,5 @@ public static IndexMetadata updateActiveAllocations(IndexRoutingTable indexRouti } return imdBuilder.build(); } + } diff --git a/server/src/test/java/org/opensearch/cluster/service/MasterServiceTests.java b/server/src/test/java/org/opensearch/cluster/service/MasterServiceTests.java index 7562dfc2e9d33..db9abe0310e40 100644 --- a/server/src/test/java/org/opensearch/cluster/service/MasterServiceTests.java +++ b/server/src/test/java/org/opensearch/cluster/service/MasterServiceTests.java @@ -534,8 +534,8 @@ public void onFailure(String source, Exception e) { }); assertBusy(mockAppender::assertAllExpectationsMatched); // verify stats values after state is published - assertEquals(1, clusterManagerService.getClusterStateStats().getUpdateSuccess()); - assertEquals(0, clusterManagerService.getClusterStateStats().getUpdateFailed()); + assertBusy(() -> assertEquals(1, clusterManagerService.getClusterStateStats().getUpdateSuccess())); + assertBusy(() -> assertEquals(0, clusterManagerService.getClusterStateStats().getUpdateFailed())); } } } @@ -699,8 +699,8 @@ public void onFailure(String source, Exception e) { }); assertBusy(mockAppender::assertAllExpectationsMatched); // verify stats values after state is published - assertEquals(1, clusterManagerService.getClusterStateStats().getUpdateSuccess()); - assertEquals(0, clusterManagerService.getClusterStateStats().getUpdateFailed()); + assertBusy(() -> assertEquals(1, clusterManagerService.getClusterStateStats().getUpdateSuccess())); + assertBusy(() -> assertEquals(0, clusterManagerService.getClusterStateStats().getUpdateFailed())); } } } diff --git a/server/src/test/java/org/opensearch/gateway/GatewayMetaStatePersistedStateTests.java b/server/src/test/java/org/opensearch/gateway/GatewayMetaStatePersistedStateTests.java index 5ac94281822b8..9972bbfff5d66 100644 --- a/server/src/test/java/org/opensearch/gateway/GatewayMetaStatePersistedStateTests.java +++ b/server/src/test/java/org/opensearch/gateway/GatewayMetaStatePersistedStateTests.java @@ -73,6 +73,7 @@ import org.opensearch.index.recovery.RemoteStoreRestoreService; import org.opensearch.index.recovery.RemoteStoreRestoreService.RemoteRestoreResult; import org.opensearch.index.remote.RemoteIndexPathUploader; +import org.opensearch.indices.DefaultRemoteStoreSettings; import org.opensearch.node.Node; import org.opensearch.repositories.RepositoriesService; import org.opensearch.repositories.fs.FsRepository; @@ -504,7 +505,15 @@ public void testDataOnlyNodePersistence() throws Exception { clusterService, () -> 0L, threadPool, - List.of(new RemoteIndexPathUploader(threadPool, settings, repositoriesServiceSupplier, clusterSettings)), + List.of( + new RemoteIndexPathUploader( + threadPool, + settings, + repositoriesServiceSupplier, + clusterSettings, + DefaultRemoteStoreSettings.INSTANCE + ) + ), writableRegistry() ); } else { 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 6a191d002b971..3f9aa1245cab3 100644 --- a/server/src/test/java/org/opensearch/gateway/remote/ClusterMetadataManifestTests.java +++ b/server/src/test/java/org/opensearch/gateway/remote/ClusterMetadataManifestTests.java @@ -9,15 +9,24 @@ package org.opensearch.gateway.remote; import org.opensearch.Version; +import org.opensearch.cluster.ClusterName; import org.opensearch.cluster.ClusterState; +import org.opensearch.cluster.coordination.CoordinationMetadata; import org.opensearch.cluster.metadata.IndexGraveyard; +import org.opensearch.cluster.metadata.IndexMetadata; +import org.opensearch.cluster.metadata.IndexTemplateMetadata; +import org.opensearch.cluster.metadata.Metadata; import org.opensearch.cluster.metadata.RepositoriesMetadata; +import org.opensearch.cluster.metadata.TemplatesMetadata; import org.opensearch.cluster.metadata.WeightedRoutingMetadata; import org.opensearch.cluster.routing.IndexRoutingTable; +import org.opensearch.cluster.routing.RoutingTable; import org.opensearch.cluster.routing.StringKeyDiffProvider; +import org.opensearch.common.settings.Settings; import org.opensearch.common.xcontent.json.JsonXContent; import org.opensearch.core.common.bytes.BytesReference; import org.opensearch.core.common.io.stream.NamedWriteableRegistry; +import org.opensearch.core.index.Index; import org.opensearch.core.xcontent.ToXContent; import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.core.xcontent.XContentParser; @@ -31,6 +40,7 @@ import java.util.Arrays; import java.util.Collections; import java.util.List; +import java.util.Map; import java.util.function.Function; import java.util.stream.Collectors; @@ -164,7 +174,7 @@ public void testClusterMetadataManifestSerializationEqualsHashCode() { .opensearchVersion(Version.CURRENT) .nodeId("B10RX1f5RJenMQvYccCgSQ") .committed(true) - .codecVersion(CODEC_V3) + .codecVersion(ClusterMetadataManifest.CODEC_V4) .indices(randomUploadedIndexMetadataList()) .previousClusterUUID("yfObdx8KSMKKrXf8UyHhM") .clusterUUIDCommitted(true) @@ -204,6 +214,7 @@ public void testClusterMetadataManifestSerializationEqualsHashCode() { "indicesRoutingDiffPath" ) ) + .checksum(new ClusterStateChecksum(createClusterState())) .build(); { // Mutate Cluster Term EqualsHashCodeTestUtils.checkEqualsAndHashCode( @@ -485,6 +496,22 @@ public void testClusterMetadataManifestSerializationEqualsHashCode() { } ); } + { + // Mutate checksum + EqualsHashCodeTestUtils.checkEqualsAndHashCode( + initialManifest, + orig -> OpenSearchTestCase.copyWriteable( + orig, + new NamedWriteableRegistry(Collections.emptyList()), + ClusterMetadataManifest::new + ), + manifest -> { + ClusterMetadataManifest.Builder builder = ClusterMetadataManifest.builder(manifest); + builder.checksum(null); + return builder.build(); + } + ); + } } public void testClusterMetadataManifestXContentV2() throws IOException { @@ -616,15 +643,11 @@ public void testClusterMetadataManifestXContentV3() throws IOException { } } - public void testClusterMetadataManifestXContentV2WithoutEphemeral() throws IOException { + public void testClusterMetadataManifestXContentV4() throws IOException { UploadedIndexMetadata uploadedIndexMetadata = new UploadedIndexMetadata("test-index", "test-uuid", "/test/upload/path"); UploadedMetadataAttribute uploadedMetadataAttribute = new UploadedMetadataAttribute("attribute_name", "testing_attribute"); - UploadedIndexMetadata uploadedIndexRoutingMetadata = new UploadedIndexMetadata( - "test-index", - "test-uuid", - "routing-path", - INDEX_ROUTING_METADATA_PREFIX - ); + final StringKeyDiffProvider routingTableIncrementalDiff = Mockito.mock(StringKeyDiffProvider.class); + ClusterStateChecksum checksum = new ClusterStateChecksum(createClusterState()); ClusterMetadataManifest originalManifest = ClusterMetadataManifest.builder() .clusterTerm(1L) .stateVersion(1L) @@ -633,7 +656,7 @@ public void testClusterMetadataManifestXContentV2WithoutEphemeral() throws IOExc .opensearchVersion(Version.CURRENT) .nodeId("test-node-id") .committed(false) - .codecVersion(CODEC_V2) + .codecVersion(ClusterMetadataManifest.CODEC_V4) .indices(Collections.singletonList(uploadedIndexMetadata)) .previousClusterUUID("prev-cluster-uuid") .clusterUUIDCommitted(true) @@ -658,7 +681,23 @@ public void testClusterMetadataManifestXContentV2WithoutEphemeral() throws IOExc ) ).stream().collect(Collectors.toMap(UploadedMetadataAttribute::getAttributeName, Function.identity())) ) - .indicesRouting(Collections.singletonList(uploadedIndexRoutingMetadata)) + .routingTableVersion(1L) + .indicesRouting(Collections.singletonList(uploadedIndexMetadata)) + .discoveryNodesMetadata(uploadedMetadataAttribute) + .clusterBlocksMetadata(uploadedMetadataAttribute) + .transientSettingsMetadata(uploadedMetadataAttribute) + .hashesOfConsistentSettings(uploadedMetadataAttribute) + .clusterStateCustomMetadataMap(Collections.emptyMap()) + .diffManifest( + new ClusterStateDiffManifest( + RemoteClusterStateServiceTests.generateClusterStateWithOneIndex().build(), + ClusterState.EMPTY_STATE, + CODEC_V4, + routingTableIncrementalDiff, + uploadedMetadataAttribute.getUploadedFilename() + ) + ) + .checksum(checksum) .build(); final XContentBuilder builder = JsonXContent.contentBuilder(); builder.startObject(); @@ -671,10 +710,15 @@ public void testClusterMetadataManifestXContentV2WithoutEphemeral() throws IOExc } } - public void testClusterMetadataManifestXContentV4() throws IOException { + public void testClusterMetadataManifestXContentV2WithoutEphemeral() throws IOException { UploadedIndexMetadata uploadedIndexMetadata = new UploadedIndexMetadata("test-index", "test-uuid", "/test/upload/path"); UploadedMetadataAttribute uploadedMetadataAttribute = new UploadedMetadataAttribute("attribute_name", "testing_attribute"); - final StringKeyDiffProvider routingTableIncrementalDiff = Mockito.mock(StringKeyDiffProvider.class); + UploadedIndexMetadata uploadedIndexRoutingMetadata = new UploadedIndexMetadata( + "test-index", + "test-uuid", + "routing-path", + INDEX_ROUTING_METADATA_PREFIX + ); ClusterMetadataManifest originalManifest = ClusterMetadataManifest.builder() .clusterTerm(1L) .stateVersion(1L) @@ -683,7 +727,7 @@ public void testClusterMetadataManifestXContentV4() throws IOException { .opensearchVersion(Version.CURRENT) .nodeId("test-node-id") .committed(false) - .codecVersion(ClusterMetadataManifest.CODEC_V4) + .codecVersion(CODEC_V2) .indices(Collections.singletonList(uploadedIndexMetadata)) .previousClusterUUID("prev-cluster-uuid") .clusterUUIDCommitted(true) @@ -708,22 +752,7 @@ public void testClusterMetadataManifestXContentV4() throws IOException { ) ).stream().collect(Collectors.toMap(UploadedMetadataAttribute::getAttributeName, Function.identity())) ) - .routingTableVersion(1L) - .indicesRouting(Collections.singletonList(uploadedIndexMetadata)) - .discoveryNodesMetadata(uploadedMetadataAttribute) - .clusterBlocksMetadata(uploadedMetadataAttribute) - .transientSettingsMetadata(uploadedMetadataAttribute) - .hashesOfConsistentSettings(uploadedMetadataAttribute) - .clusterStateCustomMetadataMap(Collections.emptyMap()) - .diffManifest( - new ClusterStateDiffManifest( - RemoteClusterStateServiceTests.generateClusterStateWithOneIndex().build(), - ClusterState.EMPTY_STATE, - CODEC_V4, - routingTableIncrementalDiff, - uploadedMetadataAttribute.getUploadedFilename() - ) - ) + .indicesRouting(Collections.singletonList(uploadedIndexRoutingMetadata)) .build(); final XContentBuilder builder = JsonXContent.contentBuilder(); builder.startObject(); @@ -815,4 +844,40 @@ private UploadedIndexMetadata randomlyChangingUploadedIndexMetadata(UploadedInde return uploadedIndexMetadata; } + static ClusterState createClusterState() { + 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(); + final CoordinationMetadata coordinationMetadata = CoordinationMetadata.builder().term(1L).build(); + final Settings settings = Settings.builder().put("mock-settings", true).build(); + final TemplatesMetadata templatesMetadata = TemplatesMetadata.builder() + .put(IndexTemplateMetadata.builder("template1").settings(idxSettings).patterns(List.of("test*")).build()) + .build(); + final RemoteClusterStateTestUtils.CustomMetadata1 customMetadata = new RemoteClusterStateTestUtils.CustomMetadata1( + "custom-metadata-1" + ); + return ClusterState.builder(ClusterName.DEFAULT) + .version(1L) + .stateUUID("state-uuid") + .metadata( + Metadata.builder() + .version(randomNonNegativeLong()) + .put(indexMetadata, true) + .clusterUUID("cluster-uuid") + .coordinationMetadata(coordinationMetadata) + .persistentSettings(settings) + .templates(templatesMetadata) + .hashesOfConsistentSettings(Map.of("key1", "value1", "key2", "value2")) + .putCustom(customMetadata.getWriteableName(), customMetadata) + .build() + ) + .routingTable(RoutingTable.builder().addAsNew(indexMetadata).version(1L).build()) + .build(); + } } diff --git a/server/src/test/java/org/opensearch/gateway/remote/ClusterStateChecksumTests.java b/server/src/test/java/org/opensearch/gateway/remote/ClusterStateChecksumTests.java new file mode 100644 index 0000000000000..0203e56dd2d5c --- /dev/null +++ b/server/src/test/java/org/opensearch/gateway/remote/ClusterStateChecksumTests.java @@ -0,0 +1,222 @@ +/* + * 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.Version; +import org.opensearch.cluster.ClusterName; +import org.opensearch.cluster.ClusterState; +import org.opensearch.cluster.block.ClusterBlock; +import org.opensearch.cluster.block.ClusterBlockLevel; +import org.opensearch.cluster.block.ClusterBlocks; +import org.opensearch.cluster.coordination.CoordinationMetadata; +import org.opensearch.cluster.metadata.IndexMetadata; +import org.opensearch.cluster.metadata.IndexTemplateMetadata; +import org.opensearch.cluster.metadata.Metadata; +import org.opensearch.cluster.metadata.TemplatesMetadata; +import org.opensearch.cluster.node.DiscoveryNode; +import org.opensearch.cluster.node.DiscoveryNodes; +import org.opensearch.cluster.routing.RoutingTable; +import org.opensearch.common.io.stream.BytesStreamOutput; +import org.opensearch.common.settings.Settings; +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.common.transport.TransportAddress; +import org.opensearch.core.index.Index; +import org.opensearch.core.rest.RestStatus; +import org.opensearch.core.xcontent.ToXContent; +import org.opensearch.core.xcontent.XContentBuilder; +import org.opensearch.core.xcontent.XContentParser; +import org.opensearch.test.OpenSearchTestCase; + +import java.io.IOException; +import java.util.EnumSet; +import java.util.List; +import java.util.Map; + +public class ClusterStateChecksumTests extends OpenSearchTestCase { + + public void testClusterStateChecksumEmptyClusterState() { + ClusterStateChecksum checksum = new ClusterStateChecksum(ClusterState.EMPTY_STATE); + assertNotNull(checksum); + } + + public void testClusterStateChecksum() { + ClusterStateChecksum checksum = new ClusterStateChecksum(generateClusterState()); + assertNotNull(checksum); + assertTrue(checksum.routingTableChecksum != 0); + assertTrue(checksum.nodesChecksum != 0); + assertTrue(checksum.blocksChecksum != 0); + assertTrue(checksum.clusterStateCustomsChecksum != 0); + assertTrue(checksum.coordinationMetadataChecksum != 0); + assertTrue(checksum.settingMetadataChecksum != 0); + assertTrue(checksum.transientSettingsMetadataChecksum != 0); + assertTrue(checksum.templatesMetadataChecksum != 0); + assertTrue(checksum.customMetadataMapChecksum != 0); + assertTrue(checksum.hashesOfConsistentSettingsChecksum != 0); + assertTrue(checksum.indicesChecksum != 0); + assertTrue(checksum.clusterStateChecksum != 0); + } + + public void testClusterStateMatchChecksum() { + ClusterStateChecksum checksum = new ClusterStateChecksum(generateClusterState()); + ClusterStateChecksum newChecksum = new ClusterStateChecksum(generateClusterState()); + assertNotNull(checksum); + assertNotNull(newChecksum); + assertEquals(checksum.routingTableChecksum, newChecksum.routingTableChecksum); + assertEquals(checksum.nodesChecksum, newChecksum.nodesChecksum); + assertEquals(checksum.blocksChecksum, newChecksum.blocksChecksum); + assertEquals(checksum.clusterStateCustomsChecksum, newChecksum.clusterStateCustomsChecksum); + assertEquals(checksum.coordinationMetadataChecksum, newChecksum.coordinationMetadataChecksum); + assertEquals(checksum.settingMetadataChecksum, newChecksum.settingMetadataChecksum); + assertEquals(checksum.transientSettingsMetadataChecksum, newChecksum.transientSettingsMetadataChecksum); + assertEquals(checksum.templatesMetadataChecksum, newChecksum.templatesMetadataChecksum); + assertEquals(checksum.customMetadataMapChecksum, newChecksum.customMetadataMapChecksum); + assertEquals(checksum.hashesOfConsistentSettingsChecksum, newChecksum.hashesOfConsistentSettingsChecksum); + assertEquals(checksum.indicesChecksum, newChecksum.indicesChecksum); + assertEquals(checksum.clusterStateChecksum, newChecksum.clusterStateChecksum); + } + + public void testXContentConversion() throws IOException { + ClusterStateChecksum checksum = new ClusterStateChecksum(generateClusterState()); + final XContentBuilder builder = JsonXContent.contentBuilder(); + builder.startObject(); + checksum.toXContent(builder, ToXContent.EMPTY_PARAMS); + builder.endObject(); + + try (XContentParser parser = createParser(JsonXContent.jsonXContent, BytesReference.bytes(builder))) { + final ClusterStateChecksum parsedChecksum = ClusterStateChecksum.fromXContent(parser); + assertEquals(checksum, parsedChecksum); + } + } + + public void testSerialization() throws IOException { + ClusterStateChecksum checksum = new ClusterStateChecksum(generateClusterState()); + BytesStreamOutput output = new BytesStreamOutput(); + checksum.writeTo(output); + + try (StreamInput in = output.bytes().streamInput()) { + ClusterStateChecksum deserializedChecksum = new ClusterStateChecksum(in); + assertEquals(checksum, deserializedChecksum); + } + } + + public void testGetMismatchEntities() { + ClusterState clsState1 = generateClusterState(); + ClusterStateChecksum checksum = new ClusterStateChecksum(clsState1); + assertTrue(checksum.getMismatchEntities(checksum).isEmpty()); + + ClusterStateChecksum checksum2 = new ClusterStateChecksum(clsState1); + assertTrue(checksum.getMismatchEntities(checksum2).isEmpty()); + + ClusterState clsState2 = ClusterState.builder(ClusterName.DEFAULT) + .routingTable(RoutingTable.builder().build()) + .nodes(DiscoveryNodes.builder().build()) + .blocks(ClusterBlocks.builder().build()) + .customs(Map.of()) + .metadata(Metadata.EMPTY_METADATA) + .build(); + ClusterStateChecksum checksum3 = new ClusterStateChecksum(clsState2); + List mismatches = checksum.getMismatchEntities(checksum3); + assertFalse(mismatches.isEmpty()); + assertEquals(11, mismatches.size()); + assertEquals(ClusterStateChecksum.ROUTING_TABLE_CS, mismatches.get(0)); + assertEquals(ClusterStateChecksum.NODES_CS, mismatches.get(1)); + assertEquals(ClusterStateChecksum.BLOCKS_CS, mismatches.get(2)); + assertEquals(ClusterStateChecksum.CUSTOMS_CS, mismatches.get(3)); + assertEquals(ClusterStateChecksum.COORDINATION_MD_CS, mismatches.get(4)); + assertEquals(ClusterStateChecksum.SETTINGS_MD_CS, mismatches.get(5)); + assertEquals(ClusterStateChecksum.TRANSIENT_SETTINGS_MD_CS, mismatches.get(6)); + assertEquals(ClusterStateChecksum.TEMPLATES_MD_CS, mismatches.get(7)); + assertEquals(ClusterStateChecksum.CUSTOM_MD_CS, mismatches.get(8)); + assertEquals(ClusterStateChecksum.HASHES_MD_CS, mismatches.get(9)); + assertEquals(ClusterStateChecksum.INDICES_CS, mismatches.get(10)); + } + + public void testGetMismatchEntitiesUnorderedInput() { + ClusterState state1 = generateClusterState(); + DiscoveryNode node1 = DiscoveryNode.createLocal(Settings.EMPTY, new TransportAddress(TransportAddress.META_ADDRESS, 9200), "node1"); + DiscoveryNode node2 = DiscoveryNode.createLocal(Settings.EMPTY, new TransportAddress(TransportAddress.META_ADDRESS, 9201), "node2"); + DiscoveryNode node3 = DiscoveryNode.createLocal(Settings.EMPTY, new TransportAddress(TransportAddress.META_ADDRESS, 9202), "node3"); + + DiscoveryNodes nodes1 = DiscoveryNodes.builder().clusterManagerNodeId("test-node").add(node1).add(node2).add(node3).build(); + DiscoveryNodes nodes2 = DiscoveryNodes.builder().clusterManagerNodeId("test-node").add(node2).add(node3).build(); + nodes2 = nodes2.newNode(node1); + ClusterState state2 = ClusterState.builder(state1).nodes(nodes1).build(); + ClusterState state3 = ClusterState.builder(state1).nodes(nodes2).build(); + + ClusterStateChecksum checksum1 = new ClusterStateChecksum(state2); + ClusterStateChecksum checksum2 = new ClusterStateChecksum(state3); + assertEquals(checksum2, checksum1); + } + + private ClusterState generateClusterState() { + 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()) + .put(IndexMetadata.INDEX_READ_ONLY_SETTING.getKey(), true) + .build(); + final IndexMetadata indexMetadata = new IndexMetadata.Builder(index.getName()).settings(idxSettings) + .numberOfShards(1) + .numberOfReplicas(0) + .build(); + final Index index2 = new Index("test-index2", "index-uuid2"); + final Settings idxSettings2 = Settings.builder() + .put(IndexMetadata.SETTING_VERSION_CREATED, Version.CURRENT) + .put(IndexMetadata.SETTING_INDEX_UUID, index2.getUUID()) + .put(IndexMetadata.INDEX_READ_ONLY_SETTING.getKey(), true) + .build(); + final IndexMetadata indexMetadata2 = new IndexMetadata.Builder(index2.getName()).settings(idxSettings2) + .numberOfShards(3) + .numberOfReplicas(2) + .build(); + final CoordinationMetadata coordinationMetadata = CoordinationMetadata.builder().term(1L).build(); + final Settings settings = Settings.builder().put("mock-settings", true).build(); + final TemplatesMetadata templatesMetadata = TemplatesMetadata.builder() + .put(IndexTemplateMetadata.builder("template1").settings(idxSettings).patterns(List.of("test*")).build()) + .build(); + final RemoteClusterStateTestUtils.CustomMetadata1 customMetadata1 = new RemoteClusterStateTestUtils.CustomMetadata1( + "custom-metadata-1" + ); + RemoteClusterStateTestUtils.TestClusterStateCustom1 clusterStateCustom1 = new RemoteClusterStateTestUtils.TestClusterStateCustom1( + "custom-1" + ); + return ClusterState.builder(ClusterName.DEFAULT) + .version(1L) + .stateUUID("state-uuid") + .metadata( + Metadata.builder() + .version(1L) + .put(indexMetadata, true) + .clusterUUID("cluster-uuid") + .coordinationMetadata(coordinationMetadata) + .persistentSettings(settings) + .transientSettings(settings) + .templates(templatesMetadata) + .hashesOfConsistentSettings(Map.of("key1", "value1", "key2", "value2")) + .putCustom(customMetadata1.getWriteableName(), customMetadata1) + .indices(Map.of(indexMetadata.getIndex().getName(), indexMetadata, indexMetadata2.getIndex().getName(), indexMetadata2)) + .build() + ) + .nodes(DiscoveryNodes.builder().clusterManagerNodeId("test-node").build()) + .blocks( + ClusterBlocks.builder() + .addBlocks(indexMetadata) + .addGlobalBlock(new ClusterBlock(1, "block", true, true, true, RestStatus.ACCEPTED, EnumSet.of(ClusterBlockLevel.READ))) + .addGlobalBlock( + new ClusterBlock(2, "block-name", false, true, true, RestStatus.OK, EnumSet.of(ClusterBlockLevel.WRITE)) + ) + .build() + ) + .customs(Map.of(clusterStateCustom1.getWriteableName(), clusterStateCustom1)) + .routingTable(RoutingTable.builder().addAsNew(indexMetadata).addAsNew(indexMetadata2).version(1L).build()) + .build(); + } +} diff --git a/server/src/test/java/org/opensearch/gateway/remote/RemoteClusterStateAttributesManagerTests.java b/server/src/test/java/org/opensearch/gateway/remote/RemoteClusterStateAttributesManagerTests.java index 4ef459e6657a1..67b1528466a9e 100644 --- a/server/src/test/java/org/opensearch/gateway/remote/RemoteClusterStateAttributesManagerTests.java +++ b/server/src/test/java/org/opensearch/gateway/remote/RemoteClusterStateAttributesManagerTests.java @@ -140,7 +140,12 @@ public void testGetAsyncReadRunnable_DiscoveryNodes() throws IOException, Interr DiscoveryNodes discoveryNodes = getDiscoveryNodes(); String fileName = randomAlphaOfLength(10); when(blobStoreTransferService.downloadBlob(anyIterable(), anyString())).thenReturn( - DISCOVERY_NODES_FORMAT.serialize(discoveryNodes, fileName, compressor).streamInput() + DISCOVERY_NODES_FORMAT.serialize( + (out, discoveryNode) -> discoveryNode.writeToWithAttribute(out), + discoveryNodes, + fileName, + compressor + ).streamInput() ); RemoteDiscoveryNodes remoteObjForDownload = new RemoteDiscoveryNodes(fileName, "cluster-uuid", compressor); CountDownLatch latch = new CountDownLatch(1); 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 835087632d0d0..ccadbfb3fbebc 100644 --- a/server/src/test/java/org/opensearch/gateway/remote/RemoteClusterStateServiceTests.java +++ b/server/src/test/java/org/opensearch/gateway/remote/RemoteClusterStateServiceTests.java @@ -62,6 +62,7 @@ import org.opensearch.gateway.remote.model.RemoteReadResult; import org.opensearch.gateway.remote.model.RemoteTransientSettingsMetadata; import org.opensearch.index.remote.RemoteIndexPathUploader; +import org.opensearch.indices.DefaultRemoteStoreSettings; import org.opensearch.indices.IndicesModule; import org.opensearch.repositories.FilterRepository; import org.opensearch.repositories.RepositoriesService; @@ -165,6 +166,7 @@ import static org.mockito.ArgumentMatchers.eq; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.doNothing; +import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; @@ -252,7 +254,15 @@ public void setup() { clusterService, () -> 0L, threadPool, - List.of(new RemoteIndexPathUploader(threadPool, settings, repositoriesServiceSupplier, clusterSettings)), + List.of( + new RemoteIndexPathUploader( + threadPool, + settings, + repositoriesServiceSupplier, + clusterSettings, + DefaultRemoteStoreSettings.INSTANCE + ) + ), namedWriteableRegistry ); } @@ -289,7 +299,15 @@ public void testFailInitializationWhenRemoteStateDisabled() { clusterService, () -> 0L, threadPool, - List.of(new RemoteIndexPathUploader(threadPool, settings, repositoriesServiceSupplier, clusterSettings)), + List.of( + new RemoteIndexPathUploader( + threadPool, + settings, + repositoriesServiceSupplier, + clusterSettings, + DefaultRemoteStoreSettings.INSTANCE + ) + ), writableRegistry() ) ); @@ -361,7 +379,15 @@ public void testWriteFullMetadataSuccessPublicationEnabled() throws IOException clusterService, () -> 0L, threadPool, - List.of(new RemoteIndexPathUploader(threadPool, settings, repositoriesServiceSupplier, clusterSettings)), + List.of( + new RemoteIndexPathUploader( + threadPool, + settings, + repositoriesServiceSupplier, + clusterSettings, + DefaultRemoteStoreSettings.INSTANCE + ) + ), writableRegistry() ); final ClusterState clusterState = generateClusterStateWithOneIndex().nodes(nodesWithLocalNodeClusterManager()) @@ -731,7 +757,15 @@ public void testWriteIncrementalMetadataSuccessWhenPublicationEnabled() throws I clusterService, () -> 0L, threadPool, - List.of(new RemoteIndexPathUploader(threadPool, settings, repositoriesServiceSupplier, clusterSettings)), + List.of( + new RemoteIndexPathUploader( + threadPool, + settings, + repositoriesServiceSupplier, + clusterSettings, + DefaultRemoteStoreSettings.INSTANCE + ) + ), writableRegistry() ); final ClusterState clusterState = generateClusterStateWithOneIndex().nodes(nodesWithLocalNodeClusterManager()).build(); @@ -1186,7 +1220,12 @@ public void testGetClusterStateUsingDiff() throws IOException { diffManifestBuilder.discoveryNodesUpdated(true); manifestBuilder.discoveryNodesMetadata(new UploadedMetadataAttribute(DISCOVERY_NODES, DISCOVERY_NODES_FILENAME)); when(blobContainer.readBlob(DISCOVERY_NODES_FILENAME)).thenAnswer(invocationOnMock -> { - BytesReference bytes = DISCOVERY_NODES_FORMAT.serialize(nodesBuilder.build(), DISCOVERY_NODES_FILENAME, compressor); + BytesReference bytes = DISCOVERY_NODES_FORMAT.serialize( + (out, nodes) -> nodes.writeToWithAttribute(out), + nodesBuilder.build(), + DISCOVERY_NODES_FILENAME, + compressor + ); return new ByteArrayInputStream(bytes.streamInput().readAllBytes()); }); } @@ -2628,7 +2667,15 @@ public void testRemoteRoutingTableInitializedWhenEnabled() { clusterService, () -> 0L, threadPool, - List.of(new RemoteIndexPathUploader(threadPool, newSettings, repositoriesServiceSupplier, clusterSettings)), + List.of( + new RemoteIndexPathUploader( + threadPool, + newSettings, + repositoriesServiceSupplier, + clusterSettings, + DefaultRemoteStoreSettings.INSTANCE + ) + ), writableRegistry() ); assertTrue(remoteClusterStateService.getRemoteRoutingTableService() instanceof InternalRemoteRoutingTableService); @@ -2899,11 +2946,380 @@ private void initializeRoutingTable() { clusterService, () -> 0L, threadPool, - List.of(new RemoteIndexPathUploader(threadPool, newSettings, repositoriesServiceSupplier, clusterSettings)), + List.of( + new RemoteIndexPathUploader( + threadPool, + newSettings, + repositoriesServiceSupplier, + clusterSettings, + DefaultRemoteStoreSettings.INSTANCE + ) + ), + writableRegistry() + ); + } + + private void initializeWithChecksumEnabled() { + Settings newSettings = Settings.builder() + .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) + .put(RemoteClusterStateService.REMOTE_CLUSTER_STATE_CHECKSUM_VALIDATION_ENABLED_SETTING.getKey(), true) + .build(); + clusterSettings.applySettings(newSettings); + + Settings nodeSettings = Settings.builder().put(REMOTE_PUBLICATION_EXPERIMENTAL, "true").build(); + FeatureFlags.initializeFeatureFlags(nodeSettings); + remoteClusterStateService = new RemoteClusterStateService( + "test-node-id", + repositoriesServiceSupplier, + newSettings, + clusterService, + () -> 0L, + threadPool, + List.of( + new RemoteIndexPathUploader( + threadPool, + newSettings, + repositoriesServiceSupplier, + clusterSettings, + DefaultRemoteStoreSettings.INSTANCE + ) + ), writableRegistry() ); } + public void testWriteFullMetadataSuccessWithChecksumValidationEnabled() throws IOException { + initializeWithChecksumEnabled(); + mockBlobStoreObjects(); + when((blobStoreRepository.basePath())).thenReturn(BlobPath.cleanPath().add("base-path")); + + final ClusterState clusterState = generateClusterStateWithOneIndex().nodes(nodesWithLocalNodeClusterManager()).build(); + remoteClusterStateService.start(); + final ClusterMetadataManifest manifest = remoteClusterStateService.writeFullMetadata( + clusterState, + "prev-cluster-uuid", + MANIFEST_CURRENT_CODEC_VERSION + ).getClusterMetadataManifest(); + final UploadedIndexMetadata uploadedIndexMetadata = new UploadedIndexMetadata("test-index", "index-uuid", "metadata-filename"); + final UploadedIndexMetadata uploadedIndiceRoutingMetadata = new UploadedIndexMetadata( + "test-index", + "index-uuid", + "routing-filename", + INDEX_ROUTING_METADATA_PREFIX + ); + final ClusterMetadataManifest expectedManifest = ClusterMetadataManifest.builder() + .indices(List.of(uploadedIndexMetadata)) + .clusterTerm(1L) + .stateVersion(1L) + .stateUUID("state-uuid") + .clusterUUID("cluster-uuid") + .previousClusterUUID("prev-cluster-uuid") + .routingTableVersion(1L) + .indicesRouting(List.of(uploadedIndiceRoutingMetadata)) + .checksum(new ClusterStateChecksum(clusterState)) + .build(); + + assertThat(manifest.getIndices().size(), is(1)); + assertThat(manifest.getClusterTerm(), is(expectedManifest.getClusterTerm())); + assertThat(manifest.getStateVersion(), is(expectedManifest.getStateVersion())); + assertThat(manifest.getClusterUUID(), is(expectedManifest.getClusterUUID())); + assertThat(manifest.getStateUUID(), is(expectedManifest.getStateUUID())); + assertThat(manifest.getPreviousClusterUUID(), is(expectedManifest.getPreviousClusterUUID())); + assertThat(manifest.getRoutingTableVersion(), is(expectedManifest.getRoutingTableVersion())); + assertThat(manifest.getIndicesRouting().get(0).getIndexName(), is(uploadedIndiceRoutingMetadata.getIndexName())); + assertThat(manifest.getIndicesRouting().get(0).getIndexUUID(), is(uploadedIndiceRoutingMetadata.getIndexUUID())); + assertThat(manifest.getIndicesRouting().get(0).getUploadedFilename(), notNullValue()); + assertThat(manifest.getClusterStateChecksum(), is(expectedManifest.getClusterStateChecksum())); + } + + public void testWriteIncrementalMetadataSuccessWithChecksumValidationEnabled() throws IOException { + initializeWithChecksumEnabled(); + final ClusterState clusterState = generateClusterStateWithOneIndex().nodes(nodesWithLocalNodeClusterManager()).build(); + mockBlobStoreObjects(); + final CoordinationMetadata coordinationMetadata = CoordinationMetadata.builder().term(1L).build(); + final ClusterState previousClusterState = ClusterState.builder(ClusterName.DEFAULT) + .metadata(Metadata.builder().coordinationMetadata(coordinationMetadata)) + .build(); + + final ClusterMetadataManifest previousManifest = ClusterMetadataManifest.builder() + .indices(Collections.emptyList()) + .checksum(new ClusterStateChecksum(clusterState)) + .build(); + when((blobStoreRepository.basePath())).thenReturn(BlobPath.cleanPath().add("base-path")); + + remoteClusterStateService.start(); + final ClusterMetadataManifest manifest = remoteClusterStateService.writeIncrementalMetadata( + previousClusterState, + clusterState, + previousManifest + ).getClusterMetadataManifest(); + final UploadedIndexMetadata uploadedIndexMetadata = new UploadedIndexMetadata("test-index", "index-uuid", "metadata-filename"); + final UploadedIndexMetadata uploadedIndiceRoutingMetadata = new UploadedIndexMetadata( + "test-index", + "index-uuid", + "routing-filename", + INDEX_ROUTING_METADATA_PREFIX + ); + final ClusterMetadataManifest expectedManifest = ClusterMetadataManifest.builder() + .indices(List.of(uploadedIndexMetadata)) + .clusterTerm(1L) + .stateVersion(1L) + .stateUUID("state-uuid") + .clusterUUID("cluster-uuid") + .previousClusterUUID("prev-cluster-uuid") + .routingTableVersion(1) + .indicesRouting(List.of(uploadedIndiceRoutingMetadata)) + .checksum(new ClusterStateChecksum(clusterState)) + .build(); + + assertThat(manifest.getIndices().size(), is(1)); + assertThat(manifest.getClusterTerm(), is(expectedManifest.getClusterTerm())); + assertThat(manifest.getStateVersion(), is(expectedManifest.getStateVersion())); + assertThat(manifest.getClusterUUID(), is(expectedManifest.getClusterUUID())); + assertThat(manifest.getStateUUID(), is(expectedManifest.getStateUUID())); + assertThat(manifest.getRoutingTableVersion(), is(expectedManifest.getRoutingTableVersion())); + assertThat(manifest.getIndicesRouting().get(0).getIndexName(), is(uploadedIndiceRoutingMetadata.getIndexName())); + assertThat(manifest.getIndicesRouting().get(0).getIndexUUID(), is(uploadedIndiceRoutingMetadata.getIndexUUID())); + assertThat(manifest.getIndicesRouting().get(0).getUploadedFilename(), notNullValue()); + assertThat(manifest.getClusterStateChecksum(), is(expectedManifest.getClusterStateChecksum())); + } + + public void testGetClusterStateForManifestWithChecksumValidationEnabledWithNullChecksum() throws IOException { + initializeWithChecksumEnabled(); + ClusterMetadataManifest manifest = generateClusterMetadataManifestWithAllAttributes().build(); + mockBlobStoreObjects(); + remoteClusterStateService.start(); + RemoteReadResult mockedResult = mock(RemoteReadResult.class); + RemoteIndexMetadataManager mockedIndexManager = mock(RemoteIndexMetadataManager.class); + RemoteGlobalMetadataManager mockedGlobalMetadataManager = mock(RemoteGlobalMetadataManager.class); + RemoteClusterStateAttributesManager mockedClusterStateAttributeManager = mock(RemoteClusterStateAttributesManager.class); + remoteClusterStateService.setRemoteIndexMetadataManager(mockedIndexManager); + remoteClusterStateService.setRemoteGlobalMetadataManager(mockedGlobalMetadataManager); + remoteClusterStateService.setRemoteClusterStateAttributesManager(mockedClusterStateAttributeManager); + ArgumentCaptor> listenerArgumentCaptor = ArgumentCaptor.forClass( + LatchedActionListener.class + ); + doAnswer(invocation -> { + listenerArgumentCaptor.getValue().onResponse(mockedResult); + return null; + }).when(mockedIndexManager).readAsync(any(), any(), listenerArgumentCaptor.capture()); + doAnswer(invocation -> { + listenerArgumentCaptor.getValue().onResponse(mockedResult); + return null; + }).when(mockedGlobalMetadataManager).readAsync(any(), any(), listenerArgumentCaptor.capture()); + doAnswer(invocation -> { + listenerArgumentCaptor.getValue().onResponse(mockedResult); + return null; + }).when(mockedClusterStateAttributeManager).readAsync(anyString(), any(), listenerArgumentCaptor.capture()); + when(mockedResult.getComponent()).thenReturn(COORDINATION_METADATA); + RemoteClusterStateService mockService = spy(remoteClusterStateService); + mockService.getClusterStateForManifest(ClusterName.DEFAULT.value(), manifest, NODE_ID, true); + verify(mockService, times(1)).readClusterStateInParallel( + any(), + eq(manifest), + eq(manifest.getClusterUUID()), + eq(NODE_ID), + eq(manifest.getIndices()), + eq(manifest.getCustomMetadataMap()), + eq(true), + eq(true), + eq(true), + eq(true), + eq(true), + eq(true), + eq(manifest.getIndicesRouting()), + eq(true), + eq(manifest.getClusterStateCustomMap()), + eq(false), + eq(true) + ); + verify(mockService, times(0)).validateClusterStateFromChecksum( + any(ClusterMetadataManifest.class), + any(ClusterState.class), + anyString(), + anyString(), + anyBoolean() + ); + } + + public void testGetClusterStateForManifestWithChecksumValidationEnabled() throws IOException { + initializeWithChecksumEnabled(); + ClusterState clusterState = generateClusterStateWithAllAttributes().build(); + ClusterMetadataManifest manifest = generateClusterMetadataManifestWithAllAttributes().checksum( + new ClusterStateChecksum(clusterState) + ).build(); + remoteClusterStateService.start(); + RemoteClusterStateService mockService = spy(remoteClusterStateService); + doReturn(clusterState).when(mockService) + .readClusterStateInParallel( + any(), + eq(manifest), + eq(manifest.getClusterUUID()), + eq(NODE_ID), + eq(manifest.getIndices()), + eq(manifest.getCustomMetadataMap()), + eq(true), + eq(true), + eq(true), + eq(true), + eq(true), + eq(true), + eq(manifest.getIndicesRouting()), + eq(true), + eq(manifest.getClusterStateCustomMap()), + eq(false), + eq(true) + ); + mockService.getClusterStateForManifest(ClusterName.DEFAULT.value(), manifest, NODE_ID, true); + verify(mockService, times(1)).validateClusterStateFromChecksum(manifest, clusterState, ClusterName.DEFAULT.value(), NODE_ID, true); + } + + public void testGetClusterStateForManifestWithChecksumValidationEnabledWithMismatch() throws IOException { + initializeWithChecksumEnabled(); + ClusterState clusterState = generateClusterStateWithAllAttributes().build(); + ClusterMetadataManifest manifest = generateClusterMetadataManifestWithAllAttributes().checksum( + new ClusterStateChecksum(clusterState) + ).build(); + remoteClusterStateService.start(); + RemoteClusterStateService mockService = spy(remoteClusterStateService); + ClusterState clusterStateWrong = ClusterState.builder(clusterState).routingTable(RoutingTable.EMPTY_ROUTING_TABLE).build(); + doReturn(clusterStateWrong).when(mockService) + .readClusterStateInParallel( + any(), + eq(manifest), + eq(manifest.getClusterUUID()), + eq(NODE_ID), + eq(manifest.getIndices()), + eq(manifest.getCustomMetadataMap()), + eq(true), + eq(true), + eq(true), + eq(true), + eq(true), + eq(true), + eq(manifest.getIndicesRouting()), + eq(true), + eq(manifest.getClusterStateCustomMap()), + eq(false), + eq(true) + ); + expectThrows( + IllegalStateException.class, + () -> mockService.getClusterStateForManifest(ClusterName.DEFAULT.value(), manifest, NODE_ID, true) + ); + verify(mockService, times(1)).validateClusterStateFromChecksum( + manifest, + clusterStateWrong, + ClusterName.DEFAULT.value(), + NODE_ID, + true + ); + } + + public void testGetClusterStateUsingDiffWithChecksum() throws IOException { + initializeWithChecksumEnabled(); + ClusterState clusterState = generateClusterStateWithAllAttributes().build(); + ClusterMetadataManifest manifest = generateClusterMetadataManifestWithAllAttributes().checksum( + new ClusterStateChecksum(clusterState) + ).diffManifest(ClusterStateDiffManifest.builder().build()).build(); + + remoteClusterStateService.start(); + RemoteClusterStateService mockService = spy(remoteClusterStateService); + + doReturn(clusterState).when(mockService) + .readClusterStateInParallel( + any(), + eq(manifest), + eq(manifest.getClusterUUID()), + eq(NODE_ID), + eq(emptyList()), + eq(emptyMap()), + anyBoolean(), + anyBoolean(), + anyBoolean(), + anyBoolean(), + anyBoolean(), + anyBoolean(), + eq(emptyList()), + anyBoolean(), + eq(emptyMap()), + anyBoolean(), + anyBoolean() + ); + mockService.getClusterStateUsingDiff(manifest, clusterState, NODE_ID); + + verify(mockService, times(1)).validateClusterStateFromChecksum( + eq(manifest), + any(ClusterState.class), + eq(ClusterName.DEFAULT.value()), + eq(NODE_ID), + eq(false) + ); + } + + public void testGetClusterStateUsingDiffWithChecksumMismatch() throws IOException { + initializeWithChecksumEnabled(); + ClusterState clusterState = generateClusterStateWithAllAttributes().build(); + ClusterMetadataManifest manifest = generateClusterMetadataManifestWithAllAttributes().checksum( + new ClusterStateChecksum(clusterState) + ).diffManifest(ClusterStateDiffManifest.builder().build()).build(); + + remoteClusterStateService.start(); + RemoteClusterStateService mockService = spy(remoteClusterStateService); + ClusterState clusterStateWrong = ClusterState.builder(clusterState).routingTable(RoutingTable.EMPTY_ROUTING_TABLE).build(); + doReturn(clusterStateWrong).when(mockService) + .readClusterStateInParallel( + any(), + eq(manifest), + eq(manifest.getClusterUUID()), + eq(NODE_ID), + eq(emptyList()), + eq(emptyMap()), + anyBoolean(), + anyBoolean(), + anyBoolean(), + anyBoolean(), + anyBoolean(), + anyBoolean(), + eq(emptyList()), + anyBoolean(), + eq(emptyMap()), + anyBoolean(), + anyBoolean() + ); + doReturn(clusterState).when(mockService) + .readClusterStateInParallel( + any(), + eq(manifest), + eq(manifest.getClusterUUID()), + eq(NODE_ID), + eq(manifest.getIndices()), + eq(manifest.getCustomMetadataMap()), + eq(true), + eq(true), + eq(true), + eq(true), + eq(true), + eq(true), + eq(manifest.getIndicesRouting()), + eq(true), + eq(manifest.getClusterStateCustomMap()), + eq(false), + eq(true) + ); + + expectThrows(IllegalStateException.class, () -> mockService.getClusterStateUsingDiff(manifest, clusterState, NODE_ID)); + verify(mockService, times(1)).validateClusterStateFromChecksum( + eq(manifest), + any(ClusterState.class), + eq(ClusterName.DEFAULT.value()), + eq(NODE_ID), + eq(false) + ); + } + private void mockObjectsForGettingPreviousClusterUUID(Map clusterUUIDsPointers) throws IOException { mockObjectsForGettingPreviousClusterUUID(clusterUUIDsPointers, false, Collections.emptyMap()); } @@ -3453,6 +3869,8 @@ static ClusterMetadataManifest.Builder generateClusterMetadataManifestWithAllAtt return ClusterMetadataManifest.builder() .codecVersion(CODEC_V2) .clusterUUID("cluster-uuid") + .stateVersion(1L) + .stateUUID("state-uuid") .indices(List.of(new UploadedIndexMetadata("test-index", "test-index-uuid", "test-index-file__2"))) .customMetadataMap( Map.of( @@ -3478,7 +3896,8 @@ static ClusterMetadataManifest.Builder generateClusterMetadataManifestWithAllAtt "custom_2", new UploadedMetadataAttribute("custom_2", "test-cluster-state-custom2-file__1") ) - ); + ) + .routingTableVersion(1L); } public static DiscoveryNodes nodesWithLocalNodeClusterManager() { diff --git a/server/src/test/java/org/opensearch/gateway/remote/model/RemoteDiscoveryNodesTests.java b/server/src/test/java/org/opensearch/gateway/remote/model/RemoteDiscoveryNodesTests.java index f1bced2bdf855..1b988ee1f37ec 100644 --- a/server/src/test/java/org/opensearch/gateway/remote/model/RemoteDiscoveryNodesTests.java +++ b/server/src/test/java/org/opensearch/gateway/remote/model/RemoteDiscoveryNodesTests.java @@ -143,7 +143,7 @@ public void testSerDe() throws IOException { public void testExceptionDuringSerialization() throws IOException { DiscoveryNodes nodes = mock(DiscoveryNodes.class); RemoteDiscoveryNodes remoteObjectForUpload = new RemoteDiscoveryNodes(nodes, METADATA_VERSION, clusterUUID, compressor); - doThrow(new IOException("mock-exception")).when(nodes).writeTo(any()); + doThrow(new IOException("mock-exception")).when(nodes).writeToWithAttribute(any()); IOException iea = assertThrows(IOException.class, remoteObjectForUpload::serialize); } 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 index d7ecd2ad3f44a..ca978654c8825 100644 --- a/server/src/test/java/org/opensearch/gateway/remote/model/RemoteTemplatesMetadataTests.java +++ b/server/src/test/java/org/opensearch/gateway/remote/model/RemoteTemplatesMetadataTests.java @@ -231,12 +231,22 @@ public static TemplatesMetadata getTemplatesMetadata() { return TemplatesMetadata.builder() .put( IndexTemplateMetadata.builder("template" + randomAlphaOfLength(3)) + .order(1234) .patterns(Arrays.asList("bar-*", "foo-*")) .settings( Settings.builder().put("index.random_index_setting_" + randomAlphaOfLength(3), randomAlphaOfLength(5)).build() ) .build() ) + .put( + IndexTemplateMetadata.builder("template" + randomAlphaOfLength(3)) + .order(5678) + .patterns(Arrays.asList("test-*")) + .settings( + Settings.builder().put("index.random_index_setting_" + randomAlphaOfLength(3), randomAlphaOfLength(5)).build() + ) + .build() + ) .build(); } } diff --git a/server/src/test/java/org/opensearch/index/IndexModuleTests.java b/server/src/test/java/org/opensearch/index/IndexModuleTests.java index 4ce4936c047d9..8618d776bdf04 100644 --- a/server/src/test/java/org/opensearch/index/IndexModuleTests.java +++ b/server/src/test/java/org/opensearch/index/IndexModuleTests.java @@ -260,7 +260,7 @@ private IndexService newIndexService(IndexModule module) throws IOException { writableRegistry(), () -> false, null, - new RemoteSegmentStoreDirectoryFactory(() -> repositoriesService, threadPool), + new RemoteSegmentStoreDirectoryFactory(() -> repositoriesService, threadPool, ""), translogFactorySupplier, () -> IndexSettings.DEFAULT_REFRESH_INTERVAL, DefaultRecoverySettings.INSTANCE, diff --git a/server/src/test/java/org/opensearch/index/codec/composite99/datacube/startree/StarTreeDocValuesFormatTests.java b/server/src/test/java/org/opensearch/index/codec/composite99/datacube/startree/StarTreeDocValuesFormatTests.java index fa492e1adec0a..4bbefeba0845b 100644 --- a/server/src/test/java/org/opensearch/index/codec/composite99/datacube/startree/StarTreeDocValuesFormatTests.java +++ b/server/src/test/java/org/opensearch/index/codec/composite99/datacube/startree/StarTreeDocValuesFormatTests.java @@ -33,14 +33,18 @@ import org.opensearch.common.settings.Settings; import org.opensearch.common.util.FeatureFlags; import org.opensearch.common.xcontent.XContentFactory; +import org.opensearch.core.common.unit.ByteSizeUnit; +import org.opensearch.core.common.unit.ByteSizeValue; import org.opensearch.core.xcontent.NamedXContentRegistry; import org.opensearch.core.xcontent.XContentBuilder; +import org.opensearch.index.IndexSettings; import org.opensearch.index.MapperTestUtils; import org.opensearch.index.codec.composite.CompositeIndexFieldInfo; import org.opensearch.index.codec.composite.CompositeIndexReader; import org.opensearch.index.codec.composite.composite99.Composite99Codec; import org.opensearch.index.compositeindex.datacube.startree.StarTreeDocument; import org.opensearch.index.compositeindex.datacube.startree.StarTreeFieldConfiguration; +import org.opensearch.index.compositeindex.datacube.startree.StarTreeIndexSettings; import org.opensearch.index.compositeindex.datacube.startree.StarTreeTestUtils; import org.opensearch.index.compositeindex.datacube.startree.aggregators.numerictype.StarTreeNumericType; import org.opensearch.index.compositeindex.datacube.startree.index.StarTreeValues; @@ -213,20 +217,19 @@ private XContentBuilder topMapping(CheckedConsumer } private void createMapperService(XContentBuilder builder) throws IOException { - IndexMetadata indexMetadata = IndexMetadata.builder("test") - .settings( - Settings.builder() - .put(IndexMetadata.SETTING_VERSION_CREATED, Version.CURRENT) - .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) - .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1) - ) - .putMapping(builder.toString()) + Settings settings = Settings.builder() + .put(IndexMetadata.SETTING_VERSION_CREATED, Version.CURRENT) + .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1) + .put(StarTreeIndexSettings.IS_COMPOSITE_INDEX_SETTING.getKey(), true) + .put(IndexSettings.INDEX_TRANSLOG_FLUSH_THRESHOLD_SIZE_SETTING.getKey(), new ByteSizeValue(512, ByteSizeUnit.MB)) .build(); + IndexMetadata indexMetadata = IndexMetadata.builder("test").settings(settings).putMapping(builder.toString()).build(); IndicesModule indicesModule = new IndicesModule(Collections.emptyList()); mapperService = MapperTestUtils.newMapperServiceWithHelperAnalyzer( new NamedXContentRegistry(ClusterModule.getNamedXWriteables()), createTempDir(), - Settings.EMPTY, + settings, indicesModule, "test" ); diff --git a/server/src/test/java/org/opensearch/index/mapper/ObjectMapperTests.java b/server/src/test/java/org/opensearch/index/mapper/ObjectMapperTests.java index 504bc622ec12e..ff0533de4fe8d 100644 --- a/server/src/test/java/org/opensearch/index/mapper/ObjectMapperTests.java +++ b/server/src/test/java/org/opensearch/index/mapper/ObjectMapperTests.java @@ -37,7 +37,11 @@ import org.opensearch.common.util.FeatureFlags; import org.opensearch.common.xcontent.XContentFactory; import org.opensearch.core.common.bytes.BytesArray; +import org.opensearch.core.common.unit.ByteSizeUnit; +import org.opensearch.core.common.unit.ByteSizeValue; import org.opensearch.core.xcontent.MediaTypeRegistry; +import org.opensearch.index.IndexSettings; +import org.opensearch.index.compositeindex.datacube.startree.StarTreeIndexSettings; import org.opensearch.index.mapper.MapperService.MergeReason; import org.opensearch.index.mapper.ObjectMapper.Dynamic; import org.opensearch.plugins.Plugin; @@ -544,7 +548,11 @@ public void testCompositeFields() throws Exception { final Settings starTreeEnabledSettings = Settings.builder().put(STAR_TREE_INDEX, "true").build(); FeatureFlags.initializeFeatureFlags(starTreeEnabledSettings); - DocumentMapper documentMapper = createIndex("test").mapperService() + Settings settings = Settings.builder() + .put(StarTreeIndexSettings.IS_COMPOSITE_INDEX_SETTING.getKey(), true) + .put(IndexSettings.INDEX_TRANSLOG_FLUSH_THRESHOLD_SIZE_SETTING.getKey(), new ByteSizeValue(512, ByteSizeUnit.MB)) + .build(); + DocumentMapper documentMapper = createIndex("test", settings).mapperService() .documentMapperParser() .parse("tweet", new CompressedXContent(mapping)); diff --git a/server/src/test/java/org/opensearch/index/mapper/StarTreeMapperTests.java b/server/src/test/java/org/opensearch/index/mapper/StarTreeMapperTests.java index 81454b210d6be..e06d9889ec905 100644 --- a/server/src/test/java/org/opensearch/index/mapper/StarTreeMapperTests.java +++ b/server/src/test/java/org/opensearch/index/mapper/StarTreeMapperTests.java @@ -13,6 +13,8 @@ import org.opensearch.common.settings.ClusterSettings; import org.opensearch.common.settings.Settings; import org.opensearch.common.util.FeatureFlags; +import org.opensearch.core.common.unit.ByteSizeUnit; +import org.opensearch.core.common.unit.ByteSizeValue; import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.index.compositeindex.CompositeIndexSettings; import org.opensearch.index.compositeindex.CompositeIndexValidator; @@ -24,6 +26,7 @@ import org.opensearch.index.compositeindex.datacube.ReadDimension; import org.opensearch.index.compositeindex.datacube.startree.StarTreeField; import org.opensearch.index.compositeindex.datacube.startree.StarTreeFieldConfiguration; +import org.opensearch.index.compositeindex.datacube.startree.StarTreeIndexSettings; import org.junit.After; import org.junit.Before; @@ -35,6 +38,9 @@ import java.util.List; import java.util.Set; +import static org.opensearch.common.xcontent.XContentFactory.jsonBuilder; +import static org.opensearch.index.IndexSettings.INDEX_TRANSLOG_FLUSH_THRESHOLD_SIZE_SETTING; +import static org.opensearch.index.compositeindex.CompositeIndexSettings.COMPOSITE_INDEX_MAX_TRANSLOG_FLUSH_THRESHOLD_SIZE_SETTING; import static org.hamcrest.Matchers.containsString; /** @@ -52,7 +58,17 @@ public void teardown() { FeatureFlags.initializeFeatureFlags(Settings.EMPTY); } + @Override + protected Settings getIndexSettings() { + return Settings.builder() + .put(StarTreeIndexSettings.IS_COMPOSITE_INDEX_SETTING.getKey(), true) + .put(INDEX_TRANSLOG_FLUSH_THRESHOLD_SIZE_SETTING.getKey(), new ByteSizeValue(512, ByteSizeUnit.MB)) + .put(SETTINGS) + .build(); + } + public void testValidStarTree() throws IOException { + MapperService mapperService = createMapperService(getExpandedMappingWithJustAvg("status", "size")); Set compositeFieldTypes = mapperService.getCompositeFieldTypes(); for (CompositeMappedFieldType type : compositeFieldTypes) { @@ -82,6 +98,40 @@ public void testValidStarTree() throws IOException { } } + public void testCompositeIndexWithArraysInCompositeField() throws IOException { + DocumentMapper mapper = createDocumentMapper(getExpandedMappingWithJustAvg("status", "status")); + MapperParsingException ex = expectThrows( + MapperParsingException.class, + () -> mapper.parse(source(b -> b.startArray("status").value(0).value(1).endArray())) + ); + assertEquals( + "object mapping for [_doc] with array for [status] cannot be accepted as field is also part of composite index mapping which does not accept arrays", + ex.getMessage() + ); + ParsedDocument doc = mapper.parse(source(b -> b.startArray("size").value(0).value(1).endArray())); + // 1 intPoint , 1 SNDV field for each value , so 4 in total + assertEquals(4, doc.rootDoc().getFields("size").length); + } + + public void testValidValueForFlushTresholdSizeWithoutCompositeIndex() { + Settings settings = Settings.builder() + .put(INDEX_TRANSLOG_FLUSH_THRESHOLD_SIZE_SETTING.getKey(), "256mb") + .put(StarTreeIndexSettings.IS_COMPOSITE_INDEX_SETTING.getKey(), false) + .build(); + + assertEquals(new ByteSizeValue(256, ByteSizeUnit.MB), INDEX_TRANSLOG_FLUSH_THRESHOLD_SIZE_SETTING.get(settings)); + } + + public void testValidValueForCompositeIndex() { + Settings settings = Settings.builder() + .put(INDEX_TRANSLOG_FLUSH_THRESHOLD_SIZE_SETTING.getKey(), "256mb") + .put(StarTreeIndexSettings.IS_COMPOSITE_INDEX_SETTING.getKey(), true) + .put(COMPOSITE_INDEX_MAX_TRANSLOG_FLUSH_THRESHOLD_SIZE_SETTING.getKey(), "512mb") + .build(); + + assertEquals(new ByteSizeValue(256, ByteSizeUnit.MB), INDEX_TRANSLOG_FLUSH_THRESHOLD_SIZE_SETTING.get(settings)); + } + public void testMetricsWithJustSum() throws IOException { MapperService mapperService = createMapperService(getExpandedMappingWithJustSum("status", "size")); Set compositeFieldTypes = mapperService.getCompositeFieldTypes(); @@ -291,6 +341,24 @@ public void testInvalidSingleDim() { ); } + public void testDuplicateDimensions() { + XContentBuilder finalMapping = getMappingWithDuplicateFields(true, false); + MapperParsingException ex = expectThrows(MapperParsingException.class, () -> createMapperService(finalMapping)); + assertEquals( + "Failed to parse mapping [_doc]: Duplicate dimension [numeric_dv] present as part star tree index field [startree-1]", + ex.getMessage() + ); + } + + public void testDuplicateMetrics() { + XContentBuilder finalMapping = getMappingWithDuplicateFields(false, true); + MapperParsingException ex = expectThrows(MapperParsingException.class, () -> createMapperService(finalMapping)); + assertEquals( + "Failed to parse mapping [_doc]: Duplicate metrics [numeric_dv] present as part star tree index field [startree-1]", + ex.getMessage() + ); + } + public void testMetric() { List m1 = new ArrayList<>(); m1.add(MetricStat.MAX); @@ -507,6 +575,56 @@ private XContentBuilder getExpandedMappingWithJustAvg(String dim, String metric) }); } + private XContentBuilder getMappingWithDuplicateFields(boolean isDuplicateDim, boolean isDuplicateMetric) { + XContentBuilder mapping = null; + try { + mapping = jsonBuilder().startObject() + .startObject("composite") + .startObject("startree-1") + .field("type", "star_tree") + .startObject("config") + .startArray("ordered_dimensions") + .startObject() + .field("name", "timestamp") + .endObject() + .startObject() + .field("name", "numeric_dv") + .endObject() + .startObject() + .field("name", isDuplicateDim ? "numeric_dv" : "numeric_dv1") // Duplicate dimension + .endObject() + .endArray() + .startArray("metrics") + .startObject() + .field("name", "numeric_dv") + .endObject() + .startObject() + .field("name", isDuplicateMetric ? "numeric_dv" : "numeric_dv1") // Duplicate metric + .endObject() + .endArray() + .endObject() + .endObject() + .endObject() + .startObject("properties") + .startObject("timestamp") + .field("type", "date") + .endObject() + .startObject("numeric_dv") + .field("type", "integer") + .field("doc_values", true) + .endObject() + .startObject("numeric_dv1") + .field("type", "integer") + .field("doc_values", true) + .endObject() + .endObject() + .endObject(); + } catch (IOException e) { + fail("Failed to create mapping: " + e.getMessage()); + } + return mapping; + } + private XContentBuilder getExpandedMappingWithJustSum(String dim, String metric) throws IOException { return topMapping(b -> { b.startObject("composite"); diff --git a/server/src/test/java/org/opensearch/index/remote/RemoteIndexPathTests.java b/server/src/test/java/org/opensearch/index/remote/RemoteIndexPathTests.java index 8ddbd383756e7..2351d165b369f 100644 --- a/server/src/test/java/org/opensearch/index/remote/RemoteIndexPathTests.java +++ b/server/src/test/java/org/opensearch/index/remote/RemoteIndexPathTests.java @@ -9,12 +9,16 @@ package org.opensearch.index.remote; import org.opensearch.common.blobstore.BlobPath; +import org.opensearch.common.settings.ClusterSettings; +import org.opensearch.common.settings.Settings; 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.index.remote.RemoteStoreEnums.PathHashAlgorithm; import org.opensearch.index.remote.RemoteStoreEnums.PathType; +import org.opensearch.indices.DefaultRemoteStoreSettings; +import org.opensearch.indices.RemoteStoreSettings; import org.opensearch.test.OpenSearchTestCase; import java.io.IOException; @@ -40,7 +44,8 @@ public void testToXContentWithSegmentRepo() throws IOException { new BlobPath().add("djsd878ndjh").add("hcs87cj8"), PathType.HASHED_PREFIX, PathHashAlgorithm.FNV_1A_BASE64, - RemoteIndexPath.SEGMENT_PATH + RemoteIndexPath.SEGMENT_PATH, + DefaultRemoteStoreSettings.INSTANCE ); XContentBuilder xContentBuilder = MediaTypeRegistry.contentBuilder(MediaTypeRegistry.JSON); xContentBuilder.startObject(); @@ -49,6 +54,28 @@ public void testToXContentWithSegmentRepo() throws IOException { String expected = "{\"version\":\"1\",\"index_uuid\":\"djjsid73he8yd7usduh\",\"shard_count\":2,\"path_type\":\"HASHED_PREFIX\",\"path_hash_algorithm\":\"FNV_1A_BASE64\",\"path_creation_map\":{\"segments\":[\"data\",\"metadata\",\"lock_files\"]},\"paths\":[\"9BmBinD5HYs/djsd878ndjh/hcs87cj8/djjsid73he8yd7usduh/0/segments/data/\",\"ExCNOD8_5ew/djsd878ndjh/hcs87cj8/djjsid73he8yd7usduh/1/segments/data/\",\"z8wtf0yr2l4/djsd878ndjh/hcs87cj8/djjsid73he8yd7usduh/0/segments/metadata/\",\"VheHVwFlExE/djsd878ndjh/hcs87cj8/djjsid73he8yd7usduh/1/segments/metadata/\",\"IgFKbsDeUpQ/djsd878ndjh/hcs87cj8/djjsid73he8yd7usduh/0/segments/lock_files/\",\"pA3gy_GZtns/djsd878ndjh/hcs87cj8/djjsid73he8yd7usduh/1/segments/lock_files/\"]}"; assertEquals(expected, xContentBuilder.toString()); + + // Fixed prefix + Settings settings = Settings.builder().put(RemoteStoreSettings.CLUSTER_REMOTE_STORE_SEGMENTS_PATH_PREFIX.getKey(), ".").build(); + ClusterSettings clusterSettings = new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); + RemoteStoreSettings remoteStoreSettings = new RemoteStoreSettings(settings, clusterSettings); + indexPath = new RemoteIndexPath( + "djjsid73he8yd7usduh", + 2, + new BlobPath().add("djsd878ndjh").add("hcs87cj8"), + PathType.HASHED_PREFIX, + PathHashAlgorithm.FNV_1A_BASE64, + RemoteIndexPath.SEGMENT_PATH, + remoteStoreSettings + ); + xContentBuilder = MediaTypeRegistry.contentBuilder(MediaTypeRegistry.JSON); + xContentBuilder.startObject(); + xContentBuilder = indexPath.toXContent(xContentBuilder, ToXContent.EMPTY_PARAMS); + xContentBuilder.endObject(); + expected = + "{\"version\":\"1\",\"index_uuid\":\"djjsid73he8yd7usduh\",\"shard_count\":2,\"path_type\":\"HASHED_PREFIX\",\"path_hash_algorithm\":\"FNV_1A_BASE64\",\"path_creation_map\":{\"segments\":[\"data\",\"metadata\",\"lock_files\"]},\"paths\":[\".9BmBinD5HYs/djsd878ndjh/hcs87cj8/djjsid73he8yd7usduh/0/segments/data/\",\".ExCNOD8_5ew/djsd878ndjh/hcs87cj8/djjsid73he8yd7usduh/1/segments/data/\",\".z8wtf0yr2l4/djsd878ndjh/hcs87cj8/djjsid73he8yd7usduh/0/segments/metadata/\",\".VheHVwFlExE/djsd878ndjh/hcs87cj8/djjsid73he8yd7usduh/1/segments/metadata/\",\".IgFKbsDeUpQ/djsd878ndjh/hcs87cj8/djjsid73he8yd7usduh/0/segments/lock_files/\",\".pA3gy_GZtns/djsd878ndjh/hcs87cj8/djjsid73he8yd7usduh/1/segments/lock_files/\"]}"; + assertEquals(expected, xContentBuilder.toString()); + } /** @@ -61,7 +88,8 @@ public void testToXContentForTranslogRepoOnly() throws IOException { new BlobPath().add("djsd878ndjh").add("hcs87cj8"), PathType.HASHED_PREFIX, PathHashAlgorithm.FNV_1A_BASE64, - RemoteIndexPath.TRANSLOG_PATH + RemoteIndexPath.TRANSLOG_PATH, + DefaultRemoteStoreSettings.INSTANCE ); XContentBuilder xContentBuilder = MediaTypeRegistry.contentBuilder(MediaTypeRegistry.JSON); xContentBuilder.startObject(); @@ -70,6 +98,27 @@ public void testToXContentForTranslogRepoOnly() throws IOException { String expected = "{\"version\":\"1\",\"index_uuid\":\"djjsid73he8yd7usduh\",\"shard_count\":2,\"path_type\":\"HASHED_PREFIX\",\"path_hash_algorithm\":\"FNV_1A_BASE64\",\"path_creation_map\":{\"translog\":[\"data\",\"metadata\"]},\"paths\":[\"2EaVODaKBck/djsd878ndjh/hcs87cj8/djjsid73he8yd7usduh/0/translog/data/\",\"dTS2VqEOUNo/djsd878ndjh/hcs87cj8/djjsid73he8yd7usduh/1/translog/data/\",\"PVNKNGonmZw/djsd878ndjh/hcs87cj8/djjsid73he8yd7usduh/0/translog/metadata/\",\"NXmt0Y6NjA8/djsd878ndjh/hcs87cj8/djjsid73he8yd7usduh/1/translog/metadata/\"]}"; assertEquals(expected, xContentBuilder.toString()); + + // Fixed prefix + Settings settings = Settings.builder().put(RemoteStoreSettings.CLUSTER_REMOTE_STORE_TRANSLOG_PATH_PREFIX.getKey(), ".").build(); + ClusterSettings clusterSettings = new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); + RemoteStoreSettings remoteStoreSettings = new RemoteStoreSettings(settings, clusterSettings); + indexPath = new RemoteIndexPath( + "djjsid73he8yd7usduh", + 2, + new BlobPath().add("djsd878ndjh").add("hcs87cj8"), + PathType.HASHED_PREFIX, + PathHashAlgorithm.FNV_1A_BASE64, + RemoteIndexPath.TRANSLOG_PATH, + remoteStoreSettings + ); + xContentBuilder = MediaTypeRegistry.contentBuilder(MediaTypeRegistry.JSON); + xContentBuilder.startObject(); + xContentBuilder = indexPath.toXContent(xContentBuilder, ToXContent.EMPTY_PARAMS); + xContentBuilder.endObject(); + expected = + "{\"version\":\"1\",\"index_uuid\":\"djjsid73he8yd7usduh\",\"shard_count\":2,\"path_type\":\"HASHED_PREFIX\",\"path_hash_algorithm\":\"FNV_1A_BASE64\",\"path_creation_map\":{\"translog\":[\"data\",\"metadata\"]},\"paths\":[\".2EaVODaKBck/djsd878ndjh/hcs87cj8/djjsid73he8yd7usduh/0/translog/data/\",\".dTS2VqEOUNo/djsd878ndjh/hcs87cj8/djjsid73he8yd7usduh/1/translog/data/\",\".PVNKNGonmZw/djsd878ndjh/hcs87cj8/djjsid73he8yd7usduh/0/translog/metadata/\",\".NXmt0Y6NjA8/djsd878ndjh/hcs87cj8/djjsid73he8yd7usduh/1/translog/metadata/\"]}"; + assertEquals(expected, xContentBuilder.toString()); } /** @@ -85,7 +134,8 @@ public void testToXContentForBothRepos() throws IOException { new BlobPath().add("nxf9yv0").add("c3ejoi"), PathType.HASHED_PREFIX, PathHashAlgorithm.FNV_1A_BASE64, - pathCreationMap + pathCreationMap, + DefaultRemoteStoreSettings.INSTANCE ); XContentBuilder xContentBuilder = MediaTypeRegistry.contentBuilder(MediaTypeRegistry.JSON); xContentBuilder.startObject(); @@ -94,9 +144,42 @@ public void testToXContentForBothRepos() throws IOException { String expected = "{\"version\":\"1\",\"index_uuid\":\"csbdqiu8a7sdnjdks\",\"shard_count\":3,\"path_type\":\"HASHED_PREFIX\",\"path_hash_algorithm\":\"FNV_1A_BASE64\",\"path_creation_map\":{\"translog\":[\"data\",\"metadata\"],\"segments\":[\"data\",\"metadata\",\"lock_files\"]},\"paths\":[\"Cjo0F6kNjYk/nxf9yv0/c3ejoi/csbdqiu8a7sdnjdks/0/segments/data/\",\"kpayyhxct1I/nxf9yv0/c3ejoi/csbdqiu8a7sdnjdks/1/segments/data/\",\"p2RlgnHeIgc/nxf9yv0/c3ejoi/csbdqiu8a7sdnjdks/2/segments/data/\",\"gkPIurBtB1w/nxf9yv0/c3ejoi/csbdqiu8a7sdnjdks/0/segments/metadata/\",\"Y4YhlbxAB1c/nxf9yv0/c3ejoi/csbdqiu8a7sdnjdks/1/segments/metadata/\",\"HYc8fyVPouI/nxf9yv0/c3ejoi/csbdqiu8a7sdnjdks/2/segments/metadata/\",\"igzyZCz1ysI/nxf9yv0/c3ejoi/csbdqiu8a7sdnjdks/0/segments/lock_files/\",\"uEluEiYmptk/nxf9yv0/c3ejoi/csbdqiu8a7sdnjdks/1/segments/lock_files/\",\"TfAD8f06_7A/nxf9yv0/c3ejoi/csbdqiu8a7sdnjdks/2/segments/lock_files/\",\"QqKEpasbEGs/nxf9yv0/c3ejoi/csbdqiu8a7sdnjdks/0/translog/data/\",\"sNyoimoe1Bw/nxf9yv0/c3ejoi/csbdqiu8a7sdnjdks/1/translog/data/\",\"d4YQtONfq50/nxf9yv0/c3ejoi/csbdqiu8a7sdnjdks/2/translog/data/\",\"zLr4UXjK8T4/nxf9yv0/c3ejoi/csbdqiu8a7sdnjdks/0/translog/metadata/\",\"_s8i7ZmlXGE/nxf9yv0/c3ejoi/csbdqiu8a7sdnjdks/1/translog/metadata/\",\"tvtD3-k5ISg/nxf9yv0/c3ejoi/csbdqiu8a7sdnjdks/2/translog/metadata/\"]}"; assertEquals(expected, xContentBuilder.toString()); + + // Fixed prefix + Settings settings = Settings.builder() + .put(RemoteStoreSettings.CLUSTER_REMOTE_STORE_TRANSLOG_PATH_PREFIX.getKey(), ".") + .put(RemoteStoreSettings.CLUSTER_REMOTE_STORE_SEGMENTS_PATH_PREFIX.getKey(), ".") + .build(); + ClusterSettings clusterSettings = new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); + RemoteStoreSettings remoteStoreSettings = new RemoteStoreSettings(settings, clusterSettings); + indexPath = new RemoteIndexPath( + "csbdqiu8a7sdnjdks", + 3, + new BlobPath().add("nxf9yv0").add("c3ejoi"), + PathType.HASHED_PREFIX, + PathHashAlgorithm.FNV_1A_BASE64, + pathCreationMap, + remoteStoreSettings + ); + xContentBuilder = MediaTypeRegistry.contentBuilder(MediaTypeRegistry.JSON); + xContentBuilder.startObject(); + xContentBuilder = indexPath.toXContent(xContentBuilder, ToXContent.EMPTY_PARAMS); + xContentBuilder.endObject(); + expected = + "{\"version\":\"1\",\"index_uuid\":\"csbdqiu8a7sdnjdks\",\"shard_count\":3,\"path_type\":\"HASHED_PREFIX\",\"path_hash_algorithm\":\"FNV_1A_BASE64\",\"path_creation_map\":{\"translog\":[\"data\",\"metadata\"],\"segments\":[\"data\",\"metadata\",\"lock_files\"]},\"paths\":[\".Cjo0F6kNjYk/nxf9yv0/c3ejoi/csbdqiu8a7sdnjdks/0/segments/data/\",\".kpayyhxct1I/nxf9yv0/c3ejoi/csbdqiu8a7sdnjdks/1/segments/data/\",\".p2RlgnHeIgc/nxf9yv0/c3ejoi/csbdqiu8a7sdnjdks/2/segments/data/\",\".gkPIurBtB1w/nxf9yv0/c3ejoi/csbdqiu8a7sdnjdks/0/segments/metadata/\",\".Y4YhlbxAB1c/nxf9yv0/c3ejoi/csbdqiu8a7sdnjdks/1/segments/metadata/\",\".HYc8fyVPouI/nxf9yv0/c3ejoi/csbdqiu8a7sdnjdks/2/segments/metadata/\",\".igzyZCz1ysI/nxf9yv0/c3ejoi/csbdqiu8a7sdnjdks/0/segments/lock_files/\",\".uEluEiYmptk/nxf9yv0/c3ejoi/csbdqiu8a7sdnjdks/1/segments/lock_files/\",\".TfAD8f06_7A/nxf9yv0/c3ejoi/csbdqiu8a7sdnjdks/2/segments/lock_files/\",\".QqKEpasbEGs/nxf9yv0/c3ejoi/csbdqiu8a7sdnjdks/0/translog/data/\",\".sNyoimoe1Bw/nxf9yv0/c3ejoi/csbdqiu8a7sdnjdks/1/translog/data/\",\".d4YQtONfq50/nxf9yv0/c3ejoi/csbdqiu8a7sdnjdks/2/translog/data/\",\".zLr4UXjK8T4/nxf9yv0/c3ejoi/csbdqiu8a7sdnjdks/0/translog/metadata/\",\"._s8i7ZmlXGE/nxf9yv0/c3ejoi/csbdqiu8a7sdnjdks/1/translog/metadata/\",\".tvtD3-k5ISg/nxf9yv0/c3ejoi/csbdqiu8a7sdnjdks/2/translog/metadata/\"]}"; + assertEquals(expected, xContentBuilder.toString()); } - public void testRemoteIndexPathWithInvalidPathCreationMap() throws IOException { + public void testRemoteIndexPathWithInvalidPathCreationMap() { + Settings.Builder builder = Settings.builder(); + if (randomBoolean()) { + builder.put(RemoteStoreSettings.CLUSTER_REMOTE_STORE_TRANSLOG_PATH_PREFIX.getKey(), "."); + } + if (randomBoolean()) { + builder.put(RemoteStoreSettings.CLUSTER_REMOTE_STORE_SEGMENTS_PATH_PREFIX.getKey(), "."); + } + ClusterSettings clusterSettings = new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); + RemoteStoreSettings remoteStoreSettings = new RemoteStoreSettings(builder.build(), clusterSettings); IllegalArgumentException ex = assertThrows( IllegalArgumentException.class, () -> new RemoteIndexPath( @@ -105,7 +188,8 @@ public void testRemoteIndexPathWithInvalidPathCreationMap() throws IOException { new BlobPath().add("djsd878ndjh").add("hcs87cj8"), PathType.HASHED_PREFIX, PathHashAlgorithm.FNV_1A_BASE64, - new HashMap<>() + new HashMap<>(), + remoteStoreSettings ) ); assertEquals( @@ -124,6 +208,15 @@ public void testFromXContent() { } public void testInvalidPathCreationMap() { + Settings.Builder builder = Settings.builder(); + if (randomBoolean()) { + builder.put(RemoteStoreSettings.CLUSTER_REMOTE_STORE_TRANSLOG_PATH_PREFIX.getKey(), "."); + } + if (randomBoolean()) { + builder.put(RemoteStoreSettings.CLUSTER_REMOTE_STORE_SEGMENTS_PATH_PREFIX.getKey(), "."); + } + ClusterSettings clusterSettings = new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); + RemoteStoreSettings remoteStoreSettings = new RemoteStoreSettings(builder.build(), clusterSettings); IllegalArgumentException ex = assertThrows( IllegalArgumentException.class, () -> new RemoteIndexPath( @@ -132,7 +225,8 @@ public void testInvalidPathCreationMap() { new BlobPath().add("djsd878ndjh").add("hcs87cj8"), PathType.HASHED_PREFIX, PathHashAlgorithm.FNV_1A_BASE64, - Map.of(TRANSLOG, List.of(LOCK_FILES)) + Map.of(TRANSLOG, List.of(LOCK_FILES)), + remoteStoreSettings ) ); assertEquals("pathCreationMap={TRANSLOG=[LOCK_FILES]} is having illegal combination of category and type", ex.getMessage()); 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 e0a75f7296705..d6519d9db8ee6 100644 --- a/server/src/test/java/org/opensearch/index/remote/RemoteIndexPathUploaderTests.java +++ b/server/src/test/java/org/opensearch/index/remote/RemoteIndexPathUploaderTests.java @@ -25,6 +25,7 @@ import org.opensearch.gateway.remote.RemoteStateTransferException; import org.opensearch.index.remote.RemoteStoreEnums.PathHashAlgorithm; import org.opensearch.index.remote.RemoteStoreEnums.PathType; +import org.opensearch.indices.DefaultRemoteStoreSettings; import org.opensearch.node.Node; import org.opensearch.node.remotestore.RemoteStoreNodeAttribute; import org.opensearch.repositories.RepositoriesService; @@ -131,7 +132,8 @@ public void testInterceptWithNoRemoteDataAttributes() { threadPool, settings, () -> repositoriesService, - clusterSettings + clusterSettings, + DefaultRemoteStoreSettings.INSTANCE ); List indexMetadataList = Mockito.mock(List.class); ActionListener actionListener = ActionListener.wrap( @@ -149,7 +151,8 @@ public void testInterceptWithEmptyIndexMetadataList() { threadPool, settings, () -> repositoriesService, - clusterSettings + clusterSettings, + DefaultRemoteStoreSettings.INSTANCE ); remoteIndexPathUploader.start(); ActionListener actionListener = ActionListener.wrap( @@ -166,7 +169,8 @@ public void testInterceptWithEmptyEligibleIndexMetadataList() { threadPool, settings, () -> repositoriesService, - clusterSettings + clusterSettings, + DefaultRemoteStoreSettings.INSTANCE ); remoteIndexPathUploader.start(); ActionListener actionListener = ActionListener.wrap( @@ -228,7 +232,8 @@ public void testInterceptWithSameRepo() throws IOException { threadPool, settings, () -> repositoriesService, - clusterSettings + clusterSettings, + DefaultRemoteStoreSettings.INSTANCE ); remoteIndexPathUploader.start(); ActionListener actionListener = ActionListener.wrap( @@ -251,7 +256,8 @@ public void testInterceptWithDifferentRepo() throws IOException { threadPool, settings, () -> repositoriesService, - clusterSettings + clusterSettings, + DefaultRemoteStoreSettings.INSTANCE ); remoteIndexPathUploader.start(); ActionListener actionListener = ActionListener.wrap( @@ -271,7 +277,8 @@ public void testInterceptWithLatchAwaitTimeout() throws IOException { threadPool, settings, () -> repositoriesService, - clusterSettings + clusterSettings, + DefaultRemoteStoreSettings.INSTANCE ); remoteIndexPathUploader.start(); @@ -302,7 +309,8 @@ public void testInterceptWithInterruptedExceptionDuringLatchAwait() throws Excep threadPool, settings, () -> repositoriesService, - clusterSettings + clusterSettings, + DefaultRemoteStoreSettings.INSTANCE ); remoteIndexPathUploader.start(); Settings settings = Settings.builder() diff --git a/server/src/test/java/org/opensearch/index/remote/RemoteStoreCustomMetadataResolverTests.java b/server/src/test/java/org/opensearch/index/remote/RemoteStoreCustomMetadataResolverTests.java index abd115732c4db..331ebd0e3fcd4 100644 --- a/server/src/test/java/org/opensearch/index/remote/RemoteStoreCustomMetadataResolverTests.java +++ b/server/src/test/java/org/opensearch/index/remote/RemoteStoreCustomMetadataResolverTests.java @@ -220,4 +220,50 @@ public void testTranslogMetadataAllowedMinVersionOlder() { assertFalse(resolver.isTranslogMetadataEnabled()); } + public void testTranslogPathFixedPathSetting() { + + // Default settings + Settings settings = Settings.builder().build(); + ClusterSettings clusterSettings = new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); + RemoteStoreSettings remoteStoreSettings = new RemoteStoreSettings(settings, clusterSettings); + assertEquals("", remoteStoreSettings.getTranslogPathFixedPrefix()); + + // Any other random value + String randomPrefix = randomAlphaOfLengthBetween(2, 5); + settings = Settings.builder().put(RemoteStoreSettings.CLUSTER_REMOTE_STORE_TRANSLOG_PATH_PREFIX.getKey(), randomPrefix).build(); + remoteStoreSettings = new RemoteStoreSettings(settings, clusterSettings); + assertEquals(randomPrefix, remoteStoreSettings.getTranslogPathFixedPrefix()); + + // Set any other random value, the setting still points to the old value + clusterSettings.applySettings( + Settings.builder() + .put(RemoteStoreSettings.CLUSTER_REMOTE_STORE_TRANSLOG_PATH_PREFIX.getKey(), randomAlphaOfLengthBetween(2, 5)) + .build() + ); + assertEquals(randomPrefix, remoteStoreSettings.getTranslogPathFixedPrefix()); + } + + public void testSegmentsPathFixedPathSetting() { + + // Default settings + Settings settings = Settings.builder().build(); + ClusterSettings clusterSettings = new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); + RemoteStoreSettings remoteStoreSettings = new RemoteStoreSettings(settings, clusterSettings); + assertEquals("", remoteStoreSettings.getSegmentsPathFixedPrefix()); + + // Any other random value + String randomPrefix = randomAlphaOfLengthBetween(2, 5); + settings = Settings.builder().put(RemoteStoreSettings.CLUSTER_REMOTE_STORE_SEGMENTS_PATH_PREFIX.getKey(), randomPrefix).build(); + remoteStoreSettings = new RemoteStoreSettings(settings, clusterSettings); + assertEquals(randomPrefix, remoteStoreSettings.getSegmentsPathFixedPrefix()); + + // Set any other random value, the setting still points to the old value + clusterSettings.applySettings( + Settings.builder() + .put(RemoteStoreSettings.CLUSTER_REMOTE_STORE_SEGMENTS_PATH_PREFIX.getKey(), randomAlphaOfLengthBetween(2, 5)) + .build() + ); + assertEquals(randomPrefix, remoteStoreSettings.getSegmentsPathFixedPrefix()); + + } } diff --git a/server/src/test/java/org/opensearch/index/remote/RemoteStoreEnumsTests.java b/server/src/test/java/org/opensearch/index/remote/RemoteStoreEnumsTests.java index cfcac5915837c..b2203d6eccd27 100644 --- a/server/src/test/java/org/opensearch/index/remote/RemoteStoreEnumsTests.java +++ b/server/src/test/java/org/opensearch/index/remote/RemoteStoreEnumsTests.java @@ -142,6 +142,7 @@ public void testGeneratePathForHashedPrefixType() { String shardId = String.valueOf(randomInt(100)); DataCategory dataCategory = TRANSLOG; DataType dataType = DATA; + String fixedPrefix = "."; String basePath = getPath(pathList) + indexUUID + SEPARATOR + shardId; // Translog Data @@ -151,11 +152,20 @@ public void testGeneratePathForHashedPrefixType() { .shardId(shardId) .dataCategory(dataCategory) .dataType(dataType) + .fixedPrefix(fixedPrefix) .build(); BlobPath result = HASHED_PREFIX.path(pathInput, FNV_1A_BASE64); assertTrue( result.buildAsString() - .startsWith(String.join(SEPARATOR, FNV_1A_BASE64.hash(pathInput), basePath, dataCategory.getName(), dataType.getName())) + .startsWith( + String.join( + SEPARATOR, + fixedPrefix + FNV_1A_BASE64.hash(pathInput), + basePath, + dataCategory.getName(), + dataType.getName() + ) + ) ); // assert with exact value for known base path @@ -168,9 +178,10 @@ public void testGeneratePathForHashedPrefixType() { .shardId(fixedShardId) .dataCategory(dataCategory) .dataType(dataType) + .fixedPrefix(fixedPrefix) .build(); result = HASHED_PREFIX.path(pathInput, FNV_1A_BASE64); - assertEquals("DgSI70IciXs/xjsdhj/ddjsha/yudy7sd/32hdhua7/89jdij/k2ijhe877d7yuhx7/10/translog/data/", result.buildAsString()); + assertEquals(".DgSI70IciXs/xjsdhj/ddjsha/yudy7sd/32hdhua7/89jdij/k2ijhe877d7yuhx7/10/translog/data/", result.buildAsString()); // Translog Metadata dataType = METADATA; @@ -180,11 +191,20 @@ public void testGeneratePathForHashedPrefixType() { .shardId(shardId) .dataCategory(dataCategory) .dataType(dataType) + .fixedPrefix(fixedPrefix) .build(); result = HASHED_PREFIX.path(pathInput, FNV_1A_BASE64); assertTrue( result.buildAsString() - .startsWith(String.join(SEPARATOR, FNV_1A_BASE64.hash(pathInput), basePath, dataCategory.getName(), dataType.getName())) + .startsWith( + String.join( + SEPARATOR, + fixedPrefix + FNV_1A_BASE64.hash(pathInput), + basePath, + dataCategory.getName(), + dataType.getName() + ) + ) ); // assert with exact value for known base path @@ -194,9 +214,10 @@ public void testGeneratePathForHashedPrefixType() { .shardId(fixedShardId) .dataCategory(dataCategory) .dataType(dataType) + .fixedPrefix(fixedPrefix) .build(); result = HASHED_PREFIX.path(pathInput, FNV_1A_BASE64); - assertEquals("oKU5SjILiy4/xjsdhj/ddjsha/yudy7sd/32hdhua7/89jdij/k2ijhe877d7yuhx7/10/translog/metadata/", result.buildAsString()); + assertEquals(".oKU5SjILiy4/xjsdhj/ddjsha/yudy7sd/32hdhua7/89jdij/k2ijhe877d7yuhx7/10/translog/metadata/", result.buildAsString()); // Segment Data dataCategory = SEGMENTS; @@ -207,11 +228,20 @@ public void testGeneratePathForHashedPrefixType() { .shardId(shardId) .dataCategory(dataCategory) .dataType(dataType) + .fixedPrefix(fixedPrefix) .build(); result = HASHED_PREFIX.path(pathInput, FNV_1A_BASE64); assertTrue( result.buildAsString() - .startsWith(String.join(SEPARATOR, FNV_1A_BASE64.hash(pathInput), basePath, dataCategory.getName(), dataType.getName())) + .startsWith( + String.join( + SEPARATOR, + fixedPrefix + FNV_1A_BASE64.hash(pathInput), + basePath, + dataCategory.getName(), + dataType.getName() + ) + ) ); // assert with exact value for known base path @@ -221,9 +251,10 @@ public void testGeneratePathForHashedPrefixType() { .shardId(fixedShardId) .dataCategory(dataCategory) .dataType(dataType) + .fixedPrefix(fixedPrefix) .build(); result = HASHED_PREFIX.path(pathInput, FNV_1A_BASE64); - assertEquals("AUBRfCIuWdk/xjsdhj/ddjsha/yudy7sd/32hdhua7/89jdij/k2ijhe877d7yuhx7/10/segments/data/", result.buildAsString()); + assertEquals(".AUBRfCIuWdk/xjsdhj/ddjsha/yudy7sd/32hdhua7/89jdij/k2ijhe877d7yuhx7/10/segments/data/", result.buildAsString()); // Segment Metadata dataType = METADATA; @@ -233,11 +264,20 @@ public void testGeneratePathForHashedPrefixType() { .shardId(shardId) .dataCategory(dataCategory) .dataType(dataType) + .fixedPrefix(fixedPrefix) .build(); result = HASHED_PREFIX.path(pathInput, FNV_1A_BASE64); assertTrue( result.buildAsString() - .startsWith(String.join(SEPARATOR, FNV_1A_BASE64.hash(pathInput), basePath, dataCategory.getName(), dataType.getName())) + .startsWith( + String.join( + SEPARATOR, + fixedPrefix + FNV_1A_BASE64.hash(pathInput), + basePath, + dataCategory.getName(), + dataType.getName() + ) + ) ); // assert with exact value for known base path @@ -247,9 +287,10 @@ public void testGeneratePathForHashedPrefixType() { .shardId(fixedShardId) .dataCategory(dataCategory) .dataType(dataType) + .fixedPrefix(fixedPrefix) .build(); result = HASHED_PREFIX.path(pathInput, FNV_1A_BASE64); - assertEquals("erwR-G735Uw/xjsdhj/ddjsha/yudy7sd/32hdhua7/89jdij/k2ijhe877d7yuhx7/10/segments/metadata/", result.buildAsString()); + assertEquals(".erwR-G735Uw/xjsdhj/ddjsha/yudy7sd/32hdhua7/89jdij/k2ijhe877d7yuhx7/10/segments/metadata/", result.buildAsString()); // Segment Lockfiles dataType = LOCK_FILES; @@ -259,11 +300,20 @@ public void testGeneratePathForHashedPrefixType() { .shardId(shardId) .dataCategory(dataCategory) .dataType(dataType) + .fixedPrefix(fixedPrefix) .build(); result = HASHED_PREFIX.path(pathInput, FNV_1A_BASE64); assertTrue( result.buildAsString() - .startsWith(String.join(SEPARATOR, FNV_1A_BASE64.hash(pathInput), basePath, dataCategory.getName(), dataType.getName())) + .startsWith( + String.join( + SEPARATOR, + fixedPrefix + FNV_1A_BASE64.hash(pathInput), + basePath, + dataCategory.getName(), + dataType.getName() + ) + ) ); // assert with exact value for known base path @@ -273,12 +323,14 @@ public void testGeneratePathForHashedPrefixType() { .shardId(fixedShardId) .dataCategory(dataCategory) .dataType(dataType) + .fixedPrefix(fixedPrefix) .build(); result = HASHED_PREFIX.path(pathInput, FNV_1A_BASE64); - assertEquals("KeYDIk0mJXI/xjsdhj/ddjsha/yudy7sd/32hdhua7/89jdij/k2ijhe877d7yuhx7/10/segments/lock_files/", result.buildAsString()); + assertEquals(".KeYDIk0mJXI/xjsdhj/ddjsha/yudy7sd/32hdhua7/89jdij/k2ijhe877d7yuhx7/10/segments/lock_files/", result.buildAsString()); } public void testGeneratePathForHashedPrefixTypeAndFNVCompositeHashAlgorithm() { + String fixedPrefix = "."; BlobPath blobPath = new BlobPath(); List pathList = getPathList(); for (String path : pathList) { @@ -298,12 +350,19 @@ public void testGeneratePathForHashedPrefixTypeAndFNVCompositeHashAlgorithm() { .shardId(shardId) .dataCategory(dataCategory) .dataType(dataType) + .fixedPrefix(fixedPrefix) .build(); BlobPath result = HASHED_PREFIX.path(pathInput, FNV_1A_COMPOSITE_1); assertTrue( result.buildAsString() .startsWith( - String.join(SEPARATOR, FNV_1A_COMPOSITE_1.hash(pathInput), basePath, dataCategory.getName(), dataType.getName()) + String.join( + SEPARATOR, + fixedPrefix + FNV_1A_COMPOSITE_1.hash(pathInput), + basePath, + dataCategory.getName(), + dataType.getName() + ) ) ); @@ -317,9 +376,10 @@ public void testGeneratePathForHashedPrefixTypeAndFNVCompositeHashAlgorithm() { .shardId(fixedShardId) .dataCategory(dataCategory) .dataType(dataType) + .fixedPrefix(fixedPrefix) .build(); result = HASHED_PREFIX.path(pathInput, FNV_1A_COMPOSITE_1); - assertEquals("D10000001001000/xjsdhj/ddjsha/yudy7sd/32hdhua7/89jdij/k2ijhe877d7yuhx7/10/translog/data/", result.buildAsString()); + assertEquals(".D10000001001000/xjsdhj/ddjsha/yudy7sd/32hdhua7/89jdij/k2ijhe877d7yuhx7/10/translog/data/", result.buildAsString()); // Translog Metadata dataType = METADATA; @@ -329,12 +389,19 @@ public void testGeneratePathForHashedPrefixTypeAndFNVCompositeHashAlgorithm() { .shardId(shardId) .dataCategory(dataCategory) .dataType(dataType) + .fixedPrefix(fixedPrefix) .build(); result = HASHED_PREFIX.path(pathInput, FNV_1A_COMPOSITE_1); assertTrue( result.buildAsString() .startsWith( - String.join(SEPARATOR, FNV_1A_COMPOSITE_1.hash(pathInput), basePath, dataCategory.getName(), dataType.getName()) + String.join( + SEPARATOR, + fixedPrefix + FNV_1A_COMPOSITE_1.hash(pathInput), + basePath, + dataCategory.getName(), + dataType.getName() + ) ) ); @@ -345,10 +412,11 @@ public void testGeneratePathForHashedPrefixTypeAndFNVCompositeHashAlgorithm() { .shardId(fixedShardId) .dataCategory(dataCategory) .dataType(dataType) + .fixedPrefix(fixedPrefix) .build(); result = HASHED_PREFIX.path(pathInput, FNV_1A_COMPOSITE_1); assertEquals( - "o00101001010011/xjsdhj/ddjsha/yudy7sd/32hdhua7/89jdij/k2ijhe877d7yuhx7/10/translog/metadata/", + ".o00101001010011/xjsdhj/ddjsha/yudy7sd/32hdhua7/89jdij/k2ijhe877d7yuhx7/10/translog/metadata/", result.buildAsString() ); @@ -361,12 +429,19 @@ public void testGeneratePathForHashedPrefixTypeAndFNVCompositeHashAlgorithm() { .shardId(shardId) .dataCategory(dataCategory) .dataType(dataType) + .fixedPrefix(fixedPrefix) .build(); result = HASHED_PREFIX.path(pathInput, FNV_1A_COMPOSITE_1); assertTrue( result.buildAsString() .startsWith( - String.join(SEPARATOR, FNV_1A_COMPOSITE_1.hash(pathInput), basePath, dataCategory.getName(), dataType.getName()) + String.join( + SEPARATOR, + fixedPrefix + FNV_1A_COMPOSITE_1.hash(pathInput), + basePath, + dataCategory.getName(), + dataType.getName() + ) ) ); @@ -377,9 +452,10 @@ public void testGeneratePathForHashedPrefixTypeAndFNVCompositeHashAlgorithm() { .shardId(fixedShardId) .dataCategory(dataCategory) .dataType(dataType) + .fixedPrefix(fixedPrefix) .build(); result = HASHED_PREFIX.path(pathInput, FNV_1A_COMPOSITE_1); - assertEquals("A01010000000101/xjsdhj/ddjsha/yudy7sd/32hdhua7/89jdij/k2ijhe877d7yuhx7/10/segments/data/", result.buildAsString()); + assertEquals(".A01010000000101/xjsdhj/ddjsha/yudy7sd/32hdhua7/89jdij/k2ijhe877d7yuhx7/10/segments/data/", result.buildAsString()); // Segment Metadata dataType = METADATA; @@ -389,12 +465,19 @@ public void testGeneratePathForHashedPrefixTypeAndFNVCompositeHashAlgorithm() { .shardId(shardId) .dataCategory(dataCategory) .dataType(dataType) + .fixedPrefix(fixedPrefix) .build(); result = HASHED_PREFIX.path(pathInput, FNV_1A_COMPOSITE_1); assertTrue( result.buildAsString() .startsWith( - String.join(SEPARATOR, FNV_1A_COMPOSITE_1.hash(pathInput), basePath, dataCategory.getName(), dataType.getName()) + String.join( + SEPARATOR, + fixedPrefix + FNV_1A_COMPOSITE_1.hash(pathInput), + basePath, + dataCategory.getName(), + dataType.getName() + ) ) ); @@ -405,10 +488,11 @@ public void testGeneratePathForHashedPrefixTypeAndFNVCompositeHashAlgorithm() { .shardId(fixedShardId) .dataCategory(dataCategory) .dataType(dataType) + .fixedPrefix(fixedPrefix) .build(); result = HASHED_PREFIX.path(pathInput, FNV_1A_COMPOSITE_1); assertEquals( - "e10101111000001/xjsdhj/ddjsha/yudy7sd/32hdhua7/89jdij/k2ijhe877d7yuhx7/10/segments/metadata/", + ".e10101111000001/xjsdhj/ddjsha/yudy7sd/32hdhua7/89jdij/k2ijhe877d7yuhx7/10/segments/metadata/", result.buildAsString() ); @@ -420,12 +504,19 @@ public void testGeneratePathForHashedPrefixTypeAndFNVCompositeHashAlgorithm() { .shardId(shardId) .dataCategory(dataCategory) .dataType(dataType) + .fixedPrefix(fixedPrefix) .build(); result = HASHED_PREFIX.path(pathInput, FNV_1A_COMPOSITE_1); assertTrue( result.buildAsString() .startsWith( - String.join(SEPARATOR, FNV_1A_COMPOSITE_1.hash(pathInput), basePath, dataCategory.getName(), dataType.getName()) + String.join( + SEPARATOR, + fixedPrefix + FNV_1A_COMPOSITE_1.hash(pathInput), + basePath, + dataCategory.getName(), + dataType.getName() + ) ) ); @@ -436,10 +527,11 @@ public void testGeneratePathForHashedPrefixTypeAndFNVCompositeHashAlgorithm() { .shardId(fixedShardId) .dataCategory(dataCategory) .dataType(dataType) + .fixedPrefix(fixedPrefix) .build(); result = HASHED_PREFIX.path(pathInput, FNV_1A_COMPOSITE_1); assertEquals( - "K01111001100000/xjsdhj/ddjsha/yudy7sd/32hdhua7/89jdij/k2ijhe877d7yuhx7/10/segments/lock_files/", + ".K01111001100000/xjsdhj/ddjsha/yudy7sd/32hdhua7/89jdij/k2ijhe877d7yuhx7/10/segments/lock_files/", result.buildAsString() ); } @@ -455,6 +547,7 @@ public void testGeneratePathForHashedInfixType() { String shardId = String.valueOf(randomInt(100)); DataCategory dataCategory = TRANSLOG; DataType dataType = DATA; + String fixedPrefix = "."; String basePath = getPath(pathList); basePath = basePath.isEmpty() ? basePath : basePath.substring(0, basePath.length() - 1); @@ -465,9 +558,10 @@ public void testGeneratePathForHashedInfixType() { .shardId(shardId) .dataCategory(dataCategory) .dataType(dataType) + .fixedPrefix(fixedPrefix) .build(); BlobPath result = HASHED_INFIX.path(pathInput, FNV_1A_BASE64); - String expected = derivePath(basePath, pathInput); + String expected = derivePath(basePath, pathInput, fixedPrefix); String actual = result.buildAsString(); assertTrue(new ParameterizedMessage("expected={} actual={}", expected, actual).getFormattedMessage(), actual.startsWith(expected)); @@ -481,9 +575,10 @@ public void testGeneratePathForHashedInfixType() { .shardId(fixedShardId) .dataCategory(dataCategory) .dataType(dataType) + .fixedPrefix(fixedPrefix) .build(); result = HASHED_INFIX.path(pathInput, FNV_1A_BASE64); - expected = "xjsdhj/ddjsha/yudy7sd/32hdhua7/89jdij/DgSI70IciXs/k2ijhe877d7yuhx7/10/translog/data/"; + expected = "xjsdhj/ddjsha/yudy7sd/32hdhua7/89jdij/.DgSI70IciXs/k2ijhe877d7yuhx7/10/translog/data/"; actual = result.buildAsString(); assertTrue(new ParameterizedMessage("expected={} actual={}", expected, actual).getFormattedMessage(), actual.startsWith(expected)); @@ -495,10 +590,11 @@ public void testGeneratePathForHashedInfixType() { .shardId(shardId) .dataCategory(dataCategory) .dataType(dataType) + .fixedPrefix(fixedPrefix) .build(); result = HASHED_INFIX.path(pathInput, FNV_1A_BASE64); - expected = derivePath(basePath, pathInput); + expected = derivePath(basePath, pathInput, fixedPrefix); actual = result.buildAsString(); assertTrue(new ParameterizedMessage("expected={} actual={}", expected, actual).getFormattedMessage(), actual.startsWith(expected)); @@ -509,9 +605,10 @@ public void testGeneratePathForHashedInfixType() { .shardId(fixedShardId) .dataCategory(dataCategory) .dataType(dataType) + .fixedPrefix(fixedPrefix) .build(); result = HASHED_INFIX.path(pathInput, FNV_1A_BASE64); - expected = "xjsdhj/ddjsha/yudy7sd/32hdhua7/89jdij/oKU5SjILiy4/k2ijhe877d7yuhx7/10/translog/metadata/"; + expected = "xjsdhj/ddjsha/yudy7sd/32hdhua7/89jdij/.oKU5SjILiy4/k2ijhe877d7yuhx7/10/translog/metadata/"; actual = result.buildAsString(); assertTrue(new ParameterizedMessage("expected={} actual={}", expected, actual).getFormattedMessage(), actual.startsWith(expected)); @@ -524,9 +621,10 @@ public void testGeneratePathForHashedInfixType() { .shardId(shardId) .dataCategory(dataCategory) .dataType(dataType) + .fixedPrefix(fixedPrefix) .build(); result = HASHED_INFIX.path(pathInput, FNV_1A_BASE64); - expected = derivePath(basePath, pathInput); + expected = derivePath(basePath, pathInput, fixedPrefix); actual = result.buildAsString(); assertTrue(new ParameterizedMessage("expected={} actual={}", expected, actual).getFormattedMessage(), actual.startsWith(expected)); @@ -537,9 +635,10 @@ public void testGeneratePathForHashedInfixType() { .shardId(fixedShardId) .dataCategory(dataCategory) .dataType(dataType) + .fixedPrefix(fixedPrefix) .build(); result = HASHED_INFIX.path(pathInput, FNV_1A_BASE64); - expected = "xjsdhj/ddjsha/yudy7sd/32hdhua7/89jdij/AUBRfCIuWdk/k2ijhe877d7yuhx7/10/segments/data/"; + expected = "xjsdhj/ddjsha/yudy7sd/32hdhua7/89jdij/.AUBRfCIuWdk/k2ijhe877d7yuhx7/10/segments/data/"; actual = result.buildAsString(); assertTrue(new ParameterizedMessage("expected={} actual={}", expected, actual).getFormattedMessage(), actual.startsWith(expected)); @@ -551,9 +650,10 @@ public void testGeneratePathForHashedInfixType() { .shardId(shardId) .dataCategory(dataCategory) .dataType(dataType) + .fixedPrefix(fixedPrefix) .build(); result = HASHED_INFIX.path(pathInput, FNV_1A_BASE64); - expected = derivePath(basePath, pathInput); + expected = derivePath(basePath, pathInput, fixedPrefix); actual = result.buildAsString(); assertTrue(new ParameterizedMessage("expected={} actual={}", expected, actual).getFormattedMessage(), actual.startsWith(expected)); @@ -564,9 +664,10 @@ public void testGeneratePathForHashedInfixType() { .shardId(fixedShardId) .dataCategory(dataCategory) .dataType(dataType) + .fixedPrefix(fixedPrefix) .build(); result = HASHED_INFIX.path(pathInput, FNV_1A_BASE64); - expected = "xjsdhj/ddjsha/yudy7sd/32hdhua7/89jdij/erwR-G735Uw/k2ijhe877d7yuhx7/10/segments/metadata/"; + expected = "xjsdhj/ddjsha/yudy7sd/32hdhua7/89jdij/.erwR-G735Uw/k2ijhe877d7yuhx7/10/segments/metadata/"; actual = result.buildAsString(); assertTrue(new ParameterizedMessage("expected={} actual={}", expected, actual).getFormattedMessage(), actual.startsWith(expected)); @@ -578,9 +679,10 @@ public void testGeneratePathForHashedInfixType() { .shardId(shardId) .dataCategory(dataCategory) .dataType(dataType) + .fixedPrefix(fixedPrefix) .build(); result = HASHED_INFIX.path(pathInput, FNV_1A_BASE64); - expected = derivePath(basePath, pathInput); + expected = derivePath(basePath, pathInput, fixedPrefix); actual = result.buildAsString(); assertTrue(new ParameterizedMessage("expected={} actual={}", expected, actual).getFormattedMessage(), actual.startsWith(expected)); @@ -591,14 +693,16 @@ public void testGeneratePathForHashedInfixType() { .shardId(fixedShardId) .dataCategory(dataCategory) .dataType(dataType) + .fixedPrefix(fixedPrefix) .build(); result = HASHED_INFIX.path(pathInput, FNV_1A_BASE64); - expected = "xjsdhj/ddjsha/yudy7sd/32hdhua7/89jdij/KeYDIk0mJXI/k2ijhe877d7yuhx7/10/segments/lock_files/"; + expected = "xjsdhj/ddjsha/yudy7sd/32hdhua7/89jdij/.KeYDIk0mJXI/k2ijhe877d7yuhx7/10/segments/lock_files/"; actual = result.buildAsString(); assertTrue(new ParameterizedMessage("expected={} actual={}", expected, actual).getFormattedMessage(), actual.startsWith(expected)); } public void testGeneratePathForSnapshotShardPathInput() { + String fixedPrefix = "snap"; BlobPath blobPath = BlobPath.cleanPath().add("xjsdhj").add("ddjsha").add("yudy7sd").add("32hdhua7").add("89jdij"); String indexUUID = "dsdkjsu8832njn"; String shardId = "10"; @@ -606,6 +710,7 @@ public void testGeneratePathForSnapshotShardPathInput() { .basePath(blobPath) .indexUUID(indexUUID) .shardId(shardId) + .fixedPrefix(fixedPrefix) .build(); // FIXED PATH @@ -616,34 +721,34 @@ public void testGeneratePathForSnapshotShardPathInput() { // HASHED_PREFIX - FNV_1A_COMPOSITE_1 result = HASHED_PREFIX.path(pathInput, FNV_1A_COMPOSITE_1); - expected = "_11001000010110/xjsdhj/ddjsha/yudy7sd/32hdhua7/89jdij/indices/dsdkjsu8832njn/10/"; + expected = "snap_11001000010110/xjsdhj/ddjsha/yudy7sd/32hdhua7/89jdij/indices/dsdkjsu8832njn/10/"; actual = result.buildAsString(); assertEquals(new ParameterizedMessage("expected={} actual={}", expected, actual).getFormattedMessage(), actual, expected); // HASHED_PREFIX - FNV_1A_BASE64 result = HASHED_PREFIX.path(pathInput, FNV_1A_BASE64); - expected = "_yFiSl_VGGM/xjsdhj/ddjsha/yudy7sd/32hdhua7/89jdij/indices/dsdkjsu8832njn/10/"; + expected = "snap_yFiSl_VGGM/xjsdhj/ddjsha/yudy7sd/32hdhua7/89jdij/indices/dsdkjsu8832njn/10/"; actual = result.buildAsString(); assertEquals(new ParameterizedMessage("expected={} actual={}", expected, actual).getFormattedMessage(), actual, expected); // HASHED_INFIX - FNV_1A_COMPOSITE_1 result = HASHED_INFIX.path(pathInput, FNV_1A_COMPOSITE_1); - expected = "xjsdhj/ddjsha/yudy7sd/32hdhua7/89jdij/_11001000010110/indices/dsdkjsu8832njn/10/"; + expected = "xjsdhj/ddjsha/yudy7sd/32hdhua7/89jdij/snap_11001000010110/indices/dsdkjsu8832njn/10/"; actual = result.buildAsString(); assertEquals(new ParameterizedMessage("expected={} actual={}", expected, actual).getFormattedMessage(), actual, expected); // HASHED_INFIX - FNV_1A_BASE64 result = HASHED_INFIX.path(pathInput, FNV_1A_BASE64); - expected = "xjsdhj/ddjsha/yudy7sd/32hdhua7/89jdij/_yFiSl_VGGM/indices/dsdkjsu8832njn/10/"; + expected = "xjsdhj/ddjsha/yudy7sd/32hdhua7/89jdij/snap_yFiSl_VGGM/indices/dsdkjsu8832njn/10/"; actual = result.buildAsString(); assertEquals(new ParameterizedMessage("expected={} actual={}", expected, actual).getFormattedMessage(), actual, expected); } - private String derivePath(String basePath, ShardDataPathInput pathInput) { + private String derivePath(String basePath, ShardDataPathInput pathInput, String fixedPrefix) { return "".equals(basePath) ? String.join( SEPARATOR, - FNV_1A_BASE64.hash(pathInput), + fixedPrefix + FNV_1A_BASE64.hash(pathInput), pathInput.indexUUID(), pathInput.shardId(), pathInput.dataCategory().getName(), @@ -652,7 +757,7 @@ private String derivePath(String basePath, ShardDataPathInput pathInput) { : String.join( SEPARATOR, basePath, - FNV_1A_BASE64.hash(pathInput), + fixedPrefix + FNV_1A_BASE64.hash(pathInput), pathInput.indexUUID(), pathInput.shardId(), pathInput.dataCategory().getName(), diff --git a/server/src/test/java/org/opensearch/index/shard/IndexShardTests.java b/server/src/test/java/org/opensearch/index/shard/IndexShardTests.java index 377e4e99e9964..96794a83ef762 100644 --- a/server/src/test/java/org/opensearch/index/shard/IndexShardTests.java +++ b/server/src/test/java/org/opensearch/index/shard/IndexShardTests.java @@ -2899,7 +2899,7 @@ public void testSyncSegmentsFromGivenRemoteSegmentStore() throws IOException { target = reinitShard(target, routing); DiscoveryNode localNode = new DiscoveryNode("foo", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT); target.markAsRecovering("from snapshot", new RecoveryState(routing, localNode, null)); - target.syncSegmentsFromGivenRemoteSegmentStore(false, tempRemoteSegmentDirectory, primaryTerm, commitGeneration); + target.syncSegmentsFromGivenRemoteSegmentStore(false, tempRemoteSegmentDirectory, null, false); RemoteSegmentStoreDirectory remoteStoreDirectory = ((RemoteSegmentStoreDirectory) ((FilterDirectory) ((FilterDirectory) target .remoteStore() .directory()).getDelegate()).getDelegate()); diff --git a/server/src/test/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryFactoryTests.java b/server/src/test/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryFactoryTests.java index cad5e47531cc6..95b5d4456baf6 100644 --- a/server/src/test/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryFactoryTests.java +++ b/server/src/test/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryFactoryTests.java @@ -57,7 +57,7 @@ public void setup() { repositoriesService = mock(RepositoriesService.class); threadPool = mock(ThreadPool.class); when(repositoriesServiceSupplier.get()).thenReturn(repositoriesService); - remoteSegmentStoreDirectoryFactory = new RemoteSegmentStoreDirectoryFactory(repositoriesServiceSupplier, threadPool); + remoteSegmentStoreDirectoryFactory = new RemoteSegmentStoreDirectoryFactory(repositoriesServiceSupplier, threadPool, ""); } public void testNewDirectory() throws IOException { diff --git a/server/src/test/java/org/opensearch/index/store/lockmanager/RemoteStoreLockManagerFactoryTests.java b/server/src/test/java/org/opensearch/index/store/lockmanager/RemoteStoreLockManagerFactoryTests.java index de3dfbdaa4778..37ae0484d5ee1 100644 --- a/server/src/test/java/org/opensearch/index/store/lockmanager/RemoteStoreLockManagerFactoryTests.java +++ b/server/src/test/java/org/opensearch/index/store/lockmanager/RemoteStoreLockManagerFactoryTests.java @@ -42,7 +42,7 @@ public void setup() throws IOException { repositoriesServiceSupplier = mock(Supplier.class); repositoriesService = mock(RepositoriesService.class); when(repositoriesServiceSupplier.get()).thenReturn(repositoriesService); - remoteStoreLockManagerFactory = new RemoteStoreLockManagerFactory(repositoriesServiceSupplier); + remoteStoreLockManagerFactory = new RemoteStoreLockManagerFactory(repositoriesServiceSupplier, ""); } public void testNewLockManager() throws IOException { diff --git a/server/src/test/java/org/opensearch/index/translog/transfer/TranslogTransferManagerTests.java b/server/src/test/java/org/opensearch/index/translog/transfer/TranslogTransferManagerTests.java index 2426a14c0c93b..8605043ddd5b5 100644 --- a/server/src/test/java/org/opensearch/index/translog/transfer/TranslogTransferManagerTests.java +++ b/server/src/test/java/org/opensearch/index/translog/transfer/TranslogTransferManagerTests.java @@ -26,6 +26,7 @@ import org.opensearch.index.remote.RemoteStoreUtils; import org.opensearch.index.remote.RemoteTranslogTransferTracker; import org.opensearch.index.translog.Translog; +import org.opensearch.index.translog.TranslogReader; import org.opensearch.index.translog.transfer.FileSnapshot.CheckpointFileSnapshot; import org.opensearch.index.translog.transfer.FileSnapshot.TransferFileSnapshot; import org.opensearch.index.translog.transfer.FileSnapshot.TranslogFileSnapshot; @@ -874,4 +875,53 @@ public void testReadMetadataForGivenTimestampException() throws IOException { assertThrows(IOException.class, () -> translogTransferManager.readMetadata(3000L)); assertNoDownloadStats(true); } + + public void testPopulateFileTrackerWithLocalStateNoReaders() { + translogTransferManager.populateFileTrackerWithLocalState(null); + assertTrue(translogTransferManager.getFileTransferTracker().allUploaded().isEmpty()); + + translogTransferManager.populateFileTrackerWithLocalState(List.of()); + assertTrue(translogTransferManager.getFileTransferTracker().allUploaded().isEmpty()); + } + + public void testPopulateFileTrackerWithLocalState() { + TranslogReader reader1 = mock(TranslogReader.class); + when(reader1.getGeneration()).thenReturn(12L); + TranslogReader reader2 = mock(TranslogReader.class); + when(reader2.getGeneration()).thenReturn(23L); + TranslogReader reader3 = mock(TranslogReader.class); + when(reader3.getGeneration()).thenReturn(34L); + TranslogReader reader4 = mock(TranslogReader.class); + when(reader4.getGeneration()).thenReturn(45L); + + translogTransferManager.populateFileTrackerWithLocalState(List.of(reader1, reader2, reader3, reader4)); + assertEquals( + Set.of("translog-12.tlog", "translog-23.tlog", "translog-34.tlog", "translog-45.tlog"), + translogTransferManager.getFileTransferTracker().allUploaded() + ); + } + + public void testPopulateFileTrackerWithLocalStateNoCkpAsMetadata() { + TranslogTransferManager translogTransferManager = new TranslogTransferManager( + shardId, + transferService, + remoteBaseTransferPath.add(TRANSLOG.getName()), + remoteBaseTransferPath.add(METADATA.getName()), + tracker, + remoteTranslogTransferTracker, + DefaultRemoteStoreSettings.INSTANCE, + true + ); + + TranslogReader reader1 = mock(TranslogReader.class); + when(reader1.getGeneration()).thenReturn(12L); + TranslogReader reader2 = mock(TranslogReader.class); + when(reader2.getGeneration()).thenReturn(23L); + + translogTransferManager.populateFileTrackerWithLocalState(List.of(reader1, reader2)); + assertEquals( + Set.of("translog-12.tlog", "translog-12.ckp", "translog-23.tlog", "translog-23.ckp"), + translogTransferManager.getFileTransferTracker().allUploaded() + ); + } } diff --git a/server/src/test/java/org/opensearch/repositories/RepositoriesServiceTests.java b/server/src/test/java/org/opensearch/repositories/RepositoriesServiceTests.java index cb0a36c870d07..4cd822c7d583b 100644 --- a/server/src/test/java/org/opensearch/repositories/RepositoriesServiceTests.java +++ b/server/src/test/java/org/opensearch/repositories/RepositoriesServiceTests.java @@ -62,6 +62,7 @@ import org.opensearch.common.io.InputStreamContainer; import org.opensearch.common.lifecycle.Lifecycle; import org.opensearch.common.lifecycle.LifecycleListener; +import org.opensearch.common.settings.ClusterSettings; import org.opensearch.common.settings.Settings; import org.opensearch.core.action.ActionListener; import org.opensearch.core.common.Strings; @@ -148,6 +149,10 @@ private RepositoriesService createRepositoriesServiceWithMockedClusterService(Cl when(currentClusterState.getNodes()).thenReturn(nodes); when(clusterService.state()).thenReturn(currentClusterState); + when(clusterService.getSettings()).thenReturn(Settings.EMPTY); + ClusterSettings clusterSettings = new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); + when(clusterService.getClusterSettings()).thenReturn(clusterSettings); + RepositoriesService repositoriesService = new RepositoriesService( Settings.EMPTY, clusterService, @@ -676,6 +681,19 @@ public void finalizeSnapshot( listener.onResponse(null); } + @Override + public void finalizeSnapshot( + ShardGenerations shardGenerations, + long repositoryStateId, + Metadata clusterMetadata, + SnapshotInfo snapshotInfo, + Version repositoryMetaVersion, + Function stateTransformer, + ActionListener listener + ) { + listener.onResponse(null); + } + @Override public void deleteSnapshots( Collection snapshotIds, diff --git a/server/src/test/java/org/opensearch/repositories/blobstore/BlobStoreRepositoryHelperTests.java b/server/src/test/java/org/opensearch/repositories/blobstore/BlobStoreRepositoryHelperTests.java index 958a499ada167..970e375055bc7 100644 --- a/server/src/test/java/org/opensearch/repositories/blobstore/BlobStoreRepositoryHelperTests.java +++ b/server/src/test/java/org/opensearch/repositories/blobstore/BlobStoreRepositoryHelperTests.java @@ -23,6 +23,7 @@ import org.opensearch.index.IndexSettings; import org.opensearch.index.store.RemoteBufferedOutputDirectory; import org.opensearch.indices.IndicesService; +import org.opensearch.indices.RemoteStoreSettings; import org.opensearch.indices.recovery.RecoverySettings; import org.opensearch.indices.replication.common.ReplicationType; import org.opensearch.plugins.Plugin; @@ -59,13 +60,16 @@ protected Collection> getPlugins() { protected String[] getLockFilesInRemoteStore(String remoteStoreIndex, String remoteStoreRepository) throws IOException { final RepositoriesService repositoriesService = getInstanceFromNode(RepositoriesService.class); final BlobStoreRepository remoteStorerepository = (BlobStoreRepository) repositoriesService.repository(remoteStoreRepository); + ClusterService clusterService = getInstanceFromNode(ClusterService.class); + String segmentsPathFixedPrefix = RemoteStoreSettings.CLUSTER_REMOTE_STORE_SEGMENTS_PATH_PREFIX.get(clusterService.getSettings()); BlobPath shardLevelBlobPath = getShardLevelBlobPath( client(), remoteStoreIndex, remoteStorerepository.basePath(), "0", SEGMENTS, - LOCK_FILES + LOCK_FILES, + segmentsPathFixedPrefix ); BlobContainer blobContainer = remoteStorerepository.blobStore().blobContainer(shardLevelBlobPath); try (RemoteBufferedOutputDirectory lockDirectory = new RemoteBufferedOutputDirectory(blobContainer)) { diff --git a/server/src/test/java/org/opensearch/repositories/blobstore/ChecksumWritableBlobStoreFormatTests.java b/server/src/test/java/org/opensearch/repositories/blobstore/ChecksumWritableBlobStoreFormatTests.java index 536df880b2597..c4e53c1eea138 100644 --- a/server/src/test/java/org/opensearch/repositories/blobstore/ChecksumWritableBlobStoreFormatTests.java +++ b/server/src/test/java/org/opensearch/repositories/blobstore/ChecksumWritableBlobStoreFormatTests.java @@ -35,7 +35,12 @@ public class ChecksumWritableBlobStoreFormatTests extends OpenSearchTestCase { public void testSerDe() throws IOException { IndexMetadata indexMetadata = getIndexMetadata(); - BytesReference bytesReference = clusterBlocksFormat.serialize(indexMetadata, TEST_BLOB_FILE_NAME, CompressorRegistry.none()); + BytesReference bytesReference = clusterBlocksFormat.serialize( + (out, metadata) -> metadata.writeTo(out), + indexMetadata, + TEST_BLOB_FILE_NAME, + CompressorRegistry.none() + ); IndexMetadata readIndexMetadata = clusterBlocksFormat.deserialize(TEST_BLOB_FILE_NAME, bytesReference); assertThat(readIndexMetadata, is(indexMetadata)); } @@ -43,6 +48,7 @@ public void testSerDe() throws IOException { public void testSerDeForCompressed() throws IOException { IndexMetadata indexMetadata = getIndexMetadata(); BytesReference bytesReference = clusterBlocksFormat.serialize( + (out, metadata) -> metadata.writeTo(out), indexMetadata, TEST_BLOB_FILE_NAME, CompressorRegistry.getCompressor(DeflateCompressor.NAME) diff --git a/server/src/test/java/org/opensearch/search/approximate/ApproximatePointRangeQueryTests.java b/server/src/test/java/org/opensearch/search/approximate/ApproximatePointRangeQueryTests.java index dd683d28f00f7..7fb75e43da6ec 100644 --- a/server/src/test/java/org/opensearch/search/approximate/ApproximatePointRangeQueryTests.java +++ b/server/src/test/java/org/opensearch/search/approximate/ApproximatePointRangeQueryTests.java @@ -112,6 +112,7 @@ protected String toString(int dimension, byte[] value) { } } + @AwaitsFix(bugUrl = "https://github.com/opensearch-project/OpenSearch/issues/15600") public void testApproximateRangeWithSizeUnderDefault() throws IOException { try (Directory directory = newDirectory()) { try (RandomIndexWriter iw = new RandomIndexWriter(random(), directory, new WhitespaceAnalyzer())) { @@ -150,6 +151,7 @@ protected String toString(int dimension, byte[] value) { } } + @AwaitsFix(bugUrl = "https://github.com/opensearch-project/OpenSearch/issues/15600") public void testApproximateRangeWithSizeOverDefault() throws IOException { try (Directory directory = newDirectory()) { try (RandomIndexWriter iw = new RandomIndexWriter(random(), directory, new WhitespaceAnalyzer())) { @@ -193,6 +195,7 @@ protected String toString(int dimension, byte[] value) { } } + @AwaitsFix(bugUrl = "https://github.com/opensearch-project/OpenSearch/issues/15600") public void testApproximateRangeShortCircuit() throws IOException { try (Directory directory = newDirectory()) { try (RandomIndexWriter iw = new RandomIndexWriter(random(), directory, new WhitespaceAnalyzer())) { diff --git a/server/src/test/java/org/opensearch/snapshots/SnapshotResiliencyTests.java b/server/src/test/java/org/opensearch/snapshots/SnapshotResiliencyTests.java index fd84c6b48df8a..71a2b42c9847a 100644 --- a/server/src/test/java/org/opensearch/snapshots/SnapshotResiliencyTests.java +++ b/server/src/test/java/org/opensearch/snapshots/SnapshotResiliencyTests.java @@ -2013,7 +2013,8 @@ public void onFailure(final Exception e) { repositoriesService, transportService, actionFilters, - null + null, + DefaultRemoteStoreSettings.INSTANCE ); nodeEnv = new NodeEnvironment(settings, environment); final NamedXContentRegistry namedXContentRegistry = new NamedXContentRegistry(Collections.emptyList()); @@ -2070,7 +2071,7 @@ public void onFailure(final Exception e) { emptyMap(), null, emptyMap(), - new RemoteSegmentStoreDirectoryFactory(() -> repositoriesService, threadPool), + new RemoteSegmentStoreDirectoryFactory(() -> repositoriesService, threadPool, ""), repositoriesServiceReference::get, null, new RemoteStoreStatsTrackerFactory(clusterService, settings), @@ -2366,7 +2367,8 @@ public void onFailure(final Exception e) { snapshotsService, threadPool, actionFilters, - indexNameExpressionResolver + indexNameExpressionResolver, + DefaultRemoteStoreSettings.INSTANCE ) ); actions.put( diff --git a/test/framework/src/main/java/org/opensearch/index/shard/RestoreOnlyRepository.java b/test/framework/src/main/java/org/opensearch/index/shard/RestoreOnlyRepository.java index 1ca1a6969ab2d..846f54bd13b40 100644 --- a/test/framework/src/main/java/org/opensearch/index/shard/RestoreOnlyRepository.java +++ b/test/framework/src/main/java/org/opensearch/index/shard/RestoreOnlyRepository.java @@ -118,6 +118,19 @@ public void getRepositoryData(ActionListener listener) { ); } + @Override + public void finalizeSnapshot( + ShardGenerations shardGenerations, + long repositoryStateId, + Metadata clusterMetadata, + SnapshotInfo snapshotInfo, + Version repositoryMetaVersion, + Function stateTransformer, + ActionListener listener + ) { + listener.onResponse(null); + } + @Override public void finalizeSnapshot( ShardGenerations shardGenerations, diff --git a/test/framework/src/main/java/org/opensearch/repositories/blobstore/BlobStoreTestUtil.java b/test/framework/src/main/java/org/opensearch/repositories/blobstore/BlobStoreTestUtil.java index e290cdee836b1..027b1bef84e7f 100644 --- a/test/framework/src/main/java/org/opensearch/repositories/blobstore/BlobStoreTestUtil.java +++ b/test/framework/src/main/java/org/opensearch/repositories/blobstore/BlobStoreTestUtil.java @@ -51,6 +51,8 @@ import org.opensearch.common.blobstore.BlobMetadata; import org.opensearch.common.blobstore.BlobPath; import org.opensearch.common.blobstore.BlobStore; +import org.opensearch.common.settings.ClusterSettings; +import org.opensearch.common.settings.Settings; import org.opensearch.common.unit.TimeValue; import org.opensearch.common.xcontent.LoggingDeprecationHandler; import org.opensearch.core.common.Strings; @@ -463,6 +465,9 @@ private static ClusterService mockClusterService(ClusterState initialState) { return null; }).when(clusterService).addStateApplier(any(ClusterStateApplier.class)); when(clusterApplierService.threadPool()).thenReturn(threadPool); + when(clusterService.getSettings()).thenReturn(Settings.EMPTY); + ClusterSettings clusterSettings = new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); + when(clusterService.getClusterSettings()).thenReturn(clusterSettings); return clusterService; } diff --git a/test/framework/src/main/java/org/opensearch/snapshots/AbstractSnapshotIntegTestCase.java b/test/framework/src/main/java/org/opensearch/snapshots/AbstractSnapshotIntegTestCase.java index 196aaf23df3c6..a8bb10fe20752 100644 --- a/test/framework/src/main/java/org/opensearch/snapshots/AbstractSnapshotIntegTestCase.java +++ b/test/framework/src/main/java/org/opensearch/snapshots/AbstractSnapshotIntegTestCase.java @@ -66,6 +66,7 @@ import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.index.IndexModule; import org.opensearch.index.store.RemoteBufferedOutputDirectory; +import org.opensearch.indices.RemoteStoreSettings; import org.opensearch.indices.replication.common.ReplicationType; import org.opensearch.node.NodeClosedException; import org.opensearch.plugins.Plugin; @@ -553,13 +554,15 @@ protected void assertDocCount(String index, long count) { protected String[] getLockFilesInRemoteStore(String remoteStoreIndex, String remoteStoreRepositoryName) throws IOException { final RepositoriesService repositoriesService = internalCluster().getCurrentClusterManagerNodeInstance(RepositoriesService.class); final BlobStoreRepository remoteStoreRepository = (BlobStoreRepository) repositoriesService.repository(remoteStoreRepositoryName); + String segmentsPathFixedPrefix = RemoteStoreSettings.CLUSTER_REMOTE_STORE_SEGMENTS_PATH_PREFIX.get(getNodeSettings()); BlobPath shardLevelBlobPath = getShardLevelBlobPath( client(), remoteStoreIndex, remoteStoreRepository.basePath(), "0", SEGMENTS, - LOCK_FILES + LOCK_FILES, + segmentsPathFixedPrefix ); BlobContainer blobContainer = remoteStoreRepository.blobStore().blobContainer(shardLevelBlobPath); try (RemoteBufferedOutputDirectory lockDirectory = new RemoteBufferedOutputDirectory(blobContainer)) { diff --git a/test/framework/src/main/java/org/opensearch/test/OpenSearchIntegTestCase.java b/test/framework/src/main/java/org/opensearch/test/OpenSearchIntegTestCase.java index f277c118c3d46..7d1fe1b000d20 100644 --- a/test/framework/src/main/java/org/opensearch/test/OpenSearchIntegTestCase.java +++ b/test/framework/src/main/java/org/opensearch/test/OpenSearchIntegTestCase.java @@ -399,6 +399,10 @@ public abstract class OpenSearchIntegTestCase extends OpenSearchTestCase { private static Boolean prefixModeVerificationEnable; + private static Boolean translogPathFixedPrefix; + + private static Boolean segmentsPathFixedPrefix; + private Path remoteStoreRepositoryPath; private ReplicationType randomReplicationType; @@ -408,6 +412,8 @@ public abstract class OpenSearchIntegTestCase extends OpenSearchTestCase { @BeforeClass public static void beforeClass() throws Exception { prefixModeVerificationEnable = randomBoolean(); + translogPathFixedPrefix = randomBoolean(); + segmentsPathFixedPrefix = randomBoolean(); testClusterRule.beforeClass(); } @@ -2585,6 +2591,12 @@ protected void updateRepository(String repoName, String type, Settings.Builder s putRepository(clusterAdmin(), repoName, type, null, settings); } + public Settings getNodeSettings() { + InternalTestCluster internalTestCluster = internalCluster(); + ClusterService clusterService = internalTestCluster.getInstance(ClusterService.class, internalTestCluster.getClusterManagerName()); + return clusterService.getSettings(); + } + public static void putRepository(ClusterAdminClient adminClient, String repoName, String type, Settings.Builder settings) { assertAcked(putRepositoryRequestBuilder(adminClient, repoName, type, true, settings, null, false)); } @@ -2887,6 +2899,8 @@ private static Settings buildRemoteStoreNodeAttributes( settings.put(RemoteStoreSettings.CLUSTER_REMOTE_STORE_PATH_TYPE_SETTING.getKey(), randomFrom(PathType.values())); settings.put(RemoteStoreSettings.CLUSTER_REMOTE_STORE_TRANSLOG_METADATA.getKey(), randomBoolean()); settings.put(RemoteStoreSettings.CLUSTER_REMOTE_STORE_PINNED_TIMESTAMP_ENABLED.getKey(), randomBoolean()); + settings.put(RemoteStoreSettings.CLUSTER_REMOTE_STORE_SEGMENTS_PATH_PREFIX.getKey(), translogPathFixedPrefix ? "a" : ""); + settings.put(RemoteStoreSettings.CLUSTER_REMOTE_STORE_TRANSLOG_PATH_PREFIX.getKey(), segmentsPathFixedPrefix ? "b" : ""); return settings.build(); } diff --git a/test/framework/src/main/java/org/opensearch/test/OpenSearchTestCase.java b/test/framework/src/main/java/org/opensearch/test/OpenSearchTestCase.java index e07d7ef3c5a7b..b180187303a60 100644 --- a/test/framework/src/main/java/org/opensearch/test/OpenSearchTestCase.java +++ b/test/framework/src/main/java/org/opensearch/test/OpenSearchTestCase.java @@ -1817,7 +1817,8 @@ public static BlobPath getShardLevelBlobPath( BlobPath basePath, String shardId, RemoteStoreEnums.DataCategory dataCategory, - RemoteStoreEnums.DataType dataType + RemoteStoreEnums.DataType dataType, + String fixedPrefix ) { String indexUUID = client.admin() .indices() @@ -1842,6 +1843,7 @@ public static BlobPath getShardLevelBlobPath( .shardId(shardId) .dataCategory(dataCategory) .dataType(dataType) + .fixedPrefix(fixedPrefix) .build(); return type.path(pathInput, hashAlgorithm); }