From a9bc1c4b51f3ca48e6e31d56073cbb441cd188cb Mon Sep 17 00:00:00 2001 From: sudarshan baliga Date: Mon, 24 Jul 2023 17:58:18 +0530 Subject: [PATCH 01/21] Added transport action for bulk async shard fetch for primary shards Signed-off-by: sudarshan baliga --- .../gateway/AsyncBatchShardFetch.java | 34 ++ ...ortNodesListGatewayStartedShardsBatch.java | 458 ++++++++++++++++++ 2 files changed, 492 insertions(+) create mode 100644 server/src/main/java/org/opensearch/gateway/AsyncBatchShardFetch.java create mode 100644 server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedShardsBatch.java diff --git a/server/src/main/java/org/opensearch/gateway/AsyncBatchShardFetch.java b/server/src/main/java/org/opensearch/gateway/AsyncBatchShardFetch.java new file mode 100644 index 0000000000000..4872d75e13740 --- /dev/null +++ b/server/src/main/java/org/opensearch/gateway/AsyncBatchShardFetch.java @@ -0,0 +1,34 @@ +/* + * 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; + +import org.opensearch.action.ActionListener; +import org.opensearch.action.support.nodes.BaseNodeResponse; +import org.opensearch.action.support.nodes.BaseNodesResponse; +import org.opensearch.cluster.node.DiscoveryNode; +import org.opensearch.common.lease.Releasable; +import org.opensearch.core.index.shard.ShardId; + +import java.util.Map; + +/** + * This class is responsible for fetching shard data from nodes. It is analogous to AsyncShardFetch class except + * that we fetch batch of shards in this class from single transport request to a node. + * @param + * + * @opensearch.internal + */ +public abstract class AsyncBatchShardFetch implements Releasable { + /** + * An action that lists the relevant shard data that needs to be fetched. + */ + public interface Lister, NodeResponse extends BaseNodeResponse> { + void list(DiscoveryNode[] nodes, Map shardIdsWithCustomDataPath, ActionListener listener); + } +} diff --git a/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedShardsBatch.java b/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedShardsBatch.java new file mode 100644 index 0000000000000..3bd2860dea094 --- /dev/null +++ b/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedShardsBatch.java @@ -0,0 +1,458 @@ +/* + * 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; + +import org.apache.logging.log4j.message.ParameterizedMessage; +import org.opensearch.OpenSearchException; +import org.opensearch.Version; +import org.opensearch.action.ActionListener; +import org.opensearch.action.ActionType; +import org.opensearch.action.FailedNodeException; +import org.opensearch.action.support.ActionFilters; +import org.opensearch.action.support.nodes.BaseNodeResponse; +import org.opensearch.action.support.nodes.BaseNodesRequest; +import org.opensearch.action.support.nodes.BaseNodesResponse; +import org.opensearch.action.support.nodes.TransportNodesAction; +import org.opensearch.cluster.ClusterName; +import org.opensearch.cluster.metadata.IndexMetadata; +import org.opensearch.cluster.node.DiscoveryNode; +import org.opensearch.cluster.service.ClusterService; +import org.opensearch.common.inject.Inject; +import org.opensearch.core.common.io.stream.StreamInput; +import org.opensearch.core.common.io.stream.StreamOutput; +import org.opensearch.common.settings.Settings; +import org.opensearch.core.xcontent.NamedXContentRegistry; +import org.opensearch.env.NodeEnvironment; +import org.opensearch.index.IndexSettings; +import org.opensearch.index.shard.IndexShard; +import org.opensearch.core.index.shard.ShardId; +import org.opensearch.index.shard.ShardPath; +import org.opensearch.index.shard.ShardStateMetadata; +import org.opensearch.index.store.Store; +import org.opensearch.indices.IndicesService; +import org.opensearch.indices.replication.checkpoint.ReplicationCheckpoint; +import org.opensearch.threadpool.ThreadPool; +import org.opensearch.transport.TransportRequest; +import org.opensearch.transport.TransportService; + +import java.io.IOException; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; + +/** + * This transport action is used to fetch all unassigned shard version from each node during primary allocation in {@link GatewayAllocator}. + * We use this to find out which node holds the latest shard version and which of them used to be a primary in order to allocate + * shards after node or cluster restarts. + * + * @opensearch.internal + */ +public class TransportNodesListGatewayStartedShardsBatch extends TransportNodesAction< + TransportNodesListGatewayStartedShardsBatch.Request, + TransportNodesListGatewayStartedShardsBatch.NodesGatewayStartedShardsBatch, + TransportNodesListGatewayStartedShardsBatch.NodeRequest, + TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShardsBatch> + implements + AsyncBatchShardFetch.Lister< + TransportNodesListGatewayStartedShardsBatch.NodesGatewayStartedShardsBatch, + TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShardsBatch> { + + public static final String ACTION_NAME = "internal:gateway/local/started_shards_batch"; + public static final ActionType TYPE = new ActionType<>( + ACTION_NAME, + NodesGatewayStartedShardsBatch::new + ); + + private final Settings settings; + private final NodeEnvironment nodeEnv; + private final IndicesService indicesService; + private final NamedXContentRegistry namedXContentRegistry; + + @Inject + public TransportNodesListGatewayStartedShardsBatch( + Settings settings, + ThreadPool threadPool, + ClusterService clusterService, + TransportService transportService, + ActionFilters actionFilters, + NodeEnvironment env, + IndicesService indicesService, + NamedXContentRegistry namedXContentRegistry + ) { + super( + ACTION_NAME, + threadPool, + clusterService, + transportService, + actionFilters, + Request::new, + NodeRequest::new, + ThreadPool.Names.FETCH_SHARD_STARTED, + NodeGatewayStartedShardsBatch.class + ); + this.settings = settings; + this.nodeEnv = env; + this.indicesService = indicesService; + this.namedXContentRegistry = namedXContentRegistry; + } + + @Override + public void list( + DiscoveryNode[] nodes, + Map shardIdsWithCustomDataPath, + ActionListener listener + ) { + execute(new Request(nodes, shardIdsWithCustomDataPath), listener); + } + + @Override + protected NodeRequest newNodeRequest(Request request) { + return new NodeRequest(request); + } + + @Override + protected NodeGatewayStartedShardsBatch newNodeResponse(StreamInput in) throws IOException { + return new NodeGatewayStartedShardsBatch(in); + } + + @Override + protected NodesGatewayStartedShardsBatch newResponse( + Request request, + List responses, + List failures + ) { + return new NodesGatewayStartedShardsBatch(clusterService.getClusterName(), responses, failures); + } + + /** + * This function is similar to nodeoperation method of {@link TransportNodesListGatewayStartedShards} we loop over + * the shards here to fetch the shard result in bulk. + * + * @param request + * @return NodeGatewayStartedShardsBatch + */ + @Override + protected NodeGatewayStartedShardsBatch nodeOperation(NodeRequest request) { + Map shardsOnNode = new HashMap<>(); + for (Map.Entry shardToCustomDataPathEntry : request.shardIdsWithCustomDataPath.entrySet()) { + final ShardId shardId = shardToCustomDataPathEntry.getKey(); + try { + logger.trace("{} loading local shard state info", shardId); + ShardStateMetadata shardStateMetadata = ShardStateMetadata.FORMAT.loadLatestState( + logger, + namedXContentRegistry, + nodeEnv.availableShardPaths(shardId) + ); + if (shardStateMetadata != null) { + if (indicesService.getShardOrNull(shardId) == null + && shardStateMetadata.indexDataLocation == ShardStateMetadata.IndexDataLocation.LOCAL) { + final String customDataPath; + if (shardToCustomDataPathEntry.getValue() != null) { + customDataPath = shardToCustomDataPathEntry.getValue(); + } else { + // TODO: Fallback for BWC with older OpenSearch versions. + // Remove once request.getCustomDataPath() always returns non-null + final IndexMetadata metadata = clusterService.state().metadata().index(shardId.getIndex()); + if (metadata != null) { + customDataPath = new IndexSettings(metadata, settings).customDataPath(); + } else { + logger.trace("{} node doesn't have meta data for the requests index", shardId); + throw new OpenSearchException("node doesn't have meta data for index " + shardId.getIndex()); + } + } + // we don't have an open shard on the store, validate the files on disk are openable + ShardPath shardPath = null; + try { + shardPath = ShardPath.loadShardPath(logger, nodeEnv, shardId, customDataPath); + if (shardPath == null) { + throw new IllegalStateException(shardId + " no shard path found"); + } + Store.tryOpenIndex(shardPath.resolveIndex(), shardId, nodeEnv::shardLock, logger); + } catch (Exception exception) { + final ShardPath finalShardPath = shardPath; + logger.trace( + () -> new ParameterizedMessage( + "{} can't open index for shard [{}] in path [{}]", + shardId, + shardStateMetadata, + (finalShardPath != null) ? finalShardPath.resolveIndex() : "" + ), + exception + ); + String allocationId = shardStateMetadata.allocationId != null ? shardStateMetadata.allocationId.getId() : null; + shardsOnNode.put( + shardId, + new NodeGatewayStartedShards(allocationId, shardStateMetadata.primary, null, exception) + ); + continue; + } + } + + logger.debug("{} shard state info found: [{}]", shardId, shardStateMetadata); + String allocationId = shardStateMetadata.allocationId != null ? shardStateMetadata.allocationId.getId() : null; + final IndexShard shard = indicesService.getShardOrNull(shardId); + shardsOnNode.put( + shardId, + new NodeGatewayStartedShards( + allocationId, + shardStateMetadata.primary, + shard != null ? shard.getLatestReplicationCheckpoint() : null + ) + ); + continue; + } + logger.trace("{} no local shard info found", shardId); + shardsOnNode.put(shardId, new NodeGatewayStartedShards(null, false, null)); + } catch (Exception e) { + shardsOnNode.put( + shardId, + new NodeGatewayStartedShards(null, false, null, new OpenSearchException("failed to load started shards", e)) + ); + } + } + return new NodeGatewayStartedShardsBatch(clusterService.localNode(), shardsOnNode); + } + + /** + * This is used in constructing the request for making the transport request to set of other node. + * Refer {@link TransportNodesAction} class start method. + * + * @opensearch.internal + */ + public static class Request extends BaseNodesRequest { + private final Map shardIdsWithCustomDataPath; + + public Request(StreamInput in) throws IOException { + super(in); + shardIdsWithCustomDataPath = in.readMap(ShardId::new, StreamInput::readString); + } + + public Request(DiscoveryNode[] nodes, Map shardIdStringMap) { + super(nodes); + this.shardIdsWithCustomDataPath = Objects.requireNonNull(shardIdStringMap); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + out.writeMap(shardIdsWithCustomDataPath, (o, k) -> k.writeTo(o), StreamOutput::writeString); + } + + public Map getShardIdsMap() { + return shardIdsWithCustomDataPath; + } + } + + /** + * Responses received from set of other nodes is clubbed into this class and sent back to the caller + * of this transport request. Refer {@link TransportNodesAction} + * + * @opensearch.internal + */ + public static class NodesGatewayStartedShardsBatch extends BaseNodesResponse { + + public NodesGatewayStartedShardsBatch(StreamInput in) throws IOException { + super(in); + } + + public NodesGatewayStartedShardsBatch( + ClusterName clusterName, + List nodes, + List failures + ) { + super(clusterName, nodes, failures); + } + + @Override + protected List readNodesFrom(StreamInput in) throws IOException { + return in.readList(NodeGatewayStartedShardsBatch::new); + } + + @Override + protected void writeNodesTo(StreamOutput out, List nodes) throws IOException { + out.writeList(nodes); + } + } + + /** + * NodeRequest class is for deserializing the request received by this node from other node for this transport action. + * This is used in {@link TransportNodesAction} + * + * @opensearch.internal + */ + public static class NodeRequest extends TransportRequest { + private final Map shardIdsWithCustomDataPath; + + public NodeRequest(StreamInput in) throws IOException { + super(in); + shardIdsWithCustomDataPath = in.readMap(ShardId::new, StreamInput::readString); + } + + public NodeRequest(Request request) { + this.shardIdsWithCustomDataPath = Objects.requireNonNull(request.getShardIdsMap()); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + out.writeMap(shardIdsWithCustomDataPath, (o, k) -> k.writeTo(o), StreamOutput::writeString); + } + + } + + /** + * Class for storing the information about the shards fetched on the node. + * + * @opensearch.internal + */ + public static class NodeGatewayStartedShards { + private final String allocationId; + private final boolean primary; + private final Exception storeException; + private final ReplicationCheckpoint replicationCheckpoint; + + public NodeGatewayStartedShards(StreamInput in) throws IOException { + allocationId = in.readOptionalString(); + primary = in.readBoolean(); + if (in.readBoolean()) { + storeException = in.readException(); + } else { + storeException = null; + } + if (in.getVersion().onOrAfter(Version.V_2_3_0) && in.readBoolean()) { + replicationCheckpoint = new ReplicationCheckpoint(in); + } else { + replicationCheckpoint = null; + } + } + + public NodeGatewayStartedShards(String allocationId, boolean primary, ReplicationCheckpoint replicationCheckpoint) { + this(allocationId, primary, replicationCheckpoint, null); + } + + public NodeGatewayStartedShards( + String allocationId, + boolean primary, + ReplicationCheckpoint replicationCheckpoint, + Exception storeException + ) { + this.allocationId = allocationId; + this.primary = primary; + this.replicationCheckpoint = replicationCheckpoint; + this.storeException = storeException; + } + + public String allocationId() { + return this.allocationId; + } + + public boolean primary() { + return this.primary; + } + + public ReplicationCheckpoint replicationCheckpoint() { + return this.replicationCheckpoint; + } + + public Exception storeException() { + return this.storeException; + } + + public void writeTo(StreamOutput out) throws IOException { + out.writeOptionalString(allocationId); + out.writeBoolean(primary); + if (storeException != null) { + out.writeBoolean(true); + out.writeException(storeException); + } else { + out.writeBoolean(false); + } + if (out.getVersion().onOrAfter(Version.V_2_3_0)) { + if (replicationCheckpoint != null) { + out.writeBoolean(true); + replicationCheckpoint.writeTo(out); + } else { + out.writeBoolean(false); + } + } + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + + NodeGatewayStartedShards that = (NodeGatewayStartedShards) o; + + return primary == that.primary + && Objects.equals(allocationId, that.allocationId) + && Objects.equals(storeException, that.storeException) + && Objects.equals(replicationCheckpoint, that.replicationCheckpoint); + } + + @Override + public int hashCode() { + int result = (allocationId != null ? allocationId.hashCode() : 0); + result = 31 * result + (primary ? 1 : 0); + result = 31 * result + (storeException != null ? storeException.hashCode() : 0); + result = 31 * result + (replicationCheckpoint != null ? replicationCheckpoint.hashCode() : 0); + return result; + } + + @Override + public String toString() { + StringBuilder buf = new StringBuilder(); + buf.append("NodeGatewayStartedShards[").append("allocationId=").append(allocationId).append(",primary=").append(primary); + if (storeException != null) { + buf.append(",storeException=").append(storeException); + } + if (replicationCheckpoint != null) { + buf.append(",ReplicationCheckpoint=").append(replicationCheckpoint.toString()); + } + buf.append("]"); + return buf.toString(); + } + } + + /** + * This is the response from a single node, this is used in {@link NodesGatewayStartedShardsBatch} for creating + * node to its response mapping for this transport request. + * Refer {@link TransportNodesAction} start method + * + * @opensearch.internal + */ + public static class NodeGatewayStartedShardsBatch extends BaseNodeResponse { + private final Map nodeGatewayStartedShardsBatch; + + public Map getNodeGatewayStartedShardsBatch() { + return nodeGatewayStartedShardsBatch; + } + + public NodeGatewayStartedShardsBatch(StreamInput in) throws IOException { + super(in); + this.nodeGatewayStartedShardsBatch = in.readMap(ShardId::new, NodeGatewayStartedShards::new); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + out.writeMap(nodeGatewayStartedShardsBatch, (o, k) -> k.writeTo(o), (o, v) -> v.writeTo(o)); + } + + public NodeGatewayStartedShardsBatch(DiscoveryNode node, Map nodeGatewayStartedShardsBatch) { + super(node); + this.nodeGatewayStartedShardsBatch = nodeGatewayStartedShardsBatch; + } + } +} From 13b026da978b8af6fe03549f2414d0f2cf46fc13 Mon Sep 17 00:00:00 2001 From: sudarshan baliga Date: Tue, 1 Aug 2023 00:18:46 +0530 Subject: [PATCH 02/21] Add PSA Async batch shard fetch transport integ test Signed-off-by: sudarshan baliga --- ...tNodesListGatewayStartedShardsBatchIT.java | 171 ++++++++++++++++++ .../org/opensearch/gateway/GatewayModule.java | 1 + 2 files changed, 172 insertions(+) create mode 100644 server/src/internalClusterTest/java/org/opensearch/gateway/TransportNodesListGatewayStartedShardsBatchIT.java diff --git a/server/src/internalClusterTest/java/org/opensearch/gateway/TransportNodesListGatewayStartedShardsBatchIT.java b/server/src/internalClusterTest/java/org/opensearch/gateway/TransportNodesListGatewayStartedShardsBatchIT.java new file mode 100644 index 0000000000000..6c53333843834 --- /dev/null +++ b/server/src/internalClusterTest/java/org/opensearch/gateway/TransportNodesListGatewayStartedShardsBatchIT.java @@ -0,0 +1,171 @@ +/* + * 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; + +import org.opensearch.action.admin.cluster.shards.ClusterSearchShardsGroup; +import org.opensearch.action.admin.cluster.shards.ClusterSearchShardsResponse; +import org.opensearch.action.admin.cluster.state.ClusterStateRequest; +import org.opensearch.action.admin.cluster.state.ClusterStateResponse; +import org.opensearch.action.support.ActionTestUtils; +import org.opensearch.cluster.metadata.IndexMetadata; +import org.opensearch.cluster.node.DiscoveryNode; +import org.opensearch.common.settings.Settings; +import org.opensearch.core.index.Index; +import org.opensearch.core.index.shard.ShardId; +import org.opensearch.env.NodeEnvironment; +import org.opensearch.index.shard.ShardPath; +import org.opensearch.test.OpenSearchIntegTestCase; + +import java.io.IOException; +import java.nio.file.DirectoryStream; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.HashMap; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ExecutionException; + +import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_NUMBER_OF_REPLICAS; +import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_NUMBER_OF_SHARDS; + +public class TransportNodesListGatewayStartedShardsBatchIT extends OpenSearchIntegTestCase { + + public void testSingleShardFetch() throws Exception { + String indexName = "test"; + Map shardIdCustomDataPathMap = prepareRequestMap(new String[] { indexName }, 1); + + ClusterSearchShardsResponse searchShardsResponse = client().admin().cluster().prepareSearchShards(indexName).get(); + + TransportNodesListGatewayStartedShardsBatch.NodesGatewayStartedShardsBatch response; + response = ActionTestUtils.executeBlocking( + internalCluster().getInstance(TransportNodesListGatewayStartedShardsBatch.class), + new TransportNodesListGatewayStartedShardsBatch.Request(searchShardsResponse.getNodes(), shardIdCustomDataPathMap) + ); + final Index index = resolveIndex(indexName); + final ShardId shardId = new ShardId(index, 0); + TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShards nodeGatewayStartedShards = response.getNodesMap() + .get(searchShardsResponse.getNodes()[0].getId()) + .getNodeGatewayStartedShardsBatch() + .get(shardId); + assertNodeGatewayStartedShardsHappyCase(nodeGatewayStartedShards); + } + + public void testShardFetchMultiNodeMultiIndexes() throws Exception { + // start second node + internalCluster().startNode(); + String indexName1 = "test1"; + String indexName2 = "test2"; + // assign one primary shard each to the data nodes + Map shardIdCustomDataPathMap = prepareRequestMap( + new String[] { indexName1, indexName2 }, + internalCluster().numDataNodes() + ); + ClusterSearchShardsResponse searchShardsResponse = client().admin().cluster().prepareSearchShards(indexName1, indexName2).get(); + assertEquals(internalCluster().numDataNodes(), searchShardsResponse.getNodes().length); + TransportNodesListGatewayStartedShardsBatch.NodesGatewayStartedShardsBatch response; + response = ActionTestUtils.executeBlocking( + internalCluster().getInstance(TransportNodesListGatewayStartedShardsBatch.class), + new TransportNodesListGatewayStartedShardsBatch.Request(searchShardsResponse.getNodes(), shardIdCustomDataPathMap) + ); + for (ClusterSearchShardsGroup clusterSearchShardsGroup : searchShardsResponse.getGroups()) { + ShardId shardId = clusterSearchShardsGroup.getShardId(); + assertEquals(1, clusterSearchShardsGroup.getShards().length); + String nodeId = clusterSearchShardsGroup.getShards()[0].currentNodeId(); + TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShards nodeGatewayStartedShards = response.getNodesMap() + .get(nodeId) + .getNodeGatewayStartedShardsBatch() + .get(shardId); + assertNodeGatewayStartedShardsHappyCase(nodeGatewayStartedShards); + } + } + + public void testShardFetchCorruptedShards() throws Exception { + String indexName = "test"; + Map shardIdCustomDataPathMap = prepareRequestMap(new String[] { indexName }, 1); + ClusterSearchShardsResponse searchShardsResponse = client().admin().cluster().prepareSearchShards(indexName).get(); + final Index index = resolveIndex(indexName); + final ShardId shardId = new ShardId(index, 0); + corruptShard(searchShardsResponse.getNodes()[0].getName(), shardId); + TransportNodesListGatewayStartedShardsBatch.NodesGatewayStartedShardsBatch response; + internalCluster().restartNode(searchShardsResponse.getNodes()[0].getName()); + response = ActionTestUtils.executeBlocking( + internalCluster().getInstance(TransportNodesListGatewayStartedShardsBatch.class), + new TransportNodesListGatewayStartedShardsBatch.Request(getDiscoveryNodes(), shardIdCustomDataPathMap) + ); + DiscoveryNode[] discoveryNodes = getDiscoveryNodes(); + TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShards nodeGatewayStartedShards = response.getNodesMap() + .get(discoveryNodes[0].getId()) + .getNodeGatewayStartedShardsBatch() + .get(shardId); + assertNotNull(nodeGatewayStartedShards.storeException()); + assertNotNull(nodeGatewayStartedShards.allocationId()); + assertTrue(nodeGatewayStartedShards.primary()); + } + + private DiscoveryNode[] getDiscoveryNodes() throws ExecutionException, InterruptedException { + final ClusterStateRequest clusterStateRequest = new ClusterStateRequest(); + clusterStateRequest.local(false); + clusterStateRequest.clear().nodes(true).routingTable(true).indices("*"); + ClusterStateResponse clusterStateResponse = client().admin().cluster().state(clusterStateRequest).get(); + final List nodes = new LinkedList<>(clusterStateResponse.getState().nodes().getDataNodes().values()); + DiscoveryNode[] disNodesArr = new DiscoveryNode[nodes.size()]; + nodes.toArray(disNodesArr); + return disNodesArr; + } + + private void assertNodeGatewayStartedShardsHappyCase( + TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShards nodeGatewayStartedShards + ) { + assertNull(nodeGatewayStartedShards.storeException()); + assertNotNull(nodeGatewayStartedShards.allocationId()); + assertTrue(nodeGatewayStartedShards.primary()); + } + + private void prepareIndex(String indexName, int numberOfPrimaryShards) { + createIndex( + indexName, + Settings.builder().put(SETTING_NUMBER_OF_SHARDS, numberOfPrimaryShards).put(SETTING_NUMBER_OF_REPLICAS, 0).build() + ); + index(indexName, "type", "1"); + flush(indexName); + } + + private Map prepareRequestMap(String[] indices, int primaryShardCount) { + Map shardIdCustomDataPathMap = new HashMap<>(); + for (String indexName : indices) { + prepareIndex(indexName, primaryShardCount); + final Index index = resolveIndex(indexName); + final String customDataPath = IndexMetadata.INDEX_DATA_PATH_SETTING.get( + client().admin().indices().prepareGetSettings(indexName).get().getIndexToSettings().get(indexName) + ); + for (int shardIdNum = 0; shardIdNum < primaryShardCount; shardIdNum++) { + final ShardId shardId = new ShardId(index, shardIdNum); + shardIdCustomDataPathMap.put(shardId, customDataPath); + } + } + return shardIdCustomDataPathMap; + } + + private void corruptShard(String nodeName, ShardId shardId) throws IOException, InterruptedException { + for (Path path : internalCluster().getInstance(NodeEnvironment.class, nodeName).availableShardPaths(shardId)) { + final Path indexPath = path.resolve(ShardPath.INDEX_FOLDER_NAME); + if (Files.exists(indexPath)) { // multi data path might only have one path in use + try (DirectoryStream stream = Files.newDirectoryStream(indexPath)) { + for (Path item : stream) { + if (item.getFileName().toString().startsWith("segments_")) { + logger.info("--> deleting [{}]", item); + Files.delete(item); + } + } + } + } + } + } +} diff --git a/server/src/main/java/org/opensearch/gateway/GatewayModule.java b/server/src/main/java/org/opensearch/gateway/GatewayModule.java index 59ec0243c88c9..847ba01737332 100644 --- a/server/src/main/java/org/opensearch/gateway/GatewayModule.java +++ b/server/src/main/java/org/opensearch/gateway/GatewayModule.java @@ -47,6 +47,7 @@ protected void configure() { bind(GatewayService.class).asEagerSingleton(); bind(TransportNodesListGatewayMetaState.class).asEagerSingleton(); bind(TransportNodesListGatewayStartedShards.class).asEagerSingleton(); + bind(TransportNodesListGatewayStartedShardsBatch.class).asEagerSingleton(); bind(LocalAllocateDangledIndices.class).asEagerSingleton(); } } From 8404fc8b52a9f1163885c75e7a8e5d41404e9d26 Mon Sep 17 00:00:00 2001 From: sudarshan baliga Date: Thu, 3 Aug 2023 17:55:52 +0530 Subject: [PATCH 03/21] Refactor PSA Async batch shard fetch transport integ test Signed-off-by: sudarshan baliga --- .../AsyncShardFetchBatchTestUtils.java | 76 +++++++++ ...tNodesListGatewayStartedShardsBatchIT.java | 154 ++++++++---------- 2 files changed, 144 insertions(+), 86 deletions(-) create mode 100644 server/src/internalClusterTest/java/org/opensearch/gateway/AsyncShardFetchBatchTestUtils.java diff --git a/server/src/internalClusterTest/java/org/opensearch/gateway/AsyncShardFetchBatchTestUtils.java b/server/src/internalClusterTest/java/org/opensearch/gateway/AsyncShardFetchBatchTestUtils.java new file mode 100644 index 0000000000000..3f6303378b911 --- /dev/null +++ b/server/src/internalClusterTest/java/org/opensearch/gateway/AsyncShardFetchBatchTestUtils.java @@ -0,0 +1,76 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.gateway; + +import org.opensearch.action.admin.cluster.state.ClusterStateRequest; +import org.opensearch.action.admin.cluster.state.ClusterStateResponse; +import org.opensearch.cluster.metadata.IndexMetadata; +import org.opensearch.cluster.node.DiscoveryNode; +import org.opensearch.core.index.Index; +import org.opensearch.core.index.shard.ShardId; +import org.opensearch.env.NodeEnvironment; +import org.opensearch.index.shard.ShardPath; + +import java.io.IOException; +import java.nio.file.DirectoryStream; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.HashMap; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ExecutionException; + +import static org.opensearch.test.OpenSearchIntegTestCase.client; +import static org.opensearch.test.OpenSearchIntegTestCase.internalCluster; +import static org.opensearch.test.OpenSearchIntegTestCase.resolveIndex; + +public class AsyncShardFetchBatchTestUtils { + + public static DiscoveryNode[] getDiscoveryNodes() throws ExecutionException, InterruptedException { + final ClusterStateRequest clusterStateRequest = new ClusterStateRequest(); + clusterStateRequest.local(false); + clusterStateRequest.clear().nodes(true).routingTable(true).indices("*"); + ClusterStateResponse clusterStateResponse = client().admin().cluster().state(clusterStateRequest).get(); + final List nodes = new LinkedList<>(clusterStateResponse.getState().nodes().getDataNodes().values()); + DiscoveryNode[] disNodesArr = new DiscoveryNode[nodes.size()]; + nodes.toArray(disNodesArr); + return disNodesArr; + } + + public static Map prepareRequestMap(String[] indices, int shardCount) { + Map shardIdCustomDataPathMap = new HashMap<>(); + for (String indexName : indices) { + final Index index = resolveIndex(indexName); + final String customDataPath = IndexMetadata.INDEX_DATA_PATH_SETTING.get( + client().admin().indices().prepareGetSettings(indexName).get().getIndexToSettings().get(indexName) + ); + for (int shardIdNum = 0; shardIdNum < shardCount; shardIdNum++) { + final ShardId shardId = new ShardId(index, shardIdNum); + shardIdCustomDataPathMap.put(shardId, customDataPath); + } + } + return shardIdCustomDataPathMap; + } + + public static void corruptShard(String nodeName, ShardId shardId) throws IOException, InterruptedException { + for (Path path : internalCluster().getInstance(NodeEnvironment.class, nodeName).availableShardPaths(shardId)) { + final Path indexPath = path.resolve(ShardPath.INDEX_FOLDER_NAME); + if (Files.exists(indexPath)) { // multi data path might only have one path in use + try (DirectoryStream stream = Files.newDirectoryStream(indexPath)) { + for (Path item : stream) { + if (item.getFileName().toString().startsWith("segments_")) { + Files.delete(item); + } + } + } + } + } + } +} diff --git a/server/src/internalClusterTest/java/org/opensearch/gateway/TransportNodesListGatewayStartedShardsBatchIT.java b/server/src/internalClusterTest/java/org/opensearch/gateway/TransportNodesListGatewayStartedShardsBatchIT.java index 6c53333843834..0b34f7fb4f9ef 100644 --- a/server/src/internalClusterTest/java/org/opensearch/gateway/TransportNodesListGatewayStartedShardsBatchIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/gateway/TransportNodesListGatewayStartedShardsBatchIT.java @@ -8,86 +8,87 @@ package org.opensearch.gateway; +import org.opensearch.Version; import org.opensearch.action.admin.cluster.shards.ClusterSearchShardsGroup; import org.opensearch.action.admin.cluster.shards.ClusterSearchShardsResponse; -import org.opensearch.action.admin.cluster.state.ClusterStateRequest; -import org.opensearch.action.admin.cluster.state.ClusterStateResponse; import org.opensearch.action.support.ActionTestUtils; -import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.cluster.node.DiscoveryNode; import org.opensearch.common.settings.Settings; import org.opensearch.core.index.Index; import org.opensearch.core.index.shard.ShardId; -import org.opensearch.env.NodeEnvironment; -import org.opensearch.index.shard.ShardPath; import org.opensearch.test.OpenSearchIntegTestCase; -import java.io.IOException; -import java.nio.file.DirectoryStream; -import java.nio.file.Files; -import java.nio.file.Path; -import java.util.HashMap; -import java.util.LinkedList; -import java.util.List; import java.util.Map; -import java.util.concurrent.ExecutionException; +import static java.util.Collections.emptyMap; +import static java.util.Collections.emptySet; import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_NUMBER_OF_REPLICAS; import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_NUMBER_OF_SHARDS; +import static org.opensearch.gateway.AsyncShardFetchBatchTestUtils.corruptShard; +import static org.opensearch.gateway.AsyncShardFetchBatchTestUtils.getDiscoveryNodes; +import static org.opensearch.gateway.AsyncShardFetchBatchTestUtils.prepareRequestMap; public class TransportNodesListGatewayStartedShardsBatchIT extends OpenSearchIntegTestCase { public void testSingleShardFetch() throws Exception { String indexName = "test"; - Map shardIdCustomDataPathMap = prepareRequestMap(new String[] { indexName }, 1); - - ClusterSearchShardsResponse searchShardsResponse = client().admin().cluster().prepareSearchShards(indexName).get(); - - TransportNodesListGatewayStartedShardsBatch.NodesGatewayStartedShardsBatch response; - response = ActionTestUtils.executeBlocking( - internalCluster().getInstance(TransportNodesListGatewayStartedShardsBatch.class), - new TransportNodesListGatewayStartedShardsBatch.Request(searchShardsResponse.getNodes(), shardIdCustomDataPathMap) + prepareIndices(new String[] { indexName }, 1, 0); + DiscoveryNode[] nodes = getDiscoveryNodes(); + TransportNodesListGatewayStartedShardsBatch.NodesGatewayStartedShardsBatch nodesGatewayStartedShardsBatch = prepareAndSendRequest( + new String[] { indexName }, + nodes ); + final Index index = resolveIndex(indexName); final ShardId shardId = new ShardId(index, 0); - TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShards nodeGatewayStartedShards = response.getNodesMap() - .get(searchShardsResponse.getNodes()[0].getId()) + TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShards nodeGatewayStartedShards = nodesGatewayStartedShardsBatch + .getNodesMap() + .get(nodes[0].getId()) .getNodeGatewayStartedShardsBatch() .get(shardId); - assertNodeGatewayStartedShardsHappyCase(nodeGatewayStartedShards); + assertNodeGatewayStartedShardsSuccessCase(nodeGatewayStartedShards); } public void testShardFetchMultiNodeMultiIndexes() throws Exception { // start second node internalCluster().startNode(); - String indexName1 = "test1"; - String indexName2 = "test2"; - // assign one primary shard each to the data nodes - Map shardIdCustomDataPathMap = prepareRequestMap( - new String[] { indexName1, indexName2 }, - internalCluster().numDataNodes() - ); - ClusterSearchShardsResponse searchShardsResponse = client().admin().cluster().prepareSearchShards(indexName1, indexName2).get(); - assertEquals(internalCluster().numDataNodes(), searchShardsResponse.getNodes().length); - TransportNodesListGatewayStartedShardsBatch.NodesGatewayStartedShardsBatch response; - response = ActionTestUtils.executeBlocking( - internalCluster().getInstance(TransportNodesListGatewayStartedShardsBatch.class), - new TransportNodesListGatewayStartedShardsBatch.Request(searchShardsResponse.getNodes(), shardIdCustomDataPathMap) + String[] indices = new String[] { "index1", "index2" }; + prepareIndices(indices, 1, 0); + DiscoveryNode[] nodes = getDiscoveryNodes(); + TransportNodesListGatewayStartedShardsBatch.NodesGatewayStartedShardsBatch nodesGatewayStartedShardsBatch = prepareAndSendRequest( + indices, + nodes ); + ClusterSearchShardsResponse searchShardsResponse = client().admin().cluster().prepareSearchShards(indices).get(); for (ClusterSearchShardsGroup clusterSearchShardsGroup : searchShardsResponse.getGroups()) { ShardId shardId = clusterSearchShardsGroup.getShardId(); assertEquals(1, clusterSearchShardsGroup.getShards().length); String nodeId = clusterSearchShardsGroup.getShards()[0].currentNodeId(); - TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShards nodeGatewayStartedShards = response.getNodesMap() + TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShards nodeGatewayStartedShards = nodesGatewayStartedShardsBatch + .getNodesMap() .get(nodeId) .getNodeGatewayStartedShardsBatch() .get(shardId); - assertNodeGatewayStartedShardsHappyCase(nodeGatewayStartedShards); + assertNodeGatewayStartedShardsSuccessCase(nodeGatewayStartedShards); } } + public void testShardFetchNodeNotConnected() { + DiscoveryNode nonExistingNode = new DiscoveryNode("foo", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT); + String indexName = "test"; + prepareIndices(new String[] { indexName }, 1, 0); + TransportNodesListGatewayStartedShardsBatch.NodesGatewayStartedShardsBatch nodesGatewayStartedShardsBatch = prepareAndSendRequest( + new String[] { indexName }, + new DiscoveryNode[] { nonExistingNode } + ); + assertTrue(nodesGatewayStartedShardsBatch.hasFailures()); + assertEquals(1, nodesGatewayStartedShardsBatch.failures().size()); + assertEquals(nonExistingNode.getId(), nodesGatewayStartedShardsBatch.failures().get(0).nodeId()); + } + public void testShardFetchCorruptedShards() throws Exception { String indexName = "test"; + prepareIndices(new String[] { indexName }, 1, 0); Map shardIdCustomDataPathMap = prepareRequestMap(new String[] { indexName }, 1); ClusterSearchShardsResponse searchShardsResponse = client().admin().cluster().prepareSearchShards(indexName).get(); final Index index = resolveIndex(indexName); @@ -104,23 +105,22 @@ public void testShardFetchCorruptedShards() throws Exception { .get(discoveryNodes[0].getId()) .getNodeGatewayStartedShardsBatch() .get(shardId); - assertNotNull(nodeGatewayStartedShards.storeException()); - assertNotNull(nodeGatewayStartedShards.allocationId()); - assertTrue(nodeGatewayStartedShards.primary()); + assertNodeGatewayStartedShardsFailureCase(nodeGatewayStartedShards); } - private DiscoveryNode[] getDiscoveryNodes() throws ExecutionException, InterruptedException { - final ClusterStateRequest clusterStateRequest = new ClusterStateRequest(); - clusterStateRequest.local(false); - clusterStateRequest.clear().nodes(true).routingTable(true).indices("*"); - ClusterStateResponse clusterStateResponse = client().admin().cluster().state(clusterStateRequest).get(); - final List nodes = new LinkedList<>(clusterStateResponse.getState().nodes().getDataNodes().values()); - DiscoveryNode[] disNodesArr = new DiscoveryNode[nodes.size()]; - nodes.toArray(disNodesArr); - return disNodesArr; + private TransportNodesListGatewayStartedShardsBatch.NodesGatewayStartedShardsBatch prepareAndSendRequest( + String[] indices, + DiscoveryNode[] nodes + ) { + Map shardIdCustomDataPathMap = prepareRequestMap(indices, 1); + TransportNodesListGatewayStartedShardsBatch.NodesGatewayStartedShardsBatch response; + return ActionTestUtils.executeBlocking( + internalCluster().getInstance(TransportNodesListGatewayStartedShardsBatch.class), + new TransportNodesListGatewayStartedShardsBatch.Request(nodes, shardIdCustomDataPathMap) + ); } - private void assertNodeGatewayStartedShardsHappyCase( + private void assertNodeGatewayStartedShardsSuccessCase( TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShards nodeGatewayStartedShards ) { assertNull(nodeGatewayStartedShards.storeException()); @@ -128,44 +128,26 @@ private void assertNodeGatewayStartedShardsHappyCase( assertTrue(nodeGatewayStartedShards.primary()); } - private void prepareIndex(String indexName, int numberOfPrimaryShards) { - createIndex( - indexName, - Settings.builder().put(SETTING_NUMBER_OF_SHARDS, numberOfPrimaryShards).put(SETTING_NUMBER_OF_REPLICAS, 0).build() - ); - index(indexName, "type", "1"); - flush(indexName); + private void assertNodeGatewayStartedShardsFailureCase( + TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShards nodeGatewayStartedShards + ) { + assertNotNull(nodeGatewayStartedShards.storeException()); + assertNotNull(nodeGatewayStartedShards.allocationId()); + assertTrue(nodeGatewayStartedShards.primary()); } - private Map prepareRequestMap(String[] indices, int primaryShardCount) { - Map shardIdCustomDataPathMap = new HashMap<>(); - for (String indexName : indices) { - prepareIndex(indexName, primaryShardCount); - final Index index = resolveIndex(indexName); - final String customDataPath = IndexMetadata.INDEX_DATA_PATH_SETTING.get( - client().admin().indices().prepareGetSettings(indexName).get().getIndexToSettings().get(indexName) + private void prepareIndices(String[] indices, int numberOfPrimaryShards, int numberOfReplicaShards) { + for (String index : indices) { + createIndex( + index, + Settings.builder() + .put(SETTING_NUMBER_OF_SHARDS, numberOfPrimaryShards) + .put(SETTING_NUMBER_OF_REPLICAS, numberOfReplicaShards) + .build() ); - for (int shardIdNum = 0; shardIdNum < primaryShardCount; shardIdNum++) { - final ShardId shardId = new ShardId(index, shardIdNum); - shardIdCustomDataPathMap.put(shardId, customDataPath); - } + index(index, "type", "1"); + flush(index); } - return shardIdCustomDataPathMap; } - private void corruptShard(String nodeName, ShardId shardId) throws IOException, InterruptedException { - for (Path path : internalCluster().getInstance(NodeEnvironment.class, nodeName).availableShardPaths(shardId)) { - final Path indexPath = path.resolve(ShardPath.INDEX_FOLDER_NAME); - if (Files.exists(indexPath)) { // multi data path might only have one path in use - try (DirectoryStream stream = Files.newDirectoryStream(indexPath)) { - for (Path item : stream) { - if (item.getFileName().toString().startsWith("segments_")) { - logger.info("--> deleting [{}]", item); - Files.delete(item); - } - } - } - } - } - } } From d80f441694263488cc1de1ccc76ad687707b1ecb Mon Sep 17 00:00:00 2001 From: sudarshan baliga Date: Sun, 6 Aug 2023 14:43:51 +0530 Subject: [PATCH 04/21] Update the documentation of TransportNodesListGatewayStartedShardsBatch. Update variable name of AsyncBatchShardFetch Signed-off-by: sudarshan baliga --- .../org/opensearch/gateway/AsyncBatchShardFetch.java | 2 +- .../TransportNodesListGatewayStartedShardsBatch.java | 10 +++++----- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/server/src/main/java/org/opensearch/gateway/AsyncBatchShardFetch.java b/server/src/main/java/org/opensearch/gateway/AsyncBatchShardFetch.java index 4872d75e13740..dcd09ed1eabf1 100644 --- a/server/src/main/java/org/opensearch/gateway/AsyncBatchShardFetch.java +++ b/server/src/main/java/org/opensearch/gateway/AsyncBatchShardFetch.java @@ -29,6 +29,6 @@ public abstract class AsyncBatchShardFetch implement * An action that lists the relevant shard data that needs to be fetched. */ public interface Lister, NodeResponse extends BaseNodeResponse> { - void list(DiscoveryNode[] nodes, Map shardIdsWithCustomDataPath, ActionListener listener); + void list(DiscoveryNode[] nodes, Map shardToCustomDataPath, ActionListener listener); } } diff --git a/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedShardsBatch.java b/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedShardsBatch.java index 3bd2860dea094..60510b49bcaf2 100644 --- a/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedShardsBatch.java +++ b/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedShardsBatch.java @@ -48,7 +48,7 @@ import java.util.Objects; /** - * This transport action is used to fetch all unassigned shard version from each node during primary allocation in {@link GatewayAllocator}. + * This transport action is used to fetch batch of unassigned shard version from each node during primary allocation in {@link GatewayAllocator}. * We use this to find out which node holds the latest shard version and which of them used to be a primary in order to allocate * shards after node or cluster restarts. * @@ -132,11 +132,11 @@ protected NodesGatewayStartedShardsBatch newResponse( } /** - * This function is similar to nodeoperation method of {@link TransportNodesListGatewayStartedShards} we loop over - * the shards here to fetch the shard result in bulk. + * This function is similar to nodeOperation method of {@link TransportNodesListGatewayStartedShards} we loop over + * the shards here and populate the data about the shards held by the local node. * - * @param request - * @return NodeGatewayStartedShardsBatch + * @param request Request containing the map shardIdsWithCustomDataPath. + * @return NodeGatewayStartedShardsBatch contains the data about the primary shards held by the local node */ @Override protected NodeGatewayStartedShardsBatch nodeOperation(NodeRequest request) { From 06e124361ab15355fe88ac9dde659a28434c2d07 Mon Sep 17 00:00:00 2001 From: Shivansh Arora Date: Tue, 26 Sep 2023 04:20:40 +0530 Subject: [PATCH 05/21] Transport PSA change Signed-off-by: Shivansh Arora --- ...ortNodesListGatewayStartedShardsBatch.java | 32 ++--- .../indices/store/ShardAttributes.java | 52 ++++++++ .../indices/store/StoreFilesMetadata.java | 115 ++++++++++++++++++ 3 files changed, 184 insertions(+), 15 deletions(-) create mode 100644 server/src/main/java/org/opensearch/indices/store/ShardAttributes.java create mode 100644 server/src/main/java/org/opensearch/indices/store/StoreFilesMetadata.java diff --git a/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedShardsBatch.java b/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedShardsBatch.java index 60510b49bcaf2..2762a98c18fd0 100644 --- a/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedShardsBatch.java +++ b/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedShardsBatch.java @@ -37,6 +37,7 @@ import org.opensearch.index.store.Store; import org.opensearch.indices.IndicesService; import org.opensearch.indices.replication.checkpoint.ReplicationCheckpoint; +import org.opensearch.indices.store.ShardAttributes; import org.opensearch.threadpool.ThreadPool; import org.opensearch.transport.TransportRequest; import org.opensearch.transport.TransportService; @@ -46,6 +47,7 @@ import java.util.List; import java.util.Map; import java.util.Objects; +import java.util.stream.Collectors; /** * This transport action is used to fetch batch of unassigned shard version from each node during primary allocation in {@link GatewayAllocator}. @@ -141,8 +143,8 @@ protected NodesGatewayStartedShardsBatch newResponse( @Override protected NodeGatewayStartedShardsBatch nodeOperation(NodeRequest request) { Map shardsOnNode = new HashMap<>(); - for (Map.Entry shardToCustomDataPathEntry : request.shardIdsWithCustomDataPath.entrySet()) { - final ShardId shardId = shardToCustomDataPathEntry.getKey(); + for (ShardAttributes shardAttr: request.shardAttributes) { + final ShardId shardId = shardAttr.getShardId(); try { logger.trace("{} loading local shard state info", shardId); ShardStateMetadata shardStateMetadata = ShardStateMetadata.FORMAT.loadLatestState( @@ -154,8 +156,8 @@ protected NodeGatewayStartedShardsBatch nodeOperation(NodeRequest request) { if (indicesService.getShardOrNull(shardId) == null && shardStateMetadata.indexDataLocation == ShardStateMetadata.IndexDataLocation.LOCAL) { final String customDataPath; - if (shardToCustomDataPathEntry.getValue() != null) { - customDataPath = shardToCustomDataPathEntry.getValue(); + if (shardAttr.getCustomDataPath() != null) { + customDataPath = shardAttr.getCustomDataPath(); } else { // TODO: Fallback for BWC with older OpenSearch versions. // Remove once request.getCustomDataPath() always returns non-null @@ -227,26 +229,27 @@ protected NodeGatewayStartedShardsBatch nodeOperation(NodeRequest request) { * @opensearch.internal */ public static class Request extends BaseNodesRequest { - private final Map shardIdsWithCustomDataPath; + private final List shardAttributes; public Request(StreamInput in) throws IOException { super(in); - shardIdsWithCustomDataPath = in.readMap(ShardId::new, StreamInput::readString); + shardAttributes = in.readList(ShardAttributes::new); } public Request(DiscoveryNode[] nodes, Map shardIdStringMap) { super(nodes); - this.shardIdsWithCustomDataPath = Objects.requireNonNull(shardIdStringMap); + this.shardAttributes = Objects.requireNonNull(shardIdStringMap).entrySet().stream().map(entry -> + new ShardAttributes(entry.getKey(), entry.getValue())).collect(Collectors.toList()); } @Override public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); - out.writeMap(shardIdsWithCustomDataPath, (o, k) -> k.writeTo(o), StreamOutput::writeString); + out.writeList(shardAttributes); } - public Map getShardIdsMap() { - return shardIdsWithCustomDataPath; + public List getShardAttributes() { + return shardAttributes; } } @@ -288,23 +291,22 @@ protected void writeNodesTo(StreamOutput out, List shardIdsWithCustomDataPath; + private final List shardAttributes; public NodeRequest(StreamInput in) throws IOException { super(in); - shardIdsWithCustomDataPath = in.readMap(ShardId::new, StreamInput::readString); + shardAttributes = in.readList(ShardAttributes::new); } public NodeRequest(Request request) { - this.shardIdsWithCustomDataPath = Objects.requireNonNull(request.getShardIdsMap()); + this.shardAttributes = Objects.requireNonNull(request.getShardAttributes()); } @Override public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); - out.writeMap(shardIdsWithCustomDataPath, (o, k) -> k.writeTo(o), StreamOutput::writeString); + out.writeList(shardAttributes); } - } /** diff --git a/server/src/main/java/org/opensearch/indices/store/ShardAttributes.java b/server/src/main/java/org/opensearch/indices/store/ShardAttributes.java new file mode 100644 index 0000000000000..6ed65f33dd0b6 --- /dev/null +++ b/server/src/main/java/org/opensearch/indices/store/ShardAttributes.java @@ -0,0 +1,52 @@ +/* + * 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.indices.store; + +import org.opensearch.common.Nullable; +import org.opensearch.core.common.io.stream.StreamInput; +import org.opensearch.core.common.io.stream.StreamOutput; +import org.opensearch.core.common.io.stream.Writeable; +import org.opensearch.core.index.shard.ShardId; + +import java.io.IOException; + +public class ShardAttributes implements Writeable { + private final ShardId shardId; + @Nullable + private final String customDataPath; + + public ShardAttributes(ShardId shardId, String customDataPath) { + this.shardId = shardId; + this.customDataPath = customDataPath; + } + + public ShardAttributes(StreamInput in) throws IOException { + shardId = new ShardId(in); + customDataPath = in.readString(); + } + + public ShardId getShardId() { + return shardId; + } + + /** + * Returns the custom data path that is used to look up information for this shard. + * Returns an empty string if no custom data path is used for this index. + * Returns null if custom data path information is not available (due to BWC). + */ + @Nullable + public String getCustomDataPath() { + return customDataPath; + } + + public void writeTo(StreamOutput out) throws IOException { + shardId.writeTo(out); + out.writeString(customDataPath); + } +} diff --git a/server/src/main/java/org/opensearch/indices/store/StoreFilesMetadata.java b/server/src/main/java/org/opensearch/indices/store/StoreFilesMetadata.java new file mode 100644 index 0000000000000..96e0589c8df9b --- /dev/null +++ b/server/src/main/java/org/opensearch/indices/store/StoreFilesMetadata.java @@ -0,0 +1,115 @@ +/* + * 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.indices.store; + +import org.opensearch.cluster.node.DiscoveryNode; +import org.opensearch.core.common.io.stream.StreamInput; +import org.opensearch.core.common.io.stream.StreamOutput; +import org.opensearch.core.common.io.stream.Writeable; +import org.opensearch.core.index.shard.ShardId; +import org.opensearch.index.seqno.ReplicationTracker; +import org.opensearch.index.seqno.RetentionLease; +import org.opensearch.index.store.Store; +import org.opensearch.index.store.StoreFileMetadata; + +import java.io.IOException; +import java.util.Iterator; +import java.util.List; + +/** + * Metadata for store files + * + * @opensearch.internal + */ +public class StoreFilesMetadata implements Iterable, Writeable { + private final ShardId shardId; + private final Store.MetadataSnapshot metadataSnapshot; + private final List peerRecoveryRetentionLeases; + + public StoreFilesMetadata( + ShardId shardId, + Store.MetadataSnapshot metadataSnapshot, + List peerRecoveryRetentionLeases + ) { + this.shardId = shardId; + this.metadataSnapshot = metadataSnapshot; + this.peerRecoveryRetentionLeases = peerRecoveryRetentionLeases; + } + + public StoreFilesMetadata(StreamInput in) throws IOException { + this.shardId = new ShardId(in); + this.metadataSnapshot = new Store.MetadataSnapshot(in); + this.peerRecoveryRetentionLeases = in.readList(RetentionLease::new); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + shardId.writeTo(out); + metadataSnapshot.writeTo(out); + out.writeList(peerRecoveryRetentionLeases); + } + + public ShardId shardId() { + return this.shardId; + } + + public boolean isEmpty() { + return metadataSnapshot.size() == 0; + } + + @Override + public Iterator iterator() { + return metadataSnapshot.iterator(); + } + + public boolean fileExists(String name) { + return metadataSnapshot.asMap().containsKey(name); + } + + public StoreFileMetadata file(String name) { + return metadataSnapshot.asMap().get(name); + } + + /** + * Returns the retaining sequence number of the peer recovery retention lease for a given node if exists; otherwise, returns -1. + */ + public long getPeerRecoveryRetentionLeaseRetainingSeqNo(DiscoveryNode node) { + assert node != null; + final String retentionLeaseId = ReplicationTracker.getPeerRecoveryRetentionLeaseId(node.getId()); + return peerRecoveryRetentionLeases.stream() + .filter(lease -> lease.id().equals(retentionLeaseId)) + .mapToLong(RetentionLease::retainingSequenceNumber) + .findFirst() + .orElse(-1L); + } + + public List peerRecoveryRetentionLeases() { + return peerRecoveryRetentionLeases; + } + + /** + * @return commit sync id if exists, else null + */ + public String syncId() { + return metadataSnapshot.getSyncId(); + } + + @Override + public String toString() { + return "StoreFilesMetadata{" + + ", shardId=" + + shardId + + ", metadataSnapshot{size=" + + metadataSnapshot.size() + + ", syncId=" + + metadataSnapshot.getSyncId() + + "}" + + '}'; + } +} From 933bcfcf16b8ef6a2c69b8470649530eccdc91f4 Mon Sep 17 00:00:00 2001 From: Shivansh Arora Date: Tue, 26 Sep 2023 04:30:30 +0530 Subject: [PATCH 06/21] Renamed Transport File to TransportNodesListGatewayStartedBatchShards Signed-off-by: Shivansh Arora --- server/src/main/java/org/opensearch/gateway/GatewayModule.java | 2 +- ...ch.java => TransportNodesListGatewayStartedBatchShards.java} | 0 2 files changed, 1 insertion(+), 1 deletion(-) rename server/src/main/java/org/opensearch/gateway/{TransportNodesListGatewayStartedShardsBatch.java => TransportNodesListGatewayStartedBatchShards.java} (100%) diff --git a/server/src/main/java/org/opensearch/gateway/GatewayModule.java b/server/src/main/java/org/opensearch/gateway/GatewayModule.java index 847ba01737332..e6c3176e8adbf 100644 --- a/server/src/main/java/org/opensearch/gateway/GatewayModule.java +++ b/server/src/main/java/org/opensearch/gateway/GatewayModule.java @@ -47,7 +47,7 @@ protected void configure() { bind(GatewayService.class).asEagerSingleton(); bind(TransportNodesListGatewayMetaState.class).asEagerSingleton(); bind(TransportNodesListGatewayStartedShards.class).asEagerSingleton(); - bind(TransportNodesListGatewayStartedShardsBatch.class).asEagerSingleton(); + bind(TransportNodesListGatewayStartedBatchShards.class).asEagerSingleton(); bind(LocalAllocateDangledIndices.class).asEagerSingleton(); } } diff --git a/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedShardsBatch.java b/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedBatchShards.java similarity index 100% rename from server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedShardsBatch.java rename to server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedBatchShards.java From 1c381dc0d5f8620ed5276d8770e080bb1c5731d9 Mon Sep 17 00:00:00 2001 From: Shivansh Arora Date: Tue, 26 Sep 2023 14:44:29 +0530 Subject: [PATCH 07/21] Removed AsyncBatchShardFetch Signed-off-by: Shivansh Arora --- .../gateway/AsyncBatchShardFetch.java | 34 ------------------- ...ortNodesListGatewayStartedBatchShards.java | 4 +-- ...ransportNodesListGatewayStartedShards.java | 6 +++- 3 files changed, 7 insertions(+), 37 deletions(-) delete mode 100644 server/src/main/java/org/opensearch/gateway/AsyncBatchShardFetch.java diff --git a/server/src/main/java/org/opensearch/gateway/AsyncBatchShardFetch.java b/server/src/main/java/org/opensearch/gateway/AsyncBatchShardFetch.java deleted file mode 100644 index dcd09ed1eabf1..0000000000000 --- a/server/src/main/java/org/opensearch/gateway/AsyncBatchShardFetch.java +++ /dev/null @@ -1,34 +0,0 @@ -/* - * 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; - -import org.opensearch.action.ActionListener; -import org.opensearch.action.support.nodes.BaseNodeResponse; -import org.opensearch.action.support.nodes.BaseNodesResponse; -import org.opensearch.cluster.node.DiscoveryNode; -import org.opensearch.common.lease.Releasable; -import org.opensearch.core.index.shard.ShardId; - -import java.util.Map; - -/** - * This class is responsible for fetching shard data from nodes. It is analogous to AsyncShardFetch class except - * that we fetch batch of shards in this class from single transport request to a node. - * @param - * - * @opensearch.internal - */ -public abstract class AsyncBatchShardFetch implements Releasable { - /** - * An action that lists the relevant shard data that needs to be fetched. - */ - public interface Lister, NodeResponse extends BaseNodeResponse> { - void list(DiscoveryNode[] nodes, Map shardToCustomDataPath, ActionListener listener); - } -} diff --git a/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedBatchShards.java b/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedBatchShards.java index 2762a98c18fd0..0215088b2f8d0 100644 --- a/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedBatchShards.java +++ b/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedBatchShards.java @@ -62,7 +62,7 @@ public class TransportNodesListGatewayStartedShardsBatch extends TransportNodesA TransportNodesListGatewayStartedShardsBatch.NodeRequest, TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShardsBatch> implements - AsyncBatchShardFetch.Lister< + AsyncShardFetch.Lister< TransportNodesListGatewayStartedShardsBatch.NodesGatewayStartedShardsBatch, TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShardsBatch> { @@ -107,8 +107,8 @@ public TransportNodesListGatewayStartedShardsBatch( @Override public void list( - DiscoveryNode[] nodes, Map shardIdsWithCustomDataPath, + DiscoveryNode[] nodes, ActionListener listener ) { execute(new Request(nodes, shardIdsWithCustomDataPath), listener); diff --git a/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedShards.java b/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedShards.java index e2a3f08bb02c6..4c21746ec9ae3 100644 --- a/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedShards.java +++ b/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedShards.java @@ -68,6 +68,7 @@ import java.io.IOException; import java.util.List; +import java.util.Map; import java.util.Objects; /** @@ -124,7 +125,10 @@ public TransportNodesListGatewayStartedShards( } @Override - public void list(ShardId shardId, String customDataPath, DiscoveryNode[] nodes, ActionListener listener) { + public void list(Map shardIdsWithCustomDataPath, DiscoveryNode[] nodes, ActionListener listener) { + assert shardIdsWithCustomDataPath.size() == 1 : "only one shard should be specified"; + final ShardId shardId = shardIdsWithCustomDataPath.keySet().iterator().next(); + final String customDataPath = shardIdsWithCustomDataPath.get(shardId); execute(new Request(shardId, customDataPath, nodes), listener); } From b8a141d389a61f2ab91681835a5fda1dff6de8c6 Mon Sep 17 00:00:00 2001 From: Shivansh Arora Date: Tue, 26 Sep 2023 16:53:41 +0530 Subject: [PATCH 08/21] Renamed test files Signed-off-by: Shivansh Arora --- ...hBatchTestUtils.java => AsyncShardFetchTestUtils.java} | 2 +- ...=> TransportNodesListGatewayStartedBatchShardsIT.java} | 8 ++++---- 2 files changed, 5 insertions(+), 5 deletions(-) rename server/src/internalClusterTest/java/org/opensearch/gateway/{AsyncShardFetchBatchTestUtils.java => AsyncShardFetchTestUtils.java} (98%) rename server/src/internalClusterTest/java/org/opensearch/gateway/{TransportNodesListGatewayStartedShardsBatchIT.java => TransportNodesListGatewayStartedBatchShardsIT.java} (95%) diff --git a/server/src/internalClusterTest/java/org/opensearch/gateway/AsyncShardFetchBatchTestUtils.java b/server/src/internalClusterTest/java/org/opensearch/gateway/AsyncShardFetchTestUtils.java similarity index 98% rename from server/src/internalClusterTest/java/org/opensearch/gateway/AsyncShardFetchBatchTestUtils.java rename to server/src/internalClusterTest/java/org/opensearch/gateway/AsyncShardFetchTestUtils.java index 3f6303378b911..7e0e61f4232c9 100644 --- a/server/src/internalClusterTest/java/org/opensearch/gateway/AsyncShardFetchBatchTestUtils.java +++ b/server/src/internalClusterTest/java/org/opensearch/gateway/AsyncShardFetchTestUtils.java @@ -31,7 +31,7 @@ import static org.opensearch.test.OpenSearchIntegTestCase.internalCluster; import static org.opensearch.test.OpenSearchIntegTestCase.resolveIndex; -public class AsyncShardFetchBatchTestUtils { +public class AsyncShardFetchTestUtils { public static DiscoveryNode[] getDiscoveryNodes() throws ExecutionException, InterruptedException { final ClusterStateRequest clusterStateRequest = new ClusterStateRequest(); diff --git a/server/src/internalClusterTest/java/org/opensearch/gateway/TransportNodesListGatewayStartedShardsBatchIT.java b/server/src/internalClusterTest/java/org/opensearch/gateway/TransportNodesListGatewayStartedBatchShardsIT.java similarity index 95% rename from server/src/internalClusterTest/java/org/opensearch/gateway/TransportNodesListGatewayStartedShardsBatchIT.java rename to server/src/internalClusterTest/java/org/opensearch/gateway/TransportNodesListGatewayStartedBatchShardsIT.java index 0b34f7fb4f9ef..1a10dfb701073 100644 --- a/server/src/internalClusterTest/java/org/opensearch/gateway/TransportNodesListGatewayStartedShardsBatchIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/gateway/TransportNodesListGatewayStartedBatchShardsIT.java @@ -24,11 +24,11 @@ import static java.util.Collections.emptySet; import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_NUMBER_OF_REPLICAS; import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_NUMBER_OF_SHARDS; -import static org.opensearch.gateway.AsyncShardFetchBatchTestUtils.corruptShard; -import static org.opensearch.gateway.AsyncShardFetchBatchTestUtils.getDiscoveryNodes; -import static org.opensearch.gateway.AsyncShardFetchBatchTestUtils.prepareRequestMap; +import static org.opensearch.gateway.AsyncShardFetchTestUtils.corruptShard; +import static org.opensearch.gateway.AsyncShardFetchTestUtils.getDiscoveryNodes; +import static org.opensearch.gateway.AsyncShardFetchTestUtils.prepareRequestMap; -public class TransportNodesListGatewayStartedShardsBatchIT extends OpenSearchIntegTestCase { +public class TransportNodesListGatewayStartedBatchShardsIT extends OpenSearchIntegTestCase { public void testSingleShardFetch() throws Exception { String indexName = "test"; From 84f63acd79cf6854568b6ff1610ab7a00e4e6370 Mon Sep 17 00:00:00 2001 From: Shivansh Arora Date: Tue, 26 Sep 2023 16:59:25 +0530 Subject: [PATCH 09/21] Removed changes part of other PRs Signed-off-by: Shivansh Arora --- ...ransportNodesListGatewayStartedShards.java | 6 +- .../indices/store/StoreFilesMetadata.java | 115 ------------------ 2 files changed, 1 insertion(+), 120 deletions(-) delete mode 100644 server/src/main/java/org/opensearch/indices/store/StoreFilesMetadata.java diff --git a/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedShards.java b/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedShards.java index 4c21746ec9ae3..e2a3f08bb02c6 100644 --- a/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedShards.java +++ b/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedShards.java @@ -68,7 +68,6 @@ import java.io.IOException; import java.util.List; -import java.util.Map; import java.util.Objects; /** @@ -125,10 +124,7 @@ public TransportNodesListGatewayStartedShards( } @Override - public void list(Map shardIdsWithCustomDataPath, DiscoveryNode[] nodes, ActionListener listener) { - assert shardIdsWithCustomDataPath.size() == 1 : "only one shard should be specified"; - final ShardId shardId = shardIdsWithCustomDataPath.keySet().iterator().next(); - final String customDataPath = shardIdsWithCustomDataPath.get(shardId); + public void list(ShardId shardId, String customDataPath, DiscoveryNode[] nodes, ActionListener listener) { execute(new Request(shardId, customDataPath, nodes), listener); } diff --git a/server/src/main/java/org/opensearch/indices/store/StoreFilesMetadata.java b/server/src/main/java/org/opensearch/indices/store/StoreFilesMetadata.java deleted file mode 100644 index 96e0589c8df9b..0000000000000 --- a/server/src/main/java/org/opensearch/indices/store/StoreFilesMetadata.java +++ /dev/null @@ -1,115 +0,0 @@ -/* - * 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.indices.store; - -import org.opensearch.cluster.node.DiscoveryNode; -import org.opensearch.core.common.io.stream.StreamInput; -import org.opensearch.core.common.io.stream.StreamOutput; -import org.opensearch.core.common.io.stream.Writeable; -import org.opensearch.core.index.shard.ShardId; -import org.opensearch.index.seqno.ReplicationTracker; -import org.opensearch.index.seqno.RetentionLease; -import org.opensearch.index.store.Store; -import org.opensearch.index.store.StoreFileMetadata; - -import java.io.IOException; -import java.util.Iterator; -import java.util.List; - -/** - * Metadata for store files - * - * @opensearch.internal - */ -public class StoreFilesMetadata implements Iterable, Writeable { - private final ShardId shardId; - private final Store.MetadataSnapshot metadataSnapshot; - private final List peerRecoveryRetentionLeases; - - public StoreFilesMetadata( - ShardId shardId, - Store.MetadataSnapshot metadataSnapshot, - List peerRecoveryRetentionLeases - ) { - this.shardId = shardId; - this.metadataSnapshot = metadataSnapshot; - this.peerRecoveryRetentionLeases = peerRecoveryRetentionLeases; - } - - public StoreFilesMetadata(StreamInput in) throws IOException { - this.shardId = new ShardId(in); - this.metadataSnapshot = new Store.MetadataSnapshot(in); - this.peerRecoveryRetentionLeases = in.readList(RetentionLease::new); - } - - @Override - public void writeTo(StreamOutput out) throws IOException { - shardId.writeTo(out); - metadataSnapshot.writeTo(out); - out.writeList(peerRecoveryRetentionLeases); - } - - public ShardId shardId() { - return this.shardId; - } - - public boolean isEmpty() { - return metadataSnapshot.size() == 0; - } - - @Override - public Iterator iterator() { - return metadataSnapshot.iterator(); - } - - public boolean fileExists(String name) { - return metadataSnapshot.asMap().containsKey(name); - } - - public StoreFileMetadata file(String name) { - return metadataSnapshot.asMap().get(name); - } - - /** - * Returns the retaining sequence number of the peer recovery retention lease for a given node if exists; otherwise, returns -1. - */ - public long getPeerRecoveryRetentionLeaseRetainingSeqNo(DiscoveryNode node) { - assert node != null; - final String retentionLeaseId = ReplicationTracker.getPeerRecoveryRetentionLeaseId(node.getId()); - return peerRecoveryRetentionLeases.stream() - .filter(lease -> lease.id().equals(retentionLeaseId)) - .mapToLong(RetentionLease::retainingSequenceNumber) - .findFirst() - .orElse(-1L); - } - - public List peerRecoveryRetentionLeases() { - return peerRecoveryRetentionLeases; - } - - /** - * @return commit sync id if exists, else null - */ - public String syncId() { - return metadataSnapshot.getSyncId(); - } - - @Override - public String toString() { - return "StoreFilesMetadata{" - + ", shardId=" - + shardId - + ", metadataSnapshot{size=" - + metadataSnapshot.size() - + ", syncId=" - + metadataSnapshot.getSyncId() - + "}" - + '}'; - } -} From ffd6031339bc0b6620edf52251e471a7d0c826c6 Mon Sep 17 00:00:00 2001 From: Shivansh Arora Date: Thu, 28 Sep 2023 12:45:45 +0530 Subject: [PATCH 10/21] Corrected the references of TransportNodesListGatewayStartedBatchShards Signed-off-by: Shivansh Arora --- ...tNodesListGatewayStartedBatchShardsIT.java | 164 ++++++++++-------- 1 file changed, 91 insertions(+), 73 deletions(-) diff --git a/server/src/internalClusterTest/java/org/opensearch/gateway/TransportNodesListGatewayStartedBatchShardsIT.java b/server/src/internalClusterTest/java/org/opensearch/gateway/TransportNodesListGatewayStartedBatchShardsIT.java index 1a10dfb701073..590cb66e2d14e 100644 --- a/server/src/internalClusterTest/java/org/opensearch/gateway/TransportNodesListGatewayStartedBatchShardsIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/gateway/TransportNodesListGatewayStartedBatchShardsIT.java @@ -8,146 +8,164 @@ package org.opensearch.gateway; -import org.opensearch.Version; import org.opensearch.action.admin.cluster.shards.ClusterSearchShardsGroup; import org.opensearch.action.admin.cluster.shards.ClusterSearchShardsResponse; +import org.opensearch.action.admin.cluster.state.ClusterStateRequest; +import org.opensearch.action.admin.cluster.state.ClusterStateResponse; import org.opensearch.action.support.ActionTestUtils; +import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.cluster.node.DiscoveryNode; import org.opensearch.common.settings.Settings; import org.opensearch.core.index.Index; import org.opensearch.core.index.shard.ShardId; +import org.opensearch.env.NodeEnvironment; +import org.opensearch.index.shard.ShardPath; import org.opensearch.test.OpenSearchIntegTestCase; +import java.io.IOException; +import java.nio.file.DirectoryStream; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.HashMap; +import java.util.LinkedList; +import java.util.List; import java.util.Map; +import java.util.concurrent.ExecutionException; -import static java.util.Collections.emptyMap; -import static java.util.Collections.emptySet; import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_NUMBER_OF_REPLICAS; import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_NUMBER_OF_SHARDS; -import static org.opensearch.gateway.AsyncShardFetchTestUtils.corruptShard; -import static org.opensearch.gateway.AsyncShardFetchTestUtils.getDiscoveryNodes; -import static org.opensearch.gateway.AsyncShardFetchTestUtils.prepareRequestMap; public class TransportNodesListGatewayStartedBatchShardsIT extends OpenSearchIntegTestCase { public void testSingleShardFetch() throws Exception { String indexName = "test"; - prepareIndices(new String[] { indexName }, 1, 0); - DiscoveryNode[] nodes = getDiscoveryNodes(); - TransportNodesListGatewayStartedShardsBatch.NodesGatewayStartedShardsBatch nodesGatewayStartedShardsBatch = prepareAndSendRequest( - new String[] { indexName }, - nodes - ); + Map shardIdCustomDataPathMap = prepareRequestMap(new String[] { indexName }, 1); + + ClusterSearchShardsResponse searchShardsResponse = client().admin().cluster().prepareSearchShards(indexName).get(); + TransportNodesListGatewayStartedBatchShards.NodesGatewayStartedShardsBatch response; + response = ActionTestUtils.executeBlocking( + internalCluster().getInstance(TransportNodesListGatewayStartedBatchShards.class), + new TransportNodesListGatewayStartedBatchShards.Request(searchShardsResponse.getNodes(), shardIdCustomDataPathMap) + ); final Index index = resolveIndex(indexName); final ShardId shardId = new ShardId(index, 0); - TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShards nodeGatewayStartedShards = nodesGatewayStartedShardsBatch - .getNodesMap() - .get(nodes[0].getId()) + TransportNodesListGatewayStartedBatchShards.NodeGatewayStartedShards nodeGatewayStartedShards = response.getNodesMap() + .get(searchShardsResponse.getNodes()[0].getId()) .getNodeGatewayStartedShardsBatch() .get(shardId); - assertNodeGatewayStartedShardsSuccessCase(nodeGatewayStartedShards); + assertNodeGatewayStartedShardsHappyCase(nodeGatewayStartedShards); } public void testShardFetchMultiNodeMultiIndexes() throws Exception { // start second node internalCluster().startNode(); - String[] indices = new String[] { "index1", "index2" }; - prepareIndices(indices, 1, 0); - DiscoveryNode[] nodes = getDiscoveryNodes(); - TransportNodesListGatewayStartedShardsBatch.NodesGatewayStartedShardsBatch nodesGatewayStartedShardsBatch = prepareAndSendRequest( - indices, - nodes + String indexName1 = "test1"; + String indexName2 = "test2"; + // assign one primary shard each to the data nodes + Map shardIdCustomDataPathMap = prepareRequestMap( + new String[] { indexName1, indexName2 }, + internalCluster().numDataNodes() + ); + ClusterSearchShardsResponse searchShardsResponse = client().admin().cluster().prepareSearchShards(indexName1, indexName2).get(); + assertEquals(internalCluster().numDataNodes(), searchShardsResponse.getNodes().length); + TransportNodesListGatewayStartedBatchShards.NodesGatewayStartedShardsBatch response; + response = ActionTestUtils.executeBlocking( + internalCluster().getInstance(TransportNodesListGatewayStartedBatchShards.class), + new TransportNodesListGatewayStartedBatchShards.Request(searchShardsResponse.getNodes(), shardIdCustomDataPathMap) ); - ClusterSearchShardsResponse searchShardsResponse = client().admin().cluster().prepareSearchShards(indices).get(); for (ClusterSearchShardsGroup clusterSearchShardsGroup : searchShardsResponse.getGroups()) { ShardId shardId = clusterSearchShardsGroup.getShardId(); assertEquals(1, clusterSearchShardsGroup.getShards().length); String nodeId = clusterSearchShardsGroup.getShards()[0].currentNodeId(); - TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShards nodeGatewayStartedShards = nodesGatewayStartedShardsBatch - .getNodesMap() + TransportNodesListGatewayStartedBatchShards.NodeGatewayStartedShards nodeGatewayStartedShards = response.getNodesMap() .get(nodeId) .getNodeGatewayStartedShardsBatch() .get(shardId); - assertNodeGatewayStartedShardsSuccessCase(nodeGatewayStartedShards); + assertNodeGatewayStartedShardsHappyCase(nodeGatewayStartedShards); } } - public void testShardFetchNodeNotConnected() { - DiscoveryNode nonExistingNode = new DiscoveryNode("foo", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT); - String indexName = "test"; - prepareIndices(new String[] { indexName }, 1, 0); - TransportNodesListGatewayStartedShardsBatch.NodesGatewayStartedShardsBatch nodesGatewayStartedShardsBatch = prepareAndSendRequest( - new String[] { indexName }, - new DiscoveryNode[] { nonExistingNode } - ); - assertTrue(nodesGatewayStartedShardsBatch.hasFailures()); - assertEquals(1, nodesGatewayStartedShardsBatch.failures().size()); - assertEquals(nonExistingNode.getId(), nodesGatewayStartedShardsBatch.failures().get(0).nodeId()); - } - public void testShardFetchCorruptedShards() throws Exception { String indexName = "test"; - prepareIndices(new String[] { indexName }, 1, 0); Map shardIdCustomDataPathMap = prepareRequestMap(new String[] { indexName }, 1); ClusterSearchShardsResponse searchShardsResponse = client().admin().cluster().prepareSearchShards(indexName).get(); final Index index = resolveIndex(indexName); final ShardId shardId = new ShardId(index, 0); corruptShard(searchShardsResponse.getNodes()[0].getName(), shardId); - TransportNodesListGatewayStartedShardsBatch.NodesGatewayStartedShardsBatch response; + TransportNodesListGatewayStartedBatchShards.NodesGatewayStartedShardsBatch response; internalCluster().restartNode(searchShardsResponse.getNodes()[0].getName()); response = ActionTestUtils.executeBlocking( - internalCluster().getInstance(TransportNodesListGatewayStartedShardsBatch.class), - new TransportNodesListGatewayStartedShardsBatch.Request(getDiscoveryNodes(), shardIdCustomDataPathMap) + internalCluster().getInstance(TransportNodesListGatewayStartedBatchShards.class), + new TransportNodesListGatewayStartedBatchShards.Request(getDiscoveryNodes(), shardIdCustomDataPathMap) ); DiscoveryNode[] discoveryNodes = getDiscoveryNodes(); - TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShards nodeGatewayStartedShards = response.getNodesMap() + TransportNodesListGatewayStartedBatchShards.NodeGatewayStartedShards nodeGatewayStartedShards = response.getNodesMap() .get(discoveryNodes[0].getId()) .getNodeGatewayStartedShardsBatch() .get(shardId); - assertNodeGatewayStartedShardsFailureCase(nodeGatewayStartedShards); + assertNotNull(nodeGatewayStartedShards.storeException()); + assertNotNull(nodeGatewayStartedShards.allocationId()); + assertTrue(nodeGatewayStartedShards.primary()); } - private TransportNodesListGatewayStartedShardsBatch.NodesGatewayStartedShardsBatch prepareAndSendRequest( - String[] indices, - DiscoveryNode[] nodes - ) { - Map shardIdCustomDataPathMap = prepareRequestMap(indices, 1); - TransportNodesListGatewayStartedShardsBatch.NodesGatewayStartedShardsBatch response; - return ActionTestUtils.executeBlocking( - internalCluster().getInstance(TransportNodesListGatewayStartedShardsBatch.class), - new TransportNodesListGatewayStartedShardsBatch.Request(nodes, shardIdCustomDataPathMap) - ); + private DiscoveryNode[] getDiscoveryNodes() throws ExecutionException, InterruptedException { + final ClusterStateRequest clusterStateRequest = new ClusterStateRequest(); + clusterStateRequest.local(false); + clusterStateRequest.clear().nodes(true).routingTable(true).indices("*"); + ClusterStateResponse clusterStateResponse = client().admin().cluster().state(clusterStateRequest).get(); + final List nodes = new LinkedList<>(clusterStateResponse.getState().nodes().getDataNodes().values()); + DiscoveryNode[] disNodesArr = new DiscoveryNode[nodes.size()]; + nodes.toArray(disNodesArr); + return disNodesArr; } - private void assertNodeGatewayStartedShardsSuccessCase( - TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShards nodeGatewayStartedShards + private void assertNodeGatewayStartedShardsHappyCase( + TransportNodesListGatewayStartedBatchShards.NodeGatewayStartedShards nodeGatewayStartedShards ) { assertNull(nodeGatewayStartedShards.storeException()); assertNotNull(nodeGatewayStartedShards.allocationId()); assertTrue(nodeGatewayStartedShards.primary()); } - private void assertNodeGatewayStartedShardsFailureCase( - TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShards nodeGatewayStartedShards - ) { - assertNotNull(nodeGatewayStartedShards.storeException()); - assertNotNull(nodeGatewayStartedShards.allocationId()); - assertTrue(nodeGatewayStartedShards.primary()); + private void prepareIndex(String indexName, int numberOfPrimaryShards) { + createIndex( + indexName, + Settings.builder().put(SETTING_NUMBER_OF_SHARDS, numberOfPrimaryShards).put(SETTING_NUMBER_OF_REPLICAS, 0).build() + ); + index(indexName, "type", "1"); + flush(indexName); } - private void prepareIndices(String[] indices, int numberOfPrimaryShards, int numberOfReplicaShards) { - for (String index : indices) { - createIndex( - index, - Settings.builder() - .put(SETTING_NUMBER_OF_SHARDS, numberOfPrimaryShards) - .put(SETTING_NUMBER_OF_REPLICAS, numberOfReplicaShards) - .build() + private Map prepareRequestMap(String[] indices, int primaryShardCount) { + Map shardIdCustomDataPathMap = new HashMap<>(); + for (String indexName : indices) { + prepareIndex(indexName, primaryShardCount); + final Index index = resolveIndex(indexName); + final String customDataPath = IndexMetadata.INDEX_DATA_PATH_SETTING.get( + client().admin().indices().prepareGetSettings(indexName).get().getIndexToSettings().get(indexName) ); - index(index, "type", "1"); - flush(index); + for (int shardIdNum = 0; shardIdNum < primaryShardCount; shardIdNum++) { + final ShardId shardId = new ShardId(index, shardIdNum); + shardIdCustomDataPathMap.put(shardId, customDataPath); + } } + return shardIdCustomDataPathMap; } + private void corruptShard(String nodeName, ShardId shardId) throws IOException, InterruptedException { + for (Path path : internalCluster().getInstance(NodeEnvironment.class, nodeName).availableShardPaths(shardId)) { + final Path indexPath = path.resolve(ShardPath.INDEX_FOLDER_NAME); + if (Files.exists(indexPath)) { // multi data path might only have one path in use + try (DirectoryStream stream = Files.newDirectoryStream(indexPath)) { + for (Path item : stream) { + if (item.getFileName().toString().startsWith("segments_")) { + logger.info("--> deleting [{}]", item); + Files.delete(item); + } + } + } + } + } + } } From bc84bd94d20c8e00906f98792c4f50d4eaaf04ee Mon Sep 17 00:00:00 2001 From: Shivansh Arora Date: Tue, 12 Dec 2023 01:48:41 +0530 Subject: [PATCH 11/21] Rename NodeGatewayStartedShards to NodeGatewayStartedShard Signed-off-by: Shivansh Arora --- ...tNodesListGatewayStartedBatchShardsIT.java | 8 ++--- ...ortNodesListGatewayStartedBatchShards.java | 30 +++++++++---------- 2 files changed, 19 insertions(+), 19 deletions(-) diff --git a/server/src/internalClusterTest/java/org/opensearch/gateway/TransportNodesListGatewayStartedBatchShardsIT.java b/server/src/internalClusterTest/java/org/opensearch/gateway/TransportNodesListGatewayStartedBatchShardsIT.java index 590cb66e2d14e..68ab35a6d45da 100644 --- a/server/src/internalClusterTest/java/org/opensearch/gateway/TransportNodesListGatewayStartedBatchShardsIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/gateway/TransportNodesListGatewayStartedBatchShardsIT.java @@ -50,7 +50,7 @@ public void testSingleShardFetch() throws Exception { ); final Index index = resolveIndex(indexName); final ShardId shardId = new ShardId(index, 0); - TransportNodesListGatewayStartedBatchShards.NodeGatewayStartedShards nodeGatewayStartedShards = response.getNodesMap() + TransportNodesListGatewayStartedBatchShards.NodeGatewayStartedShard nodeGatewayStartedShards = response.getNodesMap() .get(searchShardsResponse.getNodes()[0].getId()) .getNodeGatewayStartedShardsBatch() .get(shardId); @@ -78,7 +78,7 @@ public void testShardFetchMultiNodeMultiIndexes() throws Exception { ShardId shardId = clusterSearchShardsGroup.getShardId(); assertEquals(1, clusterSearchShardsGroup.getShards().length); String nodeId = clusterSearchShardsGroup.getShards()[0].currentNodeId(); - TransportNodesListGatewayStartedBatchShards.NodeGatewayStartedShards nodeGatewayStartedShards = response.getNodesMap() + TransportNodesListGatewayStartedBatchShards.NodeGatewayStartedShard nodeGatewayStartedShards = response.getNodesMap() .get(nodeId) .getNodeGatewayStartedShardsBatch() .get(shardId); @@ -100,7 +100,7 @@ public void testShardFetchCorruptedShards() throws Exception { new TransportNodesListGatewayStartedBatchShards.Request(getDiscoveryNodes(), shardIdCustomDataPathMap) ); DiscoveryNode[] discoveryNodes = getDiscoveryNodes(); - TransportNodesListGatewayStartedBatchShards.NodeGatewayStartedShards nodeGatewayStartedShards = response.getNodesMap() + TransportNodesListGatewayStartedBatchShards.NodeGatewayStartedShard nodeGatewayStartedShards = response.getNodesMap() .get(discoveryNodes[0].getId()) .getNodeGatewayStartedShardsBatch() .get(shardId); @@ -121,7 +121,7 @@ private DiscoveryNode[] getDiscoveryNodes() throws ExecutionException, Interrupt } private void assertNodeGatewayStartedShardsHappyCase( - TransportNodesListGatewayStartedBatchShards.NodeGatewayStartedShards nodeGatewayStartedShards + TransportNodesListGatewayStartedBatchShards.NodeGatewayStartedShard nodeGatewayStartedShards ) { assertNull(nodeGatewayStartedShards.storeException()); assertNotNull(nodeGatewayStartedShards.allocationId()); diff --git a/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedBatchShards.java b/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedBatchShards.java index 0215088b2f8d0..cff6f50547859 100644 --- a/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedBatchShards.java +++ b/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedBatchShards.java @@ -142,7 +142,7 @@ protected NodesGatewayStartedShardsBatch newResponse( */ @Override protected NodeGatewayStartedShardsBatch nodeOperation(NodeRequest request) { - Map shardsOnNode = new HashMap<>(); + Map shardsOnNode = new HashMap<>(); for (ShardAttributes shardAttr: request.shardAttributes) { final ShardId shardId = shardAttr.getShardId(); try { @@ -191,7 +191,7 @@ protected NodeGatewayStartedShardsBatch nodeOperation(NodeRequest request) { String allocationId = shardStateMetadata.allocationId != null ? shardStateMetadata.allocationId.getId() : null; shardsOnNode.put( shardId, - new NodeGatewayStartedShards(allocationId, shardStateMetadata.primary, null, exception) + new NodeGatewayStartedShard(allocationId, shardStateMetadata.primary, null, exception) ); continue; } @@ -202,7 +202,7 @@ protected NodeGatewayStartedShardsBatch nodeOperation(NodeRequest request) { final IndexShard shard = indicesService.getShardOrNull(shardId); shardsOnNode.put( shardId, - new NodeGatewayStartedShards( + new NodeGatewayStartedShard( allocationId, shardStateMetadata.primary, shard != null ? shard.getLatestReplicationCheckpoint() : null @@ -211,11 +211,11 @@ protected NodeGatewayStartedShardsBatch nodeOperation(NodeRequest request) { continue; } logger.trace("{} no local shard info found", shardId); - shardsOnNode.put(shardId, new NodeGatewayStartedShards(null, false, null)); + shardsOnNode.put(shardId, new NodeGatewayStartedShard(null, false, null)); } catch (Exception e) { shardsOnNode.put( shardId, - new NodeGatewayStartedShards(null, false, null, new OpenSearchException("failed to load started shards", e)) + new NodeGatewayStartedShard(null, false, null, new OpenSearchException("failed to load started shards", e)) ); } } @@ -310,17 +310,17 @@ public void writeTo(StreamOutput out) throws IOException { } /** - * Class for storing the information about the shards fetched on the node. + * Class for storing the information about the shard fetched on the node. * * @opensearch.internal */ - public static class NodeGatewayStartedShards { + public static class NodeGatewayStartedShard { private final String allocationId; private final boolean primary; private final Exception storeException; private final ReplicationCheckpoint replicationCheckpoint; - public NodeGatewayStartedShards(StreamInput in) throws IOException { + public NodeGatewayStartedShard(StreamInput in) throws IOException { allocationId = in.readOptionalString(); primary = in.readBoolean(); if (in.readBoolean()) { @@ -335,11 +335,11 @@ public NodeGatewayStartedShards(StreamInput in) throws IOException { } } - public NodeGatewayStartedShards(String allocationId, boolean primary, ReplicationCheckpoint replicationCheckpoint) { + public NodeGatewayStartedShard(String allocationId, boolean primary, ReplicationCheckpoint replicationCheckpoint) { this(allocationId, primary, replicationCheckpoint, null); } - public NodeGatewayStartedShards( + public NodeGatewayStartedShard( String allocationId, boolean primary, ReplicationCheckpoint replicationCheckpoint, @@ -395,7 +395,7 @@ public boolean equals(Object o) { return false; } - NodeGatewayStartedShards that = (NodeGatewayStartedShards) o; + NodeGatewayStartedShard that = (NodeGatewayStartedShard) o; return primary == that.primary && Objects.equals(allocationId, that.allocationId) @@ -435,15 +435,15 @@ public String toString() { * @opensearch.internal */ public static class NodeGatewayStartedShardsBatch extends BaseNodeResponse { - private final Map nodeGatewayStartedShardsBatch; + private final Map nodeGatewayStartedShardsBatch; - public Map getNodeGatewayStartedShardsBatch() { + public Map getNodeGatewayStartedShardsBatch() { return nodeGatewayStartedShardsBatch; } public NodeGatewayStartedShardsBatch(StreamInput in) throws IOException { super(in); - this.nodeGatewayStartedShardsBatch = in.readMap(ShardId::new, NodeGatewayStartedShards::new); + this.nodeGatewayStartedShardsBatch = in.readMap(ShardId::new, NodeGatewayStartedShard::new); } @Override @@ -452,7 +452,7 @@ public void writeTo(StreamOutput out) throws IOException { out.writeMap(nodeGatewayStartedShardsBatch, (o, k) -> k.writeTo(o), (o, v) -> v.writeTo(o)); } - public NodeGatewayStartedShardsBatch(DiscoveryNode node, Map nodeGatewayStartedShardsBatch) { + public NodeGatewayStartedShardsBatch(DiscoveryNode node, Map nodeGatewayStartedShardsBatch) { super(node); this.nodeGatewayStartedShardsBatch = nodeGatewayStartedShardsBatch; } From 94f27cc03eee9f119d6c17b2cef20d9de131d8aa Mon Sep 17 00:00:00 2001 From: Shivansh Arora Date: Tue, 12 Dec 2023 23:04:17 +0530 Subject: [PATCH 12/21] modify the request signature Signed-off-by: Shivansh Arora --- ...ortNodesListGatewayStartedBatchShards.java | 76 +++++++++---------- ...ransportNodesListGatewayStartedShards.java | 13 +++- 2 files changed, 46 insertions(+), 43 deletions(-) diff --git a/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedBatchShards.java b/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedBatchShards.java index cff6f50547859..8c2aadcb93610 100644 --- a/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedBatchShards.java +++ b/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedBatchShards.java @@ -11,7 +11,6 @@ import org.apache.logging.log4j.message.ParameterizedMessage; import org.opensearch.OpenSearchException; import org.opensearch.Version; -import org.opensearch.action.ActionListener; import org.opensearch.action.ActionType; import org.opensearch.action.FailedNodeException; import org.opensearch.action.support.ActionFilters; @@ -24,14 +23,15 @@ import org.opensearch.cluster.node.DiscoveryNode; import org.opensearch.cluster.service.ClusterService; import org.opensearch.common.inject.Inject; +import org.opensearch.common.settings.Settings; +import org.opensearch.core.action.ActionListener; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; -import org.opensearch.common.settings.Settings; +import org.opensearch.core.index.shard.ShardId; import org.opensearch.core.xcontent.NamedXContentRegistry; import org.opensearch.env.NodeEnvironment; import org.opensearch.index.IndexSettings; import org.opensearch.index.shard.IndexShard; -import org.opensearch.core.index.shard.ShardId; import org.opensearch.index.shard.ShardPath; import org.opensearch.index.shard.ShardStateMetadata; import org.opensearch.index.store.Store; @@ -47,7 +47,6 @@ import java.util.List; import java.util.Map; import java.util.Objects; -import java.util.stream.Collectors; /** * This transport action is used to fetch batch of unassigned shard version from each node during primary allocation in {@link GatewayAllocator}. @@ -56,15 +55,15 @@ * * @opensearch.internal */ -public class TransportNodesListGatewayStartedShardsBatch extends TransportNodesAction< - TransportNodesListGatewayStartedShardsBatch.Request, - TransportNodesListGatewayStartedShardsBatch.NodesGatewayStartedShardsBatch, - TransportNodesListGatewayStartedShardsBatch.NodeRequest, - TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShardsBatch> +public class TransportNodesListGatewayStartedBatchShards extends TransportNodesAction< + TransportNodesListGatewayStartedBatchShards.Request, + TransportNodesListGatewayStartedBatchShards.NodesGatewayStartedShardsBatch, + TransportNodesListGatewayStartedBatchShards.NodeRequest, + TransportNodesListGatewayStartedBatchShards.NodeGatewayStartedShardsBatch> implements AsyncShardFetch.Lister< - TransportNodesListGatewayStartedShardsBatch.NodesGatewayStartedShardsBatch, - TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShardsBatch> { + TransportNodesListGatewayStartedBatchShards.NodesGatewayStartedShardsBatch, + TransportNodesListGatewayStartedBatchShards.NodeGatewayStartedShardsBatch> { public static final String ACTION_NAME = "internal:gateway/local/started_shards_batch"; public static final ActionType TYPE = new ActionType<>( @@ -78,7 +77,7 @@ public class TransportNodesListGatewayStartedShardsBatch extends TransportNodesA private final NamedXContentRegistry namedXContentRegistry; @Inject - public TransportNodesListGatewayStartedShardsBatch( + public TransportNodesListGatewayStartedBatchShards( Settings settings, ThreadPool threadPool, ClusterService clusterService, @@ -107,11 +106,11 @@ public TransportNodesListGatewayStartedShardsBatch( @Override public void list( - Map shardIdsWithCustomDataPath, + Map shardAttributesMap, DiscoveryNode[] nodes, ActionListener listener ) { - execute(new Request(nodes, shardIdsWithCustomDataPath), listener); + execute(new Request(nodes, shardAttributesMap), listener); } @Override @@ -142,8 +141,8 @@ protected NodesGatewayStartedShardsBatch newResponse( */ @Override protected NodeGatewayStartedShardsBatch nodeOperation(NodeRequest request) { - Map shardsOnNode = new HashMap<>(); - for (ShardAttributes shardAttr: request.shardAttributes) { + Map shardsOnNode = new HashMap<>(); + for (ShardAttributes shardAttr: request.shardAttributes.values()) { final ShardId shardId = shardAttr.getShardId(); try { logger.trace("{} loading local shard state info", shardId); @@ -191,7 +190,7 @@ protected NodeGatewayStartedShardsBatch nodeOperation(NodeRequest request) { String allocationId = shardStateMetadata.allocationId != null ? shardStateMetadata.allocationId.getId() : null; shardsOnNode.put( shardId, - new NodeGatewayStartedShard(allocationId, shardStateMetadata.primary, null, exception) + new NodeGatewayStartedShards(allocationId, shardStateMetadata.primary, null, exception) ); continue; } @@ -202,7 +201,7 @@ protected NodeGatewayStartedShardsBatch nodeOperation(NodeRequest request) { final IndexShard shard = indicesService.getShardOrNull(shardId); shardsOnNode.put( shardId, - new NodeGatewayStartedShard( + new NodeGatewayStartedShards( allocationId, shardStateMetadata.primary, shard != null ? shard.getLatestReplicationCheckpoint() : null @@ -211,11 +210,11 @@ protected NodeGatewayStartedShardsBatch nodeOperation(NodeRequest request) { continue; } logger.trace("{} no local shard info found", shardId); - shardsOnNode.put(shardId, new NodeGatewayStartedShard(null, false, null)); + shardsOnNode.put(shardId, new NodeGatewayStartedShards(null, false, null)); } catch (Exception e) { shardsOnNode.put( shardId, - new NodeGatewayStartedShard(null, false, null, new OpenSearchException("failed to load started shards", e)) + new NodeGatewayStartedShards(null, false, null, new OpenSearchException("failed to load started shards", e)) ); } } @@ -229,26 +228,25 @@ protected NodeGatewayStartedShardsBatch nodeOperation(NodeRequest request) { * @opensearch.internal */ public static class Request extends BaseNodesRequest { - private final List shardAttributes; + private final Map shardAttributes; public Request(StreamInput in) throws IOException { super(in); - shardAttributes = in.readList(ShardAttributes::new); + shardAttributes = in.readMap(ShardId::new, ShardAttributes::new); } - public Request(DiscoveryNode[] nodes, Map shardIdStringMap) { + public Request(DiscoveryNode[] nodes, Map shardAttributes) { super(nodes); - this.shardAttributes = Objects.requireNonNull(shardIdStringMap).entrySet().stream().map(entry -> - new ShardAttributes(entry.getKey(), entry.getValue())).collect(Collectors.toList()); + this.shardAttributes = Objects.requireNonNull(shardAttributes); } @Override public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); - out.writeList(shardAttributes); + out.writeMap(shardAttributes, (o,k)-> k.writeTo(o), (o,v) -> v.writeTo(o)); } - public List getShardAttributes() { + public Map getShardAttributes() { return shardAttributes; } } @@ -291,11 +289,11 @@ protected void writeNodesTo(StreamOutput out, List shardAttributes; + private final Map shardAttributes; public NodeRequest(StreamInput in) throws IOException { super(in); - shardAttributes = in.readList(ShardAttributes::new); + shardAttributes = in.readMap(ShardId::new, ShardAttributes::new); } public NodeRequest(Request request) { @@ -305,7 +303,7 @@ public NodeRequest(Request request) { @Override public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); - out.writeList(shardAttributes); + out.writeMap(shardAttributes, (o,k)-> k.writeTo(o), (o,v) -> v.writeTo(o)); } } @@ -314,13 +312,13 @@ public void writeTo(StreamOutput out) throws IOException { * * @opensearch.internal */ - public static class NodeGatewayStartedShard { + public static class NodeGatewayStartedShards { private final String allocationId; private final boolean primary; private final Exception storeException; private final ReplicationCheckpoint replicationCheckpoint; - public NodeGatewayStartedShard(StreamInput in) throws IOException { + public NodeGatewayStartedShards(StreamInput in) throws IOException { allocationId = in.readOptionalString(); primary = in.readBoolean(); if (in.readBoolean()) { @@ -335,11 +333,11 @@ public NodeGatewayStartedShard(StreamInput in) throws IOException { } } - public NodeGatewayStartedShard(String allocationId, boolean primary, ReplicationCheckpoint replicationCheckpoint) { + public NodeGatewayStartedShards(String allocationId, boolean primary, ReplicationCheckpoint replicationCheckpoint) { this(allocationId, primary, replicationCheckpoint, null); } - public NodeGatewayStartedShard( + public NodeGatewayStartedShards( String allocationId, boolean primary, ReplicationCheckpoint replicationCheckpoint, @@ -395,7 +393,7 @@ public boolean equals(Object o) { return false; } - NodeGatewayStartedShard that = (NodeGatewayStartedShard) o; + NodeGatewayStartedShards that = (NodeGatewayStartedShards) o; return primary == that.primary && Objects.equals(allocationId, that.allocationId) @@ -435,15 +433,15 @@ public String toString() { * @opensearch.internal */ public static class NodeGatewayStartedShardsBatch extends BaseNodeResponse { - private final Map nodeGatewayStartedShardsBatch; + private final Map nodeGatewayStartedShardsBatch; - public Map getNodeGatewayStartedShardsBatch() { + public Map getNodeGatewayStartedShardsBatch() { return nodeGatewayStartedShardsBatch; } public NodeGatewayStartedShardsBatch(StreamInput in) throws IOException { super(in); - this.nodeGatewayStartedShardsBatch = in.readMap(ShardId::new, NodeGatewayStartedShard::new); + this.nodeGatewayStartedShardsBatch = in.readMap(ShardId::new, NodeGatewayStartedShards::new); } @Override @@ -452,7 +450,7 @@ public void writeTo(StreamOutput out) throws IOException { out.writeMap(nodeGatewayStartedShardsBatch, (o, k) -> k.writeTo(o), (o, v) -> v.writeTo(o)); } - public NodeGatewayStartedShardsBatch(DiscoveryNode node, Map nodeGatewayStartedShardsBatch) { + public NodeGatewayStartedShardsBatch(DiscoveryNode node, Map nodeGatewayStartedShardsBatch) { super(node); this.nodeGatewayStartedShardsBatch = nodeGatewayStartedShardsBatch; } diff --git a/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedShards.java b/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedShards.java index e2a3f08bb02c6..0fe7e1b7baad8 100644 --- a/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedShards.java +++ b/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedShards.java @@ -35,7 +35,6 @@ import org.apache.logging.log4j.message.ParameterizedMessage; import org.opensearch.OpenSearchException; import org.opensearch.Version; -import org.opensearch.action.ActionListener; import org.opensearch.action.ActionType; import org.opensearch.action.FailedNodeException; import org.opensearch.action.support.ActionFilters; @@ -49,25 +48,28 @@ import org.opensearch.cluster.service.ClusterService; import org.opensearch.common.Nullable; import org.opensearch.common.inject.Inject; +import org.opensearch.common.settings.Settings; +import org.opensearch.action.ActionListener; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; -import org.opensearch.common.settings.Settings; +import org.opensearch.core.index.shard.ShardId; import org.opensearch.core.xcontent.NamedXContentRegistry; import org.opensearch.env.NodeEnvironment; import org.opensearch.index.IndexSettings; import org.opensearch.index.shard.IndexShard; -import org.opensearch.core.index.shard.ShardId; import org.opensearch.index.shard.ShardPath; import org.opensearch.index.shard.ShardStateMetadata; import org.opensearch.index.store.Store; import org.opensearch.indices.IndicesService; import org.opensearch.indices.replication.checkpoint.ReplicationCheckpoint; +import org.opensearch.indices.store.ShardAttributes; import org.opensearch.threadpool.ThreadPool; import org.opensearch.transport.TransportRequest; import org.opensearch.transport.TransportService; import java.io.IOException; import java.util.List; +import java.util.Map; import java.util.Objects; /** @@ -124,7 +126,10 @@ public TransportNodesListGatewayStartedShards( } @Override - public void list(ShardId shardId, String customDataPath, DiscoveryNode[] nodes, ActionListener listener) { + public void list(Map shardAttributesMap, DiscoveryNode[] nodes, ActionListener listener) { + assert shardAttributesMap.size() == 1 : "only one shard should be specified"; + final ShardId shardId = shardAttributesMap.keySet().iterator().next(); + final String customDataPath = shardAttributesMap.get(shardId).getCustomDataPath(); execute(new Request(shardId, customDataPath, nodes), listener); } From 4a62b19e509c86699477f9397855a7e0f9bff1e9 Mon Sep 17 00:00:00 2001 From: Shivansh Arora Date: Tue, 12 Dec 2023 23:09:25 +0530 Subject: [PATCH 13/21] Remove ShardAttributes because added in #8742 Signed-off-by: Shivansh Arora --- .../indices/store/ShardAttributes.java | 52 ------------------- 1 file changed, 52 deletions(-) delete mode 100644 server/src/main/java/org/opensearch/indices/store/ShardAttributes.java diff --git a/server/src/main/java/org/opensearch/indices/store/ShardAttributes.java b/server/src/main/java/org/opensearch/indices/store/ShardAttributes.java deleted file mode 100644 index 6ed65f33dd0b6..0000000000000 --- a/server/src/main/java/org/opensearch/indices/store/ShardAttributes.java +++ /dev/null @@ -1,52 +0,0 @@ -/* - * 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.indices.store; - -import org.opensearch.common.Nullable; -import org.opensearch.core.common.io.stream.StreamInput; -import org.opensearch.core.common.io.stream.StreamOutput; -import org.opensearch.core.common.io.stream.Writeable; -import org.opensearch.core.index.shard.ShardId; - -import java.io.IOException; - -public class ShardAttributes implements Writeable { - private final ShardId shardId; - @Nullable - private final String customDataPath; - - public ShardAttributes(ShardId shardId, String customDataPath) { - this.shardId = shardId; - this.customDataPath = customDataPath; - } - - public ShardAttributes(StreamInput in) throws IOException { - shardId = new ShardId(in); - customDataPath = in.readString(); - } - - public ShardId getShardId() { - return shardId; - } - - /** - * Returns the custom data path that is used to look up information for this shard. - * Returns an empty string if no custom data path is used for this index. - * Returns null if custom data path information is not available (due to BWC). - */ - @Nullable - public String getCustomDataPath() { - return customDataPath; - } - - public void writeTo(StreamOutput out) throws IOException { - shardId.writeTo(out); - out.writeString(customDataPath); - } -} From 5c1f44631356f0e7a8356aafae7590ef66911470 Mon Sep 17 00:00:00 2001 From: Shivansh Arora Date: Tue, 12 Dec 2023 16:59:50 +0530 Subject: [PATCH 14/21] Added a Helper class to remove code duplication Signed-off-by: Shivansh Arora --- ...ansportNodesGatewayStartedShardHelper.java | 114 +++++++++++++++++ ...ortNodesListGatewayStartedBatchShards.java | 117 +++++------------- ...ransportNodesListGatewayStartedShards.java | 86 +++---------- 3 files changed, 160 insertions(+), 157 deletions(-) create mode 100644 server/src/main/java/org/opensearch/gateway/TransportNodesGatewayStartedShardHelper.java diff --git a/server/src/main/java/org/opensearch/gateway/TransportNodesGatewayStartedShardHelper.java b/server/src/main/java/org/opensearch/gateway/TransportNodesGatewayStartedShardHelper.java new file mode 100644 index 0000000000000..471b10426b7e0 --- /dev/null +++ b/server/src/main/java/org/opensearch/gateway/TransportNodesGatewayStartedShardHelper.java @@ -0,0 +1,114 @@ +/* + * 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; + +import org.apache.logging.log4j.Logger; +import org.apache.logging.log4j.message.ParameterizedMessage; +import org.opensearch.OpenSearchException; +import org.opensearch.cluster.metadata.IndexMetadata; +import org.opensearch.cluster.service.ClusterService; +import org.opensearch.common.settings.Settings; +import org.opensearch.core.index.shard.ShardId; +import org.opensearch.core.xcontent.NamedXContentRegistry; +import org.opensearch.env.NodeEnvironment; +import org.opensearch.index.IndexSettings; +import org.opensearch.index.shard.IndexShard; +import org.opensearch.index.shard.ShardPath; +import org.opensearch.index.shard.ShardStateMetadata; +import org.opensearch.index.store.Store; +import org.opensearch.indices.IndicesService; + +import java.io.IOException; + +/** + * This class has the common code used in {@link TransportNodesListGatewayStartedShards} and + * {@link TransportNodesListGatewayStartedBatchShards} to get the shard info on the local node. + *

+ * This class should not be used to add more functions and will be removed when the + * {@link TransportNodesListGatewayStartedShards} will be deprecated and all the code will be moved to + * {@link TransportNodesListGatewayStartedBatchShards} + * + * @opensearch.internal + */ +public class TransportNodesGatewayStartedShardHelper { + public static TransportNodesListGatewayStartedBatchShards.NodeGatewayStartedShard getShardInfoOnLocalNode( + Logger logger, + final ShardId shardId, + NamedXContentRegistry namedXContentRegistry, + NodeEnvironment nodeEnv, + IndicesService indicesService, + String shardDataPathInRequest, + Settings settings, + ClusterService clusterService + ) throws IOException { + logger.trace("{} loading local shard state info", shardId); + ShardStateMetadata shardStateMetadata = ShardStateMetadata.FORMAT.loadLatestState( + logger, + namedXContentRegistry, + nodeEnv.availableShardPaths(shardId) + ); + if (shardStateMetadata != null) { + if (indicesService.getShardOrNull(shardId) == null + && shardStateMetadata.indexDataLocation == ShardStateMetadata.IndexDataLocation.LOCAL) { + final String customDataPath; + if (shardDataPathInRequest != null) { + customDataPath = shardDataPathInRequest; + } else { + // TODO: Fallback for BWC with older OpenSearch versions. + // Remove once request.getCustomDataPath() always returns non-null + final IndexMetadata metadata = clusterService.state().metadata().index(shardId.getIndex()); + if (metadata != null) { + customDataPath = new IndexSettings(metadata, settings).customDataPath(); + } else { + logger.trace("{} node doesn't have meta data for the requests index", shardId); + throw new OpenSearchException("node doesn't have meta data for index " + shardId.getIndex()); + } + } + // we don't have an open shard on the store, validate the files on disk are openable + ShardPath shardPath = null; + try { + shardPath = ShardPath.loadShardPath(logger, nodeEnv, shardId, customDataPath); + if (shardPath == null) { + throw new IllegalStateException(shardId + " no shard path found"); + } + Store.tryOpenIndex(shardPath.resolveIndex(), shardId, nodeEnv::shardLock, logger); + } catch (Exception exception) { + final ShardPath finalShardPath = shardPath; + logger.trace( + () -> new ParameterizedMessage( + "{} can't open index for shard [{}] in path [{}]", + shardId, + shardStateMetadata, + (finalShardPath != null) ? finalShardPath.resolveIndex() : "" + ), + exception + ); + String allocationId = shardStateMetadata.allocationId != null ? shardStateMetadata.allocationId.getId() : null; + return new TransportNodesListGatewayStartedBatchShards.NodeGatewayStartedShard( + allocationId, + shardStateMetadata.primary, + null, + exception + ); + } + } + + logger.debug("{} shard state info found: [{}]", shardId, shardStateMetadata); + String allocationId = shardStateMetadata.allocationId != null ? shardStateMetadata.allocationId.getId() : null; + final IndexShard shard = indicesService.getShardOrNull(shardId); + return new TransportNodesListGatewayStartedBatchShards.NodeGatewayStartedShard( + allocationId, + shardStateMetadata.primary, + shard != null ? shard.getLatestReplicationCheckpoint() : null + ); + } + logger.trace("{} no local shard info found", shardId); + return new TransportNodesListGatewayStartedBatchShards.NodeGatewayStartedShard(null, false, null); + } +} diff --git a/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedBatchShards.java b/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedBatchShards.java index 8c2aadcb93610..b88efd12c958b 100644 --- a/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedBatchShards.java +++ b/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedBatchShards.java @@ -8,7 +8,6 @@ package org.opensearch.gateway; -import org.apache.logging.log4j.message.ParameterizedMessage; import org.opensearch.OpenSearchException; import org.opensearch.Version; import org.opensearch.action.ActionType; @@ -19,7 +18,6 @@ import org.opensearch.action.support.nodes.BaseNodesResponse; import org.opensearch.action.support.nodes.TransportNodesAction; import org.opensearch.cluster.ClusterName; -import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.cluster.node.DiscoveryNode; import org.opensearch.cluster.service.ClusterService; import org.opensearch.common.inject.Inject; @@ -30,11 +28,6 @@ import org.opensearch.core.index.shard.ShardId; import org.opensearch.core.xcontent.NamedXContentRegistry; import org.opensearch.env.NodeEnvironment; -import org.opensearch.index.IndexSettings; -import org.opensearch.index.shard.IndexShard; -import org.opensearch.index.shard.ShardPath; -import org.opensearch.index.shard.ShardStateMetadata; -import org.opensearch.index.store.Store; import org.opensearch.indices.IndicesService; import org.opensearch.indices.replication.checkpoint.ReplicationCheckpoint; import org.opensearch.indices.store.ShardAttributes; @@ -48,6 +41,8 @@ import java.util.Map; import java.util.Objects; +import static org.opensearch.gateway.TransportNodesGatewayStartedShardHelper.getShardInfoOnLocalNode; + /** * This transport action is used to fetch batch of unassigned shard version from each node during primary allocation in {@link GatewayAllocator}. * We use this to find out which node holds the latest shard version and which of them used to be a primary in order to allocate @@ -141,80 +136,27 @@ protected NodesGatewayStartedShardsBatch newResponse( */ @Override protected NodeGatewayStartedShardsBatch nodeOperation(NodeRequest request) { - Map shardsOnNode = new HashMap<>(); - for (ShardAttributes shardAttr: request.shardAttributes.values()) { + Map shardsOnNode = new HashMap<>(); + for (ShardAttributes shardAttr : request.shardAttributes.values()) { final ShardId shardId = shardAttr.getShardId(); try { - logger.trace("{} loading local shard state info", shardId); - ShardStateMetadata shardStateMetadata = ShardStateMetadata.FORMAT.loadLatestState( - logger, - namedXContentRegistry, - nodeEnv.availableShardPaths(shardId) - ); - if (shardStateMetadata != null) { - if (indicesService.getShardOrNull(shardId) == null - && shardStateMetadata.indexDataLocation == ShardStateMetadata.IndexDataLocation.LOCAL) { - final String customDataPath; - if (shardAttr.getCustomDataPath() != null) { - customDataPath = shardAttr.getCustomDataPath(); - } else { - // TODO: Fallback for BWC with older OpenSearch versions. - // Remove once request.getCustomDataPath() always returns non-null - final IndexMetadata metadata = clusterService.state().metadata().index(shardId.getIndex()); - if (metadata != null) { - customDataPath = new IndexSettings(metadata, settings).customDataPath(); - } else { - logger.trace("{} node doesn't have meta data for the requests index", shardId); - throw new OpenSearchException("node doesn't have meta data for index " + shardId.getIndex()); - } - } - // we don't have an open shard on the store, validate the files on disk are openable - ShardPath shardPath = null; - try { - shardPath = ShardPath.loadShardPath(logger, nodeEnv, shardId, customDataPath); - if (shardPath == null) { - throw new IllegalStateException(shardId + " no shard path found"); - } - Store.tryOpenIndex(shardPath.resolveIndex(), shardId, nodeEnv::shardLock, logger); - } catch (Exception exception) { - final ShardPath finalShardPath = shardPath; - logger.trace( - () -> new ParameterizedMessage( - "{} can't open index for shard [{}] in path [{}]", - shardId, - shardStateMetadata, - (finalShardPath != null) ? finalShardPath.resolveIndex() : "" - ), - exception - ); - String allocationId = shardStateMetadata.allocationId != null ? shardStateMetadata.allocationId.getId() : null; - shardsOnNode.put( - shardId, - new NodeGatewayStartedShards(allocationId, shardStateMetadata.primary, null, exception) - ); - continue; - } - } - - logger.debug("{} shard state info found: [{}]", shardId, shardStateMetadata); - String allocationId = shardStateMetadata.allocationId != null ? shardStateMetadata.allocationId.getId() : null; - final IndexShard shard = indicesService.getShardOrNull(shardId); - shardsOnNode.put( + shardsOnNode.put( + shardId, + getShardInfoOnLocalNode( + logger, shardId, - new NodeGatewayStartedShards( - allocationId, - shardStateMetadata.primary, - shard != null ? shard.getLatestReplicationCheckpoint() : null - ) - ); - continue; - } - logger.trace("{} no local shard info found", shardId); - shardsOnNode.put(shardId, new NodeGatewayStartedShards(null, false, null)); + namedXContentRegistry, + nodeEnv, + indicesService, + shardAttr.getCustomDataPath(), + settings, + clusterService + ) + ); } catch (Exception e) { shardsOnNode.put( shardId, - new NodeGatewayStartedShards(null, false, null, new OpenSearchException("failed to load started shards", e)) + new NodeGatewayStartedShard(null, false, null, new OpenSearchException("failed to load started shards", e)) ); } } @@ -243,7 +185,7 @@ public Request(DiscoveryNode[] nodes, Map shardAttribu @Override public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); - out.writeMap(shardAttributes, (o,k)-> k.writeTo(o), (o,v) -> v.writeTo(o)); + out.writeMap(shardAttributes, (o, k) -> k.writeTo(o), (o, v) -> v.writeTo(o)); } public Map getShardAttributes() { @@ -303,22 +245,23 @@ public NodeRequest(Request request) { @Override public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); - out.writeMap(shardAttributes, (o,k)-> k.writeTo(o), (o,v) -> v.writeTo(o)); + out.writeMap(shardAttributes, (o, k) -> k.writeTo(o), (o, v) -> v.writeTo(o)); } } /** - * Class for storing the information about the shard fetched on the node. + * Class for storing shard information received from other node. + * This is used in {@link NodeGatewayStartedShardsBatch} to construct the response for each node * * @opensearch.internal */ - public static class NodeGatewayStartedShards { + public static class NodeGatewayStartedShard { private final String allocationId; private final boolean primary; private final Exception storeException; private final ReplicationCheckpoint replicationCheckpoint; - public NodeGatewayStartedShards(StreamInput in) throws IOException { + public NodeGatewayStartedShard(StreamInput in) throws IOException { allocationId = in.readOptionalString(); primary = in.readBoolean(); if (in.readBoolean()) { @@ -333,11 +276,11 @@ public NodeGatewayStartedShards(StreamInput in) throws IOException { } } - public NodeGatewayStartedShards(String allocationId, boolean primary, ReplicationCheckpoint replicationCheckpoint) { + public NodeGatewayStartedShard(String allocationId, boolean primary, ReplicationCheckpoint replicationCheckpoint) { this(allocationId, primary, replicationCheckpoint, null); } - public NodeGatewayStartedShards( + public NodeGatewayStartedShard( String allocationId, boolean primary, ReplicationCheckpoint replicationCheckpoint, @@ -393,7 +336,7 @@ public boolean equals(Object o) { return false; } - NodeGatewayStartedShards that = (NodeGatewayStartedShards) o; + NodeGatewayStartedShard that = (NodeGatewayStartedShard) o; return primary == that.primary && Objects.equals(allocationId, that.allocationId) @@ -433,15 +376,15 @@ public String toString() { * @opensearch.internal */ public static class NodeGatewayStartedShardsBatch extends BaseNodeResponse { - private final Map nodeGatewayStartedShardsBatch; + private final Map nodeGatewayStartedShardsBatch; - public Map getNodeGatewayStartedShardsBatch() { + public Map getNodeGatewayStartedShardsBatch() { return nodeGatewayStartedShardsBatch; } public NodeGatewayStartedShardsBatch(StreamInput in) throws IOException { super(in); - this.nodeGatewayStartedShardsBatch = in.readMap(ShardId::new, NodeGatewayStartedShards::new); + this.nodeGatewayStartedShardsBatch = in.readMap(ShardId::new, NodeGatewayStartedShard::new); } @Override @@ -450,7 +393,7 @@ public void writeTo(StreamOutput out) throws IOException { out.writeMap(nodeGatewayStartedShardsBatch, (o, k) -> k.writeTo(o), (o, v) -> v.writeTo(o)); } - public NodeGatewayStartedShardsBatch(DiscoveryNode node, Map nodeGatewayStartedShardsBatch) { + public NodeGatewayStartedShardsBatch(DiscoveryNode node, Map nodeGatewayStartedShardsBatch) { super(node); this.nodeGatewayStartedShardsBatch = nodeGatewayStartedShardsBatch; } diff --git a/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedShards.java b/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedShards.java index 0fe7e1b7baad8..497ef6d6fea24 100644 --- a/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedShards.java +++ b/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedShards.java @@ -32,7 +32,6 @@ package org.opensearch.gateway; -import org.apache.logging.log4j.message.ParameterizedMessage; import org.opensearch.OpenSearchException; import org.opensearch.Version; import org.opensearch.action.ActionType; @@ -43,7 +42,6 @@ import org.opensearch.action.support.nodes.BaseNodesResponse; import org.opensearch.action.support.nodes.TransportNodesAction; import org.opensearch.cluster.ClusterName; -import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.cluster.node.DiscoveryNode; import org.opensearch.cluster.service.ClusterService; import org.opensearch.common.Nullable; @@ -55,11 +53,6 @@ import org.opensearch.core.index.shard.ShardId; import org.opensearch.core.xcontent.NamedXContentRegistry; import org.opensearch.env.NodeEnvironment; -import org.opensearch.index.IndexSettings; -import org.opensearch.index.shard.IndexShard; -import org.opensearch.index.shard.ShardPath; -import org.opensearch.index.shard.ShardStateMetadata; -import org.opensearch.index.store.Store; import org.opensearch.indices.IndicesService; import org.opensearch.indices.replication.checkpoint.ReplicationCheckpoint; import org.opensearch.indices.store.ShardAttributes; @@ -72,6 +65,8 @@ import java.util.Map; import java.util.Objects; +import static org.opensearch.gateway.TransportNodesGatewayStartedShardHelper.getShardInfoOnLocalNode; + /** * This transport action is used to fetch the shard version from each node during primary allocation in {@link GatewayAllocator}. * We use this to find out which node holds the latest shard version and which of them used to be a primary in order to allocate @@ -155,72 +150,23 @@ protected NodesGatewayStartedShards newResponse( @Override protected NodeGatewayStartedShards nodeOperation(NodeRequest request) { try { - final ShardId shardId = request.getShardId(); - logger.trace("{} loading local shard state info", shardId); - ShardStateMetadata shardStateMetadata = ShardStateMetadata.FORMAT.loadLatestState( + TransportNodesListGatewayStartedBatchShards.NodeGatewayStartedShard shardInfo = getShardInfoOnLocalNode( logger, + request.getShardId(), namedXContentRegistry, - nodeEnv.availableShardPaths(request.shardId) + nodeEnv, + indicesService, + request.getCustomDataPath(), + settings, + clusterService + ); + return new NodeGatewayStartedShards( + clusterService.localNode(), + shardInfo.allocationId(), + shardInfo.primary(), + shardInfo.replicationCheckpoint(), + shardInfo.storeException() ); - if (shardStateMetadata != null) { - if (indicesService.getShardOrNull(shardId) == null - && shardStateMetadata.indexDataLocation == ShardStateMetadata.IndexDataLocation.LOCAL) { - final String customDataPath; - if (request.getCustomDataPath() != null) { - customDataPath = request.getCustomDataPath(); - } else { - // TODO: Fallback for BWC with older OpenSearch versions. - // Remove once request.getCustomDataPath() always returns non-null - final IndexMetadata metadata = clusterService.state().metadata().index(shardId.getIndex()); - if (metadata != null) { - customDataPath = new IndexSettings(metadata, settings).customDataPath(); - } else { - logger.trace("{} node doesn't have meta data for the requests index", shardId); - throw new OpenSearchException("node doesn't have meta data for index " + shardId.getIndex()); - } - } - // we don't have an open shard on the store, validate the files on disk are openable - ShardPath shardPath = null; - try { - shardPath = ShardPath.loadShardPath(logger, nodeEnv, shardId, customDataPath); - if (shardPath == null) { - throw new IllegalStateException(shardId + " no shard path found"); - } - Store.tryOpenIndex(shardPath.resolveIndex(), shardId, nodeEnv::shardLock, logger); - } catch (Exception exception) { - final ShardPath finalShardPath = shardPath; - logger.trace( - () -> new ParameterizedMessage( - "{} can't open index for shard [{}] in path [{}]", - shardId, - shardStateMetadata, - (finalShardPath != null) ? finalShardPath.resolveIndex() : "" - ), - exception - ); - String allocationId = shardStateMetadata.allocationId != null ? shardStateMetadata.allocationId.getId() : null; - return new NodeGatewayStartedShards( - clusterService.localNode(), - allocationId, - shardStateMetadata.primary, - null, - exception - ); - } - } - - logger.debug("{} shard state info found: [{}]", shardId, shardStateMetadata); - String allocationId = shardStateMetadata.allocationId != null ? shardStateMetadata.allocationId.getId() : null; - final IndexShard shard = indicesService.getShardOrNull(shardId); - return new NodeGatewayStartedShards( - clusterService.localNode(), - allocationId, - shardStateMetadata.primary, - shard != null ? shard.getLatestReplicationCheckpoint() : null - ); - } - logger.trace("{} no local shard info found", shardId); - return new NodeGatewayStartedShards(clusterService.localNode(), null, false, null); } catch (Exception e) { throw new OpenSearchException("failed to load started shards", e); } From 89c11439336754f19df4f44fbabd62087ab1777b Mon Sep 17 00:00:00 2001 From: Shivansh Arora Date: Mon, 8 Jan 2024 16:02:20 +0530 Subject: [PATCH 15/21] Fix build failure after main merge Signed-off-by: Shivansh Arora --- ...tNodesListGatewayStartedBatchShardsIT.java | 21 ++++++++++--------- 1 file changed, 11 insertions(+), 10 deletions(-) diff --git a/server/src/internalClusterTest/java/org/opensearch/gateway/TransportNodesListGatewayStartedBatchShardsIT.java b/server/src/internalClusterTest/java/org/opensearch/gateway/TransportNodesListGatewayStartedBatchShardsIT.java index 68ab35a6d45da..45028daf41bcf 100644 --- a/server/src/internalClusterTest/java/org/opensearch/gateway/TransportNodesListGatewayStartedBatchShardsIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/gateway/TransportNodesListGatewayStartedBatchShardsIT.java @@ -20,6 +20,7 @@ import org.opensearch.core.index.shard.ShardId; import org.opensearch.env.NodeEnvironment; import org.opensearch.index.shard.ShardPath; +import org.opensearch.indices.store.ShardAttributes; import org.opensearch.test.OpenSearchIntegTestCase; import java.io.IOException; @@ -39,14 +40,14 @@ public class TransportNodesListGatewayStartedBatchShardsIT extends OpenSearchInt public void testSingleShardFetch() throws Exception { String indexName = "test"; - Map shardIdCustomDataPathMap = prepareRequestMap(new String[] { indexName }, 1); + Map shardIdShardAttributesMap = prepareRequestMap(new String[] { indexName }, 1); ClusterSearchShardsResponse searchShardsResponse = client().admin().cluster().prepareSearchShards(indexName).get(); TransportNodesListGatewayStartedBatchShards.NodesGatewayStartedShardsBatch response; response = ActionTestUtils.executeBlocking( internalCluster().getInstance(TransportNodesListGatewayStartedBatchShards.class), - new TransportNodesListGatewayStartedBatchShards.Request(searchShardsResponse.getNodes(), shardIdCustomDataPathMap) + new TransportNodesListGatewayStartedBatchShards.Request(searchShardsResponse.getNodes(), shardIdShardAttributesMap) ); final Index index = resolveIndex(indexName); final ShardId shardId = new ShardId(index, 0); @@ -63,7 +64,7 @@ public void testShardFetchMultiNodeMultiIndexes() throws Exception { String indexName1 = "test1"; String indexName2 = "test2"; // assign one primary shard each to the data nodes - Map shardIdCustomDataPathMap = prepareRequestMap( + Map shardIdShardAttributesMap = prepareRequestMap( new String[] { indexName1, indexName2 }, internalCluster().numDataNodes() ); @@ -72,7 +73,7 @@ public void testShardFetchMultiNodeMultiIndexes() throws Exception { TransportNodesListGatewayStartedBatchShards.NodesGatewayStartedShardsBatch response; response = ActionTestUtils.executeBlocking( internalCluster().getInstance(TransportNodesListGatewayStartedBatchShards.class), - new TransportNodesListGatewayStartedBatchShards.Request(searchShardsResponse.getNodes(), shardIdCustomDataPathMap) + new TransportNodesListGatewayStartedBatchShards.Request(searchShardsResponse.getNodes(), shardIdShardAttributesMap) ); for (ClusterSearchShardsGroup clusterSearchShardsGroup : searchShardsResponse.getGroups()) { ShardId shardId = clusterSearchShardsGroup.getShardId(); @@ -88,7 +89,7 @@ public void testShardFetchMultiNodeMultiIndexes() throws Exception { public void testShardFetchCorruptedShards() throws Exception { String indexName = "test"; - Map shardIdCustomDataPathMap = prepareRequestMap(new String[] { indexName }, 1); + Map shardIdShardAttributesMap = prepareRequestMap(new String[] { indexName }, 1); ClusterSearchShardsResponse searchShardsResponse = client().admin().cluster().prepareSearchShards(indexName).get(); final Index index = resolveIndex(indexName); final ShardId shardId = new ShardId(index, 0); @@ -97,7 +98,7 @@ public void testShardFetchCorruptedShards() throws Exception { internalCluster().restartNode(searchShardsResponse.getNodes()[0].getName()); response = ActionTestUtils.executeBlocking( internalCluster().getInstance(TransportNodesListGatewayStartedBatchShards.class), - new TransportNodesListGatewayStartedBatchShards.Request(getDiscoveryNodes(), shardIdCustomDataPathMap) + new TransportNodesListGatewayStartedBatchShards.Request(getDiscoveryNodes(), shardIdShardAttributesMap) ); DiscoveryNode[] discoveryNodes = getDiscoveryNodes(); TransportNodesListGatewayStartedBatchShards.NodeGatewayStartedShard nodeGatewayStartedShards = response.getNodesMap() @@ -137,8 +138,8 @@ private void prepareIndex(String indexName, int numberOfPrimaryShards) { flush(indexName); } - private Map prepareRequestMap(String[] indices, int primaryShardCount) { - Map shardIdCustomDataPathMap = new HashMap<>(); + private Map prepareRequestMap(String[] indices, int primaryShardCount) { + Map shardIdShardAttributesMap = new HashMap<>(); for (String indexName : indices) { prepareIndex(indexName, primaryShardCount); final Index index = resolveIndex(indexName); @@ -147,10 +148,10 @@ private Map prepareRequestMap(String[] indices, int primaryShar ); for (int shardIdNum = 0; shardIdNum < primaryShardCount; shardIdNum++) { final ShardId shardId = new ShardId(index, shardIdNum); - shardIdCustomDataPathMap.put(shardId, customDataPath); + shardIdShardAttributesMap.put(shardId, new ShardAttributes(shardId, customDataPath)); } } - return shardIdCustomDataPathMap; + return shardIdShardAttributesMap; } private void corruptShard(String nodeName, ShardId shardId) throws IOException, InterruptedException { From 55e43cf479a907e9402b3c51df0e5d2e978399f6 Mon Sep 17 00:00:00 2001 From: Shivansh Arora Date: Thu, 25 Jan 2024 15:06:21 +0530 Subject: [PATCH 16/21] Renamed TransportNodesListGatewayStartedShardsBatch Signed-off-by: Shivansh Arora --- ...NodesListGatewayStartedShardsBatchIT.java} | 28 +++++++++---------- .../org/opensearch/gateway/GatewayModule.java | 2 +- ...ansportNodesGatewayStartedShardHelper.java | 12 ++++---- ...ransportNodesListGatewayStartedShards.java | 2 +- ...rtNodesListGatewayStartedShardsBatch.java} | 23 ++++++++------- 5 files changed, 35 insertions(+), 32 deletions(-) rename server/src/internalClusterTest/java/org/opensearch/gateway/{TransportNodesListGatewayStartedBatchShardsIT.java => TransportNodesListGatewayStartedShardsBatchIT.java} (91%) rename server/src/main/java/org/opensearch/gateway/{TransportNodesListGatewayStartedBatchShards.java => TransportNodesListGatewayStartedShardsBatch.java} (93%) diff --git a/server/src/internalClusterTest/java/org/opensearch/gateway/TransportNodesListGatewayStartedBatchShardsIT.java b/server/src/internalClusterTest/java/org/opensearch/gateway/TransportNodesListGatewayStartedShardsBatchIT.java similarity index 91% rename from server/src/internalClusterTest/java/org/opensearch/gateway/TransportNodesListGatewayStartedBatchShardsIT.java rename to server/src/internalClusterTest/java/org/opensearch/gateway/TransportNodesListGatewayStartedShardsBatchIT.java index 45028daf41bcf..748adca66154b 100644 --- a/server/src/internalClusterTest/java/org/opensearch/gateway/TransportNodesListGatewayStartedBatchShardsIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/gateway/TransportNodesListGatewayStartedShardsBatchIT.java @@ -36,7 +36,7 @@ import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_NUMBER_OF_REPLICAS; import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_NUMBER_OF_SHARDS; -public class TransportNodesListGatewayStartedBatchShardsIT extends OpenSearchIntegTestCase { +public class TransportNodesListGatewayStartedShardsBatchIT extends OpenSearchIntegTestCase { public void testSingleShardFetch() throws Exception { String indexName = "test"; @@ -44,14 +44,14 @@ public void testSingleShardFetch() throws Exception { ClusterSearchShardsResponse searchShardsResponse = client().admin().cluster().prepareSearchShards(indexName).get(); - TransportNodesListGatewayStartedBatchShards.NodesGatewayStartedShardsBatch response; + TransportNodesListGatewayStartedShardsBatch.NodesGatewayStartedShardsBatch response; response = ActionTestUtils.executeBlocking( - internalCluster().getInstance(TransportNodesListGatewayStartedBatchShards.class), - new TransportNodesListGatewayStartedBatchShards.Request(searchShardsResponse.getNodes(), shardIdShardAttributesMap) + internalCluster().getInstance(TransportNodesListGatewayStartedShardsBatch.class), + new TransportNodesListGatewayStartedShardsBatch.Request(searchShardsResponse.getNodes(), shardIdShardAttributesMap) ); final Index index = resolveIndex(indexName); final ShardId shardId = new ShardId(index, 0); - TransportNodesListGatewayStartedBatchShards.NodeGatewayStartedShard nodeGatewayStartedShards = response.getNodesMap() + TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShard nodeGatewayStartedShards = response.getNodesMap() .get(searchShardsResponse.getNodes()[0].getId()) .getNodeGatewayStartedShardsBatch() .get(shardId); @@ -70,16 +70,16 @@ public void testShardFetchMultiNodeMultiIndexes() throws Exception { ); ClusterSearchShardsResponse searchShardsResponse = client().admin().cluster().prepareSearchShards(indexName1, indexName2).get(); assertEquals(internalCluster().numDataNodes(), searchShardsResponse.getNodes().length); - TransportNodesListGatewayStartedBatchShards.NodesGatewayStartedShardsBatch response; + TransportNodesListGatewayStartedShardsBatch.NodesGatewayStartedShardsBatch response; response = ActionTestUtils.executeBlocking( - internalCluster().getInstance(TransportNodesListGatewayStartedBatchShards.class), - new TransportNodesListGatewayStartedBatchShards.Request(searchShardsResponse.getNodes(), shardIdShardAttributesMap) + internalCluster().getInstance(TransportNodesListGatewayStartedShardsBatch.class), + new TransportNodesListGatewayStartedShardsBatch.Request(searchShardsResponse.getNodes(), shardIdShardAttributesMap) ); for (ClusterSearchShardsGroup clusterSearchShardsGroup : searchShardsResponse.getGroups()) { ShardId shardId = clusterSearchShardsGroup.getShardId(); assertEquals(1, clusterSearchShardsGroup.getShards().length); String nodeId = clusterSearchShardsGroup.getShards()[0].currentNodeId(); - TransportNodesListGatewayStartedBatchShards.NodeGatewayStartedShard nodeGatewayStartedShards = response.getNodesMap() + TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShard nodeGatewayStartedShards = response.getNodesMap() .get(nodeId) .getNodeGatewayStartedShardsBatch() .get(shardId); @@ -94,14 +94,14 @@ public void testShardFetchCorruptedShards() throws Exception { final Index index = resolveIndex(indexName); final ShardId shardId = new ShardId(index, 0); corruptShard(searchShardsResponse.getNodes()[0].getName(), shardId); - TransportNodesListGatewayStartedBatchShards.NodesGatewayStartedShardsBatch response; + TransportNodesListGatewayStartedShardsBatch.NodesGatewayStartedShardsBatch response; internalCluster().restartNode(searchShardsResponse.getNodes()[0].getName()); response = ActionTestUtils.executeBlocking( - internalCluster().getInstance(TransportNodesListGatewayStartedBatchShards.class), - new TransportNodesListGatewayStartedBatchShards.Request(getDiscoveryNodes(), shardIdShardAttributesMap) + internalCluster().getInstance(TransportNodesListGatewayStartedShardsBatch.class), + new TransportNodesListGatewayStartedShardsBatch.Request(getDiscoveryNodes(), shardIdShardAttributesMap) ); DiscoveryNode[] discoveryNodes = getDiscoveryNodes(); - TransportNodesListGatewayStartedBatchShards.NodeGatewayStartedShard nodeGatewayStartedShards = response.getNodesMap() + TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShard nodeGatewayStartedShards = response.getNodesMap() .get(discoveryNodes[0].getId()) .getNodeGatewayStartedShardsBatch() .get(shardId); @@ -122,7 +122,7 @@ private DiscoveryNode[] getDiscoveryNodes() throws ExecutionException, Interrupt } private void assertNodeGatewayStartedShardsHappyCase( - TransportNodesListGatewayStartedBatchShards.NodeGatewayStartedShard nodeGatewayStartedShards + TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShard nodeGatewayStartedShards ) { assertNull(nodeGatewayStartedShards.storeException()); assertNotNull(nodeGatewayStartedShards.allocationId()); diff --git a/server/src/main/java/org/opensearch/gateway/GatewayModule.java b/server/src/main/java/org/opensearch/gateway/GatewayModule.java index e6c3176e8adbf..847ba01737332 100644 --- a/server/src/main/java/org/opensearch/gateway/GatewayModule.java +++ b/server/src/main/java/org/opensearch/gateway/GatewayModule.java @@ -47,7 +47,7 @@ protected void configure() { bind(GatewayService.class).asEagerSingleton(); bind(TransportNodesListGatewayMetaState.class).asEagerSingleton(); bind(TransportNodesListGatewayStartedShards.class).asEagerSingleton(); - bind(TransportNodesListGatewayStartedBatchShards.class).asEagerSingleton(); + bind(TransportNodesListGatewayStartedShardsBatch.class).asEagerSingleton(); bind(LocalAllocateDangledIndices.class).asEagerSingleton(); } } diff --git a/server/src/main/java/org/opensearch/gateway/TransportNodesGatewayStartedShardHelper.java b/server/src/main/java/org/opensearch/gateway/TransportNodesGatewayStartedShardHelper.java index 471b10426b7e0..403e3e96fa209 100644 --- a/server/src/main/java/org/opensearch/gateway/TransportNodesGatewayStartedShardHelper.java +++ b/server/src/main/java/org/opensearch/gateway/TransportNodesGatewayStartedShardHelper.java @@ -28,16 +28,16 @@ /** * This class has the common code used in {@link TransportNodesListGatewayStartedShards} and - * {@link TransportNodesListGatewayStartedBatchShards} to get the shard info on the local node. + * {@link TransportNodesListGatewayStartedShardsBatch} to get the shard info on the local node. *

* This class should not be used to add more functions and will be removed when the * {@link TransportNodesListGatewayStartedShards} will be deprecated and all the code will be moved to - * {@link TransportNodesListGatewayStartedBatchShards} + * {@link TransportNodesListGatewayStartedShardsBatch} * * @opensearch.internal */ public class TransportNodesGatewayStartedShardHelper { - public static TransportNodesListGatewayStartedBatchShards.NodeGatewayStartedShard getShardInfoOnLocalNode( + public static TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShard getShardInfoOnLocalNode( Logger logger, final ShardId shardId, NamedXContentRegistry namedXContentRegistry, @@ -90,7 +90,7 @@ public static TransportNodesListGatewayStartedBatchShards.NodeGatewayStartedShar exception ); String allocationId = shardStateMetadata.allocationId != null ? shardStateMetadata.allocationId.getId() : null; - return new TransportNodesListGatewayStartedBatchShards.NodeGatewayStartedShard( + return new TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShard( allocationId, shardStateMetadata.primary, null, @@ -102,13 +102,13 @@ public static TransportNodesListGatewayStartedBatchShards.NodeGatewayStartedShar logger.debug("{} shard state info found: [{}]", shardId, shardStateMetadata); String allocationId = shardStateMetadata.allocationId != null ? shardStateMetadata.allocationId.getId() : null; final IndexShard shard = indicesService.getShardOrNull(shardId); - return new TransportNodesListGatewayStartedBatchShards.NodeGatewayStartedShard( + return new TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShard( allocationId, shardStateMetadata.primary, shard != null ? shard.getLatestReplicationCheckpoint() : null ); } logger.trace("{} no local shard info found", shardId); - return new TransportNodesListGatewayStartedBatchShards.NodeGatewayStartedShard(null, false, null); + return new TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShard(null, false, null); } } diff --git a/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedShards.java b/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedShards.java index b9c8e9023c897..0ba872aab9974 100644 --- a/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedShards.java +++ b/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedShards.java @@ -154,7 +154,7 @@ protected NodesGatewayStartedShards newResponse( @Override protected NodeGatewayStartedShards nodeOperation(NodeRequest request) { try { - TransportNodesListGatewayStartedBatchShards.NodeGatewayStartedShard shardInfo = getShardInfoOnLocalNode( + TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShard shardInfo = getShardInfoOnLocalNode( logger, request.getShardId(), namedXContentRegistry, diff --git a/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedBatchShards.java b/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedShardsBatch.java similarity index 93% rename from server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedBatchShards.java rename to server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedShardsBatch.java index b88efd12c958b..2ac216bed9fb6 100644 --- a/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedBatchShards.java +++ b/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedShardsBatch.java @@ -50,15 +50,15 @@ * * @opensearch.internal */ -public class TransportNodesListGatewayStartedBatchShards extends TransportNodesAction< - TransportNodesListGatewayStartedBatchShards.Request, - TransportNodesListGatewayStartedBatchShards.NodesGatewayStartedShardsBatch, - TransportNodesListGatewayStartedBatchShards.NodeRequest, - TransportNodesListGatewayStartedBatchShards.NodeGatewayStartedShardsBatch> +public class TransportNodesListGatewayStartedShardsBatch extends TransportNodesAction< + TransportNodesListGatewayStartedShardsBatch.Request, + TransportNodesListGatewayStartedShardsBatch.NodesGatewayStartedShardsBatch, + TransportNodesListGatewayStartedShardsBatch.NodeRequest, + TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShardsBatch> implements AsyncShardFetch.Lister< - TransportNodesListGatewayStartedBatchShards.NodesGatewayStartedShardsBatch, - TransportNodesListGatewayStartedBatchShards.NodeGatewayStartedShardsBatch> { + TransportNodesListGatewayStartedShardsBatch.NodesGatewayStartedShardsBatch, + TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShardsBatch> { public static final String ACTION_NAME = "internal:gateway/local/started_shards_batch"; public static final ActionType TYPE = new ActionType<>( @@ -72,7 +72,7 @@ public class TransportNodesListGatewayStartedBatchShards extends TransportNodesA private final NamedXContentRegistry namedXContentRegistry; @Inject - public TransportNodesListGatewayStartedBatchShards( + public TransportNodesListGatewayStartedShardsBatch( Settings settings, ThreadPool threadPool, ClusterService clusterService, @@ -250,8 +250,11 @@ public void writeTo(StreamOutput out) throws IOException { } /** - * Class for storing shard information received from other node. - * This is used in {@link NodeGatewayStartedShardsBatch} to construct the response for each node + * This class encapsulates the metadata about a started shard that needs to be persisted or sent between nodes. + * This is used in {@link NodeGatewayStartedShardsBatch} to construct the response for each node, instead of + * {@link TransportNodesListGatewayStartedShards.NodeGatewayStartedShards} because we don't need to save an extra + * {@link DiscoveryNode} object like in {@link TransportNodesListGatewayStartedShards.NodeGatewayStartedShards} + * which reduces memory footprint of its objects. * * @opensearch.internal */ From 551fc60c3e3e92b6a790d5f40e642d7ee77842d2 Mon Sep 17 00:00:00 2001 From: Shivansh Arora Date: Thu, 25 Jan 2024 17:38:40 +0530 Subject: [PATCH 17/21] Address review comment Signed-off-by: Shivansh Arora --- .../gateway/AsyncShardFetchTestUtils.java | 11 ++-- ...tNodesListGatewayStartedShardsBatchIT.java | 58 ++++--------------- ...ortNodesListGatewayStartedShardsBatch.java | 15 ++--- 3 files changed, 24 insertions(+), 60 deletions(-) diff --git a/server/src/internalClusterTest/java/org/opensearch/gateway/AsyncShardFetchTestUtils.java b/server/src/internalClusterTest/java/org/opensearch/gateway/AsyncShardFetchTestUtils.java index 7e0e61f4232c9..8c524f7115fa1 100644 --- a/server/src/internalClusterTest/java/org/opensearch/gateway/AsyncShardFetchTestUtils.java +++ b/server/src/internalClusterTest/java/org/opensearch/gateway/AsyncShardFetchTestUtils.java @@ -16,6 +16,7 @@ import org.opensearch.core.index.shard.ShardId; import org.opensearch.env.NodeEnvironment; import org.opensearch.index.shard.ShardPath; +import org.opensearch.indices.store.ShardAttributes; import java.io.IOException; import java.nio.file.DirectoryStream; @@ -44,19 +45,19 @@ public static DiscoveryNode[] getDiscoveryNodes() throws ExecutionException, Int return disNodesArr; } - public static Map prepareRequestMap(String[] indices, int shardCount) { - Map shardIdCustomDataPathMap = new HashMap<>(); + public static Map prepareRequestMap(String[] indices, int primaryShardCount) { + Map shardIdShardAttributesMap = new HashMap<>(); for (String indexName : indices) { final Index index = resolveIndex(indexName); final String customDataPath = IndexMetadata.INDEX_DATA_PATH_SETTING.get( client().admin().indices().prepareGetSettings(indexName).get().getIndexToSettings().get(indexName) ); - for (int shardIdNum = 0; shardIdNum < shardCount; shardIdNum++) { + for (int shardIdNum = 0; shardIdNum < primaryShardCount; shardIdNum++) { final ShardId shardId = new ShardId(index, shardIdNum); - shardIdCustomDataPathMap.put(shardId, customDataPath); + shardIdShardAttributesMap.put(shardId, new ShardAttributes(shardId, customDataPath)); } } - return shardIdCustomDataPathMap; + return shardIdShardAttributesMap; } public static void corruptShard(String nodeName, ShardId shardId) throws IOException, InterruptedException { diff --git a/server/src/internalClusterTest/java/org/opensearch/gateway/TransportNodesListGatewayStartedShardsBatchIT.java b/server/src/internalClusterTest/java/org/opensearch/gateway/TransportNodesListGatewayStartedShardsBatchIT.java index 748adca66154b..7bd4aa3016318 100644 --- a/server/src/internalClusterTest/java/org/opensearch/gateway/TransportNodesListGatewayStartedShardsBatchIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/gateway/TransportNodesListGatewayStartedShardsBatchIT.java @@ -13,21 +13,13 @@ import org.opensearch.action.admin.cluster.state.ClusterStateRequest; import org.opensearch.action.admin.cluster.state.ClusterStateResponse; import org.opensearch.action.support.ActionTestUtils; -import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.cluster.node.DiscoveryNode; import org.opensearch.common.settings.Settings; import org.opensearch.core.index.Index; import org.opensearch.core.index.shard.ShardId; -import org.opensearch.env.NodeEnvironment; -import org.opensearch.index.shard.ShardPath; import org.opensearch.indices.store.ShardAttributes; import org.opensearch.test.OpenSearchIntegTestCase; -import java.io.IOException; -import java.nio.file.DirectoryStream; -import java.nio.file.Files; -import java.nio.file.Path; -import java.util.HashMap; import java.util.LinkedList; import java.util.List; import java.util.Map; @@ -35,12 +27,16 @@ import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_NUMBER_OF_REPLICAS; import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_NUMBER_OF_SHARDS; +import static org.opensearch.gateway.AsyncShardFetchTestUtils.corruptShard; +import static org.opensearch.gateway.AsyncShardFetchTestUtils.prepareRequestMap; public class TransportNodesListGatewayStartedShardsBatchIT extends OpenSearchIntegTestCase { public void testSingleShardFetch() throws Exception { String indexName = "test"; - Map shardIdShardAttributesMap = prepareRequestMap(new String[] { indexName }, 1); + int numOfShards = 1; + prepareIndex(indexName, numOfShards); + Map shardIdShardAttributesMap = prepareRequestMap(new String[] { indexName }, numOfShards); ClusterSearchShardsResponse searchShardsResponse = client().admin().cluster().prepareSearchShards(indexName).get(); @@ -63,11 +59,11 @@ public void testShardFetchMultiNodeMultiIndexes() throws Exception { internalCluster().startNode(); String indexName1 = "test1"; String indexName2 = "test2"; + int numShards = internalCluster().numDataNodes(); // assign one primary shard each to the data nodes - Map shardIdShardAttributesMap = prepareRequestMap( - new String[] { indexName1, indexName2 }, - internalCluster().numDataNodes() - ); + prepareIndex(indexName1, numShards); + prepareIndex(indexName2, numShards); + Map shardIdShardAttributesMap = prepareRequestMap(new String[] { indexName1, indexName2 }, numShards); ClusterSearchShardsResponse searchShardsResponse = client().admin().cluster().prepareSearchShards(indexName1, indexName2).get(); assertEquals(internalCluster().numDataNodes(), searchShardsResponse.getNodes().length); TransportNodesListGatewayStartedShardsBatch.NodesGatewayStartedShardsBatch response; @@ -89,7 +85,9 @@ public void testShardFetchMultiNodeMultiIndexes() throws Exception { public void testShardFetchCorruptedShards() throws Exception { String indexName = "test"; - Map shardIdShardAttributesMap = prepareRequestMap(new String[] { indexName }, 1); + int numOfShards = 1; + prepareIndex(indexName, numOfShards); + Map shardIdShardAttributesMap = prepareRequestMap(new String[] { indexName }, numOfShards); ClusterSearchShardsResponse searchShardsResponse = client().admin().cluster().prepareSearchShards(indexName).get(); final Index index = resolveIndex(indexName); final ShardId shardId = new ShardId(index, 0); @@ -137,36 +135,4 @@ private void prepareIndex(String indexName, int numberOfPrimaryShards) { index(indexName, "type", "1"); flush(indexName); } - - private Map prepareRequestMap(String[] indices, int primaryShardCount) { - Map shardIdShardAttributesMap = new HashMap<>(); - for (String indexName : indices) { - prepareIndex(indexName, primaryShardCount); - final Index index = resolveIndex(indexName); - final String customDataPath = IndexMetadata.INDEX_DATA_PATH_SETTING.get( - client().admin().indices().prepareGetSettings(indexName).get().getIndexToSettings().get(indexName) - ); - for (int shardIdNum = 0; shardIdNum < primaryShardCount; shardIdNum++) { - final ShardId shardId = new ShardId(index, shardIdNum); - shardIdShardAttributesMap.put(shardId, new ShardAttributes(shardId, customDataPath)); - } - } - return shardIdShardAttributesMap; - } - - private void corruptShard(String nodeName, ShardId shardId) throws IOException, InterruptedException { - for (Path path : internalCluster().getInstance(NodeEnvironment.class, nodeName).availableShardPaths(shardId)) { - final Path indexPath = path.resolve(ShardPath.INDEX_FOLDER_NAME); - if (Files.exists(indexPath)) { // multi data path might only have one path in use - try (DirectoryStream stream = Files.newDirectoryStream(indexPath)) { - for (Path item : stream) { - if (item.getFileName().toString().startsWith("segments_")) { - logger.info("--> deleting [{}]", item); - Files.delete(item); - } - } - } - } - } - } } diff --git a/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedShardsBatch.java b/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedShardsBatch.java index 2ac216bed9fb6..bc327c1b85748 100644 --- a/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedShardsBatch.java +++ b/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedShardsBatch.java @@ -9,7 +9,6 @@ package org.opensearch.gateway; import org.opensearch.OpenSearchException; -import org.opensearch.Version; import org.opensearch.action.ActionType; import org.opensearch.action.FailedNodeException; import org.opensearch.action.support.ActionFilters; @@ -272,7 +271,7 @@ public NodeGatewayStartedShard(StreamInput in) throws IOException { } else { storeException = null; } - if (in.getVersion().onOrAfter(Version.V_2_3_0) && in.readBoolean()) { + if (in.readBoolean()) { replicationCheckpoint = new ReplicationCheckpoint(in); } else { replicationCheckpoint = null; @@ -320,13 +319,11 @@ public void writeTo(StreamOutput out) throws IOException { } else { out.writeBoolean(false); } - if (out.getVersion().onOrAfter(Version.V_2_3_0)) { - if (replicationCheckpoint != null) { - out.writeBoolean(true); - replicationCheckpoint.writeTo(out); - } else { - out.writeBoolean(false); - } + if (replicationCheckpoint != null) { + out.writeBoolean(true); + replicationCheckpoint.writeTo(out); + } else { + out.writeBoolean(false); } } From b57c3300a6f3396cd44240c2b352ce70dd02f8d6 Mon Sep 17 00:00:00 2001 From: Shivansh Arora Date: Mon, 29 Jan 2024 14:57:15 +0530 Subject: [PATCH 18/21] Move integration tests to RecoveryFromGatewayIT Signed-off-by: Shivansh Arora --- ...ils.java => GatewayRecoveryTestUtils.java} | 2 +- .../gateway/RecoveryFromGatewayIT.java | 99 +++++++++++++ ...tNodesListGatewayStartedShardsBatchIT.java | 138 ------------------ 3 files changed, 100 insertions(+), 139 deletions(-) rename server/src/internalClusterTest/java/org/opensearch/gateway/{AsyncShardFetchTestUtils.java => GatewayRecoveryTestUtils.java} (98%) delete mode 100644 server/src/internalClusterTest/java/org/opensearch/gateway/TransportNodesListGatewayStartedShardsBatchIT.java diff --git a/server/src/internalClusterTest/java/org/opensearch/gateway/AsyncShardFetchTestUtils.java b/server/src/internalClusterTest/java/org/opensearch/gateway/GatewayRecoveryTestUtils.java similarity index 98% rename from server/src/internalClusterTest/java/org/opensearch/gateway/AsyncShardFetchTestUtils.java rename to server/src/internalClusterTest/java/org/opensearch/gateway/GatewayRecoveryTestUtils.java index 8c524f7115fa1..2b6a5b4ee6867 100644 --- a/server/src/internalClusterTest/java/org/opensearch/gateway/AsyncShardFetchTestUtils.java +++ b/server/src/internalClusterTest/java/org/opensearch/gateway/GatewayRecoveryTestUtils.java @@ -32,7 +32,7 @@ import static org.opensearch.test.OpenSearchIntegTestCase.internalCluster; import static org.opensearch.test.OpenSearchIntegTestCase.resolveIndex; -public class AsyncShardFetchTestUtils { +public class GatewayRecoveryTestUtils { public static DiscoveryNode[] getDiscoveryNodes() throws ExecutionException, InterruptedException { final ClusterStateRequest clusterStateRequest = new ClusterStateRequest(); diff --git a/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java b/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java index 229cd7bffad2f..80137ba407135 100644 --- a/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java @@ -36,6 +36,8 @@ import org.opensearch.action.admin.cluster.configuration.AddVotingConfigExclusionsRequest; import org.opensearch.action.admin.cluster.configuration.ClearVotingConfigExclusionsAction; import org.opensearch.action.admin.cluster.configuration.ClearVotingConfigExclusionsRequest; +import org.opensearch.action.admin.cluster.shards.ClusterSearchShardsGroup; +import org.opensearch.action.admin.cluster.shards.ClusterSearchShardsResponse; import org.opensearch.action.admin.indices.recovery.RecoveryResponse; import org.opensearch.action.admin.indices.stats.IndexStats; import org.opensearch.action.admin.indices.stats.ShardStats; @@ -60,6 +62,7 @@ import org.opensearch.indices.IndicesService; import org.opensearch.indices.recovery.RecoveryState; import org.opensearch.indices.replication.common.ReplicationLuceneIndex; +import org.opensearch.indices.store.ShardAttributes; import org.opensearch.plugins.Plugin; import org.opensearch.test.InternalSettingsPlugin; import org.opensearch.test.InternalTestCluster.RestartCallback; @@ -85,6 +88,9 @@ import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_NUMBER_OF_REPLICAS; import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_NUMBER_OF_SHARDS; import static org.opensearch.common.xcontent.XContentFactory.jsonBuilder; +import static org.opensearch.gateway.GatewayRecoveryTestUtils.corruptShard; +import static org.opensearch.gateway.GatewayRecoveryTestUtils.getDiscoveryNodes; +import static org.opensearch.gateway.GatewayRecoveryTestUtils.prepareRequestMap; import static org.opensearch.gateway.GatewayService.RECOVER_AFTER_NODES_SETTING; import static org.opensearch.index.query.QueryBuilders.matchAllQuery; import static org.opensearch.index.query.QueryBuilders.termQuery; @@ -734,4 +740,97 @@ public void testMessyElectionsStillMakeClusterGoGreen() throws Exception { internalCluster().fullRestart(); ensureGreen("test"); } + + public void testSingleShardFetch() { + String indexName = "test"; + int numOfShards = 1; + prepareIndex(indexName, numOfShards); + Map shardIdShardAttributesMap = prepareRequestMap(new String[] { indexName }, numOfShards); + + ClusterSearchShardsResponse searchShardsResponse = client().admin().cluster().prepareSearchShards(indexName).get(); + + TransportNodesListGatewayStartedShardsBatch.NodesGatewayStartedShardsBatch response; + response = ActionTestUtils.executeBlocking( + internalCluster().getInstance(TransportNodesListGatewayStartedShardsBatch.class), + new TransportNodesListGatewayStartedShardsBatch.Request(searchShardsResponse.getNodes(), shardIdShardAttributesMap) + ); + final Index index = resolveIndex(indexName); + final ShardId shardId = new ShardId(index, 0); + TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShard nodeGatewayStartedShards = response.getNodesMap() + .get(searchShardsResponse.getNodes()[0].getId()) + .getNodeGatewayStartedShardsBatch() + .get(shardId); + assertNodeGatewayStartedShardsHappyCase(nodeGatewayStartedShards); + } + + public void testShardFetchMultiNodeMultiIndexes() { + // start second node + internalCluster().startNode(); + String indexName1 = "test1"; + String indexName2 = "test2"; + int numShards = internalCluster().numDataNodes(); + // assign one primary shard each to the data nodes + prepareIndex(indexName1, numShards); + prepareIndex(indexName2, numShards); + Map shardIdShardAttributesMap = prepareRequestMap(new String[] { indexName1, indexName2 }, numShards); + ClusterSearchShardsResponse searchShardsResponse = client().admin().cluster().prepareSearchShards(indexName1, indexName2).get(); + assertEquals(internalCluster().numDataNodes(), searchShardsResponse.getNodes().length); + TransportNodesListGatewayStartedShardsBatch.NodesGatewayStartedShardsBatch response; + response = ActionTestUtils.executeBlocking( + internalCluster().getInstance(TransportNodesListGatewayStartedShardsBatch.class), + new TransportNodesListGatewayStartedShardsBatch.Request(searchShardsResponse.getNodes(), shardIdShardAttributesMap) + ); + for (ClusterSearchShardsGroup clusterSearchShardsGroup : searchShardsResponse.getGroups()) { + ShardId shardId = clusterSearchShardsGroup.getShardId(); + assertEquals(1, clusterSearchShardsGroup.getShards().length); + String nodeId = clusterSearchShardsGroup.getShards()[0].currentNodeId(); + TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShard nodeGatewayStartedShards = response.getNodesMap() + .get(nodeId) + .getNodeGatewayStartedShardsBatch() + .get(shardId); + assertNodeGatewayStartedShardsHappyCase(nodeGatewayStartedShards); + } + } + + public void testShardFetchCorruptedShards() throws Exception { + String indexName = "test"; + int numOfShards = 1; + prepareIndex(indexName, numOfShards); + Map shardIdShardAttributesMap = prepareRequestMap(new String[] { indexName }, numOfShards); + ClusterSearchShardsResponse searchShardsResponse = client().admin().cluster().prepareSearchShards(indexName).get(); + final Index index = resolveIndex(indexName); + final ShardId shardId = new ShardId(index, 0); + corruptShard(searchShardsResponse.getNodes()[0].getName(), shardId); + TransportNodesListGatewayStartedShardsBatch.NodesGatewayStartedShardsBatch response; + internalCluster().restartNode(searchShardsResponse.getNodes()[0].getName()); + response = ActionTestUtils.executeBlocking( + internalCluster().getInstance(TransportNodesListGatewayStartedShardsBatch.class), + new TransportNodesListGatewayStartedShardsBatch.Request(getDiscoveryNodes(), shardIdShardAttributesMap) + ); + DiscoveryNode[] discoveryNodes = getDiscoveryNodes(); + TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShard nodeGatewayStartedShards = response.getNodesMap() + .get(discoveryNodes[0].getId()) + .getNodeGatewayStartedShardsBatch() + .get(shardId); + assertNotNull(nodeGatewayStartedShards.storeException()); + assertNotNull(nodeGatewayStartedShards.allocationId()); + assertTrue(nodeGatewayStartedShards.primary()); + } + + private void assertNodeGatewayStartedShardsHappyCase( + TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShard nodeGatewayStartedShards + ) { + assertNull(nodeGatewayStartedShards.storeException()); + assertNotNull(nodeGatewayStartedShards.allocationId()); + assertTrue(nodeGatewayStartedShards.primary()); + } + + private void prepareIndex(String indexName, int numberOfPrimaryShards) { + createIndex( + indexName, + Settings.builder().put(SETTING_NUMBER_OF_SHARDS, numberOfPrimaryShards).put(SETTING_NUMBER_OF_REPLICAS, 0).build() + ); + index(indexName, "type", "1", Collections.emptyMap()); + flush(indexName); + } } diff --git a/server/src/internalClusterTest/java/org/opensearch/gateway/TransportNodesListGatewayStartedShardsBatchIT.java b/server/src/internalClusterTest/java/org/opensearch/gateway/TransportNodesListGatewayStartedShardsBatchIT.java deleted file mode 100644 index 7bd4aa3016318..0000000000000 --- a/server/src/internalClusterTest/java/org/opensearch/gateway/TransportNodesListGatewayStartedShardsBatchIT.java +++ /dev/null @@ -1,138 +0,0 @@ -/* - * 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; - -import org.opensearch.action.admin.cluster.shards.ClusterSearchShardsGroup; -import org.opensearch.action.admin.cluster.shards.ClusterSearchShardsResponse; -import org.opensearch.action.admin.cluster.state.ClusterStateRequest; -import org.opensearch.action.admin.cluster.state.ClusterStateResponse; -import org.opensearch.action.support.ActionTestUtils; -import org.opensearch.cluster.node.DiscoveryNode; -import org.opensearch.common.settings.Settings; -import org.opensearch.core.index.Index; -import org.opensearch.core.index.shard.ShardId; -import org.opensearch.indices.store.ShardAttributes; -import org.opensearch.test.OpenSearchIntegTestCase; - -import java.util.LinkedList; -import java.util.List; -import java.util.Map; -import java.util.concurrent.ExecutionException; - -import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_NUMBER_OF_REPLICAS; -import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_NUMBER_OF_SHARDS; -import static org.opensearch.gateway.AsyncShardFetchTestUtils.corruptShard; -import static org.opensearch.gateway.AsyncShardFetchTestUtils.prepareRequestMap; - -public class TransportNodesListGatewayStartedShardsBatchIT extends OpenSearchIntegTestCase { - - public void testSingleShardFetch() throws Exception { - String indexName = "test"; - int numOfShards = 1; - prepareIndex(indexName, numOfShards); - Map shardIdShardAttributesMap = prepareRequestMap(new String[] { indexName }, numOfShards); - - ClusterSearchShardsResponse searchShardsResponse = client().admin().cluster().prepareSearchShards(indexName).get(); - - TransportNodesListGatewayStartedShardsBatch.NodesGatewayStartedShardsBatch response; - response = ActionTestUtils.executeBlocking( - internalCluster().getInstance(TransportNodesListGatewayStartedShardsBatch.class), - new TransportNodesListGatewayStartedShardsBatch.Request(searchShardsResponse.getNodes(), shardIdShardAttributesMap) - ); - final Index index = resolveIndex(indexName); - final ShardId shardId = new ShardId(index, 0); - TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShard nodeGatewayStartedShards = response.getNodesMap() - .get(searchShardsResponse.getNodes()[0].getId()) - .getNodeGatewayStartedShardsBatch() - .get(shardId); - assertNodeGatewayStartedShardsHappyCase(nodeGatewayStartedShards); - } - - public void testShardFetchMultiNodeMultiIndexes() throws Exception { - // start second node - internalCluster().startNode(); - String indexName1 = "test1"; - String indexName2 = "test2"; - int numShards = internalCluster().numDataNodes(); - // assign one primary shard each to the data nodes - prepareIndex(indexName1, numShards); - prepareIndex(indexName2, numShards); - Map shardIdShardAttributesMap = prepareRequestMap(new String[] { indexName1, indexName2 }, numShards); - ClusterSearchShardsResponse searchShardsResponse = client().admin().cluster().prepareSearchShards(indexName1, indexName2).get(); - assertEquals(internalCluster().numDataNodes(), searchShardsResponse.getNodes().length); - TransportNodesListGatewayStartedShardsBatch.NodesGatewayStartedShardsBatch response; - response = ActionTestUtils.executeBlocking( - internalCluster().getInstance(TransportNodesListGatewayStartedShardsBatch.class), - new TransportNodesListGatewayStartedShardsBatch.Request(searchShardsResponse.getNodes(), shardIdShardAttributesMap) - ); - for (ClusterSearchShardsGroup clusterSearchShardsGroup : searchShardsResponse.getGroups()) { - ShardId shardId = clusterSearchShardsGroup.getShardId(); - assertEquals(1, clusterSearchShardsGroup.getShards().length); - String nodeId = clusterSearchShardsGroup.getShards()[0].currentNodeId(); - TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShard nodeGatewayStartedShards = response.getNodesMap() - .get(nodeId) - .getNodeGatewayStartedShardsBatch() - .get(shardId); - assertNodeGatewayStartedShardsHappyCase(nodeGatewayStartedShards); - } - } - - public void testShardFetchCorruptedShards() throws Exception { - String indexName = "test"; - int numOfShards = 1; - prepareIndex(indexName, numOfShards); - Map shardIdShardAttributesMap = prepareRequestMap(new String[] { indexName }, numOfShards); - ClusterSearchShardsResponse searchShardsResponse = client().admin().cluster().prepareSearchShards(indexName).get(); - final Index index = resolveIndex(indexName); - final ShardId shardId = new ShardId(index, 0); - corruptShard(searchShardsResponse.getNodes()[0].getName(), shardId); - TransportNodesListGatewayStartedShardsBatch.NodesGatewayStartedShardsBatch response; - internalCluster().restartNode(searchShardsResponse.getNodes()[0].getName()); - response = ActionTestUtils.executeBlocking( - internalCluster().getInstance(TransportNodesListGatewayStartedShardsBatch.class), - new TransportNodesListGatewayStartedShardsBatch.Request(getDiscoveryNodes(), shardIdShardAttributesMap) - ); - DiscoveryNode[] discoveryNodes = getDiscoveryNodes(); - TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShard nodeGatewayStartedShards = response.getNodesMap() - .get(discoveryNodes[0].getId()) - .getNodeGatewayStartedShardsBatch() - .get(shardId); - assertNotNull(nodeGatewayStartedShards.storeException()); - assertNotNull(nodeGatewayStartedShards.allocationId()); - assertTrue(nodeGatewayStartedShards.primary()); - } - - private DiscoveryNode[] getDiscoveryNodes() throws ExecutionException, InterruptedException { - final ClusterStateRequest clusterStateRequest = new ClusterStateRequest(); - clusterStateRequest.local(false); - clusterStateRequest.clear().nodes(true).routingTable(true).indices("*"); - ClusterStateResponse clusterStateResponse = client().admin().cluster().state(clusterStateRequest).get(); - final List nodes = new LinkedList<>(clusterStateResponse.getState().nodes().getDataNodes().values()); - DiscoveryNode[] disNodesArr = new DiscoveryNode[nodes.size()]; - nodes.toArray(disNodesArr); - return disNodesArr; - } - - private void assertNodeGatewayStartedShardsHappyCase( - TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShard nodeGatewayStartedShards - ) { - assertNull(nodeGatewayStartedShards.storeException()); - assertNotNull(nodeGatewayStartedShards.allocationId()); - assertTrue(nodeGatewayStartedShards.primary()); - } - - private void prepareIndex(String indexName, int numberOfPrimaryShards) { - createIndex( - indexName, - Settings.builder().put(SETTING_NUMBER_OF_SHARDS, numberOfPrimaryShards).put(SETTING_NUMBER_OF_REPLICAS, 0).build() - ); - index(indexName, "type", "1"); - flush(indexName); - } -} From 1dbd24864823ba46b255bca9f7f541234f2e59df Mon Sep 17 00:00:00 2001 From: Shivansh Arora Date: Thu, 1 Feb 2024 16:33:33 +0530 Subject: [PATCH 19/21] Added UsingBatchAction suffix to ITs Signed-off-by: Shivansh Arora --- .../java/org/opensearch/gateway/RecoveryFromGatewayIT.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java b/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java index 80137ba407135..7d7a827209e27 100644 --- a/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java @@ -741,7 +741,7 @@ public void testMessyElectionsStillMakeClusterGoGreen() throws Exception { ensureGreen("test"); } - public void testSingleShardFetch() { + public void testSingleShardFetchUsingBatchAction() { String indexName = "test"; int numOfShards = 1; prepareIndex(indexName, numOfShards); @@ -763,7 +763,7 @@ public void testSingleShardFetch() { assertNodeGatewayStartedShardsHappyCase(nodeGatewayStartedShards); } - public void testShardFetchMultiNodeMultiIndexes() { + public void testShardFetchMultiNodeMultiIndexesUsingBatchAction() { // start second node internalCluster().startNode(); String indexName1 = "test1"; @@ -792,7 +792,7 @@ public void testShardFetchMultiNodeMultiIndexes() { } } - public void testShardFetchCorruptedShards() throws Exception { + public void testShardFetchCorruptedShardsUsingBatchAction() throws Exception { String indexName = "test"; int numOfShards = 1; prepareIndex(indexName, numOfShards); From f8d8a6aabb02cbef0647db9f138742b0598f3da4 Mon Sep 17 00:00:00 2001 From: Shivansh Arora Date: Thu, 1 Feb 2024 16:39:26 +0530 Subject: [PATCH 20/21] Fixed comment Signed-off-by: Shivansh Arora --- .../java/org/opensearch/gateway/RecoveryFromGatewayIT.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java b/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java index 7d7a827209e27..9da1336642a64 100644 --- a/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java @@ -764,7 +764,7 @@ public void testSingleShardFetchUsingBatchAction() { } public void testShardFetchMultiNodeMultiIndexesUsingBatchAction() { - // start second node + // start node internalCluster().startNode(); String indexName1 = "test1"; String indexName2 = "test2"; From 8f113fc05a9e5cfd3b46bcd7bde1d5dec18101ae Mon Sep 17 00:00:00 2001 From: Shivansh Arora Date: Fri, 2 Feb 2024 01:20:28 +0530 Subject: [PATCH 21/21] empty commit to trigger workflow Signed-off-by: Shivansh Arora