From 07ba88af4f8ba00a64a538bdbb7c3aa8229382cf Mon Sep 17 00:00:00 2001 From: sudarshan baliga Date: Mon, 24 Jul 2023 16:39:08 +0530 Subject: [PATCH 01/22] Add batch async shard fetch transport action for replica Signed-off-by: sudarshan baliga --- .../gateway/AsyncBatchShardFetch.java | 34 ++ ...sportNodesListShardStoreMetadataBatch.java | 535 ++++++++++++++++++ 2 files changed, 569 insertions(+) create mode 100644 server/src/main/java/org/opensearch/gateway/AsyncBatchShardFetch.java create mode 100644 server/src/main/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadataBatch.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/indices/store/TransportNodesListShardStoreMetadataBatch.java b/server/src/main/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadataBatch.java new file mode 100644 index 0000000000000..e6d3e69a0a2a7 --- /dev/null +++ b/server/src/main/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadataBatch.java @@ -0,0 +1,535 @@ +/* + * 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.apache.logging.log4j.message.ParameterizedMessage; +import org.opensearch.OpenSearchException; +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.core.common.io.stream.Writeable; +import org.opensearch.common.settings.Settings; +import org.opensearch.common.unit.TimeValue; +import org.opensearch.env.NodeEnvironment; +import org.opensearch.gateway.AsyncBatchShardFetch; +import org.opensearch.index.IndexService; +import org.opensearch.index.IndexSettings; +import org.opensearch.index.seqno.ReplicationTracker; +import org.opensearch.index.seqno.RetentionLease; +import org.opensearch.index.shard.IndexShard; +import org.opensearch.core.index.shard.ShardId; +import org.opensearch.index.shard.ShardPath; +import org.opensearch.index.store.Store; +import org.opensearch.index.store.StoreFileMetadata; +import org.opensearch.indices.IndicesService; +import org.opensearch.threadpool.ThreadPool; +import org.opensearch.transport.TransportRequest; +import org.opensearch.transport.TransportService; + +import java.io.IOException; +import java.util.Collections; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.concurrent.TimeUnit; + +/** + * Transport action for fetching the batch of shard stores Metadata from a list of transport nodes + * + * @opensearch.internal + */ +public class TransportNodesListShardStoreMetadataBatch extends TransportNodesAction< + TransportNodesListShardStoreMetadataBatch.Request, + TransportNodesListShardStoreMetadataBatch.NodesStoreFilesMetadataBatch, + TransportNodesListShardStoreMetadataBatch.NodeRequest, + TransportNodesListShardStoreMetadataBatch.NodeStoreFilesMetadataBatch> + implements + AsyncBatchShardFetch.Lister< + TransportNodesListShardStoreMetadataBatch.NodesStoreFilesMetadataBatch, + TransportNodesListShardStoreMetadataBatch.NodeStoreFilesMetadataBatch> { + + public static final String ACTION_NAME = "internal:cluster/nodes/indices/shard/store/batch"; + public static final ActionType TYPE = new ActionType<>( + ACTION_NAME, + TransportNodesListShardStoreMetadataBatch.NodesStoreFilesMetadataBatch::new + ); + + private final Settings settings; + private final IndicesService indicesService; + private final NodeEnvironment nodeEnv; + + @Inject + public TransportNodesListShardStoreMetadataBatch( + Settings settings, + ThreadPool threadPool, + ClusterService clusterService, + TransportService transportService, + IndicesService indicesService, + NodeEnvironment nodeEnv, + ActionFilters actionFilters + ) { + super( + ACTION_NAME, + threadPool, + clusterService, + transportService, + actionFilters, + Request::new, + NodeRequest::new, + ThreadPool.Names.FETCH_SHARD_STORE, + NodeStoreFilesMetadataBatch.class + ); + this.settings = settings; + this.indicesService = indicesService; + this.nodeEnv = nodeEnv; + } + + @Override + public void list( + DiscoveryNode[] nodes, + Map shardIdsWithCustomDataPath, + ActionListener listener + ) { + execute(new TransportNodesListShardStoreMetadataBatch.Request(shardIdsWithCustomDataPath, nodes), listener); + } + + @Override + protected NodeRequest newNodeRequest(Request request) { + return new NodeRequest(request); + } + + @Override + protected NodeStoreFilesMetadataBatch newNodeResponse(StreamInput in) throws IOException { + return new NodeStoreFilesMetadataBatch(in); + } + + @Override + protected NodesStoreFilesMetadataBatch newResponse( + Request request, + List responses, + List failures + ) { + return new NodesStoreFilesMetadataBatch(clusterService.getClusterName(), responses, failures); + } + + @Override + protected NodeStoreFilesMetadataBatch nodeOperation(NodeRequest request) { + try { + return new NodeStoreFilesMetadataBatch(clusterService.localNode(), listStoreMetadata(request)); + } catch (IOException e) { + throw new OpenSearchException("Failed to list store metadata for shards [" + request.getShardIdsWithCustomDataPath() + "]", e); + } + } + + /** + * This method is similar to listStoreMetadata method of {@link TransportNodesListShardStoreMetadata} + * In this case we fetch the shard store files for batch of shards instead of one shard. + */ + private Map listStoreMetadata(NodeRequest request) throws IOException { + Map shardStoreMetadataMap = new HashMap(); + for (Map.Entry shardToCustomDataPathEntry : request.getShardIdsWithCustomDataPath().entrySet()) { + final ShardId shardId = shardToCustomDataPathEntry.getKey(); + logger.trace("listing store meta data for {}", shardId); + long startTimeNS = System.nanoTime(); + boolean exists = false; + try { + IndexService indexService = indicesService.indexService(shardId.getIndex()); + if (indexService != null) { + IndexShard indexShard = indexService.getShardOrNull(shardId.id()); + if (indexShard != null) { + try { + final StoreFilesMetadata storeFilesMetadata = new StoreFilesMetadata( + shardId, + indexShard.snapshotStoreMetadata(), + indexShard.getPeerRecoveryRetentionLeases() + ); + exists = true; + shardStoreMetadataMap.put(shardId, new NodeStoreFilesMetadata(storeFilesMetadata, null)); + continue; + } catch (org.apache.lucene.index.IndexNotFoundException e) { + logger.trace(new ParameterizedMessage("[{}] node is missing index, responding with empty", shardId), e); + shardStoreMetadataMap.put( + shardId, + new NodeStoreFilesMetadata( + new StoreFilesMetadata(shardId, Store.MetadataSnapshot.EMPTY, Collections.emptyList()), + e + ) + ); + continue; + } catch (IOException e) { + logger.warn(new ParameterizedMessage("[{}] can't read metadata from store, responding with empty", shardId), e); + shardStoreMetadataMap.put( + shardId, + new NodeStoreFilesMetadata( + new StoreFilesMetadata(shardId, Store.MetadataSnapshot.EMPTY, Collections.emptyList()), + e + ) + ); + continue; + } + } + } + final String customDataPath; + if (shardToCustomDataPathEntry.getValue() != null) { + customDataPath = shardToCustomDataPathEntry.getValue(); + } else { + // TODO: Fallback for BWC with older predecessor (ES) versions. + // Remove this once request.getCustomDataPath() always returns non-null + if (indexService != null) { + customDataPath = indexService.getIndexSettings().customDataPath(); + } else { + 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); + shardStoreMetadataMap.put( + shardId, + new NodeStoreFilesMetadata( + new StoreFilesMetadata(shardId, Store.MetadataSnapshot.EMPTY, Collections.emptyList()), + new OpenSearchException("node doesn't have meta data for index " + shardId.getIndex()) + ) + ); + continue; + } + } + } + final ShardPath shardPath = ShardPath.loadShardPath(logger, nodeEnv, shardId, customDataPath); + if (shardPath == null) { + shardStoreMetadataMap.put( + shardId, + new NodeStoreFilesMetadata( + new StoreFilesMetadata(shardId, Store.MetadataSnapshot.EMPTY, Collections.emptyList()), + null + ) + ); + continue; + } + // note that this may fail if it can't get access to the shard lock. Since we check above there is an active shard, this + // means: + // 1) a shard is being constructed, which means the cluster-manager will not use a copy of this replica + // 2) A shard is shutting down and has not cleared it's content within lock timeout. In this case the cluster-manager may + // not + // reuse local resources. + final Store.MetadataSnapshot metadataSnapshot = Store.readMetadataSnapshot( + shardPath.resolveIndex(), + shardId, + nodeEnv::shardLock, + logger + ); + // We use peer recovery retention leases from the primary for allocating replicas. We should always have retention leases + // when + // we refresh shard info after the primary has started. Hence, we can ignore retention leases if there is no active shard. + shardStoreMetadataMap.put( + shardId, + new NodeStoreFilesMetadata(new StoreFilesMetadata(shardId, metadataSnapshot, Collections.emptyList()), null) + ); + } catch (Exception e) { + logger.trace("{} failed to load store metadata {}", shardId, e); + shardStoreMetadataMap.put( + shardId, + new NodeStoreFilesMetadata( + new StoreFilesMetadata(shardId, Store.MetadataSnapshot.EMPTY, Collections.emptyList()), + new OpenSearchException("failed to load store metadata", e) + ) + ); + } finally { + TimeValue took = new TimeValue(System.nanoTime() - startTimeNS, TimeUnit.NANOSECONDS); + if (exists) { + logger.debug("{} loaded store meta data (took [{}])", shardId, took); + } else { + logger.trace("{} didn't find any store meta data to load (took [{}])", shardId, took); + } + } + } + return shardStoreMetadataMap; + } + + /** + * Metadata for store files + * + * @opensearch.internal + */ + public static 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() + + "}" + + '}'; + } + } + + /** + * Request 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(Map shardIdsWithCustomDataPath, DiscoveryNode[] nodes) { + super(nodes); + this.shardIdsWithCustomDataPath = Objects.requireNonNull(shardIdsWithCustomDataPath); + } + + public Map getShardIdsWithCustomDataPath() { + return shardIdsWithCustomDataPath; + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + out.writeMap(shardIdsWithCustomDataPath, (o, k) -> k.writeTo(o), StreamOutput::writeString); + } + } + + /** + * Metadata for the nodes store files + * + * @opensearch.internal + */ + public static class NodesStoreFilesMetadataBatch extends BaseNodesResponse { + + public NodesStoreFilesMetadataBatch(StreamInput in) throws IOException { + super(in); + } + + public NodesStoreFilesMetadataBatch( + ClusterName clusterName, + List nodes, + List failures + ) { + super(clusterName, nodes, failures); + } + + @Override + protected List readNodesFrom(StreamInput in) throws IOException { + return in.readList(NodeStoreFilesMetadataBatch::new); + } + + @Override + protected void writeNodesTo(StreamOutput out, List nodes) throws IOException { + out.writeList(nodes); + } + } + + /** + * The metadata for the node store files + * + * @opensearch.internal + */ + public static class NodeStoreFilesMetadata { + + private StoreFilesMetadata storeFilesMetadata; + private Exception storeFileFetchException; + + public NodeStoreFilesMetadata(StoreFilesMetadata storeFilesMetadata) { + this.storeFilesMetadata = storeFilesMetadata; + this.storeFileFetchException = null; + } + + public NodeStoreFilesMetadata(StreamInput in) throws IOException { + storeFilesMetadata = new StoreFilesMetadata(in); + if (in.readBoolean()) { + this.storeFileFetchException = in.readException(); + } else { + this.storeFileFetchException = null; + } + } + + public NodeStoreFilesMetadata(StoreFilesMetadata storeFilesMetadata, Exception storeFileFetchException) { + this.storeFilesMetadata = storeFilesMetadata; + this.storeFileFetchException = storeFileFetchException; + } + + public StoreFilesMetadata storeFilesMetadata() { + return storeFilesMetadata; + } + + public static NodeStoreFilesMetadata readListShardStoreNodeOperationResponse(StreamInput in) throws IOException { + return new NodeStoreFilesMetadata(in); + } + + public void writeTo(StreamOutput out) throws IOException { + storeFilesMetadata.writeTo(out); + if (storeFileFetchException != null) { + out.writeBoolean(true); + out.writeException(storeFileFetchException); + } else { + out.writeBoolean(false); + } + } + + public Exception getStoreFileFetchException() { + return storeFileFetchException; + } + + @Override + public String toString() { + return "[[" + storeFilesMetadata + "]]"; + } + } + + /** + * 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.getShardIdsWithCustomDataPath()); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + out.writeMap(shardIdsWithCustomDataPath, (o, k) -> k.writeTo(o), StreamOutput::writeString); + } + + public Map getShardIdsWithCustomDataPath() { + return shardIdsWithCustomDataPath; + } + } + + /** + * NodeStoreFilesMetadataBatch Response received by the node from other node for this transport action. + * Refer {@link TransportNodesAction} + */ + public static class NodeStoreFilesMetadataBatch extends BaseNodeResponse { + private final Map nodeStoreFilesMetadataBatch; + + protected NodeStoreFilesMetadataBatch(StreamInput in) throws IOException { + super(in); + this.nodeStoreFilesMetadataBatch = in.readMap(ShardId::new, NodeStoreFilesMetadata::new); + } + + public NodeStoreFilesMetadataBatch(DiscoveryNode node, Map nodeStoreFilesMetadataBatch) { + super(node); + this.nodeStoreFilesMetadataBatch = nodeStoreFilesMetadataBatch; + } + + public Map getNodeStoreFilesMetadataBatch() { + return this.nodeStoreFilesMetadataBatch; + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + out.writeMap(nodeStoreFilesMetadataBatch, (o, k) -> k.writeTo(o), (o, v) -> v.writeTo(o)); + } + } + +} From 45aadc858bb7e90c94c6128a1e8448a90e3ffcbd Mon Sep 17 00:00:00 2001 From: sudarshan baliga Date: Thu, 3 Aug 2023 15:33:50 +0530 Subject: [PATCH 02/22] Add RSA Async batch shard fetch transport integ test Signed-off-by: sudarshan baliga --- .../AsyncShardFetchBatchTestUtils.java | 76 ++++++++ ...ortNodesListShardStoreMetadataBatchIT.java | 163 ++++++++++++++++++ .../org/opensearch/indices/IndicesModule.java | 6 +- 3 files changed, 243 insertions(+), 2 deletions(-) create mode 100644 server/src/internalClusterTest/java/org/opensearch/gateway/AsyncShardFetchBatchTestUtils.java create mode 100644 server/src/internalClusterTest/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadataBatchIT.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/indices/store/TransportNodesListShardStoreMetadataBatchIT.java b/server/src/internalClusterTest/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadataBatchIT.java new file mode 100644 index 0000000000000..e8fd38ca90499 --- /dev/null +++ b/server/src/internalClusterTest/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadataBatchIT.java @@ -0,0 +1,163 @@ +/* + * 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.Version; +import org.opensearch.action.admin.cluster.reroute.ClusterRerouteResponse; +import org.opensearch.action.admin.cluster.shards.ClusterSearchShardsGroup; +import org.opensearch.action.admin.cluster.shards.ClusterSearchShardsResponse; +import org.opensearch.action.support.ActionTestUtils; +import org.opensearch.cluster.node.DiscoveryNode; +import org.opensearch.cluster.routing.ShardRouting; +import org.opensearch.common.settings.Settings; +import org.opensearch.core.index.Index; +import org.opensearch.core.index.shard.ShardId; +import org.opensearch.test.OpenSearchIntegTestCase; + +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 TransportNodesListShardStoreMetadataBatchIT extends OpenSearchIntegTestCase { + + public void testSingleShardStoreFetch() throws ExecutionException, InterruptedException { + String indexName = "test"; + DiscoveryNode[] nodes = getDiscoveryNodes(); + TransportNodesListShardStoreMetadataBatch.NodesStoreFilesMetadataBatch response = prepareAndSendRequest( + new String[] { indexName }, + nodes + ); + Index index = resolveIndex(indexName); + ShardId shardId = new ShardId(index, 0); + TransportNodesListShardStoreMetadataBatch.NodeStoreFilesMetadata nodeStoreFilesMetadata = response.getNodesMap() + .get(nodes[0].getId()) + .getNodeStoreFilesMetadataBatch() + .get(shardId); + assertNodeStoreFilesMetadataSuccessCase(nodeStoreFilesMetadata, shardId); + } + + public void testShardStoreFetchMultiNodeMultiIndexes() throws Exception { + // start second node + internalCluster().startNode(); + String indexName1 = "test1"; + String indexName2 = "test2"; + DiscoveryNode[] nodes = getDiscoveryNodes(); + TransportNodesListShardStoreMetadataBatch.NodesStoreFilesMetadataBatch response = prepareAndSendRequest( + new String[] { indexName1, indexName2 }, + nodes + ); + ClusterSearchShardsResponse searchShardsResponse = client().admin().cluster().prepareSearchShards(indexName1, indexName2).get(); + for (ClusterSearchShardsGroup clusterSearchShardsGroup : searchShardsResponse.getGroups()) { + ShardId shardId = clusterSearchShardsGroup.getShardId(); + ShardRouting[] shardRoutings = clusterSearchShardsGroup.getShards(); + assertEquals(2, shardRoutings.length); + for (ShardRouting shardRouting : shardRoutings) { + TransportNodesListShardStoreMetadataBatch.NodeStoreFilesMetadata nodeStoreFilesMetadata = response.getNodesMap() + .get(shardRouting.currentNodeId()) + .getNodeStoreFilesMetadataBatch() + .get(shardId); + assertNodeStoreFilesMetadataSuccessCase(nodeStoreFilesMetadata, shardId); + } + } + } + + public void testShardStoreFetchNodeNotConnected() { + DiscoveryNode nonExistingNode = new DiscoveryNode("foo", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT); + String indexName = "test"; + TransportNodesListShardStoreMetadataBatch.NodesStoreFilesMetadataBatch response = prepareAndSendRequest( + new String[] { indexName }, + new DiscoveryNode[] { nonExistingNode } + ); + assertTrue(response.hasFailures()); + assertEquals(1, response.failures().size()); + assertEquals(nonExistingNode.getId(), response.failures().get(0).nodeId()); + } + + public void testShardStoreFetchCorruptedIndex() throws Exception { + // start second node + internalCluster().startNode(); + String indexName = "test"; + prepareIndices(new String[] { indexName }, 1, 1); + Map shardIdCustomDataPathMap = prepareRequestMap(new String[] { indexName }, 1); + Index index = resolveIndex(indexName); + ShardId shardId = new ShardId(index, 0); + ClusterSearchShardsResponse searchShardsResponse = client().admin().cluster().prepareSearchShards(indexName).get(); + assertEquals(2, searchShardsResponse.getNodes().length); + corruptShard(searchShardsResponse.getNodes()[0].getName(), shardId); + corruptShard(searchShardsResponse.getNodes()[1].getName(), shardId); + ClusterRerouteResponse clusterRerouteResponse = client().admin().cluster().prepareReroute().setRetryFailed(false).get(); + DiscoveryNode[] discoveryNodes = getDiscoveryNodes(); + TransportNodesListShardStoreMetadataBatch.NodesStoreFilesMetadataBatch response; + response = ActionTestUtils.executeBlocking( + internalCluster().getInstance(TransportNodesListShardStoreMetadataBatch.class), + new TransportNodesListShardStoreMetadataBatch.Request(shardIdCustomDataPathMap, discoveryNodes) + ); + TransportNodesListShardStoreMetadataBatch.NodeStoreFilesMetadata nodeStoreFilesMetadata = response.getNodesMap() + .get(discoveryNodes[0].getId()) + .getNodeStoreFilesMetadataBatch() + .get(shardId); + assertNodeStoreFilesMetadataFailureCase(nodeStoreFilesMetadata, shardId); + } + + 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() + ); + index(index, "type", "1"); + flush(index); + } + } + + private TransportNodesListShardStoreMetadataBatch.NodesStoreFilesMetadataBatch prepareAndSendRequest( + String[] indices, + DiscoveryNode[] nodes + ) { + Map shardIdCustomDataPathMap = null; + prepareIndices(indices, 1, 1); + shardIdCustomDataPathMap = prepareRequestMap(indices, 1); + TransportNodesListShardStoreMetadataBatch.NodesStoreFilesMetadataBatch response; + return ActionTestUtils.executeBlocking( + internalCluster().getInstance(TransportNodesListShardStoreMetadataBatch.class), + new TransportNodesListShardStoreMetadataBatch.Request(shardIdCustomDataPathMap, nodes) + ); + } + + private void assertNodeStoreFilesMetadataFailureCase( + TransportNodesListShardStoreMetadataBatch.NodeStoreFilesMetadata nodeStoreFilesMetadata, + ShardId shardId + ) { + assertNotNull(nodeStoreFilesMetadata.getStoreFileFetchException()); + TransportNodesListShardStoreMetadataBatch.StoreFilesMetadata storeFileMetadata = nodeStoreFilesMetadata.storeFilesMetadata(); + assertEquals(shardId, storeFileMetadata.shardId()); + assertTrue(storeFileMetadata.peerRecoveryRetentionLeases().isEmpty()); + } + + private void assertNodeStoreFilesMetadataSuccessCase( + TransportNodesListShardStoreMetadataBatch.NodeStoreFilesMetadata nodeStoreFilesMetadata, + ShardId shardId + ) { + assertNull(nodeStoreFilesMetadata.getStoreFileFetchException()); + TransportNodesListShardStoreMetadataBatch.StoreFilesMetadata storeFileMetadata = nodeStoreFilesMetadata.storeFilesMetadata(); + assertFalse(storeFileMetadata.isEmpty()); + assertEquals(shardId, storeFileMetadata.shardId()); + assertNotNull(storeFileMetadata.peerRecoveryRetentionLeases()); + } +} diff --git a/server/src/main/java/org/opensearch/indices/IndicesModule.java b/server/src/main/java/org/opensearch/indices/IndicesModule.java index 9d2eef5f67a86..9777a1a5416b1 100644 --- a/server/src/main/java/org/opensearch/indices/IndicesModule.java +++ b/server/src/main/java/org/opensearch/indices/IndicesModule.java @@ -37,9 +37,9 @@ import org.opensearch.action.admin.indices.rollover.MaxDocsCondition; import org.opensearch.action.admin.indices.rollover.MaxSizeCondition; import org.opensearch.action.resync.TransportResyncReplicationAction; +import org.opensearch.common.inject.AbstractModule; import org.opensearch.common.util.FeatureFlags; import org.opensearch.core.ParseField; -import org.opensearch.common.inject.AbstractModule; import org.opensearch.core.common.io.stream.NamedWriteableRegistry; import org.opensearch.core.common.io.stream.NamedWriteableRegistry.Entry; import org.opensearch.core.xcontent.NamedXContentRegistry; @@ -71,16 +71,17 @@ import org.opensearch.index.mapper.TextFieldMapper; import org.opensearch.index.mapper.VersionFieldMapper; import org.opensearch.index.remote.RemoteRefreshSegmentPressureService; +import org.opensearch.index.seqno.GlobalCheckpointSyncAction; import org.opensearch.index.seqno.RetentionLeaseBackgroundSyncAction; import org.opensearch.index.seqno.RetentionLeaseSyncAction; import org.opensearch.index.seqno.RetentionLeaseSyncer; -import org.opensearch.index.seqno.GlobalCheckpointSyncAction; import org.opensearch.index.shard.PrimaryReplicaSyncer; import org.opensearch.indices.cluster.IndicesClusterStateService; import org.opensearch.indices.mapper.MapperRegistry; import org.opensearch.indices.replication.checkpoint.SegmentReplicationCheckpointPublisher; import org.opensearch.indices.store.IndicesStore; import org.opensearch.indices.store.TransportNodesListShardStoreMetadata; +import org.opensearch.indices.store.TransportNodesListShardStoreMetadataBatch; import org.opensearch.plugins.MapperPlugin; import java.util.ArrayList; @@ -280,6 +281,7 @@ protected void configure() { bind(IndicesStore.class).asEagerSingleton(); bind(IndicesClusterStateService.class).asEagerSingleton(); bind(TransportNodesListShardStoreMetadata.class).asEagerSingleton(); + bind(TransportNodesListShardStoreMetadataBatch.class).asEagerSingleton(); bind(GlobalCheckpointSyncAction.class).asEagerSingleton(); bind(TransportResyncReplicationAction.class).asEagerSingleton(); bind(PrimaryReplicaSyncer.class).asEagerSingleton(); From e2b573c484176e853593e9cbe518dfa16d62edd3 Mon Sep 17 00:00:00 2001 From: sudarshan baliga Date: Sun, 6 Aug 2023 15:03:14 +0530 Subject: [PATCH 03/22] Update the documentation of TransportNodesListShardStoreMetadataBatch. Update variable name of AsyncBatchShardFetch Signed-off-by: sudarshan baliga Signed-off-by: sudarshan baliga --- .../java/org/opensearch/gateway/AsyncBatchShardFetch.java | 2 +- .../store/TransportNodesListShardStoreMetadataBatch.java | 7 +++++-- 2 files changed, 6 insertions(+), 3 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/indices/store/TransportNodesListShardStoreMetadataBatch.java b/server/src/main/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadataBatch.java index e6d3e69a0a2a7..7bef79f9233ea 100644 --- a/server/src/main/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadataBatch.java +++ b/server/src/main/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadataBatch.java @@ -142,8 +142,11 @@ protected NodeStoreFilesMetadataBatch nodeOperation(NodeRequest request) { } /** - * This method is similar to listStoreMetadata method of {@link TransportNodesListShardStoreMetadata} - * In this case we fetch the shard store files for batch of shards instead of one shard. + * This method is similar to listStoreMetadata method of {@link TransportNodesListShardStoreMetadata} we loop over + * the shards here and populate the data about the shard store information held by the local node. + * + * @param request Request containing the map shardIdsWithCustomDataPath. + * @return NodeStoreFilesMetadata contains the data about the shard store held by the local node */ private Map listStoreMetadata(NodeRequest request) throws IOException { Map shardStoreMetadataMap = new HashMap(); From 344a4f1de2783afe3af0fd81f6621b3dfd85a23b Mon Sep 17 00:00:00 2001 From: Shivansh Arora Date: Tue, 26 Sep 2023 04:21:40 +0530 Subject: [PATCH 04/22] Transport RSA refactor Signed-off-by: Shivansh Arora --- .../indices/store/StoreFilesMetadata.java | 115 +++++++++++++++ .../TransportNodesListShardStoreMetadata.java | 92 ------------ ...sportNodesListShardStoreMetadataBatch.java | 131 +++--------------- 3 files changed, 135 insertions(+), 203 deletions(-) create mode 100644 server/src/main/java/org/opensearch/indices/store/StoreFilesMetadata.java 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() + + "}" + + '}'; + } +} diff --git a/server/src/main/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadata.java b/server/src/main/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadata.java index 22c5c923e6322..a8e071b146667 100644 --- a/server/src/main/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadata.java +++ b/server/src/main/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadata.java @@ -229,98 +229,6 @@ private StoreFilesMetadata listStoreMetadata(NodeRequest request) throws IOExcep } } - /** - * Metadata for store files - * - * @opensearch.internal - */ - public static 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() - + "}" - + '}'; - } - } - /** * The request * diff --git a/server/src/main/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadataBatch.java b/server/src/main/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadataBatch.java index 7bef79f9233ea..e6e7cdb08a560 100644 --- a/server/src/main/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadataBatch.java +++ b/server/src/main/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadataBatch.java @@ -52,6 +52,7 @@ import java.util.Map; import java.util.Objects; import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; /** * Transport action for fetching the batch of shard stores Metadata from a list of transport nodes @@ -110,7 +111,7 @@ public void list( Map shardIdsWithCustomDataPath, ActionListener listener ) { - execute(new TransportNodesListShardStoreMetadataBatch.Request(shardIdsWithCustomDataPath, nodes), listener); + execute(new TransportNodesListShardStoreMetadataBatch.Request(shardIdsWithCustomDataPath.entrySet().stream().map(entry -> new ShardAttributes(entry.getKey(), entry.getValue())).collect(Collectors.toList()), nodes), listener); } @Override @@ -137,7 +138,7 @@ protected NodeStoreFilesMetadataBatch nodeOperation(NodeRequest request) { try { return new NodeStoreFilesMetadataBatch(clusterService.localNode(), listStoreMetadata(request)); } catch (IOException e) { - throw new OpenSearchException("Failed to list store metadata for shards [" + request.getShardIdsWithCustomDataPath() + "]", e); + throw new OpenSearchException("Failed to list store metadata for shards [" + request.getShardAttributes().stream().map(ShardAttributes::getShardId) + "]", e); } } @@ -150,8 +151,8 @@ protected NodeStoreFilesMetadataBatch nodeOperation(NodeRequest request) { */ private Map listStoreMetadata(NodeRequest request) throws IOException { Map shardStoreMetadataMap = new HashMap(); - for (Map.Entry shardToCustomDataPathEntry : request.getShardIdsWithCustomDataPath().entrySet()) { - final ShardId shardId = shardToCustomDataPathEntry.getKey(); + for (ShardAttributes shardAttributes : request.getShardAttributes()) { + final ShardId shardId = shardAttributes.getShardId(); logger.trace("listing store meta data for {}", shardId); long startTimeNS = System.nanoTime(); boolean exists = false; @@ -193,8 +194,8 @@ private Map listStoreMetadata(NodeRequest reque } } final String customDataPath; - if (shardToCustomDataPathEntry.getValue() != null) { - customDataPath = shardToCustomDataPathEntry.getValue(); + if (shardAttributes.getCustomDataPath() != null) { + customDataPath = shardAttributes.getCustomDataPath(); } else { // TODO: Fallback for BWC with older predecessor (ES) versions. // Remove this once request.getCustomDataPath() always returns non-null @@ -268,98 +269,6 @@ private Map listStoreMetadata(NodeRequest reque return shardStoreMetadataMap; } - /** - * Metadata for store files - * - * @opensearch.internal - */ - public static 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() - + "}" - + '}'; - } - } - /** * Request is used in constructing the request for making the transport request to set of other node. * Refer {@link TransportNodesAction} class start method. @@ -368,26 +277,26 @@ public String toString() { */ 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(Map shardIdsWithCustomDataPath, DiscoveryNode[] nodes) { + public Request(List shardAttributes, DiscoveryNode[] nodes) { super(nodes); - this.shardIdsWithCustomDataPath = Objects.requireNonNull(shardIdsWithCustomDataPath); + this.shardAttributes = Objects.requireNonNull(shardAttributes); } - public Map getShardIdsWithCustomDataPath() { - return shardIdsWithCustomDataPath; + public List getShardAttributes() { + return shardAttributes; } @Override public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); - out.writeMap(shardIdsWithCustomDataPath, (o, k) -> k.writeTo(o), StreamOutput::writeString); + out.writeList(shardAttributes); } } @@ -485,25 +394,25 @@ public String toString() { */ public static class NodeRequest extends TransportRequest { - private final Map 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.getShardIdsWithCustomDataPath()); + 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); } - public Map getShardIdsWithCustomDataPath() { - return shardIdsWithCustomDataPath; + public List getShardAttributes() { + return shardAttributes; } } From 24ec9a2746e1e450723cd6ef24138cff115d87a9 Mon Sep 17 00:00:00 2001 From: Shivansh Arora Date: Wed, 27 Sep 2023 18:08:55 +0530 Subject: [PATCH 05/22] Correcting the refernce of StoreFilesMetadata Signed-off-by: Shivansh Arora --- .../gateway/ReplicaShardAllocator.java | 24 +++++++++---------- .../gateway/ReplicaShardAllocatorTests.java | 7 +++--- .../opensearch/index/store/StoreTests.java | 8 +++---- 3 files changed, 20 insertions(+), 19 deletions(-) diff --git a/server/src/main/java/org/opensearch/gateway/ReplicaShardAllocator.java b/server/src/main/java/org/opensearch/gateway/ReplicaShardAllocator.java index 4c90ac1710718..67ff1a96d171c 100644 --- a/server/src/main/java/org/opensearch/gateway/ReplicaShardAllocator.java +++ b/server/src/main/java/org/opensearch/gateway/ReplicaShardAllocator.java @@ -106,7 +106,7 @@ public void processExistingRecoveries(RoutingAllocation allocation) { assert primaryShard != null : "the replica shard can be allocated on at least one node, so there must be an active primary"; assert primaryShard.currentNodeId() != null; final DiscoveryNode primaryNode = allocation.nodes().get(primaryShard.currentNodeId()); - final TransportNodesListShardStoreMetadata.StoreFilesMetadata primaryStore = findStore(primaryNode, shardStores); + final StoreFilesMetadata primaryStore = findStore(primaryNode, shardStores); if (primaryStore == null) { // if we can't find the primary data, it is probably because the primary shard is corrupted (and listing failed) // just let the recovery find it out, no need to do anything about it for the initializing shard @@ -223,7 +223,7 @@ public AllocateUnassignedDecision makeAllocationDecision( } assert primaryShard.currentNodeId() != null; final DiscoveryNode primaryNode = allocation.nodes().get(primaryShard.currentNodeId()); - final TransportNodesListShardStoreMetadata.StoreFilesMetadata primaryStore = findStore(primaryNode, shardStores); + final StoreFilesMetadata primaryStore = findStore(primaryNode, shardStores); if (primaryStore == null) { // if we can't find the primary data, it is probably because the primary shard is corrupted (and listing failed) // we want to let the replica be allocated in order to expose the actual problem with the primary that the replica @@ -357,7 +357,7 @@ private static List augmentExplanationsWithStoreInfo( /** * Finds the store for the assigned shard in the fetched data, returns null if none is found. */ - private static TransportNodesListShardStoreMetadata.StoreFilesMetadata findStore( + private static StoreFilesMetadata findStore( DiscoveryNode node, AsyncShardFetch.FetchResult data ) { @@ -373,7 +373,7 @@ private MatchingNodes findMatchingNodes( RoutingAllocation allocation, boolean noMatchFailedNodes, DiscoveryNode primaryNode, - TransportNodesListShardStoreMetadata.StoreFilesMetadata primaryStore, + StoreFilesMetadata primaryStore, AsyncShardFetch.FetchResult data, boolean explain ) { @@ -386,7 +386,7 @@ private MatchingNodes findMatchingNodes( && shard.unassignedInfo().getFailedNodeIds().contains(discoNode.getId())) { continue; } - TransportNodesListShardStoreMetadata.StoreFilesMetadata storeFilesMetadata = nodeStoreEntry.getValue().storeFilesMetadata(); + StoreFilesMetadata storeFilesMetadata = nodeStoreEntry.getValue().storeFilesMetadata(); // we don't have any files at all, it is an empty index if (storeFilesMetadata.isEmpty()) { continue; @@ -442,8 +442,8 @@ private MatchingNodes findMatchingNodes( } private static long computeMatchingBytes( - TransportNodesListShardStoreMetadata.StoreFilesMetadata primaryStore, - TransportNodesListShardStoreMetadata.StoreFilesMetadata storeFilesMetadata + StoreFilesMetadata primaryStore, + StoreFilesMetadata storeFilesMetadata ) { long sizeMatched = 0; for (StoreFileMetadata storeFileMetadata : storeFilesMetadata) { @@ -456,8 +456,8 @@ private static long computeMatchingBytes( } private static boolean hasMatchingSyncId( - TransportNodesListShardStoreMetadata.StoreFilesMetadata primaryStore, - TransportNodesListShardStoreMetadata.StoreFilesMetadata replicaStore + StoreFilesMetadata primaryStore, + StoreFilesMetadata replicaStore ) { String primarySyncId = primaryStore.syncId(); return primarySyncId != null && primarySyncId.equals(replicaStore.syncId()); @@ -465,9 +465,9 @@ private static boolean hasMatchingSyncId( private static MatchingNode computeMatchingNode( DiscoveryNode primaryNode, - TransportNodesListShardStoreMetadata.StoreFilesMetadata primaryStore, + StoreFilesMetadata primaryStore, DiscoveryNode replicaNode, - TransportNodesListShardStoreMetadata.StoreFilesMetadata replicaStore + StoreFilesMetadata replicaStore ) { final long retainingSeqNoForPrimary = primaryStore.getPeerRecoveryRetentionLeaseRetainingSeqNo(primaryNode); final long retainingSeqNoForReplica = primaryStore.getPeerRecoveryRetentionLeaseRetainingSeqNo(replicaNode); @@ -478,7 +478,7 @@ private static MatchingNode computeMatchingNode( } private static boolean canPerformOperationBasedRecovery( - TransportNodesListShardStoreMetadata.StoreFilesMetadata primaryStore, + StoreFilesMetadata primaryStore, AsyncShardFetch.FetchResult shardStores, DiscoveryNode targetNode ) { diff --git a/server/src/test/java/org/opensearch/gateway/ReplicaShardAllocatorTests.java b/server/src/test/java/org/opensearch/gateway/ReplicaShardAllocatorTests.java index 968a0474051f7..91a26408098d2 100644 --- a/server/src/test/java/org/opensearch/gateway/ReplicaShardAllocatorTests.java +++ b/server/src/test/java/org/opensearch/gateway/ReplicaShardAllocatorTests.java @@ -65,6 +65,7 @@ import org.opensearch.core.index.shard.ShardId; import org.opensearch.index.store.Store; import org.opensearch.index.store.StoreFileMetadata; +import org.opensearch.indices.store.StoreFilesMetadata; import org.opensearch.indices.store.TransportNodesListShardStoreMetadata; import org.opensearch.cluster.OpenSearchAllocationTestCase; import org.opensearch.snapshots.SnapshotShardSizeInfo; @@ -664,7 +665,7 @@ static String randomSyncId() { class TestAllocator extends ReplicaShardAllocator { - private Map data = null; + private Map data = null; private AtomicBoolean fetchDataCalled = new AtomicBoolean(false); public void clean() { @@ -702,7 +703,7 @@ TestAllocator addData( } data.put( node, - new TransportNodesListShardStoreMetadata.StoreFilesMetadata( + new StoreFilesMetadata( shardId, new Store.MetadataSnapshot(unmodifiableMap(filesAsMap), unmodifiableMap(commitData), randomInt()), peerRecoveryRetentionLeases @@ -720,7 +721,7 @@ protected AsyncShardFetch.FetchResult tData = null; if (data != null) { tData = new HashMap<>(); - for (Map.Entry entry : data.entrySet()) { + for (Map.Entry entry : data.entrySet()) { tData.put( entry.getKey(), new TransportNodesListShardStoreMetadata.NodeStoreFilesMetadata(entry.getKey(), entry.getValue()) diff --git a/server/src/test/java/org/opensearch/index/store/StoreTests.java b/server/src/test/java/org/opensearch/index/store/StoreTests.java index 9043dcce1b779..7c5767bfbab98 100644 --- a/server/src/test/java/org/opensearch/index/store/StoreTests.java +++ b/server/src/test/java/org/opensearch/index/store/StoreTests.java @@ -86,6 +86,7 @@ import org.opensearch.index.seqno.SequenceNumbers; import org.opensearch.core.index.shard.ShardId; import org.opensearch.indices.replication.common.ReplicationType; +import org.opensearch.indices.store.StoreFilesMetadata; import org.opensearch.indices.store.TransportNodesListShardStoreMetadata; import org.opensearch.test.DummyShardLock; import org.opensearch.test.FeatureFlagSetter; @@ -961,8 +962,8 @@ public void testStreamStoreFilesMetadata() throws Exception { ) ); } - TransportNodesListShardStoreMetadata.StoreFilesMetadata outStoreFileMetadata = - new TransportNodesListShardStoreMetadata.StoreFilesMetadata( + StoreFilesMetadata outStoreFileMetadata = + new StoreFilesMetadata( new ShardId("test", "_na_", 0), metadataSnapshot, peerRecoveryRetentionLeases @@ -975,8 +976,7 @@ public void testStreamStoreFilesMetadata() throws Exception { ByteArrayInputStream inBuffer = new ByteArrayInputStream(outBuffer.toByteArray()); InputStreamStreamInput in = new InputStreamStreamInput(inBuffer); in.setVersion(targetNodeVersion); - TransportNodesListShardStoreMetadata.StoreFilesMetadata inStoreFileMetadata = - new TransportNodesListShardStoreMetadata.StoreFilesMetadata(in); + StoreFilesMetadata inStoreFileMetadata = new StoreFilesMetadata(in); Iterator outFiles = outStoreFileMetadata.iterator(); for (StoreFileMetadata inFile : inStoreFileMetadata) { assertThat(inFile.name(), equalTo(outFiles.next().name())); From 88cb94ed7205169f9977f9f741338d1943320cb8 Mon Sep 17 00:00:00 2001 From: Shivansh Arora Date: Wed, 13 Dec 2023 10:52:34 +0530 Subject: [PATCH 06/22] Added helper class to remove code duplication Signed-off-by: Shivansh Arora --- .../TransportNodesListShardStoreMetadata.java | 109 +++-------- ...sportNodesListShardStoreMetadataBatch.java | 181 ++++-------------- ...portNodesListShardStoreMetadataHelper.java | 117 +++++++++++ 3 files changed, 177 insertions(+), 230 deletions(-) create mode 100644 server/src/main/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadataHelper.java diff --git a/server/src/main/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadata.java b/server/src/main/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadata.java index a8e071b146667..ab0dc2842f2d2 100644 --- a/server/src/main/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadata.java +++ b/server/src/main/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadata.java @@ -32,38 +32,27 @@ package org.opensearch.indices.store; -import org.apache.logging.log4j.message.ParameterizedMessage; import org.opensearch.OpenSearchException; -import org.opensearch.action.ActionListener; import org.opensearch.action.ActionType; import org.opensearch.action.FailedNodeException; +import org.opensearch.action.ActionListener; 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.Nullable; import org.opensearch.common.inject.Inject; +import org.opensearch.common.settings.Settings; 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.common.settings.Settings; -import org.opensearch.common.unit.TimeValue; +import org.opensearch.core.index.shard.ShardId; import org.opensearch.env.NodeEnvironment; import org.opensearch.gateway.AsyncShardFetch; -import org.opensearch.index.IndexService; -import org.opensearch.index.IndexSettings; -import org.opensearch.index.seqno.ReplicationTracker; -import org.opensearch.index.seqno.RetentionLease; -import org.opensearch.index.shard.IndexShard; -import org.opensearch.core.index.shard.ShardId; -import org.opensearch.index.shard.ShardPath; import org.opensearch.index.store.Store; -import org.opensearch.index.store.StoreFileMetadata; import org.opensearch.indices.IndicesService; import org.opensearch.threadpool.ThreadPool; import org.opensearch.transport.TransportRequest; @@ -71,10 +60,10 @@ import java.io.IOException; import java.util.Collections; -import java.util.Iterator; import java.util.List; import java.util.Objects; -import java.util.concurrent.TimeUnit; + +import static org.opensearch.indices.store.TransportNodesListShardStoreMetadataHelper.getListShardMetadataOnLocalNode; /** * Metadata for shard stores from a list of transport nodes @@ -87,9 +76,9 @@ public class TransportNodesListShardStoreMetadata extends TransportNodesAction< TransportNodesListShardStoreMetadata.NodeRequest, TransportNodesListShardStoreMetadata.NodeStoreFilesMetadata> implements - AsyncShardFetch.Lister< - TransportNodesListShardStoreMetadata.NodesStoreFilesMetadata, - TransportNodesListShardStoreMetadata.NodeStoreFilesMetadata> { + AsyncShardFetch.Lister< + TransportNodesListShardStoreMetadata.NodesStoreFilesMetadata, + TransportNodesListShardStoreMetadata.NodeStoreFilesMetadata> { public static final String ACTION_NAME = "internal:cluster/nodes/indices/shard/store"; public static final ActionType TYPE = new ActionType<>(ACTION_NAME, NodesStoreFilesMetadata::new); @@ -125,7 +114,14 @@ public TransportNodesListShardStoreMetadata( } @Override - public void list(ShardId shardId, String customDataPath, DiscoveryNode[] nodes, ActionListener listener) { + public void list( + Map shardAttributes, + DiscoveryNode[] nodes, + ActionListener listener + ) { + assert shardAttributes.size() == 1 : "only one shard should be specified"; + final ShardId shardId = shardAttributes.keySet().iterator().next(); + final String customDataPath = shardAttributes.get(shardId).getCustomDataPath(); execute(new Request(shardId, customDataPath, nodes), listener); } @@ -159,73 +155,18 @@ protected NodeStoreFilesMetadata nodeOperation(NodeRequest request) { private StoreFilesMetadata listStoreMetadata(NodeRequest request) throws IOException { final ShardId shardId = request.getShardId(); - logger.trace("listing store meta data for {}", shardId); - long startTimeNS = System.nanoTime(); - boolean exists = false; try { - IndexService indexService = indicesService.indexService(shardId.getIndex()); - if (indexService != null) { - IndexShard indexShard = indexService.getShardOrNull(shardId.id()); - if (indexShard != null) { - try { - final StoreFilesMetadata storeFilesMetadata = new StoreFilesMetadata( - shardId, - indexShard.snapshotStoreMetadata(), - indexShard.getPeerRecoveryRetentionLeases() - ); - exists = true; - return storeFilesMetadata; - } catch (org.apache.lucene.index.IndexNotFoundException e) { - logger.trace(new ParameterizedMessage("[{}] node is missing index, responding with empty", shardId), e); - return new StoreFilesMetadata(shardId, Store.MetadataSnapshot.EMPTY, Collections.emptyList()); - } catch (IOException e) { - logger.warn(new ParameterizedMessage("[{}] can't read metadata from store, responding with empty", shardId), e); - return new StoreFilesMetadata(shardId, Store.MetadataSnapshot.EMPTY, Collections.emptyList()); - } - } - } - final String customDataPath; - if (request.getCustomDataPath() != null) { - customDataPath = request.getCustomDataPath(); - } else { - // TODO: Fallback for BWC with older predecessor (ES) versions. - // Remove this once request.getCustomDataPath() always returns non-null - if (indexService != null) { - customDataPath = indexService.getIndexSettings().customDataPath(); - } else { - 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()); - } - } - } - final ShardPath shardPath = ShardPath.loadShardPath(logger, nodeEnv, shardId, customDataPath); - if (shardPath == null) { - return new StoreFilesMetadata(shardId, Store.MetadataSnapshot.EMPTY, Collections.emptyList()); - } - // note that this may fail if it can't get access to the shard lock. Since we check above there is an active shard, this means: - // 1) a shard is being constructed, which means the cluster-manager will not use a copy of this replica - // 2) A shard is shutting down and has not cleared it's content within lock timeout. In this case the cluster-manager may not - // reuse local resources. - final Store.MetadataSnapshot metadataSnapshot = Store.readMetadataSnapshot( - shardPath.resolveIndex(), + return getListShardMetadataOnLocalNode( + logger, shardId, - nodeEnv::shardLock, - logger + nodeEnv, + indicesService, + request.getCustomDataPath(), + settings, + clusterService ); - // We use peer recovery retention leases from the primary for allocating replicas. We should always have retention leases when - // we refresh shard info after the primary has started. Hence, we can ignore retention leases if there is no active shard. - return new StoreFilesMetadata(shardId, metadataSnapshot, Collections.emptyList()); - } finally { - TimeValue took = new TimeValue(System.nanoTime() - startTimeNS, TimeUnit.NANOSECONDS); - if (exists) { - logger.debug("{} loaded store meta data (took [{}])", shardId, took); - } else { - logger.trace("{} didn't find any store meta data to load (took [{}])", shardId, took); - } + } catch (IOException e) { + return new StoreFilesMetadata(shardId, Store.MetadataSnapshot.EMPTY, Collections.emptyList()); } } diff --git a/server/src/main/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadataBatch.java b/server/src/main/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadataBatch.java index e6e7cdb08a560..5533a406737c6 100644 --- a/server/src/main/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadataBatch.java +++ b/server/src/main/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadataBatch.java @@ -10,35 +10,25 @@ import org.apache.logging.log4j.message.ParameterizedMessage; import org.opensearch.OpenSearchException; -import org.opensearch.action.ActionListener; import org.opensearch.action.ActionType; import org.opensearch.action.FailedNodeException; +import org.opensearch.action.ActionListener; 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.common.settings.Settings; 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.common.settings.Settings; -import org.opensearch.common.unit.TimeValue; -import org.opensearch.env.NodeEnvironment; -import org.opensearch.gateway.AsyncBatchShardFetch; -import org.opensearch.index.IndexService; -import org.opensearch.index.IndexSettings; -import org.opensearch.index.seqno.ReplicationTracker; -import org.opensearch.index.seqno.RetentionLease; -import org.opensearch.index.shard.IndexShard; import org.opensearch.core.index.shard.ShardId; -import org.opensearch.index.shard.ShardPath; +import org.opensearch.env.NodeEnvironment; +import org.opensearch.gateway.AsyncShardFetch; import org.opensearch.index.store.Store; -import org.opensearch.index.store.StoreFileMetadata; import org.opensearch.indices.IndicesService; import org.opensearch.threadpool.ThreadPool; import org.opensearch.transport.TransportRequest; @@ -47,12 +37,9 @@ import java.io.IOException; import java.util.Collections; import java.util.HashMap; -import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Objects; -import java.util.concurrent.TimeUnit; -import java.util.stream.Collectors; /** * Transport action for fetching the batch of shard stores Metadata from a list of transport nodes @@ -65,9 +52,9 @@ public class TransportNodesListShardStoreMetadataBatch extends TransportNodesAct TransportNodesListShardStoreMetadataBatch.NodeRequest, TransportNodesListShardStoreMetadataBatch.NodeStoreFilesMetadataBatch> implements - AsyncBatchShardFetch.Lister< - TransportNodesListShardStoreMetadataBatch.NodesStoreFilesMetadataBatch, - TransportNodesListShardStoreMetadataBatch.NodeStoreFilesMetadataBatch> { + AsyncShardFetch.Lister< + TransportNodesListShardStoreMetadataBatch.NodesStoreFilesMetadataBatch, + TransportNodesListShardStoreMetadataBatch.NodeStoreFilesMetadataBatch> { public static final String ACTION_NAME = "internal:cluster/nodes/indices/shard/store/batch"; public static final ActionType TYPE = new ActionType<>( @@ -107,11 +94,11 @@ public TransportNodesListShardStoreMetadataBatch( @Override public void list( + Map shardAttributes, DiscoveryNode[] nodes, - Map shardIdsWithCustomDataPath, ActionListener listener ) { - execute(new TransportNodesListShardStoreMetadataBatch.Request(shardIdsWithCustomDataPath.entrySet().stream().map(entry -> new ShardAttributes(entry.getKey(), entry.getValue())).collect(Collectors.toList()), nodes), listener); + execute(new TransportNodesListShardStoreMetadataBatch.Request(shardAttributes, nodes), listener); } @Override @@ -138,132 +125,34 @@ protected NodeStoreFilesMetadataBatch nodeOperation(NodeRequest request) { try { return new NodeStoreFilesMetadataBatch(clusterService.localNode(), listStoreMetadata(request)); } catch (IOException e) { - throw new OpenSearchException("Failed to list store metadata for shards [" + request.getShardAttributes().stream().map(ShardAttributes::getShardId) + "]", e); + throw new OpenSearchException( + "Failed to list store metadata for shards [" + request.getShardAttributes().keySet().stream().map(ShardId::toString) + "]", + e + ); } } /** - * This method is similar to listStoreMetadata method of {@link TransportNodesListShardStoreMetadata} we loop over - * the shards here and populate the data about the shard store information held by the local node. - * - * @param request Request containing the map shardIdsWithCustomDataPath. - * @return NodeStoreFilesMetadata contains the data about the shard store held by the local node + * This method is similar to listStoreMetadata method of {@link TransportNodesListShardStoreMetadata} + * In this case we fetch the shard store files for batch of shards instead of one shard. */ private Map listStoreMetadata(NodeRequest request) throws IOException { Map shardStoreMetadataMap = new HashMap(); - for (ShardAttributes shardAttributes : request.getShardAttributes()) { + for (ShardAttributes shardAttributes : request.getShardAttributes().values()) { final ShardId shardId = shardAttributes.getShardId(); - logger.trace("listing store meta data for {}", shardId); - long startTimeNS = System.nanoTime(); - boolean exists = false; try { - IndexService indexService = indicesService.indexService(shardId.getIndex()); - if (indexService != null) { - IndexShard indexShard = indexService.getShardOrNull(shardId.id()); - if (indexShard != null) { - try { - final StoreFilesMetadata storeFilesMetadata = new StoreFilesMetadata( - shardId, - indexShard.snapshotStoreMetadata(), - indexShard.getPeerRecoveryRetentionLeases() - ); - exists = true; - shardStoreMetadataMap.put(shardId, new NodeStoreFilesMetadata(storeFilesMetadata, null)); - continue; - } catch (org.apache.lucene.index.IndexNotFoundException e) { - logger.trace(new ParameterizedMessage("[{}] node is missing index, responding with empty", shardId), e); - shardStoreMetadataMap.put( - shardId, - new NodeStoreFilesMetadata( - new StoreFilesMetadata(shardId, Store.MetadataSnapshot.EMPTY, Collections.emptyList()), - e - ) - ); - continue; - } catch (IOException e) { - logger.warn(new ParameterizedMessage("[{}] can't read metadata from store, responding with empty", shardId), e); - shardStoreMetadataMap.put( - shardId, - new NodeStoreFilesMetadata( - new StoreFilesMetadata(shardId, Store.MetadataSnapshot.EMPTY, Collections.emptyList()), - e - ) - ); - continue; - } - } - } - final String customDataPath; - if (shardAttributes.getCustomDataPath() != null) { - customDataPath = shardAttributes.getCustomDataPath(); - } else { - // TODO: Fallback for BWC with older predecessor (ES) versions. - // Remove this once request.getCustomDataPath() always returns non-null - if (indexService != null) { - customDataPath = indexService.getIndexSettings().customDataPath(); - } else { - 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); - shardStoreMetadataMap.put( - shardId, - new NodeStoreFilesMetadata( - new StoreFilesMetadata(shardId, Store.MetadataSnapshot.EMPTY, Collections.emptyList()), - new OpenSearchException("node doesn't have meta data for index " + shardId.getIndex()) - ) - ); - continue; - } - } - } - final ShardPath shardPath = ShardPath.loadShardPath(logger, nodeEnv, shardId, customDataPath); - if (shardPath == null) { - shardStoreMetadataMap.put( - shardId, - new NodeStoreFilesMetadata( - new StoreFilesMetadata(shardId, Store.MetadataSnapshot.EMPTY, Collections.emptyList()), - null - ) - ); - continue; - } - // note that this may fail if it can't get access to the shard lock. Since we check above there is an active shard, this - // means: - // 1) a shard is being constructed, which means the cluster-manager will not use a copy of this replica - // 2) A shard is shutting down and has not cleared it's content within lock timeout. In this case the cluster-manager may - // not - // reuse local resources. - final Store.MetadataSnapshot metadataSnapshot = Store.readMetadataSnapshot( - shardPath.resolveIndex(), - shardId, - nodeEnv::shardLock, - logger - ); - // We use peer recovery retention leases from the primary for allocating replicas. We should always have retention leases - // when - // we refresh shard info after the primary has started. Hence, we can ignore retention leases if there is no active shard. - shardStoreMetadataMap.put( - shardId, - new NodeStoreFilesMetadata(new StoreFilesMetadata(shardId, metadataSnapshot, Collections.emptyList()), null) - ); - } catch (Exception e) { - logger.trace("{} failed to load store metadata {}", shardId, e); - shardStoreMetadataMap.put( + StoreFilesMetadata storeFilesMetadata = TransportNodesListShardStoreMetadataHelper.getListShardMetadataOnLocalNode( + logger, shardId, - new NodeStoreFilesMetadata( - new StoreFilesMetadata(shardId, Store.MetadataSnapshot.EMPTY, Collections.emptyList()), - new OpenSearchException("failed to load store metadata", e) - ) + nodeEnv, + indicesService, + shardAttributes.getCustomDataPath(), + settings, + clusterService ); - } finally { - TimeValue took = new TimeValue(System.nanoTime() - startTimeNS, TimeUnit.NANOSECONDS); - if (exists) { - logger.debug("{} loaded store meta data (took [{}])", shardId, took); - } else { - logger.trace("{} didn't find any store meta data to load (took [{}])", shardId, took); - } + shardStoreMetadataMap.put(shardId, new NodeStoreFilesMetadata(storeFilesMetadata, null)); + } catch (IOException | OpenSearchException e) { + shardStoreMetadataMap.put(shardId, new NodeStoreFilesMetadata(new StoreFilesMetadata(shardId, Store.MetadataSnapshot.EMPTY, Collections.emptyList()), e)); } } return shardStoreMetadataMap; @@ -277,26 +166,26 @@ private Map listStoreMetadata(NodeRequest reque */ 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(List shardAttributes, DiscoveryNode[] nodes) { + public Request(Map shardAttributes, DiscoveryNode[] nodes) { super(nodes); this.shardAttributes = Objects.requireNonNull(shardAttributes); } - public List getShardAttributes() { + public Map getShardAttributes() { return 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)); } } @@ -394,11 +283,11 @@ public String toString() { */ public static class NodeRequest extends TransportRequest { - private final 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) { @@ -408,10 +297,10 @@ 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)); } - public List getShardAttributes() { + public Map getShardAttributes() { return shardAttributes; } } diff --git a/server/src/main/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadataHelper.java b/server/src/main/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadataHelper.java new file mode 100644 index 0000000000000..807ce530f3ec0 --- /dev/null +++ b/server/src/main/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadataHelper.java @@ -0,0 +1,117 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.indices.store; + +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.common.unit.TimeValue; +import org.opensearch.core.index.shard.ShardId; +import org.opensearch.env.NodeEnvironment; +import org.opensearch.index.IndexService; +import org.opensearch.index.IndexSettings; +import org.opensearch.index.shard.IndexShard; +import org.opensearch.index.shard.ShardPath; +import org.opensearch.index.store.Store; +import org.opensearch.indices.IndicesService; + +import java.io.IOException; +import java.util.Collections; +import java.util.concurrent.TimeUnit; + +/** + * This class has the common code used in {@link TransportNodesListShardStoreMetadata} and + * {@link TransportNodesListShardStoreMetadataBatch} 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 TransportNodesListShardStoreMetadata} will be deprecated and all the code will be moved to + * {@link TransportNodesListShardStoreMetadataBatch} + * + * @opensearch.internal + */ +public class TransportNodesListShardStoreMetadataHelper { + public static StoreFilesMetadata getListShardMetadataOnLocalNode( + Logger logger, + final ShardId shardId, + NodeEnvironment nodeEnv, + IndicesService indicesService, + String customDataPath, + Settings settings, + ClusterService clusterService + ) throws IOException, OpenSearchException { + logger.trace("listing store meta data for {}", shardId); + long startTimeNS = System.nanoTime(); + boolean exists = false; + try { + IndexService indexService = indicesService.indexService(shardId.getIndex()); + if (indexService != null) { + IndexShard indexShard = indexService.getShardOrNull(shardId.id()); + if (indexShard != null) { + try { + final StoreFilesMetadata storeFilesMetadata = new StoreFilesMetadata( + shardId, + indexShard.snapshotStoreMetadata(), + indexShard.getPeerRecoveryRetentionLeases() + ); + exists = true; + return storeFilesMetadata; + } catch (org.apache.lucene.index.IndexNotFoundException e) { + logger.trace(new ParameterizedMessage("[{}] node is missing index, responding with empty", shardId), e); + throw e; + } catch (IOException e) { + logger.warn(new ParameterizedMessage("[{}] can't read metadata from store, responding with empty", shardId), e); + throw e; + } + } + } + if (customDataPath == null) { + // TODO: Fallback for BWC with older predecessor (ES) versions. + // Remove this once request.getCustomDataPath() always returns non-null + if (indexService != null) { + customDataPath = indexService.getIndexSettings().customDataPath(); + } else { + 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()); + } + } + } + final ShardPath shardPath = ShardPath.loadShardPath(logger, nodeEnv, shardId, customDataPath); + if (shardPath == null) { + return new StoreFilesMetadata(shardId, Store.MetadataSnapshot.EMPTY, Collections.emptyList()); + } + // note that this may fail if it can't get access to the shard lock. Since we check above there is an active shard, this means: + // 1) a shard is being constructed, which means the cluster-manager will not use a copy of this replica + // 2) A shard is shutting down and has not cleared it's content within lock timeout. In this case the cluster-manager may not + // reuse local resources. + final Store.MetadataSnapshot metadataSnapshot = Store.readMetadataSnapshot( + shardPath.resolveIndex(), + shardId, + nodeEnv::shardLock, + logger + ); + // We use peer recovery retention leases from the primary for allocating replicas. We should always have retention leases when + // we refresh shard info after the primary has started. Hence, we can ignore retention leases if there is no active shard. + return new StoreFilesMetadata(shardId, metadataSnapshot, Collections.emptyList()); + } finally { + TimeValue took = new TimeValue(System.nanoTime() - startTimeNS, TimeUnit.NANOSECONDS); + if (exists) { + logger.debug("{} loaded store meta data (took [{}])", shardId, took); + } else { + logger.trace("{} didn't find any store meta data to load (took [{}])", shardId, took); + } + } + } +} From d623b28489ae40dbeb87569c03647ec07031d2d3 Mon Sep 17 00:00:00 2001 From: Shivansh Arora Date: Mon, 8 Jan 2024 15:39:10 +0530 Subject: [PATCH 07/22] Fix build failure after main merge Signed-off-by: Shivansh Arora --- .../gateway/AsyncShardFetchBatchTestUtils.java | 7 ++++--- ...sportNodesListShardStoreMetadataBatchIT.java | 14 +++++++------- .../gateway/AsyncBatchShardFetch.java | 2 +- .../gateway/ReplicaShardAllocator.java | 17 ++++------------- .../indices/store/StoreFilesMetadata.java | 6 +----- .../TransportNodesListShardStoreMetadata.java | 2 +- ...ansportNodesListShardStoreMetadataBatch.java | 14 ++++++++------ .../org/opensearch/index/store/StoreTests.java | 12 +++++------- 8 files changed, 31 insertions(+), 43 deletions(-) diff --git a/server/src/internalClusterTest/java/org/opensearch/gateway/AsyncShardFetchBatchTestUtils.java b/server/src/internalClusterTest/java/org/opensearch/gateway/AsyncShardFetchBatchTestUtils.java index 3f6303378b911..bfa70298ab563 100644 --- a/server/src/internalClusterTest/java/org/opensearch/gateway/AsyncShardFetchBatchTestUtils.java +++ b/server/src/internalClusterTest/java/org/opensearch/gateway/AsyncShardFetchBatchTestUtils.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,8 +45,8 @@ 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 shardCount) { + Map shardIdCustomDataPathMap = new HashMap<>(); for (String indexName : indices) { final Index index = resolveIndex(indexName); final String customDataPath = IndexMetadata.INDEX_DATA_PATH_SETTING.get( @@ -53,7 +54,7 @@ public static Map prepareRequestMap(String[] indices, int shard ); for (int shardIdNum = 0; shardIdNum < shardCount; shardIdNum++) { final ShardId shardId = new ShardId(index, shardIdNum); - shardIdCustomDataPathMap.put(shardId, customDataPath); + shardIdCustomDataPathMap.put(shardId, new ShardAttributes(shardId, customDataPath)); } } return shardIdCustomDataPathMap; diff --git a/server/src/internalClusterTest/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadataBatchIT.java b/server/src/internalClusterTest/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadataBatchIT.java index e8fd38ca90499..7b830b12e11fa 100644 --- a/server/src/internalClusterTest/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadataBatchIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadataBatchIT.java @@ -91,7 +91,7 @@ public void testShardStoreFetchCorruptedIndex() throws Exception { internalCluster().startNode(); String indexName = "test"; prepareIndices(new String[] { indexName }, 1, 1); - Map shardIdCustomDataPathMap = prepareRequestMap(new String[] { indexName }, 1); + Map shardAttributesMap = prepareRequestMap(new String[] { indexName }, 1); Index index = resolveIndex(indexName); ShardId shardId = new ShardId(index, 0); ClusterSearchShardsResponse searchShardsResponse = client().admin().cluster().prepareSearchShards(indexName).get(); @@ -103,7 +103,7 @@ public void testShardStoreFetchCorruptedIndex() throws Exception { TransportNodesListShardStoreMetadataBatch.NodesStoreFilesMetadataBatch response; response = ActionTestUtils.executeBlocking( internalCluster().getInstance(TransportNodesListShardStoreMetadataBatch.class), - new TransportNodesListShardStoreMetadataBatch.Request(shardIdCustomDataPathMap, discoveryNodes) + new TransportNodesListShardStoreMetadataBatch.Request(shardAttributesMap, discoveryNodes) ); TransportNodesListShardStoreMetadataBatch.NodeStoreFilesMetadata nodeStoreFilesMetadata = response.getNodesMap() .get(discoveryNodes[0].getId()) @@ -130,13 +130,13 @@ private TransportNodesListShardStoreMetadataBatch.NodesStoreFilesMetadataBatch p String[] indices, DiscoveryNode[] nodes ) { - Map shardIdCustomDataPathMap = null; + Map shardAttributesMap = null; prepareIndices(indices, 1, 1); - shardIdCustomDataPathMap = prepareRequestMap(indices, 1); + shardAttributesMap = prepareRequestMap(indices, 1); TransportNodesListShardStoreMetadataBatch.NodesStoreFilesMetadataBatch response; return ActionTestUtils.executeBlocking( internalCluster().getInstance(TransportNodesListShardStoreMetadataBatch.class), - new TransportNodesListShardStoreMetadataBatch.Request(shardIdCustomDataPathMap, nodes) + new TransportNodesListShardStoreMetadataBatch.Request(shardAttributesMap, nodes) ); } @@ -145,7 +145,7 @@ private void assertNodeStoreFilesMetadataFailureCase( ShardId shardId ) { assertNotNull(nodeStoreFilesMetadata.getStoreFileFetchException()); - TransportNodesListShardStoreMetadataBatch.StoreFilesMetadata storeFileMetadata = nodeStoreFilesMetadata.storeFilesMetadata(); + StoreFilesMetadata storeFileMetadata = nodeStoreFilesMetadata.storeFilesMetadata(); assertEquals(shardId, storeFileMetadata.shardId()); assertTrue(storeFileMetadata.peerRecoveryRetentionLeases().isEmpty()); } @@ -155,7 +155,7 @@ private void assertNodeStoreFilesMetadataSuccessCase( ShardId shardId ) { assertNull(nodeStoreFilesMetadata.getStoreFileFetchException()); - TransportNodesListShardStoreMetadataBatch.StoreFilesMetadata storeFileMetadata = nodeStoreFilesMetadata.storeFilesMetadata(); + StoreFilesMetadata storeFileMetadata = nodeStoreFilesMetadata.storeFilesMetadata(); assertFalse(storeFileMetadata.isEmpty()); assertEquals(shardId, storeFileMetadata.shardId()); assertNotNull(storeFileMetadata.peerRecoveryRetentionLeases()); diff --git a/server/src/main/java/org/opensearch/gateway/AsyncBatchShardFetch.java b/server/src/main/java/org/opensearch/gateway/AsyncBatchShardFetch.java index dcd09ed1eabf1..7a9ec7ae84a37 100644 --- a/server/src/main/java/org/opensearch/gateway/AsyncBatchShardFetch.java +++ b/server/src/main/java/org/opensearch/gateway/AsyncBatchShardFetch.java @@ -8,11 +8,11 @@ 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.action.ActionListener; import org.opensearch.core.index.shard.ShardId; import java.util.Map; diff --git a/server/src/main/java/org/opensearch/gateway/ReplicaShardAllocator.java b/server/src/main/java/org/opensearch/gateway/ReplicaShardAllocator.java index 5d65daff8cd41..36bb005599dbc 100644 --- a/server/src/main/java/org/opensearch/gateway/ReplicaShardAllocator.java +++ b/server/src/main/java/org/opensearch/gateway/ReplicaShardAllocator.java @@ -51,7 +51,7 @@ import org.opensearch.common.unit.TimeValue; import org.opensearch.core.common.unit.ByteSizeValue; import org.opensearch.index.store.StoreFileMetadata; -import org.opensearch.indices.store.TransportNodesListShardStoreMetadata; +import org.opensearch.indices.store.StoreFilesMetadata; import org.opensearch.indices.store.TransportNodesListShardStoreMetadata.NodeStoreFilesMetadata; import java.util.ArrayList; @@ -357,10 +357,7 @@ private static List augmentExplanationsWithStoreInfo( /** * Finds the store for the assigned shard in the fetched data, returns null if none is found. */ - private static StoreFilesMetadata findStore( - DiscoveryNode node, - AsyncShardFetch.FetchResult data - ) { + private static StoreFilesMetadata findStore(DiscoveryNode node, AsyncShardFetch.FetchResult data) { NodeStoreFilesMetadata nodeFilesStore = data.getData().get(node); if (nodeFilesStore == null) { return null; @@ -441,10 +438,7 @@ private MatchingNodes findMatchingNodes( return new MatchingNodes(matchingNodes, nodeDecisions); } - private static long computeMatchingBytes( - StoreFilesMetadata primaryStore, - StoreFilesMetadata storeFilesMetadata - ) { + private static long computeMatchingBytes(StoreFilesMetadata primaryStore, StoreFilesMetadata storeFilesMetadata) { long sizeMatched = 0; for (StoreFileMetadata storeFileMetadata : storeFilesMetadata) { String metadataFileName = storeFileMetadata.name(); @@ -455,10 +449,7 @@ private static long computeMatchingBytes( return sizeMatched; } - private static boolean hasMatchingSyncId( - StoreFilesMetadata primaryStore, - StoreFilesMetadata replicaStore - ) { + private static boolean hasMatchingSyncId(StoreFilesMetadata primaryStore, StoreFilesMetadata replicaStore) { String primarySyncId = primaryStore.syncId(); return primarySyncId != null && primarySyncId.equals(replicaStore.syncId()); } diff --git a/server/src/main/java/org/opensearch/indices/store/StoreFilesMetadata.java b/server/src/main/java/org/opensearch/indices/store/StoreFilesMetadata.java index 96e0589c8df9b..84db19879c190 100644 --- a/server/src/main/java/org/opensearch/indices/store/StoreFilesMetadata.java +++ b/server/src/main/java/org/opensearch/indices/store/StoreFilesMetadata.java @@ -32,11 +32,7 @@ public class StoreFilesMetadata implements Iterable, Writeabl private final Store.MetadataSnapshot metadataSnapshot; private final List peerRecoveryRetentionLeases; - public StoreFilesMetadata( - ShardId shardId, - Store.MetadataSnapshot metadataSnapshot, - List peerRecoveryRetentionLeases - ) { + public StoreFilesMetadata(ShardId shardId, Store.MetadataSnapshot metadataSnapshot, List peerRecoveryRetentionLeases) { this.shardId = shardId; this.metadataSnapshot = metadataSnapshot; this.peerRecoveryRetentionLeases = peerRecoveryRetentionLeases; diff --git a/server/src/main/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadata.java b/server/src/main/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadata.java index 1e031158d2eee..2c858d659521c 100644 --- a/server/src/main/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadata.java +++ b/server/src/main/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadata.java @@ -35,7 +35,6 @@ import org.opensearch.OpenSearchException; import org.opensearch.action.ActionType; import org.opensearch.action.FailedNodeException; -import org.opensearch.action.ActionListener; import org.opensearch.action.support.ActionFilters; import org.opensearch.action.support.nodes.BaseNodeResponse; import org.opensearch.action.support.nodes.BaseNodesRequest; @@ -47,6 +46,7 @@ import org.opensearch.common.Nullable; 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.core.index.shard.ShardId; diff --git a/server/src/main/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadataBatch.java b/server/src/main/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadataBatch.java index 5533a406737c6..aea7fb6d84ecc 100644 --- a/server/src/main/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadataBatch.java +++ b/server/src/main/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadataBatch.java @@ -8,11 +8,9 @@ package org.opensearch.indices.store; -import org.apache.logging.log4j.message.ParameterizedMessage; import org.opensearch.OpenSearchException; import org.opensearch.action.ActionType; import org.opensearch.action.FailedNodeException; -import org.opensearch.action.ActionListener; import org.opensearch.action.support.ActionFilters; import org.opensearch.action.support.nodes.BaseNodeResponse; import org.opensearch.action.support.nodes.BaseNodesRequest; @@ -23,6 +21,7 @@ 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.core.index.shard.ShardId; @@ -52,9 +51,9 @@ public class TransportNodesListShardStoreMetadataBatch extends TransportNodesAct TransportNodesListShardStoreMetadataBatch.NodeRequest, TransportNodesListShardStoreMetadataBatch.NodeStoreFilesMetadataBatch> implements - AsyncShardFetch.Lister< - TransportNodesListShardStoreMetadataBatch.NodesStoreFilesMetadataBatch, - TransportNodesListShardStoreMetadataBatch.NodeStoreFilesMetadataBatch> { + AsyncShardFetch.Lister< + TransportNodesListShardStoreMetadataBatch.NodesStoreFilesMetadataBatch, + TransportNodesListShardStoreMetadataBatch.NodeStoreFilesMetadataBatch> { public static final String ACTION_NAME = "internal:cluster/nodes/indices/shard/store/batch"; public static final ActionType TYPE = new ActionType<>( @@ -152,7 +151,10 @@ private Map listStoreMetadata(NodeRequest reque ); shardStoreMetadataMap.put(shardId, new NodeStoreFilesMetadata(storeFilesMetadata, null)); } catch (IOException | OpenSearchException e) { - shardStoreMetadataMap.put(shardId, new NodeStoreFilesMetadata(new StoreFilesMetadata(shardId, Store.MetadataSnapshot.EMPTY, Collections.emptyList()), e)); + shardStoreMetadataMap.put( + shardId, + new NodeStoreFilesMetadata(new StoreFilesMetadata(shardId, Store.MetadataSnapshot.EMPTY, Collections.emptyList()), e) + ); } } return shardStoreMetadataMap; diff --git a/server/src/test/java/org/opensearch/index/store/StoreTests.java b/server/src/test/java/org/opensearch/index/store/StoreTests.java index fdbc424109fbd..da967ec306fec 100644 --- a/server/src/test/java/org/opensearch/index/store/StoreTests.java +++ b/server/src/test/java/org/opensearch/index/store/StoreTests.java @@ -88,7 +88,6 @@ import org.opensearch.index.translog.Translog; import org.opensearch.indices.replication.common.ReplicationType; import org.opensearch.indices.store.StoreFilesMetadata; -import org.opensearch.indices.store.TransportNodesListShardStoreMetadata; import org.opensearch.test.DummyShardLock; import org.opensearch.test.FeatureFlagSetter; import org.opensearch.test.IndexSettingsModule; @@ -981,12 +980,11 @@ public void testStreamStoreFilesMetadata() throws Exception { ) ); } - StoreFilesMetadata outStoreFileMetadata = - new StoreFilesMetadata( - new ShardId("test", "_na_", 0), - metadataSnapshot, - peerRecoveryRetentionLeases - ); + StoreFilesMetadata outStoreFileMetadata = new StoreFilesMetadata( + new ShardId("test", "_na_", 0), + metadataSnapshot, + peerRecoveryRetentionLeases + ); ByteArrayOutputStream outBuffer = new ByteArrayOutputStream(); OutputStreamStreamOutput out = new OutputStreamStreamOutput(outBuffer); org.opensearch.Version targetNodeVersion = randomVersion(random()); From 934ba3f60ac49d7fbf52eb1c8096890872c7b911 Mon Sep 17 00:00:00 2001 From: Shivansh Arora Date: Thu, 25 Jan 2024 16:52:29 +0530 Subject: [PATCH 08/22] Removed AsyncBatchShardFetch Signed-off-by: Shivansh Arora --- .../gateway/AsyncBatchShardFetch.java | 34 ------------------- 1 file changed, 34 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 7a9ec7ae84a37..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.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.action.ActionListener; -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); - } -} From 2d96a53d4fb7917342bf9f409455aa36d1052c57 Mon Sep 17 00:00:00 2001 From: Shivansh Arora Date: Mon, 29 Jan 2024 15:27:30 +0530 Subject: [PATCH 09/22] Address review comments Signed-off-by: Shivansh Arora --- ...ils.java => GatewayRecoveryTestUtils.java} | 2 +- .../gateway/RecoveryFromGatewayIT.java | 143 +++++++++++++++ ...ortNodesListShardStoreMetadataBatchIT.java | 163 ------------------ .../gateway/ReplicaShardAllocator.java | 2 +- .../indices/store/StoreFilesMetadata.java | 111 ------------ .../TransportNodesListShardStoreMetadata.java | 1 + ...sportNodesListShardStoreMetadataBatch.java | 3 +- ...portNodesListShardStoreMetadataHelper.java | 99 ++++++++++- .../gateway/ReplicaShardAllocatorTests.java | 2 +- .../opensearch/index/store/StoreTests.java | 2 +- 10 files changed, 248 insertions(+), 280 deletions(-) rename server/src/internalClusterTest/java/org/opensearch/gateway/{AsyncShardFetchBatchTestUtils.java => GatewayRecoveryTestUtils.java} (98%) delete mode 100644 server/src/internalClusterTest/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadataBatchIT.java delete mode 100644 server/src/main/java/org/opensearch/indices/store/StoreFilesMetadata.java diff --git a/server/src/internalClusterTest/java/org/opensearch/gateway/AsyncShardFetchBatchTestUtils.java b/server/src/internalClusterTest/java/org/opensearch/gateway/GatewayRecoveryTestUtils.java similarity index 98% rename from server/src/internalClusterTest/java/org/opensearch/gateway/AsyncShardFetchBatchTestUtils.java rename to server/src/internalClusterTest/java/org/opensearch/gateway/GatewayRecoveryTestUtils.java index bfa70298ab563..e5d1ee29376cb 100644 --- a/server/src/internalClusterTest/java/org/opensearch/gateway/AsyncShardFetchBatchTestUtils.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 AsyncShardFetchBatchTestUtils { +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..34c069f74c62a 100644 --- a/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java @@ -32,10 +32,14 @@ package org.opensearch.gateway; +import org.opensearch.Version; import org.opensearch.action.admin.cluster.configuration.AddVotingConfigExclusionsAction; 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.reroute.ClusterRerouteResponse; +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; @@ -44,6 +48,7 @@ import org.opensearch.cluster.coordination.ElectionSchedulerFactory; import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.cluster.node.DiscoveryNode; +import org.opensearch.cluster.routing.ShardRouting; import org.opensearch.cluster.routing.UnassignedInfo; import org.opensearch.cluster.service.ClusterService; import org.opensearch.common.settings.Settings; @@ -60,6 +65,9 @@ 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.indices.store.TransportNodesListShardStoreMetadataBatch; +import org.opensearch.indices.store.TransportNodesListShardStoreMetadataHelper; import org.opensearch.plugins.Plugin; import org.opensearch.test.InternalSettingsPlugin; import org.opensearch.test.InternalTestCluster.RestartCallback; @@ -79,12 +87,18 @@ import java.util.List; import java.util.Map; import java.util.Set; +import java.util.concurrent.ExecutionException; import java.util.stream.IntStream; +import static java.util.Collections.emptyMap; +import static java.util.Collections.emptySet; import static org.opensearch.cluster.coordination.ClusterBootstrapService.INITIAL_CLUSTER_MANAGER_NODES_SETTING; 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 +748,133 @@ public void testMessyElectionsStillMakeClusterGoGreen() throws Exception { internalCluster().fullRestart(); ensureGreen("test"); } + + public void testSingleShardStoreFetch() throws ExecutionException, InterruptedException { + String indexName = "test"; + DiscoveryNode[] nodes = getDiscoveryNodes(); + TransportNodesListShardStoreMetadataBatch.NodesStoreFilesMetadataBatch response = prepareAndSendRequest( + new String[] { indexName }, + nodes + ); + Index index = resolveIndex(indexName); + ShardId shardId = new ShardId(index, 0); + TransportNodesListShardStoreMetadataBatch.NodeStoreFilesMetadata nodeStoreFilesMetadata = response.getNodesMap() + .get(nodes[0].getId()) + .getNodeStoreFilesMetadataBatch() + .get(shardId); + assertNodeStoreFilesMetadataSuccessCase(nodeStoreFilesMetadata, shardId); + } + + public void testShardStoreFetchMultiNodeMultiIndexes() throws Exception { + // start second node + internalCluster().startNode(); + String indexName1 = "test1"; + String indexName2 = "test2"; + DiscoveryNode[] nodes = getDiscoveryNodes(); + TransportNodesListShardStoreMetadataBatch.NodesStoreFilesMetadataBatch response = prepareAndSendRequest( + new String[] { indexName1, indexName2 }, + nodes + ); + ClusterSearchShardsResponse searchShardsResponse = client().admin().cluster().prepareSearchShards(indexName1, indexName2).get(); + for (ClusterSearchShardsGroup clusterSearchShardsGroup : searchShardsResponse.getGroups()) { + ShardId shardId = clusterSearchShardsGroup.getShardId(); + ShardRouting[] shardRoutings = clusterSearchShardsGroup.getShards(); + assertEquals(2, shardRoutings.length); + for (ShardRouting shardRouting : shardRoutings) { + TransportNodesListShardStoreMetadataBatch.NodeStoreFilesMetadata nodeStoreFilesMetadata = response.getNodesMap() + .get(shardRouting.currentNodeId()) + .getNodeStoreFilesMetadataBatch() + .get(shardId); + assertNodeStoreFilesMetadataSuccessCase(nodeStoreFilesMetadata, shardId); + } + } + } + + public void testShardStoreFetchNodeNotConnected() { + DiscoveryNode nonExistingNode = new DiscoveryNode("foo", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT); + String indexName = "test"; + TransportNodesListShardStoreMetadataBatch.NodesStoreFilesMetadataBatch response = prepareAndSendRequest( + new String[] { indexName }, + new DiscoveryNode[] { nonExistingNode } + ); + assertTrue(response.hasFailures()); + assertEquals(1, response.failures().size()); + assertEquals(nonExistingNode.getId(), response.failures().get(0).nodeId()); + } + + public void testShardStoreFetchCorruptedIndex() throws Exception { + // start second node + internalCluster().startNode(); + String indexName = "test"; + prepareIndices(new String[] { indexName }, 1, 1); + Map shardAttributesMap = prepareRequestMap(new String[] { indexName }, 1); + Index index = resolveIndex(indexName); + ShardId shardId = new ShardId(index, 0); + ClusterSearchShardsResponse searchShardsResponse = client().admin().cluster().prepareSearchShards(indexName).get(); + assertEquals(2, searchShardsResponse.getNodes().length); + corruptShard(searchShardsResponse.getNodes()[0].getName(), shardId); + corruptShard(searchShardsResponse.getNodes()[1].getName(), shardId); + ClusterRerouteResponse clusterRerouteResponse = client().admin().cluster().prepareReroute().setRetryFailed(false).get(); + DiscoveryNode[] discoveryNodes = getDiscoveryNodes(); + TransportNodesListShardStoreMetadataBatch.NodesStoreFilesMetadataBatch response; + response = ActionTestUtils.executeBlocking( + internalCluster().getInstance(TransportNodesListShardStoreMetadataBatch.class), + new TransportNodesListShardStoreMetadataBatch.Request(shardAttributesMap, discoveryNodes) + ); + TransportNodesListShardStoreMetadataBatch.NodeStoreFilesMetadata nodeStoreFilesMetadata = response.getNodesMap() + .get(discoveryNodes[0].getId()) + .getNodeStoreFilesMetadataBatch() + .get(shardId); + assertNodeStoreFilesMetadataFailureCase(nodeStoreFilesMetadata, shardId); + } + + 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() + ); + index(index, "type", "1", Collections.emptyMap()); + flush(index); + } + } + + private TransportNodesListShardStoreMetadataBatch.NodesStoreFilesMetadataBatch prepareAndSendRequest( + String[] indices, + DiscoveryNode[] nodes + ) { + Map shardAttributesMap = null; + prepareIndices(indices, 1, 1); + shardAttributesMap = prepareRequestMap(indices, 1); + TransportNodesListShardStoreMetadataBatch.NodesStoreFilesMetadataBatch response; + return ActionTestUtils.executeBlocking( + internalCluster().getInstance(TransportNodesListShardStoreMetadataBatch.class), + new TransportNodesListShardStoreMetadataBatch.Request(shardAttributesMap, nodes) + ); + } + + private void assertNodeStoreFilesMetadataFailureCase( + TransportNodesListShardStoreMetadataBatch.NodeStoreFilesMetadata nodeStoreFilesMetadata, + ShardId shardId + ) { + assertNotNull(nodeStoreFilesMetadata.getStoreFileFetchException()); + TransportNodesListShardStoreMetadataHelper.StoreFilesMetadata storeFileMetadata = nodeStoreFilesMetadata.storeFilesMetadata(); + assertEquals(shardId, storeFileMetadata.shardId()); + assertTrue(storeFileMetadata.peerRecoveryRetentionLeases().isEmpty()); + } + + private void assertNodeStoreFilesMetadataSuccessCase( + TransportNodesListShardStoreMetadataBatch.NodeStoreFilesMetadata nodeStoreFilesMetadata, + ShardId shardId + ) { + assertNull(nodeStoreFilesMetadata.getStoreFileFetchException()); + TransportNodesListShardStoreMetadataHelper.StoreFilesMetadata storeFileMetadata = nodeStoreFilesMetadata.storeFilesMetadata(); + assertFalse(storeFileMetadata.isEmpty()); + assertEquals(shardId, storeFileMetadata.shardId()); + assertNotNull(storeFileMetadata.peerRecoveryRetentionLeases()); + } + } diff --git a/server/src/internalClusterTest/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadataBatchIT.java b/server/src/internalClusterTest/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadataBatchIT.java deleted file mode 100644 index 7b830b12e11fa..0000000000000 --- a/server/src/internalClusterTest/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadataBatchIT.java +++ /dev/null @@ -1,163 +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.Version; -import org.opensearch.action.admin.cluster.reroute.ClusterRerouteResponse; -import org.opensearch.action.admin.cluster.shards.ClusterSearchShardsGroup; -import org.opensearch.action.admin.cluster.shards.ClusterSearchShardsResponse; -import org.opensearch.action.support.ActionTestUtils; -import org.opensearch.cluster.node.DiscoveryNode; -import org.opensearch.cluster.routing.ShardRouting; -import org.opensearch.common.settings.Settings; -import org.opensearch.core.index.Index; -import org.opensearch.core.index.shard.ShardId; -import org.opensearch.test.OpenSearchIntegTestCase; - -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 TransportNodesListShardStoreMetadataBatchIT extends OpenSearchIntegTestCase { - - public void testSingleShardStoreFetch() throws ExecutionException, InterruptedException { - String indexName = "test"; - DiscoveryNode[] nodes = getDiscoveryNodes(); - TransportNodesListShardStoreMetadataBatch.NodesStoreFilesMetadataBatch response = prepareAndSendRequest( - new String[] { indexName }, - nodes - ); - Index index = resolveIndex(indexName); - ShardId shardId = new ShardId(index, 0); - TransportNodesListShardStoreMetadataBatch.NodeStoreFilesMetadata nodeStoreFilesMetadata = response.getNodesMap() - .get(nodes[0].getId()) - .getNodeStoreFilesMetadataBatch() - .get(shardId); - assertNodeStoreFilesMetadataSuccessCase(nodeStoreFilesMetadata, shardId); - } - - public void testShardStoreFetchMultiNodeMultiIndexes() throws Exception { - // start second node - internalCluster().startNode(); - String indexName1 = "test1"; - String indexName2 = "test2"; - DiscoveryNode[] nodes = getDiscoveryNodes(); - TransportNodesListShardStoreMetadataBatch.NodesStoreFilesMetadataBatch response = prepareAndSendRequest( - new String[] { indexName1, indexName2 }, - nodes - ); - ClusterSearchShardsResponse searchShardsResponse = client().admin().cluster().prepareSearchShards(indexName1, indexName2).get(); - for (ClusterSearchShardsGroup clusterSearchShardsGroup : searchShardsResponse.getGroups()) { - ShardId shardId = clusterSearchShardsGroup.getShardId(); - ShardRouting[] shardRoutings = clusterSearchShardsGroup.getShards(); - assertEquals(2, shardRoutings.length); - for (ShardRouting shardRouting : shardRoutings) { - TransportNodesListShardStoreMetadataBatch.NodeStoreFilesMetadata nodeStoreFilesMetadata = response.getNodesMap() - .get(shardRouting.currentNodeId()) - .getNodeStoreFilesMetadataBatch() - .get(shardId); - assertNodeStoreFilesMetadataSuccessCase(nodeStoreFilesMetadata, shardId); - } - } - } - - public void testShardStoreFetchNodeNotConnected() { - DiscoveryNode nonExistingNode = new DiscoveryNode("foo", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT); - String indexName = "test"; - TransportNodesListShardStoreMetadataBatch.NodesStoreFilesMetadataBatch response = prepareAndSendRequest( - new String[] { indexName }, - new DiscoveryNode[] { nonExistingNode } - ); - assertTrue(response.hasFailures()); - assertEquals(1, response.failures().size()); - assertEquals(nonExistingNode.getId(), response.failures().get(0).nodeId()); - } - - public void testShardStoreFetchCorruptedIndex() throws Exception { - // start second node - internalCluster().startNode(); - String indexName = "test"; - prepareIndices(new String[] { indexName }, 1, 1); - Map shardAttributesMap = prepareRequestMap(new String[] { indexName }, 1); - Index index = resolveIndex(indexName); - ShardId shardId = new ShardId(index, 0); - ClusterSearchShardsResponse searchShardsResponse = client().admin().cluster().prepareSearchShards(indexName).get(); - assertEquals(2, searchShardsResponse.getNodes().length); - corruptShard(searchShardsResponse.getNodes()[0].getName(), shardId); - corruptShard(searchShardsResponse.getNodes()[1].getName(), shardId); - ClusterRerouteResponse clusterRerouteResponse = client().admin().cluster().prepareReroute().setRetryFailed(false).get(); - DiscoveryNode[] discoveryNodes = getDiscoveryNodes(); - TransportNodesListShardStoreMetadataBatch.NodesStoreFilesMetadataBatch response; - response = ActionTestUtils.executeBlocking( - internalCluster().getInstance(TransportNodesListShardStoreMetadataBatch.class), - new TransportNodesListShardStoreMetadataBatch.Request(shardAttributesMap, discoveryNodes) - ); - TransportNodesListShardStoreMetadataBatch.NodeStoreFilesMetadata nodeStoreFilesMetadata = response.getNodesMap() - .get(discoveryNodes[0].getId()) - .getNodeStoreFilesMetadataBatch() - .get(shardId); - assertNodeStoreFilesMetadataFailureCase(nodeStoreFilesMetadata, shardId); - } - - 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() - ); - index(index, "type", "1"); - flush(index); - } - } - - private TransportNodesListShardStoreMetadataBatch.NodesStoreFilesMetadataBatch prepareAndSendRequest( - String[] indices, - DiscoveryNode[] nodes - ) { - Map shardAttributesMap = null; - prepareIndices(indices, 1, 1); - shardAttributesMap = prepareRequestMap(indices, 1); - TransportNodesListShardStoreMetadataBatch.NodesStoreFilesMetadataBatch response; - return ActionTestUtils.executeBlocking( - internalCluster().getInstance(TransportNodesListShardStoreMetadataBatch.class), - new TransportNodesListShardStoreMetadataBatch.Request(shardAttributesMap, nodes) - ); - } - - private void assertNodeStoreFilesMetadataFailureCase( - TransportNodesListShardStoreMetadataBatch.NodeStoreFilesMetadata nodeStoreFilesMetadata, - ShardId shardId - ) { - assertNotNull(nodeStoreFilesMetadata.getStoreFileFetchException()); - StoreFilesMetadata storeFileMetadata = nodeStoreFilesMetadata.storeFilesMetadata(); - assertEquals(shardId, storeFileMetadata.shardId()); - assertTrue(storeFileMetadata.peerRecoveryRetentionLeases().isEmpty()); - } - - private void assertNodeStoreFilesMetadataSuccessCase( - TransportNodesListShardStoreMetadataBatch.NodeStoreFilesMetadata nodeStoreFilesMetadata, - ShardId shardId - ) { - assertNull(nodeStoreFilesMetadata.getStoreFileFetchException()); - StoreFilesMetadata storeFileMetadata = nodeStoreFilesMetadata.storeFilesMetadata(); - assertFalse(storeFileMetadata.isEmpty()); - assertEquals(shardId, storeFileMetadata.shardId()); - assertNotNull(storeFileMetadata.peerRecoveryRetentionLeases()); - } -} diff --git a/server/src/main/java/org/opensearch/gateway/ReplicaShardAllocator.java b/server/src/main/java/org/opensearch/gateway/ReplicaShardAllocator.java index 36bb005599dbc..9b730ded7a55d 100644 --- a/server/src/main/java/org/opensearch/gateway/ReplicaShardAllocator.java +++ b/server/src/main/java/org/opensearch/gateway/ReplicaShardAllocator.java @@ -51,7 +51,7 @@ import org.opensearch.common.unit.TimeValue; import org.opensearch.core.common.unit.ByteSizeValue; import org.opensearch.index.store.StoreFileMetadata; -import org.opensearch.indices.store.StoreFilesMetadata; +import org.opensearch.indices.store.TransportNodesListShardStoreMetadataHelper.StoreFilesMetadata; import org.opensearch.indices.store.TransportNodesListShardStoreMetadata.NodeStoreFilesMetadata; import java.util.ArrayList; 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 84db19879c190..0000000000000 --- a/server/src/main/java/org/opensearch/indices/store/StoreFilesMetadata.java +++ /dev/null @@ -1,111 +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() - + "}" - + '}'; - } -} diff --git a/server/src/main/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadata.java b/server/src/main/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadata.java index 2c858d659521c..268a7144d056d 100644 --- a/server/src/main/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadata.java +++ b/server/src/main/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadata.java @@ -57,6 +57,7 @@ import org.opensearch.threadpool.ThreadPool; import org.opensearch.transport.TransportRequest; import org.opensearch.transport.TransportService; +import org.opensearch.indices.store.TransportNodesListShardStoreMetadataHelper.StoreFilesMetadata; import java.io.IOException; import java.util.Collections; diff --git a/server/src/main/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadataBatch.java b/server/src/main/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadataBatch.java index aea7fb6d84ecc..a78931379bf64 100644 --- a/server/src/main/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadataBatch.java +++ b/server/src/main/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadataBatch.java @@ -32,6 +32,7 @@ import org.opensearch.threadpool.ThreadPool; import org.opensearch.transport.TransportRequest; import org.opensearch.transport.TransportService; +import org.opensearch.indices.store.TransportNodesListShardStoreMetadataHelper.StoreFilesMetadata; import java.io.IOException; import java.util.Collections; @@ -150,7 +151,7 @@ private Map listStoreMetadata(NodeRequest reque clusterService ); shardStoreMetadataMap.put(shardId, new NodeStoreFilesMetadata(storeFilesMetadata, null)); - } catch (IOException | OpenSearchException e) { + } catch (IOException e) { shardStoreMetadataMap.put( shardId, new NodeStoreFilesMetadata(new StoreFilesMetadata(shardId, Store.MetadataSnapshot.EMPTY, Collections.emptyList()), e) diff --git a/server/src/main/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadataHelper.java b/server/src/main/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadataHelper.java index 807ce530f3ec0..fb8f60caad4b0 100644 --- a/server/src/main/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadataHelper.java +++ b/server/src/main/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadataHelper.java @@ -12,20 +12,29 @@ import org.apache.logging.log4j.message.ParameterizedMessage; import org.opensearch.OpenSearchException; import org.opensearch.cluster.metadata.IndexMetadata; +import org.opensearch.cluster.node.DiscoveryNode; import org.opensearch.cluster.service.ClusterService; import org.opensearch.common.settings.Settings; import org.opensearch.common.unit.TimeValue; +import org.opensearch.core.common.io.stream.StreamInput; +import org.opensearch.core.common.io.stream.StreamOutput; +import org.opensearch.core.common.io.stream.Writeable; import org.opensearch.core.index.shard.ShardId; import org.opensearch.env.NodeEnvironment; import org.opensearch.index.IndexService; import org.opensearch.index.IndexSettings; +import org.opensearch.index.seqno.ReplicationTracker; +import org.opensearch.index.seqno.RetentionLease; import org.opensearch.index.shard.IndexShard; import org.opensearch.index.shard.ShardPath; import org.opensearch.index.store.Store; +import org.opensearch.index.store.StoreFileMetadata; import org.opensearch.indices.IndicesService; import java.io.IOException; import java.util.Collections; +import java.util.Iterator; +import java.util.List; import java.util.concurrent.TimeUnit; /** @@ -47,7 +56,7 @@ public static StoreFilesMetadata getListShardMetadataOnLocalNode( String customDataPath, Settings settings, ClusterService clusterService - ) throws IOException, OpenSearchException { + ) throws IOException { logger.trace("listing store meta data for {}", shardId); long startTimeNS = System.nanoTime(); boolean exists = false; @@ -114,4 +123,92 @@ public static StoreFilesMetadata getListShardMetadataOnLocalNode( } } } + + /** + * Metadata for store files + * + * @opensearch.internal + */ + public static 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() + + "}" + + '}'; + } + } } diff --git a/server/src/test/java/org/opensearch/gateway/ReplicaShardAllocatorTests.java b/server/src/test/java/org/opensearch/gateway/ReplicaShardAllocatorTests.java index 10a136d08a451..86f3755fc5fed 100644 --- a/server/src/test/java/org/opensearch/gateway/ReplicaShardAllocatorTests.java +++ b/server/src/test/java/org/opensearch/gateway/ReplicaShardAllocatorTests.java @@ -67,7 +67,7 @@ import org.opensearch.index.seqno.RetentionLease; import org.opensearch.index.store.Store; import org.opensearch.index.store.StoreFileMetadata; -import org.opensearch.indices.store.StoreFilesMetadata; +import org.opensearch.indices.store.TransportNodesListShardStoreMetadataHelper.StoreFilesMetadata; import org.opensearch.indices.store.TransportNodesListShardStoreMetadata; import org.opensearch.snapshots.SnapshotShardSizeInfo; import org.junit.Before; diff --git a/server/src/test/java/org/opensearch/index/store/StoreTests.java b/server/src/test/java/org/opensearch/index/store/StoreTests.java index da967ec306fec..ab30a4c1c435f 100644 --- a/server/src/test/java/org/opensearch/index/store/StoreTests.java +++ b/server/src/test/java/org/opensearch/index/store/StoreTests.java @@ -87,7 +87,7 @@ import org.opensearch.index.shard.ShardPath; import org.opensearch.index.translog.Translog; import org.opensearch.indices.replication.common.ReplicationType; -import org.opensearch.indices.store.StoreFilesMetadata; +import org.opensearch.indices.store.TransportNodesListShardStoreMetadataHelper.StoreFilesMetadata; import org.opensearch.test.DummyShardLock; import org.opensearch.test.FeatureFlagSetter; import org.opensearch.test.IndexSettingsModule; From 5bc620296f1786b82e591c7023c5e8153aed136a Mon Sep 17 00:00:00 2001 From: Shivansh Arora Date: Mon, 29 Jan 2024 15:34:40 +0530 Subject: [PATCH 10/22] Spotless changes Signed-off-by: Shivansh Arora --- .../java/org/opensearch/gateway/ReplicaShardAllocator.java | 2 +- .../indices/store/TransportNodesListShardStoreMetadata.java | 2 +- .../store/TransportNodesListShardStoreMetadataBatch.java | 2 +- .../store/TransportNodesListShardStoreMetadataHelper.java | 6 +++++- .../org/opensearch/gateway/ReplicaShardAllocatorTests.java | 2 +- 5 files changed, 9 insertions(+), 5 deletions(-) diff --git a/server/src/main/java/org/opensearch/gateway/ReplicaShardAllocator.java b/server/src/main/java/org/opensearch/gateway/ReplicaShardAllocator.java index 9b730ded7a55d..89db3198662fa 100644 --- a/server/src/main/java/org/opensearch/gateway/ReplicaShardAllocator.java +++ b/server/src/main/java/org/opensearch/gateway/ReplicaShardAllocator.java @@ -51,8 +51,8 @@ import org.opensearch.common.unit.TimeValue; import org.opensearch.core.common.unit.ByteSizeValue; import org.opensearch.index.store.StoreFileMetadata; -import org.opensearch.indices.store.TransportNodesListShardStoreMetadataHelper.StoreFilesMetadata; import org.opensearch.indices.store.TransportNodesListShardStoreMetadata.NodeStoreFilesMetadata; +import org.opensearch.indices.store.TransportNodesListShardStoreMetadataHelper.StoreFilesMetadata; import java.util.ArrayList; import java.util.Collections; diff --git a/server/src/main/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadata.java b/server/src/main/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadata.java index 268a7144d056d..7bf64926cdd77 100644 --- a/server/src/main/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadata.java +++ b/server/src/main/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadata.java @@ -54,10 +54,10 @@ import org.opensearch.gateway.AsyncShardFetch; import org.opensearch.index.store.Store; import org.opensearch.indices.IndicesService; +import org.opensearch.indices.store.TransportNodesListShardStoreMetadataHelper.StoreFilesMetadata; import org.opensearch.threadpool.ThreadPool; import org.opensearch.transport.TransportRequest; import org.opensearch.transport.TransportService; -import org.opensearch.indices.store.TransportNodesListShardStoreMetadataHelper.StoreFilesMetadata; import java.io.IOException; import java.util.Collections; diff --git a/server/src/main/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadataBatch.java b/server/src/main/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadataBatch.java index a78931379bf64..b840d4560f05e 100644 --- a/server/src/main/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadataBatch.java +++ b/server/src/main/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadataBatch.java @@ -29,10 +29,10 @@ import org.opensearch.gateway.AsyncShardFetch; import org.opensearch.index.store.Store; import org.opensearch.indices.IndicesService; +import org.opensearch.indices.store.TransportNodesListShardStoreMetadataHelper.StoreFilesMetadata; import org.opensearch.threadpool.ThreadPool; import org.opensearch.transport.TransportRequest; import org.opensearch.transport.TransportService; -import org.opensearch.indices.store.TransportNodesListShardStoreMetadataHelper.StoreFilesMetadata; import java.io.IOException; import java.util.Collections; diff --git a/server/src/main/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadataHelper.java b/server/src/main/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadataHelper.java index fb8f60caad4b0..e1b72774e8fe1 100644 --- a/server/src/main/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadataHelper.java +++ b/server/src/main/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadataHelper.java @@ -134,7 +134,11 @@ public static class StoreFilesMetadata implements Iterable, W private final Store.MetadataSnapshot metadataSnapshot; private final List peerRecoveryRetentionLeases; - public StoreFilesMetadata(ShardId shardId, Store.MetadataSnapshot metadataSnapshot, List peerRecoveryRetentionLeases) { + public StoreFilesMetadata( + ShardId shardId, + Store.MetadataSnapshot metadataSnapshot, + List peerRecoveryRetentionLeases + ) { this.shardId = shardId; this.metadataSnapshot = metadataSnapshot; this.peerRecoveryRetentionLeases = peerRecoveryRetentionLeases; diff --git a/server/src/test/java/org/opensearch/gateway/ReplicaShardAllocatorTests.java b/server/src/test/java/org/opensearch/gateway/ReplicaShardAllocatorTests.java index 86f3755fc5fed..ae56bc0f8b3d2 100644 --- a/server/src/test/java/org/opensearch/gateway/ReplicaShardAllocatorTests.java +++ b/server/src/test/java/org/opensearch/gateway/ReplicaShardAllocatorTests.java @@ -67,8 +67,8 @@ import org.opensearch.index.seqno.RetentionLease; import org.opensearch.index.store.Store; import org.opensearch.index.store.StoreFileMetadata; -import org.opensearch.indices.store.TransportNodesListShardStoreMetadataHelper.StoreFilesMetadata; import org.opensearch.indices.store.TransportNodesListShardStoreMetadata; +import org.opensearch.indices.store.TransportNodesListShardStoreMetadataHelper.StoreFilesMetadata; import org.opensearch.snapshots.SnapshotShardSizeInfo; import org.junit.Before; From 875f4fa3bbd610003818bf86322f183f33e4de04 Mon Sep 17 00:00:00 2001 From: Shivansh Arora Date: Thu, 1 Feb 2024 16:37:43 +0530 Subject: [PATCH 11/22] Added UsingBatchAction suffix to ITs Signed-off-by: Shivansh Arora --- .../org/opensearch/gateway/RecoveryFromGatewayIT.java | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) diff --git a/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java b/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java index 34c069f74c62a..b222b14b862e8 100644 --- a/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java @@ -749,7 +749,7 @@ public void testMessyElectionsStillMakeClusterGoGreen() throws Exception { ensureGreen("test"); } - public void testSingleShardStoreFetch() throws ExecutionException, InterruptedException { + public void testSingleShardStoreFetchUsingBatchAction() throws ExecutionException, InterruptedException { String indexName = "test"; DiscoveryNode[] nodes = getDiscoveryNodes(); TransportNodesListShardStoreMetadataBatch.NodesStoreFilesMetadataBatch response = prepareAndSendRequest( @@ -765,8 +765,7 @@ public void testSingleShardStoreFetch() throws ExecutionException, InterruptedEx assertNodeStoreFilesMetadataSuccessCase(nodeStoreFilesMetadata, shardId); } - public void testShardStoreFetchMultiNodeMultiIndexes() throws Exception { - // start second node + public void testShardStoreFetchMultiNodeMultiIndexesUsingBatchAction() throws Exception { internalCluster().startNode(); String indexName1 = "test1"; String indexName2 = "test2"; @@ -790,7 +789,7 @@ public void testShardStoreFetchMultiNodeMultiIndexes() throws Exception { } } - public void testShardStoreFetchNodeNotConnected() { + public void testShardStoreFetchNodeNotConnectedUsingBatchAction() { DiscoveryNode nonExistingNode = new DiscoveryNode("foo", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT); String indexName = "test"; TransportNodesListShardStoreMetadataBatch.NodesStoreFilesMetadataBatch response = prepareAndSendRequest( @@ -802,8 +801,7 @@ public void testShardStoreFetchNodeNotConnected() { assertEquals(nonExistingNode.getId(), response.failures().get(0).nodeId()); } - public void testShardStoreFetchCorruptedIndex() throws Exception { - // start second node + public void testShardStoreFetchCorruptedIndexUsingBatchAction() throws Exception { internalCluster().startNode(); String indexName = "test"; prepareIndices(new String[] { indexName }, 1, 1); From 5a328ca2c5051f1b0fb05a3f53724843bb5aec5b Mon Sep 17 00:00:00 2001 From: Shivansh Arora Date: Fri, 2 Feb 2024 16:32:12 +0530 Subject: [PATCH 12/22] Apply Spotless Signed-off-by: Shivansh Arora --- .../java/org/opensearch/gateway/RecoveryFromGatewayIT.java | 1 - 1 file changed, 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 809cda8b62843..427de9d7099bd 100644 --- a/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java @@ -825,7 +825,6 @@ public void testShardFetchCorruptedShardsUsingBatchAction() throws Exception { assertTrue(nodeGatewayStartedShards.primary()); } - public void testSingleShardStoreFetchUsingBatchAction() throws ExecutionException, InterruptedException { String indexName = "test"; DiscoveryNode[] nodes = getDiscoveryNodes(); From 5f5fbbabf9388c6c944020b0adcf4172e43c33ed Mon Sep 17 00:00:00 2001 From: Shivansh Arora Date: Mon, 5 Feb 2024 12:53:41 +0530 Subject: [PATCH 13/22] Fix tests Signed-off-by: Shivansh Arora --- .../java/org/opensearch/gateway/RecoveryFromGatewayIT.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java b/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java index 427de9d7099bd..0369f52e9c781 100644 --- a/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java @@ -842,7 +842,7 @@ public void testSingleShardStoreFetchUsingBatchAction() throws ExecutionExceptio } public void testShardStoreFetchMultiNodeMultiIndexesUsingBatchAction() throws Exception { - internalCluster().startNode(); + internalCluster().startNodes(2); String indexName1 = "test1"; String indexName2 = "test2"; DiscoveryNode[] nodes = getDiscoveryNodes(); @@ -878,7 +878,7 @@ public void testShardStoreFetchNodeNotConnectedUsingBatchAction() { } public void testShardStoreFetchCorruptedIndexUsingBatchAction() throws Exception { - internalCluster().startNode(); + internalCluster().startNodes(2); String indexName = "test"; prepareIndices(new String[] { indexName }, 1, 1); Map shardAttributesMap = prepareRequestMap(new String[] { indexName }, 1); From 57960f96c16d407d91aa125ba2b88bd12b3db8d5 Mon Sep 17 00:00:00 2001 From: Shivansh Arora Date: Tue, 27 Feb 2024 15:20:56 +0530 Subject: [PATCH 14/22] Catch OpenSearchException as well during the batch flow Signed-off-by: Shivansh Arora --- .../store/TransportNodesListShardStoreMetadataBatch.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/main/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadataBatch.java b/server/src/main/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadataBatch.java index b840d4560f05e..a256274cd1ccc 100644 --- a/server/src/main/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadataBatch.java +++ b/server/src/main/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadataBatch.java @@ -151,7 +151,7 @@ private Map listStoreMetadata(NodeRequest reque clusterService ); shardStoreMetadataMap.put(shardId, new NodeStoreFilesMetadata(storeFilesMetadata, null)); - } catch (IOException e) { + } catch (IOException | OpenSearchException e) { shardStoreMetadataMap.put( shardId, new NodeStoreFilesMetadata(new StoreFilesMetadata(shardId, Store.MetadataSnapshot.EMPTY, Collections.emptyList()), e) From 15af61448a6665f762637b383d53dd2769b6fdbe Mon Sep 17 00:00:00 2001 From: Shivansh Arora Date: Wed, 13 Mar 2024 01:57:42 +0530 Subject: [PATCH 15/22] Address comments Signed-off-by: Shivansh Arora --- .../store/TransportNodesListShardStoreMetadata.java | 4 ++-- .../TransportNodesListShardStoreMetadataBatch.java | 12 ++++++++++-- .../TransportNodesListShardStoreMetadataHelper.java | 2 +- 3 files changed, 13 insertions(+), 5 deletions(-) diff --git a/server/src/main/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadata.java b/server/src/main/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadata.java index 7bf64926cdd77..7d78fae762330 100644 --- a/server/src/main/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadata.java +++ b/server/src/main/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadata.java @@ -65,7 +65,7 @@ import java.util.Map; import java.util.Objects; -import static org.opensearch.indices.store.TransportNodesListShardStoreMetadataHelper.getListShardMetadataOnLocalNode; +import static org.opensearch.indices.store.TransportNodesListShardStoreMetadataHelper.listShardMetadataInternal; /** * Metadata for shard stores from a list of transport nodes @@ -158,7 +158,7 @@ protected NodeStoreFilesMetadata nodeOperation(NodeRequest request) { private StoreFilesMetadata listStoreMetadata(NodeRequest request) throws IOException { final ShardId shardId = request.getShardId(); try { - return getListShardMetadataOnLocalNode( + return listShardMetadataInternal( logger, shardId, nodeEnv, diff --git a/server/src/main/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadataBatch.java b/server/src/main/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadataBatch.java index a256274cd1ccc..9c9db7118c4bc 100644 --- a/server/src/main/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadataBatch.java +++ b/server/src/main/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadataBatch.java @@ -8,6 +8,7 @@ package org.opensearch.indices.store; +import org.apache.logging.log4j.message.ParameterizedMessage; import org.opensearch.OpenSearchException; import org.opensearch.action.ActionType; import org.opensearch.action.FailedNodeException; @@ -141,7 +142,7 @@ private Map listStoreMetadata(NodeRequest reque for (ShardAttributes shardAttributes : request.getShardAttributes().values()) { final ShardId shardId = shardAttributes.getShardId(); try { - StoreFilesMetadata storeFilesMetadata = TransportNodesListShardStoreMetadataHelper.getListShardMetadataOnLocalNode( + StoreFilesMetadata storeFilesMetadata = TransportNodesListShardStoreMetadataHelper.listShardMetadataInternal( logger, shardId, nodeEnv, @@ -151,7 +152,14 @@ private Map listStoreMetadata(NodeRequest reque clusterService ); shardStoreMetadataMap.put(shardId, new NodeStoreFilesMetadata(storeFilesMetadata, null)); - } catch (IOException | OpenSearchException e) { + } catch (Exception e) { + logger.debug( + new ParameterizedMessage( + "Faced following exception while trying to get Shard Metadata for {}: ", + shardId.toString() + ), + e + ); shardStoreMetadataMap.put( shardId, new NodeStoreFilesMetadata(new StoreFilesMetadata(shardId, Store.MetadataSnapshot.EMPTY, Collections.emptyList()), e) diff --git a/server/src/main/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadataHelper.java b/server/src/main/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadataHelper.java index e1b72774e8fe1..808d0dd61ebf6 100644 --- a/server/src/main/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadataHelper.java +++ b/server/src/main/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadataHelper.java @@ -48,7 +48,7 @@ * @opensearch.internal */ public class TransportNodesListShardStoreMetadataHelper { - public static StoreFilesMetadata getListShardMetadataOnLocalNode( + public static StoreFilesMetadata listShardMetadataInternal( Logger logger, final ShardId shardId, NodeEnvironment nodeEnv, From d0d46ab1c1d9c18856d93506140a9cda696160cc Mon Sep 17 00:00:00 2001 From: Shivansh Arora Date: Wed, 13 Mar 2024 02:31:15 +0530 Subject: [PATCH 16/22] Apply spotless Signed-off-by: Shivansh Arora --- .../store/TransportNodesListShardStoreMetadataBatch.java | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/server/src/main/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadataBatch.java b/server/src/main/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadataBatch.java index 9c9db7118c4bc..ce6a8f20dfa8f 100644 --- a/server/src/main/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadataBatch.java +++ b/server/src/main/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadataBatch.java @@ -154,10 +154,7 @@ private Map listStoreMetadata(NodeRequest reque shardStoreMetadataMap.put(shardId, new NodeStoreFilesMetadata(storeFilesMetadata, null)); } catch (Exception e) { logger.debug( - new ParameterizedMessage( - "Faced following exception while trying to get Shard Metadata for {}: ", - shardId.toString() - ), + new ParameterizedMessage("Faced following exception while trying to get Shard Metadata for {}: ", shardId.toString()), e ); shardStoreMetadataMap.put( From f05dfc28bc669e7dd6832c24ad17461c84796798 Mon Sep 17 00:00:00 2001 From: Aman Khare Date: Wed, 13 Mar 2024 12:46:45 +0530 Subject: [PATCH 17/22] Handle index not found properly and return null in response Signed-off-by: Aman Khare --- .../opensearch/gateway/BaseShardResponse.java | 51 +++++++++++++++ .../TransportNodesListShardStoreMetadata.java | 16 +---- ...sportNodesListShardStoreMetadataBatch.java | 64 +++++++++---------- ...portNodesListShardStoreMetadataHelper.java | 9 ++- 4 files changed, 89 insertions(+), 51 deletions(-) create mode 100644 server/src/main/java/org/opensearch/gateway/BaseShardResponse.java diff --git a/server/src/main/java/org/opensearch/gateway/BaseShardResponse.java b/server/src/main/java/org/opensearch/gateway/BaseShardResponse.java new file mode 100644 index 0000000000000..7ef768e62880a --- /dev/null +++ b/server/src/main/java/org/opensearch/gateway/BaseShardResponse.java @@ -0,0 +1,51 @@ +/* + * 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.core.common.io.stream.StreamInput; +import org.opensearch.core.common.io.stream.StreamOutput; + +import java.io.IOException; + +/** + * Base response class for shard response. + * + * @opensearch.internal + */ +public abstract class BaseShardResponse { + + private final Exception storeException; + + public BaseShardResponse(Exception storeException) { + this.storeException = storeException; + } + + public abstract boolean isEmpty(); + + public Exception getException() { + return storeException; + } + + public BaseShardResponse(StreamInput in) throws IOException { + if (in.readBoolean()) { + storeException = in.readException(); + } else { + storeException = null; + } + } + + public void writeTo(StreamOutput out) throws IOException { + if (storeException != null) { + out.writeBoolean(true); + out.writeException(storeException); + } else { + out.writeBoolean(false); + } + } +} diff --git a/server/src/main/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadata.java b/server/src/main/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadata.java index 7d78fae762330..eeee5d8a409aa 100644 --- a/server/src/main/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadata.java +++ b/server/src/main/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadata.java @@ -52,7 +52,6 @@ import org.opensearch.core.index.shard.ShardId; import org.opensearch.env.NodeEnvironment; import org.opensearch.gateway.AsyncShardFetch; -import org.opensearch.index.store.Store; import org.opensearch.indices.IndicesService; import org.opensearch.indices.store.TransportNodesListShardStoreMetadataHelper.StoreFilesMetadata; import org.opensearch.threadpool.ThreadPool; @@ -60,7 +59,6 @@ import org.opensearch.transport.TransportService; import java.io.IOException; -import java.util.Collections; import java.util.List; import java.util.Map; import java.util.Objects; @@ -157,19 +155,7 @@ protected NodeStoreFilesMetadata nodeOperation(NodeRequest request) { private StoreFilesMetadata listStoreMetadata(NodeRequest request) throws IOException { final ShardId shardId = request.getShardId(); - try { - return listShardMetadataInternal( - logger, - shardId, - nodeEnv, - indicesService, - request.getCustomDataPath(), - settings, - clusterService - ); - } catch (IOException e) { - return new StoreFilesMetadata(shardId, Store.MetadataSnapshot.EMPTY, Collections.emptyList()); - } + return listShardMetadataInternal(logger, shardId, nodeEnv, indicesService, request.getCustomDataPath(), settings, clusterService); } /** diff --git a/server/src/main/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadataBatch.java b/server/src/main/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadataBatch.java index ce6a8f20dfa8f..8fe277a61205a 100644 --- a/server/src/main/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadataBatch.java +++ b/server/src/main/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadataBatch.java @@ -8,7 +8,6 @@ package org.opensearch.indices.store; -import org.apache.logging.log4j.message.ParameterizedMessage; import org.opensearch.OpenSearchException; import org.opensearch.action.ActionType; import org.opensearch.action.FailedNodeException; @@ -28,6 +27,7 @@ import org.opensearch.core.index.shard.ShardId; import org.opensearch.env.NodeEnvironment; import org.opensearch.gateway.AsyncShardFetch; +import org.opensearch.gateway.BaseShardResponse; import org.opensearch.index.store.Store; import org.opensearch.indices.IndicesService; import org.opensearch.indices.store.TransportNodesListShardStoreMetadataHelper.StoreFilesMetadata; @@ -42,6 +42,8 @@ import java.util.Map; import java.util.Objects; +import static org.opensearch.indices.store.TransportNodesListShardStoreMetadataHelper.INDEX_NOT_FOUND; + /** * Transport action for fetching the batch of shard stores Metadata from a list of transport nodes * @@ -139,28 +141,33 @@ protected NodeStoreFilesMetadataBatch nodeOperation(NodeRequest request) { */ private Map listStoreMetadata(NodeRequest request) throws IOException { Map shardStoreMetadataMap = new HashMap(); - for (ShardAttributes shardAttributes : request.getShardAttributes().values()) { - final ShardId shardId = shardAttributes.getShardId(); + for (Map.Entry shardAttributes : request.getShardAttributes().entrySet()) { + final ShardId shardId = shardAttributes.getKey(); try { StoreFilesMetadata storeFilesMetadata = TransportNodesListShardStoreMetadataHelper.listShardMetadataInternal( logger, shardId, nodeEnv, indicesService, - shardAttributes.getCustomDataPath(), + shardAttributes.getValue().getCustomDataPath(), settings, clusterService ); shardStoreMetadataMap.put(shardId, new NodeStoreFilesMetadata(storeFilesMetadata, null)); - } catch (Exception e) { - logger.debug( - new ParameterizedMessage("Faced following exception while trying to get Shard Metadata for {}: ", shardId.toString()), - e - ); - shardStoreMetadataMap.put( - shardId, - new NodeStoreFilesMetadata(new StoreFilesMetadata(shardId, Store.MetadataSnapshot.EMPTY, Collections.emptyList()), e) - ); + } catch (IOException e) { + // should return null in case of known exceptions being returned from listShardMetadataInternal method. + if (e.getMessage().contains(INDEX_NOT_FOUND)) { + shardStoreMetadataMap.put(shardId, null); + } else { + // return actual exception as it is for unknown exceptions + shardStoreMetadataMap.put( + shardId, + new NodeStoreFilesMetadata( + new StoreFilesMetadata(shardId, Store.MetadataSnapshot.EMPTY, Collections.emptyList()), + e + ) + ); + } } } return shardStoreMetadataMap; @@ -232,52 +239,43 @@ protected void writeNodesTo(StreamOutput out, List * * @opensearch.internal */ - public static class NodeStoreFilesMetadata { + public static class NodeStoreFilesMetadata extends BaseShardResponse { private StoreFilesMetadata storeFilesMetadata; - private Exception storeFileFetchException; - public NodeStoreFilesMetadata(StoreFilesMetadata storeFilesMetadata) { - this.storeFilesMetadata = storeFilesMetadata; - this.storeFileFetchException = null; + @Override + public boolean isEmpty() { + return storeFilesMetadata == null || storeFilesMetadata().isEmpty() && getException() == null; } public NodeStoreFilesMetadata(StreamInput in) throws IOException { - storeFilesMetadata = new StoreFilesMetadata(in); + super(in); if (in.readBoolean()) { - this.storeFileFetchException = in.readException(); + this.storeFilesMetadata = new StoreFilesMetadata(in); } else { - this.storeFileFetchException = null; + this.storeFilesMetadata = null; } } public NodeStoreFilesMetadata(StoreFilesMetadata storeFilesMetadata, Exception storeFileFetchException) { + super(storeFileFetchException); this.storeFilesMetadata = storeFilesMetadata; - this.storeFileFetchException = storeFileFetchException; } public StoreFilesMetadata storeFilesMetadata() { return storeFilesMetadata; } - public static NodeStoreFilesMetadata readListShardStoreNodeOperationResponse(StreamInput in) throws IOException { - return new NodeStoreFilesMetadata(in); - } - public void writeTo(StreamOutput out) throws IOException { - storeFilesMetadata.writeTo(out); - if (storeFileFetchException != null) { + super.writeTo(out); + if (storeFilesMetadata != null) { out.writeBoolean(true); - out.writeException(storeFileFetchException); + storeFilesMetadata.writeTo(out); } else { out.writeBoolean(false); } } - public Exception getStoreFileFetchException() { - return storeFileFetchException; - } - @Override public String toString() { return "[[" + storeFilesMetadata + "]]"; diff --git a/server/src/main/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadataHelper.java b/server/src/main/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadataHelper.java index 808d0dd61ebf6..74b04d6c6d494 100644 --- a/server/src/main/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadataHelper.java +++ b/server/src/main/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadataHelper.java @@ -48,6 +48,9 @@ * @opensearch.internal */ public class TransportNodesListShardStoreMetadataHelper { + + public static final String INDEX_NOT_FOUND = "node doesn't have meta data for index "; + public static StoreFilesMetadata listShardMetadataInternal( Logger logger, final ShardId shardId, @@ -75,10 +78,10 @@ public static StoreFilesMetadata listShardMetadataInternal( return storeFilesMetadata; } catch (org.apache.lucene.index.IndexNotFoundException e) { logger.trace(new ParameterizedMessage("[{}] node is missing index, responding with empty", shardId), e); - throw e; + return new StoreFilesMetadata(shardId, Store.MetadataSnapshot.EMPTY, Collections.emptyList()); } catch (IOException e) { logger.warn(new ParameterizedMessage("[{}] can't read metadata from store, responding with empty", shardId), e); - throw e; + return new StoreFilesMetadata(shardId, Store.MetadataSnapshot.EMPTY, Collections.emptyList()); } } } @@ -93,7 +96,7 @@ public static StoreFilesMetadata listShardMetadataInternal( 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()); + throw new OpenSearchException(INDEX_NOT_FOUND + shardId.getIndex()); } } } From 143c3d42dbe50d9977d329fd6aad3b2615d5a8e7 Mon Sep 17 00:00:00 2001 From: Shivansh Arora Date: Wed, 13 Mar 2024 12:52:25 +0530 Subject: [PATCH 18/22] Catch all exceptions in Batch mode Signed-off-by: Shivansh Arora --- .../store/TransportNodesListShardStoreMetadataBatch.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/main/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadataBatch.java b/server/src/main/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadataBatch.java index 8fe277a61205a..b99142f0cf326 100644 --- a/server/src/main/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadataBatch.java +++ b/server/src/main/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadataBatch.java @@ -154,7 +154,7 @@ private Map listStoreMetadata(NodeRequest reque clusterService ); shardStoreMetadataMap.put(shardId, new NodeStoreFilesMetadata(storeFilesMetadata, null)); - } catch (IOException e) { + } catch (Exception e) { // should return null in case of known exceptions being returned from listShardMetadataInternal method. if (e.getMessage().contains(INDEX_NOT_FOUND)) { shardStoreMetadataMap.put(shardId, null); From 29871bfb9a510ea90fa83b62377448d28e93daaa Mon Sep 17 00:00:00 2001 From: Shivansh Arora Date: Wed, 13 Mar 2024 13:42:36 +0530 Subject: [PATCH 19/22] Fix Integ Test Signed-off-by: Shivansh Arora --- .../java/org/opensearch/gateway/RecoveryFromGatewayIT.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java b/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java index 0369f52e9c781..b1254395dcae6 100644 --- a/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java @@ -934,7 +934,7 @@ private void assertNodeStoreFilesMetadataFailureCase( TransportNodesListShardStoreMetadataBatch.NodeStoreFilesMetadata nodeStoreFilesMetadata, ShardId shardId ) { - assertNotNull(nodeStoreFilesMetadata.getStoreFileFetchException()); + assertNotNull(nodeStoreFilesMetadata.getException()); TransportNodesListShardStoreMetadataHelper.StoreFilesMetadata storeFileMetadata = nodeStoreFilesMetadata.storeFilesMetadata(); assertEquals(shardId, storeFileMetadata.shardId()); assertTrue(storeFileMetadata.peerRecoveryRetentionLeases().isEmpty()); @@ -944,7 +944,7 @@ private void assertNodeStoreFilesMetadataSuccessCase( TransportNodesListShardStoreMetadataBatch.NodeStoreFilesMetadata nodeStoreFilesMetadata, ShardId shardId ) { - assertNull(nodeStoreFilesMetadata.getStoreFileFetchException()); + assertNull(nodeStoreFilesMetadata.getException()); TransportNodesListShardStoreMetadataHelper.StoreFilesMetadata storeFileMetadata = nodeStoreFilesMetadata.storeFilesMetadata(); assertFalse(storeFileMetadata.isEmpty()); assertEquals(shardId, storeFileMetadata.shardId()); From 85facccdeb9a4cfc295d9405109651053923d28f Mon Sep 17 00:00:00 2001 From: Shivansh Arora Date: Wed, 13 Mar 2024 16:31:20 +0530 Subject: [PATCH 20/22] Update Integration test Signed-off-by: Shivansh Arora --- .../gateway/RecoveryFromGatewayIT.java | 43 +++++++++---------- 1 file changed, 21 insertions(+), 22 deletions(-) diff --git a/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java b/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java index b1254395dcae6..311382ea1e2a9 100644 --- a/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java @@ -879,15 +879,18 @@ public void testShardStoreFetchNodeNotConnectedUsingBatchAction() { public void testShardStoreFetchCorruptedIndexUsingBatchAction() throws Exception { internalCluster().startNodes(2); - String indexName = "test"; - prepareIndices(new String[] { indexName }, 1, 1); - Map shardAttributesMap = prepareRequestMap(new String[] { indexName }, 1); - Index index = resolveIndex(indexName); - ShardId shardId = new ShardId(index, 0); - ClusterSearchShardsResponse searchShardsResponse = client().admin().cluster().prepareSearchShards(indexName).get(); + String index1Name = "test1"; + String index2Name = "test2"; + prepareIndices(new String[] { index1Name, index2Name }, 1, 1); + Map shardAttributesMap = prepareRequestMap(new String[] { index1Name, index2Name }, 1); + Index index1 = resolveIndex(index1Name); + ShardId shardId1 = new ShardId(index1, 0); + ClusterSearchShardsResponse searchShardsResponse = client().admin().cluster().prepareSearchShards(index1Name).get(); assertEquals(2, searchShardsResponse.getNodes().length); - corruptShard(searchShardsResponse.getNodes()[0].getName(), shardId); - corruptShard(searchShardsResponse.getNodes()[1].getName(), shardId); + + // corrupt test1 index shards + corruptShard(searchShardsResponse.getNodes()[0].getName(), shardId1); + corruptShard(searchShardsResponse.getNodes()[1].getName(), shardId1); ClusterRerouteResponse clusterRerouteResponse = client().admin().cluster().prepareReroute().setRetryFailed(false).get(); DiscoveryNode[] discoveryNodes = getDiscoveryNodes(); TransportNodesListShardStoreMetadataBatch.NodesStoreFilesMetadataBatch response; @@ -895,11 +898,17 @@ public void testShardStoreFetchCorruptedIndexUsingBatchAction() throws Exception internalCluster().getInstance(TransportNodesListShardStoreMetadataBatch.class), new TransportNodesListShardStoreMetadataBatch.Request(shardAttributesMap, discoveryNodes) ); - TransportNodesListShardStoreMetadataBatch.NodeStoreFilesMetadata nodeStoreFilesMetadata = response.getNodesMap() + Map nodeStoreFilesMetadata = response.getNodesMap() .get(discoveryNodes[0].getId()) - .getNodeStoreFilesMetadataBatch() - .get(shardId); - assertNodeStoreFilesMetadataFailureCase(nodeStoreFilesMetadata, shardId); + .getNodeStoreFilesMetadataBatch(); + // We don't store exception in case of corrupt index, rather just return an empty response + assertNull(nodeStoreFilesMetadata.get(shardId1).getException()); + assertEquals(shardId1, nodeStoreFilesMetadata.get(shardId1).storeFilesMetadata().shardId()); + assertTrue(nodeStoreFilesMetadata.get(shardId1).storeFilesMetadata().isEmpty()); + + Index index2 = resolveIndex(index2Name); + ShardId shardId2 = new ShardId(index2, 0); + assertNodeStoreFilesMetadataSuccessCase(nodeStoreFilesMetadata.get(shardId2), shardId2); } private void prepareIndices(String[] indices, int numberOfPrimaryShards, int numberOfReplicaShards) { @@ -930,16 +939,6 @@ private TransportNodesListShardStoreMetadataBatch.NodesStoreFilesMetadataBatch p ); } - private void assertNodeStoreFilesMetadataFailureCase( - TransportNodesListShardStoreMetadataBatch.NodeStoreFilesMetadata nodeStoreFilesMetadata, - ShardId shardId - ) { - assertNotNull(nodeStoreFilesMetadata.getException()); - TransportNodesListShardStoreMetadataHelper.StoreFilesMetadata storeFileMetadata = nodeStoreFilesMetadata.storeFilesMetadata(); - assertEquals(shardId, storeFileMetadata.shardId()); - assertTrue(storeFileMetadata.peerRecoveryRetentionLeases().isEmpty()); - } - private void assertNodeStoreFilesMetadataSuccessCase( TransportNodesListShardStoreMetadataBatch.NodeStoreFilesMetadata nodeStoreFilesMetadata, ShardId shardId From 776f625c9fb15f2cba60406131cc887da664a021 Mon Sep 17 00:00:00 2001 From: Aman Khare Date: Thu, 14 Mar 2024 13:38:52 +0530 Subject: [PATCH 21/22] Remove BaseShardResponse as it'll be pushed later Signed-off-by: Aman Khare --- .../opensearch/gateway/BaseShardResponse.java | 51 ------------------- ...sportNodesListShardStoreMetadataBatch.java | 28 +++++----- 2 files changed, 16 insertions(+), 63 deletions(-) delete mode 100644 server/src/main/java/org/opensearch/gateway/BaseShardResponse.java diff --git a/server/src/main/java/org/opensearch/gateway/BaseShardResponse.java b/server/src/main/java/org/opensearch/gateway/BaseShardResponse.java deleted file mode 100644 index 7ef768e62880a..0000000000000 --- a/server/src/main/java/org/opensearch/gateway/BaseShardResponse.java +++ /dev/null @@ -1,51 +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.core.common.io.stream.StreamInput; -import org.opensearch.core.common.io.stream.StreamOutput; - -import java.io.IOException; - -/** - * Base response class for shard response. - * - * @opensearch.internal - */ -public abstract class BaseShardResponse { - - private final Exception storeException; - - public BaseShardResponse(Exception storeException) { - this.storeException = storeException; - } - - public abstract boolean isEmpty(); - - public Exception getException() { - return storeException; - } - - public BaseShardResponse(StreamInput in) throws IOException { - if (in.readBoolean()) { - storeException = in.readException(); - } else { - storeException = null; - } - } - - public void writeTo(StreamOutput out) throws IOException { - if (storeException != null) { - out.writeBoolean(true); - out.writeException(storeException); - } else { - out.writeBoolean(false); - } - } -} diff --git a/server/src/main/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadataBatch.java b/server/src/main/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadataBatch.java index b99142f0cf326..3f151fe1c5ca0 100644 --- a/server/src/main/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadataBatch.java +++ b/server/src/main/java/org/opensearch/indices/store/TransportNodesListShardStoreMetadataBatch.java @@ -27,7 +27,6 @@ import org.opensearch.core.index.shard.ShardId; import org.opensearch.env.NodeEnvironment; import org.opensearch.gateway.AsyncShardFetch; -import org.opensearch.gateway.BaseShardResponse; import org.opensearch.index.store.Store; import org.opensearch.indices.IndicesService; import org.opensearch.indices.store.TransportNodesListShardStoreMetadataHelper.StoreFilesMetadata; @@ -239,27 +238,28 @@ protected void writeNodesTo(StreamOutput out, List * * @opensearch.internal */ - public static class NodeStoreFilesMetadata extends BaseShardResponse { + public static class NodeStoreFilesMetadata { private StoreFilesMetadata storeFilesMetadata; + private Exception storeFileFetchException; - @Override - public boolean isEmpty() { - return storeFilesMetadata == null || storeFilesMetadata().isEmpty() && getException() == null; + public NodeStoreFilesMetadata(StoreFilesMetadata storeFilesMetadata) { + this.storeFilesMetadata = storeFilesMetadata; + this.storeFileFetchException = null; } public NodeStoreFilesMetadata(StreamInput in) throws IOException { - super(in); + storeFilesMetadata = new StoreFilesMetadata(in); if (in.readBoolean()) { - this.storeFilesMetadata = new StoreFilesMetadata(in); + this.storeFileFetchException = in.readException(); } else { - this.storeFilesMetadata = null; + this.storeFileFetchException = null; } } public NodeStoreFilesMetadata(StoreFilesMetadata storeFilesMetadata, Exception storeFileFetchException) { - super(storeFileFetchException); this.storeFilesMetadata = storeFilesMetadata; + this.storeFileFetchException = storeFileFetchException; } public StoreFilesMetadata storeFilesMetadata() { @@ -267,15 +267,19 @@ public StoreFilesMetadata storeFilesMetadata() { } public void writeTo(StreamOutput out) throws IOException { - super.writeTo(out); - if (storeFilesMetadata != null) { + storeFilesMetadata.writeTo(out); + if (storeFileFetchException != null) { out.writeBoolean(true); - storeFilesMetadata.writeTo(out); + out.writeException(storeFileFetchException); } else { out.writeBoolean(false); } } + public Exception getStoreFileFetchException() { + return storeFileFetchException; + } + @Override public String toString() { return "[[" + storeFilesMetadata + "]]"; From 2cc687f2e420758f1da43775d75dc7e6edc3b3a6 Mon Sep 17 00:00:00 2001 From: Aman Khare Date: Thu, 14 Mar 2024 13:49:02 +0530 Subject: [PATCH 22/22] Modify integ test accordingly Signed-off-by: Aman Khare --- .../java/org/opensearch/gateway/RecoveryFromGatewayIT.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java b/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java index 311382ea1e2a9..6c248a32c9928 100644 --- a/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java @@ -902,7 +902,7 @@ public void testShardStoreFetchCorruptedIndexUsingBatchAction() throws Exception .get(discoveryNodes[0].getId()) .getNodeStoreFilesMetadataBatch(); // We don't store exception in case of corrupt index, rather just return an empty response - assertNull(nodeStoreFilesMetadata.get(shardId1).getException()); + assertNull(nodeStoreFilesMetadata.get(shardId1).getStoreFileFetchException()); assertEquals(shardId1, nodeStoreFilesMetadata.get(shardId1).storeFilesMetadata().shardId()); assertTrue(nodeStoreFilesMetadata.get(shardId1).storeFilesMetadata().isEmpty()); @@ -943,7 +943,7 @@ private void assertNodeStoreFilesMetadataSuccessCase( TransportNodesListShardStoreMetadataBatch.NodeStoreFilesMetadata nodeStoreFilesMetadata, ShardId shardId ) { - assertNull(nodeStoreFilesMetadata.getException()); + assertNull(nodeStoreFilesMetadata.getStoreFileFetchException()); TransportNodesListShardStoreMetadataHelper.StoreFilesMetadata storeFileMetadata = nodeStoreFilesMetadata.storeFilesMetadata(); assertFalse(storeFileMetadata.isEmpty()); assertEquals(shardId, storeFileMetadata.shardId());