From 5e2034c684f51f73199d81294e1c420a1673672e Mon Sep 17 00:00:00 2001 From: Marc Handalian Date: Mon, 18 Mar 2024 16:36:20 -0700 Subject: [PATCH 01/38] Fix flaky test SegmentReplicationWithNodeToNodeIndexShardTests#testReplicaClosesWhileReplicating_AfterGetCheckpoint (#12695) This fixes a race condition in the test where the primary shard will still have an open file ref while shutting down. This happens because we are fetching file refs inside the resolveCheckpointInfoResponseListener method right after calling beforeIndexShardClosed. BeforeIndexShardClosed will resolve replication listeners immediately and leave a possibility of the primary attempting shut down before those refs are closed. We can resolve this using latches, but this test really doesn't need to simulate a primary response at all so removed it entirely. Signed-off-by: Marc Handalian --- .../shard/SegmentReplicationWithNodeToNodeIndexShardTests.java | 3 --- 1 file changed, 3 deletions(-) diff --git a/server/src/test/java/org/opensearch/index/shard/SegmentReplicationWithNodeToNodeIndexShardTests.java b/server/src/test/java/org/opensearch/index/shard/SegmentReplicationWithNodeToNodeIndexShardTests.java index f0950fe5392de..e541e988f3920 100644 --- a/server/src/test/java/org/opensearch/index/shard/SegmentReplicationWithNodeToNodeIndexShardTests.java +++ b/server/src/test/java/org/opensearch/index/shard/SegmentReplicationWithNodeToNodeIndexShardTests.java @@ -110,7 +110,6 @@ public void testReplicaClosesWhileReplicating_AfterGetCheckpoint() throws Except IndexShard primary = shards.getPrimary(); final IndexShard replica = shards.getReplicas().get(0); - final int numDocs = shards.indexDocs(randomInt(10)); primary.refresh("Test"); final SegmentReplicationSourceFactory sourceFactory = mock(SegmentReplicationSourceFactory.class); @@ -124,7 +123,6 @@ public void getCheckpointMetadata( ) { // trigger a cancellation by closing the replica. targetService.beforeIndexShardClosed(replica.shardId, replica, Settings.EMPTY); - resolveCheckpointInfoResponseListener(listener, primary); } @Override @@ -141,7 +139,6 @@ public void getSegmentFiles( }; when(sourceFactory.get(any())).thenReturn(source); startReplicationAndAssertCancellation(replica, primary, targetService); - shards.removeReplica(replica); closeShards(replica); } From a499d1eb1a0683bd93c170c6a6e90be2a1690f65 Mon Sep 17 00:00:00 2001 From: Aman Khare <85096200+amkhar@users.noreply.github.com> Date: Tue, 19 Mar 2024 09:25:55 +0530 Subject: [PATCH 02/38] Add PrimaryShardBatchAllocator to take allocation decisions for a batch of shards (#8916) * Add PrimaryShardBatchAllocator to take allocation decisions for a batch of shards Signed-off-by: Aman Khare --- .../gateway/RecoveryFromGatewayIT.java | 36 +- .../TransportIndicesShardStoresAction.java | 7 +- .../gateway/PrimaryShardAllocator.java | 72 ++-- .../gateway/PrimaryShardBatchAllocator.java | 150 ++++++++ ...ansportNodesGatewayStartedShardHelper.java | 166 ++++++++- ...ransportNodesListGatewayStartedShards.java | 97 ++--- ...ortNodesListGatewayStartedShardsBatch.java | 134 +------ .../gateway/PrimaryShardAllocatorTests.java | 10 +- .../PrimaryShardBatchAllocatorTests.java | 340 ++++++++++++++++++ .../test/gateway/TestGatewayAllocator.java | 10 +- 10 files changed, 754 insertions(+), 268 deletions(-) create mode 100644 server/src/main/java/org/opensearch/gateway/PrimaryShardBatchAllocator.java create mode 100644 server/src/test/java/org/opensearch/gateway/PrimaryShardBatchAllocatorTests.java diff --git a/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java b/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java index 6c248a32c9928..ba03532a9aa2f 100644 --- a/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java @@ -56,6 +56,7 @@ import org.opensearch.core.index.Index; import org.opensearch.core.index.shard.ShardId; import org.opensearch.env.NodeEnvironment; +import org.opensearch.gateway.TransportNodesGatewayStartedShardHelper.GatewayStartedShard; import org.opensearch.index.IndexService; import org.opensearch.index.IndexSettings; import org.opensearch.index.MergePolicyProvider; @@ -720,11 +721,11 @@ public Settings onNodeStopped(String nodeName) throws Exception { ); assertThat(response.getNodes(), hasSize(1)); - assertThat(response.getNodes().get(0).allocationId(), notNullValue()); + assertThat(response.getNodes().get(0).getGatewayShardStarted().allocationId(), notNullValue()); if (corrupt) { - assertThat(response.getNodes().get(0).storeException(), notNullValue()); + assertThat(response.getNodes().get(0).getGatewayShardStarted().storeException(), notNullValue()); } else { - assertThat(response.getNodes().get(0).storeException(), nullValue()); + assertThat(response.getNodes().get(0).getGatewayShardStarted().storeException(), nullValue()); } // start another node so cluster consistency checks won't time out due to the lack of state @@ -764,11 +765,11 @@ public void testSingleShardFetchUsingBatchAction() { ); final Index index = resolveIndex(indexName); final ShardId shardId = new ShardId(index, 0); - TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShard nodeGatewayStartedShards = response.getNodesMap() + GatewayStartedShard gatewayStartedShard = response.getNodesMap() .get(searchShardsResponse.getNodes()[0].getId()) .getNodeGatewayStartedShardsBatch() .get(shardId); - assertNodeGatewayStartedShardsHappyCase(nodeGatewayStartedShards); + assertNodeGatewayStartedShardsHappyCase(gatewayStartedShard); } public void testShardFetchMultiNodeMultiIndexesUsingBatchAction() { @@ -792,11 +793,8 @@ public void testShardFetchMultiNodeMultiIndexesUsingBatchAction() { ShardId shardId = clusterSearchShardsGroup.getShardId(); assertEquals(1, clusterSearchShardsGroup.getShards().length); String nodeId = clusterSearchShardsGroup.getShards()[0].currentNodeId(); - TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShard nodeGatewayStartedShards = response.getNodesMap() - .get(nodeId) - .getNodeGatewayStartedShardsBatch() - .get(shardId); - assertNodeGatewayStartedShardsHappyCase(nodeGatewayStartedShards); + GatewayStartedShard gatewayStartedShard = response.getNodesMap().get(nodeId).getNodeGatewayStartedShardsBatch().get(shardId); + assertNodeGatewayStartedShardsHappyCase(gatewayStartedShard); } } @@ -816,13 +814,13 @@ public void testShardFetchCorruptedShardsUsingBatchAction() throws Exception { new TransportNodesListGatewayStartedShardsBatch.Request(getDiscoveryNodes(), shardIdShardAttributesMap) ); DiscoveryNode[] discoveryNodes = getDiscoveryNodes(); - TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShard nodeGatewayStartedShards = response.getNodesMap() + GatewayStartedShard gatewayStartedShard = response.getNodesMap() .get(discoveryNodes[0].getId()) .getNodeGatewayStartedShardsBatch() .get(shardId); - assertNotNull(nodeGatewayStartedShards.storeException()); - assertNotNull(nodeGatewayStartedShards.allocationId()); - assertTrue(nodeGatewayStartedShards.primary()); + assertNotNull(gatewayStartedShard.storeException()); + assertNotNull(gatewayStartedShard.allocationId()); + assertTrue(gatewayStartedShard.primary()); } public void testSingleShardStoreFetchUsingBatchAction() throws ExecutionException, InterruptedException { @@ -950,12 +948,10 @@ private void assertNodeStoreFilesMetadataSuccessCase( assertNotNull(storeFileMetadata.peerRecoveryRetentionLeases()); } - private void assertNodeGatewayStartedShardsHappyCase( - TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShard nodeGatewayStartedShards - ) { - assertNull(nodeGatewayStartedShards.storeException()); - assertNotNull(nodeGatewayStartedShards.allocationId()); - assertTrue(nodeGatewayStartedShards.primary()); + private void assertNodeGatewayStartedShardsHappyCase(GatewayStartedShard gatewayStartedShard) { + assertNull(gatewayStartedShard.storeException()); + assertNotNull(gatewayStartedShard.allocationId()); + assertTrue(gatewayStartedShard.primary()); } private void prepareIndex(String indexName, int numberOfPrimaryShards) { diff --git a/server/src/main/java/org/opensearch/action/admin/indices/shards/TransportIndicesShardStoresAction.java b/server/src/main/java/org/opensearch/action/admin/indices/shards/TransportIndicesShardStoresAction.java index 04166c88a00ad..3fbf9ac1bb570 100644 --- a/server/src/main/java/org/opensearch/action/admin/indices/shards/TransportIndicesShardStoresAction.java +++ b/server/src/main/java/org/opensearch/action/admin/indices/shards/TransportIndicesShardStoresAction.java @@ -258,9 +258,9 @@ void finish() { storeStatuses.add( new IndicesShardStoresResponse.StoreStatus( response.getNode(), - response.allocationId(), + response.getGatewayShardStarted().allocationId(), allocationStatus, - response.storeException() + response.getGatewayShardStarted().storeException() ) ); } @@ -308,7 +308,8 @@ private IndicesShardStoresResponse.StoreStatus.AllocationStatus getAllocationSta * A shard exists/existed in a node only if shard state file exists in the node */ private boolean shardExistsInNode(final NodeGatewayStartedShards response) { - return response.storeException() != null || response.allocationId() != null; + return response.getGatewayShardStarted().storeException() != null + || response.getGatewayShardStarted().allocationId() != null; } @Override diff --git a/server/src/main/java/org/opensearch/gateway/PrimaryShardAllocator.java b/server/src/main/java/org/opensearch/gateway/PrimaryShardAllocator.java index 5046873830c01..f41545cbdf9bf 100644 --- a/server/src/main/java/org/opensearch/gateway/PrimaryShardAllocator.java +++ b/server/src/main/java/org/opensearch/gateway/PrimaryShardAllocator.java @@ -50,6 +50,7 @@ import org.opensearch.cluster.routing.allocation.decider.Decision.Type; import org.opensearch.env.ShardLockObtainFailedException; import org.opensearch.gateway.AsyncShardFetch.FetchResult; +import org.opensearch.gateway.TransportNodesGatewayStartedShardHelper.NodeGatewayStartedShard; import org.opensearch.gateway.TransportNodesListGatewayStartedShards.NodeGatewayStartedShards; import java.util.ArrayList; @@ -125,27 +126,37 @@ public AllocateUnassignedDecision makeAllocationDecision( return decision; } final FetchResult shardState = fetchData(unassignedShard, allocation); - List nodeShardStates = adaptToNodeStartedShardList(shardState); + List nodeShardStates = adaptToNodeStartedShardList(shardState); return getAllocationDecision(unassignedShard, allocation, nodeShardStates, logger); } /** - * Transforms {@link FetchResult} of {@link NodeGatewayStartedShards} to {@link List} of {@link NodeGatewayStartedShards} + * Transforms {@link FetchResult} of {@link NodeGatewayStartedShards} to {@link List} of {@link NodeGatewayStartedShard} * Returns null if {@link FetchResult} does not have any data. */ - private static List adaptToNodeStartedShardList(FetchResult shardsState) { + private static List adaptToNodeStartedShardList(FetchResult shardsState) { if (!shardsState.hasData()) { return null; } - List nodeShardStates = new ArrayList<>(); - shardsState.getData().forEach((node, nodeGatewayStartedShard) -> { nodeShardStates.add(nodeGatewayStartedShard); }); + List nodeShardStates = new ArrayList<>(); + shardsState.getData().forEach((node, nodeGatewayStartedShard) -> { + nodeShardStates.add( + new NodeGatewayStartedShard( + nodeGatewayStartedShard.getGatewayShardStarted().allocationId(), + nodeGatewayStartedShard.getGatewayShardStarted().primary(), + nodeGatewayStartedShard.getGatewayShardStarted().replicationCheckpoint(), + nodeGatewayStartedShard.getGatewayShardStarted().storeException(), + node + ) + ); + }); return nodeShardStates; } protected AllocateUnassignedDecision getAllocationDecision( ShardRouting unassignedShard, RoutingAllocation allocation, - List shardState, + List shardState, Logger logger ) { final boolean explain = allocation.debugDecision(); @@ -236,7 +247,7 @@ protected AllocateUnassignedDecision getAllocationDecision( nodesToAllocate = buildNodesToAllocate(allocation, nodeShardsResult.orderedAllocationCandidates, unassignedShard, true); if (nodesToAllocate.yesNodeShards.isEmpty() == false) { final DecidedNode decidedNode = nodesToAllocate.yesNodeShards.get(0); - final NodeGatewayStartedShards nodeShardState = decidedNode.nodeShardState; + final NodeGatewayStartedShard nodeShardState = decidedNode.nodeShardState; logger.debug( "[{}][{}]: allocating [{}] to [{}] on forced primary allocation", unassignedShard.index(), @@ -296,11 +307,11 @@ protected AllocateUnassignedDecision getAllocationDecision( */ private static List buildNodeDecisions( NodesToAllocate nodesToAllocate, - List fetchedShardData, + List fetchedShardData, Set inSyncAllocationIds ) { List nodeResults = new ArrayList<>(); - Collection ineligibleShards = new ArrayList<>(); + Collection ineligibleShards = new ArrayList<>(); if (nodesToAllocate != null) { final Set discoNodes = new HashSet<>(); nodeResults.addAll( @@ -334,21 +345,21 @@ private static List buildNodeDecisions( return nodeResults; } - private static ShardStoreInfo shardStoreInfo(NodeGatewayStartedShards nodeShardState, Set inSyncAllocationIds) { + private static ShardStoreInfo shardStoreInfo(NodeGatewayStartedShard nodeShardState, Set inSyncAllocationIds) { final Exception storeErr = nodeShardState.storeException(); final boolean inSync = nodeShardState.allocationId() != null && inSyncAllocationIds.contains(nodeShardState.allocationId()); return new ShardStoreInfo(nodeShardState.allocationId(), inSync, storeErr); } - private static final Comparator NO_STORE_EXCEPTION_FIRST_COMPARATOR = Comparator.comparing( - (NodeGatewayStartedShards state) -> state.storeException() == null + private static final Comparator NO_STORE_EXCEPTION_FIRST_COMPARATOR = Comparator.comparing( + (NodeGatewayStartedShard state) -> state.storeException() == null ).reversed(); - private static final Comparator PRIMARY_FIRST_COMPARATOR = Comparator.comparing( - NodeGatewayStartedShards::primary + private static final Comparator PRIMARY_FIRST_COMPARATOR = Comparator.comparing( + NodeGatewayStartedShard::primary ).reversed(); - private static final Comparator HIGHEST_REPLICATION_CHECKPOINT_FIRST_COMPARATOR = Comparator.comparing( - NodeGatewayStartedShards::replicationCheckpoint, + private static final Comparator HIGHEST_REPLICATION_CHECKPOINT_FIRST_COMPARATOR = Comparator.comparing( + NodeGatewayStartedShard::replicationCheckpoint, Comparator.nullsLast(Comparator.naturalOrder()) ); @@ -362,12 +373,12 @@ protected static NodeShardsResult buildNodeShardsResult( boolean matchAnyShard, Set ignoreNodes, Set inSyncAllocationIds, - List shardState, + List shardState, Logger logger ) { - List nodeShardStates = new ArrayList<>(); + List nodeShardStates = new ArrayList<>(); int numberOfAllocationsFound = 0; - for (NodeGatewayStartedShards nodeShardState : shardState) { + for (NodeGatewayStartedShard nodeShardState : shardState) { DiscoveryNode node = nodeShardState.getNode(); String allocationId = nodeShardState.allocationId(); @@ -432,21 +443,18 @@ protected static NodeShardsResult buildNodeShardsResult( return new NodeShardsResult(nodeShardStates, numberOfAllocationsFound); } - private static Comparator createActiveShardComparator( - boolean matchAnyShard, - Set inSyncAllocationIds - ) { + private static Comparator createActiveShardComparator(boolean matchAnyShard, Set inSyncAllocationIds) { /** * Orders the active shards copies based on below comparators * 1. No store exception i.e. shard copy is readable * 2. Prefer previous primary shard * 3. Prefer shard copy with the highest replication checkpoint. It is NO-OP for doc rep enabled indices. */ - final Comparator comparator; // allocation preference + final Comparator comparator; // allocation preference if (matchAnyShard) { // prefer shards with matching allocation ids - Comparator matchingAllocationsFirst = Comparator.comparing( - (NodeGatewayStartedShards state) -> inSyncAllocationIds.contains(state.allocationId()) + Comparator matchingAllocationsFirst = Comparator.comparing( + (NodeGatewayStartedShard state) -> inSyncAllocationIds.contains(state.allocationId()) ).reversed(); comparator = matchingAllocationsFirst.thenComparing(NO_STORE_EXCEPTION_FIRST_COMPARATOR) .thenComparing(PRIMARY_FIRST_COMPARATOR) @@ -464,14 +472,14 @@ private static Comparator createActiveShardComparator( */ private static NodesToAllocate buildNodesToAllocate( RoutingAllocation allocation, - List nodeShardStates, + List nodeShardStates, ShardRouting shardRouting, boolean forceAllocate ) { List yesNodeShards = new ArrayList<>(); List throttledNodeShards = new ArrayList<>(); List noNodeShards = new ArrayList<>(); - for (NodeGatewayStartedShards nodeShardState : nodeShardStates) { + for (NodeGatewayStartedShard nodeShardState : nodeShardStates) { RoutingNode node = allocation.routingNodes().node(nodeShardState.getNode().getId()); if (node == null) { continue; @@ -502,10 +510,10 @@ private static NodesToAllocate buildNodesToAllocate( * This class encapsulates the result of a call to {@link #buildNodeShardsResult} */ static class NodeShardsResult { - final List orderedAllocationCandidates; + final List orderedAllocationCandidates; final int allocationsFound; - NodeShardsResult(List orderedAllocationCandidates, int allocationsFound) { + NodeShardsResult(List orderedAllocationCandidates, int allocationsFound) { this.orderedAllocationCandidates = orderedAllocationCandidates; this.allocationsFound = allocationsFound; } @@ -531,10 +539,10 @@ protected static class NodesToAllocate { * by the allocator for allocating to the node that holds the shard copy. */ private static class DecidedNode { - final NodeGatewayStartedShards nodeShardState; + final NodeGatewayStartedShard nodeShardState; final Decision decision; - private DecidedNode(NodeGatewayStartedShards nodeShardState, Decision decision) { + private DecidedNode(NodeGatewayStartedShard nodeShardState, Decision decision) { this.nodeShardState = nodeShardState; this.decision = decision; } diff --git a/server/src/main/java/org/opensearch/gateway/PrimaryShardBatchAllocator.java b/server/src/main/java/org/opensearch/gateway/PrimaryShardBatchAllocator.java new file mode 100644 index 0000000000000..8d222903b6f29 --- /dev/null +++ b/server/src/main/java/org/opensearch/gateway/PrimaryShardBatchAllocator.java @@ -0,0 +1,150 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.gateway; + +import org.apache.logging.log4j.Logger; +import org.opensearch.cluster.node.DiscoveryNode; +import org.opensearch.cluster.routing.RoutingNodes; +import org.opensearch.cluster.routing.ShardRouting; +import org.opensearch.cluster.routing.allocation.AllocateUnassignedDecision; +import org.opensearch.cluster.routing.allocation.RoutingAllocation; +import org.opensearch.gateway.AsyncShardFetch.FetchResult; +import org.opensearch.gateway.TransportNodesGatewayStartedShardHelper.NodeGatewayStartedShard; +import org.opensearch.gateway.TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShardsBatch; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * PrimaryShardBatchAllocator is similar to {@link org.opensearch.gateway.PrimaryShardAllocator} only difference is + * that it can allocate multiple unassigned primary shards wherein PrimaryShardAllocator can only allocate single + * unassigned shard. + * The primary shard batch allocator allocates multiple unassigned primary shards to nodes that hold + * valid copies of the unassigned primaries. It does this by iterating over all unassigned + * primary shards in the routing table and fetching shard metadata from each node in the cluster + * that holds a copy of the shard. The shard metadata from each node is compared against the + * set of valid allocation IDs and for all valid shard copies (if any), the primary shard batch allocator + * executes the allocation deciders to chose a copy to assign the primary shard to. + *

+ * Note that the PrimaryShardBatchAllocator does *not* allocate primaries on index creation + * (see {@link org.opensearch.cluster.routing.allocation.allocator.BalancedShardsAllocator}), + * nor does it allocate primaries when a primary shard failed and there is a valid replica + * copy that can immediately be promoted to primary, as this takes place in {@link RoutingNodes#failShard}. + * + * @opensearch.internal + */ +public abstract class PrimaryShardBatchAllocator extends PrimaryShardAllocator { + + abstract protected FetchResult fetchData( + List eligibleShards, + List inEligibleShards, + RoutingAllocation allocation + ); + + protected FetchResult fetchData( + ShardRouting shard, + RoutingAllocation allocation + ) { + logger.error("fetchData for single shard called via batch allocator, shard id {}", shard.shardId()); + throw new IllegalStateException("PrimaryShardBatchAllocator should only be used for a batch of shards"); + } + + @Override + public AllocateUnassignedDecision makeAllocationDecision(ShardRouting unassignedShard, RoutingAllocation allocation, Logger logger) { + return makeAllocationDecision(Collections.singletonList(unassignedShard), allocation, logger).get(unassignedShard); + } + + /** + * Build allocation decisions for all the shards present in the batch identified by batchId. + * + * @param shards set of shards given for allocation + * @param allocation current allocation of all the shards + * @param logger logger used for logging + * @return shard to allocation decision map + */ + @Override + public HashMap makeAllocationDecision( + List shards, + RoutingAllocation allocation, + Logger logger + ) { + HashMap shardAllocationDecisions = new HashMap<>(); + List eligibleShards = new ArrayList<>(); + List inEligibleShards = new ArrayList<>(); + // identify ineligible shards + for (ShardRouting shard : shards) { + AllocateUnassignedDecision decision = getInEligibleShardDecision(shard, allocation); + if (decision != null) { + inEligibleShards.add(shard); + shardAllocationDecisions.put(shard, decision); + } else { + eligibleShards.add(shard); + } + } + // Do not call fetchData if there are no eligible shards + if (eligibleShards.isEmpty()) { + return shardAllocationDecisions; + } + // only fetch data for eligible shards + final FetchResult shardsState = fetchData(eligibleShards, inEligibleShards, allocation); + + // process the received data + for (ShardRouting unassignedShard : eligibleShards) { + List nodeShardStates = adaptToNodeShardStates(unassignedShard, shardsState); + // get allocation decision for this shard + shardAllocationDecisions.put(unassignedShard, getAllocationDecision(unassignedShard, allocation, nodeShardStates, logger)); + } + return shardAllocationDecisions; + } + + /** + * Transforms {@link FetchResult} of {@link NodeGatewayStartedShardsBatch} to {@link List} of {@link TransportNodesListGatewayStartedShards.NodeGatewayStartedShards}. + *

+ * Returns null if {@link FetchResult} does not have any data. + *

+ * shardsState contain the Data, there key is DiscoveryNode but value is Map of ShardId + * and NodeGatewayStartedShardsBatch so to get one shard level data (from all the nodes), we'll traverse the map + * and construct the nodeShardState along the way before making any allocation decision. As metadata for a + * particular shard is needed from all the discovery nodes. + * + * @param unassignedShard unassigned shard + * @param shardsState fetch data result for the whole batch + * @return shard state returned from each node + */ + private static List adaptToNodeShardStates( + ShardRouting unassignedShard, + FetchResult shardsState + ) { + if (!shardsState.hasData()) { + return null; + } + List nodeShardStates = new ArrayList<>(); + Map nodeResponses = shardsState.getData(); + + // build data for a shard from all the nodes + nodeResponses.forEach((node, nodeGatewayStartedShardsBatch) -> { + TransportNodesGatewayStartedShardHelper.GatewayStartedShard shardData = nodeGatewayStartedShardsBatch + .getNodeGatewayStartedShardsBatch() + .get(unassignedShard.shardId()); + nodeShardStates.add( + new NodeGatewayStartedShard( + shardData.allocationId(), + shardData.primary(), + shardData.replicationCheckpoint(), + shardData.storeException(), + node + ) + ); + }); + return nodeShardStates; + } +} diff --git a/server/src/main/java/org/opensearch/gateway/TransportNodesGatewayStartedShardHelper.java b/server/src/main/java/org/opensearch/gateway/TransportNodesGatewayStartedShardHelper.java index 403e3e96fa209..27cce76b1b694 100644 --- a/server/src/main/java/org/opensearch/gateway/TransportNodesGatewayStartedShardHelper.java +++ b/server/src/main/java/org/opensearch/gateway/TransportNodesGatewayStartedShardHelper.java @@ -12,8 +12,11 @@ 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.core.common.io.stream.StreamInput; +import org.opensearch.core.common.io.stream.StreamOutput; import org.opensearch.core.index.shard.ShardId; import org.opensearch.core.xcontent.NamedXContentRegistry; import org.opensearch.env.NodeEnvironment; @@ -23,8 +26,10 @@ import org.opensearch.index.shard.ShardStateMetadata; import org.opensearch.index.store.Store; import org.opensearch.indices.IndicesService; +import org.opensearch.indices.replication.checkpoint.ReplicationCheckpoint; import java.io.IOException; +import java.util.Objects; /** * This class has the common code used in {@link TransportNodesListGatewayStartedShards} and @@ -37,7 +42,7 @@ * @opensearch.internal */ public class TransportNodesGatewayStartedShardHelper { - public static TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShard getShardInfoOnLocalNode( + public static GatewayStartedShard getShardInfoOnLocalNode( Logger logger, final ShardId shardId, NamedXContentRegistry namedXContentRegistry, @@ -90,25 +95,168 @@ public static TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShar exception ); String allocationId = shardStateMetadata.allocationId != null ? shardStateMetadata.allocationId.getId() : null; - return new TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShard( - allocationId, - shardStateMetadata.primary, - null, - exception - ); + return new GatewayStartedShard(allocationId, shardStateMetadata.primary, null, exception); } } logger.debug("{} shard state info found: [{}]", shardId, shardStateMetadata); String allocationId = shardStateMetadata.allocationId != null ? shardStateMetadata.allocationId.getId() : null; final IndexShard shard = indicesService.getShardOrNull(shardId); - return new TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShard( + return new GatewayStartedShard( allocationId, shardStateMetadata.primary, shard != null ? shard.getLatestReplicationCheckpoint() : null ); } logger.trace("{} no local shard info found", shardId); - return new TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShard(null, false, null); + return new GatewayStartedShard(null, false, null); + } + + /** + * This class encapsulates the metadata about a started shard that needs to be persisted or sent between nodes. + * This is used in {@link TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShardsBatch} to construct the response for each node, instead of + * {@link TransportNodesListGatewayStartedShards.NodeGatewayStartedShards} because we don't need to save an extra + * {@link DiscoveryNode} object like in {@link TransportNodesListGatewayStartedShards.NodeGatewayStartedShards} + * which reduces memory footprint of its objects. + * + * @opensearch.internal + */ + public static class GatewayStartedShard { + private final String allocationId; + private final boolean primary; + private final Exception storeException; + private final ReplicationCheckpoint replicationCheckpoint; + + public GatewayStartedShard(StreamInput in) throws IOException { + allocationId = in.readOptionalString(); + primary = in.readBoolean(); + if (in.readBoolean()) { + storeException = in.readException(); + } else { + storeException = null; + } + if (in.readBoolean()) { + replicationCheckpoint = new ReplicationCheckpoint(in); + } else { + replicationCheckpoint = null; + } + } + + public GatewayStartedShard(String allocationId, boolean primary, ReplicationCheckpoint replicationCheckpoint) { + this(allocationId, primary, replicationCheckpoint, null); + } + + public GatewayStartedShard( + String allocationId, + boolean primary, + ReplicationCheckpoint replicationCheckpoint, + Exception storeException + ) { + this.allocationId = allocationId; + this.primary = primary; + this.replicationCheckpoint = replicationCheckpoint; + this.storeException = storeException; + } + + public String allocationId() { + return this.allocationId; + } + + public boolean primary() { + return this.primary; + } + + public ReplicationCheckpoint replicationCheckpoint() { + return this.replicationCheckpoint; + } + + public Exception storeException() { + return this.storeException; + } + + public void writeTo(StreamOutput out) throws IOException { + out.writeOptionalString(allocationId); + out.writeBoolean(primary); + if (storeException != null) { + out.writeBoolean(true); + out.writeException(storeException); + } else { + out.writeBoolean(false); + } + if (replicationCheckpoint != null) { + out.writeBoolean(true); + replicationCheckpoint.writeTo(out); + } else { + out.writeBoolean(false); + } + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + + GatewayStartedShard that = (GatewayStartedShard) o; + + return primary == that.primary + && Objects.equals(allocationId, that.allocationId) + && Objects.equals(storeException, that.storeException) + && Objects.equals(replicationCheckpoint, that.replicationCheckpoint); + } + + @Override + public int hashCode() { + int result = (allocationId != null ? allocationId.hashCode() : 0); + result = 31 * result + (primary ? 1 : 0); + result = 31 * result + (storeException != null ? storeException.hashCode() : 0); + result = 31 * result + (replicationCheckpoint != null ? replicationCheckpoint.hashCode() : 0); + return result; + } + + @Override + public String toString() { + StringBuilder buf = new StringBuilder(); + buf.append("NodeGatewayStartedShards[").append("allocationId=").append(allocationId).append(",primary=").append(primary); + if (storeException != null) { + buf.append(",storeException=").append(storeException); + } + if (replicationCheckpoint != null) { + buf.append(",ReplicationCheckpoint=").append(replicationCheckpoint.toString()); + } + buf.append("]"); + return buf.toString(); + } + } + + /** + * This class extends the {@link GatewayStartedShard} which contains all necessary shard metadata like + * allocationId and replication checkpoint. It also has DiscoveryNode which is needed by + * {@link PrimaryShardAllocator} and {@link PrimaryShardBatchAllocator} to make allocation decision. + * This class removes the dependency of + * {@link TransportNodesListGatewayStartedShards.NodeGatewayStartedShards} to make allocation decisions by + * {@link PrimaryShardAllocator} or {@link PrimaryShardBatchAllocator}. + */ + public static class NodeGatewayStartedShard extends GatewayStartedShard { + + private final DiscoveryNode node; + + public NodeGatewayStartedShard( + String allocationId, + boolean primary, + ReplicationCheckpoint replicationCheckpoint, + Exception storeException, + DiscoveryNode node + ) { + super(allocationId, primary, replicationCheckpoint, storeException); + this.node = node; + } + + public DiscoveryNode getNode() { + return node; + } } } diff --git a/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedShards.java b/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedShards.java index 0ba872aab9974..4b1f611bb88ab 100644 --- a/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedShards.java +++ b/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedShards.java @@ -53,6 +53,7 @@ import org.opensearch.core.index.shard.ShardId; import org.opensearch.core.xcontent.NamedXContentRegistry; import org.opensearch.env.NodeEnvironment; +import org.opensearch.gateway.TransportNodesGatewayStartedShardHelper.GatewayStartedShard; import org.opensearch.indices.IndicesService; import org.opensearch.indices.replication.checkpoint.ReplicationCheckpoint; import org.opensearch.indices.store.ShardAttributes; @@ -154,7 +155,7 @@ protected NodesGatewayStartedShards newResponse( @Override protected NodeGatewayStartedShards nodeOperation(NodeRequest request) { try { - TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShard shardInfo = getShardInfoOnLocalNode( + GatewayStartedShard shardInfo = getShardInfoOnLocalNode( logger, request.getShardId(), namedXContentRegistry, @@ -166,10 +167,12 @@ protected NodeGatewayStartedShards nodeOperation(NodeRequest request) { ); return new NodeGatewayStartedShards( clusterService.localNode(), - shardInfo.allocationId(), - shardInfo.primary(), - shardInfo.replicationCheckpoint(), - shardInfo.storeException() + new GatewayStartedShard( + shardInfo.allocationId(), + shardInfo.primary(), + shardInfo.replicationCheckpoint(), + shardInfo.storeException() + ) ); } catch (Exception e) { throw new OpenSearchException("failed to load started shards", e); @@ -302,81 +305,51 @@ public String getCustomDataPath() { * @opensearch.internal */ public static class NodeGatewayStartedShards extends BaseNodeResponse { - private final String allocationId; - private final boolean primary; - private final Exception storeException; - private final ReplicationCheckpoint replicationCheckpoint; + private final GatewayStartedShard gatewayStartedShard; public NodeGatewayStartedShards(StreamInput in) throws IOException { super(in); - allocationId = in.readOptionalString(); - primary = in.readBoolean(); + String allocationId = in.readOptionalString(); + boolean primary = in.readBoolean(); + Exception storeException; if (in.readBoolean()) { storeException = in.readException(); } else { storeException = null; } + ReplicationCheckpoint replicationCheckpoint; if (in.getVersion().onOrAfter(Version.V_2_3_0) && in.readBoolean()) { replicationCheckpoint = new ReplicationCheckpoint(in); } else { replicationCheckpoint = null; } + this.gatewayStartedShard = new GatewayStartedShard(allocationId, primary, replicationCheckpoint, storeException); } - public NodeGatewayStartedShards( - DiscoveryNode node, - String allocationId, - boolean primary, - ReplicationCheckpoint replicationCheckpoint - ) { - this(node, allocationId, primary, replicationCheckpoint, null); + public GatewayStartedShard getGatewayShardStarted() { + return gatewayStartedShard; } - public NodeGatewayStartedShards( - DiscoveryNode node, - String allocationId, - boolean primary, - ReplicationCheckpoint replicationCheckpoint, - Exception storeException - ) { + public NodeGatewayStartedShards(DiscoveryNode node, GatewayStartedShard gatewayStartedShard) { super(node); - this.allocationId = allocationId; - this.primary = primary; - this.replicationCheckpoint = replicationCheckpoint; - this.storeException = storeException; - } - - public String allocationId() { - return this.allocationId; - } - - public boolean primary() { - return this.primary; - } - - public ReplicationCheckpoint replicationCheckpoint() { - return this.replicationCheckpoint; - } - - public Exception storeException() { - return this.storeException; + this.gatewayStartedShard = gatewayStartedShard; } @Override public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); - out.writeOptionalString(allocationId); - out.writeBoolean(primary); - if (storeException != null) { + out.writeOptionalString(gatewayStartedShard.allocationId()); + out.writeBoolean(gatewayStartedShard.primary()); + if (gatewayStartedShard.storeException() != null) { out.writeBoolean(true); - out.writeException(storeException); + out.writeException(gatewayStartedShard.storeException()); } else { out.writeBoolean(false); } if (out.getVersion().onOrAfter(Version.V_2_3_0)) { - if (replicationCheckpoint != null) { + if (gatewayStartedShard.replicationCheckpoint() != null) { out.writeBoolean(true); - replicationCheckpoint.writeTo(out); + gatewayStartedShard.replicationCheckpoint().writeTo(out); } else { out.writeBoolean(false); } @@ -394,33 +367,17 @@ public boolean equals(Object o) { NodeGatewayStartedShards that = (NodeGatewayStartedShards) o; - return primary == that.primary - && Objects.equals(allocationId, that.allocationId) - && Objects.equals(storeException, that.storeException) - && Objects.equals(replicationCheckpoint, that.replicationCheckpoint); + return gatewayStartedShard.equals(that.gatewayStartedShard); } @Override public int hashCode() { - int result = (allocationId != null ? allocationId.hashCode() : 0); - result = 31 * result + (primary ? 1 : 0); - result = 31 * result + (storeException != null ? storeException.hashCode() : 0); - result = 31 * result + (replicationCheckpoint != null ? replicationCheckpoint.hashCode() : 0); - return result; + return gatewayStartedShard.hashCode(); } @Override public String toString() { - StringBuilder buf = new StringBuilder(); - buf.append("NodeGatewayStartedShards[").append("allocationId=").append(allocationId).append(",primary=").append(primary); - if (storeException != null) { - buf.append(",storeException=").append(storeException); - } - if (replicationCheckpoint != null) { - buf.append(",ReplicationCheckpoint=").append(replicationCheckpoint.toString()); - } - buf.append("]"); - return buf.toString(); + return gatewayStartedShard.toString(); } } } diff --git a/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedShardsBatch.java b/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedShardsBatch.java index bc327c1b85748..dc5d85b17bc32 100644 --- a/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedShardsBatch.java +++ b/server/src/main/java/org/opensearch/gateway/TransportNodesListGatewayStartedShardsBatch.java @@ -27,8 +27,8 @@ import org.opensearch.core.index.shard.ShardId; import org.opensearch.core.xcontent.NamedXContentRegistry; import org.opensearch.env.NodeEnvironment; +import org.opensearch.gateway.TransportNodesGatewayStartedShardHelper.GatewayStartedShard; import org.opensearch.indices.IndicesService; -import org.opensearch.indices.replication.checkpoint.ReplicationCheckpoint; import org.opensearch.indices.store.ShardAttributes; import org.opensearch.threadpool.ThreadPool; import org.opensearch.transport.TransportRequest; @@ -135,7 +135,7 @@ protected NodesGatewayStartedShardsBatch newResponse( */ @Override protected NodeGatewayStartedShardsBatch nodeOperation(NodeRequest request) { - Map shardsOnNode = new HashMap<>(); + Map shardsOnNode = new HashMap<>(); for (ShardAttributes shardAttr : request.shardAttributes.values()) { final ShardId shardId = shardAttr.getShardId(); try { @@ -155,7 +155,7 @@ protected NodeGatewayStartedShardsBatch nodeOperation(NodeRequest request) { } catch (Exception e) { shardsOnNode.put( shardId, - new NodeGatewayStartedShard(null, false, null, new OpenSearchException("failed to load started shards", e)) + new GatewayStartedShard(null, false, null, new OpenSearchException("failed to load started shards", e)) ); } } @@ -248,126 +248,6 @@ public void writeTo(StreamOutput out) throws IOException { } } - /** - * This class encapsulates the metadata about a started shard that needs to be persisted or sent between nodes. - * This is used in {@link NodeGatewayStartedShardsBatch} to construct the response for each node, instead of - * {@link TransportNodesListGatewayStartedShards.NodeGatewayStartedShards} because we don't need to save an extra - * {@link DiscoveryNode} object like in {@link TransportNodesListGatewayStartedShards.NodeGatewayStartedShards} - * which reduces memory footprint of its objects. - * - * @opensearch.internal - */ - public static class NodeGatewayStartedShard { - private final String allocationId; - private final boolean primary; - private final Exception storeException; - private final ReplicationCheckpoint replicationCheckpoint; - - public NodeGatewayStartedShard(StreamInput in) throws IOException { - allocationId = in.readOptionalString(); - primary = in.readBoolean(); - if (in.readBoolean()) { - storeException = in.readException(); - } else { - storeException = null; - } - if (in.readBoolean()) { - replicationCheckpoint = new ReplicationCheckpoint(in); - } else { - replicationCheckpoint = null; - } - } - - public NodeGatewayStartedShard(String allocationId, boolean primary, ReplicationCheckpoint replicationCheckpoint) { - this(allocationId, primary, replicationCheckpoint, null); - } - - public NodeGatewayStartedShard( - String allocationId, - boolean primary, - ReplicationCheckpoint replicationCheckpoint, - Exception storeException - ) { - this.allocationId = allocationId; - this.primary = primary; - this.replicationCheckpoint = replicationCheckpoint; - this.storeException = storeException; - } - - public String allocationId() { - return this.allocationId; - } - - public boolean primary() { - return this.primary; - } - - public ReplicationCheckpoint replicationCheckpoint() { - return this.replicationCheckpoint; - } - - public Exception storeException() { - return this.storeException; - } - - public void writeTo(StreamOutput out) throws IOException { - out.writeOptionalString(allocationId); - out.writeBoolean(primary); - if (storeException != null) { - out.writeBoolean(true); - out.writeException(storeException); - } else { - out.writeBoolean(false); - } - if (replicationCheckpoint != null) { - out.writeBoolean(true); - replicationCheckpoint.writeTo(out); - } else { - out.writeBoolean(false); - } - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - - NodeGatewayStartedShard that = (NodeGatewayStartedShard) o; - - return primary == that.primary - && Objects.equals(allocationId, that.allocationId) - && Objects.equals(storeException, that.storeException) - && Objects.equals(replicationCheckpoint, that.replicationCheckpoint); - } - - @Override - public int hashCode() { - int result = (allocationId != null ? allocationId.hashCode() : 0); - result = 31 * result + (primary ? 1 : 0); - result = 31 * result + (storeException != null ? storeException.hashCode() : 0); - result = 31 * result + (replicationCheckpoint != null ? replicationCheckpoint.hashCode() : 0); - return result; - } - - @Override - public String toString() { - StringBuilder buf = new StringBuilder(); - buf.append("NodeGatewayStartedShards[").append("allocationId=").append(allocationId).append(",primary=").append(primary); - if (storeException != null) { - buf.append(",storeException=").append(storeException); - } - if (replicationCheckpoint != null) { - buf.append(",ReplicationCheckpoint=").append(replicationCheckpoint.toString()); - } - buf.append("]"); - return buf.toString(); - } - } - /** * This is the response from a single node, this is used in {@link NodesGatewayStartedShardsBatch} for creating * node to its response mapping for this transport request. @@ -376,15 +256,15 @@ public String toString() { * @opensearch.internal */ public static class NodeGatewayStartedShardsBatch extends BaseNodeResponse { - private final Map nodeGatewayStartedShardsBatch; + private final Map nodeGatewayStartedShardsBatch; - public Map getNodeGatewayStartedShardsBatch() { + public Map getNodeGatewayStartedShardsBatch() { return nodeGatewayStartedShardsBatch; } public NodeGatewayStartedShardsBatch(StreamInput in) throws IOException { super(in); - this.nodeGatewayStartedShardsBatch = in.readMap(ShardId::new, NodeGatewayStartedShard::new); + this.nodeGatewayStartedShardsBatch = in.readMap(ShardId::new, GatewayStartedShard::new); } @Override @@ -393,7 +273,7 @@ public void writeTo(StreamOutput out) throws IOException { out.writeMap(nodeGatewayStartedShardsBatch, (o, k) -> k.writeTo(o), (o, v) -> v.writeTo(o)); } - public NodeGatewayStartedShardsBatch(DiscoveryNode node, Map nodeGatewayStartedShardsBatch) { + public NodeGatewayStartedShardsBatch(DiscoveryNode node, Map nodeGatewayStartedShardsBatch) { super(node); this.nodeGatewayStartedShardsBatch = nodeGatewayStartedShardsBatch; } diff --git a/server/src/test/java/org/opensearch/gateway/PrimaryShardAllocatorTests.java b/server/src/test/java/org/opensearch/gateway/PrimaryShardAllocatorTests.java index dceda6433575c..e849f12143b4d 100644 --- a/server/src/test/java/org/opensearch/gateway/PrimaryShardAllocatorTests.java +++ b/server/src/test/java/org/opensearch/gateway/PrimaryShardAllocatorTests.java @@ -843,10 +843,12 @@ public TestAllocator addData( node, new TransportNodesListGatewayStartedShards.NodeGatewayStartedShards( node, - allocationId, - primary, - replicationCheckpoint, - storeException + new TransportNodesGatewayStartedShardHelper.GatewayStartedShard( + allocationId, + primary, + replicationCheckpoint, + storeException + ) ) ); return this; diff --git a/server/src/test/java/org/opensearch/gateway/PrimaryShardBatchAllocatorTests.java b/server/src/test/java/org/opensearch/gateway/PrimaryShardBatchAllocatorTests.java new file mode 100644 index 0000000000000..4796def2b8902 --- /dev/null +++ b/server/src/test/java/org/opensearch/gateway/PrimaryShardBatchAllocatorTests.java @@ -0,0 +1,340 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ +package org.opensearch.gateway; + +import org.apache.lucene.codecs.Codec; +import org.opensearch.Version; +import org.opensearch.cluster.ClusterName; +import org.opensearch.cluster.ClusterState; +import org.opensearch.cluster.OpenSearchAllocationTestCase; +import org.opensearch.cluster.metadata.IndexMetadata; +import org.opensearch.cluster.metadata.Metadata; +import org.opensearch.cluster.node.DiscoveryNode; +import org.opensearch.cluster.node.DiscoveryNodes; +import org.opensearch.cluster.routing.RoutingNodes; +import org.opensearch.cluster.routing.RoutingTable; +import org.opensearch.cluster.routing.ShardRouting; +import org.opensearch.cluster.routing.UnassignedInfo; +import org.opensearch.cluster.routing.allocation.AllocateUnassignedDecision; +import org.opensearch.cluster.routing.allocation.AllocationDecision; +import org.opensearch.cluster.routing.allocation.RoutingAllocation; +import org.opensearch.cluster.routing.allocation.decider.AllocationDeciders; +import org.opensearch.common.Nullable; +import org.opensearch.common.settings.Settings; +import org.opensearch.common.util.set.Sets; +import org.opensearch.core.index.shard.ShardId; +import org.opensearch.env.Environment; +import org.opensearch.index.IndexSettings; +import org.opensearch.index.codec.CodecService; +import org.opensearch.indices.replication.checkpoint.ReplicationCheckpoint; +import org.opensearch.test.IndexSettingsModule; +import org.junit.Before; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import static org.opensearch.cluster.routing.UnassignedInfo.Reason.CLUSTER_RECOVERED; + +public class PrimaryShardBatchAllocatorTests extends OpenSearchAllocationTestCase { + + private final ShardId shardId = new ShardId("test", "_na_", 0); + private static Set shardsInBatch; + private final DiscoveryNode node1 = newNode("node1"); + private final DiscoveryNode node2 = newNode("node2"); + private final DiscoveryNode node3 = newNode("node3"); + private TestBatchAllocator batchAllocator; + + public static void setUpShards(int numberOfShards) { + shardsInBatch = new HashSet<>(); + for (int shardNumber = 0; shardNumber < numberOfShards; shardNumber++) { + ShardId shardId = new ShardId("test", "_na_", shardNumber); + shardsInBatch.add(shardId); + } + } + + @Before + public void buildTestAllocator() { + this.batchAllocator = new TestBatchAllocator(); + } + + private void allocateAllUnassigned(final RoutingAllocation allocation) { + final RoutingNodes.UnassignedShards.UnassignedIterator iterator = allocation.routingNodes().unassigned().iterator(); + while (iterator.hasNext()) { + batchAllocator.allocateUnassigned(iterator.next(), allocation, iterator); + } + } + + private void allocateAllUnassignedBatch(final RoutingAllocation allocation) { + final RoutingNodes.UnassignedShards.UnassignedIterator iterator = allocation.routingNodes().unassigned().iterator(); + List shardsToBatch = new ArrayList<>(); + while (iterator.hasNext()) { + shardsToBatch.add(iterator.next()); + } + batchAllocator.allocateUnassignedBatch(shardsToBatch, allocation); + } + + public void testMakeAllocationDecisionDataFetching() { + final RoutingAllocation allocation = routingAllocationWithOnePrimary(noAllocationDeciders(), CLUSTER_RECOVERED, "allocId1"); + + List shards = new ArrayList<>(); + allocateAllUnassignedBatch(allocation); + ShardRouting shard = allocation.routingTable().getIndicesRouting().get("test").shard(shardId.id()).primaryShard(); + shards.add(shard); + HashMap allDecisions = batchAllocator.makeAllocationDecision(shards, allocation, logger); + // verify we get decisions for all the shards + assertEquals(shards.size(), allDecisions.size()); + assertEquals(shards, new ArrayList<>(allDecisions.keySet())); + assertEquals(AllocationDecision.AWAITING_INFO, allDecisions.get(shard).getAllocationDecision()); + } + + public void testMakeAllocationDecisionForReplicaShard() { + final RoutingAllocation allocation = routingAllocationWithOnePrimary(noAllocationDeciders(), CLUSTER_RECOVERED, "allocId1"); + + List replicaShards = allocation.routingTable().getIndicesRouting().get("test").shard(shardId.id()).replicaShards(); + List shards = new ArrayList<>(replicaShards); + HashMap allDecisions = batchAllocator.makeAllocationDecision(shards, allocation, logger); + // verify we get decisions for all the shards + assertEquals(shards.size(), allDecisions.size()); + assertEquals(shards, new ArrayList<>(allDecisions.keySet())); + assertFalse(allDecisions.get(replicaShards.get(0)).isDecisionTaken()); + } + + public void testMakeAllocationDecisionDataFetched() { + final RoutingAllocation allocation = routingAllocationWithOnePrimary(noAllocationDeciders(), CLUSTER_RECOVERED, "allocId1"); + + List shards = new ArrayList<>(); + ShardRouting shard = allocation.routingTable().getIndicesRouting().get("test").shard(shardId.id()).primaryShard(); + shards.add(shard); + batchAllocator.addData(node1, "allocId1", true, new ReplicationCheckpoint(shardId, 20, 101, 1, Codec.getDefault().getName())); + HashMap allDecisions = batchAllocator.makeAllocationDecision(shards, allocation, logger); + // verify we get decisions for all the shards + assertEquals(shards.size(), allDecisions.size()); + assertEquals(shards, new ArrayList<>(allDecisions.keySet())); + assertEquals(AllocationDecision.YES, allDecisions.get(shard).getAllocationDecision()); + } + + public void testMakeAllocationDecisionDataFetchedMultipleShards() { + setUpShards(2); + final RoutingAllocation allocation = routingAllocationWithMultiplePrimaries( + noAllocationDeciders(), + CLUSTER_RECOVERED, + 2, + 0, + "allocId-0", + "allocId-1" + ); + List shards = new ArrayList<>(); + for (ShardId shardId : shardsInBatch) { + ShardRouting shard = allocation.routingTable().getIndicesRouting().get("test").shard(shardId.id()).primaryShard(); + allocation.routingTable().getIndicesRouting().get("test").shard(shardId.id()).primaryShard().recoverySource(); + shards.add(shard); + batchAllocator.addShardData( + node1, + "allocId-" + shardId.id(), + shardId, + true, + new ReplicationCheckpoint(shardId, 20, 101, 1, Codec.getDefault().getName()), + null + ); + } + HashMap allDecisions = batchAllocator.makeAllocationDecision(shards, allocation, logger); + // verify we get decisions for all the shards + assertEquals(shards.size(), allDecisions.size()); + assertEquals(new HashSet<>(shards), allDecisions.keySet()); + for (ShardRouting shard : shards) { + assertEquals(AllocationDecision.YES, allDecisions.get(shard).getAllocationDecision()); + } + } + + private RoutingAllocation routingAllocationWithOnePrimary( + AllocationDeciders deciders, + UnassignedInfo.Reason reason, + String... activeAllocationIds + ) { + Metadata metadata = Metadata.builder() + .put( + IndexMetadata.builder(shardId.getIndexName()) + .settings(settings(Version.CURRENT)) + .numberOfShards(1) + .numberOfReplicas(1) + .putInSyncAllocationIds(shardId.id(), Sets.newHashSet(activeAllocationIds)) + ) + .build(); + RoutingTable.Builder routingTableBuilder = RoutingTable.builder(); + switch (reason) { + + case INDEX_CREATED: + routingTableBuilder.addAsNew(metadata.index(shardId.getIndex())); + break; + case CLUSTER_RECOVERED: + routingTableBuilder.addAsRecovery(metadata.index(shardId.getIndex())); + break; + case INDEX_REOPENED: + routingTableBuilder.addAsFromCloseToOpen(metadata.index(shardId.getIndex())); + break; + default: + throw new IllegalArgumentException("can't do " + reason + " for you. teach me"); + } + ClusterState state = ClusterState.builder(org.opensearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)) + .metadata(metadata) + .routingTable(routingTableBuilder.build()) + .nodes(DiscoveryNodes.builder().add(node1).add(node2).add(node3)) + .build(); + return new RoutingAllocation(deciders, new RoutingNodes(state, false), state, null, null, System.nanoTime()); + } + + private RoutingAllocation routingAllocationWithMultiplePrimaries( + AllocationDeciders deciders, + UnassignedInfo.Reason reason, + int numberOfShards, + int replicas, + String... activeAllocationIds + ) { + Iterator shardIterator = shardsInBatch.iterator(); + Metadata metadata = Metadata.builder() + .put( + IndexMetadata.builder(shardId.getIndexName()) + .settings(settings(Version.CURRENT)) + .numberOfShards(numberOfShards) + .numberOfReplicas(replicas) + .putInSyncAllocationIds(shardIterator.next().id(), Sets.newHashSet(activeAllocationIds[0])) + .putInSyncAllocationIds(shardIterator.next().id(), Sets.newHashSet(activeAllocationIds[1])) + ) + .build(); + + RoutingTable.Builder routingTableBuilder = RoutingTable.builder(); + for (ShardId shardIdFromBatch : shardsInBatch) { + switch (reason) { + case INDEX_CREATED: + routingTableBuilder.addAsNew(metadata.index(shardIdFromBatch.getIndex())); + break; + case CLUSTER_RECOVERED: + routingTableBuilder.addAsRecovery(metadata.index(shardIdFromBatch.getIndex())); + break; + case INDEX_REOPENED: + routingTableBuilder.addAsFromCloseToOpen(metadata.index(shardIdFromBatch.getIndex())); + break; + default: + throw new IllegalArgumentException("can't do " + reason + " for you. teach me"); + } + } + ClusterState state = ClusterState.builder(ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)) + .metadata(metadata) + .routingTable(routingTableBuilder.build()) + .nodes(DiscoveryNodes.builder().add(node1).add(node2).add(node3)) + .build(); + return new RoutingAllocation(deciders, new RoutingNodes(state, false), state, null, null, System.nanoTime()); + } + + class TestBatchAllocator extends PrimaryShardBatchAllocator { + + private Map data; + + public TestBatchAllocator clear() { + data = null; + return this; + } + + public TestBatchAllocator addData( + DiscoveryNode node, + String allocationId, + boolean primary, + ReplicationCheckpoint replicationCheckpoint + ) { + return addData(node, allocationId, primary, replicationCheckpoint, null); + } + + public TestBatchAllocator addData(DiscoveryNode node, String allocationId, boolean primary) { + Settings nodeSettings = Settings.builder().put(Environment.PATH_HOME_SETTING.getKey(), createTempDir()).build(); + IndexSettings indexSettings = IndexSettingsModule.newIndexSettings("test", nodeSettings); + return addData( + node, + allocationId, + primary, + ReplicationCheckpoint.empty(shardId, new CodecService(null, indexSettings, null).codec("default").getName()), + null + ); + } + + public TestBatchAllocator addData(DiscoveryNode node, String allocationId, boolean primary, @Nullable Exception storeException) { + Settings nodeSettings = Settings.builder().put(Environment.PATH_HOME_SETTING.getKey(), createTempDir()).build(); + IndexSettings indexSettings = IndexSettingsModule.newIndexSettings("test", nodeSettings); + return addData( + node, + allocationId, + primary, + ReplicationCheckpoint.empty(shardId, new CodecService(null, indexSettings, null).codec("default").getName()), + storeException + ); + } + + public TestBatchAllocator addData( + DiscoveryNode node, + String allocationId, + boolean primary, + ReplicationCheckpoint replicationCheckpoint, + @Nullable Exception storeException + ) { + if (data == null) { + data = new HashMap<>(); + } + Map shardData = Map.of( + shardId, + new TransportNodesGatewayStartedShardHelper.GatewayStartedShard( + allocationId, + primary, + replicationCheckpoint, + storeException + ) + ); + data.put(node, new TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShardsBatch(node, shardData)); + return this; + } + + public TestBatchAllocator addShardData( + DiscoveryNode node, + String allocationId, + ShardId shardId, + boolean primary, + ReplicationCheckpoint replicationCheckpoint, + @Nullable Exception storeException + ) { + if (data == null) { + data = new HashMap<>(); + } + Map shardData = new HashMap<>(); + shardData.put( + shardId, + new TransportNodesGatewayStartedShardHelper.GatewayStartedShard( + allocationId, + primary, + replicationCheckpoint, + storeException + ) + ); + if (data.get(node) != null) shardData.putAll(data.get(node).getNodeGatewayStartedShardsBatch()); + data.put(node, new TransportNodesListGatewayStartedShardsBatch.NodeGatewayStartedShardsBatch(node, shardData)); + return this; + } + + @Override + protected AsyncShardFetch.FetchResult fetchData( + List shardsEligibleForFetch, + List inEligibleShards, + RoutingAllocation allocation + ) { + return new AsyncShardFetch.FetchResult<>(data, Collections.>emptyMap()); + } + } +} diff --git a/test/framework/src/main/java/org/opensearch/test/gateway/TestGatewayAllocator.java b/test/framework/src/main/java/org/opensearch/test/gateway/TestGatewayAllocator.java index f123b926f5bad..b1695ff00e0cc 100644 --- a/test/framework/src/main/java/org/opensearch/test/gateway/TestGatewayAllocator.java +++ b/test/framework/src/main/java/org/opensearch/test/gateway/TestGatewayAllocator.java @@ -42,6 +42,7 @@ import org.opensearch.gateway.GatewayAllocator; import org.opensearch.gateway.PrimaryShardAllocator; import org.opensearch.gateway.ReplicaShardAllocator; +import org.opensearch.gateway.TransportNodesGatewayStartedShardHelper; import org.opensearch.gateway.TransportNodesListGatewayStartedShards.NodeGatewayStartedShards; import org.opensearch.indices.replication.checkpoint.ReplicationCheckpoint; import org.opensearch.indices.store.TransportNodesListShardStoreMetadata.NodeStoreFilesMetadata; @@ -91,9 +92,12 @@ protected AsyncShardFetch.FetchResult fetchData(ShardR routing -> currentNodes.get(routing.currentNodeId()), routing -> new NodeGatewayStartedShards( currentNodes.get(routing.currentNodeId()), - routing.allocationId().getId(), - routing.primary(), - getReplicationCheckpoint(shardId, routing.currentNodeId()) + new TransportNodesGatewayStartedShardHelper.GatewayStartedShard( + routing.allocationId().getId(), + routing.primary(), + getReplicationCheckpoint(shardId, routing.currentNodeId()), + null + ) ) ) ); From 2567792fade43d99f1ca7a0d004520ef1d23a4f0 Mon Sep 17 00:00:00 2001 From: Gagan Juneja Date: Tue, 19 Mar 2024 10:33:55 +0530 Subject: [PATCH 03/38] Fixes flaky test org.opensearch.telemetry.metrics.TelemetryMetricsEnabledSanityIT.testGauge (#12745) Signed-off-by: Gagan Juneja Co-authored-by: Gagan Juneja --- .../telemetry/metrics/TelemetryMetricsEnabledSanityIT.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/plugins/telemetry-otel/src/internalClusterTest/java/org/opensearch/telemetry/metrics/TelemetryMetricsEnabledSanityIT.java b/plugins/telemetry-otel/src/internalClusterTest/java/org/opensearch/telemetry/metrics/TelemetryMetricsEnabledSanityIT.java index 8d03b3d6a9983..90143d907cd99 100644 --- a/plugins/telemetry-otel/src/internalClusterTest/java/org/opensearch/telemetry/metrics/TelemetryMetricsEnabledSanityIT.java +++ b/plugins/telemetry-otel/src/internalClusterTest/java/org/opensearch/telemetry/metrics/TelemetryMetricsEnabledSanityIT.java @@ -124,8 +124,8 @@ public void testHistogram() throws Exception { assertEquals(1.0, histogramPointData.getMin(), 1.0); } - public void testObservableGauge() throws Exception { - String metricName = "test-observable-gauge"; + public void testGauge() throws Exception { + String metricName = "test-gauge"; MetricsRegistry metricsRegistry = internalCluster().getInstance(MetricsRegistry.class); InMemorySingletonMetricsExporter.INSTANCE.reset(); Tags tags = Tags.create().addTag("test", "integ-test"); @@ -137,7 +137,7 @@ public void testObservableGauge() throws Exception { InMemorySingletonMetricsExporter exporter = InMemorySingletonMetricsExporter.INSTANCE; - assertEquals(2.0, getMaxObservableGaugeValue(exporter, metricName), 0.0); + assertTrue(getMaxObservableGaugeValue(exporter, metricName) >= 2.0); gaugeCloseable.close(); double observableGaugeValueAfterStop = getMaxObservableGaugeValue(exporter, metricName); From 902a10bdf278bb2160858c594432dcd6ab820559 Mon Sep 17 00:00:00 2001 From: Sagar <99425694+sgup432@users.noreply.github.com> Date: Mon, 18 Mar 2024 22:17:14 -0700 Subject: [PATCH 04/38] [Tiered Caching] Fix test testComputeIfAbsentWithFactoryBasedCacheCreation (#12700) --------- Signed-off-by: Sagar Upadhyaya --- .../common/tier/TieredSpilloverCache.java | 4 +-- .../tier/TieredSpilloverCacheTests.java | 30 ++++++++++--------- 2 files changed, 18 insertions(+), 16 deletions(-) diff --git a/modules/cache-common/src/main/java/org/opensearch/cache/common/tier/TieredSpilloverCache.java b/modules/cache-common/src/main/java/org/opensearch/cache/common/tier/TieredSpilloverCache.java index 99c9c77ff3872..00a8eec93acc9 100644 --- a/modules/cache-common/src/main/java/org/opensearch/cache/common/tier/TieredSpilloverCache.java +++ b/modules/cache-common/src/main/java/org/opensearch/cache/common/tier/TieredSpilloverCache.java @@ -79,8 +79,8 @@ public void onRemoval(RemovalNotification notification) { .setValueType(builder.cacheConfig.getValueType()) .setSettings(builder.cacheConfig.getSettings()) .setWeigher(builder.cacheConfig.getWeigher()) - .setMaxSizeInBytes(builder.cacheConfig.getMaxSizeInBytes()) // TODO: Part of a workaround for an issue in TSC. Overall fix - // coming soon + .setMaxSizeInBytes(builder.cacheConfig.getMaxSizeInBytes()) + .setExpireAfterAccess(builder.cacheConfig.getExpireAfterAccess()) .build(), builder.cacheType, builder.cacheFactories diff --git a/modules/cache-common/src/test/java/org/opensearch/cache/common/tier/TieredSpilloverCacheTests.java b/modules/cache-common/src/test/java/org/opensearch/cache/common/tier/TieredSpilloverCacheTests.java index 3e4fb0efd092e..b132952834f06 100644 --- a/modules/cache-common/src/test/java/org/opensearch/cache/common/tier/TieredSpilloverCacheTests.java +++ b/modules/cache-common/src/test/java/org/opensearch/cache/common/tier/TieredSpilloverCacheTests.java @@ -119,6 +119,11 @@ public void testComputeIfAbsentWithFactoryBasedCacheCreation() throws Exception .getKey(), onHeapCacheSize * keyValueSize + "b" ) + .put( + CacheSettings.getConcreteStoreNameSettingForCacheType(CacheType.INDICES_REQUEST_CACHE).getKey(), + TieredSpilloverCache.TieredSpilloverCacheFactory.TIERED_SPILLOVER_CACHE_NAME + ) + .put(FeatureFlags.PLUGGABLE_CACHE, "true") .build(); ICache tieredSpilloverICache = new TieredSpilloverCache.TieredSpilloverCacheFactory().create( @@ -127,12 +132,8 @@ public void testComputeIfAbsentWithFactoryBasedCacheCreation() throws Exception .setWeigher((k, v) -> keyValueSize) .setRemovalListener(removalListener) .setSettings(settings) - .setCachedResultParser(new Function() { - @Override - public CachedQueryResult.PolicyValues apply(String s) { - return new CachedQueryResult.PolicyValues(20_000_000L); - } - }) // Values will always appear to have taken 20_000_000 ns = 20 ms to compute + .setCachedResultParser(s -> new CachedQueryResult.PolicyValues(20_000_000L)) // Values will always appear to have taken + // 20_000_000 ns = 20 ms to compute .build(), CacheType.INDICES_REQUEST_CACHE, Map.of( @@ -145,20 +146,16 @@ public CachedQueryResult.PolicyValues apply(String s) { TieredSpilloverCache tieredSpilloverCache = (TieredSpilloverCache) tieredSpilloverICache; - // Put values in cache more than it's size and cause evictions from onHeap. int numOfItems1 = randomIntBetween(onHeapCacheSize + 1, totalSize); - List onHeapKeys = new ArrayList<>(); - List diskTierKeys = new ArrayList<>(); for (int iter = 0; iter < numOfItems1; iter++) { String key = UUID.randomUUID().toString(); LoadAwareCacheLoader tieredCacheLoader = getLoadAwareCacheLoader(); tieredSpilloverCache.computeIfAbsent(key, tieredCacheLoader); } - tieredSpilloverCache.getOnHeapCache().keys().forEach(onHeapKeys::add); - tieredSpilloverCache.getDiskCache().keys().forEach(diskTierKeys::add); - - assertEquals(tieredSpilloverCache.getOnHeapCache().count(), onHeapKeys.size()); - assertEquals(tieredSpilloverCache.getDiskCache().count(), diskTierKeys.size()); + // Verify on heap cache size. + assertEquals(onHeapCacheSize, tieredSpilloverCache.getOnHeapCache().count()); + // Verify disk cache size. + assertEquals(numOfItems1 - onHeapCacheSize, tieredSpilloverCache.getDiskCache().count()); } public void testWithFactoryCreationWithOnHeapCacheNotPresent() { @@ -180,6 +177,11 @@ public void testWithFactoryCreationWithOnHeapCacheNotPresent() { .getKey(), onHeapCacheSize * keyValueSize + "b" ) + .put( + CacheSettings.getConcreteStoreNameSettingForCacheType(CacheType.INDICES_REQUEST_CACHE).getKey(), + TieredSpilloverCache.TieredSpilloverCacheFactory.TIERED_SPILLOVER_CACHE_NAME + ) + .put(FeatureFlags.PLUGGABLE_CACHE, "true") .build(); IllegalArgumentException ex = assertThrows( From dda3e3d78f9dc465b07c7641c0277599dc67701a Mon Sep 17 00:00:00 2001 From: Sagar <99425694+sgup432@users.noreply.github.com> Date: Mon, 18 Mar 2024 22:20:52 -0700 Subject: [PATCH 05/38] [Tiered Caching] Clear up disk cache(ehcache) files during node shutdown (#12734) * Adding logic to clear up the disk cache files during close() * Adding logic to update entries count after invalidateAll() * Removing unneeded system log statement * Added comment in test for readability * Fixing issue where we were sending compacted byte[] array to ehcache but calculating size with padded byte[] --------- Signed-off-by: Sagar Upadhyaya Signed-off-by: Sagar <99425694+sgup432@users.noreply.github.com> --- .../core/common/bytes/BytesReference.java | 5 +++ .../cache/store/disk/EhcacheDiskCache.java | 19 +++++++++- .../store/disk/EhCacheDiskCacheTests.java | 35 +++++++++++++++++++ .../serializer/BytesReferenceSerializer.java | 2 +- .../indices/IndicesRequestCache.java | 3 +- 5 files changed, 61 insertions(+), 3 deletions(-) diff --git a/libs/core/src/main/java/org/opensearch/core/common/bytes/BytesReference.java b/libs/core/src/main/java/org/opensearch/core/common/bytes/BytesReference.java index 8cb65c9feb1ca..6b60e7448cd03 100644 --- a/libs/core/src/main/java/org/opensearch/core/common/bytes/BytesReference.java +++ b/libs/core/src/main/java/org/opensearch/core/common/bytes/BytesReference.java @@ -82,6 +82,11 @@ static byte[] toBytes(BytesReference reference) { return ArrayUtil.copyOfSubArray(bytesRef.bytes, bytesRef.offset, bytesRef.offset + bytesRef.length); } + static byte[] toBytesWithoutCompact(BytesReference reference) { + final BytesRef bytesRef = reference.toBytesRef(); + return bytesRef.bytes; + } + /** * Returns an array of byte buffers from the given BytesReference. */ diff --git a/plugins/cache-ehcache/src/main/java/org/opensearch/cache/store/disk/EhcacheDiskCache.java b/plugins/cache-ehcache/src/main/java/org/opensearch/cache/store/disk/EhcacheDiskCache.java index d83a55e60fd2b..edb2c900be46c 100644 --- a/plugins/cache-ehcache/src/main/java/org/opensearch/cache/store/disk/EhcacheDiskCache.java +++ b/plugins/cache-ehcache/src/main/java/org/opensearch/cache/store/disk/EhcacheDiskCache.java @@ -10,6 +10,7 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; +import org.apache.logging.log4j.message.ParameterizedMessage; import org.opensearch.OpenSearchException; import org.opensearch.cache.EhcacheDiskCacheSettings; import org.opensearch.common.SuppressForbidden; @@ -28,9 +29,14 @@ import org.opensearch.common.settings.Setting; import org.opensearch.common.settings.Settings; import org.opensearch.common.unit.TimeValue; +import org.opensearch.common.util.io.IOUtils; import java.io.File; +import java.io.IOException; import java.nio.ByteBuffer; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; import java.time.Duration; import java.util.Arrays; import java.util.Iterator; @@ -363,7 +369,10 @@ public void invalidate(K key) { } @Override - public void invalidateAll() {} + public void invalidateAll() { + cache.clear(); + this.entries.dec(this.entries.count()); // reset to zero. + } /** * Provides a way to iterate over disk cache keys. @@ -389,13 +398,21 @@ public void refresh() { } @Override + @SuppressForbidden(reason = "Ehcache uses File.io") public void close() { cacheManager.removeCache(this.diskCacheAlias); cacheManager.close(); try { cacheManager.destroyCache(this.diskCacheAlias); + // Delete all the disk cache related files/data + Path ehcacheDirectory = Paths.get(this.storagePath); + if (Files.exists(ehcacheDirectory)) { + IOUtils.rm(ehcacheDirectory); + } } catch (CachePersistenceException e) { throw new OpenSearchException("Exception occurred while destroying ehcache and associated data", e); + } catch (IOException e) { + logger.error(() -> new ParameterizedMessage("Failed to delete ehcache disk cache data under path: {}", this.storagePath)); } } diff --git a/plugins/cache-ehcache/src/test/java/org/opensearch/cache/store/disk/EhCacheDiskCacheTests.java b/plugins/cache-ehcache/src/test/java/org/opensearch/cache/store/disk/EhCacheDiskCacheTests.java index ac6bf6d6e7c2c..8c0ab62baee54 100644 --- a/plugins/cache-ehcache/src/test/java/org/opensearch/cache/store/disk/EhCacheDiskCacheTests.java +++ b/plugins/cache-ehcache/src/test/java/org/opensearch/cache/store/disk/EhCacheDiskCacheTests.java @@ -554,6 +554,41 @@ public void testEhcacheKeyIteratorWithRemove() throws IOException { } + public void testInvalidateAll() throws Exception { + Settings settings = Settings.builder().build(); + MockRemovalListener removalListener = new MockRemovalListener<>(); + try (NodeEnvironment env = newNodeEnvironment(settings)) { + ICache ehcacheTest = new EhcacheDiskCache.Builder().setThreadPoolAlias("ehcacheTest") + .setStoragePath(env.nodePaths()[0].indicesPath.toString() + "/request_cache") + .setIsEventListenerModeSync(true) + .setKeyType(String.class) + .setValueType(String.class) + .setKeySerializer(new StringSerializer()) + .setValueSerializer(new StringSerializer()) + .setCacheType(CacheType.INDICES_REQUEST_CACHE) + .setSettings(settings) + .setExpireAfterAccess(TimeValue.MAX_VALUE) + .setMaximumWeightInBytes(CACHE_SIZE_IN_BYTES) + .setRemovalListener(removalListener) + .build(); + int randomKeys = randomIntBetween(10, 100); + Map keyValueMap = new HashMap<>(); + for (int i = 0; i < randomKeys; i++) { + keyValueMap.put(UUID.randomUUID().toString(), UUID.randomUUID().toString()); + } + for (Map.Entry entry : keyValueMap.entrySet()) { + ehcacheTest.put(entry.getKey(), entry.getValue()); + } + ehcacheTest.invalidateAll(); // clear all the entries. + for (Map.Entry entry : keyValueMap.entrySet()) { + // Verify that value is null for a removed entry. + assertNull(ehcacheTest.get(entry.getKey())); + } + assertEquals(0, ehcacheTest.count()); + ehcacheTest.close(); + } + } + public void testBasicGetAndPutBytesReference() throws Exception { Settings settings = Settings.builder().build(); try (NodeEnvironment env = newNodeEnvironment(settings)) { diff --git a/server/src/main/java/org/opensearch/common/cache/serializer/BytesReferenceSerializer.java b/server/src/main/java/org/opensearch/common/cache/serializer/BytesReferenceSerializer.java index d1cd872f5801f..c26e1191888df 100644 --- a/server/src/main/java/org/opensearch/common/cache/serializer/BytesReferenceSerializer.java +++ b/server/src/main/java/org/opensearch/common/cache/serializer/BytesReferenceSerializer.java @@ -24,7 +24,7 @@ public BytesReferenceSerializer() {} @Override public byte[] serialize(BytesReference object) { - return BytesReference.toBytes(object); + return BytesReference.toBytesWithoutCompact(object); } @Override diff --git a/server/src/main/java/org/opensearch/indices/IndicesRequestCache.java b/server/src/main/java/org/opensearch/indices/IndicesRequestCache.java index 6f1181167b6b2..34c8d6cf5e840 100644 --- a/server/src/main/java/org/opensearch/indices/IndicesRequestCache.java +++ b/server/src/main/java/org/opensearch/indices/IndicesRequestCache.java @@ -185,8 +185,9 @@ public final class IndicesRequestCache implements RemovalListener Date: Tue, 19 Mar 2024 12:23:00 +0530 Subject: [PATCH 06/38] Introduce remote store path type in customData in IndexMetadata (#12607) Signed-off-by: Ashish Singh --- .../cluster/metadata/IndexMetadata.java | 1 + .../metadata/MetadataCreateIndexService.java | 36 ++++++-- .../common/settings/ClusterSettings.java | 2 + .../index/remote/RemoteStorePathResolver.java | 30 ++++++ .../index/remote/RemoteStorePathType.java | 36 ++++++++ .../opensearch/indices/IndicesService.java | 13 +++ .../MetadataCreateIndexServiceTests.java | 92 ++++++++++++++++++- 7 files changed, 201 insertions(+), 9 deletions(-) create mode 100644 server/src/main/java/org/opensearch/index/remote/RemoteStorePathResolver.java create mode 100644 server/src/main/java/org/opensearch/index/remote/RemoteStorePathType.java diff --git a/server/src/main/java/org/opensearch/cluster/metadata/IndexMetadata.java b/server/src/main/java/org/opensearch/cluster/metadata/IndexMetadata.java index 03784df509ed6..80b78cfe154f1 100644 --- a/server/src/main/java/org/opensearch/cluster/metadata/IndexMetadata.java +++ b/server/src/main/java/org/opensearch/cluster/metadata/IndexMetadata.java @@ -635,6 +635,7 @@ public static APIBlock readFrom(StreamInput input) throws IOException { static final String KEY_ROLLOVER_INFOS = "rollover_info"; static final String KEY_SYSTEM = "system"; public static final String KEY_PRIMARY_TERMS = "primary_terms"; + public static final String REMOTE_STORE_CUSTOM_KEY = "remote_store"; public static final String INDEX_STATE_FILE_PREFIX = "state-"; diff --git a/server/src/main/java/org/opensearch/cluster/metadata/MetadataCreateIndexService.java b/server/src/main/java/org/opensearch/cluster/metadata/MetadataCreateIndexService.java index acc2f3a294745..f117d1a4a11a2 100644 --- a/server/src/main/java/org/opensearch/cluster/metadata/MetadataCreateIndexService.java +++ b/server/src/main/java/org/opensearch/cluster/metadata/MetadataCreateIndexService.java @@ -88,6 +88,8 @@ import org.opensearch.index.mapper.MapperService; import org.opensearch.index.mapper.MapperService.MergeReason; import org.opensearch.index.query.QueryShardContext; +import org.opensearch.index.remote.RemoteStorePathResolver; +import org.opensearch.index.remote.RemoteStorePathType; import org.opensearch.index.shard.IndexSettingProvider; import org.opensearch.index.translog.Translog; import org.opensearch.indices.IndexCreationException; @@ -167,6 +169,9 @@ public class MetadataCreateIndexService { private final ClusterManagerTaskThrottler.ThrottlingKey createIndexTaskKey; private AwarenessReplicaBalance awarenessReplicaBalance; + @Nullable + private final RemoteStorePathResolver remoteStorePathResolver; + public MetadataCreateIndexService( final Settings settings, final ClusterService clusterService, @@ -198,6 +203,9 @@ public MetadataCreateIndexService( // Task is onboarded for throttling, it will get retried from associated TransportClusterManagerNodeAction. createIndexTaskKey = clusterService.registerClusterManagerTask(ClusterManagerTaskKeys.CREATE_INDEX_KEY, true); + remoteStorePathResolver = isRemoteDataAttributePresent(settings) + ? new RemoteStorePathResolver(clusterService.getClusterSettings()) + : null; } /** @@ -498,7 +506,8 @@ private ClusterState applyCreateIndexWithTemporaryService( temporaryIndexMeta.getSettings(), temporaryIndexMeta.getRoutingNumShards(), sourceMetadata, - temporaryIndexMeta.isSystem() + temporaryIndexMeta.isSystem(), + temporaryIndexMeta.getCustomData() ); } catch (Exception e) { logger.info("failed to build index metadata [{}]", request.index()); @@ -522,10 +531,11 @@ private ClusterState applyCreateIndexWithTemporaryService( /** * Given a state and index settings calculated after applying templates, validate metadata for - * the new index, returning an {@link IndexMetadata} for the new index + * the new index, returning an {@link IndexMetadata} for the new index. + *

+ * The access level of the method changed to default level for visibility to test. */ - private IndexMetadata buildAndValidateTemporaryIndexMetadata( - final ClusterState currentState, + IndexMetadata buildAndValidateTemporaryIndexMetadata( final Settings aggregatedIndexSettings, final CreateIndexClusterStateUpdateRequest request, final int routingNumShards @@ -544,6 +554,11 @@ private IndexMetadata buildAndValidateTemporaryIndexMetadata( tmpImdBuilder.settings(indexSettings); tmpImdBuilder.system(isSystem); + if (remoteStorePathResolver != null) { + String pathType = remoteStorePathResolver.resolveType().toString(); + tmpImdBuilder.putCustom(IndexMetadata.REMOTE_STORE_CUSTOM_KEY, Map.of(RemoteStorePathType.NAME, pathType)); + } + // Set up everything, now locally create the index to see that things are ok, and apply IndexMetadata tempMetadata = tmpImdBuilder.build(); validateActiveShardCount(request.waitForActiveShards(), tempMetadata); @@ -582,7 +597,7 @@ private ClusterState applyCreateIndexRequestWithV1Templates( clusterService.getClusterSettings() ); int routingNumShards = getIndexNumberOfRoutingShards(aggregatedIndexSettings, null); - IndexMetadata tmpImd = buildAndValidateTemporaryIndexMetadata(currentState, aggregatedIndexSettings, request, routingNumShards); + IndexMetadata tmpImd = buildAndValidateTemporaryIndexMetadata(aggregatedIndexSettings, request, routingNumShards); return applyCreateIndexWithTemporaryService( currentState, @@ -647,7 +662,7 @@ private ClusterState applyCreateIndexRequestWithV2Template( clusterService.getClusterSettings() ); int routingNumShards = getIndexNumberOfRoutingShards(aggregatedIndexSettings, null); - IndexMetadata tmpImd = buildAndValidateTemporaryIndexMetadata(currentState, aggregatedIndexSettings, request, routingNumShards); + IndexMetadata tmpImd = buildAndValidateTemporaryIndexMetadata(aggregatedIndexSettings, request, routingNumShards); return applyCreateIndexWithTemporaryService( currentState, @@ -728,7 +743,7 @@ private ClusterState applyCreateIndexRequestWithExistingMetadata( clusterService.getClusterSettings() ); final int routingNumShards = getIndexNumberOfRoutingShards(aggregatedIndexSettings, sourceMetadata); - IndexMetadata tmpImd = buildAndValidateTemporaryIndexMetadata(currentState, aggregatedIndexSettings, request, routingNumShards); + IndexMetadata tmpImd = buildAndValidateTemporaryIndexMetadata(aggregatedIndexSettings, request, routingNumShards); return applyCreateIndexWithTemporaryService( currentState, @@ -1147,7 +1162,8 @@ static IndexMetadata buildIndexMetadata( Settings indexSettings, int routingNumShards, @Nullable IndexMetadata sourceMetadata, - boolean isSystem + boolean isSystem, + Map customData ) { IndexMetadata.Builder indexMetadataBuilder = createIndexMetadataBuilder(indexName, sourceMetadata, indexSettings, routingNumShards); indexMetadataBuilder.system(isSystem); @@ -1168,6 +1184,10 @@ static IndexMetadata buildIndexMetadata( indexMetadataBuilder.putAlias(aliases.get(i)); } + for (Map.Entry entry : customData.entrySet()) { + indexMetadataBuilder.putCustom(entry.getKey(), entry.getValue()); + } + indexMetadataBuilder.state(IndexMetadata.State.OPEN); return indexMetadataBuilder.build(); } diff --git a/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java b/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java index 6b4be45929553..730a330b6ca60 100644 --- a/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java +++ b/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java @@ -713,6 +713,8 @@ public void apply(Settings value, Settings current, Settings previous) { IoBasedAdmissionControllerSettings.SEARCH_IO_USAGE_LIMIT, IoBasedAdmissionControllerSettings.INDEXING_IO_USAGE_LIMIT, IndicesService.CLUSTER_INDEX_RESTRICT_REPLICATION_TYPE_SETTING, + IndicesService.CLUSTER_REMOTE_STORE_PATH_PREFIX_TYPE_SETTING, + // Concurrent segment search settings SearchService.CLUSTER_CONCURRENT_SEGMENT_SEARCH_SETTING, SearchService.CONCURRENT_SEGMENT_SEARCH_TARGET_MAX_SLICE_COUNT_SETTING diff --git a/server/src/main/java/org/opensearch/index/remote/RemoteStorePathResolver.java b/server/src/main/java/org/opensearch/index/remote/RemoteStorePathResolver.java new file mode 100644 index 0000000000000..6e8126fcce0ca --- /dev/null +++ b/server/src/main/java/org/opensearch/index/remote/RemoteStorePathResolver.java @@ -0,0 +1,30 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.index.remote; + +import org.opensearch.common.settings.ClusterSettings; +import org.opensearch.indices.IndicesService; + +/** + * Determines the {@link RemoteStorePathType} at the time of index metadata creation. + * + * @opensearch.internal + */ +public class RemoteStorePathResolver { + + private final ClusterSettings clusterSettings; + + public RemoteStorePathResolver(ClusterSettings clusterSettings) { + this.clusterSettings = clusterSettings; + } + + public RemoteStorePathType resolveType() { + return clusterSettings.get(IndicesService.CLUSTER_REMOTE_STORE_PATH_PREFIX_TYPE_SETTING); + } +} diff --git a/server/src/main/java/org/opensearch/index/remote/RemoteStorePathType.java b/server/src/main/java/org/opensearch/index/remote/RemoteStorePathType.java new file mode 100644 index 0000000000000..a64e07ab1f66f --- /dev/null +++ b/server/src/main/java/org/opensearch/index/remote/RemoteStorePathType.java @@ -0,0 +1,36 @@ +/* + * 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.index.remote; + +import java.util.Locale; + +/** + * Enumerates the types of remote store paths resolution techniques supported by OpenSearch. + * For more information, see Github issue #12567. + * + * @opensearch.internal + */ +public enum RemoteStorePathType { + + FIXED, + HASHED_PREFIX; + + public static RemoteStorePathType parseString(String remoteStoreBlobPathType) { + try { + return RemoteStorePathType.valueOf(remoteStoreBlobPathType.toUpperCase(Locale.ROOT)); + } catch (IllegalArgumentException e) { + throw new IllegalArgumentException("Could not parse RemoteStorePathType for [" + remoteStoreBlobPathType + "]"); + } + } + + /** + * This string is used as key for storing information in the custom data in index settings. + */ + public static final String NAME = "path_type"; +} diff --git a/server/src/main/java/org/opensearch/indices/IndicesService.java b/server/src/main/java/org/opensearch/indices/IndicesService.java index 40c10e3a2fe96..0e64894e6f708 100644 --- a/server/src/main/java/org/opensearch/indices/IndicesService.java +++ b/server/src/main/java/org/opensearch/indices/IndicesService.java @@ -123,6 +123,7 @@ import org.opensearch.index.query.QueryRewriteContext; import org.opensearch.index.recovery.RecoveryStats; import org.opensearch.index.refresh.RefreshStats; +import org.opensearch.index.remote.RemoteStorePathType; import org.opensearch.index.remote.RemoteStoreStatsTrackerFactory; import org.opensearch.index.search.stats.SearchStats; import org.opensearch.index.seqno.RetentionLeaseStats; @@ -313,6 +314,18 @@ public class IndicesService extends AbstractLifecycleComponent Property.Final ); + /** + * This setting is used to set the remote store blob store path prefix strategy. This setting is effective only for + * remote store enabled cluster. + */ + public static final Setting CLUSTER_REMOTE_STORE_PATH_PREFIX_TYPE_SETTING = new Setting<>( + "cluster.remote_store.index.path.prefix.type", + RemoteStorePathType.FIXED.toString(), + RemoteStorePathType::parseString, + Property.NodeScope, + Property.Dynamic + ); + /** * The node's settings. */ diff --git a/server/src/test/java/org/opensearch/cluster/metadata/MetadataCreateIndexServiceTests.java b/server/src/test/java/org/opensearch/cluster/metadata/MetadataCreateIndexServiceTests.java index cc605878119a2..cf4de32890a2a 100644 --- a/server/src/test/java/org/opensearch/cluster/metadata/MetadataCreateIndexServiceTests.java +++ b/server/src/test/java/org/opensearch/cluster/metadata/MetadataCreateIndexServiceTests.java @@ -71,6 +71,7 @@ import org.opensearch.index.IndexSettings; import org.opensearch.index.mapper.MapperService; import org.opensearch.index.query.QueryShardContext; +import org.opensearch.index.remote.RemoteStorePathType; import org.opensearch.index.translog.Translog; import org.opensearch.indices.IndexCreationException; import org.opensearch.indices.IndicesService; @@ -1563,13 +1564,102 @@ public void testBuildIndexMetadata() { .put(SETTING_NUMBER_OF_SHARDS, 1) .build(); List aliases = singletonList(AliasMetadata.builder("alias1").build()); - IndexMetadata indexMetadata = buildIndexMetadata("test", aliases, () -> null, indexSettings, 4, sourceIndexMetadata, false); + IndexMetadata indexMetadata = buildIndexMetadata( + "test", + aliases, + () -> null, + indexSettings, + 4, + sourceIndexMetadata, + false, + new HashMap<>() + ); assertThat(indexMetadata.getAliases().size(), is(1)); assertThat(indexMetadata.getAliases().keySet().iterator().next(), is("alias1")); assertThat("The source index primary term must be used", indexMetadata.primaryTerm(0), is(3L)); } + /** + * This test checks if the cluster is a remote store cluster then we populate custom data for remote settings in + * index metadata of the underlying index. This captures information around the resolution pattern of the path for + * remote segments and translog. + */ + public void testRemoteCustomData() { + // Case 1 - Remote store is not enabled + IndexMetadata indexMetadata = testRemoteCustomData(false, randomFrom(RemoteStorePathType.values())); + assertNull(indexMetadata.getCustomData(IndexMetadata.REMOTE_STORE_CUSTOM_KEY)); + + // Case 2 - cluster.remote_store.index.path.prefix.optimised=fixed (default value) + indexMetadata = testRemoteCustomData(true, RemoteStorePathType.FIXED); + validateRemoteCustomData( + indexMetadata.getCustomData(IndexMetadata.REMOTE_STORE_CUSTOM_KEY), + RemoteStorePathType.NAME, + RemoteStorePathType.FIXED.toString() + ); + + // Case 3 - cluster.remote_store.index.path.prefix.optimised=hashed_prefix + indexMetadata = testRemoteCustomData(true, RemoteStorePathType.HASHED_PREFIX); + validateRemoteCustomData( + indexMetadata.getCustomData(IndexMetadata.REMOTE_STORE_CUSTOM_KEY), + RemoteStorePathType.NAME, + RemoteStorePathType.HASHED_PREFIX.toString() + ); + } + + private IndexMetadata testRemoteCustomData(boolean remoteStoreEnabled, RemoteStorePathType remoteStorePathType) { + Settings.Builder settingsBuilder = Settings.builder(); + if (remoteStoreEnabled) { + settingsBuilder.put(NODE_ATTRIBUTES.getKey() + REMOTE_STORE_SEGMENT_REPOSITORY_NAME_ATTRIBUTE_KEY, "test"); + } + settingsBuilder.put(IndicesService.CLUSTER_REMOTE_STORE_PATH_PREFIX_TYPE_SETTING.getKey(), remoteStorePathType.toString()); + Settings settings = settingsBuilder.build(); + + ClusterService clusterService = mock(ClusterService.class); + Metadata metadata = Metadata.builder() + .transientSettings(Settings.builder().put(Metadata.DEFAULT_REPLICA_COUNT_SETTING.getKey(), 1).build()) + .build(); + ClusterState clusterState = ClusterState.builder(org.opensearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)) + .metadata(metadata) + .build(); + ClusterSettings clusterSettings = new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); + when(clusterService.getSettings()).thenReturn(settings); + when(clusterService.getClusterSettings()).thenReturn(clusterSettings); + when(clusterService.state()).thenReturn(clusterState); + + ThreadPool threadPool = new TestThreadPool(getTestName()); + MetadataCreateIndexService metadataCreateIndexService = new MetadataCreateIndexService( + settings, + clusterService, + null, + null, + null, + createTestShardLimitService(randomIntBetween(1, 1000), false, clusterService), + new Environment(Settings.builder().put("path.home", "dummy").build(), null), + IndexScopedSettings.DEFAULT_SCOPED_SETTINGS, + threadPool, + null, + new SystemIndices(Collections.emptyMap()), + true, + new AwarenessReplicaBalance(settings, clusterService.getClusterSettings()) + ); + CreateIndexClusterStateUpdateRequest request = new CreateIndexClusterStateUpdateRequest("create index", "test", "test"); + Settings indexSettings = Settings.builder() + .put("index.version.created", Version.CURRENT) + .put(INDEX_NUMBER_OF_SHARDS_SETTING.getKey(), 3) + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1) + .build(); + + IndexMetadata indexMetadata = metadataCreateIndexService.buildAndValidateTemporaryIndexMetadata(indexSettings, request, 0); + threadPool.shutdown(); + return indexMetadata; + } + + private void validateRemoteCustomData(Map customData, String expectedKey, String expectedValue) { + assertTrue(customData.containsKey(expectedKey)); + assertEquals(expectedValue, customData.get(expectedKey)); + } + public void testGetIndexNumberOfRoutingShardsWithNullSourceIndex() { Settings indexSettings = Settings.builder() .put("index.version.created", Version.CURRENT) From 7a6f8b0eead5371fa18d49534ac76edf199e9c4c Mon Sep 17 00:00:00 2001 From: Andriy Redko Date: Tue, 19 Mar 2024 07:16:07 -0400 Subject: [PATCH 07/38] Bump mockito & bytebuddy dependencies (#12720) Signed-off-by: Andriy Redko --- buildSrc/version.properties | 4 ++-- .../org/opensearch/transport/RemoteClusterClientTests.java | 1 + 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/buildSrc/version.properties b/buildSrc/version.properties index 6da095473b520..8705588babe97 100644 --- a/buildSrc/version.properties +++ b/buildSrc/version.properties @@ -55,9 +55,9 @@ bouncycastle=1.77 randomizedrunner = 2.7.1 junit = 4.13.2 hamcrest = 2.1 -mockito = 5.10.0 +mockito = 5.11.0 objenesis = 3.2 -bytebuddy = 1.14.7 +bytebuddy = 1.14.9 # benchmark dependencies jmh = 1.35 diff --git a/server/src/test/java/org/opensearch/transport/RemoteClusterClientTests.java b/server/src/test/java/org/opensearch/transport/RemoteClusterClientTests.java index f3b7f9916d460..59c0206a87fb3 100644 --- a/server/src/test/java/org/opensearch/transport/RemoteClusterClientTests.java +++ b/server/src/test/java/org/opensearch/transport/RemoteClusterClientTests.java @@ -63,6 +63,7 @@ public void tearDown() throws Exception { ThreadPool.terminate(threadPool, 10, TimeUnit.SECONDS); } + @AwaitsFix(bugUrl = "https://github.com/opensearch-project/OpenSearch/issues/12338") public void testConnectAndExecuteRequest() throws Exception { Settings remoteSettings = Settings.builder().put(ClusterName.CLUSTER_NAME_SETTING.getKey(), "foo_bar_cluster").build(); try ( From d4e1ab135dfcb058b476fce09a669f4be5854702 Mon Sep 17 00:00:00 2001 From: Sooraj Sinha <81695996+soosinha@users.noreply.github.com> Date: Tue, 19 Mar 2024 21:52:43 +0530 Subject: [PATCH 08/38] Election scheduler should be cancelled after cluster state publication (#11699) Signed-off-by: Sooraj Sinha --- CHANGELOG.md | 1 + .../cluster/coordination/Coordinator.java | 20 +++++++++++++++---- .../coordination/CoordinatorTests.java | 4 ++-- .../AbstractCoordinatorTestCase.java | 7 +++++++ 4 files changed, 26 insertions(+), 6 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index c4558cf3fe251..3fb141e189bd3 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -146,6 +146,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Allow composite aggregation to run under a parent filter aggregation ([#11499](https://github.com/opensearch-project/OpenSearch/pull/11499)) - Quickly compute terms aggregations when the top-level query is functionally match-all for a segment ([#11643](https://github.com/opensearch-project/OpenSearch/pull/11643)) - Mark fuzzy filter GA and remove experimental setting ([12631](https://github.com/opensearch-project/OpenSearch/pull/12631)) +- Keep the election scheduler open until cluster state has been applied ([#11699](https://github.com/opensearch-project/OpenSearch/pull/11699)) ### Deprecated diff --git a/server/src/main/java/org/opensearch/cluster/coordination/Coordinator.java b/server/src/main/java/org/opensearch/cluster/coordination/Coordinator.java index 5a07f964f94a4..3d74feddfa261 100644 --- a/server/src/main/java/org/opensearch/cluster/coordination/Coordinator.java +++ b/server/src/main/java/org/opensearch/cluster/coordination/Coordinator.java @@ -386,6 +386,7 @@ public void onFailure(String source, Exception e) { @Override public void onSuccess(String source) { + closePrevotingAndElectionScheduler(); applyListener.onResponse(null); } }); @@ -472,17 +473,29 @@ private static Optional joinWithDestination(Optional lastJoin, Disco } private void closePrevotingAndElectionScheduler() { + closePrevoting(); + closeElectionScheduler(); + } + + private void closePrevoting() { if (prevotingRound != null) { prevotingRound.close(); prevotingRound = null; } + } + private void closeElectionScheduler() { if (electionScheduler != null) { electionScheduler.close(); electionScheduler = null; } } + // package-visible for testing + boolean isElectionSchedulerRunning() { + return electionScheduler != null; + } + private void updateMaxTermSeen(final long term) { synchronized (mutex) { maxTermSeen = Math.max(maxTermSeen, term); @@ -724,7 +737,7 @@ void becomeLeader(String method) { lastKnownLeader = Optional.of(getLocalNode()); peerFinder.deactivate(getLocalNode()); clusterFormationFailureHelper.stop(); - closePrevotingAndElectionScheduler(); + closePrevoting(); preVoteCollector.update(getPreVoteResponse(), getLocalNode()); assert leaderChecker.leader() == null : leaderChecker.leader(); @@ -761,7 +774,7 @@ void becomeFollower(String method, DiscoveryNode leaderNode) { lastKnownLeader = Optional.of(leaderNode); peerFinder.deactivate(leaderNode); clusterFormationFailureHelper.stop(); - closePrevotingAndElectionScheduler(); + closePrevoting(); cancelActivePublication("become follower: " + method); preVoteCollector.update(getPreVoteResponse(), leaderNode); @@ -927,7 +940,6 @@ public void invariant() { assert lastKnownLeader.isPresent() && lastKnownLeader.get().equals(getLocalNode()); assert joinAccumulator instanceof JoinHelper.LeaderJoinAccumulator; assert peerFinderLeader.equals(lastKnownLeader) : peerFinderLeader; - assert electionScheduler == null : electionScheduler; assert prevotingRound == null : prevotingRound; assert becomingClusterManager || getStateForClusterManagerService().nodes().getClusterManagerNodeId() != null : getStateForClusterManagerService(); @@ -972,7 +984,6 @@ assert getLocalNode().equals(applierState.nodes().getClusterManagerNode()) assert lastKnownLeader.isPresent() && (lastKnownLeader.get().equals(getLocalNode()) == false); assert joinAccumulator instanceof JoinHelper.FollowerJoinAccumulator; assert peerFinderLeader.equals(lastKnownLeader) : peerFinderLeader; - assert electionScheduler == null : electionScheduler; assert prevotingRound == null : prevotingRound; assert getStateForClusterManagerService().nodes().getClusterManagerNodeId() == null : getStateForClusterManagerService(); assert leaderChecker.currentNodeIsClusterManager() == false; @@ -1693,6 +1704,7 @@ public void onSuccess(String source) { updateMaxTermSeen(getCurrentTerm()); if (mode == Mode.LEADER) { + closePrevotingAndElectionScheduler(); // if necessary, abdicate to another node or improve the voting configuration boolean attemptReconfiguration = true; final ClusterState state = getLastAcceptedState(); // committed state diff --git a/server/src/test/java/org/opensearch/cluster/coordination/CoordinatorTests.java b/server/src/test/java/org/opensearch/cluster/coordination/CoordinatorTests.java index a3129655148ab..5eeebd2588416 100644 --- a/server/src/test/java/org/opensearch/cluster/coordination/CoordinatorTests.java +++ b/server/src/test/java/org/opensearch/cluster/coordination/CoordinatorTests.java @@ -270,7 +270,7 @@ public void testNodesJoinAfterStableCluster() { public void testExpandsConfigurationWhenGrowingFromOneNodeToThreeButDoesNotShrink() { try (Cluster cluster = new Cluster(1)) { cluster.runRandomly(); - cluster.stabilise(); + cluster.stabilise(DEFAULT_STABILISATION_TIME * 2); final ClusterNode leader = cluster.getAnyLeader(); @@ -1750,7 +1750,7 @@ public void testDoesNotPerformElectionWhenRestartingFollower() { public void testImproveConfigurationPerformsVotingConfigExclusionStateCheck() { try (Cluster cluster = new Cluster(1)) { cluster.runRandomly(); - cluster.stabilise(); + cluster.stabilise(DEFAULT_STABILISATION_TIME * 2); final Coordinator coordinator = cluster.getAnyLeader().coordinator; final ClusterState currentState = coordinator.getLastAcceptedState(); diff --git a/test/framework/src/main/java/org/opensearch/cluster/coordination/AbstractCoordinatorTestCase.java b/test/framework/src/main/java/org/opensearch/cluster/coordination/AbstractCoordinatorTestCase.java index 28d7706fb1493..0754cc1793dc8 100644 --- a/test/framework/src/main/java/org/opensearch/cluster/coordination/AbstractCoordinatorTestCase.java +++ b/test/framework/src/main/java/org/opensearch/cluster/coordination/AbstractCoordinatorTestCase.java @@ -47,6 +47,7 @@ import org.opensearch.cluster.OpenSearchAllocationTestCase; import org.opensearch.cluster.coordination.AbstractCoordinatorTestCase.Cluster.ClusterNode; import org.opensearch.cluster.coordination.CoordinationMetadata.VotingConfiguration; +import org.opensearch.cluster.coordination.Coordinator.Mode; import org.opensearch.cluster.coordination.LinearizabilityChecker.History; import org.opensearch.cluster.coordination.LinearizabilityChecker.SequentialSpec; import org.opensearch.cluster.coordination.PersistedStateRegistry.PersistedStateType; @@ -653,6 +654,12 @@ void stabilise(long stabilisationDurationMillis) { leader.getLastAppliedClusterState().getNodes().nodeExists(nodeId) ); } + if (clusterNode.coordinator.getMode() == Mode.LEADER || clusterNode.coordinator.getMode() == Mode.FOLLOWER) { + assertFalse( + "Election scheduler should stop after cluster has stabilised", + clusterNode.coordinator.isElectionSchedulerRunning() + ); + } } final Set connectedNodeIds = clusterNodes.stream() From 479b65bfd4337ede96105739f5611ae9e8dbf971 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 19 Mar 2024 21:16:50 -0500 Subject: [PATCH 09/38] Bump peter-evans/create-pull-request from 5 to 6 (#12724) Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> Co-authored-by: dependabot[bot] --- .github/workflows/version.yml | 6 +++--- CHANGELOG.md | 1 + 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/.github/workflows/version.yml b/.github/workflows/version.yml index be2a89ac931e9..7f120b65d7c2e 100644 --- a/.github/workflows/version.yml +++ b/.github/workflows/version.yml @@ -62,7 +62,7 @@ jobs: - name: Create PR for BASE id: base_pr - uses: peter-evans/create-pull-request@v5 + uses: peter-evans/create-pull-request@v6 with: base: ${{ env.BASE }} branch: 'create-pull-request/patch-${{ env.BASE }}' @@ -88,7 +88,7 @@ jobs: - name: Create PR for BASE_X id: base_x_pr - uses: peter-evans/create-pull-request@v5 + uses: peter-evans/create-pull-request@v6 with: base: ${{ env.BASE_X }} branch: 'create-pull-request/patch-${{ env.BASE_X }}' @@ -114,7 +114,7 @@ jobs: - name: Create PR for main id: main_pr - uses: peter-evans/create-pull-request@v5 + uses: peter-evans/create-pull-request@v6 with: base: main branch: 'create-pull-request/patch-main' diff --git a/CHANGELOG.md b/CHANGELOG.md index 3fb141e189bd3..4e5426797c69f 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -141,6 +141,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Bump `aws-sdk-java` from 2.20.55 to 2.20.86 ([#12251](https://github.com/opensearch-project/OpenSearch/pull/12251)) - Bump `reactor-netty` from 1.1.15 to 1.1.17 ([#12633](https://github.com/opensearch-project/OpenSearch/pull/12633)) - Bump `reactor` from 3.5.14 to 3.5.15 ([#12633](https://github.com/opensearch-project/OpenSearch/pull/12633)) +- Bump `peter-evans/create-pull-request` from 5 to 6 ([#12724](https://github.com/opensearch-project/OpenSearch/pull/12724)) ### Changed - Allow composite aggregation to run under a parent filter aggregation ([#11499](https://github.com/opensearch-project/OpenSearch/pull/11499)) From f4a8d2b752fd2b444cc3ab35fc463300cf31f870 Mon Sep 17 00:00:00 2001 From: Sagar <99425694+sgup432@users.noreply.github.com> Date: Tue, 19 Mar 2024 19:41:59 -0700 Subject: [PATCH 10/38] Fixing ehcache flaky test (#12764) * Fixing ehcache flaky test Signed-off-by: Sagar Upadhyaya * Adding a ehcache issue reference for thread leak issue Signed-off-by: Sagar Upadhyaya * Updating comment Signed-off-by: Sagar Upadhyaya --------- Signed-off-by: Sagar Upadhyaya --- .../store/disk/EhCacheDiskCacheTests.java | 44 +++++++++++++++++++ .../store/disk/EhcacheThreadLeakFilter.java | 29 ++++++++++++ 2 files changed, 73 insertions(+) create mode 100644 plugins/cache-ehcache/src/test/java/org/opensearch/cache/store/disk/EhcacheThreadLeakFilter.java diff --git a/plugins/cache-ehcache/src/test/java/org/opensearch/cache/store/disk/EhCacheDiskCacheTests.java b/plugins/cache-ehcache/src/test/java/org/opensearch/cache/store/disk/EhCacheDiskCacheTests.java index 8c0ab62baee54..3a98ad2fef6b1 100644 --- a/plugins/cache-ehcache/src/test/java/org/opensearch/cache/store/disk/EhCacheDiskCacheTests.java +++ b/plugins/cache-ehcache/src/test/java/org/opensearch/cache/store/disk/EhCacheDiskCacheTests.java @@ -8,6 +8,8 @@ package org.opensearch.cache.store.disk; +import com.carrotsearch.randomizedtesting.annotations.ThreadLeakFilters; + import org.opensearch.cache.EhcacheDiskCacheSettings; import org.opensearch.common.Randomness; import org.opensearch.common.cache.CacheType; @@ -47,6 +49,7 @@ import static org.opensearch.cache.EhcacheDiskCacheSettings.DISK_STORAGE_PATH_KEY; import static org.hamcrest.CoreMatchers.instanceOf; +@ThreadLeakFilters(filters = { EhcacheThreadLeakFilter.class }) public class EhCacheDiskCacheTests extends OpenSearchSingleNodeTestCase { private static final int CACHE_SIZE_IN_BYTES = 1024 * 101; @@ -633,6 +636,47 @@ public void testBasicGetAndPutBytesReference() throws Exception { } } + public void testInvalidate() throws Exception { + Settings settings = Settings.builder().build(); + MockRemovalListener removalListener = new MockRemovalListener<>(); + try (NodeEnvironment env = newNodeEnvironment(settings)) { + ICache ehcacheTest = new EhcacheDiskCache.Builder().setThreadPoolAlias("ehcacheTest") + .setStoragePath(env.nodePaths()[0].indicesPath.toString() + "/request_cache") + .setIsEventListenerModeSync(true) + .setKeyType(String.class) + .setKeySerializer(new StringSerializer()) + .setValueSerializer(new StringSerializer()) + .setValueType(String.class) + .setCacheType(CacheType.INDICES_REQUEST_CACHE) + .setSettings(settings) + .setExpireAfterAccess(TimeValue.MAX_VALUE) + .setMaximumWeightInBytes(CACHE_SIZE_IN_BYTES) + .setRemovalListener(removalListener) + .build(); + int randomKeys = randomIntBetween(10, 100); + Map keyValueMap = new HashMap<>(); + for (int i = 0; i < randomKeys; i++) { + keyValueMap.put(UUID.randomUUID().toString(), UUID.randomUUID().toString()); + } + for (Map.Entry entry : keyValueMap.entrySet()) { + ehcacheTest.put(entry.getKey(), entry.getValue()); + } + assertEquals(keyValueMap.size(), ehcacheTest.count()); + List removedKeyList = new ArrayList<>(); + for (Map.Entry entry : keyValueMap.entrySet()) { + if (randomBoolean()) { + removedKeyList.add(entry.getKey()); + ehcacheTest.invalidate(entry.getKey()); + } + } + for (String removedKey : removedKeyList) { + assertNull(ehcacheTest.get(removedKey)); + } + assertEquals(keyValueMap.size() - removedKeyList.size(), ehcacheTest.count()); + ehcacheTest.close(); + } + } + private static String generateRandomString(int length) { String characters = "ABCDEFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz0123456789"; StringBuilder randomString = new StringBuilder(length); diff --git a/plugins/cache-ehcache/src/test/java/org/opensearch/cache/store/disk/EhcacheThreadLeakFilter.java b/plugins/cache-ehcache/src/test/java/org/opensearch/cache/store/disk/EhcacheThreadLeakFilter.java new file mode 100644 index 0000000000000..6b54c3be10466 --- /dev/null +++ b/plugins/cache-ehcache/src/test/java/org/opensearch/cache/store/disk/EhcacheThreadLeakFilter.java @@ -0,0 +1,29 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.cache.store.disk; + +import com.carrotsearch.randomizedtesting.ThreadFilter; + +/** + * In Ehcache(as of 3.10.8), while calling remove/invalidate() on entries causes to start a daemon thread in the + * background to clean up the stale offheap memory associated with the disk cache. And this thread is not closed even + * after we try to close the cache or cache manager. Considering that it requires a node restart to switch between + * different cache plugins, this shouldn't be a problem for now. + * + * See: https://github.com/ehcache/ehcache3/issues/3204 + */ +public class EhcacheThreadLeakFilter implements ThreadFilter { + + private static final String OFFENDING_THREAD_NAME = "MappedByteBufferSource"; + + @Override + public boolean reject(Thread t) { + return t.getName().startsWith(OFFENDING_THREAD_NAME); + } +} From 1b4c76ddd9a4363312d5950655ecbc36c530bb96 Mon Sep 17 00:00:00 2001 From: gaobinlong Date: Wed, 20 Mar 2024 11:06:16 +0800 Subject: [PATCH 11/38] Update supported version for the primary_only parameter in force-merge API (#12657) * Update supported version for adding primary_only parameter to force-merge API Signed-off-by: Gao Binlong * Modify change log Signed-off-by: Gao Binlong * Remove some unused code Signed-off-by: Gao Binlong * Remove change log Signed-off-by: Gao Binlong * Fix test issue Signed-off-by: Gao Binlong --------- Signed-off-by: Gao Binlong --- .../20_wait_for_completion.yml | 33 ++++++++++-- .../indices/forcemerge/ForceMergeRequest.java | 4 +- .../forcemerge/ForceMergeRequestTests.java | 53 ++++++++++--------- 3 files changed, 60 insertions(+), 30 deletions(-) diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.forcemerge/20_wait_for_completion.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.forcemerge/20_wait_for_completion.yml index efa239547e84a..a0bddd1cbd13f 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.forcemerge/20_wait_for_completion.yml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.forcemerge/20_wait_for_completion.yml @@ -4,15 +4,17 @@ # will return a task immediately and the merge process will run in background. - skip: - version: " - 2.99.99" - reason: "only available in 3.0+" - features: allowed_warnings + version: " - 2.6.99, 2.13.0 - " + reason: "wait_for_completion was introduced in 2.7.0 and task description was changed in 2.13.0" + features: allowed_warnings, node_selector - do: indices.create: index: test_index - do: + node_selector: + version: " 2.7.0 - 2.12.99" indices.forcemerge: index: test_index wait_for_completion: false @@ -25,8 +27,31 @@ wait_for_completion: true task_id: $taskId - match: { task.action: "indices:admin/forcemerge" } - - match: { task.description: "Force-merge indices [test_index], maxSegments[1], onlyExpungeDeletes[false], flush[true], primaryOnly[false]" } + - match: { task.description: "Force-merge indices [test_index], maxSegments[1], onlyExpungeDeletes[false], flush[true]" } + +--- +"Force merge index with wait_for_completion after task description changed": + - skip: + version: " - 2.12.99 " + reason: "task description was changed in 2.13.0" + features: allowed_warnings, node_selector + + - do: + node_selector: + version: " 2.13.0 - " + indices.forcemerge: + index: test_index + wait_for_completion: false + max_num_segments: 1 + - match: { task: /^.+$/ } + - set: { task: taskId } + - do: + tasks.get: + wait_for_completion: true + task_id: $taskId + - match: { task.action: "indices:admin/forcemerge" } + - match: { task.description: "Force-merge indices [test_index], maxSegments[1], onlyExpungeDeletes[false], flush[true], primaryOnly[false]" } # .tasks index is created when the force-merge operation completes, so we should delete .tasks index finally, # if not, the .tasks index may introduce unexpected warnings and then cause other test cases to fail. # Delete the .tasks index directly will also introduce warning, but currently we don't have such APIs which can delete one diff --git a/server/src/main/java/org/opensearch/action/admin/indices/forcemerge/ForceMergeRequest.java b/server/src/main/java/org/opensearch/action/admin/indices/forcemerge/ForceMergeRequest.java index bf6ee9ca43755..3efc4db21afbc 100644 --- a/server/src/main/java/org/opensearch/action/admin/indices/forcemerge/ForceMergeRequest.java +++ b/server/src/main/java/org/opensearch/action/admin/indices/forcemerge/ForceMergeRequest.java @@ -102,7 +102,7 @@ public ForceMergeRequest(StreamInput in) throws IOException { maxNumSegments = in.readInt(); onlyExpungeDeletes = in.readBoolean(); flush = in.readBoolean(); - if (in.getVersion().onOrAfter(Version.V_3_0_0)) { + if (in.getVersion().onOrAfter(Version.V_2_13_0)) { primaryOnly = in.readBoolean(); } if (in.getVersion().onOrAfter(FORCE_MERGE_UUID_VERSION)) { @@ -219,7 +219,7 @@ public void writeTo(StreamOutput out) throws IOException { out.writeInt(maxNumSegments); out.writeBoolean(onlyExpungeDeletes); out.writeBoolean(flush); - if (out.getVersion().onOrAfter(Version.V_3_0_0)) { + if (out.getVersion().onOrAfter(Version.V_2_13_0)) { out.writeBoolean(primaryOnly); } if (out.getVersion().onOrAfter(FORCE_MERGE_UUID_VERSION)) { diff --git a/server/src/test/java/org/opensearch/action/admin/indices/forcemerge/ForceMergeRequestTests.java b/server/src/test/java/org/opensearch/action/admin/indices/forcemerge/ForceMergeRequestTests.java index a80141c52b6b4..03cf38548a8cd 100644 --- a/server/src/test/java/org/opensearch/action/admin/indices/forcemerge/ForceMergeRequestTests.java +++ b/server/src/test/java/org/opensearch/action/admin/indices/forcemerge/ForceMergeRequestTests.java @@ -32,8 +32,10 @@ package org.opensearch.action.admin.indices.forcemerge; import org.opensearch.Version; +import org.opensearch.action.support.IndicesOptions; import org.opensearch.common.io.stream.BytesStreamOutput; import org.opensearch.core.common.io.stream.StreamInput; +import org.opensearch.core.tasks.TaskId; import org.opensearch.test.OpenSearchTestCase; import org.opensearch.test.VersionUtils; @@ -95,42 +97,48 @@ public void testSerialization() throws Exception { public void testBwcSerialization() throws Exception { { final ForceMergeRequest sample = randomRequest(); - final Version compatibleVersion = VersionUtils.randomCompatibleVersion(random(), Version.CURRENT); + final Version version = VersionUtils.randomCompatibleVersion(random(), Version.CURRENT); try (BytesStreamOutput out = new BytesStreamOutput()) { - out.setVersion(compatibleVersion); + out.setVersion(version); sample.writeTo(out); - final ForceMergeRequest deserializedRequest; try (StreamInput in = out.bytes().streamInput()) { - in.setVersion(Version.CURRENT); - deserializedRequest = new ForceMergeRequest(in); - } - - assertEquals(sample.maxNumSegments(), deserializedRequest.maxNumSegments()); - assertEquals(sample.onlyExpungeDeletes(), deserializedRequest.onlyExpungeDeletes()); - assertEquals(sample.flush(), deserializedRequest.flush()); - if (compatibleVersion.onOrAfter(Version.V_3_0_0)) { - assertEquals(sample.primaryOnly(), deserializedRequest.primaryOnly()); - assertEquals(sample.forceMergeUUID(), deserializedRequest.forceMergeUUID()); + in.setVersion(version); + TaskId.readFromStream(in); + in.readStringArray(); + IndicesOptions.readIndicesOptions(in); + int maxNumSegments = in.readInt(); + boolean onlyExpungeDeletes = in.readBoolean(); + boolean flush = in.readBoolean(); + boolean primaryOnly = in.readBoolean(); + String forceMergeUUID; + if (version.onOrAfter(Version.V_3_0_0)) { + forceMergeUUID = in.readString(); + } else { + forceMergeUUID = in.readOptionalString(); + } + assertEquals(sample.maxNumSegments(), maxNumSegments); + assertEquals(sample.onlyExpungeDeletes(), onlyExpungeDeletes); + assertEquals(sample.flush(), flush); + assertEquals(sample.primaryOnly(), primaryOnly); + assertEquals(sample.forceMergeUUID(), forceMergeUUID); } } } { final ForceMergeRequest sample = randomRequest(); - final Version compatibleVersion = VersionUtils.randomCompatibleVersion(random(), Version.CURRENT); + final Version version = VersionUtils.randomCompatibleVersion(random(), Version.CURRENT); try (BytesStreamOutput out = new BytesStreamOutput()) { - out.setVersion(Version.CURRENT); + out.setVersion(version); sample.getParentTask().writeTo(out); out.writeStringArray(sample.indices()); sample.indicesOptions().writeIndicesOptions(out); out.writeInt(sample.maxNumSegments()); out.writeBoolean(sample.onlyExpungeDeletes()); out.writeBoolean(sample.flush()); - if (compatibleVersion.onOrAfter(Version.V_3_0_0)) { - out.writeBoolean(sample.primaryOnly()); - } - if (compatibleVersion.onOrAfter(Version.V_3_0_0)) { + out.writeBoolean(sample.primaryOnly()); + if (version.onOrAfter(Version.V_3_0_0)) { out.writeString(sample.forceMergeUUID()); } else { out.writeOptionalString(sample.forceMergeUUID()); @@ -138,18 +146,15 @@ public void testBwcSerialization() throws Exception { final ForceMergeRequest deserializedRequest; try (StreamInput in = out.bytes().streamInput()) { - in.setVersion(compatibleVersion); + in.setVersion(version); deserializedRequest = new ForceMergeRequest(in); } assertEquals(sample.maxNumSegments(), deserializedRequest.maxNumSegments()); assertEquals(sample.onlyExpungeDeletes(), deserializedRequest.onlyExpungeDeletes()); assertEquals(sample.flush(), deserializedRequest.flush()); - if (compatibleVersion.onOrAfter(Version.V_3_0_0)) { - assertEquals(sample.primaryOnly(), deserializedRequest.primaryOnly()); - } + assertEquals(sample.primaryOnly(), deserializedRequest.primaryOnly()); assertEquals(sample.forceMergeUUID(), deserializedRequest.forceMergeUUID()); - } } } From ad6b53834229da724312ad6a64faa2dc41a8d4c5 Mon Sep 17 00:00:00 2001 From: shwetathareja Date: Wed, 20 Mar 2024 16:28:42 +0530 Subject: [PATCH 12/38] Fixing flaky test GeoPointShapeQueryTests.testQueryLinearRing (#12783) Signed-off-by: Shweta Thareja Co-authored-by: Shweta Thareja --- .../java/org/opensearch/search/geo/GeoPointShapeQueryTests.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/server/src/test/java/org/opensearch/search/geo/GeoPointShapeQueryTests.java b/server/src/test/java/org/opensearch/search/geo/GeoPointShapeQueryTests.java index b6b2a86ac7549..b5d34a78ab5a4 100644 --- a/server/src/test/java/org/opensearch/search/geo/GeoPointShapeQueryTests.java +++ b/server/src/test/java/org/opensearch/search/geo/GeoPointShapeQueryTests.java @@ -143,6 +143,8 @@ public void testQueryLinearRing() throws Exception { e.getCause().getMessage(), containsString("Field [" + defaultGeoFieldName + "] does not support LINEARRING queries") ); + } catch (UnsupportedOperationException e) { + assertThat(e.getMessage(), containsString("line ring cannot be serialized using GeoJson")); } } From c369ec44e381ddb85db647d89389842952a1c06a Mon Sep 17 00:00:00 2001 From: Prabhat <20185657+CaptainDredge@users.noreply.github.com> Date: Wed, 20 Mar 2024 04:50:28 -0700 Subject: [PATCH 13/38] Added static setting for checkPendingFlushUpdate functionality of lucene writer (#12710) Signed-off-by: Prabhat Sharma Co-authored-by: Prabhat Sharma --- CHANGELOG.md | 1 + .../common/settings/IndexScopedSettings.java | 1 + .../org/opensearch/index/IndexSettings.java | 20 ++++++++++++++++++- .../index/engine/InternalEngine.java | 1 + 4 files changed, 22 insertions(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 4e5426797c69f..3cfdcd86f6bf0 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -118,6 +118,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - [Tiered caching] Add Stale keys Management and CacheCleaner to IndicesRequestCache ([#12625](https://github.com/opensearch-project/OpenSearch/pull/12625)) - [Tiered caching] Add serializer integration to allow ehcache disk cache to use non-primitive values ([#12709](https://github.com/opensearch-project/OpenSearch/pull/12709)) - [Admission Control] Integrated IO Based AdmissionController to AdmissionControl Framework ([#12583](https://github.com/opensearch-project/OpenSearch/pull/12583)) +- Introduce a new setting `index.check_pending_flush.enabled` to expose the ability to disable the check for pending flushes by write threads ([#12710](https://github.com/opensearch-project/OpenSearch/pull/12710)) ### Dependencies - Bump `peter-evans/find-comment` from 2 to 3 ([#12288](https://github.com/opensearch-project/OpenSearch/pull/12288)) diff --git a/server/src/main/java/org/opensearch/common/settings/IndexScopedSettings.java b/server/src/main/java/org/opensearch/common/settings/IndexScopedSettings.java index 49bb3abf1decd..c6c312d6b6eea 100644 --- a/server/src/main/java/org/opensearch/common/settings/IndexScopedSettings.java +++ b/server/src/main/java/org/opensearch/common/settings/IndexScopedSettings.java @@ -207,6 +207,7 @@ public final class IndexScopedSettings extends AbstractScopedSettings { IndexSettings.INDEX_MERGE_ON_FLUSH_MAX_FULL_FLUSH_MERGE_WAIT_TIME, IndexSettings.INDEX_MERGE_ON_FLUSH_POLICY, IndexSettings.INDEX_MERGE_POLICY, + IndexSettings.INDEX_CHECK_PENDING_FLUSH_ENABLED, LogByteSizeMergePolicyProvider.INDEX_LBS_MERGE_POLICY_MERGE_FACTOR_SETTING, LogByteSizeMergePolicyProvider.INDEX_LBS_MERGE_POLICY_MIN_MERGE_SETTING, LogByteSizeMergePolicyProvider.INDEX_LBS_MAX_MERGE_SEGMENT_SETTING, diff --git a/server/src/main/java/org/opensearch/index/IndexSettings.java b/server/src/main/java/org/opensearch/index/IndexSettings.java index d750a13dece64..5aaea2c498701 100644 --- a/server/src/main/java/org/opensearch/index/IndexSettings.java +++ b/server/src/main/java/org/opensearch/index/IndexSettings.java @@ -624,6 +624,16 @@ public static IndexMergePolicy fromString(String text) { Property.IndexScope ); + /** + * Expert: Makes indexing threads check for pending flushes on update in order to help out + * flushing indexing buffers to disk. This is an experimental Apache Lucene feature. + */ + public static final Setting INDEX_CHECK_PENDING_FLUSH_ENABLED = Setting.boolSetting( + "index.check_pending_flush.enabled", + true, + Property.IndexScope + ); + public static final Setting TIME_SERIES_INDEX_MERGE_POLICY = Setting.simpleString( "indices.time_series_index.default_index_merge_policy", DEFAULT_POLICY, @@ -816,7 +826,10 @@ private void setRetentionLeaseMillis(final TimeValue retentionLease) { * Specialized merge-on-flush policy if provided */ private volatile UnaryOperator mergeOnFlushPolicy; - + /** + * Is flush check by write threads enabled or not + */ + private final boolean checkPendingFlushEnabled; /** * Is fuzzy set enabled for doc id */ @@ -959,6 +972,7 @@ public IndexSettings(final IndexMetadata indexMetadata, final Settings nodeSetti maxFullFlushMergeWaitTime = scopedSettings.get(INDEX_MERGE_ON_FLUSH_MAX_FULL_FLUSH_MERGE_WAIT_TIME); mergeOnFlushEnabled = scopedSettings.get(INDEX_MERGE_ON_FLUSH_ENABLED); setMergeOnFlushPolicy(scopedSettings.get(INDEX_MERGE_ON_FLUSH_POLICY)); + checkPendingFlushEnabled = scopedSettings.get(INDEX_CHECK_PENDING_FLUSH_ENABLED); defaultSearchPipeline = scopedSettings.get(DEFAULT_SEARCH_PIPELINE); /* There was unintentional breaking change got introduced with [OpenSearch-6424](https://github.com/opensearch-project/OpenSearch/pull/6424) (version 2.7). * For indices created prior version (prior to 2.7) which has IndexSort type, they used to type cast the SortField.Type @@ -1844,6 +1858,10 @@ private void setMergeOnFlushPolicy(String policy) { } } + public boolean isCheckPendingFlushEnabled() { + return checkPendingFlushEnabled; + } + public Optional> getMergeOnFlushPolicy() { return Optional.ofNullable(mergeOnFlushPolicy); } diff --git a/server/src/main/java/org/opensearch/index/engine/InternalEngine.java b/server/src/main/java/org/opensearch/index/engine/InternalEngine.java index e204656d3f106..a25ec95f58e05 100644 --- a/server/src/main/java/org/opensearch/index/engine/InternalEngine.java +++ b/server/src/main/java/org/opensearch/index/engine/InternalEngine.java @@ -2336,6 +2336,7 @@ private IndexWriterConfig getIndexWriterConfig() { iwc.setMaxFullFlushMergeWaitMillis(0); } + iwc.setCheckPendingFlushUpdate(config().getIndexSettings().isCheckPendingFlushEnabled()); iwc.setMergePolicy(new OpenSearchMergePolicy(mergePolicy)); iwc.setSimilarity(engineConfig.getSimilarity()); iwc.setRAMBufferSizeMB(engineConfig.getIndexingBufferSize().getMbFrac()); From 773a93945429d99e3551fad6e4746668778aa563 Mon Sep 17 00:00:00 2001 From: Andriy Redko Date: Wed, 20 Mar 2024 10:01:59 -0400 Subject: [PATCH 14/38] [FEATURE] Built-in secure transports support (#12435) * [FEATURE] Built-in secure transports support Signed-off-by: Andriy Redko * Added more tests, addressing code review comments Signed-off-by: Andriy Redko * Address code review comments Signed-off-by: Andriy Redko * Address code review comments Signed-off-by: Andriy Redko * Address code review comments Signed-off-by: Andriy Redko --------- Signed-off-by: Andriy Redko --- CHANGELOG.md | 1 + .../ssl/SecureNetty4HttpServerTransport.java | 169 +++++ .../transport/Netty4ModulePlugin.java | 64 ++ .../netty4/ssl/DualModeSslHandler.java | 106 +++ .../netty4/ssl/SecureConnectionTestUtil.java | 214 +++++++ .../netty4/ssl/SecureNetty4Transport.java | 316 +++++++++ .../transport/netty4/ssl/SslUtils.java | 107 ++++ .../http/netty4/Netty4HttpClient.java | 60 +- .../SecureNetty4HttpServerTransportTests.java | 603 ++++++++++++++++++ .../ssl/SimpleSecureNetty4TransportTests.java | 234 +++++++ .../transport/netty4/ssl/TrustAllManager.java | 28 + .../src/test/resources/README.txt | 17 + .../src/test/resources/certificate.crt | 22 + .../src/test/resources/certificate.key | 28 + .../src/test/resources/netty4-secure.jks | Bin 0 -> 2790 bytes .../common/network/NetworkModule.java | 69 +- .../common/settings/ClusterSettings.java | 3 + .../main/java/org/opensearch/node/Node.java | 11 +- .../org/opensearch/plugins/NetworkPlugin.java | 37 ++ .../java/org/opensearch/plugins/Plugin.java | 9 + .../plugins/SecureSettingsFactory.java | 29 + .../SecureTransportSettingsProvider.java | 90 +++ .../common/network/NetworkModuleTests.java | 95 ++- 23 files changed, 2296 insertions(+), 16 deletions(-) create mode 100644 modules/transport-netty4/src/main/java/org/opensearch/http/netty4/ssl/SecureNetty4HttpServerTransport.java create mode 100644 modules/transport-netty4/src/main/java/org/opensearch/transport/netty4/ssl/DualModeSslHandler.java create mode 100644 modules/transport-netty4/src/main/java/org/opensearch/transport/netty4/ssl/SecureConnectionTestUtil.java create mode 100644 modules/transport-netty4/src/main/java/org/opensearch/transport/netty4/ssl/SecureNetty4Transport.java create mode 100644 modules/transport-netty4/src/main/java/org/opensearch/transport/netty4/ssl/SslUtils.java create mode 100644 modules/transport-netty4/src/test/java/org/opensearch/http/netty4/ssl/SecureNetty4HttpServerTransportTests.java create mode 100644 modules/transport-netty4/src/test/java/org/opensearch/transport/netty4/ssl/SimpleSecureNetty4TransportTests.java create mode 100644 modules/transport-netty4/src/test/java/org/opensearch/transport/netty4/ssl/TrustAllManager.java create mode 100644 modules/transport-netty4/src/test/resources/README.txt create mode 100644 modules/transport-netty4/src/test/resources/certificate.crt create mode 100644 modules/transport-netty4/src/test/resources/certificate.key create mode 100644 modules/transport-netty4/src/test/resources/netty4-secure.jks create mode 100644 server/src/main/java/org/opensearch/plugins/SecureSettingsFactory.java create mode 100644 server/src/main/java/org/opensearch/plugins/SecureTransportSettingsProvider.java diff --git a/CHANGELOG.md b/CHANGELOG.md index 3cfdcd86f6bf0..0b5a7cc705a79 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -119,6 +119,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - [Tiered caching] Add serializer integration to allow ehcache disk cache to use non-primitive values ([#12709](https://github.com/opensearch-project/OpenSearch/pull/12709)) - [Admission Control] Integrated IO Based AdmissionController to AdmissionControl Framework ([#12583](https://github.com/opensearch-project/OpenSearch/pull/12583)) - Introduce a new setting `index.check_pending_flush.enabled` to expose the ability to disable the check for pending flushes by write threads ([#12710](https://github.com/opensearch-project/OpenSearch/pull/12710)) +- Built-in secure transports support ([#12435](https://github.com/opensearch-project/OpenSearch/pull/12435)) ### Dependencies - Bump `peter-evans/find-comment` from 2 to 3 ([#12288](https://github.com/opensearch-project/OpenSearch/pull/12288)) diff --git a/modules/transport-netty4/src/main/java/org/opensearch/http/netty4/ssl/SecureNetty4HttpServerTransport.java b/modules/transport-netty4/src/main/java/org/opensearch/http/netty4/ssl/SecureNetty4HttpServerTransport.java new file mode 100644 index 0000000000000..51a76903e284d --- /dev/null +++ b/modules/transport-netty4/src/main/java/org/opensearch/http/netty4/ssl/SecureNetty4HttpServerTransport.java @@ -0,0 +1,169 @@ +/* + * Copyright 2015-2017 floragunn GmbH + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +/* + * 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. + * + * Modifications Copyright OpenSearch Contributors. See + * GitHub history for details. + */ + +package org.opensearch.http.netty4.ssl; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.opensearch.common.network.NetworkService; +import org.opensearch.common.settings.ClusterSettings; +import org.opensearch.common.settings.Settings; +import org.opensearch.common.util.BigArrays; +import org.opensearch.core.xcontent.NamedXContentRegistry; +import org.opensearch.http.HttpChannel; +import org.opensearch.http.HttpHandlingSettings; +import org.opensearch.http.netty4.Netty4HttpChannel; +import org.opensearch.http.netty4.Netty4HttpServerTransport; +import org.opensearch.plugins.SecureTransportSettingsProvider; +import org.opensearch.telemetry.tracing.Tracer; +import org.opensearch.threadpool.ThreadPool; +import org.opensearch.transport.SharedGroupFactory; +import org.opensearch.transport.netty4.ssl.SslUtils; + +import javax.net.ssl.SSLEngine; + +import io.netty.channel.Channel; +import io.netty.channel.ChannelHandler; +import io.netty.channel.ChannelHandlerContext; +import io.netty.handler.codec.DecoderException; +import io.netty.handler.ssl.ApplicationProtocolNames; +import io.netty.handler.ssl.ApplicationProtocolNegotiationHandler; +import io.netty.handler.ssl.SslHandler; + +/** + * @see SecuritySSLNettyHttpServerTransport + */ +public class SecureNetty4HttpServerTransport extends Netty4HttpServerTransport { + private static final Logger logger = LogManager.getLogger(SecureNetty4HttpServerTransport.class); + private final SecureTransportSettingsProvider secureTransportSettingsProvider; + private final SecureTransportSettingsProvider.ServerExceptionHandler exceptionHandler; + + public SecureNetty4HttpServerTransport( + final Settings settings, + final NetworkService networkService, + final BigArrays bigArrays, + final ThreadPool threadPool, + final NamedXContentRegistry namedXContentRegistry, + final Dispatcher dispatcher, + final ClusterSettings clusterSettings, + final SharedGroupFactory sharedGroupFactory, + final SecureTransportSettingsProvider secureTransportSettingsProvider, + final Tracer tracer + ) { + super( + settings, + networkService, + bigArrays, + threadPool, + namedXContentRegistry, + dispatcher, + clusterSettings, + sharedGroupFactory, + tracer + ); + this.secureTransportSettingsProvider = secureTransportSettingsProvider; + this.exceptionHandler = secureTransportSettingsProvider.buildHttpServerExceptionHandler(settings, this) + .orElse(SecureTransportSettingsProvider.ServerExceptionHandler.NOOP); + } + + @Override + public ChannelHandler configureServerChannelHandler() { + return new SslHttpChannelHandler(this, handlingSettings); + } + + @Override + public void onException(HttpChannel channel, Exception cause0) { + Throwable cause = cause0; + + if (cause0 instanceof DecoderException && cause0 != null) { + cause = cause0.getCause(); + } + + exceptionHandler.onError(cause); + logger.error("Exception during establishing a SSL connection: " + cause, cause); + super.onException(channel, cause0); + } + + protected class SslHttpChannelHandler extends Netty4HttpServerTransport.HttpChannelHandler { + /** + * Application negotiation handler to select either HTTP 1.1 or HTTP 2 protocol, based + * on client/server ALPN negotiations. + */ + private class Http2OrHttpHandler extends ApplicationProtocolNegotiationHandler { + protected Http2OrHttpHandler() { + super(ApplicationProtocolNames.HTTP_1_1); + } + + @Override + protected void configurePipeline(ChannelHandlerContext ctx, String protocol) throws Exception { + if (ApplicationProtocolNames.HTTP_2.equals(protocol)) { + configureDefaultHttp2Pipeline(ctx.pipeline()); + } else if (ApplicationProtocolNames.HTTP_1_1.equals(protocol)) { + configureDefaultHttpPipeline(ctx.pipeline()); + } else { + throw new IllegalStateException("Unknown application protocol: " + protocol); + } + } + + @Override + public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws Exception { + super.exceptionCaught(ctx, cause); + Netty4HttpChannel channel = ctx.channel().attr(HTTP_CHANNEL_KEY).get(); + if (channel != null) { + if (cause instanceof Error) { + onException(channel, new Exception(cause)); + } else { + onException(channel, (Exception) cause); + } + } + } + } + + protected SslHttpChannelHandler(final Netty4HttpServerTransport transport, final HttpHandlingSettings handlingSettings) { + super(transport, handlingSettings); + } + + @Override + protected void initChannel(Channel ch) throws Exception { + super.initChannel(ch); + + final SSLEngine sslEngine = secureTransportSettingsProvider.buildSecureHttpServerEngine( + settings, + SecureNetty4HttpServerTransport.this + ).orElseGet(SslUtils::createDefaultServerSSLEngine); + + final SslHandler sslHandler = new SslHandler(sslEngine); + ch.pipeline().addFirst("ssl_http", sslHandler); + } + + @Override + protected void configurePipeline(Channel ch) { + ch.pipeline().addLast(new Http2OrHttpHandler()); + } + } +} diff --git a/modules/transport-netty4/src/main/java/org/opensearch/transport/Netty4ModulePlugin.java b/modules/transport-netty4/src/main/java/org/opensearch/transport/Netty4ModulePlugin.java index 2bc795d11ed5d..56163c18949a4 100644 --- a/modules/transport-netty4/src/main/java/org/opensearch/transport/Netty4ModulePlugin.java +++ b/modules/transport-netty4/src/main/java/org/opensearch/transport/Netty4ModulePlugin.java @@ -46,11 +46,14 @@ import org.opensearch.core.xcontent.NamedXContentRegistry; import org.opensearch.http.HttpServerTransport; import org.opensearch.http.netty4.Netty4HttpServerTransport; +import org.opensearch.http.netty4.ssl.SecureNetty4HttpServerTransport; import org.opensearch.plugins.NetworkPlugin; import org.opensearch.plugins.Plugin; +import org.opensearch.plugins.SecureTransportSettingsProvider; import org.opensearch.telemetry.tracing.Tracer; import org.opensearch.threadpool.ThreadPool; import org.opensearch.transport.netty4.Netty4Transport; +import org.opensearch.transport.netty4.ssl.SecureNetty4Transport; import java.util.Arrays; import java.util.Collections; @@ -61,7 +64,9 @@ public class Netty4ModulePlugin extends Plugin implements NetworkPlugin { public static final String NETTY_TRANSPORT_NAME = "netty4"; + public static final String NETTY_SECURE_TRANSPORT_NAME = "netty4-secure"; public static final String NETTY_HTTP_TRANSPORT_NAME = "netty4"; + public static final String NETTY_SECURE_HTTP_TRANSPORT_NAME = "netty4-secure"; private final SetOnce groupFactory = new SetOnce<>(); @@ -144,6 +149,65 @@ public Map> getHttpTransports( ); } + @Override + public Map> getSecureHttpTransports( + Settings settings, + ThreadPool threadPool, + BigArrays bigArrays, + PageCacheRecycler pageCacheRecycler, + CircuitBreakerService circuitBreakerService, + NamedXContentRegistry xContentRegistry, + NetworkService networkService, + HttpServerTransport.Dispatcher dispatcher, + ClusterSettings clusterSettings, + SecureTransportSettingsProvider secureTransportSettingsProvider, + Tracer tracer + ) { + return Collections.singletonMap( + NETTY_SECURE_HTTP_TRANSPORT_NAME, + () -> new SecureNetty4HttpServerTransport( + settings, + networkService, + bigArrays, + threadPool, + xContentRegistry, + dispatcher, + clusterSettings, + getSharedGroupFactory(settings), + secureTransportSettingsProvider, + tracer + ) + ); + } + + @Override + public Map> getSecureTransports( + Settings settings, + ThreadPool threadPool, + PageCacheRecycler pageCacheRecycler, + CircuitBreakerService circuitBreakerService, + NamedWriteableRegistry namedWriteableRegistry, + NetworkService networkService, + SecureTransportSettingsProvider secureTransportSettingsProvider, + Tracer tracer + ) { + return Collections.singletonMap( + NETTY_SECURE_TRANSPORT_NAME, + () -> new SecureNetty4Transport( + settings, + Version.CURRENT, + threadPool, + networkService, + pageCacheRecycler, + namedWriteableRegistry, + circuitBreakerService, + getSharedGroupFactory(settings), + secureTransportSettingsProvider, + tracer + ) + ); + } + SharedGroupFactory getSharedGroupFactory(Settings settings) { SharedGroupFactory groupFactory = this.groupFactory.get(); if (groupFactory != null) { diff --git a/modules/transport-netty4/src/main/java/org/opensearch/transport/netty4/ssl/DualModeSslHandler.java b/modules/transport-netty4/src/main/java/org/opensearch/transport/netty4/ssl/DualModeSslHandler.java new file mode 100644 index 0000000000000..1bf4cdb0eb438 --- /dev/null +++ b/modules/transport-netty4/src/main/java/org/opensearch/transport/netty4/ssl/DualModeSslHandler.java @@ -0,0 +1,106 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + * Modifications Copyright OpenSearch Contributors. See + * GitHub history for details. + */ +package org.opensearch.transport.netty4.ssl; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.opensearch.common.settings.Settings; +import org.opensearch.plugins.SecureTransportSettingsProvider; +import org.opensearch.transport.TcpTransport; + +import javax.net.ssl.SSLEngine; +import javax.net.ssl.SSLException; + +import java.nio.charset.StandardCharsets; +import java.security.NoSuchAlgorithmException; +import java.util.List; + +import io.netty.buffer.ByteBuf; +import io.netty.buffer.Unpooled; +import io.netty.channel.ChannelFutureListener; +import io.netty.channel.ChannelHandlerContext; +import io.netty.channel.ChannelPipeline; +import io.netty.handler.codec.ByteToMessageDecoder; +import io.netty.handler.ssl.SslHandler; + +/** + * Modifies the current pipeline dynamically to enable TLS + * + * @see DualModeSSLHandler + */ +public class DualModeSslHandler extends ByteToMessageDecoder { + + private static final Logger logger = LogManager.getLogger(DualModeSslHandler.class); + private final Settings settings; + private final SecureTransportSettingsProvider secureTransportSettingsProvider; + private final TcpTransport transport; + private final SslHandler providedSSLHandler; + + public DualModeSslHandler( + final Settings settings, + final SecureTransportSettingsProvider secureTransportSettingsProvider, + final TcpTransport transport + ) { + this(settings, secureTransportSettingsProvider, transport, null); + } + + protected DualModeSslHandler( + final Settings settings, + final SecureTransportSettingsProvider secureTransportSettingsProvider, + final TcpTransport transport, + SslHandler providedSSLHandler + ) { + this.settings = settings; + this.secureTransportSettingsProvider = secureTransportSettingsProvider; + this.transport = transport; + this.providedSSLHandler = providedSSLHandler; + } + + @Override + protected void decode(ChannelHandlerContext ctx, ByteBuf in, List out) throws Exception { + // Will use the first six bytes to detect a protocol. + if (in.readableBytes() < 6) { + return; + } + int offset = in.readerIndex(); + if (in.getCharSequence(offset, 6, StandardCharsets.UTF_8).equals(SecureConnectionTestUtil.DUAL_MODE_CLIENT_HELLO_MSG)) { + logger.debug("Received DualSSL Client Hello message"); + ByteBuf responseBuffer = Unpooled.buffer(6); + responseBuffer.writeCharSequence(SecureConnectionTestUtil.DUAL_MODE_SERVER_HELLO_MSG, StandardCharsets.UTF_8); + ctx.writeAndFlush(responseBuffer).addListener(ChannelFutureListener.CLOSE); + return; + } + + if (SslUtils.isTLS(in)) { + logger.debug("Identified request as SSL request"); + enableSsl(ctx); + } else { + logger.debug("Identified request as non SSL request, running in HTTP mode as dual mode is enabled"); + ctx.pipeline().remove(this); + } + } + + private void enableSsl(ChannelHandlerContext ctx) throws SSLException, NoSuchAlgorithmException { + final SSLEngine sslEngine = secureTransportSettingsProvider.buildSecureServerTransportEngine(settings, transport) + .orElseGet(SslUtils::createDefaultServerSSLEngine); + + SslHandler sslHandler; + if (providedSSLHandler != null) { + sslHandler = providedSSLHandler; + } else { + sslHandler = new SslHandler(sslEngine); + } + ChannelPipeline p = ctx.pipeline(); + p.addAfter("port_unification_handler", "ssl_server", sslHandler); + p.remove(this); + logger.debug("Removed port unification handler and added SSL handler as incoming request is SSL"); + } +} diff --git a/modules/transport-netty4/src/main/java/org/opensearch/transport/netty4/ssl/SecureConnectionTestUtil.java b/modules/transport-netty4/src/main/java/org/opensearch/transport/netty4/ssl/SecureConnectionTestUtil.java new file mode 100644 index 0000000000000..d5667475ea007 --- /dev/null +++ b/modules/transport-netty4/src/main/java/org/opensearch/transport/netty4/ssl/SecureConnectionTestUtil.java @@ -0,0 +1,214 @@ +/* + * 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. + * + * Modifications Copyright OpenSearch Contributors. See + * GitHub history for details. + */ + +package org.opensearch.transport.netty4.ssl; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +import java.io.IOException; +import java.io.InputStream; +import java.io.InputStreamReader; +import java.io.OutputStream; +import java.io.OutputStreamWriter; +import java.net.InetAddress; +import java.net.Socket; +import java.nio.charset.StandardCharsets; + +/** + * Utility class to test if the server supports SSL connections. + * SSL Check will be done by sending an OpenSearch Ping to see if server is replying to pings. + * Following that a custom client hello message will be sent to the server, if the server + * side has OpenSearchPortUnificationHandler it will reply with server hello message. + * + * @see SSLConnectionTestUtil + */ +class SecureConnectionTestUtil { + private static final Logger logger = LogManager.getLogger(SecureConnectionTestUtil.class); + + /** + * Return codes for SSLConnectionTestUtil.testConnection() + */ + enum SSLConnectionTestResult { + /** + * OpenSearch Ping to the server failed. + */ + OPENSEARCH_PING_FAILED, + /** + * Server does not support SSL. + */ + SSL_NOT_AVAILABLE, + /** + * Server supports SSL. + */ + SSL_AVAILABLE + } + + public static final byte[] OPENSEARCH_PING_MSG = new byte[] { + (byte) 'E', + (byte) 'S', + (byte) 0xFF, + (byte) 0xFF, + (byte) 0xFF, + (byte) 0xFF }; + public static final String DUAL_MODE_CLIENT_HELLO_MSG = "DUALCM"; + public static final String DUAL_MODE_SERVER_HELLO_MSG = "DUALSM"; + private static final int SOCKET_TIMEOUT_MILLIS = 10 * 1000; + private final String host; + private final int port; + private Socket overriddenSocket = null; + private OutputStreamWriter testOutputStreamWriter = null; + private InputStreamReader testInputStreamReader = null; + + public SecureConnectionTestUtil(final String host, final int port) { + this.host = host; + this.port = port; + } + + protected SecureConnectionTestUtil( + final String host, + final int port, + final Socket overriddenSocket, + final OutputStreamWriter testOutputStreamWriter, + final InputStreamReader testInputStreamReader + ) { + this.overriddenSocket = overriddenSocket; + this.testOutputStreamWriter = testOutputStreamWriter; + this.testInputStreamReader = testInputStreamReader; + + this.host = host; + this.port = port; + } + + /** + * Test connection to server by performing the below steps: + * - Send Client Hello to check if the server replies with Server Hello which indicates that Server understands SSL + * - Send OpenSearch Ping to check if the server replies to the OpenSearch Ping message + * + * @return SSLConnectionTestResult i.e. OPENSEARCH_PING_FAILED or SSL_NOT_AVAILABLE or SSL_AVAILABLE + */ + public SSLConnectionTestResult testConnection() { + if (sendDualSSLClientHello()) { + return SSLConnectionTestResult.SSL_AVAILABLE; + } + + if (sendOpenSearchPing()) { + return SSLConnectionTestResult.SSL_NOT_AVAILABLE; + } + + return SSLConnectionTestResult.OPENSEARCH_PING_FAILED; + } + + private boolean sendDualSSLClientHello() { + boolean dualSslSupported = false; + Socket socket = null; + try { + OutputStreamWriter outputStreamWriter; + InputStreamReader inputStreamReader; + if (overriddenSocket != null) { + socket = overriddenSocket; + outputStreamWriter = testOutputStreamWriter; + inputStreamReader = testInputStreamReader; + } else { + socket = new Socket(InetAddress.getByName(host), port); + outputStreamWriter = new OutputStreamWriter(socket.getOutputStream(), StandardCharsets.UTF_8); + inputStreamReader = new InputStreamReader(socket.getInputStream(), StandardCharsets.UTF_8); + } + + socket.setSoTimeout(SOCKET_TIMEOUT_MILLIS); + outputStreamWriter.write(DUAL_MODE_CLIENT_HELLO_MSG); + outputStreamWriter.flush(); + logger.debug("Sent DualSSL Client Hello msg to {}", host); + + StringBuilder sb = new StringBuilder(); + int currentChar; + while ((currentChar = inputStreamReader.read()) != -1) { + sb.append((char) currentChar); + } + + if (sb.toString().equals(DUAL_MODE_SERVER_HELLO_MSG)) { + logger.debug("Received DualSSL Server Hello msg from {}", host); + dualSslSupported = true; + } + } catch (IOException e) { + logger.debug("DualSSL client check failed for {}, exception {}", host, e.getMessage()); + } finally { + logger.debug("Closing DualSSL check client socket for {}", host); + if (socket != null) { + try { + socket.close(); + } catch (IOException e) { + logger.error( + "Exception occurred while closing DualSSL check client socket for {}. Exception: {}", + host, + e.getMessage() + ); + } + } + } + logger.debug("dualSslClient check with server {}, server supports ssl = {}", host, dualSslSupported); + return dualSslSupported; + } + + private boolean sendOpenSearchPing() { + boolean pingSucceeded = false; + Socket socket = null; + try { + if (overriddenSocket != null) { + socket = overriddenSocket; + } else { + socket = new Socket(InetAddress.getByName(host), port); + } + + socket.setSoTimeout(SOCKET_TIMEOUT_MILLIS); + OutputStream outputStream = socket.getOutputStream(); + InputStream inputStream = socket.getInputStream(); + + logger.debug("Sending OpenSearch Ping to {}", host); + outputStream.write(OPENSEARCH_PING_MSG); + outputStream.flush(); + + int currentByte; + int byteBufIndex = 0; + byte[] response = new byte[6]; + while ((byteBufIndex < 6) && ((currentByte = inputStream.read()) != -1)) { + response[byteBufIndex] = (byte) currentByte; + byteBufIndex++; + } + if (byteBufIndex == 6) { + logger.debug("Received reply for OpenSearch Ping. from {}", host); + pingSucceeded = true; + for (int i = 0; i < 6; i++) { + if (response[i] != OPENSEARCH_PING_MSG[i]) { + // Unexpected byte in response + logger.error("Received unexpected byte in OpenSearch Ping reply from {}", host); + pingSucceeded = false; + break; + } + } + } + } catch (IOException ex) { + logger.error("OpenSearch Ping failed for {}, exception: {}", host, ex.getMessage()); + } finally { + logger.debug("Closing OpenSearch Ping client socket for connection to {}", host); + if (socket != null) { + try { + socket.close(); + } catch (IOException e) { + logger.error("Exception occurred while closing socket for {}. Exception: {}", host, e.getMessage()); + } + } + } + + logger.debug("OpenSearch Ping check to server {} result = {}", host, pingSucceeded); + return pingSucceeded; + } +} diff --git a/modules/transport-netty4/src/main/java/org/opensearch/transport/netty4/ssl/SecureNetty4Transport.java b/modules/transport-netty4/src/main/java/org/opensearch/transport/netty4/ssl/SecureNetty4Transport.java new file mode 100644 index 0000000000000..9c63a1ab9161b --- /dev/null +++ b/modules/transport-netty4/src/main/java/org/opensearch/transport/netty4/ssl/SecureNetty4Transport.java @@ -0,0 +1,316 @@ +/* + * Copyright 2015-2017 floragunn GmbH + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +/* + * 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. + * + * Modifications Copyright OpenSearch Contributors. See + * GitHub history for details. + */ + +package org.opensearch.transport.netty4.ssl; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.opensearch.ExceptionsHelper; +import org.opensearch.OpenSearchSecurityException; +import org.opensearch.Version; +import org.opensearch.cluster.node.DiscoveryNode; +import org.opensearch.common.SuppressForbidden; +import org.opensearch.common.network.NetworkModule; +import org.opensearch.common.network.NetworkService; +import org.opensearch.common.settings.Settings; +import org.opensearch.common.util.PageCacheRecycler; +import org.opensearch.core.common.io.stream.NamedWriteableRegistry; +import org.opensearch.core.indices.breaker.CircuitBreakerService; +import org.opensearch.plugins.SecureTransportSettingsProvider; +import org.opensearch.telemetry.tracing.Tracer; +import org.opensearch.threadpool.ThreadPool; +import org.opensearch.transport.SharedGroupFactory; +import org.opensearch.transport.TcpChannel; +import org.opensearch.transport.netty4.Netty4Transport; +import org.opensearch.transport.netty4.ssl.SecureConnectionTestUtil.SSLConnectionTestResult; + +import javax.net.ssl.SSLEngine; +import javax.net.ssl.SSLException; + +import java.net.InetSocketAddress; +import java.net.SocketAddress; +import java.security.AccessController; +import java.security.PrivilegedAction; + +import io.netty.channel.Channel; +import io.netty.channel.ChannelHandler; +import io.netty.channel.ChannelHandlerContext; +import io.netty.channel.ChannelOutboundHandlerAdapter; +import io.netty.channel.ChannelPromise; +import io.netty.handler.codec.DecoderException; +import io.netty.handler.ssl.SslHandler; + +/** + * @see SecuritySSLNettyTransport + */ +public class SecureNetty4Transport extends Netty4Transport { + + private static final Logger logger = LogManager.getLogger(SecureNetty4Transport.class); + private final SecureTransportSettingsProvider secureTransportSettingsProvider; + private final SecureTransportSettingsProvider.ServerExceptionHandler exceptionHandler; + + public SecureNetty4Transport( + final Settings settings, + final Version version, + final ThreadPool threadPool, + final NetworkService networkService, + final PageCacheRecycler pageCacheRecycler, + final NamedWriteableRegistry namedWriteableRegistry, + final CircuitBreakerService circuitBreakerService, + final SharedGroupFactory sharedGroupFactory, + final SecureTransportSettingsProvider secureTransportSettingsProvider, + final Tracer tracer + ) { + super( + settings, + version, + threadPool, + networkService, + pageCacheRecycler, + namedWriteableRegistry, + circuitBreakerService, + sharedGroupFactory, + tracer + ); + + this.secureTransportSettingsProvider = secureTransportSettingsProvider; + this.exceptionHandler = secureTransportSettingsProvider.buildServerTransportExceptionHandler(settings, this) + .orElse(SecureTransportSettingsProvider.ServerExceptionHandler.NOOP); + } + + @Override + public void onException(TcpChannel channel, Exception e) { + + Throwable cause = e; + + if (e instanceof DecoderException && e != null) { + cause = e.getCause(); + } + + exceptionHandler.onError(cause); + logger.error("Exception during establishing a SSL connection: " + cause, cause); + + if (channel == null || !channel.isOpen()) { + throw new OpenSearchSecurityException("The provided TCP channel is invalid.", e); + } + super.onException(channel, e); + } + + @Override + protected ChannelHandler getServerChannelInitializer(String name) { + return new SSLServerChannelInitializer(name); + } + + @Override + protected ChannelHandler getClientChannelInitializer(DiscoveryNode node) { + return new SSLClientChannelInitializer(node); + } + + protected class SSLServerChannelInitializer extends Netty4Transport.ServerChannelInitializer { + + public SSLServerChannelInitializer(String name) { + super(name); + } + + @Override + protected void initChannel(Channel ch) throws Exception { + super.initChannel(ch); + + final boolean dualModeEnabled = NetworkModule.TRANSPORT_SSL_DUAL_MODE_ENABLED.get(settings); + if (dualModeEnabled) { + logger.info("SSL Dual mode enabled, using port unification handler"); + final ChannelHandler portUnificationHandler = new DualModeSslHandler( + settings, + secureTransportSettingsProvider, + SecureNetty4Transport.this + ); + ch.pipeline().addFirst("port_unification_handler", portUnificationHandler); + } else { + final SSLEngine sslEngine = secureTransportSettingsProvider.buildSecureServerTransportEngine( + settings, + SecureNetty4Transport.this + ).orElseGet(SslUtils::createDefaultServerSSLEngine); + final SslHandler sslHandler = new SslHandler(sslEngine); + ch.pipeline().addFirst("ssl_server", sslHandler); + } + } + + @Override + public final void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws Exception { + if (cause instanceof DecoderException && cause != null) { + cause = cause.getCause(); + } + + logger.error("Exception during establishing a SSL connection: " + cause, cause); + + super.exceptionCaught(ctx, cause); + } + } + + protected static class ClientSSLHandler extends ChannelOutboundHandlerAdapter { + private final Logger log = LogManager.getLogger(this.getClass()); + private final Settings settings; + private final SecureTransportSettingsProvider secureTransportSettingsProvider; + private final boolean hostnameVerificationEnabled; + private final boolean hostnameVerificationResovleHostName; + + private ClientSSLHandler( + final Settings settings, + final SecureTransportSettingsProvider secureTransportSettingsProvider, + final boolean hostnameVerificationEnabled, + final boolean hostnameVerificationResovleHostName + ) { + this.settings = settings; + this.secureTransportSettingsProvider = secureTransportSettingsProvider; + this.hostnameVerificationEnabled = hostnameVerificationEnabled; + this.hostnameVerificationResovleHostName = hostnameVerificationResovleHostName; + } + + @Override + public final void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws Exception { + if (cause instanceof DecoderException && cause != null) { + cause = cause.getCause(); + } + + logger.error("Exception during establishing a SSL connection: " + cause, cause); + + super.exceptionCaught(ctx, cause); + } + + @SuppressForbidden(reason = "The java.net.InetSocketAddress#getHostName() needs to be used") + @Override + public void connect(ChannelHandlerContext ctx, SocketAddress remoteAddress, SocketAddress localAddress, ChannelPromise promise) + throws Exception { + SSLEngine sslEngine = null; + try { + if (hostnameVerificationEnabled) { + final InetSocketAddress inetSocketAddress = (InetSocketAddress) remoteAddress; + final String hostname = (hostnameVerificationResovleHostName == true) + ? inetSocketAddress.getHostName() + : inetSocketAddress.getHostString(); + + if (log.isDebugEnabled()) { + log.debug( + "Hostname of peer is {} ({}/{}) with hostnameVerificationResolveHostName: {}", + hostname, + inetSocketAddress.getHostName(), + inetSocketAddress.getHostString(), + hostnameVerificationResovleHostName + ); + } + + sslEngine = secureTransportSettingsProvider.buildSecureClientTransportEngine( + settings, + hostname, + inetSocketAddress.getPort() + ).orElse(null); + + } else { + sslEngine = secureTransportSettingsProvider.buildSecureClientTransportEngine(settings, null, -1).orElse(null); + } + + if (sslEngine == null) { + sslEngine = SslUtils.createDefaultClientSSLEngine(); + } + } catch (final SSLException e) { + throw ExceptionsHelper.convertToOpenSearchException(e); + } + + final SslHandler sslHandler = new SslHandler(sslEngine); + ctx.pipeline().replace(this, "ssl_client", sslHandler); + super.connect(ctx, remoteAddress, localAddress, promise); + } + } + + protected class SSLClientChannelInitializer extends Netty4Transport.ClientChannelInitializer { + private final boolean hostnameVerificationEnabled; + private final boolean hostnameVerificationResolveHostName; + private final DiscoveryNode node; + private SSLConnectionTestResult connectionTestResult; + + @SuppressWarnings("removal") + public SSLClientChannelInitializer(DiscoveryNode node) { + this.node = node; + + final boolean dualModeEnabled = NetworkModule.TRANSPORT_SSL_DUAL_MODE_ENABLED.get(settings); + hostnameVerificationEnabled = NetworkModule.TRANSPORT_SSL_ENFORCE_HOSTNAME_VERIFICATION.get(settings); + hostnameVerificationResolveHostName = NetworkModule.TRANSPORT_SSL_ENFORCE_HOSTNAME_VERIFICATION_RESOLVE_HOST_NAME.get(settings); + + connectionTestResult = SSLConnectionTestResult.SSL_AVAILABLE; + if (dualModeEnabled) { + SecureConnectionTestUtil sslConnectionTestUtil = new SecureConnectionTestUtil( + node.getAddress().getAddress(), + node.getAddress().getPort() + ); + connectionTestResult = AccessController.doPrivileged( + (PrivilegedAction) sslConnectionTestUtil::testConnection + ); + } + } + + @Override + protected void initChannel(Channel ch) throws Exception { + super.initChannel(ch); + + if (connectionTestResult == SSLConnectionTestResult.OPENSEARCH_PING_FAILED) { + logger.error( + "SSL dual mode is enabled but dual mode handshake and OpenSearch ping has failed during client connection setup, closing channel" + ); + ch.close(); + return; + } + + if (connectionTestResult == SSLConnectionTestResult.SSL_AVAILABLE) { + logger.debug("Connection to {} needs to be ssl, adding ssl handler to the client channel ", node.getHostName()); + ch.pipeline() + .addFirst( + "client_ssl_handler", + new ClientSSLHandler( + settings, + secureTransportSettingsProvider, + hostnameVerificationEnabled, + hostnameVerificationResolveHostName + ) + ); + } else { + logger.debug("Connection to {} needs to be non ssl", node.getHostName()); + } + } + + @Override + public final void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws Exception { + if (cause instanceof DecoderException && cause != null) { + cause = cause.getCause(); + } + + logger.error("Exception during establishing a SSL connection: " + cause, cause); + + super.exceptionCaught(ctx, cause); + } + } +} diff --git a/modules/transport-netty4/src/main/java/org/opensearch/transport/netty4/ssl/SslUtils.java b/modules/transport-netty4/src/main/java/org/opensearch/transport/netty4/ssl/SslUtils.java new file mode 100644 index 0000000000000..8b8223da70c08 --- /dev/null +++ b/modules/transport-netty4/src/main/java/org/opensearch/transport/netty4/ssl/SslUtils.java @@ -0,0 +1,107 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + * Modifications Copyright OpenSearch Contributors. See + * GitHub history for details. + */ +package org.opensearch.transport.netty4.ssl; + +import org.opensearch.OpenSearchSecurityException; + +import javax.net.ssl.SSLContext; +import javax.net.ssl.SSLEngine; + +import java.nio.ByteOrder; +import java.security.NoSuchAlgorithmException; + +import io.netty.buffer.ByteBuf; + +/** + * @see TLSUtil + */ +public class SslUtils { + private static final String[] DEFAULT_SSL_PROTOCOLS = { "TLSv1.3", "TLSv1.2", "TLSv1.1" }; + + private static final int SSL_CONTENT_TYPE_CHANGE_CIPHER_SPEC = 20; + private static final int SSL_CONTENT_TYPE_ALERT = 21; + private static final int SSL_CONTENT_TYPE_HANDSHAKE = 22; + private static final int SSL_CONTENT_TYPE_APPLICATION_DATA = 23; + // CS-SUPPRESS-SINGLE: RegexpSingleline Extensions heartbeat needs special handling by security extension + private static final int SSL_CONTENT_TYPE_EXTENSION_HEARTBEAT = 24; + // CS-ENFORCE-SINGLE + private static final int SSL_RECORD_HEADER_LENGTH = 5; + + private SslUtils() { + + } + + public static SSLEngine createDefaultServerSSLEngine() { + try { + final SSLEngine engine = SSLContext.getDefault().createSSLEngine(); + engine.setEnabledProtocols(DEFAULT_SSL_PROTOCOLS); + engine.setUseClientMode(false); + return engine; + } catch (final NoSuchAlgorithmException ex) { + throw new OpenSearchSecurityException("Unable to initialize default server SSL engine", ex); + } + } + + public static SSLEngine createDefaultClientSSLEngine() { + try { + final SSLEngine engine = SSLContext.getDefault().createSSLEngine(); + engine.setEnabledProtocols(DEFAULT_SSL_PROTOCOLS); + engine.setUseClientMode(true); + return engine; + } catch (final NoSuchAlgorithmException ex) { + throw new OpenSearchSecurityException("Unable to initialize default client SSL engine", ex); + } + } + + static boolean isTLS(ByteBuf buffer) { + int packetLength = 0; + int offset = buffer.readerIndex(); + + // SSLv3 or TLS - Check ContentType + boolean tls; + switch (buffer.getUnsignedByte(offset)) { + case SSL_CONTENT_TYPE_CHANGE_CIPHER_SPEC: + case SSL_CONTENT_TYPE_ALERT: + case SSL_CONTENT_TYPE_HANDSHAKE: + case SSL_CONTENT_TYPE_APPLICATION_DATA: + // CS-SUPPRESS-SINGLE: RegexpSingleline Extensions heartbeat needs special handling by security extension + case SSL_CONTENT_TYPE_EXTENSION_HEARTBEAT: + tls = true; + break; + // CS-ENFORCE-SINGLE + default: + // SSLv2 or bad data + tls = false; + } + + if (tls) { + // SSLv3 or TLS - Check ProtocolVersion + int majorVersion = buffer.getUnsignedByte(offset + 1); + if (majorVersion == 3) { + // SSLv3 or TLS + packetLength = unsignedShortBE(buffer, offset + 3) + SSL_RECORD_HEADER_LENGTH; + if (packetLength <= SSL_RECORD_HEADER_LENGTH) { + // Neither SSLv3 or TLSv1 (i.e. SSLv2 or bad data) + tls = false; + } + } else { + // Neither SSLv3 or TLSv1 (i.e. SSLv2 or bad data) + tls = false; + } + } + + return tls; + } + + private static int unsignedShortBE(ByteBuf buffer, int offset) { + return buffer.order() == ByteOrder.BIG_ENDIAN ? buffer.getUnsignedShort(offset) : buffer.getUnsignedShortLE(offset); + } +} diff --git a/modules/transport-netty4/src/test/java/org/opensearch/http/netty4/Netty4HttpClient.java b/modules/transport-netty4/src/test/java/org/opensearch/http/netty4/Netty4HttpClient.java index 1c381e8000f6b..7cc1a47a5d2a4 100644 --- a/modules/transport-netty4/src/test/java/org/opensearch/http/netty4/Netty4HttpClient.java +++ b/modules/transport-netty4/src/test/java/org/opensearch/http/netty4/Netty4HttpClient.java @@ -32,11 +32,13 @@ package org.opensearch.http.netty4; +import org.opensearch.common.TriFunction; import org.opensearch.common.collect.Tuple; import org.opensearch.core.common.unit.ByteSizeUnit; import org.opensearch.core.common.unit.ByteSizeValue; import org.opensearch.tasks.Task; import org.opensearch.transport.NettyAllocator; +import org.opensearch.transport.netty4.ssl.TrustAllManager; import java.io.Closeable; import java.net.SocketAddress; @@ -47,7 +49,6 @@ import java.util.List; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; -import java.util.function.BiFunction; import io.netty.bootstrap.Bootstrap; import io.netty.buffer.ByteBuf; @@ -86,6 +87,9 @@ import io.netty.handler.codec.http2.HttpToHttp2ConnectionHandler; import io.netty.handler.codec.http2.HttpToHttp2ConnectionHandlerBuilder; import io.netty.handler.codec.http2.InboundHttp2ToHttpAdapterBuilder; +import io.netty.handler.ssl.ClientAuth; +import io.netty.handler.ssl.SslContextBuilder; +import io.netty.handler.ssl.SslHandler; import io.netty.util.AttributeKey; import static io.netty.handler.codec.http.HttpHeaderNames.HOST; @@ -95,7 +99,7 @@ /** * Tiny helper to send http requests over netty. */ -class Netty4HttpClient implements Closeable { +public class Netty4HttpClient implements Closeable { static Collection returnHttpResponseBodies(Collection responses) { List list = new ArrayList<>(responses.size()); @@ -114,31 +118,46 @@ static Collection returnOpaqueIds(Collection responses } private final Bootstrap clientBootstrap; - private final BiFunction, AwaitableChannelInitializer> handlerFactory; + private final TriFunction, Boolean, AwaitableChannelInitializer> handlerFactory; + private final boolean secure; Netty4HttpClient( Bootstrap clientBootstrap, - BiFunction, AwaitableChannelInitializer> handlerFactory + TriFunction, Boolean, AwaitableChannelInitializer> handlerFactory, + boolean secure ) { this.clientBootstrap = clientBootstrap; this.handlerFactory = handlerFactory; + this.secure = secure; } - static Netty4HttpClient http() { + public static Netty4HttpClient https() { return new Netty4HttpClient( new Bootstrap().channel(NettyAllocator.getChannelType()) .option(ChannelOption.ALLOCATOR, NettyAllocator.getAllocator()) .group(new NioEventLoopGroup(1)), - CountDownLatchHandlerHttp::new + CountDownLatchHandlerHttp::new, + true ); } - static Netty4HttpClient http2() { + public static Netty4HttpClient http() { return new Netty4HttpClient( new Bootstrap().channel(NettyAllocator.getChannelType()) .option(ChannelOption.ALLOCATOR, NettyAllocator.getAllocator()) .group(new NioEventLoopGroup(1)), - CountDownLatchHandlerHttp2::new + CountDownLatchHandlerHttp::new, + false + ); + } + + public static Netty4HttpClient http2() { + return new Netty4HttpClient( + new Bootstrap().channel(NettyAllocator.getChannelType()) + .option(ChannelOption.ALLOCATOR, NettyAllocator.getAllocator()) + .group(new NioEventLoopGroup(1)), + CountDownLatchHandlerHttp2::new, + false ); } @@ -148,7 +167,7 @@ public List get(SocketAddress remoteAddress, String... uris) t final HttpRequest httpRequest = new DefaultFullHttpRequest(HTTP_1_1, HttpMethod.GET, uris[i]); httpRequest.headers().add(HOST, "localhost"); httpRequest.headers().add("X-Opaque-ID", String.valueOf(i)); - httpRequest.headers().add(HttpConversionUtil.ExtensionHeaderNames.SCHEME.text(), "http"); + httpRequest.headers().add(HttpConversionUtil.ExtensionHeaderNames.SCHEME.text(), secure ? "http" : "https"); requests.add(httpRequest); } return sendRequests(remoteAddress, requests); @@ -195,7 +214,7 @@ private synchronized List sendRequests(final SocketAddress rem final CountDownLatch latch = new CountDownLatch(requests.size()); final List content = Collections.synchronizedList(new ArrayList<>(requests.size())); - final AwaitableChannelInitializer handler = handlerFactory.apply(latch, content); + final AwaitableChannelInitializer handler = handlerFactory.apply(latch, content, secure); clientBootstrap.handler(handler); ChannelFuture channelFuture = null; @@ -232,19 +251,32 @@ private static class CountDownLatchHandlerHttp extends AwaitableChannelInitializ private final CountDownLatch latch; private final Collection content; + private final boolean secure; - CountDownLatchHandlerHttp(final CountDownLatch latch, final Collection content) { + CountDownLatchHandlerHttp(final CountDownLatch latch, final Collection content, final boolean secure) { this.latch = latch; this.content = content; + this.secure = secure; } @Override - protected void initChannel(SocketChannel ch) { + protected void initChannel(SocketChannel ch) throws Exception { final int maxContentLength = new ByteSizeValue(100, ByteSizeUnit.MB).bytesAsInt(); ch.pipeline().addLast(new HttpResponseDecoder()); ch.pipeline().addLast(new HttpRequestEncoder()); ch.pipeline().addLast(new HttpContentDecompressor()); ch.pipeline().addLast(new HttpObjectAggregator(maxContentLength)); + if (secure) { + final SslHandler sslHandler = new SslHandler( + SslContextBuilder.forClient() + .clientAuth(ClientAuth.NONE) + .trustManager(TrustAllManager.INSTANCE) + .build() + .newEngine(ch.alloc()) + ); + ch.pipeline().addFirst("client_ssl_handler", sslHandler); + } + ch.pipeline().addLast(new SimpleChannelInboundHandler() { @Override protected void channelRead0(ChannelHandlerContext ctx, HttpObject msg) { @@ -283,11 +315,13 @@ private static class CountDownLatchHandlerHttp2 extends AwaitableChannelInitiali private final CountDownLatch latch; private final Collection content; + private final boolean secure; private Http2SettingsHandler settingsHandler; - CountDownLatchHandlerHttp2(final CountDownLatch latch, final Collection content) { + CountDownLatchHandlerHttp2(final CountDownLatch latch, final Collection content, final boolean secure) { this.latch = latch; this.content = content; + this.secure = secure; } @Override diff --git a/modules/transport-netty4/src/test/java/org/opensearch/http/netty4/ssl/SecureNetty4HttpServerTransportTests.java b/modules/transport-netty4/src/test/java/org/opensearch/http/netty4/ssl/SecureNetty4HttpServerTransportTests.java new file mode 100644 index 0000000000000..9ea49d0b24d44 --- /dev/null +++ b/modules/transport-netty4/src/test/java/org/opensearch/http/netty4/ssl/SecureNetty4HttpServerTransportTests.java @@ -0,0 +1,603 @@ +/* + * 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.http.netty4.ssl; + +import org.apache.logging.log4j.message.ParameterizedMessage; +import org.opensearch.OpenSearchException; +import org.opensearch.common.network.NetworkAddress; +import org.opensearch.common.network.NetworkService; +import org.opensearch.common.settings.ClusterSettings; +import org.opensearch.common.settings.Setting; +import org.opensearch.common.settings.Settings; +import org.opensearch.common.unit.TimeValue; +import org.opensearch.common.util.MockBigArrays; +import org.opensearch.common.util.MockPageCacheRecycler; +import org.opensearch.common.util.concurrent.ThreadContext; +import org.opensearch.core.common.bytes.BytesArray; +import org.opensearch.core.common.transport.TransportAddress; +import org.opensearch.core.common.unit.ByteSizeValue; +import org.opensearch.core.indices.breaker.NoneCircuitBreakerService; +import org.opensearch.http.BindHttpException; +import org.opensearch.http.CorsHandler; +import org.opensearch.http.HttpServerTransport; +import org.opensearch.http.HttpTransportSettings; +import org.opensearch.http.NullDispatcher; +import org.opensearch.http.netty4.Netty4HttpClient; +import org.opensearch.plugins.SecureTransportSettingsProvider; +import org.opensearch.rest.BytesRestResponse; +import org.opensearch.rest.RestChannel; +import org.opensearch.rest.RestRequest; +import org.opensearch.telemetry.tracing.noop.NoopTracer; +import org.opensearch.test.OpenSearchTestCase; +import org.opensearch.test.rest.FakeRestRequest; +import org.opensearch.threadpool.TestThreadPool; +import org.opensearch.threadpool.ThreadPool; +import org.opensearch.transport.NettyAllocator; +import org.opensearch.transport.SharedGroupFactory; +import org.opensearch.transport.TcpTransport; +import org.opensearch.transport.netty4.ssl.TrustAllManager; +import org.junit.After; +import org.junit.Before; + +import javax.net.ssl.KeyManagerFactory; +import javax.net.ssl.SSLEngine; +import javax.net.ssl.SSLException; + +import java.io.IOException; +import java.nio.charset.Charset; +import java.nio.charset.StandardCharsets; +import java.security.KeyStore; +import java.security.KeyStoreException; +import java.security.NoSuchAlgorithmException; +import java.security.UnrecoverableKeyException; +import java.security.cert.CertificateException; +import java.util.Collections; +import java.util.Optional; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReference; + +import io.netty.bootstrap.Bootstrap; +import io.netty.buffer.ByteBufUtil; +import io.netty.buffer.Unpooled; +import io.netty.channel.ChannelFuture; +import io.netty.channel.ChannelHandlerAdapter; +import io.netty.channel.ChannelInitializer; +import io.netty.channel.ChannelOption; +import io.netty.channel.nio.NioEventLoopGroup; +import io.netty.channel.socket.SocketChannel; +import io.netty.channel.socket.nio.NioSocketChannel; +import io.netty.handler.codec.TooLongFrameException; +import io.netty.handler.codec.http.DefaultFullHttpRequest; +import io.netty.handler.codec.http.FullHttpRequest; +import io.netty.handler.codec.http.FullHttpResponse; +import io.netty.handler.codec.http.HttpHeaderNames; +import io.netty.handler.codec.http.HttpHeaderValues; +import io.netty.handler.codec.http.HttpMethod; +import io.netty.handler.codec.http.HttpResponseStatus; +import io.netty.handler.codec.http.HttpUtil; +import io.netty.handler.codec.http.HttpVersion; +import io.netty.handler.ssl.ClientAuth; +import io.netty.handler.ssl.SslContextBuilder; + +import static org.opensearch.core.rest.RestStatus.BAD_REQUEST; +import static org.opensearch.core.rest.RestStatus.OK; +import static org.opensearch.http.HttpTransportSettings.SETTING_CORS_ALLOW_ORIGIN; +import static org.opensearch.http.HttpTransportSettings.SETTING_CORS_ENABLED; +import static org.hamcrest.Matchers.containsString; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.instanceOf; +import static org.hamcrest.Matchers.is; + +/** + * Tests for the {@link SecureNetty4HttpServerTransport} class. + */ +public class SecureNetty4HttpServerTransportTests extends OpenSearchTestCase { + + private NetworkService networkService; + private ThreadPool threadPool; + private MockBigArrays bigArrays; + private ClusterSettings clusterSettings; + private SecureTransportSettingsProvider secureTransportSettingsProvider; + + @Before + public void setup() throws Exception { + networkService = new NetworkService(Collections.emptyList()); + threadPool = new TestThreadPool("test"); + bigArrays = new MockBigArrays(new MockPageCacheRecycler(Settings.EMPTY), new NoneCircuitBreakerService()); + clusterSettings = new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); + + secureTransportSettingsProvider = new SecureTransportSettingsProvider() { + @Override + public Optional buildHttpServerExceptionHandler(Settings settings, HttpServerTransport transport) { + return Optional.empty(); + } + + @Override + public Optional buildServerTransportExceptionHandler(Settings settings, TcpTransport transport) { + return Optional.empty(); + } + + @Override + public Optional buildSecureHttpServerEngine(Settings settings, HttpServerTransport transport) throws SSLException { + try { + final KeyStore keyStore = KeyStore.getInstance("PKCS12"); + keyStore.load( + SecureNetty4HttpServerTransportTests.class.getResourceAsStream("/netty4-secure.jks"), + "password".toCharArray() + ); + + final KeyManagerFactory keyManagerFactory = KeyManagerFactory.getInstance("SunX509"); + keyManagerFactory.init(keyStore, "password".toCharArray()); + + SSLEngine engine = SslContextBuilder.forServer(keyManagerFactory) + .trustManager(TrustAllManager.INSTANCE) + .build() + .newEngine(NettyAllocator.getAllocator()); + return Optional.of(engine); + } catch (final IOException | NoSuchAlgorithmException | UnrecoverableKeyException | KeyStoreException + | CertificateException ex) { + throw new SSLException(ex); + } + } + + @Override + public Optional buildSecureServerTransportEngine(Settings settings, TcpTransport transport) throws SSLException { + return Optional.empty(); + } + + @Override + public Optional buildSecureClientTransportEngine(Settings settings, String hostname, int port) throws SSLException { + return Optional.of( + SslContextBuilder.forClient() + .clientAuth(ClientAuth.NONE) + .trustManager(TrustAllManager.INSTANCE) + .build() + .newEngine(NettyAllocator.getAllocator()) + ); + } + }; + } + + @After + public void shutdown() throws Exception { + if (threadPool != null) { + threadPool.shutdownNow(); + } + threadPool = null; + networkService = null; + bigArrays = null; + clusterSettings = null; + } + + /** + * Test that {@link SecureNetty4HttpServerTransport} supports the "Expect: 100-continue" HTTP header + * @throws InterruptedException if the client communication with the server is interrupted + */ + public void testExpectContinueHeader() throws InterruptedException { + final Settings settings = createSettings(); + final int contentLength = randomIntBetween(1, HttpTransportSettings.SETTING_HTTP_MAX_CONTENT_LENGTH.get(settings).bytesAsInt()); + runExpectHeaderTest(settings, HttpHeaderValues.CONTINUE.toString(), contentLength, HttpResponseStatus.CONTINUE); + } + + /** + * Test that {@link SecureNetty4HttpServerTransport} responds to a + * 100-continue expectation with too large a content-length + * with a 413 status. + * @throws InterruptedException if the client communication with the server is interrupted + */ + public void testExpectContinueHeaderContentLengthTooLong() throws InterruptedException { + final String key = HttpTransportSettings.SETTING_HTTP_MAX_CONTENT_LENGTH.getKey(); + final int maxContentLength = randomIntBetween(1, 104857600); + final Settings settings = createBuilderWithPort().put(key, maxContentLength + "b").build(); + final int contentLength = randomIntBetween(maxContentLength + 1, Integer.MAX_VALUE); + runExpectHeaderTest(settings, HttpHeaderValues.CONTINUE.toString(), contentLength, HttpResponseStatus.REQUEST_ENTITY_TOO_LARGE); + } + + /** + * Test that {@link SecureNetty4HttpServerTransport} responds to an unsupported expectation with a 417 status. + * @throws InterruptedException if the client communication with the server is interrupted + */ + public void testExpectUnsupportedExpectation() throws InterruptedException { + Settings settings = createSettings(); + runExpectHeaderTest(settings, "chocolate=yummy", 0, HttpResponseStatus.EXPECTATION_FAILED); + } + + private void runExpectHeaderTest( + final Settings settings, + final String expectation, + final int contentLength, + final HttpResponseStatus expectedStatus + ) throws InterruptedException { + + final HttpServerTransport.Dispatcher dispatcher = new HttpServerTransport.Dispatcher() { + @Override + public void dispatchRequest(RestRequest request, RestChannel channel, ThreadContext threadContext) { + channel.sendResponse(new BytesRestResponse(OK, BytesRestResponse.TEXT_CONTENT_TYPE, new BytesArray("done"))); + } + + @Override + public void dispatchBadRequest(RestChannel channel, ThreadContext threadContext, Throwable cause) { + logger.error( + new ParameterizedMessage("--> Unexpected bad request [{}]", FakeRestRequest.requestToString(channel.request())), + cause + ); + throw new AssertionError(); + } + }; + try ( + SecureNetty4HttpServerTransport transport = new SecureNetty4HttpServerTransport( + settings, + networkService, + bigArrays, + threadPool, + xContentRegistry(), + dispatcher, + clusterSettings, + new SharedGroupFactory(settings), + secureTransportSettingsProvider, + NoopTracer.INSTANCE + ) + ) { + transport.start(); + final TransportAddress remoteAddress = randomFrom(transport.boundAddress().boundAddresses()); + try (Netty4HttpClient client = Netty4HttpClient.https()) { + final FullHttpRequest request = new DefaultFullHttpRequest(HttpVersion.HTTP_1_1, HttpMethod.POST, "/"); + request.headers().set(HttpHeaderNames.EXPECT, expectation); + HttpUtil.setContentLength(request, contentLength); + + final FullHttpResponse response = client.send(remoteAddress.address(), request); + try { + assertThat(response.status(), equalTo(expectedStatus)); + if (expectedStatus.equals(HttpResponseStatus.CONTINUE)) { + final FullHttpRequest continuationRequest = new DefaultFullHttpRequest( + HttpVersion.HTTP_1_1, + HttpMethod.POST, + "/", + Unpooled.EMPTY_BUFFER + ); + final FullHttpResponse continuationResponse = client.send(remoteAddress.address(), continuationRequest); + try { + assertThat(continuationResponse.status(), is(HttpResponseStatus.OK)); + assertThat( + new String(ByteBufUtil.getBytes(continuationResponse.content()), StandardCharsets.UTF_8), + is("done") + ); + } finally { + continuationResponse.release(); + } + } + } finally { + response.release(); + } + } + } + } + + public void testBindUnavailableAddress() { + Settings initialSettings = createSettings(); + try ( + SecureNetty4HttpServerTransport transport = new SecureNetty4HttpServerTransport( + initialSettings, + networkService, + bigArrays, + threadPool, + xContentRegistry(), + new NullDispatcher(), + clusterSettings, + new SharedGroupFactory(Settings.EMPTY), + secureTransportSettingsProvider, + NoopTracer.INSTANCE + ) + ) { + transport.start(); + TransportAddress remoteAddress = randomFrom(transport.boundAddress().boundAddresses()); + Settings settings = Settings.builder() + .put("http.port", remoteAddress.getPort()) + .put("network.host", remoteAddress.getAddress()) + .build(); + try ( + SecureNetty4HttpServerTransport otherTransport = new SecureNetty4HttpServerTransport( + settings, + networkService, + bigArrays, + threadPool, + xContentRegistry(), + new NullDispatcher(), + clusterSettings, + new SharedGroupFactory(settings), + secureTransportSettingsProvider, + NoopTracer.INSTANCE + ) + ) { + BindHttpException bindHttpException = expectThrows(BindHttpException.class, otherTransport::start); + assertEquals("Failed to bind to " + NetworkAddress.format(remoteAddress.address()), bindHttpException.getMessage()); + } + } + } + + public void testBadRequest() throws InterruptedException { + final AtomicReference causeReference = new AtomicReference<>(); + final HttpServerTransport.Dispatcher dispatcher = new HttpServerTransport.Dispatcher() { + + @Override + public void dispatchRequest(final RestRequest request, final RestChannel channel, final ThreadContext threadContext) { + logger.error("--> Unexpected successful request [{}]", FakeRestRequest.requestToString(request)); + throw new AssertionError(); + } + + @Override + public void dispatchBadRequest(final RestChannel channel, final ThreadContext threadContext, final Throwable cause) { + causeReference.set(cause); + try { + final OpenSearchException e = new OpenSearchException("you sent a bad request and you should feel bad"); + channel.sendResponse(new BytesRestResponse(channel, BAD_REQUEST, e)); + } catch (final IOException e) { + throw new AssertionError(e); + } + } + + }; + + final Settings settings; + final int maxInitialLineLength; + final Setting httpMaxInitialLineLengthSetting = HttpTransportSettings.SETTING_HTTP_MAX_INITIAL_LINE_LENGTH; + if (randomBoolean()) { + maxInitialLineLength = httpMaxInitialLineLengthSetting.getDefault(Settings.EMPTY).bytesAsInt(); + settings = createSettings(); + } else { + maxInitialLineLength = randomIntBetween(1, 8192); + settings = createBuilderWithPort().put(httpMaxInitialLineLengthSetting.getKey(), maxInitialLineLength + "b").build(); + } + + try ( + SecureNetty4HttpServerTransport transport = new SecureNetty4HttpServerTransport( + settings, + networkService, + bigArrays, + threadPool, + xContentRegistry(), + dispatcher, + clusterSettings, + new SharedGroupFactory(settings), + secureTransportSettingsProvider, + NoopTracer.INSTANCE + ) + ) { + transport.start(); + final TransportAddress remoteAddress = randomFrom(transport.boundAddress().boundAddresses()); + + try (Netty4HttpClient client = Netty4HttpClient.https()) { + final String url = "/" + new String(new byte[maxInitialLineLength], Charset.forName("UTF-8")); + final FullHttpRequest request = new DefaultFullHttpRequest(HttpVersion.HTTP_1_1, HttpMethod.GET, url); + + final FullHttpResponse response = client.send(remoteAddress.address(), request); + try { + assertThat(response.status(), equalTo(HttpResponseStatus.BAD_REQUEST)); + assertThat( + new String(response.content().array(), Charset.forName("UTF-8")), + containsString("you sent a bad request and you should feel bad") + ); + } finally { + response.release(); + } + } + } + + assertNotNull(causeReference.get()); + assertThat(causeReference.get(), instanceOf(TooLongFrameException.class)); + } + + public void testLargeCompressedResponse() throws InterruptedException { + final String responseString = randomAlphaOfLength(4 * 1024 * 1024); + final String url = "/thing"; + final HttpServerTransport.Dispatcher dispatcher = new HttpServerTransport.Dispatcher() { + + @Override + public void dispatchRequest(final RestRequest request, final RestChannel channel, final ThreadContext threadContext) { + if (url.equals(request.uri())) { + channel.sendResponse(new BytesRestResponse(OK, responseString)); + } else { + logger.error("--> Unexpected successful uri [{}]", request.uri()); + throw new AssertionError(); + } + } + + @Override + public void dispatchBadRequest(final RestChannel channel, final ThreadContext threadContext, final Throwable cause) { + logger.error( + new ParameterizedMessage("--> Unexpected bad request [{}]", FakeRestRequest.requestToString(channel.request())), + cause + ); + throw new AssertionError(); + } + + }; + + try ( + SecureNetty4HttpServerTransport transport = new SecureNetty4HttpServerTransport( + Settings.EMPTY, + networkService, + bigArrays, + threadPool, + xContentRegistry(), + dispatcher, + clusterSettings, + new SharedGroupFactory(Settings.EMPTY), + secureTransportSettingsProvider, + NoopTracer.INSTANCE + ) + ) { + transport.start(); + final TransportAddress remoteAddress = randomFrom(transport.boundAddress().boundAddresses()); + + try (Netty4HttpClient client = Netty4HttpClient.https()) { + DefaultFullHttpRequest request = new DefaultFullHttpRequest(HttpVersion.HTTP_1_1, HttpMethod.GET, url); + request.headers().add(HttpHeaderNames.ACCEPT_ENCODING, randomFrom("deflate", "gzip")); + final FullHttpResponse response = client.send(remoteAddress.address(), request); + try { + assertThat(response.status(), equalTo(HttpResponseStatus.OK)); + byte[] bytes = new byte[response.content().readableBytes()]; + response.content().readBytes(bytes); + assertThat(new String(bytes, StandardCharsets.UTF_8), equalTo(responseString)); + } finally { + response.release(); + } + } + } + } + + public void testCorsRequest() throws InterruptedException { + final HttpServerTransport.Dispatcher dispatcher = new HttpServerTransport.Dispatcher() { + + @Override + public void dispatchRequest(final RestRequest request, final RestChannel channel, final ThreadContext threadContext) { + logger.error("--> Unexpected successful request [{}]", FakeRestRequest.requestToString(request)); + throw new AssertionError(); + } + + @Override + public void dispatchBadRequest(final RestChannel channel, final ThreadContext threadContext, final Throwable cause) { + logger.error( + new ParameterizedMessage("--> Unexpected bad request [{}]", FakeRestRequest.requestToString(channel.request())), + cause + ); + throw new AssertionError(); + } + + }; + + final Settings settings = createBuilderWithPort().put(SETTING_CORS_ENABLED.getKey(), true) + .put(SETTING_CORS_ALLOW_ORIGIN.getKey(), "test-cors.org") + .build(); + + try ( + SecureNetty4HttpServerTransport transport = new SecureNetty4HttpServerTransport( + settings, + networkService, + bigArrays, + threadPool, + xContentRegistry(), + dispatcher, + new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS), + new SharedGroupFactory(settings), + secureTransportSettingsProvider, + NoopTracer.INSTANCE + ) + ) { + transport.start(); + final TransportAddress remoteAddress = randomFrom(transport.boundAddress().boundAddresses()); + + // Test pre-flight request + try (Netty4HttpClient client = Netty4HttpClient.https()) { + final FullHttpRequest request = new DefaultFullHttpRequest(HttpVersion.HTTP_1_1, HttpMethod.OPTIONS, "/"); + request.headers().add(CorsHandler.ORIGIN, "test-cors.org"); + request.headers().add(CorsHandler.ACCESS_CONTROL_REQUEST_METHOD, "POST"); + + final FullHttpResponse response = client.send(remoteAddress.address(), request); + try { + assertThat(response.status(), equalTo(HttpResponseStatus.OK)); + assertThat(response.headers().get(CorsHandler.ACCESS_CONTROL_ALLOW_ORIGIN), equalTo("test-cors.org")); + assertThat(response.headers().get(CorsHandler.VARY), equalTo(CorsHandler.ORIGIN)); + assertTrue(response.headers().contains(CorsHandler.DATE)); + } finally { + response.release(); + } + } + + // Test short-circuited request + try (Netty4HttpClient client = Netty4HttpClient.https()) { + final FullHttpRequest request = new DefaultFullHttpRequest(HttpVersion.HTTP_1_1, HttpMethod.GET, "/"); + request.headers().add(CorsHandler.ORIGIN, "google.com"); + + final FullHttpResponse response = client.send(remoteAddress.address(), request); + try { + assertThat(response.status(), equalTo(HttpResponseStatus.FORBIDDEN)); + } finally { + response.release(); + } + } + } + } + + public void testReadTimeout() throws Exception { + final HttpServerTransport.Dispatcher dispatcher = new HttpServerTransport.Dispatcher() { + + @Override + public void dispatchRequest(final RestRequest request, final RestChannel channel, final ThreadContext threadContext) { + logger.error("--> Unexpected successful request [{}]", FakeRestRequest.requestToString(request)); + throw new AssertionError("Should not have received a dispatched request"); + } + + @Override + public void dispatchBadRequest(final RestChannel channel, final ThreadContext threadContext, final Throwable cause) { + logger.error( + new ParameterizedMessage("--> Unexpected bad request [{}]", FakeRestRequest.requestToString(channel.request())), + cause + ); + throw new AssertionError("Should not have received a dispatched request"); + } + + }; + + Settings settings = createBuilderWithPort().put( + HttpTransportSettings.SETTING_HTTP_READ_TIMEOUT.getKey(), + new TimeValue(randomIntBetween(100, 300)) + ).build(); + + NioEventLoopGroup group = new NioEventLoopGroup(); + try ( + SecureNetty4HttpServerTransport transport = new SecureNetty4HttpServerTransport( + settings, + networkService, + bigArrays, + threadPool, + xContentRegistry(), + dispatcher, + new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS), + new SharedGroupFactory(settings), + secureTransportSettingsProvider, + NoopTracer.INSTANCE + ) + ) { + transport.start(); + final TransportAddress remoteAddress = randomFrom(transport.boundAddress().boundAddresses()); + + CountDownLatch channelClosedLatch = new CountDownLatch(1); + + Bootstrap clientBootstrap = new Bootstrap().option(ChannelOption.ALLOCATOR, NettyAllocator.getAllocator()) + .channel(NioSocketChannel.class) + .handler(new ChannelInitializer() { + + @Override + protected void initChannel(SocketChannel ch) { + ch.pipeline().addLast(new ChannelHandlerAdapter() { + }); + + } + }) + .group(group); + ChannelFuture connect = clientBootstrap.connect(remoteAddress.address()); + connect.channel().closeFuture().addListener(future -> channelClosedLatch.countDown()); + + assertTrue("Channel should be closed due to read timeout", channelClosedLatch.await(1, TimeUnit.MINUTES)); + + } finally { + group.shutdownGracefully().await(); + } + } + + private Settings createSettings() { + return createBuilderWithPort().build(); + } + + private Settings.Builder createBuilderWithPort() { + return Settings.builder().put(HttpTransportSettings.SETTING_HTTP_PORT.getKey(), getPortRange()); + } +} diff --git a/modules/transport-netty4/src/test/java/org/opensearch/transport/netty4/ssl/SimpleSecureNetty4TransportTests.java b/modules/transport-netty4/src/test/java/org/opensearch/transport/netty4/ssl/SimpleSecureNetty4TransportTests.java new file mode 100644 index 0000000000000..0cae58b8efa2a --- /dev/null +++ b/modules/transport-netty4/src/test/java/org/opensearch/transport/netty4/ssl/SimpleSecureNetty4TransportTests.java @@ -0,0 +1,234 @@ +/* + * 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.transport.netty4.ssl; + +import org.opensearch.Version; +import org.opensearch.cluster.node.DiscoveryNode; +import org.opensearch.common.network.NetworkService; +import org.opensearch.common.settings.ClusterSettings; +import org.opensearch.common.settings.Settings; +import org.opensearch.common.util.PageCacheRecycler; +import org.opensearch.common.util.io.IOUtils; +import org.opensearch.common.util.net.NetUtils; +import org.opensearch.core.action.ActionListener; +import org.opensearch.core.common.io.stream.NamedWriteableRegistry; +import org.opensearch.core.common.transport.TransportAddress; +import org.opensearch.core.indices.breaker.NoneCircuitBreakerService; +import org.opensearch.http.HttpServerTransport; +import org.opensearch.plugins.SecureTransportSettingsProvider; +import org.opensearch.telemetry.tracing.noop.NoopTracer; +import org.opensearch.test.transport.MockTransportService; +import org.opensearch.test.transport.StubbableTransport; +import org.opensearch.transport.AbstractSimpleTransportTestCase; +import org.opensearch.transport.ConnectTransportException; +import org.opensearch.transport.ConnectionProfile; +import org.opensearch.transport.Netty4NioSocketChannel; +import org.opensearch.transport.NettyAllocator; +import org.opensearch.transport.SharedGroupFactory; +import org.opensearch.transport.TcpChannel; +import org.opensearch.transport.TcpTransport; +import org.opensearch.transport.TestProfiles; +import org.opensearch.transport.Transport; +import org.opensearch.transport.netty4.Netty4TcpChannel; + +import javax.net.ssl.KeyManagerFactory; +import javax.net.ssl.SSLEngine; +import javax.net.ssl.SSLException; + +import java.io.IOException; +import java.net.InetAddress; +import java.net.UnknownHostException; +import java.nio.channels.SocketChannel; +import java.security.KeyStore; +import java.security.KeyStoreException; +import java.security.NoSuchAlgorithmException; +import java.security.UnrecoverableKeyException; +import java.security.cert.CertificateException; +import java.util.Collections; +import java.util.Optional; + +import io.netty.handler.ssl.ClientAuth; +import io.netty.handler.ssl.SslContextBuilder; + +import static java.util.Collections.emptyMap; +import static java.util.Collections.emptySet; +import static org.hamcrest.Matchers.containsString; +import static org.hamcrest.Matchers.hasItem; +import static org.hamcrest.Matchers.instanceOf; +import static org.hamcrest.Matchers.lessThanOrEqualTo; + +public class SimpleSecureNetty4TransportTests extends AbstractSimpleTransportTestCase { + @Override + protected Transport build(Settings settings, final Version version, ClusterSettings clusterSettings, boolean doHandshake) { + NamedWriteableRegistry namedWriteableRegistry = new NamedWriteableRegistry(Collections.emptyList()); + final SecureTransportSettingsProvider secureTransportSettingsProvider = new SecureTransportSettingsProvider() { + @Override + public Optional buildHttpServerExceptionHandler(Settings settings, HttpServerTransport transport) { + return Optional.empty(); + } + + @Override + public Optional buildServerTransportExceptionHandler(Settings settings, TcpTransport transport) { + return Optional.empty(); + } + + @Override + public Optional buildSecureHttpServerEngine(Settings settings, HttpServerTransport transport) throws SSLException { + try { + final KeyStore keyStore = KeyStore.getInstance("PKCS12"); + keyStore.load( + SimpleSecureNetty4TransportTests.class.getResourceAsStream("/netty4-secure.jks"), + "password".toCharArray() + ); + + final KeyManagerFactory keyManagerFactory = KeyManagerFactory.getInstance("SunX509"); + keyManagerFactory.init(keyStore, "password".toCharArray()); + + SSLEngine engine = SslContextBuilder.forServer(keyManagerFactory) + .trustManager(TrustAllManager.INSTANCE) + .build() + .newEngine(NettyAllocator.getAllocator()); + return Optional.of(engine); + } catch (final IOException | NoSuchAlgorithmException | UnrecoverableKeyException | KeyStoreException + | CertificateException ex) { + throw new SSLException(ex); + } + } + + @Override + public Optional buildSecureServerTransportEngine(Settings settings, TcpTransport transport) throws SSLException { + try { + final KeyStore keyStore = KeyStore.getInstance("PKCS12"); + keyStore.load( + SimpleSecureNetty4TransportTests.class.getResourceAsStream("/netty4-secure.jks"), + "password".toCharArray() + ); + + final KeyManagerFactory keyManagerFactory = KeyManagerFactory.getInstance("SunX509"); + keyManagerFactory.init(keyStore, "password".toCharArray()); + + SSLEngine engine = SslContextBuilder.forServer(keyManagerFactory) + .clientAuth(ClientAuth.NONE) + .trustManager(TrustAllManager.INSTANCE) + .build() + .newEngine(NettyAllocator.getAllocator()); + return Optional.of(engine); + } catch (final IOException | NoSuchAlgorithmException | UnrecoverableKeyException | KeyStoreException + | CertificateException ex) { + throw new SSLException(ex); + } + + } + + @Override + public Optional buildSecureClientTransportEngine(Settings settings, String hostname, int port) throws SSLException { + return Optional.of( + SslContextBuilder.forClient() + .clientAuth(ClientAuth.NONE) + .trustManager(TrustAllManager.INSTANCE) + .build() + .newEngine(NettyAllocator.getAllocator()) + ); + } + }; + + return new SecureNetty4Transport( + settings, + version, + threadPool, + new NetworkService(Collections.emptyList()), + PageCacheRecycler.NON_RECYCLING_INSTANCE, + namedWriteableRegistry, + new NoneCircuitBreakerService(), + new SharedGroupFactory(settings), + secureTransportSettingsProvider, + NoopTracer.INSTANCE + ) { + + @Override + public void executeHandshake( + DiscoveryNode node, + TcpChannel channel, + ConnectionProfile profile, + ActionListener listener + ) { + if (doHandshake) { + super.executeHandshake(node, channel, profile, listener); + } else { + listener.onResponse(version.minimumCompatibilityVersion()); + } + } + }; + } + + public void testConnectException() throws UnknownHostException { + try { + serviceA.connectToNode( + new DiscoveryNode( + "C", + new TransportAddress(InetAddress.getByName("localhost"), 9876), + emptyMap(), + emptySet(), + Version.CURRENT + ) + ); + fail("Expected ConnectTransportException"); + } catch (ConnectTransportException e) { + assertThat(e.getMessage(), containsString("connect_exception")); + assertThat(e.getMessage(), containsString("[127.0.0.1:9876]")); + } + } + + public void testDefaultKeepAliveSettings() throws IOException { + assumeTrue("setting default keepalive options not supported on this platform", (IOUtils.LINUX || IOUtils.MAC_OS_X)); + try ( + MockTransportService serviceC = buildService("TS_C", Version.CURRENT, Settings.EMPTY); + MockTransportService serviceD = buildService("TS_D", Version.CURRENT, Settings.EMPTY) + ) { + serviceC.start(); + serviceC.acceptIncomingRequests(); + serviceD.start(); + serviceD.acceptIncomingRequests(); + + try (Transport.Connection connection = serviceC.openConnection(serviceD.getLocalDiscoNode(), TestProfiles.LIGHT_PROFILE)) { + assertThat(connection, instanceOf(StubbableTransport.WrappedConnection.class)); + Transport.Connection conn = ((StubbableTransport.WrappedConnection) connection).getConnection(); + assertThat(conn, instanceOf(TcpTransport.NodeChannels.class)); + TcpTransport.NodeChannels nodeChannels = (TcpTransport.NodeChannels) conn; + for (TcpChannel channel : nodeChannels.getChannels()) { + assertFalse(channel.isServerChannel()); + checkDefaultKeepAliveOptions(channel); + } + + assertThat(serviceD.getOriginalTransport(), instanceOf(TcpTransport.class)); + for (TcpChannel channel : getAcceptedChannels((TcpTransport) serviceD.getOriginalTransport())) { + assertTrue(channel.isServerChannel()); + checkDefaultKeepAliveOptions(channel); + } + } + } + } + + private void checkDefaultKeepAliveOptions(TcpChannel channel) throws IOException { + assertThat(channel, instanceOf(Netty4TcpChannel.class)); + Netty4TcpChannel nettyChannel = (Netty4TcpChannel) channel; + assertThat(nettyChannel.getNettyChannel(), instanceOf(Netty4NioSocketChannel.class)); + Netty4NioSocketChannel netty4NioSocketChannel = (Netty4NioSocketChannel) nettyChannel.getNettyChannel(); + SocketChannel socketChannel = netty4NioSocketChannel.javaChannel(); + assertThat(socketChannel.supportedOptions(), hasItem(NetUtils.getTcpKeepIdleSocketOptionOrNull())); + Integer keepIdle = socketChannel.getOption(NetUtils.getTcpKeepIdleSocketOptionOrNull()); + assertNotNull(keepIdle); + assertThat(keepIdle, lessThanOrEqualTo(500)); + assertThat(socketChannel.supportedOptions(), hasItem(NetUtils.getTcpKeepIntervalSocketOptionOrNull())); + Integer keepInterval = socketChannel.getOption(NetUtils.getTcpKeepIntervalSocketOptionOrNull()); + assertNotNull(keepInterval); + assertThat(keepInterval, lessThanOrEqualTo(500)); + } + +} diff --git a/modules/transport-netty4/src/test/java/org/opensearch/transport/netty4/ssl/TrustAllManager.java b/modules/transport-netty4/src/test/java/org/opensearch/transport/netty4/ssl/TrustAllManager.java new file mode 100644 index 0000000000000..a38c542b5780e --- /dev/null +++ b/modules/transport-netty4/src/test/java/org/opensearch/transport/netty4/ssl/TrustAllManager.java @@ -0,0 +1,28 @@ +/* + * 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.transport.netty4.ssl; + +import javax.net.ssl.X509TrustManager; + +import java.security.cert.CertificateException; +import java.security.cert.X509Certificate; + +public class TrustAllManager implements X509TrustManager { + public static final X509TrustManager INSTANCE = new TrustAllManager(); + + private TrustAllManager() {} + + public void checkClientTrusted(X509Certificate[] chain, String authType) throws CertificateException {} + + public void checkServerTrusted(X509Certificate[] chain, String authType) throws CertificateException {} + + public X509Certificate[] getAcceptedIssuers() { + return new X509Certificate[0]; + } +} diff --git a/modules/transport-netty4/src/test/resources/README.txt b/modules/transport-netty4/src/test/resources/README.txt new file mode 100644 index 0000000000000..c8cec5d3803a4 --- /dev/null +++ b/modules/transport-netty4/src/test/resources/README.txt @@ -0,0 +1,17 @@ +#!/usr/bin/env bash +# +# This is README describes how the certificates in this directory were created. +# This file can also be executed as a script +# + +# 1. Create certificate key + +openssl req -x509 -sha256 -newkey rsa:2048 -keyout certificate.key -out certificate.crt -days 1024 -nodes + +# 2. Export the certificate in pkcs12 format + +openssl pkcs12 -export -in certificate.crt -inkey certificate.key -out server.p12 -name netty4-secure -password pass:password + +# 3. Import the certificate into JDK keystore (PKCS12 type) + +keytool -importkeystore -srcstorepass password -destkeystore netty4-secure.jks -srckeystore server.p12 -srcstoretype PKCS12 -alias netty4-secure -deststorepass password \ No newline at end of file diff --git a/modules/transport-netty4/src/test/resources/certificate.crt b/modules/transport-netty4/src/test/resources/certificate.crt new file mode 100644 index 0000000000000..54c78fdbcf6de --- /dev/null +++ b/modules/transport-netty4/src/test/resources/certificate.crt @@ -0,0 +1,22 @@ +-----BEGIN CERTIFICATE----- +MIIDkzCCAnugAwIBAgIUddAawr5zygcd+Dcn9WVDpO4BJ7YwDQYJKoZIhvcNAQEL +BQAwWTELMAkGA1UEBhMCQVUxEzARBgNVBAgMClNvbWUtU3RhdGUxITAfBgNVBAoM +GEludGVybmV0IFdpZGdpdHMgUHR5IEx0ZDESMBAGA1UEAwwJbG9jYWxob3N0MB4X +DTI0MDMxNDE5NDQzOVoXDTI3MDEwMjE5NDQzOVowWTELMAkGA1UEBhMCQVUxEzAR +BgNVBAgMClNvbWUtU3RhdGUxITAfBgNVBAoMGEludGVybmV0IFdpZGdpdHMgUHR5 +IEx0ZDESMBAGA1UEAwwJbG9jYWxob3N0MIIBIjANBgkqhkiG9w0BAQEFAAOCAQ8A +MIIBCgKCAQEAzjOKkg6Iba5zfZ8b/RYw+PGmGEfbdGuuF10Wz4Jmx/Nk4VfDLxdh +TW8VllUL2JD7uPkjABj7pW3awAbvIJ+VGbKqfBr1Nsz0mPPzhT8cfuMH/FDZgQs3 +4HuqDKr0LfC1Kw5E3WF0GVMBDNu0U+nKoeqySeYjGdxDnd3W4cqK5AnUxL0RnIny +Bw7ZuhcU55XndH/Xauro/2EpvJduDsWMdqt7ZfIf1TOmaiQHK+82yb/drVaJbczK +uTpn1Kv2bnzkQEckgq+z1dLNOOyvP2xf+nsziw5ilJe92e5GJOUJYFAlEgUAGpfD +dv6j/gTRYvdJCJItOQEQtektNCAZsoc0wwIDAQABo1MwUTAdBgNVHQ4EFgQUzHts +wIt+zhB/R4U4Do2P6rr0YhkwHwYDVR0jBBgwFoAUzHtswIt+zhB/R4U4Do2P6rr0 +YhkwDwYDVR0TAQH/BAUwAwEB/zANBgkqhkiG9w0BAQsFAAOCAQEAveh870jJX7vt +oLCrdugsyo79pR4f7Nr1kUy3jJrfoaoUmrjiiiHWgT22fGwp7j1GZF2mVfo8YVaK +63YNn5gB2NNZhguPOFC4AdvHRYOKRBOaOvWK8oq7BcJ//18JYI/pPnpgkYvJjqv4 +gFKaZX9qWtujHpAmKiVGs7pwYGNXfixPHRNV4owcfHMIH5dhbbqT49j94xVpjbXs +OymKtFl4kpCE/0LzKFrFcuu55Am1VLBHx2cPpHLOipgUcF5BHFlQ8AXiCMOwfPAw +d22mLB6Gt1oVEpyvQHYd3e04FetEXQ9E8T+NKWZx/8Ucf+IWBYmZBRxch6O83xgk +bAbGzqkbzQ== +-----END CERTIFICATE----- diff --git a/modules/transport-netty4/src/test/resources/certificate.key b/modules/transport-netty4/src/test/resources/certificate.key new file mode 100644 index 0000000000000..228350180935d --- /dev/null +++ b/modules/transport-netty4/src/test/resources/certificate.key @@ -0,0 +1,28 @@ +-----BEGIN PRIVATE KEY----- +MIIEvgIBADANBgkqhkiG9w0BAQEFAASCBKgwggSkAgEAAoIBAQDOM4qSDohtrnN9 +nxv9FjD48aYYR9t0a64XXRbPgmbH82ThV8MvF2FNbxWWVQvYkPu4+SMAGPulbdrA +Bu8gn5UZsqp8GvU2zPSY8/OFPxx+4wf8UNmBCzfge6oMqvQt8LUrDkTdYXQZUwEM +27RT6cqh6rJJ5iMZ3EOd3dbhyorkCdTEvRGcifIHDtm6FxTnled0f9dq6uj/YSm8 +l24OxYx2q3tl8h/VM6ZqJAcr7zbJv92tVoltzMq5OmfUq/ZufORARySCr7PV0s04 +7K8/bF/6ezOLDmKUl73Z7kYk5QlgUCUSBQAal8N2/qP+BNFi90kIki05ARC16S00 +IBmyhzTDAgMBAAECggEAVOdiElvLjyX6xeoC00YU6hxOIMdNtHU2HMamwtDV01UD +38mMQ9KjrQelYt4n34drLrHe2IZw75/5J4JzagJrmUY47psHBwaDXItuZRokeJaw +zhLYTEs7OcKRtV+a5WOspUrdzi33aQoFb67zZG3qkpsZyFXrdBV+/fy/Iv+MCvLH +xR0jQ5mzE3cw20R7S4nddChBA/y8oKGOo6QRf2SznC1jL/+yolHvJPEn1v8AUxYm +BMPHxj1O0c4M4IxnJQ3Y5Jy9OaFMyMsFlF1hVhc/3LDDxDyOuBsVsFDicojyrRea +GKngIke0yezy7Wo4NUcp8YQhafonpWVsSJJdOUotcQKBgQD0rihFBXVtcG1d/Vy7 +FvLHrmccD56JNV744LSn2CDM7W1IulNbDUZINdCFqL91u5LpxozeE1FPY1nhwncJ +N7V7XYCaSLCuV1YJzRmUCjnzk2RyopGpzWog3f9uUFGgrk1HGbNAv99k/REya6Iu +IRSkuQhaJOj3bRXzonh0K4GjewKBgQDXvamtCioOUMSP8vq919YMkBw7F+z/fr0p +pamO8HL9eewAUg6N92JQ9kobSo/GptdmdHIjs8LqnS5C3H13GX5Qlf5GskOlCpla +V55ElaSp0gvKwWE168U7gQH4etPQAXXJrOGFaGbPj9W81hTUud7HVE88KYdfWTBo +I7TuE25tWQKBgBRjcr2Vn9xXsvVTCGgamG5lLPhcoNREGz7X0pXt34XT/vhBdnKu +331i5pZMom+YCrzqK5DRwUPBPpseTjb5amj2OKIijn5ojqXQbmI0m/GdBZC71TF2 +CXLlrMQvcy3VeGEFVjd+BYpvwAAYkfIQFZ1IQdbpHnSHpX2guzLK8UmDAoGBANUy +PIcf0EetUVHfkCIjNQfdMcjD8BTcLhsF9vWmcDxFTA9VB8ULf0D64mjt2f85yQsa +b+EQN8KZ6alxMxuLOeRxFYLPj0F9o+Y/R8wHBV48kCKhz2r1v0b6SfQ/jSm1B61x +BrxLW64qOdIOzS8bLyhUDKkrcPesr8V548aRtUKhAoGBAKlNJFd8BCGKD9Td+3dE +oP1iHTX5XZ+cQIqL0e+GMQlK4HnQP566DFZU5/GHNNAfmyxd5iSRwhTqPMHRAmOb +pqQwsyufx0dFeIBxeSO3Z6jW5h2sl4nBipZpw9bzv6EBL1xRr0SfMNZzdnf4JFzc +0htGo/VO93Z2pv8w7uGUz1nN +-----END PRIVATE KEY----- diff --git a/modules/transport-netty4/src/test/resources/netty4-secure.jks b/modules/transport-netty4/src/test/resources/netty4-secure.jks new file mode 100644 index 0000000000000000000000000000000000000000..59dfd31c2a1567c6fbae386aa8f15c563bc66ae0 GIT binary patch literal 2790 zcma)8XEYm(8crgiLaZ7k_^3S_Tdh)5I#8=SvDFOy$b^} z0rT-tEEo?BJ)r{;cu4DiNstOK9`fphE~rNH!5#!!9K_}1H^u_V z=usLU2dZ9W+tu&&HP&rr0Gj`32Xv({(IBcgf--PNjPP?Wl@c*PRlG~?w#y{$$E7Ac zx~e{WW#twe_l9Rk?!G}q_9k~-h|^Nd;KI<0tW*OJCTydW5K?zK18ZR;_&+8)QRWs&D)HUeQu z9uJg@R_z>@?>MI)N$iZs?C70IBKa$HMTmwtRZM2#Eoh>qRjH=pY|48!KPy$z*ryQVZ6bbD&&t4TF&?ZKHwA0q_P!85hGFhoG?V+nrEvu|YdG zNrL!x5hXK9XG!dHbKAhpa*yK2X+&`U9}I8`ON-SR2AaO}$md&6HEJY?sp&0?74^a& zVGN_jRIazY8Xhs~^9fS?Fi-gwcl8p=<-=HX>-`)aSrru|SJ3LA(S5lX(T8(;a;TEk zjAO5JZ;+6Ri7Fk+ojzl_j>FsNxien-@nif$*x?i|H~&{et(-zmOA(y|b9$qWo>F~6 zki+c@3Ez0iXk}m9oTR;sSPhbZ=SK~bRJ`f>5rk8+I0fjlFBn~o1e<4(6(c5WFg`wy)X+ZN{sOnlx=ETJarFP_~xRzcNqSZ7%QHR!VzITeKkh$31~ zImEggf>lp$FXnU=Ts#bWExW*Y9;7%4Oe?((yn8vIeg2W0PfQBbZg?H7S6jR7_Ab`i zFt+ORFZV!dxP5(el{8;w<@pHgN@T-AM|1c0R#EBHJ|EF9tDR?*`HCSSS-NChYvG5g86>yVnk$g?6B#Mb(2mG-cf!r7rvHyVnf8E8TJzgSB$KPWTl~g|#-Rw(h>E!kcXTQ3Q6z`cP z)fS*)cgRrG{5|EtiN~1M7vJ;J2n&CXaT*Pb--Bc$Zi!bnJlbwCvj3*7ebDIBSRV6) zJ(Pz1F*1RFZ#H99b! zFAr=1?f}O-Ds0l4sX~>=qET8%wZFXa-H{oyX`N12eqs0c0QyH-0D8xlm&3z#uIe`1 zrI9EslWc5kB75)nwQj9;_~TDgCgpG=C=?yr`DQL%YWUdLLKt~BF_>(ozPEDYfmq5| z0JoEQ)AV!wN+o_w*8sM82I;8gUaTdYv=QY$QCfcLv(IxGSQ+S> z%RxCJI*K1%Di#OIIA!I`gNrV;_(=luy&a6iR5jIAl$8;9aP{vcCdPa`xZs4!1pxskBjXCrY< zOcTKv`EhagL&xGTrIIUt2v?a2%l?bgMOV&&+|WU;e$=RpZpz1|dNV>*m~d#i)d2TYZVO)~XMlNxBrxoWuJ<66oc1(`c8LIHSqMFI@oVB5 zCTi-Btx2^*IwaoUOI*HGf^JQ#GU_^GnY6KL3-W;<6D`7ahRD>wHVTL3(9$baE*+Uh zI?Qh*6kS`O(EC~T8Xj=&-O3)}wcfX~t$yft z0#%1s3udIR?zCsDgD^((_g@(9316i|*L)iFc0tc}Oh8!c)=u|})&8|LYjG)q`SIr(O$~<-27o6EP z#<~$zeByPj(`59rPL~;qQy3J_7#W=XF0H@rb!E#Ub}O0BW;u}u1Ml#*1v|V#XY%_PSfaE#-vVR6s9p13SJS(vq?@w&x z1io}4NWAA0VtOK@D-a#+({tI}OZlv_g9^ZdL~a$%7K051T_}A1FpPpLUh51%m?D%A zu;0HQ2nYdyMN)q`vbBY0Z04-Vi-TU!{Ws&)J05FHCn{A|`7rGNj0AIUHubc-71}?2 T^$whsN!%rkGx+QOE291eC9)6< literal 0 HcmV?d00001 diff --git a/server/src/main/java/org/opensearch/common/network/NetworkModule.java b/server/src/main/java/org/opensearch/common/network/NetworkModule.java index f97d5b2f80eeb..d0f5dd9e4581d 100644 --- a/server/src/main/java/org/opensearch/common/network/NetworkModule.java +++ b/server/src/main/java/org/opensearch/common/network/NetworkModule.java @@ -55,6 +55,7 @@ import org.opensearch.http.HttpServerTransport; import org.opensearch.index.shard.PrimaryReplicaSyncer.ResyncTask; import org.opensearch.plugins.NetworkPlugin; +import org.opensearch.plugins.SecureTransportSettingsProvider; import org.opensearch.ratelimitting.admissioncontrol.enums.AdmissionControlActionType; import org.opensearch.tasks.RawTaskStatus; import org.opensearch.tasks.Task; @@ -67,6 +68,7 @@ import java.io.IOException; import java.util.ArrayList; +import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.List; @@ -85,6 +87,9 @@ public final class NetworkModule { public static final String HTTP_TYPE_KEY = "http.type"; public static final String HTTP_TYPE_DEFAULT_KEY = "http.type.default"; public static final String TRANSPORT_TYPE_DEFAULT_KEY = "transport.type.default"; + public static final String TRANSPORT_SSL_ENFORCE_HOSTNAME_VERIFICATION_KEY = "transport.ssl.enforce_hostname_verification"; + public static final String TRANSPORT_SSL_ENFORCE_HOSTNAME_VERIFICATION_RESOLVE_HOST_NAME_KEY = "transport.ssl.resolve_hostname"; + public static final String TRANSPORT_SSL_DUAL_MODE_ENABLED_KEY = "transport.ssl.dual_mode.enabled"; public static final Setting TRANSPORT_DEFAULT_TYPE_SETTING = Setting.simpleString( TRANSPORT_TYPE_DEFAULT_KEY, @@ -94,6 +99,22 @@ public final class NetworkModule { public static final Setting HTTP_TYPE_SETTING = Setting.simpleString(HTTP_TYPE_KEY, Property.NodeScope); public static final Setting TRANSPORT_TYPE_SETTING = Setting.simpleString(TRANSPORT_TYPE_KEY, Property.NodeScope); + public static final Setting TRANSPORT_SSL_ENFORCE_HOSTNAME_VERIFICATION = Setting.boolSetting( + TRANSPORT_SSL_ENFORCE_HOSTNAME_VERIFICATION_KEY, + true, + Property.NodeScope + ); + public static final Setting TRANSPORT_SSL_ENFORCE_HOSTNAME_VERIFICATION_RESOLVE_HOST_NAME = Setting.boolSetting( + TRANSPORT_SSL_ENFORCE_HOSTNAME_VERIFICATION_RESOLVE_HOST_NAME_KEY, + true, + Property.NodeScope + ); + public static final Setting TRANSPORT_SSL_DUAL_MODE_ENABLED = Setting.boolSetting( + TRANSPORT_SSL_DUAL_MODE_ENABLED_KEY, + false, + Property.NodeScope + ); + private final Settings settings; private static final List namedWriteables = new ArrayList<>(); @@ -151,9 +172,17 @@ public NetworkModule( HttpServerTransport.Dispatcher dispatcher, ClusterSettings clusterSettings, Tracer tracer, - List transportInterceptors + List transportInterceptors, + Collection secureTransportSettingsProvider ) { this.settings = settings; + + if (secureTransportSettingsProvider.size() > 1) { + throw new IllegalArgumentException( + "there is more than one secure transport settings provider: " + secureTransportSettingsProvider + ); + } + for (NetworkPlugin plugin : plugins) { Map> httpTransportFactory = plugin.getHttpTransports( settings, @@ -170,6 +199,7 @@ public NetworkModule( for (Map.Entry> entry : httpTransportFactory.entrySet()) { registerHttpTransport(entry.getKey(), entry.getValue()); } + Map> transportFactory = plugin.getTransports( settings, threadPool, @@ -182,6 +212,43 @@ public NetworkModule( for (Map.Entry> entry : transportFactory.entrySet()) { registerTransport(entry.getKey(), entry.getValue()); } + + // Register any secure transports if available + if (secureTransportSettingsProvider.isEmpty() == false) { + final SecureTransportSettingsProvider secureSettingProvider = secureTransportSettingsProvider.iterator().next(); + + final Map> secureHttpTransportFactory = plugin.getSecureHttpTransports( + settings, + threadPool, + bigArrays, + pageCacheRecycler, + circuitBreakerService, + xContentRegistry, + networkService, + dispatcher, + clusterSettings, + secureSettingProvider, + tracer + ); + for (Map.Entry> entry : secureHttpTransportFactory.entrySet()) { + registerHttpTransport(entry.getKey(), entry.getValue()); + } + + final Map> secureTransportFactory = plugin.getSecureTransports( + settings, + threadPool, + pageCacheRecycler, + circuitBreakerService, + namedWriteableRegistry, + networkService, + secureSettingProvider, + tracer + ); + for (Map.Entry> entry : secureTransportFactory.entrySet()) { + registerTransport(entry.getKey(), entry.getValue()); + } + } + List pluginTransportInterceptors = plugin.getTransportInterceptors( namedWriteableRegistry, threadPool.getThreadContext() diff --git a/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java b/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java index 730a330b6ca60..7c8afb6b5c1b5 100644 --- a/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java +++ b/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java @@ -331,6 +331,9 @@ public void apply(Settings value, Settings current, Settings previous) { NetworkModule.TRANSPORT_DEFAULT_TYPE_SETTING, NetworkModule.HTTP_TYPE_SETTING, NetworkModule.TRANSPORT_TYPE_SETTING, + NetworkModule.TRANSPORT_SSL_DUAL_MODE_ENABLED, + NetworkModule.TRANSPORT_SSL_ENFORCE_HOSTNAME_VERIFICATION, + NetworkModule.TRANSPORT_SSL_ENFORCE_HOSTNAME_VERIFICATION_RESOLVE_HOST_NAME, HttpTransportSettings.SETTING_CORS_ALLOW_CREDENTIALS, HttpTransportSettings.SETTING_CORS_ENABLED, HttpTransportSettings.SETTING_CORS_MAX_AGE, diff --git a/server/src/main/java/org/opensearch/node/Node.java b/server/src/main/java/org/opensearch/node/Node.java index 3ef3ae4f6230e..ea449afe1c811 100644 --- a/server/src/main/java/org/opensearch/node/Node.java +++ b/server/src/main/java/org/opensearch/node/Node.java @@ -202,6 +202,7 @@ import org.opensearch.plugins.ScriptPlugin; import org.opensearch.plugins.SearchPipelinePlugin; import org.opensearch.plugins.SearchPlugin; +import org.opensearch.plugins.SecureTransportSettingsProvider; import org.opensearch.plugins.SystemIndexPlugin; import org.opensearch.plugins.TelemetryPlugin; import org.opensearch.ratelimitting.admissioncontrol.AdmissionControlService; @@ -945,6 +946,13 @@ protected Node( admissionControlService ); + final Collection secureTransportSettingsProviders = pluginsService.filterPlugins(Plugin.class) + .stream() + .map(p -> p.getSecureSettingFactory(settings).flatMap(f -> f.getSecureTransportSettingsProvider(settings))) + .filter(Optional::isPresent) + .map(Optional::get) + .collect(Collectors.toList()); + List transportInterceptors = List.of(admissionControlTransportInterceptor); final NetworkModule networkModule = new NetworkModule( settings, @@ -959,7 +967,8 @@ protected Node( restController, clusterService.getClusterSettings(), tracer, - transportInterceptors + transportInterceptors, + secureTransportSettingsProviders ); Collection>> indexTemplateMetadataUpgraders = pluginsService.filterPlugins( diff --git a/server/src/main/java/org/opensearch/plugins/NetworkPlugin.java b/server/src/main/java/org/opensearch/plugins/NetworkPlugin.java index 07df40bafe6a1..679833c9f6e0d 100644 --- a/server/src/main/java/org/opensearch/plugins/NetworkPlugin.java +++ b/server/src/main/java/org/opensearch/plugins/NetworkPlugin.java @@ -107,4 +107,41 @@ default Map> getHttpTransports( ) { return Collections.emptyMap(); } + + /** + * Returns a map of secure {@link Transport} suppliers. + * See {@link org.opensearch.common.network.NetworkModule#TRANSPORT_TYPE_KEY} to configure a specific implementation. + */ + default Map> getSecureTransports( + Settings settings, + ThreadPool threadPool, + PageCacheRecycler pageCacheRecycler, + CircuitBreakerService circuitBreakerService, + NamedWriteableRegistry namedWriteableRegistry, + NetworkService networkService, + SecureTransportSettingsProvider secureTransportSettingsProvider, + Tracer tracer + ) { + return Collections.emptyMap(); + } + + /** + * Returns a map of secure {@link HttpServerTransport} suppliers. + * See {@link org.opensearch.common.network.NetworkModule#HTTP_TYPE_SETTING} to configure a specific implementation. + */ + default Map> getSecureHttpTransports( + Settings settings, + ThreadPool threadPool, + BigArrays bigArrays, + PageCacheRecycler pageCacheRecycler, + CircuitBreakerService circuitBreakerService, + NamedXContentRegistry xContentRegistry, + NetworkService networkService, + HttpServerTransport.Dispatcher dispatcher, + ClusterSettings clusterSettings, + SecureTransportSettingsProvider secureTransportSettingsProvider, + Tracer tracer + ) { + return Collections.emptyMap(); + } } diff --git a/server/src/main/java/org/opensearch/plugins/Plugin.java b/server/src/main/java/org/opensearch/plugins/Plugin.java index 48486a6b55dfd..33c4155d12c25 100644 --- a/server/src/main/java/org/opensearch/plugins/Plugin.java +++ b/server/src/main/java/org/opensearch/plugins/Plugin.java @@ -269,4 +269,13 @@ public void close() throws IOException { public Collection getAdditionalIndexSettingProviders() { return Collections.emptyList(); } + + /** + * Returns the {@link SecureSettingsFactory} instance that could be used to configure the + * security related components (fe. transports) + * @return the {@link SecureSettingsFactory} instance + */ + public Optional getSecureSettingFactory(Settings settings) { + return Optional.empty(); + } } diff --git a/server/src/main/java/org/opensearch/plugins/SecureSettingsFactory.java b/server/src/main/java/org/opensearch/plugins/SecureSettingsFactory.java new file mode 100644 index 0000000000000..b98d9cf51c129 --- /dev/null +++ b/server/src/main/java/org/opensearch/plugins/SecureSettingsFactory.java @@ -0,0 +1,29 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.plugins; + +import org.opensearch.common.annotation.ExperimentalApi; +import org.opensearch.common.settings.Settings; + +import java.util.Optional; + +/** + * A factory for creating the instance of the {@link SecureTransportSettingsProvider}, taking into account current settings. + * + * @opensearch.experimental + */ +@ExperimentalApi +public interface SecureSettingsFactory { + /** + * Creates (or provides pre-created) instance of the {@link SecureTransportSettingsProvider} + * @param settings settings + * @return optionally, the instance of the {@link SecureTransportSettingsProvider} + */ + Optional getSecureTransportSettingsProvider(Settings settings); +} diff --git a/server/src/main/java/org/opensearch/plugins/SecureTransportSettingsProvider.java b/server/src/main/java/org/opensearch/plugins/SecureTransportSettingsProvider.java new file mode 100644 index 0000000000000..6d038ed30c8ff --- /dev/null +++ b/server/src/main/java/org/opensearch/plugins/SecureTransportSettingsProvider.java @@ -0,0 +1,90 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.plugins; + +import org.opensearch.common.annotation.ExperimentalApi; +import org.opensearch.common.settings.Settings; +import org.opensearch.http.HttpServerTransport; +import org.opensearch.transport.TcpTransport; + +import javax.net.ssl.SSLEngine; +import javax.net.ssl.SSLException; + +import java.util.Optional; + +/** + * A provider for security related settings for transports. + * + * @opensearch.experimental + */ +@ExperimentalApi +public interface SecureTransportSettingsProvider { + /** + * An exception handler for errors that might happen while secure transport handle the requests. + * + * @see SslExceptionHandler + * + * @opensearch.experimental + */ + @ExperimentalApi + @FunctionalInterface + interface ServerExceptionHandler { + static ServerExceptionHandler NOOP = t -> {}; + + /** + * Handler for errors happening during the server side processing of the requests + * @param t the error + */ + void onError(Throwable t); + } + + /** + * If supported, builds the {@link ServerExceptionHandler} instance for {@link HttpServerTransport} instance + * @param settings settings + * @param transport {@link HttpServerTransport} instance + * @return if supported, builds the {@link ServerExceptionHandler} instance + */ + Optional buildHttpServerExceptionHandler(Settings settings, HttpServerTransport transport); + + /** + * If supported, builds the {@link ServerExceptionHandler} instance for {@link TcpTransport} instance + * @param settings settings + * @param transport {@link TcpTransport} instance + * @return if supported, builds the {@link ServerExceptionHandler} instance + */ + Optional buildServerTransportExceptionHandler(Settings settings, TcpTransport transport); + + /** + * If supported, builds the {@link SSLEngine} instance for {@link HttpServerTransport} instance + * @param settings settings + * @param transport {@link HttpServerTransport} instance + * @return if supported, builds the {@link SSLEngine} instance + * @throws SSLException throws SSLException if the {@link SSLEngine} instance cannot be built + */ + Optional buildSecureHttpServerEngine(Settings settings, HttpServerTransport transport) throws SSLException; + + /** + * If supported, builds the {@link SSLEngine} instance for {@link TcpTransport} instance + * @param settings settings + * @param transport {@link TcpTransport} instance + * @return if supported, builds the {@link SSLEngine} instance + * @throws SSLException throws SSLException if the {@link SSLEngine} instance cannot be built + */ + Optional buildSecureServerTransportEngine(Settings settings, TcpTransport transport) throws SSLException; + + /** + * If supported, builds the {@link SSLEngine} instance for client transport instance + * @param settings settings + * @param hostname host name + * @param port port + * @return if supported, builds the {@link SSLEngine} instance + * @throws SSLException throws SSLException if the {@link SSLEngine} instance cannot be built + */ + Optional buildSecureClientTransportEngine(Settings settings, String hostname, int port) throws SSLException; +} diff --git a/server/src/test/java/org/opensearch/common/network/NetworkModuleTests.java b/server/src/test/java/org/opensearch/common/network/NetworkModuleTests.java index de4bdcac6c2b2..1c607ca0dc98b 100644 --- a/server/src/test/java/org/opensearch/common/network/NetworkModuleTests.java +++ b/server/src/test/java/org/opensearch/common/network/NetworkModuleTests.java @@ -47,33 +47,66 @@ import org.opensearch.http.HttpStats; import org.opensearch.http.NullDispatcher; import org.opensearch.plugins.NetworkPlugin; +import org.opensearch.plugins.SecureTransportSettingsProvider; import org.opensearch.telemetry.tracing.Tracer; import org.opensearch.telemetry.tracing.noop.NoopTracer; import org.opensearch.test.OpenSearchTestCase; import org.opensearch.threadpool.TestThreadPool; import org.opensearch.threadpool.ThreadPool; +import org.opensearch.transport.TcpTransport; import org.opensearch.transport.Transport; import org.opensearch.transport.TransportInterceptor; import org.opensearch.transport.TransportRequest; import org.opensearch.transport.TransportRequestHandler; +import javax.net.ssl.SSLEngine; +import javax.net.ssl.SSLException; + import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Supplier; public class NetworkModuleTests extends OpenSearchTestCase { private ThreadPool threadPool; + private SecureTransportSettingsProvider secureTransportSettingsProvider; @Override public void setUp() throws Exception { super.setUp(); threadPool = new TestThreadPool(NetworkModuleTests.class.getName()); + secureTransportSettingsProvider = new SecureTransportSettingsProvider() { + @Override + public Optional buildHttpServerExceptionHandler(Settings settings, HttpServerTransport transport) { + return Optional.empty(); + } + + @Override + public Optional buildServerTransportExceptionHandler(Settings settings, TcpTransport transport) { + return Optional.empty(); + } + + @Override + public Optional buildSecureHttpServerEngine(Settings settings, HttpServerTransport transport) throws SSLException { + return Optional.empty(); + } + + @Override + public Optional buildSecureServerTransportEngine(Settings settings, TcpTransport transport) throws SSLException { + return Optional.empty(); + } + + @Override + public Optional buildSecureClientTransportEngine(Settings settings, String hostname, int port) throws SSLException { + return Optional.empty(); + } + }; } @Override @@ -160,6 +193,56 @@ public Map> getHttpTransports( expectThrows(IllegalStateException.class, () -> newModule.getHttpServerTransportSupplier()); } + public void testRegisterSecureTransport() { + Settings settings = Settings.builder().put(NetworkModule.TRANSPORT_TYPE_KEY, "custom-secure").build(); + Supplier custom = () -> null; // content doesn't matter we check reference equality + NetworkPlugin plugin = new NetworkPlugin() { + @Override + public Map> getSecureTransports( + Settings settings, + ThreadPool threadPool, + PageCacheRecycler pageCacheRecycler, + CircuitBreakerService circuitBreakerService, + NamedWriteableRegistry namedWriteableRegistry, + NetworkService networkService, + SecureTransportSettingsProvider secureTransportSettingsProvider, + Tracer tracer + ) { + return Collections.singletonMap("custom-secure", custom); + } + }; + NetworkModule module = newNetworkModule(settings, null, List.of(secureTransportSettingsProvider), plugin); + assertSame(custom, module.getTransportSupplier()); + } + + public void testRegisterSecureHttpTransport() { + Settings settings = Settings.builder() + .put(NetworkModule.HTTP_TYPE_SETTING.getKey(), "custom-secure") + .put(NetworkModule.TRANSPORT_TYPE_KEY, "local") + .build(); + Supplier custom = FakeHttpTransport::new; + + NetworkModule module = newNetworkModule(settings, null, List.of(secureTransportSettingsProvider), new NetworkPlugin() { + @Override + public Map> getSecureHttpTransports( + Settings settings, + ThreadPool threadPool, + BigArrays bigArrays, + PageCacheRecycler pageCacheRecycler, + CircuitBreakerService circuitBreakerService, + NamedXContentRegistry xContentRegistry, + NetworkService networkService, + HttpServerTransport.Dispatcher requestDispatcher, + ClusterSettings clusterSettings, + SecureTransportSettingsProvider secureTransportSettingsProvider, + Tracer tracer + ) { + return Collections.singletonMap("custom-secure", custom); + } + }); + assertSame(custom, module.getHttpServerTransportSupplier()); + } + public void testOverrideDefault() { Settings settings = Settings.builder() .put(NetworkModule.HTTP_TYPE_SETTING.getKey(), "custom") @@ -505,6 +588,15 @@ private NetworkModule newNetworkModule( Settings settings, List coreTransportInterceptors, NetworkPlugin... plugins + ) { + return newNetworkModule(settings, coreTransportInterceptors, List.of(), plugins); + } + + private NetworkModule newNetworkModule( + Settings settings, + List coreTransportInterceptors, + List secureTransportSettingsProviders, + NetworkPlugin... plugins ) { return new NetworkModule( settings, @@ -519,7 +611,8 @@ private NetworkModule newNetworkModule( new NullDispatcher(), new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS), NoopTracer.INSTANCE, - coreTransportInterceptors + coreTransportInterceptors, + secureTransportSettingsProviders ); } } From 0510c5bc80d708d096d167474cbac7cead7b43db Mon Sep 17 00:00:00 2001 From: Ashish Date: Wed, 20 Mar 2024 21:55:26 +0530 Subject: [PATCH 15/38] Use cluster default remote store path type during snapshot restore (#12753) * Use cluster default remote store path type as fallback during snapshot restore --------- Signed-off-by: Ashish Singh --- .../remotestore/RemoteRestoreSnapshotIT.java | 84 ++++++++++++++++++- .../snapshots/RestoreSnapshotIT.java | 57 +++++++++++++ .../metadata/MetadataCreateIndexService.java | 22 +++-- .../opensearch/snapshots/RestoreService.java | 2 +- 4 files changed, 158 insertions(+), 7 deletions(-) diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteRestoreSnapshotIT.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteRestoreSnapshotIT.java index 21ce4be9981fb..fff99e65054dc 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteRestoreSnapshotIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteRestoreSnapshotIT.java @@ -18,6 +18,7 @@ import org.opensearch.action.support.PlainActionFuture; import org.opensearch.client.Client; import org.opensearch.client.Requests; +import org.opensearch.cluster.ClusterState; import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.common.io.PathUtils; import org.opensearch.common.settings.Settings; @@ -26,6 +27,7 @@ import org.opensearch.core.rest.RestStatus; import org.opensearch.index.IndexService; import org.opensearch.index.IndexSettings; +import org.opensearch.index.remote.RemoteStorePathType; import org.opensearch.index.shard.IndexShard; import org.opensearch.indices.IndicesService; import org.opensearch.indices.replication.common.ReplicationType; @@ -43,6 +45,7 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.List; +import java.util.Map; import java.util.Optional; import java.util.concurrent.ExecutionException; import java.util.stream.Collectors; @@ -50,7 +53,7 @@ import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_REMOTE_SEGMENT_STORE_REPOSITORY; import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_REMOTE_STORE_ENABLED; -import static org.opensearch.remotestore.RemoteStoreBaseIntegTestCase.remoteStoreClusterSettings; +import static org.opensearch.indices.IndicesService.CLUSTER_REMOTE_STORE_PATH_PREFIX_TYPE_SETTING; import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertAcked; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.greaterThan; @@ -257,6 +260,85 @@ public void testRestoreOperationsShallowCopyEnabled() throws IOException, Execut assertDocsPresentInIndex(client, restoredIndexName1Doc, numDocsInIndex1 + 2); } + /** + * In this test, we validate presence of remote_store custom data in index metadata for standard index creation and + * on snapshot restore. + */ + public void testRemoteStoreCustomDataOnIndexCreationAndRestore() { + String clusterManagerNode = internalCluster().startClusterManagerOnlyNode(); + internalCluster().startDataOnlyNode(); + String indexName1 = "testindex1"; + String indexName2 = "testindex2"; + String snapshotRepoName = "test-restore-snapshot-repo"; + String snapshotName1 = "test-restore-snapshot1"; + Path absolutePath1 = randomRepoPath().toAbsolutePath(); + logger.info("Snapshot Path [{}]", absolutePath1); + String restoredIndexName1version1 = indexName1 + "-restored-1"; + String restoredIndexName1version2 = indexName1 + "-restored-2"; + + createRepository(snapshotRepoName, "fs", getRepositorySettings(absolutePath1, true)); + Client client = client(); + Settings indexSettings = getIndexSettings(1, 0).build(); + createIndex(indexName1, indexSettings); + + indexDocuments(client, indexName1, randomIntBetween(5, 10)); + ensureGreen(indexName1); + validateRemoteStorePathType(indexName1, RemoteStorePathType.FIXED); + + logger.info("--> snapshot"); + SnapshotInfo snapshotInfo = createSnapshot(snapshotRepoName, snapshotName1, new ArrayList<>(Arrays.asList(indexName1))); + assertEquals(SnapshotState.SUCCESS, snapshotInfo.state()); + assertTrue(snapshotInfo.successfulShards() > 0); + assertEquals(snapshotInfo.totalShards(), snapshotInfo.successfulShards()); + + RestoreSnapshotResponse restoreSnapshotResponse = client.admin() + .cluster() + .prepareRestoreSnapshot(snapshotRepoName, snapshotName1) + .setWaitForCompletion(false) + .setRenamePattern(indexName1) + .setRenameReplacement(restoredIndexName1version1) + .get(); + assertEquals(RestStatus.ACCEPTED, restoreSnapshotResponse.status()); + ensureGreen(restoredIndexName1version1); + validateRemoteStorePathType(restoredIndexName1version1, RemoteStorePathType.FIXED); + + client(clusterManagerNode).admin() + .cluster() + .prepareUpdateSettings() + .setTransientSettings( + Settings.builder().put(CLUSTER_REMOTE_STORE_PATH_PREFIX_TYPE_SETTING.getKey(), RemoteStorePathType.HASHED_PREFIX) + ) + .get(); + + restoreSnapshotResponse = client.admin() + .cluster() + .prepareRestoreSnapshot(snapshotRepoName, snapshotName1) + .setWaitForCompletion(false) + .setRenamePattern(indexName1) + .setRenameReplacement(restoredIndexName1version2) + .get(); + assertEquals(RestStatus.ACCEPTED, restoreSnapshotResponse.status()); + ensureGreen(restoredIndexName1version2); + validateRemoteStorePathType(restoredIndexName1version2, RemoteStorePathType.HASHED_PREFIX); + + // Create index with cluster setting cluster.remote_store.index.path.prefix.type as hashed_prefix. + indexSettings = getIndexSettings(1, 0).build(); + createIndex(indexName2, indexSettings); + ensureGreen(indexName2); + validateRemoteStorePathType(indexName2, RemoteStorePathType.HASHED_PREFIX); + + // Validating that custom data has not changed for indexes which were created before the cluster setting got updated + validateRemoteStorePathType(indexName1, RemoteStorePathType.FIXED); + } + + private void validateRemoteStorePathType(String index, RemoteStorePathType pathType) { + ClusterState state = client().admin().cluster().prepareState().execute().actionGet().getState(); + // Validate that the remote_store custom data is present in index metadata for the created index. + Map remoteCustomData = state.metadata().index(index).getCustomData(IndexMetadata.REMOTE_STORE_CUSTOM_KEY); + assertNotNull(remoteCustomData); + assertEquals(pathType.toString(), remoteCustomData.get(RemoteStorePathType.NAME)); + } + public void testRestoreInSameRemoteStoreEnabledIndex() throws IOException { String clusterManagerNode = internalCluster().startClusterManagerOnlyNode(); String primary = internalCluster().startDataOnlyNode(); diff --git a/server/src/internalClusterTest/java/org/opensearch/snapshots/RestoreSnapshotIT.java b/server/src/internalClusterTest/java/org/opensearch/snapshots/RestoreSnapshotIT.java index 7117818451e14..e76587653e99a 100644 --- a/server/src/internalClusterTest/java/org/opensearch/snapshots/RestoreSnapshotIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/snapshots/RestoreSnapshotIT.java @@ -39,6 +39,7 @@ import org.opensearch.action.admin.indices.template.get.GetIndexTemplatesResponse; import org.opensearch.action.index.IndexRequestBuilder; import org.opensearch.client.Client; +import org.opensearch.cluster.ClusterState; import org.opensearch.cluster.block.ClusterBlocks; import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.cluster.metadata.MappingMetadata; @@ -151,6 +152,62 @@ public void testParallelRestoreOperations() { assertThat(client.prepareGet(restoredIndexName2, docId2).get().isExists(), equalTo(true)); } + /** + * In this test, we test that an index created does not have any remote_store custom data in index metadata at the + * time of index creation and after snapshot restore. + */ + public void testNoRemoteStoreCustomDataOnIndexCreationAndRestore() { + String indexName1 = "testindex1"; + String repoName = "test-restore-snapshot-repo"; + String snapshotName1 = "test-restore-snapshot1"; + Path absolutePath = randomRepoPath().toAbsolutePath(); + logger.info("Path [{}]", absolutePath); + String restoredIndexName1 = indexName1 + "-restored"; + String expectedValue = "expected"; + + Client client = client(); + // Write a document + String docId = Integer.toString(randomInt()); + index(indexName1, "_doc", docId, "value", expectedValue); + + createRepository(repoName, "fs", absolutePath); + + logger.info("--> snapshot"); + CreateSnapshotResponse createSnapshotResponse = client.admin() + .cluster() + .prepareCreateSnapshot(repoName, snapshotName1) + .setWaitForCompletion(true) + .setIndices(indexName1) + .get(); + assertThat(createSnapshotResponse.getSnapshotInfo().successfulShards(), greaterThan(0)); + assertThat( + createSnapshotResponse.getSnapshotInfo().successfulShards(), + equalTo(createSnapshotResponse.getSnapshotInfo().totalShards()) + ); + assertThat(createSnapshotResponse.getSnapshotInfo().state(), equalTo(SnapshotState.SUCCESS)); + + ClusterState state = client().admin().cluster().prepareState().execute().actionGet().getState(); + + // Validate that the remote_store custom data is not present in index metadata for the created index. + assertNull(state.metadata().index(indexName1).getCustomData(IndexMetadata.REMOTE_STORE_CUSTOM_KEY)); + + RestoreSnapshotResponse restoreSnapshotResponse1 = client.admin() + .cluster() + .prepareRestoreSnapshot(repoName, snapshotName1) + .setWaitForCompletion(false) + .setRenamePattern(indexName1) + .setRenameReplacement(restoredIndexName1) + .get(); + assertThat(restoreSnapshotResponse1.status(), equalTo(RestStatus.ACCEPTED)); + ensureGreen(restoredIndexName1); + assertThat(client.prepareGet(restoredIndexName1, docId).get().isExists(), equalTo(true)); + + state = client().admin().cluster().prepareState().execute().actionGet().getState(); + + // Validate that the remote_store custom data is not present in index metadata for the restored index. + assertNull(state.metadata().index(restoredIndexName1).getCustomData(IndexMetadata.REMOTE_STORE_CUSTOM_KEY)); + } + public void testParallelRestoreOperationsFromSingleSnapshot() throws Exception { String indexName1 = "testindex1"; String indexName2 = "testindex2"; diff --git a/server/src/main/java/org/opensearch/cluster/metadata/MetadataCreateIndexService.java b/server/src/main/java/org/opensearch/cluster/metadata/MetadataCreateIndexService.java index f117d1a4a11a2..f6a14d8ec9d63 100644 --- a/server/src/main/java/org/opensearch/cluster/metadata/MetadataCreateIndexService.java +++ b/server/src/main/java/org/opensearch/cluster/metadata/MetadataCreateIndexService.java @@ -553,11 +553,7 @@ IndexMetadata buildAndValidateTemporaryIndexMetadata( tmpImdBuilder.setRoutingNumShards(routingNumShards); tmpImdBuilder.settings(indexSettings); tmpImdBuilder.system(isSystem); - - if (remoteStorePathResolver != null) { - String pathType = remoteStorePathResolver.resolveType().toString(); - tmpImdBuilder.putCustom(IndexMetadata.REMOTE_STORE_CUSTOM_KEY, Map.of(RemoteStorePathType.NAME, pathType)); - } + addRemoteCustomData(tmpImdBuilder); // Set up everything, now locally create the index to see that things are ok, and apply IndexMetadata tempMetadata = tmpImdBuilder.build(); @@ -566,6 +562,22 @@ IndexMetadata buildAndValidateTemporaryIndexMetadata( return tempMetadata; } + public void addRemoteCustomData(IndexMetadata.Builder tmpImdBuilder) { + if (remoteStorePathResolver != null) { + // It is possible that remote custom data exists already. In such cases, we need to only update the path type + // in the remote store custom data map. + Map existingRemoteCustomData = tmpImdBuilder.removeCustom(IndexMetadata.REMOTE_STORE_CUSTOM_KEY); + Map remoteCustomData = existingRemoteCustomData == null + ? new HashMap<>() + : new HashMap<>(existingRemoteCustomData); + // Determine the path type for use using the remoteStorePathResolver. + String newPathType = remoteStorePathResolver.resolveType().toString(); + String oldPathType = remoteCustomData.put(RemoteStorePathType.NAME, newPathType); + logger.trace(() -> new ParameterizedMessage("Added new path type {}, replaced old path type {}", newPathType, oldPathType)); + tmpImdBuilder.putCustom(IndexMetadata.REMOTE_STORE_CUSTOM_KEY, remoteCustomData); + } + } + private ClusterState applyCreateIndexRequestWithV1Templates( final ClusterState currentState, final CreateIndexClusterStateUpdateRequest request, diff --git a/server/src/main/java/org/opensearch/snapshots/RestoreService.java b/server/src/main/java/org/opensearch/snapshots/RestoreService.java index bf2c7fc74be92..e5ac604e0a5e3 100644 --- a/server/src/main/java/org/opensearch/snapshots/RestoreService.java +++ b/server/src/main/java/org/opensearch/snapshots/RestoreService.java @@ -216,7 +216,6 @@ public RestoreService( // Task is onboarded for throttling, it will get retried from associated TransportClusterManagerNodeAction. restoreSnapshotTaskKey = clusterService.registerClusterManagerTask(ClusterManagerTaskKeys.RESTORE_SNAPSHOT_KEY, true); - } /** @@ -452,6 +451,7 @@ public ClusterState execute(ClusterState currentState) { .put(snapshotIndexMetadata.getSettings()) .put(IndexMetadata.SETTING_INDEX_UUID, UUIDs.randomBase64UUID()) ); + createIndexService.addRemoteCustomData(indexMdBuilder); shardLimitValidator.validateShardLimit( renamedIndexName, snapshotIndexMetadata.getSettings(), From e6975e412b09a8d82675edd9a43c20f7c325c0f9 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Wed, 20 Mar 2024 12:58:33 -0400 Subject: [PATCH 16/38] Bump org.apache.commons:commons-configuration2 from 2.9.0 to 2.10.0 in /plugins/repository-hdfs (#12721) * Bump org.apache.commons:commons-configuration2 Bumps org.apache.commons:commons-configuration2 from 2.9.0 to 2.10.0. --- updated-dependencies: - dependency-name: org.apache.commons:commons-configuration2 dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] * Updating SHAs Signed-off-by: dependabot[bot] * Update changelog Signed-off-by: dependabot[bot] --------- Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> Co-authored-by: dependabot[bot] --- CHANGELOG.md | 1 + plugins/repository-hdfs/build.gradle | 2 +- .../licenses/commons-configuration2-2.10.0.jar.sha1 | 1 + .../licenses/commons-configuration2-2.9.0.jar.sha1 | 1 - 4 files changed, 3 insertions(+), 2 deletions(-) create mode 100644 plugins/repository-hdfs/licenses/commons-configuration2-2.10.0.jar.sha1 delete mode 100644 plugins/repository-hdfs/licenses/commons-configuration2-2.9.0.jar.sha1 diff --git a/CHANGELOG.md b/CHANGELOG.md index 0b5a7cc705a79..4ba3dbcdc8d9b 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -144,6 +144,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Bump `reactor-netty` from 1.1.15 to 1.1.17 ([#12633](https://github.com/opensearch-project/OpenSearch/pull/12633)) - Bump `reactor` from 3.5.14 to 3.5.15 ([#12633](https://github.com/opensearch-project/OpenSearch/pull/12633)) - Bump `peter-evans/create-pull-request` from 5 to 6 ([#12724](https://github.com/opensearch-project/OpenSearch/pull/12724)) +- Bump `org.apache.commons:commons-configuration2` from 2.9.0 to 2.10.0 ([#12721](https://github.com/opensearch-project/OpenSearch/pull/12721)) ### Changed - Allow composite aggregation to run under a parent filter aggregation ([#11499](https://github.com/opensearch-project/OpenSearch/pull/11499)) diff --git a/plugins/repository-hdfs/build.gradle b/plugins/repository-hdfs/build.gradle index 36843e3bc8700..c1f94320f2681 100644 --- a/plugins/repository-hdfs/build.gradle +++ b/plugins/repository-hdfs/build.gradle @@ -74,7 +74,7 @@ dependencies { api "commons-codec:commons-codec:${versions.commonscodec}" api 'commons-collections:commons-collections:3.2.2' api "org.apache.commons:commons-compress:${versions.commonscompress}" - api 'org.apache.commons:commons-configuration2:2.9.0' + api 'org.apache.commons:commons-configuration2:2.10.0' api 'commons-io:commons-io:2.15.1' api 'org.apache.commons:commons-lang3:3.14.0' implementation 'com.google.re2j:re2j:1.7' diff --git a/plugins/repository-hdfs/licenses/commons-configuration2-2.10.0.jar.sha1 b/plugins/repository-hdfs/licenses/commons-configuration2-2.10.0.jar.sha1 new file mode 100644 index 0000000000000..17d1b64781e5b --- /dev/null +++ b/plugins/repository-hdfs/licenses/commons-configuration2-2.10.0.jar.sha1 @@ -0,0 +1 @@ +2b93eff3c83e5372262ed4996b609336305a810f \ No newline at end of file diff --git a/plugins/repository-hdfs/licenses/commons-configuration2-2.9.0.jar.sha1 b/plugins/repository-hdfs/licenses/commons-configuration2-2.9.0.jar.sha1 deleted file mode 100644 index 086c769fe600c..0000000000000 --- a/plugins/repository-hdfs/licenses/commons-configuration2-2.9.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -adc3ee6b84fc62a6e75e901d080adacb72aac61e \ No newline at end of file From 4dbd6fa6c35f7a3d0c5e2ee284c2f8b3cc829a32 Mon Sep 17 00:00:00 2001 From: Shivansh Arora <31575408+shiv0408@users.noreply.github.com> Date: Wed, 20 Mar 2024 22:59:30 +0530 Subject: [PATCH 17/38] Created new ReplicaShardBatchAllocator (#8992) * Created new ReplicaShardBatchAllocator to be used instead of ReplicaShardAllocator for batch calls Signed-off-by: Shivansh Arora --- .../gateway/ReplicaShardBatchAllocator.java | 188 ++++ .../ReplicaShardBatchAllocatorTests.java | 849 ++++++++++++++++++ 2 files changed, 1037 insertions(+) create mode 100644 server/src/main/java/org/opensearch/gateway/ReplicaShardBatchAllocator.java create mode 100644 server/src/test/java/org/opensearch/gateway/ReplicaShardBatchAllocatorTests.java diff --git a/server/src/main/java/org/opensearch/gateway/ReplicaShardBatchAllocator.java b/server/src/main/java/org/opensearch/gateway/ReplicaShardBatchAllocator.java new file mode 100644 index 0000000000000..3459f1591b633 --- /dev/null +++ b/server/src/main/java/org/opensearch/gateway/ReplicaShardBatchAllocator.java @@ -0,0 +1,188 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.gateway; + +import org.apache.logging.log4j.Logger; +import org.opensearch.cluster.node.DiscoveryNode; +import org.opensearch.cluster.routing.ShardRouting; +import org.opensearch.cluster.routing.UnassignedInfo; +import org.opensearch.cluster.routing.allocation.AllocateUnassignedDecision; +import org.opensearch.cluster.routing.allocation.NodeAllocationResult; +import org.opensearch.cluster.routing.allocation.RoutingAllocation; +import org.opensearch.cluster.routing.allocation.decider.Decision; +import org.opensearch.common.collect.Tuple; +import org.opensearch.core.index.shard.ShardId; +import org.opensearch.gateway.AsyncShardFetch.FetchResult; +import org.opensearch.indices.store.TransportNodesListShardStoreMetadata; +import org.opensearch.indices.store.TransportNodesListShardStoreMetadataBatch.NodeStoreFilesMetadata; +import org.opensearch.indices.store.TransportNodesListShardStoreMetadataBatch.NodeStoreFilesMetadataBatch; +import org.opensearch.indices.store.TransportNodesListShardStoreMetadataHelper.StoreFilesMetadata; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * Allocates replica shards in a batch mode + * + * @opensearch.internal + */ +public abstract class ReplicaShardBatchAllocator extends ReplicaShardAllocator { + + /** + * Process existing recoveries of replicas and see if we need to cancel them if we find a better + * match. Today, a better match is one that can perform a no-op recovery while the previous recovery + * has to copy segment files. + * + * @param allocation the overall routing allocation + * @param shardBatches a list of shard batches to check for existing recoveries + */ + public void processExistingRecoveries(RoutingAllocation allocation, List> shardBatches) { + List shardCancellationActions = new ArrayList<>(); + // iterate through the batches, each batch needs to be processed together as fetch call should be made for shards from same batch + for (List shardBatch : shardBatches) { + List eligibleShards = new ArrayList<>(); + List ineligibleShards = new ArrayList<>(); + // iterate over shards to check for match for each of those + for (ShardRouting shard : shardBatch) { + if (shard != null && !shard.primary()) { + // need to iterate over all the nodes to find matching shard + if (shouldSkipFetchForRecovery(shard)) { + ineligibleShards.add(shard); + continue; + } + eligibleShards.add(shard); + } + } + AsyncShardFetch.FetchResult shardState = fetchData(eligibleShards, ineligibleShards, allocation); + if (!shardState.hasData()) { + logger.trace("{}: fetching new stores for initializing shard batch", eligibleShards); + continue; // still fetching + } + for (ShardRouting shard : eligibleShards) { + Map nodeShardStores = convertToNodeStoreFilesMetadataMap(shard, shardState); + + Runnable cancellationAction = cancelExistingRecoveryForBetterMatch(shard, allocation, nodeShardStores); + if (cancellationAction != null) { + shardCancellationActions.add(cancellationAction); + } + } + } + for (Runnable action : shardCancellationActions) { + action.run(); + } + } + + abstract protected FetchResult fetchData( + List eligibleShards, + List ineligibleShards, + RoutingAllocation allocation + ); + + @Override + protected FetchResult fetchData( + ShardRouting shard, + RoutingAllocation allocation + ) { + logger.error("fetchData for single shard called via batch allocator"); + throw new IllegalStateException("ReplicaShardBatchAllocator should only be used for a batch of shards"); + } + + @Override + public AllocateUnassignedDecision makeAllocationDecision(ShardRouting unassignedShard, RoutingAllocation allocation, Logger logger) { + return makeAllocationDecision(Collections.singletonList(unassignedShard), allocation, logger).get(unassignedShard); + } + + @Override + public HashMap makeAllocationDecision( + List shards, + RoutingAllocation allocation, + Logger logger + ) { + HashMap shardAllocationDecisions = new HashMap<>(); + final boolean explain = allocation.debugDecision(); + List eligibleShards = new ArrayList<>(); + List ineligibleShards = new ArrayList<>(); + HashMap>> nodeAllocationDecisions = new HashMap<>(); + for (ShardRouting shard : shards) { + if (!isResponsibleFor(shard)) { + // this allocator n is not responsible for allocating this shard + ineligibleShards.add(shard); + shardAllocationDecisions.put(shard, AllocateUnassignedDecision.NOT_TAKEN); + continue; + } + + Tuple> result = canBeAllocatedToAtLeastOneNode(shard, allocation); + Decision allocationDecision = result.v1(); + if (allocationDecision.type() != Decision.Type.YES && (!explain || !hasInitiatedFetching(shard))) { + // only return early if we are not in explain mode, or we are in explain mode but we have not + // yet attempted to fetch any shard data + logger.trace("{}: ignoring allocation, can't be allocated on any node", shard); + shardAllocationDecisions.put( + shard, + AllocateUnassignedDecision.no( + UnassignedInfo.AllocationStatus.fromDecision(allocationDecision.type()), + result.v2() != null ? new ArrayList<>(result.v2().values()) : null + ) + ); + continue; + } + // storing the nodeDecisions in nodeAllocationDecisions if the decision is not YES + // so that we don't have to compute the decisions again + nodeAllocationDecisions.put(shard, result); + + eligibleShards.add(shard); + } + + // Do not call fetchData if there are no eligible shards + if (eligibleShards.isEmpty()) { + return shardAllocationDecisions; + } + // only fetch data for eligible shards + final FetchResult shardsState = fetchData(eligibleShards, ineligibleShards, allocation); + + for (ShardRouting unassignedShard : eligibleShards) { + Tuple> result = nodeAllocationDecisions.get(unassignedShard); + shardAllocationDecisions.put( + unassignedShard, + getAllocationDecision( + unassignedShard, + allocation, + convertToNodeStoreFilesMetadataMap(unassignedShard, shardsState), + result, + logger + ) + ); + } + return shardAllocationDecisions; + } + + private Map convertToNodeStoreFilesMetadataMap( + ShardRouting unassignedShard, + FetchResult data + ) { + if (!data.hasData()) { + return null; + } + + Map map = new HashMap<>(); + + data.getData().forEach((discoveryNode, value) -> { + Map batch = value.getNodeStoreFilesMetadataBatch(); + NodeStoreFilesMetadata metadata = batch.get(unassignedShard.shardId()); + if (metadata != null) { + map.put(discoveryNode, metadata.storeFilesMetadata()); + } + }); + + return map; + } +} diff --git a/server/src/test/java/org/opensearch/gateway/ReplicaShardBatchAllocatorTests.java b/server/src/test/java/org/opensearch/gateway/ReplicaShardBatchAllocatorTests.java new file mode 100644 index 0000000000000..464038c93228b --- /dev/null +++ b/server/src/test/java/org/opensearch/gateway/ReplicaShardBatchAllocatorTests.java @@ -0,0 +1,849 @@ +/* + * 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 com.carrotsearch.randomizedtesting.generators.RandomPicks; + +import org.opensearch.Version; +import org.opensearch.cluster.ClusterInfo; +import org.opensearch.cluster.ClusterState; +import org.opensearch.cluster.OpenSearchAllocationTestCase; +import org.opensearch.cluster.metadata.IndexMetadata; +import org.opensearch.cluster.metadata.Metadata; +import org.opensearch.cluster.node.DiscoveryNode; +import org.opensearch.cluster.node.DiscoveryNodes; +import org.opensearch.cluster.routing.IndexRoutingTable; +import org.opensearch.cluster.routing.IndexShardRoutingTable; +import org.opensearch.cluster.routing.RecoverySource; +import org.opensearch.cluster.routing.RoutingNode; +import org.opensearch.cluster.routing.RoutingNodes; +import org.opensearch.cluster.routing.RoutingTable; +import org.opensearch.cluster.routing.ShardRouting; +import org.opensearch.cluster.routing.ShardRoutingState; +import org.opensearch.cluster.routing.TestShardRouting; +import org.opensearch.cluster.routing.UnassignedInfo; +import org.opensearch.cluster.routing.allocation.RoutingAllocation; +import org.opensearch.cluster.routing.allocation.decider.AllocationDecider; +import org.opensearch.cluster.routing.allocation.decider.AllocationDeciders; +import org.opensearch.cluster.routing.allocation.decider.Decision; +import org.opensearch.cluster.routing.allocation.decider.SameShardAllocationDecider; +import org.opensearch.common.Nullable; +import org.opensearch.common.settings.ClusterSettings; +import org.opensearch.common.settings.Settings; +import org.opensearch.common.unit.TimeValue; +import org.opensearch.common.util.set.Sets; +import org.opensearch.core.index.shard.ShardId; +import org.opensearch.index.engine.Engine; +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 org.opensearch.indices.store.TransportNodesListShardStoreMetadataBatch; +import org.opensearch.indices.store.TransportNodesListShardStoreMetadataBatch.NodeStoreFilesMetadataBatch; +import org.opensearch.indices.store.TransportNodesListShardStoreMetadataHelper; +import org.opensearch.snapshots.SnapshotShardSizeInfo; +import org.junit.Before; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.EnumSet; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; + +import static java.util.Collections.unmodifiableMap; +import static org.hamcrest.Matchers.empty; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.hasSize; + +public class ReplicaShardBatchAllocatorTests extends OpenSearchAllocationTestCase { + private static final org.apache.lucene.util.Version MIN_SUPPORTED_LUCENE_VERSION = org.opensearch.Version.CURRENT + .minimumIndexCompatibilityVersion().luceneVersion; + private final ShardId shardId = new ShardId("test", "_na_", 0); + private final DiscoveryNode node1 = newNode("node1"); + private final DiscoveryNode node2 = newNode("node2"); + private final DiscoveryNode node3 = newNode("node3"); + + private TestBatchAllocator testBatchAllocator; + + @Before + public void buildTestAllocator() { + this.testBatchAllocator = new TestBatchAllocator(); + } + + private void allocateAllUnassignedBatch(final RoutingAllocation allocation) { + final RoutingNodes.UnassignedShards.UnassignedIterator iterator = allocation.routingNodes().unassigned().iterator(); + List shardToBatch = new ArrayList<>(); + while (iterator.hasNext()) { + shardToBatch.add(iterator.next()); + } + testBatchAllocator.allocateUnassignedBatch(shardToBatch, allocation); + } + + /** + * Verifies that when we are still fetching data in an async manner, the replica shard moves to ignore unassigned. + */ + public void testNoAsyncFetchData() { + RoutingAllocation allocation = onePrimaryOnNode1And1Replica(yesAllocationDeciders()); + testBatchAllocator.clean(); + allocateAllUnassignedBatch(allocation); + assertThat(allocation.routingNodes().unassigned().ignored().size(), equalTo(1)); + assertThat(allocation.routingNodes().unassigned().ignored().get(0).shardId(), equalTo(shardId)); + } + + /** + * Verifies that on index creation, we don't fetch data for any shards, but keep the replica shard unassigned to let + * the shard allocator to allocate it. There isn't a copy around to find anyhow. + */ + public void testAsyncFetchWithNoShardOnIndexCreation() { + RoutingAllocation allocation = onePrimaryOnNode1And1Replica( + yesAllocationDeciders(), + Settings.EMPTY, + UnassignedInfo.Reason.INDEX_CREATED + ); + testBatchAllocator.clean(); + allocateAllUnassignedBatch(allocation); + assertThat(testBatchAllocator.getFetchDataCalledAndClean(), equalTo(false)); + assertThat(testBatchAllocator.getShardEligibleFetchDataCountAndClean(), equalTo(0)); + assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.UNASSIGNED).size(), equalTo(1)); + assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.UNASSIGNED).get(0).shardId(), equalTo(shardId)); + } + + /** + * Verifies that for anything but index creation, fetch data ends up being called, since we need to go and try + * and find a better copy for the shard. + */ + public void testAsyncFetchOnAnythingButIndexCreation() { + UnassignedInfo.Reason reason = RandomPicks.randomFrom( + random(), + EnumSet.complementOf(EnumSet.of(UnassignedInfo.Reason.INDEX_CREATED)) + ); + RoutingAllocation allocation = onePrimaryOnNode1And1Replica(yesAllocationDeciders(), Settings.EMPTY, reason); + testBatchAllocator.clean(); + allocateAllUnassignedBatch(allocation); + assertThat("failed with reason " + reason, testBatchAllocator.getFetchDataCalledAndClean(), equalTo(true)); + assertThat("failed with reason" + reason, testBatchAllocator.getShardEligibleFetchDataCountAndClean(), equalTo(1)); + } + + /** + * Verifies that when there is a full match (syncId and files) we allocate it to matching node. + */ + public void testSimpleFullMatchAllocation() { + RoutingAllocation allocation = onePrimaryOnNode1And1Replica(yesAllocationDeciders()); + DiscoveryNode nodeToMatch = randomBoolean() ? node2 : node3; + testBatchAllocator.addData(node1, "MATCH", null, new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION)) + .addData(nodeToMatch, "MATCH", null, new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION)); + allocateAllUnassignedBatch(allocation); + assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING).size(), equalTo(1)); + assertThat( + allocation.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING).get(0).currentNodeId(), + equalTo(nodeToMatch.getId()) + ); + } + + /** + * Verifies that when there is a sync id match but no files match, we allocate it to matching node. + */ + public void testSyncIdMatch() { + RoutingAllocation allocation = onePrimaryOnNode1And1Replica(yesAllocationDeciders()); + DiscoveryNode nodeToMatch = randomBoolean() ? node2 : node3; + testBatchAllocator.addData(node1, "MATCH", null, new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION)) + .addData(nodeToMatch, "MATCH", null, new StoreFileMetadata("file1", 10, "NO_MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION)); + allocateAllUnassignedBatch(allocation); + assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING).size(), equalTo(1)); + assertThat( + allocation.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING).get(0).currentNodeId(), + equalTo(nodeToMatch.getId()) + ); + } + + /** + * Verifies that when there is no sync id match but files match, we allocate it to matching node. + */ + public void testFileChecksumMatch() { + RoutingAllocation allocation = onePrimaryOnNode1And1Replica(yesAllocationDeciders()); + DiscoveryNode nodeToMatch = randomBoolean() ? node2 : node3; + testBatchAllocator.addData(node1, "MATCH", null, new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION)) + .addData(nodeToMatch, "NO_MATCH", null, new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION)); + allocateAllUnassignedBatch(allocation); + assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING).size(), equalTo(1)); + assertThat( + allocation.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING).get(0).currentNodeId(), + equalTo(nodeToMatch.getId()) + ); + } + + public void testPreferCopyWithHighestMatchingOperations() { + RoutingAllocation allocation = onePrimaryOnNode1And1Replica(yesAllocationDeciders()); + long retainingSeqNoOnPrimary = randomLongBetween(1, Integer.MAX_VALUE); + long retainingSeqNoForNode2 = randomLongBetween(0, retainingSeqNoOnPrimary - 1); + // Rarely use a seqNo above retainingSeqNoOnPrimary, which could in theory happen when primary fails and comes back quickly. + long retainingSeqNoForNode3 = randomLongBetween(retainingSeqNoForNode2 + 1, retainingSeqNoOnPrimary + 100); + List retentionLeases = Arrays.asList( + newRetentionLease(node1, retainingSeqNoOnPrimary), + newRetentionLease(node2, retainingSeqNoForNode2), + newRetentionLease(node3, retainingSeqNoForNode3) + ); + testBatchAllocator.addData( + node1, + retentionLeases, + "MATCH", + null, + new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION) + ); + testBatchAllocator.addData( + node2, + "NOT_MATCH", + null, + new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION) + ); + testBatchAllocator.addData( + node3, + randomSyncId(), + null, + new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION) + ); + allocateAllUnassignedBatch(allocation); + assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING).size(), equalTo(1)); + assertThat( + allocation.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING).get(0).currentNodeId(), + equalTo(node3.getId()) + ); + } + + public void testCancelRecoveryIfFoundCopyWithNoopRetentionLease() { + final UnassignedInfo unassignedInfo; + final Set failedNodes; + if (randomBoolean()) { + failedNodes = Collections.emptySet(); + unassignedInfo = new UnassignedInfo(UnassignedInfo.Reason.CLUSTER_RECOVERED, null); + } else { + failedNodes = new HashSet<>(randomSubsetOf(Arrays.asList("node-4", "node-5", "node-6"))); + unassignedInfo = new UnassignedInfo( + UnassignedInfo.Reason.ALLOCATION_FAILED, + null, + null, + randomIntBetween(1, 10), + System.nanoTime(), + System.currentTimeMillis(), + false, + UnassignedInfo.AllocationStatus.NO_ATTEMPT, + failedNodes + ); + } + RoutingAllocation allocation = onePrimaryOnNode1And1ReplicaRecovering(yesAllocationDeciders(), unassignedInfo); + long retainingSeqNo = randomLongBetween(1, Long.MAX_VALUE); + testBatchAllocator.addData( + node1, + Arrays.asList(newRetentionLease(node1, retainingSeqNo), newRetentionLease(node3, retainingSeqNo)), + "MATCH", + null, + new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION) + ); + testBatchAllocator.addData( + node2, + "NO_MATCH", + null, + new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION) + ); + testBatchAllocator.addData( + node3, + randomSyncId(), + null, + new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION) + ); + Collection replicaShards = allocation.routingNodes().shardsWithState(ShardRoutingState.UNASSIGNED); + List shardRoutingBatch = new ArrayList<>(replicaShards); + List> shardBatchList = Collections.singletonList( + new ArrayList<>(allocation.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING)) + ); + + testBatchAllocator.processExistingRecoveries(allocation, shardBatchList); + assertThat(allocation.routingNodesChanged(), equalTo(true)); + List unassignedShards = allocation.routingNodes().shardsWithState(ShardRoutingState.UNASSIGNED); + assertThat(unassignedShards, hasSize(1)); + assertThat(unassignedShards.get(0).shardId(), equalTo(shardId)); + assertThat(unassignedShards.get(0).unassignedInfo().getNumFailedAllocations(), equalTo(0)); + assertThat(unassignedShards.get(0).unassignedInfo().getFailedNodeIds(), equalTo(failedNodes)); + } + + public void testNotCancellingRecoveryIfCurrentRecoveryHasRetentionLease() { + RoutingAllocation allocation = onePrimaryOnNode1And1ReplicaRecovering(yesAllocationDeciders()); + List peerRecoveryRetentionLeasesOnPrimary = new ArrayList<>(); + long retainingSeqNo = randomLongBetween(1, Long.MAX_VALUE); + peerRecoveryRetentionLeasesOnPrimary.add(newRetentionLease(node1, retainingSeqNo)); + peerRecoveryRetentionLeasesOnPrimary.add(newRetentionLease(node2, randomLongBetween(1, retainingSeqNo))); + if (randomBoolean()) { + peerRecoveryRetentionLeasesOnPrimary.add(newRetentionLease(node3, randomLongBetween(0, retainingSeqNo))); + } + testBatchAllocator.addData( + node1, + peerRecoveryRetentionLeasesOnPrimary, + "MATCH", + null, + new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION) + ); + testBatchAllocator.addData( + node2, + randomSyncId(), + null, + new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION) + ); + testBatchAllocator.addData( + node3, + randomSyncId(), + null, + new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION) + ); + testBatchAllocator.processExistingRecoveries( + allocation, + Collections.singletonList(new ArrayList<>(allocation.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING))) + ); + assertThat(allocation.routingNodesChanged(), equalTo(false)); + assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.UNASSIGNED).size(), equalTo(0)); + } + + public void testNotCancelIfPrimaryDoesNotHaveValidRetentionLease() { + RoutingAllocation allocation = onePrimaryOnNode1And1ReplicaRecovering(yesAllocationDeciders()); + testBatchAllocator.addData( + node1, + Collections.singletonList(newRetentionLease(node3, randomNonNegativeLong())), + "MATCH", + null, + new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION) + ); + testBatchAllocator.addData( + node2, + "NOT_MATCH", + null, + new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION) + ); + testBatchAllocator.addData( + node3, + "NOT_MATCH", + null, + new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION) + ); + testBatchAllocator.processExistingRecoveries( + allocation, + Collections.singletonList(new ArrayList<>(allocation.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING))) + ); + assertThat(allocation.routingNodesChanged(), equalTo(false)); + assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.UNASSIGNED).size(), equalTo(0)); + } + + public void testIgnoreRetentionLeaseIfCopyIsEmpty() { + RoutingAllocation allocation = onePrimaryOnNode1And1Replica(yesAllocationDeciders()); + long retainingSeqNo = randomLongBetween(1, Long.MAX_VALUE); + List retentionLeases = new ArrayList<>(); + retentionLeases.add(newRetentionLease(node1, retainingSeqNo)); + retentionLeases.add(newRetentionLease(node2, randomLongBetween(0, retainingSeqNo))); + if (randomBoolean()) { + retentionLeases.add(newRetentionLease(node3, randomLongBetween(0, retainingSeqNo))); + } + testBatchAllocator.addData( + node1, + retentionLeases, + randomSyncId(), + null, + new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION) + ); + testBatchAllocator.addData(node2, null, null); // has retention lease but store is empty + testBatchAllocator.addData( + node3, + randomSyncId(), + null, + new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION) + ); + allocateAllUnassignedBatch(allocation); + assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING).size(), equalTo(1)); + assertThat( + allocation.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING).get(0).currentNodeId(), + equalTo(node3.getId()) + ); + } + + /** + * When we can't find primary data, but still find replica data, we go ahead and keep it unassigned + * to be allocated. This is today behavior, which relies on a primary corruption identified with + * adding a replica and having that replica actually recover and cause the corruption to be identified + * See CorruptFileTest# + */ + public void testNoPrimaryData() { + RoutingAllocation allocation = onePrimaryOnNode1And1Replica(yesAllocationDeciders()); + testBatchAllocator.addData( + node2, + "MATCH", + null, + new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION) + ); + allocateAllUnassignedBatch(allocation); + assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.UNASSIGNED).size(), equalTo(1)); + assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.UNASSIGNED).get(0).shardId(), equalTo(shardId)); + } + + /** + * Verifies that when there is primary data, but no data at all on other nodes, the shard keeps + * unassigned to be allocated later on. + */ + public void testNoDataForReplicaOnAnyNode() { + RoutingAllocation allocation = onePrimaryOnNode1And1Replica(yesAllocationDeciders()); + testBatchAllocator.addData( + node1, + "MATCH", + null, + new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION) + ); + allocateAllUnassignedBatch(allocation); + assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.UNASSIGNED).size(), equalTo(1)); + assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.UNASSIGNED).get(0).shardId(), equalTo(shardId)); + } + + /** + * Verifies that when there is primary data, but no matching data at all on other nodes, the shard keeps + * unassigned to be allocated later on. + */ + public void testNoMatchingFilesForReplicaOnAnyNode() { + RoutingAllocation allocation = onePrimaryOnNode1And1Replica(yesAllocationDeciders()); + testBatchAllocator.addData(node1, "MATCH", null, new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION)) + .addData(node2, "NO_MATCH", null, new StoreFileMetadata("file1", 10, "NO_MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION)); + allocateAllUnassignedBatch(allocation); + assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.UNASSIGNED).size(), equalTo(1)); + assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.UNASSIGNED).get(0).shardId(), equalTo(shardId)); + } + + /** + * When there is no decision or throttle decision across all nodes for the shard, make sure the shard + * moves to the ignore unassigned list. + */ + public void testNoOrThrottleDecidersRemainsInUnassigned() { + RoutingAllocation allocation = onePrimaryOnNode1And1Replica( + randomBoolean() ? noAllocationDeciders() : throttleAllocationDeciders() + ); + testBatchAllocator.addData(node1, "MATCH", null, new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION)) + .addData(node2, "MATCH", null, new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION)); + allocateAllUnassignedBatch(allocation); + assertThat(allocation.routingNodes().unassigned().ignored().size(), equalTo(1)); + assertThat(allocation.routingNodes().unassigned().ignored().get(0).shardId(), equalTo(shardId)); + } + + /** + * Tests when the node to allocate to due to matching is being throttled, we move the shard to ignored + * to wait till throttling on it is done. + */ + public void testThrottleWhenAllocatingToMatchingNode() { + RoutingAllocation allocation = onePrimaryOnNode1And1Replica( + new AllocationDeciders( + Arrays.asList( + new TestAllocateDecision(Decision.YES), + new SameShardAllocationDecider( + Settings.EMPTY, + new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS) + ), + new AllocationDecider() { + @Override + public Decision canAllocate(ShardRouting shardRouting, RoutingNode node, RoutingAllocation allocation) { + if (node.node().equals(node2)) { + return Decision.THROTTLE; + } + return Decision.YES; + } + } + ) + ) + ); + testBatchAllocator.addData(node1, "MATCH", null, new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION)) + .addData(node2, "MATCH", null, new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION)); + allocateAllUnassignedBatch(allocation); + assertThat(allocation.routingNodes().unassigned().ignored().size(), equalTo(1)); + assertThat(allocation.routingNodes().unassigned().ignored().get(0).shardId(), equalTo(shardId)); + } + + public void testDelayedAllocation() { + RoutingAllocation allocation = onePrimaryOnNode1And1Replica( + yesAllocationDeciders(), + Settings.builder().put(UnassignedInfo.INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING.getKey(), TimeValue.timeValueHours(1)).build(), + UnassignedInfo.Reason.NODE_LEFT + ); + testBatchAllocator.addData( + node1, + "MATCH", + null, + new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION) + ); + if (randomBoolean()) { + // we sometime return empty list of files, make sure we test this as well + testBatchAllocator.addData(node2, null, null); + } + allocateAllUnassignedBatch(allocation); + assertThat(allocation.routingNodesChanged(), equalTo(false)); + assertThat(allocation.routingNodes().unassigned().ignored().size(), equalTo(1)); + assertThat(allocation.routingNodes().unassigned().ignored().get(0).shardId(), equalTo(shardId)); + + allocation = onePrimaryOnNode1And1Replica( + yesAllocationDeciders(), + Settings.builder().put(UnassignedInfo.INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING.getKey(), TimeValue.timeValueHours(1)).build(), + UnassignedInfo.Reason.NODE_LEFT + ); + testBatchAllocator.addData( + node2, + "MATCH", + null, + new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION) + ); + allocateAllUnassignedBatch(allocation); + assertThat(allocation.routingNodesChanged(), equalTo(true)); + assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING).size(), equalTo(1)); + assertThat( + allocation.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING).get(0).currentNodeId(), + equalTo(node2.getId()) + ); + } + + public void testCancelRecoveryBetterSyncId() { + RoutingAllocation allocation = onePrimaryOnNode1And1ReplicaRecovering(yesAllocationDeciders()); + testBatchAllocator.addData(node1, "MATCH", null, new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION)) + .addData(node2, "NO_MATCH", null, new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION)) + .addData(node3, "MATCH", null, new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION)); + testBatchAllocator.processExistingRecoveries( + allocation, + Collections.singletonList(new ArrayList<>(allocation.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING))) + ); + assertThat(allocation.routingNodesChanged(), equalTo(true)); + assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.UNASSIGNED).size(), equalTo(1)); + assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.UNASSIGNED).get(0).shardId(), equalTo(shardId)); + } + + public void testNotCancellingRecoveryIfSyncedOnExistingRecovery() { + final UnassignedInfo unassignedInfo; + if (randomBoolean()) { + unassignedInfo = new UnassignedInfo(UnassignedInfo.Reason.CLUSTER_RECOVERED, null); + } else { + unassignedInfo = new UnassignedInfo( + UnassignedInfo.Reason.ALLOCATION_FAILED, + null, + null, + randomIntBetween(1, 10), + System.nanoTime(), + System.currentTimeMillis(), + false, + UnassignedInfo.AllocationStatus.NO_ATTEMPT, + Collections.singleton("node-4") + ); + } + RoutingAllocation allocation = onePrimaryOnNode1And1ReplicaRecovering(yesAllocationDeciders(), unassignedInfo); + List retentionLeases = new ArrayList<>(); + if (randomBoolean()) { + long retainingSeqNoOnPrimary = randomLongBetween(0, Long.MAX_VALUE); + retentionLeases.add(newRetentionLease(node1, retainingSeqNoOnPrimary)); + if (randomBoolean()) { + retentionLeases.add(newRetentionLease(node2, randomLongBetween(0, retainingSeqNoOnPrimary))); + } + if (randomBoolean()) { + retentionLeases.add(newRetentionLease(node3, randomLongBetween(0, retainingSeqNoOnPrimary))); + } + } + testBatchAllocator.addData( + node1, + retentionLeases, + "MATCH", + null, + new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION) + ); + testBatchAllocator.addData( + node2, + "MATCH", + null, + new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION) + ); + testBatchAllocator.addData( + node3, + randomSyncId(), + null, + new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION) + ); + testBatchAllocator.processExistingRecoveries( + allocation, + Collections.singletonList(new ArrayList<>(allocation.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING))) + ); + assertThat(allocation.routingNodesChanged(), equalTo(false)); + assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.UNASSIGNED).size(), equalTo(0)); + } + + public void testNotCancellingRecovery() { + RoutingAllocation allocation = onePrimaryOnNode1And1ReplicaRecovering(yesAllocationDeciders()); + testBatchAllocator.addData(node1, "MATCH", null, new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION)) + .addData(node2, "MATCH", null, new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION)); + testBatchAllocator.processExistingRecoveries( + allocation, + Collections.singletonList(new ArrayList<>(allocation.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING))) + ); + assertThat(allocation.routingNodesChanged(), equalTo(false)); + assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.UNASSIGNED).size(), equalTo(0)); + } + + public void testDoNotCancelForBrokenNode() { + Set failedNodes = new HashSet<>(); + failedNodes.add(node3.getId()); + if (randomBoolean()) { + failedNodes.add("node4"); + } + UnassignedInfo unassignedInfo = new UnassignedInfo( + UnassignedInfo.Reason.ALLOCATION_FAILED, + null, + null, + randomIntBetween(failedNodes.size(), 10), + System.nanoTime(), + System.currentTimeMillis(), + false, + UnassignedInfo.AllocationStatus.NO_ATTEMPT, + failedNodes + ); + RoutingAllocation allocation = onePrimaryOnNode1And1ReplicaRecovering(yesAllocationDeciders(), unassignedInfo); + long retainingSeqNoOnPrimary = randomLongBetween(0, Long.MAX_VALUE); + List retentionLeases = Arrays.asList( + newRetentionLease(node1, retainingSeqNoOnPrimary), + newRetentionLease(node3, retainingSeqNoOnPrimary) + ); + testBatchAllocator.addData( + node1, + retentionLeases, + "MATCH", + null, + new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION) + ) + .addData(node2, randomSyncId(), null, new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION)) + .addData(node3, randomSyncId(), null, new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION)); + testBatchAllocator.processExistingRecoveries( + allocation, + Collections.singletonList(new ArrayList<>(allocation.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING))) + ); + assertThat(allocation.routingNodesChanged(), equalTo(false)); + assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.UNASSIGNED), empty()); + } + + private RoutingAllocation onePrimaryOnNode1And1Replica(AllocationDeciders deciders) { + return onePrimaryOnNode1And1Replica(deciders, Settings.EMPTY, UnassignedInfo.Reason.CLUSTER_RECOVERED); + } + + private RoutingAllocation onePrimaryOnNode1And1Replica(AllocationDeciders deciders, Settings settings, UnassignedInfo.Reason reason) { + ShardRouting primaryShard = TestShardRouting.newShardRouting(shardId, node1.getId(), true, ShardRoutingState.STARTED); + IndexMetadata.Builder indexMetadata = IndexMetadata.builder(shardId.getIndexName()) + .settings(settings(Version.CURRENT).put(settings)) + .numberOfShards(1) + .numberOfReplicas(1) + .putInSyncAllocationIds(0, Sets.newHashSet(primaryShard.allocationId().getId())); + Metadata metadata = Metadata.builder().put(indexMetadata).build(); + // mark shard as delayed if reason is NODE_LEFT + boolean delayed = reason == UnassignedInfo.Reason.NODE_LEFT + && UnassignedInfo.INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING.get(settings).nanos() > 0; + int failedAllocations = reason == UnassignedInfo.Reason.ALLOCATION_FAILED ? 1 : 0; + RoutingTable routingTable = RoutingTable.builder() + .add( + IndexRoutingTable.builder(shardId.getIndex()) + .addIndexShard( + new IndexShardRoutingTable.Builder(shardId).addShard(primaryShard) + .addShard( + ShardRouting.newUnassigned( + shardId, + false, + RecoverySource.PeerRecoverySource.INSTANCE, + new UnassignedInfo( + reason, + null, + null, + failedAllocations, + System.nanoTime(), + System.currentTimeMillis(), + delayed, + UnassignedInfo.AllocationStatus.NO_ATTEMPT, + Collections.emptySet() + ) + ) + ) + .build() + ) + ) + .build(); + ClusterState state = ClusterState.builder(org.opensearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)) + .metadata(metadata) + .routingTable(routingTable) + .nodes(DiscoveryNodes.builder().add(node1).add(node2).add(node3)) + .build(); + return new RoutingAllocation( + deciders, + new RoutingNodes(state, false), + state, + ClusterInfo.EMPTY, + SnapshotShardSizeInfo.EMPTY, + System.nanoTime() + ); + } + + private RoutingAllocation onePrimaryOnNode1And1ReplicaRecovering(AllocationDeciders deciders, UnassignedInfo unassignedInfo) { + ShardRouting primaryShard = TestShardRouting.newShardRouting(shardId, node1.getId(), true, ShardRoutingState.STARTED); + Metadata metadata = Metadata.builder() + .put( + IndexMetadata.builder(shardId.getIndexName()) + .settings(settings(Version.CURRENT)) + .numberOfShards(1) + .numberOfReplicas(1) + .putInSyncAllocationIds(0, Sets.newHashSet(primaryShard.allocationId().getId())) + ) + .build(); + RoutingTable routingTable = RoutingTable.builder() + .add( + IndexRoutingTable.builder(shardId.getIndex()) + .addIndexShard( + new IndexShardRoutingTable.Builder(shardId).addShard(primaryShard) + .addShard( + TestShardRouting.newShardRouting( + shardId, + node2.getId(), + null, + false, + ShardRoutingState.INITIALIZING, + unassignedInfo + ) + ) + .build() + ) + ) + .build(); + ClusterState state = ClusterState.builder(org.opensearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)) + .metadata(metadata) + .routingTable(routingTable) + .nodes(DiscoveryNodes.builder().add(node1).add(node2).add(node3)) + .build(); + return new RoutingAllocation( + deciders, + new RoutingNodes(state, false), + state, + ClusterInfo.EMPTY, + SnapshotShardSizeInfo.EMPTY, + System.nanoTime() + ); + } + + private RoutingAllocation onePrimaryOnNode1And1ReplicaRecovering(AllocationDeciders deciders) { + return onePrimaryOnNode1And1ReplicaRecovering(deciders, new UnassignedInfo(UnassignedInfo.Reason.CLUSTER_RECOVERED, null)); + } + + static RetentionLease newRetentionLease(DiscoveryNode node, long retainingSeqNo) { + return new RetentionLease( + ReplicationTracker.getPeerRecoveryRetentionLeaseId(node.getId()), + retainingSeqNo, + randomNonNegativeLong(), + ReplicationTracker.PEER_RECOVERY_RETENTION_LEASE_SOURCE + ); + } + + static String randomSyncId() { + return randomFrom("MATCH", "NOT_MATCH", null); + } + + class TestBatchAllocator extends ReplicaShardBatchAllocator { + private Map data = null; + private AtomicBoolean fetchDataCalled = new AtomicBoolean(false); + private AtomicInteger eligibleShardFetchDataCount = new AtomicInteger(0); + + public void clean() { + data = null; + } + + public boolean getFetchDataCalledAndClean() { + return fetchDataCalled.getAndSet(false); + } + + public int getShardEligibleFetchDataCountAndClean() { + return eligibleShardFetchDataCount.getAndSet(0); + } + + public TestBatchAllocator addData( + DiscoveryNode node, + String syncId, + @Nullable Exception storeFileFetchException, + StoreFileMetadata... files + ) { + return addData(node, Collections.emptyList(), syncId, storeFileFetchException, files); + } + + public TestBatchAllocator addData( + DiscoveryNode node, + List peerRecoveryRetentionLeases, + String syncId, + @Nullable Exception storeFileFetchException, + StoreFileMetadata... files + ) { + if (data == null) { + data = new HashMap<>(); + } + Map filesAsMap = new HashMap<>(); + for (StoreFileMetadata file : files) { + filesAsMap.put(file.name(), file); + } + Map commitData = new HashMap<>(); + if (syncId != null) { + commitData.put(Engine.SYNC_COMMIT_ID, syncId); + } + data.put( + node, + new TransportNodesListShardStoreMetadataBatch.NodeStoreFilesMetadata( + new TransportNodesListShardStoreMetadataHelper.StoreFilesMetadata( + shardId, + new Store.MetadataSnapshot(unmodifiableMap(filesAsMap), unmodifiableMap(commitData), randomInt()), + peerRecoveryRetentionLeases + ), + storeFileFetchException + ) + ); + return this; + } + + @Override + protected AsyncShardFetch.FetchResult fetchData( + List eligibleShards, + List ineligibleShards, + RoutingAllocation allocation + ) { + fetchDataCalled.set(true); + eligibleShardFetchDataCount.set(eligibleShards.size()); + Map tData = null; + if (data != null) { + tData = new HashMap<>(); + for (Map.Entry entry : data.entrySet()) { + Map shardData = Map.of( + shardId, + entry.getValue() + ); + tData.put( + entry.getKey(), + new TransportNodesListShardStoreMetadataBatch.NodeStoreFilesMetadataBatch(entry.getKey(), shardData) + ); + } + } + return new AsyncShardFetch.FetchResult<>(tData, new HashMap<>() { + { + put(shardId, Collections.emptySet()); + } + }); + } + + @Override + protected boolean hasInitiatedFetching(ShardRouting shard) { + return fetchDataCalled.get(); + } + } +} From d5b27b120705857f29fdc3e5ada47e2b087a050a Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Wed, 20 Mar 2024 15:46:06 -0400 Subject: [PATCH 18/38] Bump com.azure:azure-json from 1.0.1 to 1.1.0 in /plugins/repository-azure (#12723) * Bump com.azure:azure-json in /plugins/repository-azure Bumps [com.azure:azure-json](https://github.com/Azure/azure-sdk-for-java) from 1.0.1 to 1.1.0. - [Release notes](https://github.com/Azure/azure-sdk-for-java/releases) - [Commits](https://github.com/Azure/azure-sdk-for-java/compare/azure-json_1.0.1...v1.1.0) --- updated-dependencies: - dependency-name: com.azure:azure-json dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] * Updating SHAs Signed-off-by: dependabot[bot] * Update changelog Signed-off-by: dependabot[bot] --------- Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> Co-authored-by: dependabot[bot] --- CHANGELOG.md | 1 + plugins/repository-azure/build.gradle | 2 +- plugins/repository-azure/licenses/azure-json-1.0.1.jar.sha1 | 1 - plugins/repository-azure/licenses/azure-json-1.1.0.jar.sha1 | 1 + 4 files changed, 3 insertions(+), 2 deletions(-) delete mode 100644 plugins/repository-azure/licenses/azure-json-1.0.1.jar.sha1 create mode 100644 plugins/repository-azure/licenses/azure-json-1.1.0.jar.sha1 diff --git a/CHANGELOG.md b/CHANGELOG.md index 4ba3dbcdc8d9b..026783a0d6089 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -145,6 +145,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Bump `reactor` from 3.5.14 to 3.5.15 ([#12633](https://github.com/opensearch-project/OpenSearch/pull/12633)) - Bump `peter-evans/create-pull-request` from 5 to 6 ([#12724](https://github.com/opensearch-project/OpenSearch/pull/12724)) - Bump `org.apache.commons:commons-configuration2` from 2.9.0 to 2.10.0 ([#12721](https://github.com/opensearch-project/OpenSearch/pull/12721)) +- Bump `com.azure:azure-json` from 1.0.1 to 1.1.0 ([#12723](https://github.com/opensearch-project/OpenSearch/pull/12723)) ### Changed - Allow composite aggregation to run under a parent filter aggregation ([#11499](https://github.com/opensearch-project/OpenSearch/pull/11499)) diff --git a/plugins/repository-azure/build.gradle b/plugins/repository-azure/build.gradle index 31db767b2c68e..c7836170d658f 100644 --- a/plugins/repository-azure/build.gradle +++ b/plugins/repository-azure/build.gradle @@ -45,7 +45,7 @@ opensearchplugin { dependencies { api 'com.azure:azure-core:1.47.0' - api 'com.azure:azure-json:1.0.1' + api 'com.azure:azure-json:1.1.0' api 'com.azure:azure-storage-common:12.21.2' api 'com.azure:azure-core-http-netty:1.12.8' api "io.netty:netty-codec-dns:${versions.netty}" diff --git a/plugins/repository-azure/licenses/azure-json-1.0.1.jar.sha1 b/plugins/repository-azure/licenses/azure-json-1.0.1.jar.sha1 deleted file mode 100644 index 128a82717fef9..0000000000000 --- a/plugins/repository-azure/licenses/azure-json-1.0.1.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -abdfdb0c49eebe75ed8532d047dea0c9f13c30ac \ No newline at end of file diff --git a/plugins/repository-azure/licenses/azure-json-1.1.0.jar.sha1 b/plugins/repository-azure/licenses/azure-json-1.1.0.jar.sha1 new file mode 100644 index 0000000000000..e44ee47c40253 --- /dev/null +++ b/plugins/repository-azure/licenses/azure-json-1.1.0.jar.sha1 @@ -0,0 +1 @@ +1f21cea72f54a6af3b0bb6831eb3874bd4afd213 \ No newline at end of file From 29086216427c26f0a82bb692a854dea988aa1193 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Wed, 20 Mar 2024 16:07:09 -0400 Subject: [PATCH 19/38] Bump com.google.api.grpc:proto-google-common-protos from 2.33.0 to 2.34.0 in /plugins/repository-gcs (#12365) * Bump com.google.api.grpc:proto-google-common-protos Bumps [com.google.api.grpc:proto-google-common-protos](https://github.com/googleapis/sdk-platform-java) from 2.33.0 to 2.34.0. - [Release notes](https://github.com/googleapis/sdk-platform-java/releases) - [Changelog](https://github.com/googleapis/sdk-platform-java/blob/main/CHANGELOG.md) - [Commits](https://github.com/googleapis/sdk-platform-java/compare/v2.33.0...v2.34.0) --- updated-dependencies: - dependency-name: com.google.api.grpc:proto-google-common-protos dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] * Updating SHAs Signed-off-by: dependabot[bot] * Update changelog Signed-off-by: dependabot[bot] Signed-off-by: Andriy Redko --------- Signed-off-by: dependabot[bot] Signed-off-by: Andriy Redko Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> Co-authored-by: dependabot[bot] --- CHANGELOG.md | 2 +- plugins/repository-gcs/build.gradle | 5 ++++- .../licenses/proto-google-common-protos-2.33.0.jar.sha1 | 1 - .../licenses/proto-google-common-protos-2.37.1.jar.sha1 | 1 + 4 files changed, 6 insertions(+), 3 deletions(-) delete mode 100644 plugins/repository-gcs/licenses/proto-google-common-protos-2.33.0.jar.sha1 create mode 100644 plugins/repository-gcs/licenses/proto-google-common-protos-2.37.1.jar.sha1 diff --git a/CHANGELOG.md b/CHANGELOG.md index 026783a0d6089..504eba04993a2 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -123,7 +123,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), ### Dependencies - Bump `peter-evans/find-comment` from 2 to 3 ([#12288](https://github.com/opensearch-project/OpenSearch/pull/12288)) -- Bump `com.google.api.grpc:proto-google-common-protos` from 2.25.1 to 2.33.0 ([#12289](https://github.com/opensearch-project/OpenSearch/pull/12289)) +- Bump `com.google.api.grpc:proto-google-common-protos` from 2.25.1 to 2.37.1 ([#12289](https://github.com/opensearch-project/OpenSearch/pull/12289), [#12365](https://github.com/opensearch-project/OpenSearch/pull/12365)) - Bump `com.squareup.okio:okio` from 3.7.0 to 3.8.0 ([#12290](https://github.com/opensearch-project/OpenSearch/pull/12290)) - Bump `gradle/wrapper-validation-action` from 1 to 2 ([#12367](https://github.com/opensearch-project/OpenSearch/pull/12367)) - Bump `netty` from 4.1.106.Final to 4.1.107.Final ([#12372](https://github.com/opensearch-project/OpenSearch/pull/12372)) diff --git a/plugins/repository-gcs/build.gradle b/plugins/repository-gcs/build.gradle index 0ddcf0f6dddca..1dfc64e19601c 100644 --- a/plugins/repository-gcs/build.gradle +++ b/plugins/repository-gcs/build.gradle @@ -60,7 +60,7 @@ dependencies { api 'com.google.api-client:google-api-client:2.2.0' - api 'com.google.api.grpc:proto-google-common-protos:2.33.0' + api 'com.google.api.grpc:proto-google-common-protos:2.37.1' api 'com.google.api.grpc:proto-google-iam-v1:0.12.0' api "com.google.auth:google-auth-library-credentials:${versions.google_auth}" @@ -149,6 +149,9 @@ thirdPartyAudit { 'com.google.appengine.api.urlfetch.URLFetchService', 'com.google.appengine.api.urlfetch.URLFetchServiceFactory', 'com.google.auth.oauth2.GdchCredentials', + 'com.google.protobuf.MapFieldBuilder', + 'com.google.protobuf.MapFieldBuilder$Converter', + 'com.google.protobuf.MapFieldReflectionAccessor', 'com.google.protobuf.util.JsonFormat', 'com.google.protobuf.util.JsonFormat$Parser', 'com.google.protobuf.util.JsonFormat$Printer', diff --git a/plugins/repository-gcs/licenses/proto-google-common-protos-2.33.0.jar.sha1 b/plugins/repository-gcs/licenses/proto-google-common-protos-2.33.0.jar.sha1 deleted file mode 100644 index 746e4e99fd881..0000000000000 --- a/plugins/repository-gcs/licenses/proto-google-common-protos-2.33.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -644e11df1cec6d38a63a9a06a701e48c398b87d0 \ No newline at end of file diff --git a/plugins/repository-gcs/licenses/proto-google-common-protos-2.37.1.jar.sha1 b/plugins/repository-gcs/licenses/proto-google-common-protos-2.37.1.jar.sha1 new file mode 100644 index 0000000000000..92f991778ccc3 --- /dev/null +++ b/plugins/repository-gcs/licenses/proto-google-common-protos-2.37.1.jar.sha1 @@ -0,0 +1 @@ +3b8759ef0468cced72f8f0d4fc3cc57aeb8139f8 \ No newline at end of file From 7ad30171a8869f659c1feeb6185acf023ebaa203 Mon Sep 17 00:00:00 2001 From: rajiv-kv <157019998+rajiv-kv@users.noreply.github.com> Date: Thu, 21 Mar 2024 03:01:15 +0530 Subject: [PATCH 20/38] Light weight Transport action to verify local term before fetching cluster-state from remote (#12252) Signed-off-by: Rajiv Kumar Vaidyanathan --- CHANGELOG.md | 1 + .../cluster/state/FetchByTermVersionIT.java | 161 +++++++++ .../org/opensearch/action/ActionModule.java | 3 + .../state/TransportClusterStateAction.java | 11 +- .../state/term/GetTermVersionAction.java | 26 ++ .../state/term/GetTermVersionRequest.java | 34 ++ .../state/term/GetTermVersionResponse.java | 50 +++ .../term/TransportGetTermVersionAction.java | 85 +++++ .../cluster/state/term/package-info.java | 10 + .../TransportClusterManagerNodeAction.java | 183 +++++++--- .../coordination/ClusterStateTermVersion.java | 110 ++++++ .../state/term/ClusterTermVersionIT.java | 121 +++++++ .../state/term/ClusterTermVersionTests.java | 26 ++ ...ransportClusterManagerNodeActionTests.java | 18 - ...TransportClusterManagerTermCheckTests.java | 320 ++++++++++++++++++ .../snapshots/SnapshotResiliencyTests.java | 14 + 16 files changed, 1111 insertions(+), 62 deletions(-) create mode 100644 server/src/internalClusterTest/java/org/opensearch/cluster/state/FetchByTermVersionIT.java create mode 100644 server/src/main/java/org/opensearch/action/admin/cluster/state/term/GetTermVersionAction.java create mode 100644 server/src/main/java/org/opensearch/action/admin/cluster/state/term/GetTermVersionRequest.java create mode 100644 server/src/main/java/org/opensearch/action/admin/cluster/state/term/GetTermVersionResponse.java create mode 100644 server/src/main/java/org/opensearch/action/admin/cluster/state/term/TransportGetTermVersionAction.java create mode 100644 server/src/main/java/org/opensearch/action/admin/cluster/state/term/package-info.java create mode 100644 server/src/main/java/org/opensearch/cluster/coordination/ClusterStateTermVersion.java create mode 100644 server/src/test/java/org/opensearch/action/admin/cluster/state/term/ClusterTermVersionIT.java create mode 100644 server/src/test/java/org/opensearch/action/admin/cluster/state/term/ClusterTermVersionTests.java create mode 100644 server/src/test/java/org/opensearch/action/support/clustermanager/TransportClusterManagerTermCheckTests.java diff --git a/CHANGELOG.md b/CHANGELOG.md index 504eba04993a2..c65b20b99bc25 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -120,6 +120,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - [Admission Control] Integrated IO Based AdmissionController to AdmissionControl Framework ([#12583](https://github.com/opensearch-project/OpenSearch/pull/12583)) - Introduce a new setting `index.check_pending_flush.enabled` to expose the ability to disable the check for pending flushes by write threads ([#12710](https://github.com/opensearch-project/OpenSearch/pull/12710)) - Built-in secure transports support ([#12435](https://github.com/opensearch-project/OpenSearch/pull/12435)) +- Lightweight Transport action to verify local term before fetching cluster-state from remote ([#12252](https://github.com/opensearch-project/OpenSearch/pull/12252/)) ### Dependencies - Bump `peter-evans/find-comment` from 2 to 3 ([#12288](https://github.com/opensearch-project/OpenSearch/pull/12288)) diff --git a/server/src/internalClusterTest/java/org/opensearch/cluster/state/FetchByTermVersionIT.java b/server/src/internalClusterTest/java/org/opensearch/cluster/state/FetchByTermVersionIT.java new file mode 100644 index 0000000000000..cef184b3fddf9 --- /dev/null +++ b/server/src/internalClusterTest/java/org/opensearch/cluster/state/FetchByTermVersionIT.java @@ -0,0 +1,161 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.cluster.state; + +import org.opensearch.action.admin.cluster.state.ClusterStateRequest; +import org.opensearch.action.admin.cluster.state.ClusterStateResponse; +import org.opensearch.action.admin.cluster.state.term.GetTermVersionAction; +import org.opensearch.action.admin.cluster.state.term.GetTermVersionResponse; +import org.opensearch.action.admin.indices.mapping.put.PutMappingRequest; +import org.opensearch.cluster.ClusterName; +import org.opensearch.cluster.coordination.ClusterStateTermVersion; +import org.opensearch.cluster.metadata.IndexMetadata; +import org.opensearch.common.settings.Settings; +import org.opensearch.index.mapper.MapperService; +import org.opensearch.plugins.Plugin; +import org.opensearch.test.OpenSearchIntegTestCase; +import org.opensearch.test.transport.MockTransportService; +import org.opensearch.transport.TransportService; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.stream.IntStream; + +import static org.hamcrest.Matchers.is; + +@SuppressWarnings("unchecked") +@OpenSearchIntegTestCase.ClusterScope(scope = OpenSearchIntegTestCase.Scope.TEST, numDataNodes = 0) +public class FetchByTermVersionIT extends OpenSearchIntegTestCase { + + AtomicBoolean isTermVersionCheckEnabled = new AtomicBoolean(); + + protected Collection> nodePlugins() { + return List.of(MockTransportService.TestPlugin.class); + } + + AtomicBoolean forceFetchFromCM = new AtomicBoolean(); + + public void testClusterStateResponseFromDataNode() throws Exception { + String cm = internalCluster().startClusterManagerOnlyNode(); + List dns = internalCluster().startDataOnlyNodes(5); + int numberOfShards = dns.size(); + stubClusterTermResponse(cm); + + ensureClusterSizeConsistency(); + ensureGreen(); + + List indices = new ArrayList<>(); + + // Create a large sized cluster-state by creating field mappings + IntStream.range(0, 20).forEachOrdered(n -> { + String index = "index_" + n; + createIndex( + index, + Settings.builder() + .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, numberOfShards) + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0) + .put(MapperService.INDEX_MAPPING_TOTAL_FIELDS_LIMIT_SETTING.getKey(), Long.MAX_VALUE) + .build() + ); + indices.add(index); + }); + IntStream.range(0, 5).forEachOrdered(n -> { + List mappings = new ArrayList<>(); + for (int i = 0; i < 2000; i++) { + mappings.add("t-123456789-123456789-" + n + "-" + i); + mappings.add("type=keyword"); + } + PutMappingRequest request = new PutMappingRequest().source(mappings.toArray(new String[0])) + .indices(indices.toArray(new String[0])); + internalCluster().dataNodeClient().admin().indices().putMapping(request).actionGet(); + }); + ensureGreen(); + + ClusterStateResponse stateResponseM = internalCluster().clusterManagerClient() + .admin() + .cluster() + .state(new ClusterStateRequest()) + .actionGet(); + + waitUntil(() -> { + ClusterStateResponse stateResponseD = internalCluster().dataNodeClient() + .admin() + .cluster() + .state(new ClusterStateRequest()) + .actionGet(); + return stateResponseD.getState().stateUUID().equals(stateResponseM.getState().stateUUID()); + }); + // cluster state response time with term check enabled on datanode + isTermVersionCheckEnabled.set(true); + { + List latencies = new ArrayList<>(); + IntStream.range(0, 50).forEachOrdered(n1 -> { + ClusterStateRequest clusterStateRequest = new ClusterStateRequest(); + long start = System.currentTimeMillis(); + ClusterStateResponse stateResponse = dataNodeClient().admin().cluster().state(clusterStateRequest).actionGet(); + latencies.add(System.currentTimeMillis() - start); + assertThat(stateResponse.getClusterName().value(), is(internalCluster().getClusterName())); + assertThat(stateResponse.getState().nodes().getSize(), is(internalCluster().getNodeNames().length)); + assertThat(stateResponse.getState().metadata().indices().size(), is(indices.size())); + Map fieldMappings = (Map) stateResponse.getState() + .metadata() + .index(indices.get(0)) + .mapping() + .sourceAsMap() + .get("properties"); + + assertThat(fieldMappings.size(), is(10000)); + }); + Collections.sort(latencies); + + logger.info("cluster().state() fetch with Term Version enabled took {} milliseconds", (latencies.get(latencies.size() / 2))); + } + // cluster state response time with term check disabled on datanode + isTermVersionCheckEnabled.set(false); + { + List latencies = new ArrayList<>(); + IntStream.range(0, 50).forEachOrdered(n1 -> { + ClusterStateRequest clusterStateRequest = new ClusterStateRequest(); + long start = System.currentTimeMillis(); + ClusterStateResponse stateResponse = dataNodeClient().admin().cluster().state(clusterStateRequest).actionGet(); + latencies.add(System.currentTimeMillis() - start); + assertThat(stateResponse.getClusterName().value(), is(internalCluster().getClusterName())); + assertThat(stateResponse.getState().nodes().getSize(), is(internalCluster().getNodeNames().length)); + assertThat(stateResponse.getState().metadata().indices().size(), is(indices.size())); + Map typeProperties = (Map) stateResponse.getState() + .metadata() + .index(indices.get(0)) + .mapping() + .sourceAsMap() + .get("properties"); + assertThat(typeProperties.size(), is(10000)); + + }); + Collections.sort(latencies); + logger.info("cluster().state() fetch with Term Version disabled took {} milliseconds", (latencies.get(latencies.size() / 2))); + } + + } + + private void stubClusterTermResponse(String master) { + MockTransportService primaryService = (MockTransportService) internalCluster().getInstance(TransportService.class, master); + primaryService.addRequestHandlingBehavior(GetTermVersionAction.NAME, (handler, request, channel, task) -> { + if (isTermVersionCheckEnabled.get()) { + handler.messageReceived(request, channel, task); + } else { + // always return response that does not match + channel.sendResponse(new GetTermVersionResponse(new ClusterStateTermVersion(new ClusterName("test"), "1", -1, -1))); + } + }); + } +} diff --git a/server/src/main/java/org/opensearch/action/ActionModule.java b/server/src/main/java/org/opensearch/action/ActionModule.java index b19bf9590f43b..f827b7f3f0097 100644 --- a/server/src/main/java/org/opensearch/action/ActionModule.java +++ b/server/src/main/java/org/opensearch/action/ActionModule.java @@ -107,6 +107,8 @@ import org.opensearch.action.admin.cluster.snapshots.status.TransportSnapshotsStatusAction; import org.opensearch.action.admin.cluster.state.ClusterStateAction; import org.opensearch.action.admin.cluster.state.TransportClusterStateAction; +import org.opensearch.action.admin.cluster.state.term.GetTermVersionAction; +import org.opensearch.action.admin.cluster.state.term.TransportGetTermVersionAction; import org.opensearch.action.admin.cluster.stats.ClusterStatsAction; import org.opensearch.action.admin.cluster.stats.TransportClusterStatsAction; import org.opensearch.action.admin.cluster.storedscripts.DeleteStoredScriptAction; @@ -614,6 +616,7 @@ public void reg actions.register(ClusterAllocationExplainAction.INSTANCE, TransportClusterAllocationExplainAction.class); actions.register(ClusterStatsAction.INSTANCE, TransportClusterStatsAction.class); actions.register(ClusterStateAction.INSTANCE, TransportClusterStateAction.class); + actions.register(GetTermVersionAction.INSTANCE, TransportGetTermVersionAction.class); actions.register(ClusterHealthAction.INSTANCE, TransportClusterHealthAction.class); actions.register(ClusterUpdateSettingsAction.INSTANCE, TransportClusterUpdateSettingsAction.class); actions.register(ClusterRerouteAction.INSTANCE, TransportClusterRerouteAction.class); diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/state/TransportClusterStateAction.java b/server/src/main/java/org/opensearch/action/admin/cluster/state/TransportClusterStateAction.java index 4aaa7f1950823..cae465a90446e 100644 --- a/server/src/main/java/org/opensearch/action/admin/cluster/state/TransportClusterStateAction.java +++ b/server/src/main/java/org/opensearch/action/admin/cluster/state/TransportClusterStateAction.java @@ -125,9 +125,12 @@ protected void clusterManagerOperation( ? clusterState -> true : clusterState -> clusterState.metadata().version() >= request.waitForMetadataVersion(); + // action will be executed on local node, if either the request is local only (or) the local node has the same cluster-state as + // ClusterManager final Predicate acceptableClusterStateOrNotMasterPredicate = request.local() - ? acceptableClusterStatePredicate - : acceptableClusterStatePredicate.or(clusterState -> clusterState.nodes().isLocalNodeElectedClusterManager() == false); + || !state.nodes().isLocalNodeElectedClusterManager() + ? acceptableClusterStatePredicate + : acceptableClusterStatePredicate.or(clusterState -> clusterState.nodes().isLocalNodeElectedClusterManager() == false); if (acceptableClusterStatePredicate.test(state)) { ActionListener.completeWith(listener, () -> buildResponse(request, state)); @@ -231,4 +234,8 @@ private ClusterStateResponse buildResponse(final ClusterStateRequest request, fi return new ClusterStateResponse(currentState.getClusterName(), builder.build(), false); } + @Override + protected boolean localExecuteSupportedByAction() { + return true; + } } diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/state/term/GetTermVersionAction.java b/server/src/main/java/org/opensearch/action/admin/cluster/state/term/GetTermVersionAction.java new file mode 100644 index 0000000000000..3344fd549b23f --- /dev/null +++ b/server/src/main/java/org/opensearch/action/admin/cluster/state/term/GetTermVersionAction.java @@ -0,0 +1,26 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.action.admin.cluster.state.term; + +import org.opensearch.action.ActionType; + +/** + * Transport action for fetching cluster term and version + * + * @opensearch.internal + */ +public class GetTermVersionAction extends ActionType { + + public static final GetTermVersionAction INSTANCE = new GetTermVersionAction(); + public static final String NAME = "cluster:monitor/term"; + + private GetTermVersionAction() { + super(NAME, GetTermVersionResponse::new); + } +} diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/state/term/GetTermVersionRequest.java b/server/src/main/java/org/opensearch/action/admin/cluster/state/term/GetTermVersionRequest.java new file mode 100644 index 0000000000000..b099f8087bd15 --- /dev/null +++ b/server/src/main/java/org/opensearch/action/admin/cluster/state/term/GetTermVersionRequest.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.action.admin.cluster.state.term; + +import org.opensearch.action.ActionRequestValidationException; +import org.opensearch.action.support.clustermanager.ClusterManagerNodeReadRequest; +import org.opensearch.core.common.io.stream.StreamInput; + +import java.io.IOException; + +/** + * Request object to get cluster term and version + * + * @opensearch.internal + */ +public class GetTermVersionRequest extends ClusterManagerNodeReadRequest { + + public GetTermVersionRequest() {} + + public GetTermVersionRequest(StreamInput in) throws IOException { + super(in); + } + + @Override + public ActionRequestValidationException validate() { + return null; + } +} diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/state/term/GetTermVersionResponse.java b/server/src/main/java/org/opensearch/action/admin/cluster/state/term/GetTermVersionResponse.java new file mode 100644 index 0000000000000..16b355a80d1f2 --- /dev/null +++ b/server/src/main/java/org/opensearch/action/admin/cluster/state/term/GetTermVersionResponse.java @@ -0,0 +1,50 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.action.admin.cluster.state.term; + +import org.opensearch.cluster.ClusterState; +import org.opensearch.cluster.coordination.ClusterStateTermVersion; +import org.opensearch.core.action.ActionResponse; +import org.opensearch.core.common.io.stream.StreamInput; +import org.opensearch.core.common.io.stream.StreamOutput; + +import java.io.IOException; + +/** + * Response object of cluster term + * + * @opensearch.internal + */ +public class GetTermVersionResponse extends ActionResponse { + + private final ClusterStateTermVersion clusterStateTermVersion; + + public GetTermVersionResponse(ClusterStateTermVersion clusterStateTermVersion) { + this.clusterStateTermVersion = clusterStateTermVersion; + } + + public GetTermVersionResponse(StreamInput in) throws IOException { + super(in); + this.clusterStateTermVersion = new ClusterStateTermVersion(in); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + clusterStateTermVersion.writeTo(out); + } + + public ClusterStateTermVersion getClusterStateTermVersion() { + return clusterStateTermVersion; + } + + public boolean matches(ClusterState clusterState) { + return clusterStateTermVersion != null && clusterStateTermVersion.equals(new ClusterStateTermVersion(clusterState)); + } + +} diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/state/term/TransportGetTermVersionAction.java b/server/src/main/java/org/opensearch/action/admin/cluster/state/term/TransportGetTermVersionAction.java new file mode 100644 index 0000000000000..88305252aa99c --- /dev/null +++ b/server/src/main/java/org/opensearch/action/admin/cluster/state/term/TransportGetTermVersionAction.java @@ -0,0 +1,85 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.action.admin.cluster.state.term; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.opensearch.action.support.ActionFilters; +import org.opensearch.action.support.clustermanager.TransportClusterManagerNodeReadAction; +import org.opensearch.cluster.ClusterState; +import org.opensearch.cluster.block.ClusterBlockException; +import org.opensearch.cluster.coordination.ClusterStateTermVersion; +import org.opensearch.cluster.metadata.IndexNameExpressionResolver; +import org.opensearch.cluster.service.ClusterService; +import org.opensearch.common.inject.Inject; +import org.opensearch.core.action.ActionListener; +import org.opensearch.core.common.io.stream.StreamInput; +import org.opensearch.threadpool.ThreadPool; +import org.opensearch.transport.TransportService; + +import java.io.IOException; + +/** + * Transport action for obtaining cluster term and version from cluster-manager + * + * @opensearch.internal + */ +public class TransportGetTermVersionAction extends TransportClusterManagerNodeReadAction { + + private final Logger logger = LogManager.getLogger(getClass()); + + @Inject + public TransportGetTermVersionAction( + TransportService transportService, + ClusterService clusterService, + ThreadPool threadPool, + ActionFilters actionFilters, + IndexNameExpressionResolver indexNameExpressionResolver + ) { + super( + GetTermVersionAction.NAME, + false, + transportService, + clusterService, + threadPool, + actionFilters, + GetTermVersionRequest::new, + indexNameExpressionResolver + ); + } + + @Override + protected String executor() { + return ThreadPool.Names.SAME; + } + + @Override + public GetTermVersionResponse read(StreamInput in) throws IOException { + return new GetTermVersionResponse(in); + } + + @Override + protected ClusterBlockException checkBlock(GetTermVersionRequest request, ClusterState state) { + // cluster state term and version needs to be retrieved even on a fully blocked cluster + return null; + } + + @Override + protected void clusterManagerOperation( + GetTermVersionRequest request, + ClusterState state, + ActionListener listener + ) throws Exception { + ActionListener.completeWith(listener, () -> buildResponse(request, state)); + } + + private GetTermVersionResponse buildResponse(GetTermVersionRequest request, ClusterState state) { + return new GetTermVersionResponse(new ClusterStateTermVersion(state)); + } +} diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/state/term/package-info.java b/server/src/main/java/org/opensearch/action/admin/cluster/state/term/package-info.java new file mode 100644 index 0000000000000..0ee559c527d7d --- /dev/null +++ b/server/src/main/java/org/opensearch/action/admin/cluster/state/term/package-info.java @@ -0,0 +1,10 @@ +/* + * 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. + */ + +/** Cluster Term transport handler. */ +package org.opensearch.action.admin.cluster.state.term; diff --git a/server/src/main/java/org/opensearch/action/support/clustermanager/TransportClusterManagerNodeAction.java b/server/src/main/java/org/opensearch/action/support/clustermanager/TransportClusterManagerNodeAction.java index 536ddcdd402e2..6a081f9dfecde 100644 --- a/server/src/main/java/org/opensearch/action/support/clustermanager/TransportClusterManagerNodeAction.java +++ b/server/src/main/java/org/opensearch/action/support/clustermanager/TransportClusterManagerNodeAction.java @@ -37,6 +37,9 @@ import org.apache.logging.log4j.message.ParameterizedMessage; import org.opensearch.action.ActionListenerResponseHandler; import org.opensearch.action.ActionRunnable; +import org.opensearch.action.admin.cluster.state.term.GetTermVersionAction; +import org.opensearch.action.admin.cluster.state.term.GetTermVersionRequest; +import org.opensearch.action.admin.cluster.state.term.GetTermVersionResponse; import org.opensearch.action.bulk.BackoffPolicy; import org.opensearch.action.support.ActionFilters; import org.opensearch.action.support.HandledTransportAction; @@ -66,11 +69,16 @@ import org.opensearch.transport.ConnectTransportException; import org.opensearch.transport.RemoteTransportException; import org.opensearch.transport.TransportException; +import org.opensearch.transport.TransportResponseHandler; import org.opensearch.transport.TransportService; import java.io.IOException; +import java.util.function.BiConsumer; +import java.util.function.Consumer; import java.util.function.Predicate; +import static org.opensearch.Version.V_3_0_0; + /** * A base class for operations that needs to be performed on the cluster-manager node. * @@ -252,23 +260,13 @@ protected void doStart(ClusterState clusterState) { }); } } else { - ActionListener delegate = ActionListener.delegateResponse(listener, (delegatedListener, t) -> { - if (t instanceof FailedToCommitClusterStateException || t instanceof NotClusterManagerException) { - logger.debug( - () -> new ParameterizedMessage( - "master could not publish cluster state or " - + "stepped down before publishing action [{}], scheduling a retry", - actionName - ), - t - ); - retryOnMasterChange(clusterState, t); - } else { - delegatedListener.onFailure(t); - } - }); threadPool.executor(executor) - .execute(ActionRunnable.wrap(delegate, l -> clusterManagerOperation(task, request, clusterState, l))); + .execute( + ActionRunnable.wrap( + getDelegateForLocalExecute(clusterState), + l -> clusterManagerOperation(task, request, clusterState, l) + ) + ); } } else { if (nodes.getClusterManagerNode() == null) { @@ -276,32 +274,15 @@ protected void doStart(ClusterState clusterState) { retryOnMasterChange(clusterState, null); } else { DiscoveryNode clusterManagerNode = nodes.getClusterManagerNode(); - final String actionName = getClusterManagerActionName(clusterManagerNode); - transportService.sendRequest( - clusterManagerNode, - actionName, - request, - new ActionListenerResponseHandler(listener, TransportClusterManagerNodeAction.this::read) { - @Override - public void handleException(final TransportException exp) { - Throwable cause = exp.unwrapCause(); - if (cause instanceof ConnectTransportException - || (exp instanceof RemoteTransportException && cause instanceof NodeClosedException)) { - // we want to retry here a bit to see if a new cluster-manager is elected - logger.debug( - "connection exception while trying to forward request with action name [{}] to " - + "master node [{}], scheduling a retry. Error: [{}]", - actionName, - nodes.getClusterManagerNode(), - exp.getDetailedMessage() - ); - retryOnMasterChange(clusterState, cause); - } else { - listener.onFailure(exp); - } - } - } - ); + if (clusterManagerNode.getVersion().onOrAfter(V_3_0_0) && localExecuteSupportedByAction()) { + BiConsumer executeOnLocalOrClusterManager = clusterStateLatestChecker( + this::executeOnLocalNode, + this::executeOnClusterManager + ); + executeOnLocalOrClusterManager.accept(clusterManagerNode, clusterState); + } else { + executeOnClusterManager(clusterManagerNode, clusterState); + } } } } catch (Exception e) { @@ -351,6 +332,114 @@ public void onTimeout(TimeValue timeout) { } }, statePredicate); } + + private ActionListener getDelegateForLocalExecute(ClusterState clusterState) { + return ActionListener.delegateResponse(listener, (delegatedListener, t) -> { + if (t instanceof FailedToCommitClusterStateException || t instanceof NotClusterManagerException) { + logger.debug( + () -> new ParameterizedMessage( + "cluster-manager could not publish cluster state or " + + "stepped down before publishing action [{}], scheduling a retry", + actionName + ), + t + ); + + retryOnMasterChange(clusterState, t); + } else { + delegatedListener.onFailure(t); + } + }); + } + + protected BiConsumer clusterStateLatestChecker( + Consumer onLatestLocalState, + BiConsumer onStaleLocalState + ) { + return (clusterManagerNode, clusterState) -> { + transportService.sendRequest( + clusterManagerNode, + GetTermVersionAction.NAME, + new GetTermVersionRequest(), + new TransportResponseHandler() { + @Override + public void handleResponse(GetTermVersionResponse response) { + boolean isLatestClusterStatePresentOnLocalNode = response.matches(clusterState); + logger.trace( + "Received GetTermVersionResponse response : ClusterStateTermVersion {}, latest-on-local {}", + response.getClusterStateTermVersion(), + isLatestClusterStatePresentOnLocalNode + ); + if (isLatestClusterStatePresentOnLocalNode) { + onLatestLocalState.accept(clusterState); + } else { + onStaleLocalState.accept(clusterManagerNode, clusterState); + } + } + + @Override + public void handleException(TransportException exp) { + handleTransportException(clusterManagerNode, clusterState, exp); + } + + @Override + public String executor() { + return ThreadPool.Names.SAME; + } + + @Override + public GetTermVersionResponse read(StreamInput in) throws IOException { + return new GetTermVersionResponse(in); + } + + } + ); + }; + } + + private void executeOnLocalNode(ClusterState localClusterState) { + Runnable runTask = ActionRunnable.wrap( + getDelegateForLocalExecute(localClusterState), + l -> clusterManagerOperation(task, request, localClusterState, l) + ); + threadPool.executor(executor).execute(runTask); + } + + private void executeOnClusterManager(DiscoveryNode clusterManagerNode, ClusterState clusterState) { + final String actionName = getClusterManagerActionName(clusterManagerNode); + + transportService.sendRequest( + clusterManagerNode, + actionName, + request, + new ActionListenerResponseHandler(listener, TransportClusterManagerNodeAction.this::read) { + @Override + public void handleException(final TransportException exp) { + handleTransportException(clusterManagerNode, clusterState, exp); + } + } + ); + } + + private void handleTransportException(DiscoveryNode clusterManagerNode, ClusterState clusterState, final TransportException exp) { + Throwable cause = exp.unwrapCause(); + if (cause instanceof ConnectTransportException + || (exp instanceof RemoteTransportException && cause instanceof NodeClosedException)) { + // we want to retry here a bit to see if a new cluster-manager is elected + + logger.debug( + "connection exception while trying to forward request with action name [{}] to " + + "master node [{}], scheduling a retry. Error: [{}]", + actionName, + clusterManagerNode, + exp.getDetailedMessage() + ); + + retryOnMasterChange(clusterState, cause); + } else { + listener.onFailure(exp); + } + } } /** @@ -372,4 +461,14 @@ protected String getMasterActionName(DiscoveryNode node) { return getClusterManagerActionName(node); } + /** + * Override to true if the transport action can be executed locally and need NOT be executed always on cluster-manager (Read actions). + * The action is executed locally if this method returns true AND + * the ClusterState on local node is in-sync with ClusterManager. + * + * @return - boolean if the action can be run locally + */ + protected boolean localExecuteSupportedByAction() { + return false; + } } diff --git a/server/src/main/java/org/opensearch/cluster/coordination/ClusterStateTermVersion.java b/server/src/main/java/org/opensearch/cluster/coordination/ClusterStateTermVersion.java new file mode 100644 index 0000000000000..b317b0d362825 --- /dev/null +++ b/server/src/main/java/org/opensearch/cluster/coordination/ClusterStateTermVersion.java @@ -0,0 +1,110 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.cluster.coordination; + +import org.opensearch.cluster.ClusterName; +import org.opensearch.cluster.ClusterState; +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 java.io.IOException; + +/** + * Identifies a specific version of ClusterState at a node. + */ +public class ClusterStateTermVersion implements Writeable { + + private final ClusterName clusterName; + private final String clusterUUID; + private final long term; + private final long version; + + public ClusterStateTermVersion(ClusterName clusterName, String clusterUUID, long term, long version) { + this.clusterName = clusterName; + this.clusterUUID = clusterUUID; + this.term = term; + this.version = version; + } + + public ClusterStateTermVersion(StreamInput in) throws IOException { + this.clusterName = new ClusterName(in); + this.clusterUUID = in.readString(); + this.term = in.readLong(); + this.version = in.readLong(); + } + + public ClusterStateTermVersion(ClusterState state) { + this.clusterName = state.getClusterName(); + this.clusterUUID = state.metadata().clusterUUID(); + this.term = state.term(); + this.version = state.version(); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + clusterName.writeTo(out); + out.writeString(clusterUUID); + out.writeLong(term); + out.writeLong(version); + } + + public ClusterName getClusterName() { + return clusterName; + } + + public String getClusterUUID() { + return clusterUUID; + } + + public long getTerm() { + return term; + } + + public long getVersion() { + return version; + } + + @Override + public String toString() { + return "ClusterStateTermVersion{" + + "clusterName=" + + clusterName + + ", clusterUUID='" + + clusterUUID + + '\'' + + ", term=" + + term + + ", version=" + + version + + '}'; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + ClusterStateTermVersion that = (ClusterStateTermVersion) o; + + if (term != that.term) return false; + if (version != that.version) return false; + if (!clusterName.equals(that.clusterName)) return false; + return clusterUUID.equals(that.clusterUUID); + } + + @Override + public int hashCode() { + int result = clusterName.hashCode(); + result = 31 * result + clusterUUID.hashCode(); + result = 31 * result + (int) (term ^ (term >>> 32)); + result = 31 * result + (int) (version ^ (version >>> 32)); + return result; + } +} diff --git a/server/src/test/java/org/opensearch/action/admin/cluster/state/term/ClusterTermVersionIT.java b/server/src/test/java/org/opensearch/action/admin/cluster/state/term/ClusterTermVersionIT.java new file mode 100644 index 0000000000000..fa2a6121af349 --- /dev/null +++ b/server/src/test/java/org/opensearch/action/admin/cluster/state/term/ClusterTermVersionIT.java @@ -0,0 +1,121 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.action.admin.cluster.state.term; + +import org.opensearch.action.admin.cluster.state.ClusterStateAction; +import org.opensearch.action.admin.cluster.state.ClusterStateRequest; +import org.opensearch.action.admin.cluster.state.ClusterStateResponse; +import org.opensearch.cluster.ClusterName; +import org.opensearch.cluster.coordination.ClusterStateTermVersion; +import org.opensearch.common.unit.TimeValue; +import org.opensearch.plugins.Plugin; +import org.opensearch.test.OpenSearchIntegTestCase; +import org.opensearch.test.transport.MockTransportService; +import org.opensearch.transport.TransportService; + +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.concurrent.atomic.AtomicInteger; + +import static org.hamcrest.Matchers.is; + +@OpenSearchIntegTestCase.ClusterScope(scope = OpenSearchIntegTestCase.Scope.TEST, numDataNodes = 0) +public class ClusterTermVersionIT extends OpenSearchIntegTestCase { + + @Override + protected Collection> nodePlugins() { + return List.of(MockTransportService.TestPlugin.class); + } + + public void testClusterStateResponseFromDataNode() throws Exception { + internalCluster().startClusterManagerOnlyNode(); + internalCluster().startDataOnlyNode(); + + ensureClusterSizeConsistency(); + ensureGreen(); + + ClusterStateRequest clusterStateRequest = new ClusterStateRequest(); + clusterStateRequest.waitForTimeout(TimeValue.timeValueHours(1)); + ClusterStateResponse stateResponse = dataNodeClient().admin().cluster().state(clusterStateRequest).get(); + assertThat(stateResponse.getClusterName().value(), is(internalCluster().getClusterName())); + assertThat(stateResponse.getState().nodes().getSize(), is(internalCluster().getNodeNames().length)); + assertThat(stateResponse.isWaitForTimedOut(), is(false)); + + } + + public void testClusterStateResponseFromClusterManagerNode() throws Exception { + String master = internalCluster().startClusterManagerOnlyNode(); + String data = internalCluster().startDataOnlyNode(); + ensureClusterSizeConsistency(); + ensureGreen(); + Map callCounters = Map.ofEntries( + Map.entry(ClusterStateAction.NAME, new AtomicInteger()), + Map.entry(GetTermVersionAction.NAME, new AtomicInteger()) + ); + + addCallCountInterceptor(master, callCounters); + + ClusterStateResponse stateResponse = dataNodeClient().admin().cluster().state(new ClusterStateRequest()).get(); + + AtomicInteger clusterStateCallsOnMaster = callCounters.get(ClusterStateAction.NAME); + AtomicInteger termCallsOnMaster = callCounters.get(GetTermVersionAction.NAME); + + assertThat(clusterStateCallsOnMaster.get(), is(0)); + assertThat(termCallsOnMaster.get(), is(1)); + + assertThat(stateResponse.getClusterName().value(), is(internalCluster().getClusterName())); + assertThat(stateResponse.getState().nodes().getSize(), is(internalCluster().getNodeNames().length)); + + } + + public void testDatanodeOutOfSync() throws Exception { + String master = internalCluster().startClusterManagerOnlyNode(); + String data = internalCluster().startDataOnlyNode(); + ensureClusterSizeConsistency(); + ensureGreen(); + Map callCounters = Map.ofEntries( + Map.entry(ClusterStateAction.NAME, new AtomicInteger()), + Map.entry(GetTermVersionAction.NAME, new AtomicInteger()) + ); + + stubClusterTermResponse(master); + addCallCountInterceptor(master, callCounters); + + ClusterStateResponse stateResponse = dataNodeClient().admin().cluster().state(new ClusterStateRequest()).get(); + + AtomicInteger clusterStateCallsOnMaster = callCounters.get(ClusterStateAction.NAME); + AtomicInteger termCallsOnMaster = callCounters.get(GetTermVersionAction.NAME); + + assertThat(clusterStateCallsOnMaster.get(), is(1)); + assertThat(termCallsOnMaster.get(), is(1)); + + assertThat(stateResponse.getClusterName().value(), is(internalCluster().getClusterName())); + assertThat(stateResponse.getState().nodes().getSize(), is(internalCluster().getNodeNames().length)); + } + + private void addCallCountInterceptor(String nodeName, Map callCounters) { + MockTransportService primaryService = (MockTransportService) internalCluster().getInstance(TransportService.class, nodeName); + for (var ctrEnty : callCounters.entrySet()) { + primaryService.addRequestHandlingBehavior(ctrEnty.getKey(), (handler, request, channel, task) -> { + ctrEnty.getValue().incrementAndGet(); + logger.info("--> {} response redirect", ClusterStateAction.NAME); + handler.messageReceived(request, channel, task); + }); + } + } + + private void stubClusterTermResponse(String master) { + MockTransportService primaryService = (MockTransportService) internalCluster().getInstance(TransportService.class, master); + primaryService.addRequestHandlingBehavior(GetTermVersionAction.NAME, (handler, request, channel, task) -> { + channel.sendResponse(new GetTermVersionResponse(new ClusterStateTermVersion(new ClusterName("test"), "1", -1, -1))); + }); + } + +} diff --git a/server/src/test/java/org/opensearch/action/admin/cluster/state/term/ClusterTermVersionTests.java b/server/src/test/java/org/opensearch/action/admin/cluster/state/term/ClusterTermVersionTests.java new file mode 100644 index 0000000000000..22d9623eebdbe --- /dev/null +++ b/server/src/test/java/org/opensearch/action/admin/cluster/state/term/ClusterTermVersionTests.java @@ -0,0 +1,26 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.action.admin.cluster.state.term; + +import org.opensearch.cluster.service.ClusterService; +import org.opensearch.test.OpenSearchSingleNodeTestCase; + +import java.util.concurrent.ExecutionException; + +public class ClusterTermVersionTests extends OpenSearchSingleNodeTestCase { + + public void testTransportTermResponse() throws ExecutionException, InterruptedException { + GetTermVersionRequest request = new GetTermVersionRequest(); + GetTermVersionResponse resp = client().execute(GetTermVersionAction.INSTANCE, request).get(); + + final ClusterService clusterService = getInstanceFromNode(ClusterService.class); + + assertTrue(resp.matches(clusterService.state())); + } +} diff --git a/server/src/test/java/org/opensearch/action/support/clustermanager/TransportClusterManagerNodeActionTests.java b/server/src/test/java/org/opensearch/action/support/clustermanager/TransportClusterManagerNodeActionTests.java index 9ae1310a8b15c..538416e1137f5 100644 --- a/server/src/test/java/org/opensearch/action/support/clustermanager/TransportClusterManagerNodeActionTests.java +++ b/server/src/test/java/org/opensearch/action/support/clustermanager/TransportClusterManagerNodeActionTests.java @@ -6,24 +6,6 @@ * compatible open source license. */ -/* - * Licensed to Elasticsearch under one or more contributor - * license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright - * ownership. Elasticsearch licenses this file to you under - * the Apache License, Version 2.0 (the "License"); you may - * not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ /* * Modifications Copyright OpenSearch Contributors. See * GitHub history for details. diff --git a/server/src/test/java/org/opensearch/action/support/clustermanager/TransportClusterManagerTermCheckTests.java b/server/src/test/java/org/opensearch/action/support/clustermanager/TransportClusterManagerTermCheckTests.java new file mode 100644 index 0000000000000..8c7b7a0940c82 --- /dev/null +++ b/server/src/test/java/org/opensearch/action/support/clustermanager/TransportClusterManagerTermCheckTests.java @@ -0,0 +1,320 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +/* + * Modifications Copyright OpenSearch Contributors. See + * GitHub history for details. + */ + +package org.opensearch.action.support.clustermanager; + +import org.opensearch.Version; +import org.opensearch.action.ActionRequestValidationException; +import org.opensearch.action.admin.cluster.state.term.GetTermVersionResponse; +import org.opensearch.action.support.ActionFilters; +import org.opensearch.action.support.PlainActionFuture; +import org.opensearch.action.support.ThreadedActionListener; +import org.opensearch.action.support.replication.ClusterStateCreationUtils; +import org.opensearch.cluster.ClusterState; +import org.opensearch.cluster.block.ClusterBlockException; +import org.opensearch.cluster.coordination.ClusterStateTermVersion; +import org.opensearch.cluster.metadata.IndexNameExpressionResolver; +import org.opensearch.cluster.node.DiscoveryNode; +import org.opensearch.cluster.node.DiscoveryNodeRole; +import org.opensearch.cluster.service.ClusterService; +import org.opensearch.common.settings.Settings; +import org.opensearch.common.util.concurrent.ThreadContext; +import org.opensearch.core.action.ActionListener; +import org.opensearch.core.action.ActionResponse; +import org.opensearch.core.common.io.stream.StreamInput; +import org.opensearch.core.common.io.stream.StreamOutput; +import org.opensearch.tasks.Task; +import org.opensearch.telemetry.tracing.noop.NoopTracer; +import org.opensearch.test.OpenSearchTestCase; +import org.opensearch.test.transport.CapturingTransport; +import org.opensearch.threadpool.TestThreadPool; +import org.opensearch.threadpool.ThreadPool; +import org.opensearch.transport.TransportService; +import org.junit.After; +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.BeforeClass; + +import java.io.IOException; +import java.util.Collections; +import java.util.HashSet; +import java.util.Objects; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; + +import static org.opensearch.test.ClusterServiceUtils.createClusterService; +import static org.opensearch.test.ClusterServiceUtils.setState; +import static org.hamcrest.Matchers.equalTo; + +public class TransportClusterManagerTermCheckTests extends OpenSearchTestCase { + private static ThreadPool threadPool; + + private ClusterService clusterService; + private TransportService transportService; + private CapturingTransport transport; + private DiscoveryNode localNode; + private DiscoveryNode remoteNode; + private DiscoveryNode[] allNodes; + + @BeforeClass + public static void beforeClass() { + threadPool = new TestThreadPool("TransportMasterNodeActionTests"); + } + + @Override + @Before + public void setUp() throws Exception { + super.setUp(); + transport = new CapturingTransport(); + clusterService = createClusterService(threadPool); + transportService = transport.createTransportService( + clusterService.getSettings(), + threadPool, + TransportService.NOOP_TRANSPORT_INTERCEPTOR, + x -> clusterService.localNode(), + null, + Collections.emptySet(), + NoopTracer.INSTANCE + ); + transportService.start(); + transportService.acceptIncomingRequests(); + + } + + @After + public void tearDown() throws Exception { + super.tearDown(); + clusterService.close(); + transportService.close(); + } + + @AfterClass + public static void afterClass() { + ThreadPool.terminate(threadPool, 30, TimeUnit.SECONDS); + threadPool = null; + } + + public static class Request extends ClusterManagerNodeRequest { + Request() {} + + Request(StreamInput in) throws IOException { + super(in); + } + + @Override + public ActionRequestValidationException validate() { + return null; + } + } + + class Response extends ActionResponse { + private long identity = randomLong(); + + Response() {} + + Response(StreamInput in) throws IOException { + super(in); + identity = in.readLong(); + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + Response response = (Response) o; + return identity == response.identity; + } + + @Override + public int hashCode() { + return Objects.hash(identity); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeLong(identity); + } + } + + class Action extends TransportClusterManagerNodeAction { + Action(String actionName, TransportService transportService, ClusterService clusterService, ThreadPool threadPool) { + super( + actionName, + transportService, + clusterService, + threadPool, + new ActionFilters(new HashSet<>()), + Request::new, + new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY)) + ); + } + + @Override + protected void doExecute(Task task, final Request request, ActionListener listener) { + // remove unneeded threading by wrapping listener with SAME to prevent super.doExecute from wrapping it with LISTENER + super.doExecute(task, request, new ThreadedActionListener<>(logger, threadPool, ThreadPool.Names.SAME, listener, false)); + } + + @Override + protected String executor() { + // very lightweight operation in memory, no need to fork to a thread + return ThreadPool.Names.SAME; + } + + @Override + protected boolean localExecuteSupportedByAction() { + return true; + } + + @Override + protected Response read(StreamInput in) throws IOException { + return new Response(in); + } + + @Override + protected void clusterManagerOperation(Request request, ClusterState state, ActionListener listener) throws Exception { + listener.onResponse(new Response()); // default implementation, overridden in specific tests + } + + @Override + protected ClusterBlockException checkBlock(Request request, ClusterState state) { + return null; // default implementation, overridden in specific tests + } + } + + public void testTermCheckMatchWithClusterManager() throws ExecutionException, InterruptedException { + setUpCluster(Version.CURRENT); + + TransportClusterManagerTermCheckTests.Request request = new TransportClusterManagerTermCheckTests.Request(); + PlainActionFuture listener = new PlainActionFuture<>(); + new TransportClusterManagerTermCheckTests.Action("internal:testAction", transportService, clusterService, threadPool).execute( + request, + listener + ); + + assertThat(transport.capturedRequests().length, equalTo(1)); + CapturingTransport.CapturedRequest capturedRequest = transport.capturedRequests()[0]; + assertTrue(capturedRequest.node.isClusterManagerNode()); + assertThat(capturedRequest.action, equalTo("cluster:monitor/term")); + GetTermVersionResponse response = new GetTermVersionResponse( + new ClusterStateTermVersion( + clusterService.state().getClusterName(), + clusterService.state().metadata().clusterUUID(), + clusterService.state().term(), + clusterService.state().version() + ) + ); + transport.handleResponse(capturedRequest.requestId, response); + assertTrue(listener.isDone()); + } + + public void testTermCheckNoMatchWithClusterManager() throws ExecutionException, InterruptedException { + setUpCluster(Version.CURRENT); + TransportClusterManagerTermCheckTests.Request request = new TransportClusterManagerTermCheckTests.Request(); + + PlainActionFuture listener = new PlainActionFuture<>(); + new TransportClusterManagerTermCheckTests.Action("internal:testAction", transportService, clusterService, threadPool).execute( + request, + listener + ); + + assertThat(transport.capturedRequests().length, equalTo(1)); + CapturingTransport.CapturedRequest termCheckRequest = transport.capturedRequests()[0]; + assertTrue(termCheckRequest.node.isClusterManagerNode()); + assertThat(termCheckRequest.action, equalTo("cluster:monitor/term")); + GetTermVersionResponse noMatchResponse = new GetTermVersionResponse( + new ClusterStateTermVersion( + clusterService.state().getClusterName(), + clusterService.state().metadata().clusterUUID(), + clusterService.state().term(), + clusterService.state().version() - 1 + ) + ); + transport.handleResponse(termCheckRequest.requestId, noMatchResponse); + assertFalse(listener.isDone()); + + assertThat(transport.capturedRequests().length, equalTo(2)); + CapturingTransport.CapturedRequest capturedRequest = transport.capturedRequests()[1]; + assertTrue(capturedRequest.node.isClusterManagerNode()); + assertThat(capturedRequest.request, equalTo(request)); + assertThat(capturedRequest.action, equalTo("internal:testAction")); + + TransportClusterManagerTermCheckTests.Response response = new TransportClusterManagerTermCheckTests.Response(); + transport.handleResponse(capturedRequest.requestId, response); + assertTrue(listener.isDone()); + assertThat(listener.get(), equalTo(response)); + + } + + public void testTermCheckOnOldVersionClusterManager() throws ExecutionException, InterruptedException { + + setUpCluster(Version.V_2_12_0); + TransportClusterManagerTermCheckTests.Request request = new TransportClusterManagerTermCheckTests.Request(); + + PlainActionFuture listener = new PlainActionFuture<>(); + new TransportClusterManagerTermCheckTests.Action("internal:testAction", transportService, clusterService, threadPool).execute( + request, + listener + ); + + assertThat(transport.capturedRequests().length, equalTo(1)); + CapturingTransport.CapturedRequest capturedRequest = transport.capturedRequests()[0]; + assertTrue(capturedRequest.node.isClusterManagerNode()); + assertThat(capturedRequest.request, equalTo(request)); + assertThat(capturedRequest.action, equalTo("internal:testAction")); + + TransportClusterManagerTermCheckTests.Response response = new TransportClusterManagerTermCheckTests.Response(); + transport.handleResponse(capturedRequest.requestId, response); + assertTrue(listener.isDone()); + assertThat(listener.get(), equalTo(response)); + + } + + private void setUpCluster(Version clusterManagerVersion) { + localNode = new DiscoveryNode( + "local_node", + buildNewFakeTransportAddress(), + Collections.emptyMap(), + Collections.singleton(DiscoveryNodeRole.DATA_ROLE), + Version.CURRENT + ); + remoteNode = new DiscoveryNode( + "remote_node", + buildNewFakeTransportAddress(), + Collections.emptyMap(), + Collections.singleton(DiscoveryNodeRole.CLUSTER_MANAGER_ROLE), + clusterManagerVersion + ); + allNodes = new DiscoveryNode[] { localNode, remoteNode }; + setState(clusterService, ClusterStateCreationUtils.state(localNode, remoteNode, allNodes)); + + } +} diff --git a/server/src/test/java/org/opensearch/snapshots/SnapshotResiliencyTests.java b/server/src/test/java/org/opensearch/snapshots/SnapshotResiliencyTests.java index 635939e68de71..58315ba031b84 100644 --- a/server/src/test/java/org/opensearch/snapshots/SnapshotResiliencyTests.java +++ b/server/src/test/java/org/opensearch/snapshots/SnapshotResiliencyTests.java @@ -66,6 +66,8 @@ import org.opensearch.action.admin.cluster.state.ClusterStateRequest; import org.opensearch.action.admin.cluster.state.ClusterStateResponse; import org.opensearch.action.admin.cluster.state.TransportClusterStateAction; +import org.opensearch.action.admin.cluster.state.term.GetTermVersionAction; +import org.opensearch.action.admin.cluster.state.term.TransportGetTermVersionAction; import org.opensearch.action.admin.indices.create.CreateIndexAction; import org.opensearch.action.admin.indices.create.CreateIndexRequest; import org.opensearch.action.admin.indices.create.CreateIndexResponse; @@ -2437,6 +2439,18 @@ public void onFailure(final Exception e) { indexNameExpressionResolver ) ); + + actions.put( + GetTermVersionAction.INSTANCE, + new TransportGetTermVersionAction( + transportService, + clusterService, + threadPool, + actionFilters, + indexNameExpressionResolver + ) + ); + DynamicActionRegistry dynamicActionRegistry = new DynamicActionRegistry(); dynamicActionRegistry.registerUnmodifiableActionMap(actions); client.initialize( From f3d2beee637f63e38c8f26dbcee9f2a82f9c87b6 Mon Sep 17 00:00:00 2001 From: Gaurav Bafna <85113518+gbbafna@users.noreply.github.com> Date: Thu, 21 Mar 2024 10:35:24 +0530 Subject: [PATCH 21/38] [Remote Migration] Changes for Primary Relocation during migration (#12494) Changes for Primary Relocation during migration --------- Signed-off-by: Gaurav Bafna --- .../opensearch/index/shard/IndexShardIT.java | 3 +- .../SegmentReplicationClusterSettingIT.java | 8 +- .../MigrationBaseTestCase.java | 23 +- .../RemotePrimaryRelocationIT.java | 223 ++++++++++++++++++ .../RemoteIndexPrimaryRelocationIT.java | 1 - .../RemoteStoreBaseIntegTestCase.java | 121 +++++++++- .../SegmentReplicationSnapshotIT.java | 2 +- ...ransportSegmentReplicationStatsAction.java | 2 +- .../coordination/JoinTaskExecutor.java | 2 +- .../org/opensearch/index/IndexService.java | 35 ++- .../org/opensearch/index/IndexSettings.java | 16 +- .../SegmentReplicationPressureService.java | 7 +- .../index/SegmentReplicationStatsTracker.java | 2 +- .../opensearch/index/engine/EngineConfig.java | 4 +- .../index/engine/InternalEngine.java | 8 +- .../index/engine/NRTReplicationEngine.java | 3 +- .../RemoteStoreStatsTrackerFactory.java | 2 +- .../index/seqno/ReplicationTracker.java | 10 +- .../opensearch/index/shard/IndexShard.java | 198 +++++++++++++--- .../shard/RemoteStoreRefreshListener.java | 2 +- .../opensearch/index/shard/StoreRecovery.java | 35 +-- .../org/opensearch/index/store/Store.java | 4 +- .../index/translog/RemoteFsTranslog.java | 2 +- .../opensearch/index/translog/Translog.java | 2 +- .../opensearch/indices/IndicesService.java | 26 +- .../indices/recovery/MultiFileWriter.java | 2 +- .../recovery/PeerRecoverySourceService.java | 2 +- .../recovery/PeerRecoveryTargetService.java | 7 +- .../recovery/RecoverySourceHandler.java | 10 +- .../RecoverySourceHandlerFactory.java | 3 +- .../indices/recovery/RecoveryTarget.java | 26 +- .../SegmentReplicationSourceService.java | 9 +- .../SegmentReplicationTargetService.java | 12 +- .../remotestore/RemoteStoreNodeAttribute.java | 8 + .../snapshots/SnapshotShardsService.java | 2 +- .../IndexLevelReplicationTests.java | 23 +- .../RecoveryDuringReplicationTests.java | 13 +- .../index/shard/IndexShardTests.java | 6 +- ...overyWithRemoteTranslogOnPrimaryTests.java | 2 +- .../SegmentReplicationIndexShardTests.java | 2 +- .../index/translog/RemoteFsTranslogTests.java | 5 +- ...dicesLifecycleListenerSingleNodeTests.java | 18 +- .../PeerRecoveryTargetServiceTests.java | 6 +- .../indices/recovery/RecoveryTests.java | 7 +- .../recovery/ReplicationCollectionTests.java | 2 +- ...enSearchIndexLevelReplicationTestCase.java | 2 +- .../index/shard/IndexShardTestCase.java | 22 +- 47 files changed, 745 insertions(+), 185 deletions(-) create mode 100644 server/src/internalClusterTest/java/org/opensearch/remotemigration/RemotePrimaryRelocationIT.java diff --git a/server/src/internalClusterTest/java/org/opensearch/index/shard/IndexShardIT.java b/server/src/internalClusterTest/java/org/opensearch/index/shard/IndexShardIT.java index c394a1f631690..7e0c1630a76e4 100644 --- a/server/src/internalClusterTest/java/org/opensearch/index/shard/IndexShardIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/index/shard/IndexShardIT.java @@ -713,7 +713,8 @@ public static final IndexShard newIndexShard( null, () -> IndexSettings.DEFAULT_REMOTE_TRANSLOG_BUFFER_INTERVAL, nodeId, - null + null, + false ); } diff --git a/server/src/internalClusterTest/java/org/opensearch/indices/replication/SegmentReplicationClusterSettingIT.java b/server/src/internalClusterTest/java/org/opensearch/indices/replication/SegmentReplicationClusterSettingIT.java index f2cb7c9c6bfc8..d2f1e6313db07 100644 --- a/server/src/internalClusterTest/java/org/opensearch/indices/replication/SegmentReplicationClusterSettingIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/indices/replication/SegmentReplicationClusterSettingIT.java @@ -89,8 +89,8 @@ public void testIndexReplicationSettingOverridesSegRepClusterSetting() throws Ex Index index = resolveIndex(INDEX_NAME); Index anotherIndex = resolveIndex(ANOTHER_INDEX); IndicesService indicesService = internalCluster().getInstance(IndicesService.class, primaryNode); - assertEquals(indicesService.indexService(index).getIndexSettings().isSegRepEnabled(), false); - assertEquals(indicesService.indexService(anotherIndex).getIndexSettings().isSegRepEnabled(), true); + assertEquals(indicesService.indexService(index).getIndexSettings().isSegRepEnabledOrRemoteNode(), false); + assertEquals(indicesService.indexService(anotherIndex).getIndexSettings().isSegRepEnabledOrRemoteNode(), true); } public void testIndexReplicationSettingOverridesDocRepClusterSetting() throws Exception { @@ -119,8 +119,8 @@ public void testIndexReplicationSettingOverridesDocRepClusterSetting() throws Ex Index index = resolveIndex(INDEX_NAME); Index anotherIndex = resolveIndex(ANOTHER_INDEX); IndicesService indicesService = internalCluster().getInstance(IndicesService.class, primaryNode); - assertEquals(indicesService.indexService(index).getIndexSettings().isSegRepEnabled(), true); - assertEquals(indicesService.indexService(anotherIndex).getIndexSettings().isSegRepEnabled(), false); + assertEquals(indicesService.indexService(index).getIndexSettings().isSegRepEnabledOrRemoteNode(), true); + assertEquals(indicesService.indexService(anotherIndex).getIndexSettings().isSegRepEnabledOrRemoteNode(), false); } public void testReplicationTypesOverrideNotAllowed_IndexAPI() { diff --git a/server/src/internalClusterTest/java/org/opensearch/remotemigration/MigrationBaseTestCase.java b/server/src/internalClusterTest/java/org/opensearch/remotemigration/MigrationBaseTestCase.java index 88d6f6897ee68..19da668c432cf 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotemigration/MigrationBaseTestCase.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotemigration/MigrationBaseTestCase.java @@ -8,13 +8,19 @@ package org.opensearch.remotemigration; +import org.opensearch.action.admin.cluster.repositories.get.GetRepositoriesRequest; +import org.opensearch.action.admin.cluster.repositories.get.GetRepositoriesResponse; +import org.opensearch.cluster.metadata.RepositoryMetadata; import org.opensearch.common.settings.Settings; import org.opensearch.common.util.FeatureFlags; +import org.opensearch.repositories.fs.ReloadableFsRepository; import org.opensearch.test.OpenSearchIntegTestCase; import java.nio.file.Path; +import java.util.concurrent.ExecutionException; -import static org.opensearch.remotestore.RemoteStoreBaseIntegTestCase.remoteStoreClusterSettings; +import static org.opensearch.repositories.fs.ReloadableFsRepository.REPOSITORIES_FAILRATE_SETTING; +import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertAcked; public class MigrationBaseTestCase extends OpenSearchIntegTestCase { protected static final String REPOSITORY_NAME = "test-remote-store-repo"; @@ -35,11 +41,10 @@ protected Settings nodeSettings(int nodeOrdinal) { return Settings.builder() .put(super.nodeSettings(nodeOrdinal)) .put(remoteStoreClusterSettings(REPOSITORY_NAME, segmentRepoPath, REPOSITORY_2_NAME, translogRepoPath)) - .put("discovery.initial_state_timeout", "500ms") .build(); } else { logger.info("Adding docrep node"); - return Settings.builder().put(super.nodeSettings(nodeOrdinal)).put("discovery.initial_state_timeout", "500ms").build(); + return Settings.builder().put(super.nodeSettings(nodeOrdinal)).build(); } } @@ -47,4 +52,16 @@ protected Settings nodeSettings(int nodeOrdinal) { protected Settings featureFlagSettings() { return Settings.builder().put(super.featureFlagSettings()).put(FeatureFlags.REMOTE_STORE_MIGRATION_EXPERIMENTAL, "true").build(); } + + protected void setFailRate(String repoName, int value) throws ExecutionException, InterruptedException { + GetRepositoriesRequest gr = new GetRepositoriesRequest(new String[] { repoName }); + GetRepositoriesResponse res = client().admin().cluster().getRepositories(gr).get(); + RepositoryMetadata rmd = res.repositories().get(0); + Settings.Builder settings = Settings.builder() + .put("location", rmd.settings().get("location")) + .put(REPOSITORIES_FAILRATE_SETTING.getKey(), value); + assertAcked( + client().admin().cluster().preparePutRepository(repoName).setType(ReloadableFsRepository.TYPE).setSettings(settings).get() + ); + } } diff --git a/server/src/internalClusterTest/java/org/opensearch/remotemigration/RemotePrimaryRelocationIT.java b/server/src/internalClusterTest/java/org/opensearch/remotemigration/RemotePrimaryRelocationIT.java new file mode 100644 index 0000000000000..b1c429a45a1a1 --- /dev/null +++ b/server/src/internalClusterTest/java/org/opensearch/remotemigration/RemotePrimaryRelocationIT.java @@ -0,0 +1,223 @@ +/* + * 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.remotemigration; + +import com.carrotsearch.randomizedtesting.generators.RandomNumbers; + +import org.opensearch.action.DocWriteResponse; +import org.opensearch.action.admin.cluster.health.ClusterHealthResponse; +import org.opensearch.action.admin.cluster.repositories.get.GetRepositoriesRequest; +import org.opensearch.action.admin.cluster.repositories.get.GetRepositoriesResponse; +import org.opensearch.action.admin.cluster.settings.ClusterUpdateSettingsRequest; +import org.opensearch.action.delete.DeleteResponse; +import org.opensearch.action.index.IndexResponse; +import org.opensearch.client.Client; +import org.opensearch.cluster.routing.allocation.command.MoveAllocationCommand; +import org.opensearch.common.Priority; +import org.opensearch.common.settings.Settings; +import org.opensearch.common.unit.TimeValue; +import org.opensearch.index.query.QueryBuilders; +import org.opensearch.plugins.Plugin; +import org.opensearch.test.OpenSearchIntegTestCase; +import org.opensearch.test.hamcrest.OpenSearchAssertions; +import org.opensearch.test.transport.MockTransportService; + +import java.util.Collection; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; + +import static java.util.Arrays.asList; +import static org.opensearch.node.remotestore.RemoteStoreNodeService.REMOTE_STORE_COMPATIBILITY_MODE_SETTING; +import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertAcked; + +@OpenSearchIntegTestCase.ClusterScope(scope = OpenSearchIntegTestCase.Scope.TEST, numDataNodes = 0) +public class RemotePrimaryRelocationIT extends MigrationBaseTestCase { + protected int maximumNumberOfShards() { + return 1; + } + + // ToDo : Fix me when we support migration of replicas + protected int maximumNumberOfReplicas() { + return 0; + } + + protected Collection> nodePlugins() { + return asList(MockTransportService.TestPlugin.class); + } + + public void testMixedModeRelocation() throws Exception { + String docRepNode = internalCluster().startNode(); + Client client = internalCluster().client(docRepNode); + ClusterUpdateSettingsRequest updateSettingsRequest = new ClusterUpdateSettingsRequest(); + updateSettingsRequest.persistentSettings(Settings.builder().put(REMOTE_STORE_COMPATIBILITY_MODE_SETTING.getKey(), "mixed")); + assertAcked(client().admin().cluster().updateSettings(updateSettingsRequest).actionGet()); + + // create shard with 0 replica and 1 shard + client().admin().indices().prepareCreate("test").setSettings(indexSettings()).setMapping("field", "type=text").get(); + ensureGreen("test"); + + AtomicInteger numAutoGenDocs = new AtomicInteger(); + final AtomicBoolean finished = new AtomicBoolean(false); + Thread indexingThread = getIndexingThread(finished, numAutoGenDocs); + + refresh("test"); + + // add remote node in mixed mode cluster + addRemote = true; + String remoteNode = internalCluster().startNode(); + internalCluster().validateClusterFormed(); + + String remoteNode2 = internalCluster().startNode(); + internalCluster().validateClusterFormed(); + + // assert repo gets registered + GetRepositoriesRequest gr = new GetRepositoriesRequest(new String[] { REPOSITORY_NAME }); + GetRepositoriesResponse getRepositoriesResponse = client.admin().cluster().getRepositories(gr).actionGet(); + assertEquals(1, getRepositoriesResponse.repositories().size()); + + // Index some more docs + int currentDoc = numAutoGenDocs.get(); + int finalCurrentDoc1 = currentDoc; + waitUntil(() -> numAutoGenDocs.get() > finalCurrentDoc1 + 5); + + logger.info("--> relocating from {} to {} ", docRepNode, remoteNode); + client().admin().cluster().prepareReroute().add(new MoveAllocationCommand("test", 0, docRepNode, remoteNode)).execute().actionGet(); + ClusterHealthResponse clusterHealthResponse = client().admin() + .cluster() + .prepareHealth() + .setTimeout(TimeValue.timeValueSeconds(60)) + .setWaitForEvents(Priority.LANGUID) + .setWaitForNoRelocatingShards(true) + .execute() + .actionGet(); + + assertEquals(0, clusterHealthResponse.getRelocatingShards()); + assertEquals(remoteNode, primaryNodeName("test")); + logger.info("--> relocation from docrep to remote complete"); + + // Index some more docs + currentDoc = numAutoGenDocs.get(); + int finalCurrentDoc = currentDoc; + waitUntil(() -> numAutoGenDocs.get() > finalCurrentDoc + 5); + + client().admin() + .cluster() + .prepareReroute() + .add(new MoveAllocationCommand("test", 0, remoteNode, remoteNode2)) + .execute() + .actionGet(); + clusterHealthResponse = client().admin() + .cluster() + .prepareHealth() + .setTimeout(TimeValue.timeValueSeconds(60)) + .setWaitForEvents(Priority.LANGUID) + .setWaitForNoRelocatingShards(true) + .execute() + .actionGet(); + + assertEquals(0, clusterHealthResponse.getRelocatingShards()); + assertEquals(remoteNode2, primaryNodeName("test")); + + logger.info("--> relocation from remote to remote complete"); + + finished.set(true); + indexingThread.join(); + refresh("test"); + OpenSearchAssertions.assertHitCount(client().prepareSearch("test").setTrackTotalHits(true).get(), numAutoGenDocs.get()); + OpenSearchAssertions.assertHitCount( + client().prepareSearch("test") + .setTrackTotalHits(true)// extra paranoia ;) + .setQuery(QueryBuilders.termQuery("auto", true)) + .get(), + numAutoGenDocs.get() + ); + + } + + public void testMixedModeRelocation_RemoteSeedingFail() throws Exception { + String docRepNode = internalCluster().startNode(); + Client client = internalCluster().client(docRepNode); + ClusterUpdateSettingsRequest updateSettingsRequest = new ClusterUpdateSettingsRequest(); + updateSettingsRequest.persistentSettings(Settings.builder().put(REMOTE_STORE_COMPATIBILITY_MODE_SETTING.getKey(), "mixed")); + assertAcked(client().admin().cluster().updateSettings(updateSettingsRequest).actionGet()); + + // create shard with 0 replica and 1 shard + client().admin().indices().prepareCreate("test").setSettings(indexSettings()).setMapping("field", "type=text").get(); + ensureGreen("test"); + + AtomicInteger numAutoGenDocs = new AtomicInteger(); + final AtomicBoolean finished = new AtomicBoolean(false); + Thread indexingThread = getIndexingThread(finished, numAutoGenDocs); + + refresh("test"); + + // add remote node in mixed mode cluster + addRemote = true; + String remoteNode = internalCluster().startNode(); + internalCluster().validateClusterFormed(); + + // assert repo gets registered + GetRepositoriesRequest gr = new GetRepositoriesRequest(new String[] { REPOSITORY_NAME }); + GetRepositoriesResponse getRepositoriesResponse = client.admin().cluster().getRepositories(gr).actionGet(); + assertEquals(1, getRepositoriesResponse.repositories().size()); + + setFailRate(REPOSITORY_NAME, 100); + + logger.info("--> relocating from {} to {} ", docRepNode, remoteNode); + client().admin().cluster().prepareReroute().add(new MoveAllocationCommand("test", 0, docRepNode, remoteNode)).execute().actionGet(); + ClusterHealthResponse clusterHealthResponse = client().admin() + .cluster() + .prepareHealth() + .setTimeout(TimeValue.timeValueSeconds(5)) + .setWaitForEvents(Priority.LANGUID) + .setWaitForNoRelocatingShards(true) + .execute() + .actionGet(); + + assertTrue(clusterHealthResponse.getRelocatingShards() == 1); + setFailRate(REPOSITORY_NAME, 0); + Thread.sleep(RandomNumbers.randomIntBetween(random(), 0, 2000)); + clusterHealthResponse = client().admin() + .cluster() + .prepareHealth() + .setTimeout(TimeValue.timeValueSeconds(45)) + .setWaitForEvents(Priority.LANGUID) + .setWaitForNoRelocatingShards(true) + .execute() + .actionGet(); + assertTrue(clusterHealthResponse.getRelocatingShards() == 0); + logger.info("--> remote to remote relocation complete"); + finished.set(true); + indexingThread.join(); + refresh("test"); + OpenSearchAssertions.assertHitCount(client().prepareSearch("test").setTrackTotalHits(true).get(), numAutoGenDocs.get()); + OpenSearchAssertions.assertHitCount( + client().prepareSearch("test") + .setTrackTotalHits(true)// extra paranoia ;) + .setQuery(QueryBuilders.termQuery("auto", true)) + .get(), + numAutoGenDocs.get() + ); + } + + private static Thread getIndexingThread(AtomicBoolean finished, AtomicInteger numAutoGenDocs) { + Thread indexingThread = new Thread(() -> { + while (finished.get() == false && numAutoGenDocs.get() < 10_000) { + IndexResponse indexResponse = client().prepareIndex("test").setId("id").setSource("field", "value").get(); + assertEquals(DocWriteResponse.Result.CREATED, indexResponse.getResult()); + DeleteResponse deleteResponse = client().prepareDelete("test", "id").get(); + assertEquals(DocWriteResponse.Result.DELETED, deleteResponse.getResult()); + client().prepareIndex("test").setSource("auto", true).get(); + numAutoGenDocs.incrementAndGet(); + } + }); + indexingThread.start(); + return indexingThread; + } +} diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteIndexPrimaryRelocationIT.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteIndexPrimaryRelocationIT.java index 869032a84c2c2..67316ed0e6e6b 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteIndexPrimaryRelocationIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteIndexPrimaryRelocationIT.java @@ -44,7 +44,6 @@ public Settings indexSettings() { .build(); } - @AwaitsFix(bugUrl = "https://github.com/opensearch-project/OpenSearch/issues/9191") public void testPrimaryRelocationWhileIndexing() throws Exception { internalCluster().startClusterManagerOnlyNode(); super.testPrimaryRelocationWhileIndexing(); diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreBaseIntegTestCase.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreBaseIntegTestCase.java index 3899c8a80f442..ba90cbe96e157 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreBaseIntegTestCase.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreBaseIntegTestCase.java @@ -28,6 +28,7 @@ import org.opensearch.cluster.service.ClusterService; import org.opensearch.common.UUIDs; import org.opensearch.common.settings.Settings; +import org.opensearch.core.common.unit.ByteSizeUnit; import org.opensearch.core.index.Index; import org.opensearch.index.IndexModule; import org.opensearch.index.IndexService; @@ -56,8 +57,11 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.Collectors; +import static org.opensearch.node.remotestore.RemoteStoreNodeAttribute.REMOTE_STORE_CLUSTER_STATE_REPOSITORY_NAME_ATTRIBUTE_KEY; import static org.opensearch.node.remotestore.RemoteStoreNodeAttribute.REMOTE_STORE_REPOSITORY_SETTINGS_ATTRIBUTE_KEY_PREFIX; import static org.opensearch.node.remotestore.RemoteStoreNodeAttribute.REMOTE_STORE_REPOSITORY_TYPE_ATTRIBUTE_KEY_FORMAT; +import static org.opensearch.node.remotestore.RemoteStoreNodeAttribute.REMOTE_STORE_SEGMENT_REPOSITORY_NAME_ATTRIBUTE_KEY; +import static org.opensearch.node.remotestore.RemoteStoreNodeAttribute.REMOTE_STORE_TRANSLOG_REPOSITORY_NAME_ATTRIBUTE_KEY; import static org.opensearch.repositories.fs.ReloadableFsRepository.REPOSITORIES_FAILRATE_SETTING; import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertAcked; @@ -187,7 +191,122 @@ protected BulkResponse indexBulk(String indexName, int numDocs) { return client().bulk(bulkRequest).actionGet(); } - private Settings defaultIndexSettings() { + public static Settings remoteStoreClusterSettings(String name, Path path) { + return remoteStoreClusterSettings(name, path, name, path); + } + + public static Settings remoteStoreClusterSettings( + String segmentRepoName, + Path segmentRepoPath, + String segmentRepoType, + String translogRepoName, + Path translogRepoPath, + String translogRepoType + ) { + Settings.Builder settingsBuilder = Settings.builder(); + settingsBuilder.put( + buildRemoteStoreNodeAttributes( + segmentRepoName, + segmentRepoPath, + segmentRepoType, + translogRepoName, + translogRepoPath, + translogRepoType, + false + ) + ); + return settingsBuilder.build(); + } + + public static Settings remoteStoreClusterSettings( + String segmentRepoName, + Path segmentRepoPath, + String translogRepoName, + Path translogRepoPath + ) { + Settings.Builder settingsBuilder = Settings.builder(); + settingsBuilder.put(buildRemoteStoreNodeAttributes(segmentRepoName, segmentRepoPath, translogRepoName, translogRepoPath, false)); + return settingsBuilder.build(); + } + + public static Settings buildRemoteStoreNodeAttributes( + String segmentRepoName, + Path segmentRepoPath, + String translogRepoName, + Path translogRepoPath, + boolean withRateLimiterAttributes + ) { + return buildRemoteStoreNodeAttributes( + segmentRepoName, + segmentRepoPath, + ReloadableFsRepository.TYPE, + translogRepoName, + translogRepoPath, + ReloadableFsRepository.TYPE, + withRateLimiterAttributes + ); + } + + public static Settings buildRemoteStoreNodeAttributes( + String segmentRepoName, + Path segmentRepoPath, + String segmentRepoType, + String translogRepoName, + Path translogRepoPath, + String translogRepoType, + boolean withRateLimiterAttributes + ) { + String segmentRepoTypeAttributeKey = String.format( + Locale.getDefault(), + "node.attr." + REMOTE_STORE_REPOSITORY_TYPE_ATTRIBUTE_KEY_FORMAT, + segmentRepoName + ); + String segmentRepoSettingsAttributeKeyPrefix = String.format( + Locale.getDefault(), + "node.attr." + REMOTE_STORE_REPOSITORY_SETTINGS_ATTRIBUTE_KEY_PREFIX, + segmentRepoName + ); + String translogRepoTypeAttributeKey = String.format( + Locale.getDefault(), + "node.attr." + REMOTE_STORE_REPOSITORY_TYPE_ATTRIBUTE_KEY_FORMAT, + translogRepoName + ); + String translogRepoSettingsAttributeKeyPrefix = String.format( + Locale.getDefault(), + "node.attr." + REMOTE_STORE_REPOSITORY_SETTINGS_ATTRIBUTE_KEY_PREFIX, + translogRepoName + ); + String stateRepoTypeAttributeKey = String.format( + Locale.getDefault(), + "node.attr." + REMOTE_STORE_REPOSITORY_TYPE_ATTRIBUTE_KEY_FORMAT, + segmentRepoName + ); + String stateRepoSettingsAttributeKeyPrefix = String.format( + Locale.getDefault(), + "node.attr." + REMOTE_STORE_REPOSITORY_SETTINGS_ATTRIBUTE_KEY_PREFIX, + segmentRepoName + ); + + Settings.Builder settings = Settings.builder() + .put("node.attr." + REMOTE_STORE_SEGMENT_REPOSITORY_NAME_ATTRIBUTE_KEY, segmentRepoName) + .put(segmentRepoTypeAttributeKey, segmentRepoType) + .put(segmentRepoSettingsAttributeKeyPrefix + "location", segmentRepoPath) + .put("node.attr." + REMOTE_STORE_TRANSLOG_REPOSITORY_NAME_ATTRIBUTE_KEY, translogRepoName) + .put(translogRepoTypeAttributeKey, translogRepoType) + .put(translogRepoSettingsAttributeKeyPrefix + "location", translogRepoPath) + .put("node.attr." + REMOTE_STORE_CLUSTER_STATE_REPOSITORY_NAME_ATTRIBUTE_KEY, segmentRepoName) + .put(stateRepoTypeAttributeKey, segmentRepoType) + .put(stateRepoSettingsAttributeKeyPrefix + "location", segmentRepoPath); + + if (withRateLimiterAttributes) { + settings.put(segmentRepoSettingsAttributeKeyPrefix + "compress", randomBoolean()) + .put(segmentRepoSettingsAttributeKeyPrefix + "chunk_size", 200, ByteSizeUnit.BYTES); + } + + return settings.build(); + } + + Settings defaultIndexSettings() { return Settings.builder() .put(super.indexSettings()) .put(IndexModule.INDEX_QUERY_CACHE_ENABLED_SETTING.getKey(), false) diff --git a/server/src/internalClusterTest/java/org/opensearch/snapshots/SegmentReplicationSnapshotIT.java b/server/src/internalClusterTest/java/org/opensearch/snapshots/SegmentReplicationSnapshotIT.java index c649c4ab13e7e..b019bb57743c9 100644 --- a/server/src/internalClusterTest/java/org/opensearch/snapshots/SegmentReplicationSnapshotIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/snapshots/SegmentReplicationSnapshotIT.java @@ -303,7 +303,7 @@ public void testSnapshotRestoreOnIndexWithSegRepClusterSetting() throws Exceptio // Verify index setting isSegRepEnabled. Index index = resolveIndex(RESTORED_INDEX_NAME); IndicesService indicesService = internalCluster().getInstance(IndicesService.class); - assertEquals(indicesService.indexService(index).getIndexSettings().isSegRepEnabled(), false); + assertEquals(indicesService.indexService(index).getIndexSettings().isSegRepEnabledOrRemoteNode(), false); } /** diff --git a/server/src/main/java/org/opensearch/action/admin/indices/replication/TransportSegmentReplicationStatsAction.java b/server/src/main/java/org/opensearch/action/admin/indices/replication/TransportSegmentReplicationStatsAction.java index 1b912518d7e04..fc97d67c6c3af 100644 --- a/server/src/main/java/org/opensearch/action/admin/indices/replication/TransportSegmentReplicationStatsAction.java +++ b/server/src/main/java/org/opensearch/action/admin/indices/replication/TransportSegmentReplicationStatsAction.java @@ -148,7 +148,7 @@ protected SegmentReplicationShardStatsResponse shardOperation(SegmentReplication IndexShard indexShard = indexService.getShard(shardRouting.shardId().id()); ShardId shardId = shardRouting.shardId(); - if (indexShard.indexSettings().isSegRepEnabled() == false) { + if (indexShard.indexSettings().isSegRepEnabledOrRemoteNode() == false) { return null; } diff --git a/server/src/main/java/org/opensearch/cluster/coordination/JoinTaskExecutor.java b/server/src/main/java/org/opensearch/cluster/coordination/JoinTaskExecutor.java index bc365b9872037..5d896e392e6bc 100644 --- a/server/src/main/java/org/opensearch/cluster/coordination/JoinTaskExecutor.java +++ b/server/src/main/java/org/opensearch/cluster/coordination/JoinTaskExecutor.java @@ -215,7 +215,7 @@ public ClusterTasksResult execute(ClusterState currentState, List jo ensureNodeCommissioned(node, currentState.metadata()); nodesBuilder.add(node); - if (remoteDN.isEmpty()) { + if (remoteDN.isEmpty() && node.isRemoteStoreNode()) { // This is hit only on cases where we encounter first remote node logger.info("Updating system repository now for remote store"); repositoriesMetadata = remoteStoreNodeService.updateRepositoriesMetadata( diff --git a/server/src/main/java/org/opensearch/index/IndexService.java b/server/src/main/java/org/opensearch/index/IndexService.java index 0909e2d5c8ff0..11dc4474cfa42 100644 --- a/server/src/main/java/org/opensearch/index/IndexService.java +++ b/server/src/main/java/org/opensearch/index/IndexService.java @@ -90,6 +90,7 @@ import org.opensearch.index.shard.ShardNotInPrimaryModeException; import org.opensearch.index.shard.ShardPath; import org.opensearch.index.similarity.SimilarityService; +import org.opensearch.index.store.RemoteSegmentStoreDirectoryFactory; import org.opensearch.index.store.Store; import org.opensearch.index.translog.Translog; import org.opensearch.index.translog.TranslogFactory; @@ -99,7 +100,9 @@ import org.opensearch.indices.recovery.RecoverySettings; import org.opensearch.indices.recovery.RecoveryState; import org.opensearch.indices.replication.checkpoint.SegmentReplicationCheckpointPublisher; +import org.opensearch.node.remotestore.RemoteStoreNodeAttribute; import org.opensearch.plugins.IndexStorePlugin; +import org.opensearch.repositories.RepositoriesService; import org.opensearch.script.ScriptService; import org.opensearch.search.aggregations.support.ValuesSourceRegistry; import org.opensearch.threadpool.ThreadPool; @@ -455,7 +458,10 @@ public synchronized IndexShard createShard( final Consumer globalCheckpointSyncer, final RetentionLeaseSyncer retentionLeaseSyncer, final SegmentReplicationCheckpointPublisher checkpointPublisher, - final RemoteStoreStatsTrackerFactory remoteStoreStatsTrackerFactory + final RemoteStoreStatsTrackerFactory remoteStoreStatsTrackerFactory, + final RepositoriesService repositoriesService, + final DiscoveryNode targetNode, + @Nullable DiscoveryNode sourceNode ) throws IOException { Objects.requireNonNull(retentionLeaseSyncer); /* @@ -484,10 +490,26 @@ public synchronized IndexShard createShard( warmer.warm(reader, shard, IndexService.this.indexSettings); } }; - Store remoteStore = null; - if (this.indexSettings.isRemoteStoreEnabled()) { - Directory remoteDirectory = remoteDirectoryFactory.newDirectory(this.indexSettings, path); + boolean seedRemote = false; + if (targetNode.isRemoteStoreNode()) { + final Directory remoteDirectory; + if (this.indexSettings.isRemoteStoreEnabled()) { + remoteDirectory = remoteDirectoryFactory.newDirectory(this.indexSettings, path); + } else { + if (sourceNode != null && sourceNode.isRemoteStoreNode() == false) { + if (routing.primary() == false) { + throw new IllegalStateException("Can't migrate a remote shard to replica before primary " + routing.shardId()); + } + logger.info("DocRep shard {} is migrating to remote", shardId); + seedRemote = true; + } + remoteDirectory = ((RemoteSegmentStoreDirectoryFactory) remoteDirectoryFactory).newDirectory( + RemoteStoreNodeAttribute.getRemoteStoreSegmentRepo(this.indexSettings.getNodeSettings()), + this.indexSettings.getUUID(), + shardId + ); + } remoteStore = new Store(shardId, this.indexSettings, remoteDirectory, lock, Store.OnClose.EMPTY, path); } @@ -523,12 +545,13 @@ public synchronized IndexShard createShard( retentionLeaseSyncer, circuitBreakerService, translogFactorySupplier, - this.indexSettings.isSegRepEnabled() ? checkpointPublisher : null, + this.indexSettings.isSegRepEnabledOrRemoteNode() ? checkpointPublisher : null, remoteStore, remoteStoreStatsTrackerFactory, clusterRemoteTranslogBufferIntervalSupplier, nodeEnv.nodeId(), - recoverySettings + recoverySettings, + seedRemote ); eventListener.indexShardStateChanged(indexShard, null, indexShard.state(), "shard created"); eventListener.afterIndexShardCreated(indexShard); diff --git a/server/src/main/java/org/opensearch/index/IndexSettings.java b/server/src/main/java/org/opensearch/index/IndexSettings.java index 5aaea2c498701..7e49726c259cb 100644 --- a/server/src/main/java/org/opensearch/index/IndexSettings.java +++ b/server/src/main/java/org/opensearch/index/IndexSettings.java @@ -52,6 +52,7 @@ import org.opensearch.indices.replication.common.ReplicationType; import org.opensearch.ingest.IngestService; import org.opensearch.node.Node; +import org.opensearch.node.remotestore.RemoteStoreNodeAttribute; import org.opensearch.search.pipeline.SearchPipelineService; import java.util.Arrays; @@ -1221,17 +1222,20 @@ public int getNumberOfReplicas() { /** * Returns true if segment replication is enabled on the index. + * + * Every shard on a remote node would also have SegRep enabled even without + * proper index setting during the migration. */ - public boolean isSegRepEnabled() { - return ReplicationType.SEGMENT.equals(replicationType); + public boolean isSegRepEnabledOrRemoteNode() { + return ReplicationType.SEGMENT.equals(replicationType) || isRemoteNode(); } public boolean isSegRepLocalEnabled() { - return isSegRepEnabled() && !isRemoteStoreEnabled(); + return isSegRepEnabledOrRemoteNode() && !isRemoteStoreEnabled(); } public boolean isSegRepWithRemoteEnabled() { - return isSegRepEnabled() && isRemoteStoreEnabled(); + return isSegRepEnabledOrRemoteNode() && isRemoteStoreEnabled(); } /** @@ -1241,6 +1245,10 @@ public boolean isRemoteStoreEnabled() { return isRemoteStoreEnabled; } + public boolean isRemoteNode() { + return RemoteStoreNodeAttribute.isRemoteDataAttributePresent(this.getNodeSettings()); + } + /** * Returns if remote translog store is enabled for this index. */ diff --git a/server/src/main/java/org/opensearch/index/SegmentReplicationPressureService.java b/server/src/main/java/org/opensearch/index/SegmentReplicationPressureService.java index ce38dd3bb236c..297fe093f7f4e 100644 --- a/server/src/main/java/org/opensearch/index/SegmentReplicationPressureService.java +++ b/server/src/main/java/org/opensearch/index/SegmentReplicationPressureService.java @@ -145,7 +145,9 @@ public void isSegrepLimitBreached(ShardId shardId) { final IndexService indexService = indicesService.indexService(shardId.getIndex()); if (indexService != null) { final IndexShard shard = indexService.getShard(shardId.id()); - if (isSegmentReplicationBackpressureEnabled && shard.indexSettings().isSegRepEnabled() && shard.routingEntry().primary()) { + if (isSegmentReplicationBackpressureEnabled + && shard.indexSettings().isSegRepEnabledOrRemoteNode() + && shard.routingEntry().primary()) { validateReplicationGroup(shard); } } @@ -264,7 +266,8 @@ protected void runInternal() { stats.getShardStats().get(shardId).getReplicaStats() ); final IndexService indexService = pressureService.indicesService.indexService(shardId.getIndex()); - if (indexService.getIndexSettings() != null && indexService.getIndexSettings().isSegRepEnabled() == false) { + if (indexService.getIndexSettings() != null + && indexService.getIndexSettings().isSegRepEnabledOrRemoteNode() == false) { return; } final IndexShard primaryShard = indexService.getShard(shardId.getId()); diff --git a/server/src/main/java/org/opensearch/index/SegmentReplicationStatsTracker.java b/server/src/main/java/org/opensearch/index/SegmentReplicationStatsTracker.java index f5fc8aa1c1eea..e48a76c438057 100644 --- a/server/src/main/java/org/opensearch/index/SegmentReplicationStatsTracker.java +++ b/server/src/main/java/org/opensearch/index/SegmentReplicationStatsTracker.java @@ -45,7 +45,7 @@ public SegmentReplicationStats getStats() { Map stats = new HashMap<>(); for (IndexService indexService : indicesService) { for (IndexShard indexShard : indexService) { - if (indexShard.indexSettings().isSegRepEnabled() && indexShard.routingEntry().primary()) { + if (indexShard.indexSettings().isSegRepEnabledOrRemoteNode() && indexShard.routingEntry().primary()) { stats.putIfAbsent(indexShard.shardId(), getStatsForShard(indexShard)); } } diff --git a/server/src/main/java/org/opensearch/index/engine/EngineConfig.java b/server/src/main/java/org/opensearch/index/engine/EngineConfig.java index bf3e10d684c94..8106b65bddeec 100644 --- a/server/src/main/java/org/opensearch/index/engine/EngineConfig.java +++ b/server/src/main/java/org/opensearch/index/engine/EngineConfig.java @@ -244,7 +244,7 @@ private static void doValidateCodecSettings(final String codec) { * Creates a new {@link org.opensearch.index.engine.EngineConfig} */ private EngineConfig(Builder builder) { - if (builder.isReadOnlyReplica && builder.indexSettings.isSegRepEnabled() == false) { + if (builder.isReadOnlyReplica && builder.indexSettings.isSegRepEnabledOrRemoteNode() == false) { throw new IllegalArgumentException("Shard can only be wired as a read only replica with Segment Replication enabled"); } this.shardId = builder.shardId; @@ -491,7 +491,7 @@ public LongSupplier getPrimaryTermSupplier() { * @return true if this engine should be wired as read only. */ public boolean isReadOnlyReplica() { - return indexSettings.isSegRepEnabled() && isReadOnlyReplica; + return indexSettings.isSegRepEnabledOrRemoteNode() && isReadOnlyReplica; } /** diff --git a/server/src/main/java/org/opensearch/index/engine/InternalEngine.java b/server/src/main/java/org/opensearch/index/engine/InternalEngine.java index a25ec95f58e05..7bacec22fc850 100644 --- a/server/src/main/java/org/opensearch/index/engine/InternalEngine.java +++ b/server/src/main/java/org/opensearch/index/engine/InternalEngine.java @@ -710,7 +710,7 @@ private OpVsLuceneDocStatus compareOpToLuceneDocBasedOnSeqNo(final Operation op) final OpVsLuceneDocStatus status; VersionValue versionValue = getVersionFromMap(op.uid().bytes()); assert incrementVersionLookup(); - boolean segRepEnabled = engineConfig.getIndexSettings().isSegRepEnabled(); + boolean segRepEnabled = engineConfig.getIndexSettings().isSegRepEnabledOrRemoteNode(); if (versionValue != null) { status = compareOpToVersionMapOnSeqNo(op.id(), op.seqNo(), op.primaryTerm(), versionValue); } else { @@ -1005,7 +1005,7 @@ protected final IndexingStrategy planIndexingAsNonPrimary(Index index) throws IO assert maxSeqNoOfUpdatesOrDeletes < index.seqNo() : index.seqNo() + ">=" + maxSeqNoOfUpdatesOrDeletes; plan = IndexingStrategy.optimizedAppendOnly(index.version(), 0); } else { - boolean segRepEnabled = engineConfig.getIndexSettings().isSegRepEnabled(); + boolean segRepEnabled = engineConfig.getIndexSettings().isSegRepEnabledOrRemoteNode(); versionMap.enforceSafeAccess(); final OpVsLuceneDocStatus opVsLucene = compareOpToLuceneDocBasedOnSeqNo(index); if (opVsLucene == OpVsLuceneDocStatus.OP_STALE_OR_EQUAL) { @@ -1452,7 +1452,7 @@ protected final DeletionStrategy planDeletionAsNonPrimary(Delete delete) throws // See testRecoveryWithOutOfOrderDelete for an example of peer recovery plan = DeletionStrategy.processButSkipLucene(false, delete.version()); } else { - boolean segRepEnabled = engineConfig.getIndexSettings().isSegRepEnabled(); + boolean segRepEnabled = engineConfig.getIndexSettings().isSegRepEnabledOrRemoteNode(); final OpVsLuceneDocStatus opVsLucene = compareOpToLuceneDocBasedOnSeqNo(delete); if (opVsLucene == OpVsLuceneDocStatus.OP_STALE_OR_EQUAL) { if (segRepEnabled) { @@ -1868,7 +1868,7 @@ public void flush(boolean force, boolean waitIfOngoing) throws EngineException { // only after the active reader is updated. This ensures that a flush does not wipe out a required commit point file // while we are // in refresh listeners. - final GatedCloseable latestCommit = engineConfig.getIndexSettings().isSegRepEnabled() + final GatedCloseable latestCommit = engineConfig.getIndexSettings().isSegRepEnabledOrRemoteNode() ? acquireLastIndexCommit(false) : null; commitIndexWriter(indexWriter, translogManager.getTranslogUUID()); diff --git a/server/src/main/java/org/opensearch/index/engine/NRTReplicationEngine.java b/server/src/main/java/org/opensearch/index/engine/NRTReplicationEngine.java index ed8dba2f8902d..1e1825e1f8ace 100644 --- a/server/src/main/java/org/opensearch/index/engine/NRTReplicationEngine.java +++ b/server/src/main/java/org/opensearch/index/engine/NRTReplicationEngine.java @@ -436,7 +436,8 @@ protected final void closeNoLock(String reason, CountDownLatch closedLatch) { This is not required for remote store implementations given on failover the replica re-syncs with the store during promotion. */ - if (engineConfig.getIndexSettings().isRemoteStoreEnabled() == false) { + if (engineConfig.getIndexSettings().isRemoteStoreEnabled() == false + && engineConfig.getIndexSettings().isRemoteNode() == false) { latestSegmentInfos.counter = latestSegmentInfos.counter + SI_COUNTER_INCREMENT; latestSegmentInfos.changed(); } diff --git a/server/src/main/java/org/opensearch/index/remote/RemoteStoreStatsTrackerFactory.java b/server/src/main/java/org/opensearch/index/remote/RemoteStoreStatsTrackerFactory.java index 9a146be96c9de..e4c7eb56d02c6 100644 --- a/server/src/main/java/org/opensearch/index/remote/RemoteStoreStatsTrackerFactory.java +++ b/server/src/main/java/org/opensearch/index/remote/RemoteStoreStatsTrackerFactory.java @@ -68,7 +68,7 @@ public RemoteStoreStatsTrackerFactory(ClusterService clusterService, Settings se @Override public void afterIndexShardCreated(IndexShard indexShard) { - if (indexShard.indexSettings().isRemoteStoreEnabled() == false) { + if (indexShard.indexSettings().isRemoteStoreEnabled() == false && indexShard.indexSettings().isRemoteNode() == false) { return; } ShardId shardId = indexShard.shardId(); diff --git a/server/src/main/java/org/opensearch/index/seqno/ReplicationTracker.java b/server/src/main/java/org/opensearch/index/seqno/ReplicationTracker.java index 7b9c1d3aa548f..0e625e9f30320 100644 --- a/server/src/main/java/org/opensearch/index/seqno/ReplicationTracker.java +++ b/server/src/main/java/org/opensearch/index/seqno/ReplicationTracker.java @@ -1059,7 +1059,7 @@ public ReplicationTracker( this.fileBasedRecoveryThreshold = IndexSettings.FILE_BASED_RECOVERY_THRESHOLD_SETTING.get(indexSettings.getSettings()); this.safeCommitInfoSupplier = safeCommitInfoSupplier; this.onReplicationGroupUpdated = onReplicationGroupUpdated; - this.latestReplicationCheckpoint = indexSettings.isSegRepEnabled() ? ReplicationCheckpoint.empty(shardId) : null; + this.latestReplicationCheckpoint = indexSettings.isSegRepEnabledOrRemoteNode() ? ReplicationCheckpoint.empty(shardId) : null; assert Version.V_EMPTY.equals(indexSettings.getIndexVersionCreated()) == false; assert invariant(); } @@ -1173,7 +1173,7 @@ public synchronized void updateGlobalCheckpointForShard(final String allocationI * @param visibleCheckpoint the visible checkpoint */ public synchronized void updateVisibleCheckpointForShard(final String allocationId, final ReplicationCheckpoint visibleCheckpoint) { - assert indexSettings.isSegRepEnabled(); + assert indexSettings.isSegRepEnabledOrRemoteNode(); assert primaryMode; assert handoffInProgress == false; assert invariant(); @@ -1217,7 +1217,7 @@ public synchronized void updateVisibleCheckpointForShard(final String allocation * @param checkpoint {@link ReplicationCheckpoint} */ public synchronized void setLatestReplicationCheckpoint(ReplicationCheckpoint checkpoint) { - assert indexSettings.isSegRepEnabled(); + assert indexSettings.isSegRepEnabledOrRemoteNode(); if (checkpoint.equals(latestReplicationCheckpoint) == false) { this.latestReplicationCheckpoint = checkpoint; } @@ -1269,7 +1269,7 @@ && isPrimaryRelocation(allocationId) == false * @param checkpoint {@link ReplicationCheckpoint} */ public synchronized void startReplicationLagTimers(ReplicationCheckpoint checkpoint) { - assert indexSettings.isSegRepEnabled(); + assert indexSettings.isSegRepEnabledOrRemoteNode(); if (checkpoint.equals(latestReplicationCheckpoint) == false) { this.latestReplicationCheckpoint = checkpoint; } @@ -1294,7 +1294,7 @@ && isPrimaryRelocation(e.getKey()) == false * V2 - Set of {@link SegmentReplicationShardStats} per shard in this primary's replication group. */ public synchronized Set getSegmentReplicationStats() { - assert indexSettings.isSegRepEnabled(); + assert indexSettings.isSegRepEnabledOrRemoteNode(); if (primaryMode) { return this.checkpoints.entrySet() .stream() diff --git a/server/src/main/java/org/opensearch/index/shard/IndexShard.java b/server/src/main/java/org/opensearch/index/shard/IndexShard.java index 977155a1cbb72..72ce858661031 100644 --- a/server/src/main/java/org/opensearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/opensearch/index/shard/IndexShard.java @@ -234,6 +234,9 @@ import static org.opensearch.index.seqno.SequenceNumbers.LOCAL_CHECKPOINT_KEY; import static org.opensearch.index.seqno.SequenceNumbers.MAX_SEQ_NO; import static org.opensearch.index.seqno.SequenceNumbers.UNASSIGNED_SEQ_NO; +import static org.opensearch.index.shard.IndexShard.ShardMigrationState.REMOTE_MIGRATING_SEEDED; +import static org.opensearch.index.shard.IndexShard.ShardMigrationState.REMOTE_MIGRATING_UNSEEDED; +import static org.opensearch.index.shard.IndexShard.ShardMigrationState.REMOTE_NON_MIGRATING; import static org.opensearch.index.translog.Translog.Durability; import static org.opensearch.index.translog.Translog.TRANSLOG_UUID_KEY; @@ -346,6 +349,12 @@ Runnable getGlobalCheckpointSyncer() { private final List internalRefreshListener = new ArrayList<>(); private final RemoteStoreFileDownloader fileDownloader; private final RecoverySettings recoverySettings; + /* + On source doc rep node, It will be DOCREP_NON_MIGRATING. + On source remote node , it will be REMOTE_MIGRATING_SEEDED when relocating from remote node + On source remote node , it will be REMOTE_MIGRATING_UNSEEDED when relocating from docrep node + */ + private final ShardMigrationState shardMigrationState; public IndexShard( final ShardRouting shardRouting, @@ -374,7 +383,8 @@ public IndexShard( final RemoteStoreStatsTrackerFactory remoteStoreStatsTrackerFactory, final Supplier clusterRemoteTranslogBufferIntervalSupplier, final String nodeId, - final RecoverySettings recoverySettings + final RecoverySettings recoverySettings, + boolean seedRemote ) throws IOException { super(shardRouting.shardId(), indexSettings); assert shardRouting.initializing(); @@ -394,7 +404,7 @@ public IndexShard( logger, threadPool, this::getEngine, - indexSettings.isRemoteTranslogStoreEnabled(), + indexSettings.isRemoteNode(), () -> getRemoteTranslogUploadBufferInterval(clusterRemoteTranslogBufferIntervalSupplier) ); this.mapperService = mapperService; @@ -472,6 +482,7 @@ public boolean shouldCache(Query query) { this.remoteStoreStatsTrackerFactory = remoteStoreStatsTrackerFactory; this.recoverySettings = recoverySettings; this.fileDownloader = new RemoteStoreFileDownloader(shardRouting.shardId(), threadPool, recoverySettings); + this.shardMigrationState = getShardMigrationState(indexSettings, seedRemote); } public ThreadPool getThreadPool() { @@ -482,6 +493,20 @@ public Store store() { return this.store; } + public boolean isMigratingToRemote() { + // set it true only if shard is remote, but index setting doesn't say so + return shardMigrationState == REMOTE_MIGRATING_UNSEEDED || shardMigrationState == REMOTE_MIGRATING_SEEDED; + } + + public boolean shouldSeedRemoteStore() { + // set it true only if relocating from docrep to remote store + return shardMigrationState == REMOTE_MIGRATING_UNSEEDED; + } + + public boolean isRemoteSeeded() { + return shardMigrationState == REMOTE_MIGRATING_SEEDED; + } + public Store remoteStore() { return this.remoteStore; } @@ -625,7 +650,7 @@ public void updateShardState( // Flush here after relocation of primary, so that replica get all changes from new primary rather than waiting for more // docs to get indexed. - if (indexSettings.isSegRepEnabled()) { + if (indexSettings.isSegRepEnabledOrRemoteNode()) { flush(new FlushRequest().waitIfOngoing(true).force(true)); } } else if (currentRouting.primary() @@ -705,7 +730,7 @@ public void updateShardState( + newRouting; assert getOperationPrimaryTerm() == newPrimaryTerm; try { - if (indexSettings.isSegRepEnabled()) { + if (indexSettings.isSegRepEnabledOrRemoteNode()) { // this Shard's engine was read only, we need to update its engine before restoring local history from xlog. assert newRouting.primary() && currentRouting.primary() == false; ReplicationTimer timer = new ReplicationTimer(); @@ -725,7 +750,7 @@ public void updateShardState( } replicationTracker.activatePrimaryMode(getLocalCheckpoint()); - if (indexSettings.isSegRepEnabled()) { + if (indexSettings.isSegRepEnabledOrRemoteNode()) { // force publish a checkpoint once in primary mode so that replicas not caught up to previous primary // are brought up to date. checkpointPublisher.publish(this, getLatestReplicationCheckpoint()); @@ -839,8 +864,8 @@ public IndexShardState markAsRecovering(String reason, RecoveryState recoverySta * relocated. After all operations are successfully blocked, performSegRep is executed followed by target relocation * handoff. * + * @param consumer a {@link Runnable} that is executed after performSegRep * @param performSegRep a {@link Runnable} that is executed after operations are blocked - * @param consumer a {@link Runnable} that is executed after performSegRep * @throws IllegalIndexShardStateException if the shard is not relocating due to concurrent cancellation * @throws IllegalStateException if the relocation target is no longer part of the replication group * @throws InterruptedException if blocking operations is interrupted @@ -858,7 +883,8 @@ public void relocated( indexShardOperationPermits.blockOperations(30, TimeUnit.MINUTES, () -> { forceRefreshes.close(); - boolean syncTranslog = isRemoteTranslogEnabled() && Durability.ASYNC == indexSettings.getTranslogDurability(); + boolean syncTranslog = (isRemoteTranslogEnabled() || this.isMigratingToRemote()) + && Durability.ASYNC == indexSettings.getTranslogDurability(); // Since all the index permits are acquired at this point, the translog buffer will not change. // It is safe to perform sync of translogs now as this will ensure for remote-backed indexes, the // translogs has been uploaded to the remote store. @@ -881,6 +907,7 @@ public void relocated( : "in-flight operations in progress while moving shard state to relocated"; performSegRep.run(); + /* * We should not invoke the runnable under the mutex as the expected implementation is to handoff the primary context via a * network operation. Doing this under the mutex can implicitly block the cluster state update thread on network operations. @@ -1041,7 +1068,7 @@ private Engine.IndexResult applyIndexOperation( // For Segment Replication enabled replica shards we can be skip parsing the documents as we directly copy segments from primary // shard. - if (indexSettings.isSegRepEnabled() && routingEntry().primary() == false) { + if (indexSettings.isSegRepEnabledOrRemoteNode() && routingEntry().primary() == false) { Engine.Index index = new Engine.Index( new Term(IdFieldMapper.NAME, Uid.encodeId(id)), new ParsedDocument(null, null, id, null, null, sourceToParse.source(), sourceToParse.getMediaType(), null), @@ -1240,7 +1267,7 @@ public Engine.DeleteResult applyDeleteOperationOnPrimary( } public Engine.DeleteResult applyDeleteOperationOnReplica(long seqNo, long opPrimaryTerm, long version, String id) throws IOException { - if (indexSettings.isSegRepEnabled()) { + if (indexSettings.isSegRepEnabledOrRemoteNode()) { final Engine.Delete delete = new Engine.Delete( id, new Term(IdFieldMapper.NAME, Uid.encodeId(id)), @@ -1435,12 +1462,12 @@ public SegmentsStats segmentStats(boolean includeSegmentFileSizes, boolean inclu SegmentsStats segmentsStats = getEngine().segmentsStats(includeSegmentFileSizes, includeUnloadedSegments); segmentsStats.addBitsetMemoryInBytes(shardBitsetFilterCache.getMemorySizeInBytes()); // Populate remote_store stats only if the index is remote store backed - if (indexSettings.isRemoteStoreEnabled()) { + if (indexSettings().isRemoteNode()) { segmentsStats.addRemoteSegmentStats( new RemoteSegmentStats(remoteStoreStatsTrackerFactory.getRemoteSegmentTransferTracker(shardId).stats()) ); } - if (indexSettings.isSegRepEnabled()) { + if (indexSettings.isSegRepEnabledOrRemoteNode()) { segmentsStats.addReplicationStats(getReplicationStats()); } return segmentsStats; @@ -1457,7 +1484,7 @@ public FieldDataStats fieldDataStats(String... fields) { public TranslogStats translogStats() { TranslogStats translogStats = getEngine().translogManager().getTranslogStats(); // Populate remote_store stats only if the index is remote store backed - if (indexSettings.isRemoteStoreEnabled()) { + if (indexSettings.isRemoteNode()) { translogStats.addRemoteTranslogStats( new RemoteTranslogStats(remoteStoreStatsTrackerFactory.getRemoteTranslogTransferTracker(shardId).stats()) ); @@ -1496,7 +1523,7 @@ public void flush(FlushRequest request) { * {@link org.opensearch.index.translog.TranslogDeletionPolicy} for details */ public void trimTranslog() { - if (isRemoteTranslogEnabled()) { + if (indexSettings.isRemoteNode()) { return; } verifyNotClosed(); @@ -1661,7 +1688,7 @@ public ReplicationCheckpoint getLatestReplicationCheckpoint() { * */ public Tuple, ReplicationCheckpoint> getLatestSegmentInfosAndCheckpoint() { - assert indexSettings.isSegRepEnabled(); + assert indexSettings.isSegRepEnabledOrRemoteNode(); // do not close the snapshot - caller will close it. GatedCloseable snapshot = null; @@ -1720,7 +1747,7 @@ ReplicationCheckpoint computeReplicationCheckpoint(SegmentInfos segmentInfos) th * @return - True if the shard is able to perform segment replication. */ public boolean isSegmentReplicationAllowed() { - if (indexSettings.isSegRepEnabled() == false) { + if (indexSettings.isSegRepEnabledOrRemoteNode() == false) { logger.trace("Attempting to perform segment replication when it is not enabled on the index"); return false; } @@ -2016,7 +2043,7 @@ public void close(String reason, boolean flushEngine, boolean deleted) throws IO ToDo : Fix this https://github.com/opensearch-project/OpenSearch/issues/8003 */ public RemoteSegmentStoreDirectory getRemoteDirectory() { - assert indexSettings.isRemoteStoreEnabled(); + assert indexSettings.isRemoteNode(); assert remoteStore.directory() instanceof FilterDirectory : "Store.directory is not an instance of FilterDirectory"; FilterDirectory remoteStoreDirectory = (FilterDirectory) remoteStore.directory(); FilterDirectory byteSizeCachingStoreDirectory = (FilterDirectory) remoteStoreDirectory.getDelegate(); @@ -2028,8 +2055,8 @@ public RemoteSegmentStoreDirectory getRemoteDirectory() { * Returns true iff it is able to verify that remote segment store * is in sync with local */ - boolean isRemoteSegmentStoreInSync() { - assert indexSettings.isRemoteStoreEnabled(); + public boolean isRemoteSegmentStoreInSync() { + assert indexSettings.isRemoteNode(); try { RemoteSegmentStoreDirectory directory = getRemoteDirectory(); if (directory.readLatestMetadataFile() != null) { @@ -2059,6 +2086,46 @@ boolean isRemoteSegmentStoreInSync() { return false; } + public void waitForRemoteStoreSync() { + waitForRemoteStoreSync(() -> {}); + } + + /* + Blocks the calling thread, waiting for the remote store to get synced till internal Remote Upload Timeout + Calls onProgress on seeing an increased file count on remote + */ + public void waitForRemoteStoreSync(Runnable onProgress) { + assert indexSettings.isRemoteNode(); + RemoteSegmentStoreDirectory directory = getRemoteDirectory(); + int segmentUploadeCount = 0; + if (shardRouting.primary() == false) { + return; + } + long startNanos = System.nanoTime(); + + while (System.nanoTime() - startNanos < getRecoverySettings().internalRemoteUploadTimeout().nanos()) { + try { + if (isRemoteSegmentStoreInSync()) { + break; + } else { + if (directory.getSegmentsUploadedToRemoteStore().size() > segmentUploadeCount) { + onProgress.run(); + logger.debug("Uploaded segment count {}", directory.getSegmentsUploadedToRemoteStore().size()); + segmentUploadeCount = directory.getSegmentsUploadedToRemoteStore().size(); + } + try { + Thread.sleep(TimeValue.timeValueSeconds(30).seconds()); + } catch (InterruptedException ie) { + throw new OpenSearchException("Interrupted waiting for completion of [{}]", ie); + } + } + } catch (AlreadyClosedException e) { + // There is no point in waiting as shard is now closed . + return; + } + } + } + public void preRecovery() { final IndexShardState currentState = this.state; // single volatile read if (currentState == IndexShardState.CLOSED) { @@ -2203,7 +2270,7 @@ public long recoverLocallyAndFetchStartSeqNo(boolean localTranslog) { * @return the starting sequence number from which the recovery should start. */ private long recoverLocallyUptoLastCommit() { - assert isRemoteTranslogEnabled() : "Remote translog store is not enabled"; + assert indexSettings.isRemoteNode() : "Remote translog store is not enabled"; long seqNo; validateLocalRecoveryState(); @@ -2449,7 +2516,7 @@ private void innerOpenEngineAndTranslog(LongSupplier globalCheckpointSupplier, b synchronized (engineMutex) { assert currentEngineReference.get() == null : "engine is running"; verifyNotClosed(); - if (indexSettings.isRemoteStoreEnabled()) { + if (indexSettings.isRemoteStoreEnabled() || this.isRemoteSeeded()) { // Download missing segments from remote segment store. if (syncFromRemote) { syncSegmentsFromRemoteSegmentStore(false); @@ -2488,7 +2555,7 @@ private void innerOpenEngineAndTranslog(LongSupplier globalCheckpointSupplier, b onNewEngine(newEngine); currentEngineReference.set(newEngine); - if (indexSettings.isSegRepEnabled()) { + if (indexSettings.isSegRepEnabledOrRemoteNode()) { // set initial replication checkpoints into tracker. updateReplicationCheckpoint(); } @@ -2900,7 +2967,7 @@ public Translog.Snapshot getHistoryOperations(String reason, long startingSeqNo, * This method should only be invoked if Segment Replication or Remote Store is not enabled. */ public Translog.Snapshot getHistoryOperationsFromTranslog(long startingSeqNo, long endSeqNo) throws IOException { - assert (indexSettings.isSegRepEnabled() || indexSettings.isRemoteStoreEnabled()) == false + assert indexSettings.isSegRepEnabledOrRemoteNode() == false : "unsupported operation for segment replication enabled indices or remote store backed indices"; return getEngine().translogManager().newChangesSnapshot(startingSeqNo, endSeqNo, true); } @@ -3067,7 +3134,7 @@ public Set getReplicationStatsForTrackedReplicas() } public ReplicationStats getReplicationStats() { - if (indexSettings.isSegRepEnabled() && routingEntry().primary()) { + if (indexSettings.isSegRepEnabledOrRemoteNode() && routingEntry().primary()) { final Set stats = getReplicationStatsForTrackedReplicas(); long maxBytesBehind = stats.stream().mapToLong(SegmentReplicationShardStats::getBytesBehindCount).max().orElse(0L); long totalBytesBehind = stats.stream().mapToLong(SegmentReplicationShardStats::getBytesBehindCount).sum(); @@ -3446,7 +3513,14 @@ public void activateWithPrimaryContext(final ReplicationTracker.PrimaryContext p + "] does not contain relocation target [" + routingEntry() + "]"; - assert getLocalCheckpoint() == primaryContext.getCheckpointStates().get(routingEntry().allocationId().getId()).getLocalCheckpoint() + String allocationId = routingEntry().allocationId().getId(); + if (isRemoteStoreEnabled() || isMigratingToRemote()) { + // For remote backed indexes, old primary may not have updated value of local checkpoint of new primary. + // But the new primary is always updated with data in remote sore and is at par with old primary. + // So, we can use a stricter check where local checkpoint of new primary is checked against that of old primary. + allocationId = primaryContext.getRoutingTable().primaryShard().allocationId().getId(); + } + assert getLocalCheckpoint() == primaryContext.getCheckpointStates().get(allocationId).getLocalCheckpoint() || indexSettings().getTranslogDurability() == Durability.ASYNC : "local checkpoint [" + getLocalCheckpoint() + "] does not match checkpoint from primary context [" @@ -3459,7 +3533,7 @@ assert getLocalCheckpoint() == primaryContext.getCheckpointStates().get(routingE } private void postActivatePrimaryMode() { - if (indexSettings.isRemoteStoreEnabled()) { + if (indexSettings.isRemoteNode()) { // We make sure to upload translog (even if it does not contain any operations) to remote translog. // This helps to get a consistent state in remote store where both remote segment store and remote // translog contains data. @@ -3846,14 +3920,14 @@ private EngineConfig newEngineConfig(LongSupplier globalCheckpointSupplier) thro internalRefreshListener.clear(); internalRefreshListener.add(new RefreshMetricUpdater(refreshMetric)); - if (indexSettings.isSegRepEnabled()) { + if (indexSettings.isSegRepEnabledOrRemoteNode()) { internalRefreshListener.add(new ReplicationCheckpointUpdater()); } if (this.checkpointPublisher != null && shardRouting.primary() && indexSettings.isSegRepLocalEnabled()) { internalRefreshListener.add(new CheckpointRefreshListener(this, this.checkpointPublisher)); } - if (isRemoteStoreEnabled()) { + if (isRemoteStoreEnabled() || isMigratingToRemote()) { internalRefreshListener.add( new RemoteStoreRefreshListener( this, @@ -3867,10 +3941,15 @@ private EngineConfig newEngineConfig(LongSupplier globalCheckpointSupplier) thro With segment replication enabled for primary relocation, recover replica shard initially as read only and change to a writeable engine during relocation handoff after a round of segment replication. */ - boolean isReadOnlyReplica = indexSettings.isSegRepEnabled() + boolean isReadOnlyReplica = indexSettings.isSegRepEnabledOrRemoteNode() && (shardRouting.primary() == false || (shardRouting.isRelocationTarget() && recoveryState.getStage() != RecoveryState.Stage.FINALIZE)); + // For mixed mode, when relocating from doc rep to remote node, we use a writeable engine + if (shouldSeedRemoteStore()) { + isReadOnlyReplica = false; + } + return this.engineConfigFactory.newEngineConfig( shardId, threadPool, @@ -3895,7 +3974,7 @@ private EngineConfig newEngineConfig(LongSupplier globalCheckpointSupplier) thro this::getOperationPrimaryTerm, tombstoneDocSupplier(), isReadOnlyReplica, - this::isStartedPrimary, + this::enableUploadToRemoteTranslog, translogFactorySupplier.apply(indexSettings, shardRouting), isTimeSeriesDescSortOptimizationEnabled() ? DataStream.TIMESERIES_LEAF_SORTER : null // DESC @timestamp default order for // timeseries @@ -3916,7 +3995,24 @@ public boolean isRemoteTranslogEnabled() { * translog uploads. */ public boolean isStartedPrimary() { - return getReplicationTracker().isPrimaryMode() && state() == IndexShardState.STARTED; + return (getReplicationTracker().isPrimaryMode() && state() == IndexShardState.STARTED); + } + + public boolean enableUploadToRemoteTranslog() { + return isStartedPrimary() || (shouldSeedRemoteStore() && hasOneRemoteSegmentSyncHappened()); + } + + private boolean hasOneRemoteSegmentSyncHappened() { + assert indexSettings.isRemoteNode(); + // We upload remote translog only after one remote segment upload in case of migration + RemoteSegmentStoreDirectory rd = getRemoteDirectory(); + AtomicBoolean segment_n_uploaded = new AtomicBoolean(false); + rd.getSegmentsUploadedToRemoteStore().forEach((key, value) -> { + if (key.startsWith("segments")) { + segment_n_uploaded.set(true); + } + }); + return segment_n_uploaded.get(); } /** @@ -4229,7 +4325,7 @@ private void innerAcquireReplicaOperationPermit( ); // With Segment Replication enabled, we never want to reset a replica's engine unless // it is promoted to primary. - if (currentGlobalCheckpoint < maxSeqNo && indexSettings.isSegRepEnabled() == false) { + if (currentGlobalCheckpoint < maxSeqNo && indexSettings.isSegRepEnabledOrRemoteNode() == false) { resetEngineToGlobalCheckpoint(); } else { getEngine().translogManager().rollTranslogGeneration(); @@ -4521,10 +4617,10 @@ public final boolean isSearchIdle() { public final boolean isSearchIdleSupported() { // If the index is remote store backed, then search idle is not supported. This is to ensure that async refresh // task continues to upload to remote store periodically. - if (isRemoteTranslogEnabled()) { + if (isRemoteTranslogEnabled() || indexSettings.isRemoteNode()) { return false; } - return indexSettings.isSegRepEnabled() == false || indexSettings.getNumberOfReplicas() == 0; + return indexSettings.isSegRepEnabledOrRemoteNode() == false || indexSettings.getNumberOfReplicas() == 0; } /** @@ -4786,10 +4882,10 @@ public void close() throws IOException { } }; IOUtils.close(currentEngineReference.getAndSet(readOnlyEngine)); - if (indexSettings.isRemoteStoreEnabled()) { + if (indexSettings.isRemoteStoreEnabled() || this.isRemoteSeeded()) { syncSegmentsFromRemoteSegmentStore(false); } - if (indexSettings.isRemoteTranslogStoreEnabled() && shardRouting.primary()) { + if ((indexSettings.isRemoteTranslogStoreEnabled() || this.isRemoteSeeded()) && shardRouting.primary()) { syncRemoteTranslogAndUpdateGlobalCheckpoint(); } newEngineReference.set(engineFactory.newReadWriteEngine(newEngineConfig(replicationTracker))); @@ -4808,7 +4904,9 @@ public void close() throws IOException { // of truth for translog, we play all translogs that exists locally. Otherwise, the recoverUpto happens upto global checkpoint. // We also replay all local translog ops with Segment replication, because on engine swap our local translog may // hold more ops than the global checkpoint. - long recoverUpto = this.isRemoteTranslogEnabled() || indexSettings().isSegRepEnabled() ? Long.MAX_VALUE : globalCheckpoint; + long recoverUpto = this.isRemoteTranslogEnabled() || indexSettings().isSegRepEnabledOrRemoteNode() + ? Long.MAX_VALUE + : globalCheckpoint; newEngineReference.get() .translogManager() .recoverFromTranslog(translogRunner, newEngineReference.get().getProcessedLocalCheckpoint(), recoverUpto); @@ -4837,6 +4935,16 @@ public void deleteTranslogFilesFromRemoteTranslog() throws IOException { RemoteFsTranslog.cleanup(repository, shardId, getThreadPool()); } + /* + Cleans up remote store and remote translog contents. + This is used in remote store migration, where we want to clean up all stale segment and translog data + and seed the remote store afresh + */ + public void deleteRemoteStoreContents() throws IOException { + deleteTranslogFilesFromRemoteTranslog(); + getRemoteDirectory().deleteStaleSegments(0); + } + public void syncTranslogFilesFromRemoteTranslog() throws IOException { TranslogFactory translogFactory = translogFactorySupplier.apply(indexSettings, shardRouting); assert translogFactory instanceof RemoteBlobStoreInternalTranslogFactory; @@ -4862,7 +4970,7 @@ public void syncSegmentsFromRemoteSegmentStore(boolean overrideLocal) throws IOE public void syncSegmentsFromRemoteSegmentStore(boolean overrideLocal, final Runnable onFileSync) throws IOException { boolean syncSegmentSuccess = false; long startTimeMs = System.currentTimeMillis(); - assert indexSettings.isRemoteStoreEnabled(); + assert indexSettings.isRemoteStoreEnabled() || this.isRemoteSeeded(); logger.trace("Downloading segments from remote segment store"); RemoteSegmentStoreDirectory remoteDirectory = getRemoteDirectory(); // We need to call RemoteSegmentStoreDirectory.init() in order to get latest metadata of the files that @@ -5127,4 +5235,20 @@ private TimeValue getRemoteTranslogUploadBufferInterval(Supplier clus public AsyncIOProcessor getTranslogSyncProcessor() { return translogSyncProcessor; } + + enum ShardMigrationState { + REMOTE_NON_MIGRATING, + REMOTE_MIGRATING_SEEDED, + REMOTE_MIGRATING_UNSEEDED, + DOCREP_NON_MIGRATING + } + + static ShardMigrationState getShardMigrationState(IndexSettings indexSettings, boolean shouldSeed) { + if (indexSettings.isRemoteNode() && indexSettings.isRemoteStoreEnabled()) { + return REMOTE_NON_MIGRATING; + } else if (indexSettings.isRemoteNode()) { + return shouldSeed ? REMOTE_MIGRATING_UNSEEDED : REMOTE_MIGRATING_SEEDED; + } + return ShardMigrationState.DOCREP_NON_MIGRATING; + } } diff --git a/server/src/main/java/org/opensearch/index/shard/RemoteStoreRefreshListener.java b/server/src/main/java/org/opensearch/index/shard/RemoteStoreRefreshListener.java index 7bb80b736693f..fb96102bc6094 100644 --- a/server/src/main/java/org/opensearch/index/shard/RemoteStoreRefreshListener.java +++ b/server/src/main/java/org/opensearch/index/shard/RemoteStoreRefreshListener.java @@ -528,7 +528,7 @@ private void initializeRemoteDirectoryOnTermUpdate() throws IOException { * @return true iff the shard is a started with primary mode true or it is local or snapshot recovery. */ private boolean isReadyForUpload() { - boolean isReady = indexShard.isStartedPrimary() || isLocalOrSnapshotRecovery(); + boolean isReady = indexShard.isStartedPrimary() || isLocalOrSnapshotRecovery() || indexShard.shouldSeedRemoteStore(); if (isReady == false) { StringBuilder sb = new StringBuilder("Skipped syncing segments with"); diff --git a/server/src/main/java/org/opensearch/index/shard/StoreRecovery.java b/server/src/main/java/org/opensearch/index/shard/StoreRecovery.java index 3faef2da05320..5f09b1a0802f3 100644 --- a/server/src/main/java/org/opensearch/index/shard/StoreRecovery.java +++ b/server/src/main/java/org/opensearch/index/shard/StoreRecovery.java @@ -38,13 +38,11 @@ import org.apache.lucene.index.NoMergePolicy; import org.apache.lucene.index.SegmentInfos; import org.apache.lucene.search.Sort; -import org.apache.lucene.store.AlreadyClosedException; import org.apache.lucene.store.Directory; import org.apache.lucene.store.FilterDirectory; import org.apache.lucene.store.IOContext; import org.apache.lucene.store.IndexInput; import org.opensearch.ExceptionsHelper; -import org.opensearch.OpenSearchException; import org.opensearch.action.StepListener; import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.cluster.metadata.MappingMetadata; @@ -194,7 +192,7 @@ void recoverFromLocalShards( // copied segments - we will also see them in stats etc. indexShard.getEngine().forceMerge(false, -1, false, false, false, UUIDs.randomBase64UUID()); if (indexShard.isRemoteTranslogEnabled() && indexShard.shardRouting.primary()) { - waitForRemoteStoreSync(indexShard); + indexShard.waitForRemoteStoreSync(); if (indexShard.isRemoteSegmentStoreInSync() == false) { throw new IndexShardRecoveryException( indexShard.shardId(), @@ -436,7 +434,7 @@ void recoverFromSnapshotAndRemoteStore( indexShard.getEngine().fillSeqNoGaps(indexShard.getPendingPrimaryTerm()); indexShard.finalizeRecovery(); if (indexShard.isRemoteTranslogEnabled() && indexShard.shardRouting.primary()) { - waitForRemoteStoreSync(indexShard); + indexShard.waitForRemoteStoreSync(); if (indexShard.isRemoteSegmentStoreInSync() == false) { listener.onFailure(new IndexShardRestoreFailedException(shardId, "Failed to upload to remote segment store")); return; @@ -722,7 +720,7 @@ private void restore( indexShard.getEngine().fillSeqNoGaps(indexShard.getPendingPrimaryTerm()); indexShard.finalizeRecovery(); if (indexShard.isRemoteTranslogEnabled() && indexShard.shardRouting.primary()) { - waitForRemoteStoreSync(indexShard); + indexShard.waitForRemoteStoreSync(); if (indexShard.isRemoteSegmentStoreInSync() == false) { listener.onFailure(new IndexShardRestoreFailedException(shardId, "Failed to upload to remote segment store")); return; @@ -796,31 +794,4 @@ private void bootstrap(final IndexShard indexShard, final Store store) throws IO ); store.associateIndexWithNewTranslog(translogUUID); } - - /* - Blocks the calling thread, waiting for the remote store to get synced till internal Remote Upload Timeout - */ - private void waitForRemoteStoreSync(IndexShard indexShard) { - if (indexShard.shardRouting.primary() == false) { - return; - } - long startNanos = System.nanoTime(); - - while (System.nanoTime() - startNanos < indexShard.getRecoverySettings().internalRemoteUploadTimeout().nanos()) { - try { - if (indexShard.isRemoteSegmentStoreInSync()) { - break; - } else { - try { - Thread.sleep(TimeValue.timeValueMinutes(1).seconds()); - } catch (InterruptedException ie) { - throw new OpenSearchException("Interrupted waiting for completion of [{}]", ie); - } - } - } catch (AlreadyClosedException e) { - // There is no point in waiting as shard is now closed . - return; - } - } - } } diff --git a/server/src/main/java/org/opensearch/index/store/Store.java b/server/src/main/java/org/opensearch/index/store/Store.java index 1930a37daa400..0992d86d6f0aa 100644 --- a/server/src/main/java/org/opensearch/index/store/Store.java +++ b/server/src/main/java/org/opensearch/index/store/Store.java @@ -385,7 +385,7 @@ public MetadataSnapshot getMetadata(SegmentInfos segmentInfos) throws IOExceptio * @return {@link Map} map file name to {@link StoreFileMetadata}. */ public Map getSegmentMetadataMap(SegmentInfos segmentInfos) throws IOException { - assert indexSettings.isSegRepEnabled(); + assert indexSettings.isSegRepEnabledOrRemoteNode(); failIfCorrupted(); try { return loadMetadata(segmentInfos, directory, logger, true).fileMetadata; @@ -893,7 +893,7 @@ public void beforeClose() { * @throws IOException when there is an IO error committing. */ public void commitSegmentInfos(SegmentInfos latestSegmentInfos, long maxSeqNo, long processedCheckpoint) throws IOException { - assert indexSettings.isSegRepEnabled(); + assert indexSettings.isSegRepEnabledOrRemoteNode() || indexSettings.isRemoteNode(); metadataLock.writeLock().lock(); try { final Map userData = new HashMap<>(latestSegmentInfos.getUserData()); diff --git a/server/src/main/java/org/opensearch/index/translog/RemoteFsTranslog.java b/server/src/main/java/org/opensearch/index/translog/RemoteFsTranslog.java index 7b969a37e4aa6..43eec01b2d365 100644 --- a/server/src/main/java/org/opensearch/index/translog/RemoteFsTranslog.java +++ b/server/src/main/java/org/opensearch/index/translog/RemoteFsTranslog.java @@ -334,7 +334,7 @@ private boolean prepareAndUpload(Long primaryTerm, Long generation) throws IOExc } private boolean upload(long primaryTerm, long generation, long maxSeqNo) throws IOException { - logger.trace("uploading translog for {} {}", primaryTerm, generation); + logger.trace("uploading translog for primary term {} generation {}", primaryTerm, generation); try ( TranslogCheckpointTransferSnapshot transferSnapshotProvider = new TranslogCheckpointTransferSnapshot.Builder( primaryTerm, diff --git a/server/src/main/java/org/opensearch/index/translog/Translog.java b/server/src/main/java/org/opensearch/index/translog/Translog.java index 9f877e87415dd..e78300e368099 100644 --- a/server/src/main/java/org/opensearch/index/translog/Translog.java +++ b/server/src/main/java/org/opensearch/index/translog/Translog.java @@ -525,7 +525,7 @@ TranslogWriter createWriter( tragedy, persistedSequenceNumberConsumer, bigArrays, - indexSettings.isRemoteTranslogStoreEnabled() + indexSettings.isRemoteNode() ); } catch (final IOException e) { throw new TranslogException(shardId, "failed to create new translog file", e); diff --git a/server/src/main/java/org/opensearch/indices/IndicesService.java b/server/src/main/java/org/opensearch/indices/IndicesService.java index 0e64894e6f708..9bc81c1826c2d 100644 --- a/server/src/main/java/org/opensearch/indices/IndicesService.java +++ b/server/src/main/java/org/opensearch/indices/IndicesService.java @@ -150,6 +150,7 @@ import org.opensearch.indices.replication.checkpoint.SegmentReplicationCheckpointPublisher; import org.opensearch.indices.replication.common.ReplicationType; import org.opensearch.node.Node; +import org.opensearch.node.remotestore.RemoteStoreNodeAttribute; import org.opensearch.plugins.IndexStorePlugin; import org.opensearch.plugins.PluginsService; import org.opensearch.repositories.RepositoriesService; @@ -201,6 +202,7 @@ import static org.opensearch.index.IndexService.IndexCreationContext.CREATE_INDEX; import static org.opensearch.index.IndexService.IndexCreationContext.METADATA_VERIFICATION; import static org.opensearch.index.query.AbstractQueryBuilder.parseInnerQueryBuilder; +import static org.opensearch.node.remotestore.RemoteStoreNodeAttribute.isRemoteDataAttributePresent; import static org.opensearch.search.SearchService.ALLOW_EXPENSIVE_QUERIES; /** @@ -503,7 +505,12 @@ protected void closeInternal() { this.allowExpensiveQueries = ALLOW_EXPENSIVE_QUERIES.get(clusterService.getSettings()); clusterService.getClusterSettings().addSettingsUpdateConsumer(ALLOW_EXPENSIVE_QUERIES, this::setAllowExpensiveQueries); this.remoteDirectoryFactory = remoteDirectoryFactory; - this.translogFactorySupplier = getTranslogFactorySupplier(repositoriesServiceSupplier, threadPool, remoteStoreStatsTrackerFactory); + this.translogFactorySupplier = getTranslogFactorySupplier( + repositoriesServiceSupplier, + threadPool, + remoteStoreStatsTrackerFactory, + settings + ); this.searchRequestStats = searchRequestStats; this.clusterDefaultRefreshInterval = CLUSTER_DEFAULT_INDEX_REFRESH_INTERVAL_SETTING.get(clusterService.getSettings()); clusterService.getClusterSettings() @@ -533,7 +540,8 @@ private void onRefreshIntervalUpdate(TimeValue clusterDefaultRefreshInterval) { private static BiFunction getTranslogFactorySupplier( Supplier repositoriesServiceSupplier, ThreadPool threadPool, - RemoteStoreStatsTrackerFactory remoteStoreStatsTrackerFactory + RemoteStoreStatsTrackerFactory remoteStoreStatsTrackerFactory, + Settings settings ) { return (indexSettings, shardRouting) -> { if (indexSettings.isRemoteTranslogStoreEnabled() && shardRouting.primary()) { @@ -543,6 +551,13 @@ private static BiFunction getTrans indexSettings.getRemoteStoreTranslogRepository(), remoteStoreStatsTrackerFactory.getRemoteTranslogTransferTracker(shardRouting.shardId()) ); + } else if (isRemoteDataAttributePresent(settings) && shardRouting.primary()) { + return new RemoteBlobStoreInternalTranslogFactory( + repositoriesServiceSupplier, + threadPool, + RemoteStoreNodeAttribute.getRemoteStoreTranslogRepo(indexSettings.getNodeSettings()), + remoteStoreStatsTrackerFactory.getRemoteTranslogTransferTracker(shardRouting.shardId()) + ); } return new InternalTranslogFactory(); }; @@ -932,7 +947,7 @@ private EngineFactory getEngineFactory(final IndexSettings idxSettings) { if (idxSettings.isRemoteSnapshot()) { return config -> new ReadOnlyEngine(config, new SeqNoStats(0, 0, 0), new TranslogStats(), true, Function.identity(), false); } - if (idxSettings.isSegRepEnabled()) { + if (idxSettings.isSegRepEnabledOrRemoteNode() || idxSettings.isRemoteNode()) { return new NRTReplicationEngineFactory(); } return new InternalEngineFactory(); @@ -1032,7 +1047,10 @@ public IndexShard createShard( globalCheckpointSyncer, retentionLeaseSyncer, checkpointPublisher, - remoteStoreStatsTrackerFactory + remoteStoreStatsTrackerFactory, + repositoriesService, + targetNode, + sourceNode ); indexShard.addShardFailureCallback(onShardFailure); indexShard.startRecovery(recoveryState, recoveryTargetService, recoveryListener, repositoriesService, mapping -> { diff --git a/server/src/main/java/org/opensearch/indices/recovery/MultiFileWriter.java b/server/src/main/java/org/opensearch/indices/recovery/MultiFileWriter.java index 29ee097d36cac..fac6924435cf3 100644 --- a/server/src/main/java/org/opensearch/indices/recovery/MultiFileWriter.java +++ b/server/src/main/java/org/opensearch/indices/recovery/MultiFileWriter.java @@ -161,7 +161,7 @@ private void innerWriteFileChunk(StoreFileMetadata fileMetadata, long position, + "] in " + Arrays.toString(store.directory().listAll()); // With Segment Replication, we will fsync after a full commit has been received. - if (store.indexSettings().isSegRepEnabled() == false) { + if (store.indexSettings().isSegRepEnabledOrRemoteNode() == false) { store.directory().sync(Collections.singleton(temporaryFileName)); } IndexOutput remove = removeOpenIndexOutputs(name); diff --git a/server/src/main/java/org/opensearch/indices/recovery/PeerRecoverySourceService.java b/server/src/main/java/org/opensearch/indices/recovery/PeerRecoverySourceService.java index cb2bedf00de99..30f517fda9931 100644 --- a/server/src/main/java/org/opensearch/indices/recovery/PeerRecoverySourceService.java +++ b/server/src/main/java/org/opensearch/indices/recovery/PeerRecoverySourceService.java @@ -377,7 +377,7 @@ private Tuple createRecovery request.targetNode(), recoverySettings, throttleTime -> shard.recoveryStats().addThrottleTime(throttleTime), - shard.isRemoteTranslogEnabled() + shard.isRemoteTranslogEnabled() || request.targetNode().isRemoteStoreNode() ); handler = RecoverySourceHandlerFactory.create(shard, recoveryTarget, request, recoverySettings); return Tuple.tuple(handler, recoveryTarget); diff --git a/server/src/main/java/org/opensearch/indices/recovery/PeerRecoveryTargetService.java b/server/src/main/java/org/opensearch/indices/recovery/PeerRecoveryTargetService.java index 4232d32987e86..227496f72f83d 100644 --- a/server/src/main/java/org/opensearch/indices/recovery/PeerRecoveryTargetService.java +++ b/server/src/main/java/org/opensearch/indices/recovery/PeerRecoveryTargetService.java @@ -189,7 +189,7 @@ public void beforeIndexShardClosed(ShardId shardId, @Nullable IndexShard indexSh public void startRecovery(final IndexShard indexShard, final DiscoveryNode sourceNode, final RecoveryListener listener) { // create a new recovery status, and process... final long recoveryId = onGoingRecoveries.start( - new RecoveryTarget(indexShard, sourceNode, listener), + new RecoveryTarget(indexShard, sourceNode, listener, threadPool), recoverySettings.activityTimeout() ); // we fork off quickly here and go async but this is called from the cluster state applier thread too and that can cause @@ -246,7 +246,7 @@ private void doRecovery(final long recoveryId, final StartRecoveryRequest preExi logger.trace("{} preparing shard for peer recovery", recoveryTarget.shardId()); indexShard.prepareForIndexRecovery(); final boolean hasRemoteSegmentStore = indexShard.indexSettings().isRemoteStoreEnabled(); - if (hasRemoteSegmentStore) { + if (hasRemoteSegmentStore || indexShard.isRemoteSeeded()) { // ToDo: This is a temporary mitigation to not fail the peer recovery flow in case there is // an exception while downloading segments from remote store. For remote backed indexes, we // plan to revamp this flow so that node-node segment copy will not happen. @@ -260,7 +260,8 @@ private void doRecovery(final long recoveryId, final StartRecoveryRequest preExi ); } } - final boolean hasRemoteTranslog = recoveryTarget.state().getPrimary() == false && indexShard.isRemoteTranslogEnabled(); + final boolean hasRemoteTranslog = recoveryTarget.state().getPrimary() == false + && indexShard.indexSettings().isRemoteNode(); final boolean hasNoTranslog = indexShard.indexSettings().isRemoteSnapshot(); final boolean verifyTranslog = (hasRemoteTranslog || hasNoTranslog || hasRemoteSegmentStore) == false; final long startingSeqNo = indexShard.recoverLocallyAndFetchStartSeqNo(!hasRemoteTranslog); diff --git a/server/src/main/java/org/opensearch/indices/recovery/RecoverySourceHandler.java b/server/src/main/java/org/opensearch/indices/recovery/RecoverySourceHandler.java index 7996c48b2b04b..abf9b1aaeb2cc 100644 --- a/server/src/main/java/org/opensearch/indices/recovery/RecoverySourceHandler.java +++ b/server/src/main/java/org/opensearch/indices/recovery/RecoverySourceHandler.java @@ -841,9 +841,11 @@ void finalizeRecovery(long targetLocalCheckpoint, long trimAboveSeqNo, ActionLis if (request.isPrimaryRelocation()) { logger.trace("performing relocation hand-off"); - final Runnable forceSegRepRunnable = shard.indexSettings().isSegRepEnabled() - ? recoveryTarget::forceSegmentFileSync - : () -> {}; + final Runnable forceSegRepRunnable = shard.indexSettings().isSegRepEnabledOrRemoteNode() + || (request.sourceNode().isRemoteStoreNode() && request.targetNode().isRemoteStoreNode()) + ? recoveryTarget::forceSegmentFileSync + : () -> {}; + // TODO: make relocated async // this acquires all IndexShard operation permits and will thus delay new recoveries until it is done cancellableThreads.execute( @@ -855,7 +857,7 @@ void finalizeRecovery(long targetLocalCheckpoint, long trimAboveSeqNo, ActionLis */ } else { // Force round of segment replication to update its checkpoint to primary's - if (shard.indexSettings().isSegRepEnabled()) { + if (shard.indexSettings().isSegRepEnabledOrRemoteNode()) { cancellableThreads.execute(recoveryTarget::forceSegmentFileSync); } } diff --git a/server/src/main/java/org/opensearch/indices/recovery/RecoverySourceHandlerFactory.java b/server/src/main/java/org/opensearch/indices/recovery/RecoverySourceHandlerFactory.java index ea13ca18bbfca..0ccb1ac2133cf 100644 --- a/server/src/main/java/org/opensearch/indices/recovery/RecoverySourceHandlerFactory.java +++ b/server/src/main/java/org/opensearch/indices/recovery/RecoverySourceHandlerFactory.java @@ -23,7 +23,8 @@ public static RecoverySourceHandler create( StartRecoveryRequest request, RecoverySettings recoverySettings ) { - boolean isReplicaRecoveryWithRemoteTranslog = request.isPrimaryRelocation() == false && shard.isRemoteTranslogEnabled(); + boolean isReplicaRecoveryWithRemoteTranslog = request.isPrimaryRelocation() == false + && (shard.isRemoteTranslogEnabled() || shard.isMigratingToRemote()); if (isReplicaRecoveryWithRemoteTranslog) { return new RemoteStorePeerRecoverySourceHandler( shard, diff --git a/server/src/main/java/org/opensearch/indices/recovery/RecoveryTarget.java b/server/src/main/java/org/opensearch/indices/recovery/RecoveryTarget.java index f3b5d0d790f83..16311d5d2cfb7 100644 --- a/server/src/main/java/org/opensearch/indices/recovery/RecoveryTarget.java +++ b/server/src/main/java/org/opensearch/indices/recovery/RecoveryTarget.java @@ -61,6 +61,7 @@ import org.opensearch.indices.replication.common.ReplicationListener; import org.opensearch.indices.replication.common.ReplicationLuceneIndex; import org.opensearch.indices.replication.common.ReplicationTarget; +import org.opensearch.threadpool.ThreadPool; import java.io.IOException; import java.nio.channels.FileChannel; @@ -87,16 +88,20 @@ public class RecoveryTarget extends ReplicationTarget implements RecoveryTargetH // latch that can be used to blockingly wait for RecoveryTarget to be closed private final CountDownLatch closedLatch = new CountDownLatch(1); + private final ThreadPool threadPool; + /** * Creates a new recovery target object that represents a recovery to the provided shard. * - * @param indexShard local shard where we want to recover to - * @param sourceNode source node of the recovery where we recover from - * @param listener called when recovery is completed/failed + * @param indexShard local shard where we want to recover to + * @param sourceNode source node of the recovery where we recover from + * @param listener called when recovery is completed/failed + * @param threadPool threadpool instance */ - public RecoveryTarget(IndexShard indexShard, DiscoveryNode sourceNode, ReplicationListener listener) { + public RecoveryTarget(IndexShard indexShard, DiscoveryNode sourceNode, ReplicationListener listener, ThreadPool threadPool) { super("recovery_status", indexShard, indexShard.recoveryState().getIndex(), listener); this.sourceNode = sourceNode; + this.threadPool = threadPool; indexShard.recoveryStats().incCurrentAsTarget(); final String tempFilePrefix = getPrefix() + UUIDs.randomBase64UUID() + "."; this.multiFileWriter = new MultiFileWriter(indexShard.store(), stateIndex, tempFilePrefix, logger, this::ensureRefCount); @@ -108,7 +113,7 @@ public RecoveryTarget(IndexShard indexShard, DiscoveryNode sourceNode, Replicati * @return a copy of this recovery target */ public RecoveryTarget retryCopy() { - return new RecoveryTarget(indexShard, sourceNode, listener); + return new RecoveryTarget(indexShard, sourceNode, listener, threadPool); } public String source() { @@ -209,6 +214,15 @@ public void prepareForTranslogOperations(int totalTranslogOps, ActionListener { indexShard.refresh("remote store migration"); }); + indexShard.waitForRemoteStoreSync(this::setLastAccessTime); + logger.info("Remote Store is now seeded for {}", indexShard.shardId()); + } return null; }); } @@ -360,7 +374,7 @@ public void cleanFiles( // Replicas for segment replication or remote snapshot indices do not create // their own commit points and therefore do not modify the commit user data // in their store. In these cases, reuse the primary's translog UUID. - final boolean reuseTranslogUUID = indexShard.indexSettings().isSegRepEnabled() + final boolean reuseTranslogUUID = indexShard.indexSettings().isSegRepEnabledOrRemoteNode() || indexShard.indexSettings().isRemoteSnapshot(); if (reuseTranslogUUID) { final String translogUUID = store.getMetadata().getCommitUserData().get(TRANSLOG_UUID_KEY); diff --git a/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationSourceService.java b/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationSourceService.java index 4062f9702fb3a..a393faabae0ea 100644 --- a/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationSourceService.java +++ b/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationSourceService.java @@ -175,7 +175,7 @@ public void clusterChanged(ClusterChangedEvent event) { // we need to ensure its state has cleared up in ongoing replications. if (event.routingTableChanged()) { for (IndexService indexService : indicesService) { - if (indexService.getIndexSettings().isSegRepEnabled()) { + if (indexService.getIndexSettings().isSegRepEnabledOrRemoteNode()) { for (IndexShard indexShard : indexService) { if (indexShard.routingEntry().primary()) { final IndexMetadata indexMetadata = indexService.getIndexSettings().getIndexMetadata(); @@ -221,7 +221,7 @@ protected void doClose() throws IOException { */ @Override public void beforeIndexShardClosed(ShardId shardId, @Nullable IndexShard indexShard, Settings indexSettings) { - if (indexShard != null && indexShard.indexSettings().isSegRepEnabled()) { + if (indexShard != null && indexShard.indexSettings().isSegRepEnabledOrRemoteNode()) { ongoingSegmentReplications.cancel(indexShard, "shard is closed"); } } @@ -231,7 +231,10 @@ public void beforeIndexShardClosed(ShardId shardId, @Nullable IndexShard indexSh */ @Override public void shardRoutingChanged(IndexShard indexShard, @Nullable ShardRouting oldRouting, ShardRouting newRouting) { - if (indexShard != null && indexShard.indexSettings().isSegRepEnabled() && oldRouting.primary() == false && newRouting.primary()) { + if (indexShard != null + && indexShard.indexSettings().isSegRepEnabledOrRemoteNode() + && oldRouting.primary() == false + && newRouting.primary()) { ongoingSegmentReplications.cancel(indexShard.routingEntry().allocationId().getId(), "Relocating primary shard."); } } diff --git a/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationTargetService.java b/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationTargetService.java index f28f829545d59..4942d39cfa48a 100644 --- a/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationTargetService.java +++ b/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationTargetService.java @@ -168,7 +168,8 @@ protected void doClose() throws IOException { public void clusterChanged(ClusterChangedEvent event) { if (event.routingTableChanged()) { for (IndexService indexService : indicesService) { - if (indexService.getIndexSettings().isSegRepEnabled() && event.indexRoutingTableChanged(indexService.index().getName())) { + if (indexService.getIndexSettings().isSegRepEnabledOrRemoteNode() + && event.indexRoutingTableChanged(indexService.index().getName())) { for (IndexShard shard : indexService) { if (shard.routingEntry().primary() == false) { // for this shard look up its primary routing, if it has completed a relocation trigger replication @@ -197,7 +198,7 @@ public void clusterChanged(ClusterChangedEvent event) { */ @Override public void beforeIndexShardClosed(ShardId shardId, @Nullable IndexShard indexShard, Settings indexSettings) { - if (indexShard != null && indexShard.indexSettings().isSegRepEnabled()) { + if (indexShard != null && indexShard.indexSettings().isSegRepEnabledOrRemoteNode()) { onGoingReplications.cancelForShard(indexShard.shardId(), "Shard closing"); latestReceivedCheckpoint.remove(shardId); } @@ -209,7 +210,7 @@ public void beforeIndexShardClosed(ShardId shardId, @Nullable IndexShard indexSh */ @Override public void afterIndexShardStarted(IndexShard indexShard) { - if (indexShard.indexSettings().isSegRepEnabled() && indexShard.routingEntry().primary() == false) { + if (indexShard.indexSettings().isSegRepEnabledOrRemoteNode() && indexShard.routingEntry().primary() == false) { processLatestReceivedCheckpoint(indexShard, Thread.currentThread()); } } @@ -219,7 +220,10 @@ public void afterIndexShardStarted(IndexShard indexShard) { */ @Override public void shardRoutingChanged(IndexShard indexShard, @Nullable ShardRouting oldRouting, ShardRouting newRouting) { - if (oldRouting != null && indexShard.indexSettings().isSegRepEnabled() && oldRouting.primary() == false && newRouting.primary()) { + if (oldRouting != null + && indexShard.indexSettings().isSegRepEnabledOrRemoteNode() + && oldRouting.primary() == false + && newRouting.primary()) { onGoingReplications.cancelForShard(indexShard.shardId(), "Shard has been promoted to primary"); latestReceivedCheckpoint.remove(indexShard.shardId()); } diff --git a/server/src/main/java/org/opensearch/node/remotestore/RemoteStoreNodeAttribute.java b/server/src/main/java/org/opensearch/node/remotestore/RemoteStoreNodeAttribute.java index 7575c6ff5fb34..a3bfe1195d8cc 100644 --- a/server/src/main/java/org/opensearch/node/remotestore/RemoteStoreNodeAttribute.java +++ b/server/src/main/java/org/opensearch/node/remotestore/RemoteStoreNodeAttribute.java @@ -168,6 +168,14 @@ public static boolean isRemoteClusterStateAttributePresent(Settings settings) { .isEmpty() == false; } + public static String getRemoteStoreSegmentRepo(Settings settings) { + return settings.get(Node.NODE_ATTRIBUTES.getKey() + RemoteStoreNodeAttribute.REMOTE_STORE_SEGMENT_REPOSITORY_NAME_ATTRIBUTE_KEY); + } + + public static String getRemoteStoreTranslogRepo(Settings settings) { + return settings.get(Node.NODE_ATTRIBUTES.getKey() + RemoteStoreNodeAttribute.REMOTE_STORE_TRANSLOG_REPOSITORY_NAME_ATTRIBUTE_KEY); + } + public static boolean isRemoteStoreClusterStateEnabled(Settings settings) { return RemoteClusterStateService.REMOTE_CLUSTER_STATE_ENABLED_SETTING.get(settings) && isRemoteClusterStateAttributePresent(settings); diff --git a/server/src/main/java/org/opensearch/snapshots/SnapshotShardsService.java b/server/src/main/java/org/opensearch/snapshots/SnapshotShardsService.java index 1c25d8c71f948..89f1ea142336e 100644 --- a/server/src/main/java/org/opensearch/snapshots/SnapshotShardsService.java +++ b/server/src/main/java/org/opensearch/snapshots/SnapshotShardsService.java @@ -381,7 +381,7 @@ private void snapshot( if (indexShard.routingEntry().primary() == false) { throw new IndexShardSnapshotFailedException(shardId, "snapshot should be performed only on primary"); } - if (indexShard.indexSettings().isSegRepEnabled() && indexShard.isPrimaryMode() == false) { + if (indexShard.indexSettings().isSegRepEnabledOrRemoteNode() && indexShard.isPrimaryMode() == false) { throw new IndexShardSnapshotFailedException( shardId, "snapshot triggered on a new primary following failover and cannot proceed until promotion is complete" diff --git a/server/src/test/java/org/opensearch/index/replication/IndexLevelReplicationTests.java b/server/src/test/java/org/opensearch/index/replication/IndexLevelReplicationTests.java index 33e08a482b9c3..ec1600094084a 100644 --- a/server/src/test/java/org/opensearch/index/replication/IndexLevelReplicationTests.java +++ b/server/src/test/java/org/opensearch/index/replication/IndexLevelReplicationTests.java @@ -142,7 +142,7 @@ public void run() { IndexShard replica = shards.addReplica(); Future future = shards.asyncRecoverReplica( replica, - (indexShard, node) -> new RecoveryTarget(indexShard, node, recoveryListener) { + (indexShard, node) -> new RecoveryTarget(indexShard, node, recoveryListener, threadPool) { @Override public void cleanFiles( int totalTranslogOps, @@ -223,17 +223,20 @@ public IndexResult index(Index op) throws IOException { }); thread.start(); IndexShard replica = shards.addReplica(); - Future fut = shards.asyncRecoverReplica(replica, (shard, node) -> new RecoveryTarget(shard, node, recoveryListener) { - @Override - public void prepareForTranslogOperations(int totalTranslogOps, ActionListener listener) { - try { - indexedOnPrimary.await(); - } catch (InterruptedException e) { - throw new AssertionError(e); + Future fut = shards.asyncRecoverReplica( + replica, + (shard, node) -> new RecoveryTarget(shard, node, recoveryListener, threadPool) { + @Override + public void prepareForTranslogOperations(int totalTranslogOps, ActionListener listener) { + try { + indexedOnPrimary.await(); + } catch (InterruptedException e) { + throw new AssertionError(e); + } + super.prepareForTranslogOperations(totalTranslogOps, listener); } - super.prepareForTranslogOperations(totalTranslogOps, listener); } - }); + ); fut.get(); recoveryDone.countDown(); thread.join(); diff --git a/server/src/test/java/org/opensearch/index/replication/RecoveryDuringReplicationTests.java b/server/src/test/java/org/opensearch/index/replication/RecoveryDuringReplicationTests.java index 17b5440ab5424..b891ac63378ac 100644 --- a/server/src/test/java/org/opensearch/index/replication/RecoveryDuringReplicationTests.java +++ b/server/src/test/java/org/opensearch/index/replication/RecoveryDuringReplicationTests.java @@ -72,6 +72,7 @@ import org.opensearch.indices.recovery.RecoveryState; import org.opensearch.indices.recovery.RecoveryTarget; import org.opensearch.indices.replication.common.ReplicationListener; +import org.opensearch.threadpool.ThreadPool; import java.io.IOException; import java.util.ArrayList; @@ -118,7 +119,8 @@ public void testIndexingDuringFileRecovery() throws Exception { indexShard, node, recoveryListener, - logger + logger, + threadPool ) ); @@ -482,7 +484,7 @@ protected EngineFactory getEngineFactory(ShardRouting routing) { AtomicBoolean recoveryDone = new AtomicBoolean(false); final Future recoveryFuture = shards.asyncRecoverReplica(newReplica, (indexShard, node) -> { recoveryStart.countDown(); - return new RecoveryTarget(indexShard, node, recoveryListener) { + return new RecoveryTarget(indexShard, node, recoveryListener, threadPool) { @Override public void finalizeRecovery(long globalCheckpoint, long trimAboveSeqNo, ActionListener listener) { recoveryDone.set(true); @@ -536,7 +538,7 @@ protected EngineFactory getEngineFactory(final ShardRouting routing) { final IndexShard replica = shards.addReplica(); final Future recoveryFuture = shards.asyncRecoverReplica( replica, - (indexShard, node) -> new RecoveryTarget(indexShard, node, recoveryListener) { + (indexShard, node) -> new RecoveryTarget(indexShard, node, recoveryListener, threadPool) { @Override public void indexTranslogOperations( final List operations, @@ -812,9 +814,10 @@ public BlockingTarget( IndexShard shard, DiscoveryNode sourceNode, ReplicationListener listener, - Logger logger + Logger logger, + ThreadPool threadPool ) { - super(shard, sourceNode, listener); + super(shard, sourceNode, listener, threadPool); this.recoveryBlocked = recoveryBlocked; this.releaseRecovery = releaseRecovery; this.stageToBlock = stageToBlock; diff --git a/server/src/test/java/org/opensearch/index/shard/IndexShardTests.java b/server/src/test/java/org/opensearch/index/shard/IndexShardTests.java index 46be10ce62840..537bfcf8f8a6b 100644 --- a/server/src/test/java/org/opensearch/index/shard/IndexShardTests.java +++ b/server/src/test/java/org/opensearch/index/shard/IndexShardTests.java @@ -3208,7 +3208,7 @@ public void testTranslogRecoverySyncsTranslog() throws IOException { indexDoc(primary, "_doc", "0", "{\"foo\" : \"bar\"}"); IndexShard replica = newShard(primary.shardId(), false, "n2", metadata, null); - recoverReplica(replica, primary, (shard, discoveryNode) -> new RecoveryTarget(shard, discoveryNode, recoveryListener) { + recoverReplica(replica, primary, (shard, discoveryNode) -> new RecoveryTarget(shard, discoveryNode, recoveryListener, threadPool) { @Override public void indexTranslogOperations( final List operations, @@ -3340,7 +3340,7 @@ public void testShardActiveDuringPeerRecovery() throws IOException { replica.markAsRecovering("for testing", new RecoveryState(replica.routingEntry(), localNode, localNode)); // Shard is still inactive since we haven't started recovering yet assertFalse(replica.isActive()); - recoverReplica(replica, primary, (shard, discoveryNode) -> new RecoveryTarget(shard, discoveryNode, recoveryListener) { + recoverReplica(replica, primary, (shard, discoveryNode) -> new RecoveryTarget(shard, discoveryNode, recoveryListener, threadPool) { @Override public void indexTranslogOperations( final List operations, @@ -3397,7 +3397,7 @@ public void testRefreshListenersDuringPeerRecovery() throws IOException { DiscoveryNode localNode = new DiscoveryNode("foo", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT); replica.markAsRecovering("for testing", new RecoveryState(replica.routingEntry(), localNode, localNode)); assertListenerCalled.accept(replica); - recoverReplica(replica, primary, (shard, discoveryNode) -> new RecoveryTarget(shard, discoveryNode, recoveryListener) { + recoverReplica(replica, primary, (shard, discoveryNode) -> new RecoveryTarget(shard, discoveryNode, recoveryListener, threadPool) { // we're only checking that listeners are called when the engine is open, before there is no point @Override public void prepareForTranslogOperations(int totalTranslogOps, ActionListener listener) { diff --git a/server/src/test/java/org/opensearch/index/shard/ReplicaRecoveryWithRemoteTranslogOnPrimaryTests.java b/server/src/test/java/org/opensearch/index/shard/ReplicaRecoveryWithRemoteTranslogOnPrimaryTests.java index 4f5cad70fd643..85864eebd6d0d 100644 --- a/server/src/test/java/org/opensearch/index/shard/ReplicaRecoveryWithRemoteTranslogOnPrimaryTests.java +++ b/server/src/test/java/org/opensearch/index/shard/ReplicaRecoveryWithRemoteTranslogOnPrimaryTests.java @@ -86,7 +86,7 @@ public void testStartSequenceForReplicaRecovery() throws Exception { ); shards.addReplica(newReplicaShard); AtomicBoolean assertDone = new AtomicBoolean(false); - shards.recoverReplica(newReplicaShard, (r, sourceNode) -> new RecoveryTarget(r, sourceNode, recoveryListener) { + shards.recoverReplica(newReplicaShard, (r, sourceNode) -> new RecoveryTarget(r, sourceNode, recoveryListener, threadPool) { @Override public IndexShard indexShard() { IndexShard idxShard = super.indexShard(); diff --git a/server/src/test/java/org/opensearch/index/shard/SegmentReplicationIndexShardTests.java b/server/src/test/java/org/opensearch/index/shard/SegmentReplicationIndexShardTests.java index 7caff3e5f5479..e93d266dcab4c 100644 --- a/server/src/test/java/org/opensearch/index/shard/SegmentReplicationIndexShardTests.java +++ b/server/src/test/java/org/opensearch/index/shard/SegmentReplicationIndexShardTests.java @@ -256,7 +256,7 @@ public void onDone(ReplicationState state) { public void onFailure(ReplicationState state, ReplicationFailedException e, boolean sendShardFailure) { assertEquals(ExceptionsHelper.unwrap(e, IOException.class).getMessage(), "Expected failure"); } - }), + }, threadPool), true, true, replicatePrimaryFunction diff --git a/server/src/test/java/org/opensearch/index/translog/RemoteFsTranslogTests.java b/server/src/test/java/org/opensearch/index/translog/RemoteFsTranslogTests.java index a83e737dc25c1..7ff4c3ecf5236 100644 --- a/server/src/test/java/org/opensearch/index/translog/RemoteFsTranslogTests.java +++ b/server/src/test/java/org/opensearch/index/translog/RemoteFsTranslogTests.java @@ -219,8 +219,9 @@ private TranslogConfig getTranslogConfig(final Path path, final Settings setting new ByteSizeValue(8, ByteSizeUnit.KB), new ByteSizeValue(10 + randomInt(128 * 1024), ByteSizeUnit.BYTES) ); - - final IndexSettings indexSettings = IndexSettingsModule.newIndexSettings(shardId.getIndex(), settings); + // To simulate that the node is remote backed + Settings nodeSettings = Settings.builder().put("node.attr.remote_store.translog.repository", "my-repo-1").build(); + final IndexSettings indexSettings = IndexSettingsModule.newIndexSettings(shardId.getIndex(), settings, nodeSettings); return new TranslogConfig(shardId, path, indexSettings, NON_RECYCLING_INSTANCE, bufferSize, ""); } diff --git a/server/src/test/java/org/opensearch/indices/IndicesLifecycleListenerSingleNodeTests.java b/server/src/test/java/org/opensearch/indices/IndicesLifecycleListenerSingleNodeTests.java index 5e6398da6fa1b..0e16e81b1bb70 100644 --- a/server/src/test/java/org/opensearch/indices/IndicesLifecycleListenerSingleNodeTests.java +++ b/server/src/test/java/org/opensearch/indices/IndicesLifecycleListenerSingleNodeTests.java @@ -149,22 +149,26 @@ public void afterIndexRemoved(Index index, IndexSettings indexSettings, IndexRem newRouting = newRouting.moveToUnassigned(unassignedInfo) .updateUnassigned(unassignedInfo, RecoverySource.EmptyStoreRecoverySource.INSTANCE); newRouting = ShardRoutingHelper.initialize(newRouting, nodeId); + final DiscoveryNode localNode = new DiscoveryNode( + "foo", + buildNewFakeTransportAddress(), + emptyMap(), + emptySet(), + Version.CURRENT + ); IndexShard shard = index.createShard( newRouting, s -> {}, RetentionLeaseSyncer.EMPTY, SegmentReplicationCheckpointPublisher.EMPTY, + null, + null, + localNode, null ); IndexShardTestCase.updateRoutingEntry(shard, newRouting); assertEquals(5, counter.get()); - final DiscoveryNode localNode = new DiscoveryNode( - "foo", - buildNewFakeTransportAddress(), - emptyMap(), - emptySet(), - Version.CURRENT - ); + shard.markAsRecovering("store", new RecoveryState(newRouting, localNode, null)); IndexShardTestCase.recoverFromStore(shard); newRouting = ShardRoutingHelper.moveToStarted(newRouting); diff --git a/server/src/test/java/org/opensearch/indices/recovery/PeerRecoveryTargetServiceTests.java b/server/src/test/java/org/opensearch/indices/recovery/PeerRecoveryTargetServiceTests.java index 34f854cae56ba..1e6cc43703672 100644 --- a/server/src/test/java/org/opensearch/indices/recovery/PeerRecoveryTargetServiceTests.java +++ b/server/src/test/java/org/opensearch/indices/recovery/PeerRecoveryTargetServiceTests.java @@ -95,7 +95,7 @@ public void testWriteFileChunksConcurrently() throws Exception { final DiscoveryNode pNode = getFakeDiscoNode(sourceShard.routingEntry().currentNodeId()); final DiscoveryNode rNode = getFakeDiscoNode(targetShard.routingEntry().currentNodeId()); targetShard.markAsRecovering("test-peer-recovery", new RecoveryState(targetShard.routingEntry(), rNode, pNode)); - final RecoveryTarget recoveryTarget = new RecoveryTarget(targetShard, null, null); + final RecoveryTarget recoveryTarget = new RecoveryTarget(targetShard, null, null, threadPool); final PlainActionFuture receiveFileInfoFuture = new PlainActionFuture<>(); recoveryTarget.receiveFileInfo( mdFiles.stream().map(StoreFileMetadata::name).collect(Collectors.toList()), @@ -355,7 +355,7 @@ public void testResetStartingSeqNoIfLastCommitCorrupted() throws Exception { shard.prepareForIndexRecovery(); long startingSeqNo = shard.recoverLocallyAndFetchStartSeqNo(true); shard.store().markStoreCorrupted(new IOException("simulated")); - RecoveryTarget recoveryTarget = new RecoveryTarget(shard, null, null); + RecoveryTarget recoveryTarget = new RecoveryTarget(shard, null, null, threadPool); StartRecoveryRequest request = PeerRecoveryTargetService.getStartRecoveryRequest(logger, rNode, recoveryTarget, startingSeqNo); assertThat(request.startingSeqNo(), equalTo(UNASSIGNED_SEQ_NO)); assertThat(request.metadataSnapshot().size(), equalTo(0)); @@ -396,7 +396,7 @@ public void testResetStartRequestIfTranslogIsCorrupted() throws Exception { shard = reinitShard(shard, ShardRoutingHelper.initWithSameId(shard.routingEntry(), RecoverySource.PeerRecoverySource.INSTANCE)); shard.markAsRecovering("peer recovery", new RecoveryState(shard.routingEntry(), pNode, rNode)); shard.prepareForIndexRecovery(); - RecoveryTarget recoveryTarget = new RecoveryTarget(shard, null, null); + RecoveryTarget recoveryTarget = new RecoveryTarget(shard, null, null, threadPool); StartRecoveryRequest request = PeerRecoveryTargetService.getStartRecoveryRequest( logger, rNode, diff --git a/server/src/test/java/org/opensearch/indices/recovery/RecoveryTests.java b/server/src/test/java/org/opensearch/indices/recovery/RecoveryTests.java index ad90255a3cc3f..71d89e2856c6e 100644 --- a/server/src/test/java/org/opensearch/indices/recovery/RecoveryTests.java +++ b/server/src/test/java/org/opensearch/indices/recovery/RecoveryTests.java @@ -137,7 +137,8 @@ public void testRetentionPolicyChangeDuringRecovery() throws Exception { indexShard, node, recoveryListener, - logger + logger, + threadPool ) ); recoveryBlocked.await(); @@ -348,7 +349,7 @@ public void testPeerRecoverySendSafeCommitInFileBased() throws Exception { } IndexShard replicaShard = newShard(primaryShard.shardId(), false); updateMappings(replicaShard, primaryShard.indexSettings().getIndexMetadata()); - recoverReplica(replicaShard, primaryShard, (r, sourceNode) -> new RecoveryTarget(r, sourceNode, recoveryListener) { + recoverReplica(replicaShard, primaryShard, (r, sourceNode) -> new RecoveryTarget(r, sourceNode, recoveryListener, threadPool) { @Override public void prepareForTranslogOperations(int totalTranslogOps, ActionListener listener) { super.prepareForTranslogOperations(totalTranslogOps, listener); @@ -480,7 +481,7 @@ public void onDone(ReplicationState state) { public void onFailure(ReplicationState state, ReplicationFailedException e, boolean sendShardFailure) { assertThat(ExceptionsHelper.unwrap(e, IOException.class).getMessage(), equalTo("simulated")); } - })) + }, threadPool)) ); expectThrows(AlreadyClosedException.class, () -> replica.refresh("test")); group.removeReplica(replica); diff --git a/server/src/test/java/org/opensearch/recovery/ReplicationCollectionTests.java b/server/src/test/java/org/opensearch/recovery/ReplicationCollectionTests.java index fb4dc97435512..4ce4e28690697 100644 --- a/server/src/test/java/org/opensearch/recovery/ReplicationCollectionTests.java +++ b/server/src/test/java/org/opensearch/recovery/ReplicationCollectionTests.java @@ -225,6 +225,6 @@ long startRecovery( final DiscoveryNode rNode = getDiscoveryNode(indexShard.routingEntry().currentNodeId()); indexShard.markAsRecovering("remote", new RecoveryState(indexShard.routingEntry(), sourceNode, rNode)); indexShard.prepareForIndexRecovery(); - return collection.start(new RecoveryTarget(indexShard, sourceNode, listener), timeValue); + return collection.start(new RecoveryTarget(indexShard, sourceNode, listener, threadPool), timeValue); } } diff --git a/test/framework/src/main/java/org/opensearch/index/replication/OpenSearchIndexLevelReplicationTestCase.java b/test/framework/src/main/java/org/opensearch/index/replication/OpenSearchIndexLevelReplicationTestCase.java index 9800782272ede..e6e20ce8f8566 100644 --- a/test/framework/src/main/java/org/opensearch/index/replication/OpenSearchIndexLevelReplicationTestCase.java +++ b/test/framework/src/main/java/org/opensearch/index/replication/OpenSearchIndexLevelReplicationTestCase.java @@ -520,7 +520,7 @@ public synchronized boolean removeReplica(IndexShard replica) throws IOException } public void recoverReplica(IndexShard replica) throws IOException { - recoverReplica(replica, (r, sourceNode) -> new RecoveryTarget(r, sourceNode, recoveryListener)); + recoverReplica(replica, (r, sourceNode) -> new RecoveryTarget(r, sourceNode, recoveryListener, threadPool)); } public void recoverReplica(IndexShard replica, BiFunction targetSupplier) diff --git a/test/framework/src/main/java/org/opensearch/index/shard/IndexShardTestCase.java b/test/framework/src/main/java/org/opensearch/index/shard/IndexShardTestCase.java index bf1c4d4c94e04..a2f9eb677c0ac 100644 --- a/test/framework/src/main/java/org/opensearch/index/shard/IndexShardTestCase.java +++ b/test/framework/src/main/java/org/opensearch/index/shard/IndexShardTestCase.java @@ -617,7 +617,14 @@ protected IndexShard newShard( @Nullable Path remotePath, IndexingOperationListener... listeners ) throws IOException { - final Settings nodeSettings = Settings.builder().put("node.name", routing.currentNodeId()).build(); + Settings nodeSettings = Settings.builder().put("node.name", routing.currentNodeId()).build(); + // To simulate that the node is remote backed + if (indexMetadata.getSettings().get(IndexMetadata.SETTING_REMOTE_STORE_ENABLED) == "true") { + nodeSettings = Settings.builder() + .put("node.name", routing.currentNodeId()) + .put("node.attr.remote_store.translog.repository", "seg_repo") + .build(); + } final IndexSettings indexSettings = new IndexSettings(indexMetadata, nodeSettings); final IndexShard indexShard; if (storeProvider == null) { @@ -646,7 +653,7 @@ protected IndexShard newShard( RemoteStoreStatsTrackerFactory remoteStoreStatsTrackerFactory = null; RepositoriesService mockRepoSvc = mock(RepositoriesService.class); - if (indexSettings.isRemoteStoreEnabled()) { + if (indexSettings.isRemoteStoreEnabled() || indexSettings.isRemoteNode()) { String remoteStoreRepository = indexSettings.getRemoteStoreRepository(); // remote path via setting a repository . This is a hack used for shards are created using reset . // since we can't get remote path from IndexShard directly, we are using repository to store it . @@ -703,7 +710,8 @@ protected IndexShard newShard( remoteStoreStatsTrackerFactory, () -> IndexSettings.DEFAULT_REMOTE_TRANSLOG_BUFFER_INTERVAL, "dummy-node", - DefaultRecoverySettings.INSTANCE + DefaultRecoverySettings.INSTANCE, + false ); indexShard.addShardFailureCallback(DEFAULT_SHARD_FAILURE_HANDLER); if (remoteStoreStatsTrackerFactory != null) { @@ -1001,7 +1009,7 @@ public static void updateRoutingEntry(IndexShard shard, ShardRouting shardRoutin protected void recoveryEmptyReplica(IndexShard replica, boolean startReplica) throws IOException { IndexShard primary = null; try { - primary = newStartedShard(true); + primary = newStartedShard(true, replica.indexSettings.getSettings()); recoverReplica(replica, primary, startReplica); } finally { closeShards(primary); @@ -1033,7 +1041,7 @@ protected void recoverReplica( recoverReplica( replica, primary, - (r, sourceNode) -> new RecoveryTarget(r, sourceNode, recoveryListener), + (r, sourceNode) -> new RecoveryTarget(r, sourceNode, recoveryListener, threadPool), true, startReplica, replicatePrimaryFunction @@ -1051,7 +1059,7 @@ protected void recoverReplica( } public Function, List> getReplicationFunc(final IndexShard target) { - return target.indexSettings().isSegRepEnabled() ? (shardList) -> { + return target.indexSettings().isSegRepEnabledOrRemoteNode() ? (shardList) -> { try { assert shardList.size() >= 2; final IndexShard primary = shardList.get(0); @@ -1489,7 +1497,7 @@ private SegmentReplicationTargetService prepareForReplication( SegmentReplicationSourceFactory sourceFactory = null; SegmentReplicationTargetService targetService; - if (primaryShard.indexSettings.isRemoteStoreEnabled()) { + if (primaryShard.indexSettings.isRemoteStoreEnabled() || primaryShard.indexSettings.isRemoteNode()) { RecoverySettings recoverySettings = new RecoverySettings( Settings.EMPTY, new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS) From 2069bd805804dd93bd69695a4c6521cc6f2b9bb6 Mon Sep 17 00:00:00 2001 From: rajiv-kv <157019998+rajiv-kv@users.noreply.github.com> Date: Thu, 21 Mar 2024 14:42:58 +0530 Subject: [PATCH 22/38] Integrate with CPU admission controller for cluster-manager Read API's. (#12496) * Integrate with CPU admission controller for cluster-manager Read API's. The admission control is enforced at the transport layer. Signed-off-by: Rajiv Kumar Vaidyanathan --- CHANGELOG.md | 1 + .../AdmissionForClusterManagerIT.java | 198 ++++++++++++++++++ .../support/HandledTransportAction.java | 37 +++- .../TransportClusterManagerNodeAction.java | 29 ++- ...TransportClusterManagerNodeReadAction.java | 38 +++- .../common/settings/ClusterSettings.java | 7 +- .../CpuBasedAdmissionController.java | 5 +- .../IoBasedAdmissionController.java | 5 +- .../enums/AdmissionControlActionType.java | 5 +- .../CpuBasedAdmissionControllerSettings.java | 21 ++ .../IoBasedAdmissionControllerSettings.java | 18 ++ .../transport/TransportService.java | 6 +- ...BasedAdmissionControllerSettingsTests.java | 32 ++- ...BasedAdmissionControllerSettingsTests.java | 16 ++ 14 files changed, 405 insertions(+), 13 deletions(-) create mode 100644 server/src/internalClusterTest/java/org/opensearch/ratelimitting/admissioncontrol/AdmissionForClusterManagerIT.java diff --git a/CHANGELOG.md b/CHANGELOG.md index c65b20b99bc25..9dced730cbcad 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -121,6 +121,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Introduce a new setting `index.check_pending_flush.enabled` to expose the ability to disable the check for pending flushes by write threads ([#12710](https://github.com/opensearch-project/OpenSearch/pull/12710)) - Built-in secure transports support ([#12435](https://github.com/opensearch-project/OpenSearch/pull/12435)) - Lightweight Transport action to verify local term before fetching cluster-state from remote ([#12252](https://github.com/opensearch-project/OpenSearch/pull/12252/)) +- Integrate with admission controller for cluster-manager Read API. ([#12496](https://github.com/opensearch-project/OpenSearch/pull/12496)) ### Dependencies - Bump `peter-evans/find-comment` from 2 to 3 ([#12288](https://github.com/opensearch-project/OpenSearch/pull/12288)) diff --git a/server/src/internalClusterTest/java/org/opensearch/ratelimitting/admissioncontrol/AdmissionForClusterManagerIT.java b/server/src/internalClusterTest/java/org/opensearch/ratelimitting/admissioncontrol/AdmissionForClusterManagerIT.java new file mode 100644 index 0000000000000..4d1964326820e --- /dev/null +++ b/server/src/internalClusterTest/java/org/opensearch/ratelimitting/admissioncontrol/AdmissionForClusterManagerIT.java @@ -0,0 +1,198 @@ +/* + * 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.ratelimitting.admissioncontrol; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.opensearch.action.admin.indices.alias.get.GetAliasesRequest; +import org.opensearch.action.admin.indices.alias.get.GetAliasesResponse; +import org.opensearch.client.node.NodeClient; +import org.opensearch.common.settings.Settings; +import org.opensearch.common.unit.TimeValue; +import org.opensearch.core.concurrency.OpenSearchRejectedExecutionException; +import org.opensearch.core.rest.RestStatus; +import org.opensearch.node.IoUsageStats; +import org.opensearch.node.ResourceUsageCollectorService; +import org.opensearch.node.resource.tracker.ResourceTrackerSettings; +import org.opensearch.ratelimitting.admissioncontrol.controllers.CpuBasedAdmissionController; +import org.opensearch.ratelimitting.admissioncontrol.enums.AdmissionControlActionType; +import org.opensearch.ratelimitting.admissioncontrol.enums.AdmissionControlMode; +import org.opensearch.ratelimitting.admissioncontrol.stats.AdmissionControllerStats; +import org.opensearch.rest.AbstractRestChannel; +import org.opensearch.rest.RestResponse; +import org.opensearch.rest.action.admin.indices.RestGetAliasesAction; +import org.opensearch.test.OpenSearchIntegTestCase; +import org.opensearch.test.rest.FakeRestRequest; +import org.junit.Before; + +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.atomic.AtomicReference; + +import static org.opensearch.ratelimitting.admissioncontrol.AdmissionControlSettings.ADMISSION_CONTROL_TRANSPORT_LAYER_MODE; +import static org.opensearch.ratelimitting.admissioncontrol.settings.CpuBasedAdmissionControllerSettings.CLUSTER_ADMIN_CPU_USAGE_LIMIT; +import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertAcked; +import static org.hamcrest.Matchers.equalTo; + +@OpenSearchIntegTestCase.ClusterScope(scope = OpenSearchIntegTestCase.Scope.TEST, numDataNodes = 0) +public class AdmissionForClusterManagerIT extends OpenSearchIntegTestCase { + + private static final Logger LOGGER = LogManager.getLogger(AdmissionForClusterManagerIT.class); + + public static final String INDEX_NAME = "test_index"; + + private String clusterManagerNodeId; + private String datanode; + private ResourceUsageCollectorService cMResourceCollector; + + private static final Settings DISABLE_ADMISSION_CONTROL = Settings.builder() + .put(ADMISSION_CONTROL_TRANSPORT_LAYER_MODE.getKey(), AdmissionControlMode.DISABLED.getMode()) + .build(); + + private static final Settings ENFORCE_ADMISSION_CONTROL = Settings.builder() + .put(ResourceTrackerSettings.GLOBAL_CPU_USAGE_AC_WINDOW_DURATION_SETTING.getKey(), TimeValue.timeValueMillis(500)) + .put(ADMISSION_CONTROL_TRANSPORT_LAYER_MODE.getKey(), AdmissionControlMode.ENFORCED) + .put(CLUSTER_ADMIN_CPU_USAGE_LIMIT.getKey(), 50) + .build(); + + @Before + public void init() { + String clusterManagerNode = internalCluster().startClusterManagerOnlyNode( + Settings.builder().put(DISABLE_ADMISSION_CONTROL).build() + ); + datanode = internalCluster().startDataOnlyNode(Settings.builder().put(DISABLE_ADMISSION_CONTROL).build()); + + ensureClusterSizeConsistency(); + ensureGreen(); + + // Disable the automatic resource collection + clusterManagerNodeId = internalCluster().clusterService(clusterManagerNode).localNode().getId(); + cMResourceCollector = internalCluster().getClusterManagerNodeInstance(ResourceUsageCollectorService.class); + cMResourceCollector.stop(); + + // Enable admission control + client().admin().cluster().prepareUpdateSettings().setTransientSettings(ENFORCE_ADMISSION_CONTROL).execute().actionGet(); + } + + public void testAdmissionControlEnforced() throws Exception { + cMResourceCollector.collectNodeResourceUsageStats(clusterManagerNodeId, System.currentTimeMillis(), 97, 99, new IoUsageStats(98)); + + // Write API on ClusterManager + assertAcked(prepareCreate("test").setMapping("field", "type=text").setAliases("{\"alias1\" : {}}")); + + // Read API on ClusterManager + GetAliasesRequest aliasesRequest = new GetAliasesRequest(); + aliasesRequest.aliases("alias1"); + try { + dataNodeClient().admin().indices().getAliases(aliasesRequest).actionGet(); + fail("expected failure"); + } catch (Exception e) { + assertTrue(e instanceof OpenSearchRejectedExecutionException); + assertTrue(e.getMessage().contains("CPU usage admission controller rejected the request")); + assertTrue(e.getMessage().contains("[indices:admin/aliases/get]")); + assertTrue(e.getMessage().contains("action-type [CLUSTER_ADMIN]")); + } + + client().admin().cluster().prepareUpdateSettings().setTransientSettings(DISABLE_ADMISSION_CONTROL).execute().actionGet(); + GetAliasesResponse getAliasesResponse = dataNodeClient().admin().indices().getAliases(aliasesRequest).actionGet(); + assertThat(getAliasesResponse.getAliases().get("test").size(), equalTo(1)); + + AdmissionControlService admissionControlServiceCM = internalCluster().getClusterManagerNodeInstance(AdmissionControlService.class); + + AdmissionControllerStats admissionStats = getAdmissionControlStats(admissionControlServiceCM).get( + CpuBasedAdmissionController.CPU_BASED_ADMISSION_CONTROLLER + ); + + assertEquals(admissionStats.rejectionCount.get(AdmissionControlActionType.CLUSTER_ADMIN.getType()).longValue(), 1); + assertNull(admissionStats.rejectionCount.get(AdmissionControlActionType.SEARCH.getType())); + assertNull(admissionStats.rejectionCount.get(AdmissionControlActionType.INDEXING.getType())); + } + + public void testAdmissionControlEnabledOnNoBreach() throws InterruptedException { + // CPU usage is less than threshold 50% + cMResourceCollector.collectNodeResourceUsageStats(clusterManagerNodeId, System.currentTimeMillis(), 97, 35, new IoUsageStats(98)); + + // Write API on ClusterManager + assertAcked(prepareCreate("test").setMapping("field", "type=text").setAliases("{\"alias1\" : {}}").execute().actionGet()); + + // Read API on ClusterManager + GetAliasesRequest aliasesRequest = new GetAliasesRequest(); + aliasesRequest.aliases("alias1"); + GetAliasesResponse getAliasesResponse = dataNodeClient().admin().indices().getAliases(aliasesRequest).actionGet(); + assertThat(getAliasesResponse.getAliases().get("test").size(), equalTo(1)); + } + + public void testAdmissionControlMonitorOnBreach() throws InterruptedException { + admissionControlDisabledOnBreach( + Settings.builder().put(ADMISSION_CONTROL_TRANSPORT_LAYER_MODE.getKey(), AdmissionControlMode.MONITOR.getMode()).build() + ); + } + + public void testAdmissionControlDisabledOnBreach() throws InterruptedException { + admissionControlDisabledOnBreach(DISABLE_ADMISSION_CONTROL); + } + + public void admissionControlDisabledOnBreach(Settings admission) throws InterruptedException { + client().admin().cluster().prepareUpdateSettings().setTransientSettings(admission).execute().actionGet(); + + cMResourceCollector.collectNodeResourceUsageStats(clusterManagerNodeId, System.currentTimeMillis(), 97, 97, new IoUsageStats(98)); + + // Write API on ClusterManager + assertAcked(prepareCreate("test").setMapping("field", "type=text").setAliases("{\"alias1\" : {}}").execute().actionGet()); + + // Read API on ClusterManager + GetAliasesRequest aliasesRequest = new GetAliasesRequest(); + aliasesRequest.aliases("alias1"); + GetAliasesResponse getAliasesResponse = dataNodeClient().admin().indices().getAliases(aliasesRequest).actionGet(); + assertThat(getAliasesResponse.getAliases().get("test").size(), equalTo(1)); + + } + + public void testAdmissionControlResponseStatus() throws Exception { + cMResourceCollector.collectNodeResourceUsageStats(clusterManagerNodeId, System.currentTimeMillis(), 97, 99, new IoUsageStats(98)); + + // Write API on ClusterManager + assertAcked(prepareCreate("test").setMapping("field", "type=text").setAliases("{\"alias1\" : {}}")); + + // Read API on ClusterManager + FakeRestRequest aliasesRequest = new FakeRestRequest(); + aliasesRequest.params().put("name", "alias1"); + CountDownLatch waitForResponse = new CountDownLatch(1); + AtomicReference aliasResponse = new AtomicReference<>(); + AbstractRestChannel channel = new AbstractRestChannel(aliasesRequest, true) { + + @Override + public void sendResponse(RestResponse response) { + waitForResponse.countDown(); + aliasResponse.set(response); + } + }; + + RestGetAliasesAction restHandler = internalCluster().getInstance(RestGetAliasesAction.class, datanode); + restHandler.handleRequest(aliasesRequest, channel, internalCluster().getInstance(NodeClient.class, datanode)); + + waitForResponse.await(); + assertEquals(RestStatus.TOO_MANY_REQUESTS, aliasResponse.get().status()); + } + + @Override + public void tearDown() throws Exception { + client().admin().cluster().prepareUpdateSettings().setTransientSettings(DISABLE_ADMISSION_CONTROL).execute().actionGet(); + super.tearDown(); + } + + Map getAdmissionControlStats(AdmissionControlService admissionControlService) { + Map acStats = new HashMap<>(); + for (AdmissionControllerStats admissionControllerStats : admissionControlService.stats().getAdmissionControllerStatsList()) { + acStats.put(admissionControllerStats.getAdmissionControllerName(), admissionControllerStats); + } + return acStats; + } +} diff --git a/server/src/main/java/org/opensearch/action/support/HandledTransportAction.java b/server/src/main/java/org/opensearch/action/support/HandledTransportAction.java index 786d8cfb6fa1d..a5054b966b2f9 100644 --- a/server/src/main/java/org/opensearch/action/support/HandledTransportAction.java +++ b/server/src/main/java/org/opensearch/action/support/HandledTransportAction.java @@ -34,6 +34,7 @@ import org.opensearch.action.ActionRequest; import org.opensearch.core.action.ActionResponse; import org.opensearch.core.common.io.stream.Writeable; +import org.opensearch.ratelimitting.admissioncontrol.enums.AdmissionControlActionType; import org.opensearch.tasks.Task; import org.opensearch.threadpool.ThreadPool; import org.opensearch.transport.TransportChannel; @@ -65,7 +66,7 @@ protected HandledTransportAction( Writeable.Reader requestReader, String executor ) { - this(actionName, true, transportService, actionFilters, requestReader, executor); + this(actionName, true, null, transportService, actionFilters, requestReader, executor); } protected HandledTransportAction( @@ -75,19 +76,49 @@ protected HandledTransportAction( ActionFilters actionFilters, Writeable.Reader requestReader ) { - this(actionName, canTripCircuitBreaker, transportService, actionFilters, requestReader, ThreadPool.Names.SAME); + this(actionName, canTripCircuitBreaker, null, transportService, actionFilters, requestReader, ThreadPool.Names.SAME); } protected HandledTransportAction( String actionName, boolean canTripCircuitBreaker, + AdmissionControlActionType admissionControlActionType, + TransportService transportService, + ActionFilters actionFilters, + Writeable.Reader requestReader + ) { + this( + actionName, + canTripCircuitBreaker, + admissionControlActionType, + transportService, + actionFilters, + requestReader, + ThreadPool.Names.SAME + ); + } + + protected HandledTransportAction( + String actionName, + boolean canTripCircuitBreaker, + AdmissionControlActionType admissionControlActionType, TransportService transportService, ActionFilters actionFilters, Writeable.Reader requestReader, String executor ) { super(actionName, actionFilters, transportService.getTaskManager()); - transportService.registerRequestHandler(actionName, executor, false, canTripCircuitBreaker, requestReader, new TransportHandler()); + + transportService.registerRequestHandler( + actionName, + executor, + false, + canTripCircuitBreaker, + admissionControlActionType, + requestReader, + new TransportHandler() + ); + } /** diff --git a/server/src/main/java/org/opensearch/action/support/clustermanager/TransportClusterManagerNodeAction.java b/server/src/main/java/org/opensearch/action/support/clustermanager/TransportClusterManagerNodeAction.java index 6a081f9dfecde..12d572b30272a 100644 --- a/server/src/main/java/org/opensearch/action/support/clustermanager/TransportClusterManagerNodeAction.java +++ b/server/src/main/java/org/opensearch/action/support/clustermanager/TransportClusterManagerNodeAction.java @@ -64,6 +64,7 @@ import org.opensearch.core.common.io.stream.Writeable; import org.opensearch.discovery.ClusterManagerNotDiscoveredException; import org.opensearch.node.NodeClosedException; +import org.opensearch.ratelimitting.admissioncontrol.enums.AdmissionControlActionType; import org.opensearch.tasks.Task; import org.opensearch.threadpool.ThreadPool; import org.opensearch.transport.ConnectTransportException; @@ -105,7 +106,7 @@ protected TransportClusterManagerNodeAction( Writeable.Reader request, IndexNameExpressionResolver indexNameExpressionResolver ) { - this(actionName, true, transportService, clusterService, threadPool, actionFilters, request, indexNameExpressionResolver); + this(actionName, true, null, transportService, clusterService, threadPool, actionFilters, request, indexNameExpressionResolver); } protected TransportClusterManagerNodeAction( @@ -118,7 +119,31 @@ protected TransportClusterManagerNodeAction( Writeable.Reader request, IndexNameExpressionResolver indexNameExpressionResolver ) { - super(actionName, canTripCircuitBreaker, transportService, actionFilters, request); + this( + actionName, + canTripCircuitBreaker, + null, + transportService, + clusterService, + threadPool, + actionFilters, + request, + indexNameExpressionResolver + ); + } + + protected TransportClusterManagerNodeAction( + String actionName, + boolean canTripCircuitBreaker, + AdmissionControlActionType admissionControlActionType, + TransportService transportService, + ClusterService clusterService, + ThreadPool threadPool, + ActionFilters actionFilters, + Writeable.Reader request, + IndexNameExpressionResolver indexNameExpressionResolver + ) { + super(actionName, canTripCircuitBreaker, admissionControlActionType, transportService, actionFilters, request); this.transportService = transportService; this.clusterService = clusterService; this.threadPool = threadPool; diff --git a/server/src/main/java/org/opensearch/action/support/clustermanager/TransportClusterManagerNodeReadAction.java b/server/src/main/java/org/opensearch/action/support/clustermanager/TransportClusterManagerNodeReadAction.java index d8cd5af992028..d58487a475bcf 100644 --- a/server/src/main/java/org/opensearch/action/support/clustermanager/TransportClusterManagerNodeReadAction.java +++ b/server/src/main/java/org/opensearch/action/support/clustermanager/TransportClusterManagerNodeReadAction.java @@ -37,6 +37,7 @@ import org.opensearch.cluster.service.ClusterService; import org.opensearch.core.action.ActionResponse; import org.opensearch.core.common.io.stream.Writeable; +import org.opensearch.ratelimitting.admissioncontrol.enums.AdmissionControlActionType; import org.opensearch.threadpool.ThreadPool; import org.opensearch.transport.TransportService; @@ -59,12 +60,46 @@ protected TransportClusterManagerNodeReadAction( Writeable.Reader request, IndexNameExpressionResolver indexNameExpressionResolver ) { - this(actionName, true, transportService, clusterService, threadPool, actionFilters, request, indexNameExpressionResolver); + this( + actionName, + true, + AdmissionControlActionType.CLUSTER_ADMIN, + transportService, + clusterService, + threadPool, + actionFilters, + request, + indexNameExpressionResolver + ); + } + + protected TransportClusterManagerNodeReadAction( + String actionName, + boolean checkSizeLimit, + TransportService transportService, + ClusterService clusterService, + ThreadPool threadPool, + ActionFilters actionFilters, + Writeable.Reader request, + IndexNameExpressionResolver indexNameExpressionResolver + ) { + super( + actionName, + checkSizeLimit, + null, + transportService, + clusterService, + threadPool, + actionFilters, + request, + indexNameExpressionResolver + ); } protected TransportClusterManagerNodeReadAction( String actionName, boolean checkSizeLimit, + AdmissionControlActionType admissionControlActionType, TransportService transportService, ClusterService clusterService, ThreadPool threadPool, @@ -75,6 +110,7 @@ protected TransportClusterManagerNodeReadAction( super( actionName, checkSizeLimit, + admissionControlActionType, transportService, clusterService, threadPool, diff --git a/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java b/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java index 7c8afb6b5c1b5..4f1815de224db 100644 --- a/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java +++ b/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java @@ -708,15 +708,18 @@ public void apply(Settings value, Settings current, Settings previous) { RemoteStoreNodeService.MIGRATION_DIRECTION_SETTING, IndicesService.CLUSTER_REMOTE_TRANSLOG_BUFFER_INTERVAL_SETTING, IndicesService.CLUSTER_REMOTE_INDEX_RESTRICT_ASYNC_DURABILITY_SETTING, + IndicesService.CLUSTER_INDEX_RESTRICT_REPLICATION_TYPE_SETTING, + IndicesService.CLUSTER_REMOTE_STORE_PATH_PREFIX_TYPE_SETTING, + + // Admission Control Settings AdmissionControlSettings.ADMISSION_CONTROL_TRANSPORT_LAYER_MODE, CpuBasedAdmissionControllerSettings.CPU_BASED_ADMISSION_CONTROLLER_TRANSPORT_LAYER_MODE, CpuBasedAdmissionControllerSettings.INDEXING_CPU_USAGE_LIMIT, CpuBasedAdmissionControllerSettings.SEARCH_CPU_USAGE_LIMIT, + CpuBasedAdmissionControllerSettings.CLUSTER_ADMIN_CPU_USAGE_LIMIT, IoBasedAdmissionControllerSettings.IO_BASED_ADMISSION_CONTROLLER_TRANSPORT_LAYER_MODE, IoBasedAdmissionControllerSettings.SEARCH_IO_USAGE_LIMIT, IoBasedAdmissionControllerSettings.INDEXING_IO_USAGE_LIMIT, - IndicesService.CLUSTER_INDEX_RESTRICT_REPLICATION_TYPE_SETTING, - IndicesService.CLUSTER_REMOTE_STORE_PATH_PREFIX_TYPE_SETTING, // Concurrent segment search settings SearchService.CLUSTER_CONCURRENT_SEGMENT_SEARCH_SETTING, diff --git a/server/src/main/java/org/opensearch/ratelimitting/admissioncontrol/controllers/CpuBasedAdmissionController.java b/server/src/main/java/org/opensearch/ratelimitting/admissioncontrol/controllers/CpuBasedAdmissionController.java index 5c180346c05e1..7ad0715a2a38e 100644 --- a/server/src/main/java/org/opensearch/ratelimitting/admissioncontrol/controllers/CpuBasedAdmissionController.java +++ b/server/src/main/java/org/opensearch/ratelimitting/admissioncontrol/controllers/CpuBasedAdmissionController.java @@ -67,7 +67,8 @@ private void applyForTransportLayer(String actionName, AdmissionControlActionTyp throw new OpenSearchRejectedExecutionException( String.format( Locale.ROOT, - "CPU usage admission controller rejected the request for action [%s] as CPU limit reached", + "CPU usage admission controller rejected the request for action [%s] as CPU limit reached for action-type [%s]", + actionName, admissionControlActionType.name() ) ); @@ -112,6 +113,8 @@ private long getCpuRejectionThreshold(AdmissionControlActionType admissionContro return this.settings.getSearchCPULimit(); case INDEXING: return this.settings.getIndexingCPULimit(); + case CLUSTER_ADMIN: + return this.settings.getClusterAdminCPULimit(); default: throw new IllegalArgumentException( String.format( diff --git a/server/src/main/java/org/opensearch/ratelimitting/admissioncontrol/controllers/IoBasedAdmissionController.java b/server/src/main/java/org/opensearch/ratelimitting/admissioncontrol/controllers/IoBasedAdmissionController.java index ad6cc3ff378f0..d03b2050cd5f3 100644 --- a/server/src/main/java/org/opensearch/ratelimitting/admissioncontrol/controllers/IoBasedAdmissionController.java +++ b/server/src/main/java/org/opensearch/ratelimitting/admissioncontrol/controllers/IoBasedAdmissionController.java @@ -68,7 +68,8 @@ private void applyForTransportLayer(String actionName, AdmissionControlActionTyp throw new OpenSearchRejectedExecutionException( String.format( Locale.ROOT, - "Io usage admission controller rejected the request for action [%s] as IO limit reached", + "IO usage admission controller rejected the request for action [%s] as IO limit reached for action-type [%s]", + actionName, admissionControlActionType.name() ) ); @@ -113,6 +114,8 @@ private long getIoRejectionThreshold(AdmissionControlActionType admissionControl return this.settings.getSearchIOUsageLimit(); case INDEXING: return this.settings.getIndexingIOUsageLimit(); + case CLUSTER_ADMIN: + return this.settings.getClusterAdminIOUsageLimit(); default: throw new IllegalArgumentException( String.format( diff --git a/server/src/main/java/org/opensearch/ratelimitting/admissioncontrol/enums/AdmissionControlActionType.java b/server/src/main/java/org/opensearch/ratelimitting/admissioncontrol/enums/AdmissionControlActionType.java index 8cf6e973ceb64..6acc440180281 100644 --- a/server/src/main/java/org/opensearch/ratelimitting/admissioncontrol/enums/AdmissionControlActionType.java +++ b/server/src/main/java/org/opensearch/ratelimitting/admissioncontrol/enums/AdmissionControlActionType.java @@ -15,7 +15,8 @@ */ public enum AdmissionControlActionType { INDEXING("indexing"), - SEARCH("search"); + SEARCH("search"), + CLUSTER_ADMIN("cluster_admin"); private final String type; @@ -38,6 +39,8 @@ public static AdmissionControlActionType fromName(String name) { return INDEXING; case "search": return SEARCH; + case "cluster_admin": + return CLUSTER_ADMIN; default: throw new IllegalArgumentException("Not Supported TransportAction Type: " + name); } diff --git a/server/src/main/java/org/opensearch/ratelimitting/admissioncontrol/settings/CpuBasedAdmissionControllerSettings.java b/server/src/main/java/org/opensearch/ratelimitting/admissioncontrol/settings/CpuBasedAdmissionControllerSettings.java index 1bddd1446a4c4..30012176d59af 100644 --- a/server/src/main/java/org/opensearch/ratelimitting/admissioncontrol/settings/CpuBasedAdmissionControllerSettings.java +++ b/server/src/main/java/org/opensearch/ratelimitting/admissioncontrol/settings/CpuBasedAdmissionControllerSettings.java @@ -30,6 +30,8 @@ public static class Defaults { private AdmissionControlMode transportLayerMode; private Long searchCPULimit; private Long indexingCPULimit; + private Long clusterInfoCPULimit; + /** * Feature level setting to operate in shadow-mode or in enforced-mode. If enforced field is set * rejection will be performed, otherwise only rejection metrics will be populated. @@ -62,14 +64,24 @@ public static class Defaults { Setting.Property.NodeScope ); + public static final Setting CLUSTER_ADMIN_CPU_USAGE_LIMIT = Setting.longSetting( + "admission_control.cluster.admin.cpu_usage.limit", + Defaults.CPU_USAGE_LIMIT, + Setting.Property.Dynamic, + Setting.Property.NodeScope + ); + // currently limited to one setting will add further more settings in follow-up PR's public CpuBasedAdmissionControllerSettings(ClusterSettings clusterSettings, Settings settings) { this.transportLayerMode = CPU_BASED_ADMISSION_CONTROLLER_TRANSPORT_LAYER_MODE.get(settings); clusterSettings.addSettingsUpdateConsumer(CPU_BASED_ADMISSION_CONTROLLER_TRANSPORT_LAYER_MODE, this::setTransportLayerMode); this.searchCPULimit = SEARCH_CPU_USAGE_LIMIT.get(settings); this.indexingCPULimit = INDEXING_CPU_USAGE_LIMIT.get(settings); + this.clusterInfoCPULimit = CLUSTER_ADMIN_CPU_USAGE_LIMIT.get(settings); clusterSettings.addSettingsUpdateConsumer(INDEXING_CPU_USAGE_LIMIT, this::setIndexingCPULimit); clusterSettings.addSettingsUpdateConsumer(SEARCH_CPU_USAGE_LIMIT, this::setSearchCPULimit); + clusterSettings.addSettingsUpdateConsumer(CLUSTER_ADMIN_CPU_USAGE_LIMIT, this::setClusterInfoCPULimit); + } private void setTransportLayerMode(AdmissionControlMode admissionControlMode) { @@ -88,6 +100,10 @@ public Long getIndexingCPULimit() { return indexingCPULimit; } + public Long getClusterAdminCPULimit() { + return clusterInfoCPULimit; + } + public void setIndexingCPULimit(Long indexingCPULimit) { this.indexingCPULimit = indexingCPULimit; } @@ -95,4 +111,9 @@ public void setIndexingCPULimit(Long indexingCPULimit) { public void setSearchCPULimit(Long searchCPULimit) { this.searchCPULimit = searchCPULimit; } + + public void setClusterInfoCPULimit(Long clusterInfoCPULimit) { + this.clusterInfoCPULimit = clusterInfoCPULimit; + } + } diff --git a/server/src/main/java/org/opensearch/ratelimitting/admissioncontrol/settings/IoBasedAdmissionControllerSettings.java b/server/src/main/java/org/opensearch/ratelimitting/admissioncontrol/settings/IoBasedAdmissionControllerSettings.java index e58ed28d21605..e442906ea77d7 100644 --- a/server/src/main/java/org/opensearch/ratelimitting/admissioncontrol/settings/IoBasedAdmissionControllerSettings.java +++ b/server/src/main/java/org/opensearch/ratelimitting/admissioncontrol/settings/IoBasedAdmissionControllerSettings.java @@ -25,11 +25,14 @@ public class IoBasedAdmissionControllerSettings { */ public static class Defaults { public static final long IO_USAGE_LIMIT = 95; + public static final long CLUSTER_ADMIN_IO_USAGE_LIMIT = 100; + } private AdmissionControlMode transportLayerMode; private Long searchIOUsageLimit; private Long indexingIOUsageLimit; + private Long clusterAdminIOUsageLimit; /** * Feature level setting to operate in shadow-mode or in enforced-mode. If enforced field is set @@ -63,11 +66,22 @@ public static class Defaults { Setting.Property.NodeScope ); + /** + * This setting used to set the limits for cluster admin requests by default it will use default cluster_admin IO usage limit + */ + public static final Setting CLUSTER_ADMIN_IO_USAGE_LIMIT = Setting.longSetting( + "admission_control.cluster_admin.io_usage.limit", + Defaults.CLUSTER_ADMIN_IO_USAGE_LIMIT, + Setting.Property.Final, + Setting.Property.NodeScope + ); + public IoBasedAdmissionControllerSettings(ClusterSettings clusterSettings, Settings settings) { this.transportLayerMode = IO_BASED_ADMISSION_CONTROLLER_TRANSPORT_LAYER_MODE.get(settings); clusterSettings.addSettingsUpdateConsumer(IO_BASED_ADMISSION_CONTROLLER_TRANSPORT_LAYER_MODE, this::setTransportLayerMode); this.searchIOUsageLimit = SEARCH_IO_USAGE_LIMIT.get(settings); this.indexingIOUsageLimit = INDEXING_IO_USAGE_LIMIT.get(settings); + this.clusterAdminIOUsageLimit = CLUSTER_ADMIN_IO_USAGE_LIMIT.get(settings); clusterSettings.addSettingsUpdateConsumer(INDEXING_IO_USAGE_LIMIT, this::setIndexingIOUsageLimit); clusterSettings.addSettingsUpdateConsumer(SEARCH_IO_USAGE_LIMIT, this::setSearchIOUsageLimit); } @@ -95,4 +109,8 @@ public Long getIndexingIOUsageLimit() { public Long getSearchIOUsageLimit() { return searchIOUsageLimit; } + + public Long getClusterAdminIOUsageLimit() { + return clusterAdminIOUsageLimit; + } } diff --git a/server/src/main/java/org/opensearch/transport/TransportService.java b/server/src/main/java/org/opensearch/transport/TransportService.java index 652d57f4c5348..d08b28730d417 100644 --- a/server/src/main/java/org/opensearch/transport/TransportService.java +++ b/server/src/main/java/org/opensearch/transport/TransportService.java @@ -1214,7 +1214,11 @@ public void registerRequestHandler( TransportRequestHandler handler ) { validateActionName(action); - handler = interceptor.interceptHandler(action, executor, forceExecution, handler, admissionControlActionType); + if (admissionControlActionType != null) { + handler = interceptor.interceptHandler(action, executor, forceExecution, handler, admissionControlActionType); + } else { + handler = interceptor.interceptHandler(action, executor, forceExecution, handler); + } RequestHandlerRegistry reg = new RequestHandlerRegistry<>( action, requestReader, diff --git a/server/src/test/java/org/opensearch/ratelimitting/admissioncontrol/settings/CPUBasedAdmissionControllerSettingsTests.java b/server/src/test/java/org/opensearch/ratelimitting/admissioncontrol/settings/CPUBasedAdmissionControllerSettingsTests.java index 9ce28bc7fdb40..6836ecb3d615f 100644 --- a/server/src/test/java/org/opensearch/ratelimitting/admissioncontrol/settings/CPUBasedAdmissionControllerSettingsTests.java +++ b/server/src/test/java/org/opensearch/ratelimitting/admissioncontrol/settings/CPUBasedAdmissionControllerSettingsTests.java @@ -49,7 +49,8 @@ public void testSettingsExists() { Arrays.asList( CpuBasedAdmissionControllerSettings.CPU_BASED_ADMISSION_CONTROLLER_TRANSPORT_LAYER_MODE, CpuBasedAdmissionControllerSettings.SEARCH_CPU_USAGE_LIMIT, - CpuBasedAdmissionControllerSettings.INDEXING_CPU_USAGE_LIMIT + CpuBasedAdmissionControllerSettings.INDEXING_CPU_USAGE_LIMIT, + CpuBasedAdmissionControllerSettings.CLUSTER_ADMIN_CPU_USAGE_LIMIT ) ) ); @@ -149,4 +150,33 @@ public void testUpdateAfterGetConfiguredSettings() { assertEquals(cpuBasedAdmissionControllerSettings.getSearchCPULimit().longValue(), searchPercent); assertEquals(cpuBasedAdmissionControllerSettings.getIndexingCPULimit().longValue(), indexingPercent); } + + public void testConfiguredSettingsForAdmin() { + Settings settings = Settings.builder() + .put( + CpuBasedAdmissionControllerSettings.CPU_BASED_ADMISSION_CONTROLLER_TRANSPORT_LAYER_MODE.getKey(), + AdmissionControlMode.ENFORCED.getMode() + ) + .put(CpuBasedAdmissionControllerSettings.CLUSTER_ADMIN_CPU_USAGE_LIMIT.getKey(), 50) + .build(); + + CpuBasedAdmissionControllerSettings cpuBasedAdmissionControllerSettings = new CpuBasedAdmissionControllerSettings( + clusterService.getClusterSettings(), + settings + ); + assertEquals(cpuBasedAdmissionControllerSettings.getTransportLayerAdmissionControllerMode(), AdmissionControlMode.ENFORCED); + assertEquals(cpuBasedAdmissionControllerSettings.getClusterAdminCPULimit().longValue(), 50); + + Settings updatedSettings = Settings.builder() + .put( + CpuBasedAdmissionControllerSettings.CPU_BASED_ADMISSION_CONTROLLER_TRANSPORT_LAYER_MODE.getKey(), + AdmissionControlMode.MONITOR.getMode() + ) + .put(CpuBasedAdmissionControllerSettings.CLUSTER_ADMIN_CPU_USAGE_LIMIT.getKey(), 90) + .build(); + clusterService.getClusterSettings().applySettings(updatedSettings); + assertEquals(cpuBasedAdmissionControllerSettings.getTransportLayerAdmissionControllerMode(), AdmissionControlMode.MONITOR); + assertEquals(cpuBasedAdmissionControllerSettings.getClusterAdminCPULimit().longValue(), 90); + + } } diff --git a/server/src/test/java/org/opensearch/ratelimitting/admissioncontrol/settings/IoBasedAdmissionControllerSettingsTests.java b/server/src/test/java/org/opensearch/ratelimitting/admissioncontrol/settings/IoBasedAdmissionControllerSettingsTests.java index ff777c175ec0e..c462f9700264d 100644 --- a/server/src/test/java/org/opensearch/ratelimitting/admissioncontrol/settings/IoBasedAdmissionControllerSettingsTests.java +++ b/server/src/test/java/org/opensearch/ratelimitting/admissioncontrol/settings/IoBasedAdmissionControllerSettingsTests.java @@ -72,6 +72,10 @@ public void testDefaultSettings() { assertEquals(ioBasedAdmissionControllerSettings.getTransportLayerAdmissionControllerMode(), AdmissionControlMode.DISABLED); assertEquals(ioBasedAdmissionControllerSettings.getIndexingIOUsageLimit().longValue(), percent); assertEquals(ioBasedAdmissionControllerSettings.getSearchIOUsageLimit().longValue(), percent); + assertEquals( + ioBasedAdmissionControllerSettings.getClusterAdminIOUsageLimit().longValue(), + IoBasedAdmissionControllerSettings.Defaults.CLUSTER_ADMIN_IO_USAGE_LIMIT + ); } public void testGetConfiguredSettings() { @@ -134,6 +138,10 @@ public void testUpdateAfterGetConfiguredSettings() { assertEquals(ioBasedAdmissionControllerSettings.getTransportLayerAdmissionControllerMode(), AdmissionControlMode.ENFORCED); assertEquals(ioBasedAdmissionControllerSettings.getSearchIOUsageLimit().longValue(), searchPercent); assertEquals(ioBasedAdmissionControllerSettings.getIndexingIOUsageLimit().longValue(), percent); + assertEquals( + ioBasedAdmissionControllerSettings.getClusterAdminIOUsageLimit().longValue(), + IoBasedAdmissionControllerSettings.Defaults.CLUSTER_ADMIN_IO_USAGE_LIMIT + ); Settings updatedSettings = Settings.builder() .put( @@ -146,6 +154,10 @@ public void testUpdateAfterGetConfiguredSettings() { assertEquals(ioBasedAdmissionControllerSettings.getTransportLayerAdmissionControllerMode(), AdmissionControlMode.MONITOR); assertEquals(ioBasedAdmissionControllerSettings.getSearchIOUsageLimit().longValue(), searchPercent); assertEquals(ioBasedAdmissionControllerSettings.getIndexingIOUsageLimit().longValue(), indexingPercent); + assertEquals( + ioBasedAdmissionControllerSettings.getClusterAdminIOUsageLimit().longValue(), + IoBasedAdmissionControllerSettings.Defaults.CLUSTER_ADMIN_IO_USAGE_LIMIT + ); searchPercent = 70; updatedSettings = Settings.builder() @@ -156,5 +168,9 @@ public void testUpdateAfterGetConfiguredSettings() { clusterService.getClusterSettings().applySettings(updatedSettings); assertEquals(ioBasedAdmissionControllerSettings.getSearchIOUsageLimit().longValue(), searchPercent); assertEquals(ioBasedAdmissionControllerSettings.getIndexingIOUsageLimit().longValue(), indexingPercent); + assertEquals( + ioBasedAdmissionControllerSettings.getClusterAdminIOUsageLimit().longValue(), + IoBasedAdmissionControllerSettings.Defaults.CLUSTER_ADMIN_IO_USAGE_LIMIT + ); } } From 53c0ce38b26569c90f8980f1c2b0b70e5ebbf6fc Mon Sep 17 00:00:00 2001 From: gaobinlong Date: Thu, 21 Mar 2024 20:40:05 +0800 Subject: [PATCH 23/38] Update supported version for the wait_for_completion parameter in open&clone&shrink&split APIs (#12819) Signed-off-by: Gao Binlong --- .../test/indices.clone/40_wait_for_completion.yml | 4 ++-- .../test/indices.open/30_wait_for_completion.yml | 4 ++-- .../test/indices.shrink/50_wait_for_completion.yml | 4 ++-- .../test/indices.split/40_wait_for_completion.yml | 4 ++-- 4 files changed, 8 insertions(+), 8 deletions(-) diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.clone/40_wait_for_completion.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.clone/40_wait_for_completion.yml index b298575d15410..c9c1558797a35 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.clone/40_wait_for_completion.yml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.clone/40_wait_for_completion.yml @@ -4,8 +4,8 @@ # will return a task immediately and the clone operation will run in background. - skip: - version: " - 2.99.99" - reason: "only available in 3.0+" + version: " - 2.6.99" + reason: "wait_for_completion was introduced in 2.7.0" features: allowed_warnings - do: diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.open/30_wait_for_completion.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.open/30_wait_for_completion.yml index 2caf604eb4296..b93c75f6819c7 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.open/30_wait_for_completion.yml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.open/30_wait_for_completion.yml @@ -4,8 +4,8 @@ # will return a task immediately and the open operation will run in background. - skip: - version: " - 2.99.99" - reason: "only available in 3.0+" + version: " - 2.6.99" + reason: "wait_for_completion was introduced in 2.7.0" features: allowed_warnings - do: diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.shrink/50_wait_for_completion.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.shrink/50_wait_for_completion.yml index f7568b1446967..53df9f61700cd 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.shrink/50_wait_for_completion.yml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.shrink/50_wait_for_completion.yml @@ -4,8 +4,8 @@ # will return a task immediately and the shrink operation will run in background. - skip: - version: " - 2.99.99" - reason: "only available in 3.0+" + version: " - 2.6.99" + reason: "wait_for_completion was introduced in 2.7.0" features: allowed_warnings - do: diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.split/40_wait_for_completion.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.split/40_wait_for_completion.yml index 2ce4fc620742a..9d56cc0800b09 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.split/40_wait_for_completion.yml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.split/40_wait_for_completion.yml @@ -4,8 +4,8 @@ # will return a task immediately and the split operation will run in background. - skip: - version: " - 2.99.99" - reason: "only available in 3.0+" + version: " - 2.6.99" + reason: "wait_for_completion was introduced in 2.7.0" features: allowed_warnings - do: From fd458d62faa0a264d3e7378d9a0a8ffd90f6f235 Mon Sep 17 00:00:00 2001 From: kkewwei Date: Thu, 21 Mar 2024 21:35:58 +0800 Subject: [PATCH 24/38] Fix Flaky SimpleQueryStringIT Tests (#12575) * Fix Flaky SimpleQueryStringIT Tests Signed-off-by: kkewwei * add the comment to unit test Signed-off-by: kkewwei --------- Signed-off-by: kkewwei --- .../org/opensearch/search/query/SimpleQueryStringIT.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/server/src/internalClusterTest/java/org/opensearch/search/query/SimpleQueryStringIT.java b/server/src/internalClusterTest/java/org/opensearch/search/query/SimpleQueryStringIT.java index 31678d3f018a1..cae543506f919 100644 --- a/server/src/internalClusterTest/java/org/opensearch/search/query/SimpleQueryStringIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/search/query/SimpleQueryStringIT.java @@ -112,7 +112,10 @@ public static Collection parameters() { @BeforeClass public static void createRandomClusterSetting() { - CLUSTER_MAX_CLAUSE_COUNT = randomIntBetween(60, 100); + // Lower bound can't be small(such as 60), simpleQueryStringQuery("foo Bar 19 127.0.0.1") in testDocWithAllTypes + // will create many clauses of BooleanClause, In that way, it will throw too_many_nested_clauses exception. + // So we need to set a higher bound(such as 80) to avoid failures. + CLUSTER_MAX_CLAUSE_COUNT = randomIntBetween(80, 100); } @Override From e673b6194fab6f672b62dfdaf8f082bc8dec403f Mon Sep 17 00:00:00 2001 From: rajiv-kv <157019998+rajiv-kv@users.noreply.github.com> Date: Thu, 21 Mar 2024 22:36:47 +0530 Subject: [PATCH 25/38] Update the support version to 2.13 as term-check is merged to 2.x (#12830) Signed-off-by: Rajiv Kumar Vaidyanathan --- .../clustermanager/TransportClusterManagerNodeAction.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/server/src/main/java/org/opensearch/action/support/clustermanager/TransportClusterManagerNodeAction.java b/server/src/main/java/org/opensearch/action/support/clustermanager/TransportClusterManagerNodeAction.java index 12d572b30272a..5f57658e33924 100644 --- a/server/src/main/java/org/opensearch/action/support/clustermanager/TransportClusterManagerNodeAction.java +++ b/server/src/main/java/org/opensearch/action/support/clustermanager/TransportClusterManagerNodeAction.java @@ -78,7 +78,7 @@ import java.util.function.Consumer; import java.util.function.Predicate; -import static org.opensearch.Version.V_3_0_0; +import static org.opensearch.Version.V_2_13_0; /** * A base class for operations that needs to be performed on the cluster-manager node. @@ -299,7 +299,7 @@ protected void doStart(ClusterState clusterState) { retryOnMasterChange(clusterState, null); } else { DiscoveryNode clusterManagerNode = nodes.getClusterManagerNode(); - if (clusterManagerNode.getVersion().onOrAfter(V_3_0_0) && localExecuteSupportedByAction()) { + if (clusterManagerNode.getVersion().onOrAfter(V_2_13_0) && localExecuteSupportedByAction()) { BiConsumer executeOnLocalOrClusterManager = clusterStateLatestChecker( this::executeOnLocalNode, this::executeOnClusterManager From 7bd3715c81acdaa3c3eaf2c50032db139718f485 Mon Sep 17 00:00:00 2001 From: Andriy Redko Date: Thu, 21 Mar 2024 17:56:02 -0400 Subject: [PATCH 26/38] Add 2.14.0 to version & BWC (#12842) Signed-off-by: Andriy Redko --- .ci/bwcVersions | 1 + libs/core/src/main/java/org/opensearch/Version.java | 1 + 2 files changed, 2 insertions(+) diff --git a/.ci/bwcVersions b/.ci/bwcVersions index 1e3b913c5cb5a..78d8796c624d7 100644 --- a/.ci/bwcVersions +++ b/.ci/bwcVersions @@ -30,3 +30,4 @@ BWC_VERSION: - "2.12.0" - "2.12.1" - "2.13.0" + - "2.14.0" diff --git a/libs/core/src/main/java/org/opensearch/Version.java b/libs/core/src/main/java/org/opensearch/Version.java index 66ba446d4fc54..56df46ae94d44 100644 --- a/libs/core/src/main/java/org/opensearch/Version.java +++ b/libs/core/src/main/java/org/opensearch/Version.java @@ -101,6 +101,7 @@ public class Version implements Comparable, ToXContentFragment { public static final Version V_2_12_0 = new Version(2120099, org.apache.lucene.util.Version.LUCENE_9_9_2); public static final Version V_2_12_1 = new Version(2120199, org.apache.lucene.util.Version.LUCENE_9_9_2); public static final Version V_2_13_0 = new Version(2130099, org.apache.lucene.util.Version.LUCENE_9_10_0); + public static final Version V_2_14_0 = new Version(2140099, org.apache.lucene.util.Version.LUCENE_9_10_0); public static final Version V_3_0_0 = new Version(3000099, org.apache.lucene.util.Version.LUCENE_9_11_0); public static final Version CURRENT = V_3_0_0; From 0c0bcd9ffa10ba7999418f2f7d400ba3a1d446dd Mon Sep 17 00:00:00 2001 From: Varun Bansal Date: Fri, 22 Mar 2024 05:16:45 +0530 Subject: [PATCH 27/38] Add release-notes for 2.13.0 (#12841) Signed-off-by: Varun Bansal --- .../opensearch.release-notes-2.13.0.md | 71 +++++++++++++++++++ 1 file changed, 71 insertions(+) create mode 100644 release-notes/opensearch.release-notes-2.13.0.md diff --git a/release-notes/opensearch.release-notes-2.13.0.md b/release-notes/opensearch.release-notes-2.13.0.md new file mode 100644 index 0000000000000..517092564545d --- /dev/null +++ b/release-notes/opensearch.release-notes-2.13.0.md @@ -0,0 +1,71 @@ +## 2024-03-21 Version 2.13.0 Release Notes + +## [2.13.0] +### Added +- [Tiered caching] Introducing cache plugins and exposing Ehcache as one of the pluggable disk cache option ([#11874](https://github.com/opensearch-project/OpenSearch/pull/11874)) +- Add support for dependencies in plugin descriptor properties with semver range ([#11441](https://github.com/opensearch-project/OpenSearch/pull/11441)) +- Add community_id ingest processor ([#12121](https://github.com/opensearch-project/OpenSearch/pull/12121)) +- Introduce query level setting `index.query.max_nested_depth` limiting nested queries ([#3268](https://github.com/opensearch-project/OpenSearch/issues/3268) +- Add toString methods to MultiSearchRequest, MultiGetRequest and CreateIndexRequest ([#12163](https://github.com/opensearch-project/OpenSearch/pull/12163)) +- Fix error in RemoteSegmentStoreDirectory when debug logging is enabled ([#12328](https://github.com/opensearch-project/OpenSearch/pull/12328)) +- Support for returning scores in matched queries ([#11626](https://github.com/opensearch-project/OpenSearch/pull/11626)) +- Add shard id property to SearchLookup for use in field types provided by plugins ([#1063](https://github.com/opensearch-project/OpenSearch/pull/1063)) +- [Tiered caching] Make IndicesRequestCache implementation configurable [EXPERIMENTAL] ([#12533](https://github.com/opensearch-project/OpenSearch/pull/12533)) +- Force merge API supports performing on primary shards only ([#11269](https://github.com/opensearch-project/OpenSearch/pull/11269)) +- Add kuromoji_completion analyzer and filter ([#4835](https://github.com/opensearch-project/OpenSearch/issues/4835)) +- [Admission Control] Integrate IO Usage Tracker to the Resource Usage Collector Service and Emit IO Usage Stats ([#11880](https://github.com/opensearch-project/OpenSearch/pull/11880)) +- The org.opensearch.bootstrap.Security should support codebase for JAR files with classifiers ([#12586](https://github.com/opensearch-project/OpenSearch/issues/12586)) +- Remote reindex: Add support for configurable retry mechanism ([#12561](https://github.com/opensearch-project/OpenSearch/pull/12561)) +- Tracing for deep search path ([#12103](https://github.com/opensearch-project/OpenSearch/pull/12103)) +- [Metrics Framework] Adds support for asynchronous gauge metric type. ([#12642](https://github.com/opensearch-project/OpenSearch/issues/12642)) +- [Tiered caching] Add Stale keys Management and CacheCleaner to IndicesRequestCache ([#12625](https://github.com/opensearch-project/OpenSearch/pull/12625)) +- Make search query counters dynamic to support all query types ([#12601](https://github.com/opensearch-project/OpenSearch/pull/12601)) +- [Tiered caching] Add policies controlling which values can enter pluggable caches [EXPERIMENTAL] ([#12542](https://github.com/opensearch-project/OpenSearch/pull/12542)) +- [Tiered caching] Add serializer integration to allow ehcache disk cache to use non-primitive values ([#12709](https://github.com/opensearch-project/OpenSearch/pull/12709)) +- [Admission Control] Integrated IO Based AdmissionController to AdmissionControl Framework ([#12583](https://github.com/opensearch-project/OpenSearch/pull/12583)) +- Add Remote Store Migration Experimental flag and allow mixed mode clusters under same ([#11986](https://github.com/opensearch-project/OpenSearch/pull/11986)) +- Built-in secure transports support ([#12435](https://github.com/opensearch-project/OpenSearch/pull/12435)) +- Lightweight Transport action to verify local term before fetching cluster-state from remote ([#12252](https://github.com/opensearch-project/OpenSearch/pull/12252/)) + +### Dependencies +- Bump `com.squareup.okio:okio` from 3.7.0 to 3.8.0 ([#12290](https://github.com/opensearch-project/OpenSearch/pull/12290)) +- Bump `org.bouncycastle:bcprov-jdk15to18` to `org.bouncycastle:bcprov-jdk18on` version 1.77 ([#12326](https://github.com/opensearch-project/OpenSearch/pull/12326)) +- Bump `org.bouncycastle:bcmail-jdk15to18` to `org.bouncycastle:bcmail-jdk18on` version 1.77 ([#12326](https://github.com/opensearch-project/OpenSearch/pull/12326)) +- Bump `org.bouncycastle:bcpkix-jdk15to18` to `org.bouncycastle:bcpkix-jdk18on` version 1.77 ([#12326](https://github.com/opensearch-project/OpenSearch/pull/12326)) +- Bump `gradle/wrapper-validation-action` from 1 to 2 ([#12367](https://github.com/opensearch-project/OpenSearch/pull/12367)) +- Bump `netty` from 4.1.106.Final to 4.1.107.Final ([#12372](https://github.com/opensearch-project/OpenSearch/pull/12372)) +- Bump `opentelemetry` from 1.34.1 to 1.36.0 ([#12388](https://github.com/opensearch-project/OpenSearch/pull/12388), [#12618](https://github.com/opensearch-project/OpenSearch/pull/12618)) +- Bump Apache Lucene from 9.9.2 to 9.10.0 ([#12392](https://github.com/opensearch-project/OpenSearch/pull/12392)) +- Bump `org.apache.logging.log4j:log4j-core` from 2.22.1 to 2.23.1 ([#12464](https://github.com/opensearch-project/OpenSearch/pull/12464), [#12587](https://github.com/opensearch-project/OpenSearch/pull/12587)) +- Bump `antlr4` from 4.11.1 to 4.13.1 ([#12445](https://github.com/opensearch-project/OpenSearch/pull/12445)) +- Bump `com.netflix.nebula.ospackage-base` from 11.8.0 to 11.8.1 ([#12461](https://github.com/opensearch-project/OpenSearch/pull/12461)) +- Bump `peter-evans/create-or-update-comment` from 3 to 4 ([#12462](https://github.com/opensearch-project/OpenSearch/pull/12462)) +- Bump `lycheeverse/lychee-action` from 1.9.1 to 1.9.3 ([#12521](https://github.com/opensearch-project/OpenSearch/pull/12521)) +- Bump `com.azure:azure-core` from 1.39.0 to 1.47.0 ([#12520](https://github.com/opensearch-project/OpenSearch/pull/12520)) +- Bump `ch.qos.logback:logback-core` from 1.2.13 to 1.5.3 ([#12519](https://github.com/opensearch-project/OpenSearch/pull/12519)) +- Bump `codecov/codecov-action` from 3 to 4 ([#12585](https://github.com/opensearch-project/OpenSearch/pull/12585)) +- Bump `org.apache.zookeeper:zookeeper` from 3.9.1 to 3.9.2 ([#12580](https://github.com/opensearch-project/OpenSearch/pull/12580)) +- Bump `org.codehaus.woodstox:stax2-api` from 4.2.1 to 4.2.2 ([#12579](https://github.com/opensearch-project/OpenSearch/pull/12579)) +- Bump Jackson version from 2.16.1 to 2.17.0 ([#12611](https://github.com/opensearch-project/OpenSearch/pull/12611), [#12662](https://github.com/opensearch-project/OpenSearch/pull/12662)) +- Bump `reactor-netty` from 1.1.15 to 1.1.17 ([#12633](https://github.com/opensearch-project/OpenSearch/pull/12633)) +- Bump `reactor` from 3.5.14 to 3.5.15 ([#12633](https://github.com/opensearch-project/OpenSearch/pull/12633)) +- Bump `aws-sdk-java` from 2.20.55 to 2.20.86 ([#12251](https://github.com/opensearch-project/OpenSearch/pull/12251)) + +### Changed +- Allow composite aggregation to run under a parent filter aggregation ([#11499](https://github.com/opensearch-project/OpenSearch/pull/11499)) +- Quickly compute terms aggregations when the top-level query is functionally match-all for a segment ([#11643](https://github.com/opensearch-project/OpenSearch/pull/11643)) +- Mark fuzzy filter GA and remove experimental setting ([12631](https://github.com/opensearch-project/OpenSearch/pull/12631)) +- Keep the election scheduler open until cluster state has been applied ([#11699](https://github.com/opensearch-project/OpenSearch/pull/11699)) + +### Fixed +- [Revert] [Bug] Check phase name before SearchRequestOperationsListener onPhaseStart ([#12035](https://github.com/opensearch-project/OpenSearch/pull/12035)) +- Add support of special WrappingSearchAsyncActionPhase so the onPhaseStart() will always be followed by onPhaseEnd() within AbstractSearchAsyncAction ([#12293](https://github.com/opensearch-project/OpenSearch/pull/12293)) +- Add a system property to configure YamlParser codepoint limits ([#12298](https://github.com/opensearch-project/OpenSearch/pull/12298)) +- Prevent read beyond slice boundary in ByteArrayIndexInput ([#10481](https://github.com/opensearch-project/OpenSearch/issues/10481)) +- Fix the "highlight.max_analyzer_offset" request parameter with "plain" highlighter ([#10919](https://github.com/opensearch-project/OpenSearch/pull/10919)) +- Prevent unnecessary fetch sub phase processor initialization during fetch phase execution ([#12503](https://github.com/opensearch-project/OpenSearch/pull/12503)) +- Fix `terms` query on `float` field when `doc_values` are turned off by reverting back to `FloatPoint` from `FloatField` ([#12499](https://github.com/opensearch-project/OpenSearch/pull/12499)) +- Fix get task API does not refresh resource stats ([#11531](https://github.com/opensearch-project/OpenSearch/pull/11531)) +- Fix for deserilization bug in weighted round-robin metadata ([#11679](https://github.com/opensearch-project/OpenSearch/pull/11679)) +- onShardResult and onShardFailure are executed on one shard causes opensearch jvm crashed ([#12158](https://github.com/opensearch-project/OpenSearch/pull/12158)) +- Avoid overflow when sorting missing last on `epoch_millis` datetime field ([#12676](https://github.com/opensearch-project/OpenSearch/pull/12676)) From 0c956e81373c9d5878710c1bafe471431000a64f Mon Sep 17 00:00:00 2001 From: "Daniel (dB.) Doubrovkine" Date: Thu, 21 Mar 2024 20:35:21 -0400 Subject: [PATCH 28/38] Catch task description error (#12834) * Always return a task description even when it cannot be serialized. Signed-off-by: dblock * Expect tasks to fail. Signed-off-by: dblock * Only catch exceptions when getting description. Signed-off-by: dblock * Added to mark error more clearly. Signed-off-by: dblock --------- Signed-off-by: dblock --- .../opensearch/action/search/SearchRequest.java | 9 ++++++++- .../search/builder/SearchSourceBuilder.java | 2 +- .../action/search/SearchRequestTests.java | 15 +++++++++++++++ .../search/geo/GeoPointShapeQueryTests.java | 8 ++++++-- 4 files changed, 30 insertions(+), 4 deletions(-) diff --git a/server/src/main/java/org/opensearch/action/search/SearchRequest.java b/server/src/main/java/org/opensearch/action/search/SearchRequest.java index f738c182c06da..3b8a6937815aa 100644 --- a/server/src/main/java/org/opensearch/action/search/SearchRequest.java +++ b/server/src/main/java/org/opensearch/action/search/SearchRequest.java @@ -32,6 +32,7 @@ package org.opensearch.action.search; +import org.opensearch.OpenSearchException; import org.opensearch.Version; import org.opensearch.action.ActionRequest; import org.opensearch.action.ActionRequestValidationException; @@ -712,7 +713,13 @@ public final String buildDescription() { sb.append("scroll[").append(scroll.keepAlive()).append("], "); } if (source != null) { - sb.append("source[").append(source.toString(FORMAT_PARAMS)).append("]"); + sb.append("source["); + try { + sb.append(source.toString(FORMAT_PARAMS)); + } catch (final OpenSearchException ex) { + sb.append(""); + } + sb.append("]"); } else { sb.append("source[]"); } diff --git a/server/src/main/java/org/opensearch/search/builder/SearchSourceBuilder.java b/server/src/main/java/org/opensearch/search/builder/SearchSourceBuilder.java index 1a5a9dc6d1f03..bf7045d43ba67 100644 --- a/server/src/main/java/org/opensearch/search/builder/SearchSourceBuilder.java +++ b/server/src/main/java/org/opensearch/search/builder/SearchSourceBuilder.java @@ -1842,7 +1842,7 @@ public String toString() { public String toString(Params params) { try { return XContentHelper.toXContent(this, MediaTypeRegistry.JSON, params, true).utf8ToString(); - } catch (IOException e) { + } catch (IOException | UnsupportedOperationException e) { throw new OpenSearchException(e); } } diff --git a/server/src/test/java/org/opensearch/action/search/SearchRequestTests.java b/server/src/test/java/org/opensearch/action/search/SearchRequestTests.java index f025e3a63b9bf..9ee314e77ca7e 100644 --- a/server/src/test/java/org/opensearch/action/search/SearchRequestTests.java +++ b/server/src/test/java/org/opensearch/action/search/SearchRequestTests.java @@ -39,6 +39,8 @@ import org.opensearch.common.util.ArrayUtils; import org.opensearch.core.common.Strings; import org.opensearch.core.tasks.TaskId; +import org.opensearch.geometry.LinearRing; +import org.opensearch.index.query.GeoShapeQueryBuilder; import org.opensearch.index.query.QueryBuilders; import org.opensearch.search.AbstractSearchTestCase; import org.opensearch.search.Scroll; @@ -269,6 +271,19 @@ public void testDescriptionIncludesScroll() { ); } + public void testDescriptionOnSourceError() { + LinearRing linearRing = new LinearRing(new double[] { -25, -35, -25 }, new double[] { -25, -35, -25 }); + GeoShapeQueryBuilder queryBuilder = new GeoShapeQueryBuilder("geo", linearRing); + SearchRequest request = new SearchRequest(); + request.source(new SearchSourceBuilder().query(queryBuilder)); + assertThat( + toDescription(request), + equalTo( + "indices[], search_type[QUERY_THEN_FETCH], source[]" + ) + ); + } + private String toDescription(SearchRequest request) { return request.createTask(0, "test", SearchAction.NAME, TaskId.EMPTY_TASK_ID, emptyMap()).getDescription(); } diff --git a/server/src/test/java/org/opensearch/search/geo/GeoPointShapeQueryTests.java b/server/src/test/java/org/opensearch/search/geo/GeoPointShapeQueryTests.java index b5d34a78ab5a4..b00f36ef52d4a 100644 --- a/server/src/test/java/org/opensearch/search/geo/GeoPointShapeQueryTests.java +++ b/server/src/test/java/org/opensearch/search/geo/GeoPointShapeQueryTests.java @@ -100,6 +100,7 @@ public void testProcessRelationSupport() throws Exception { client().prepareSearch("test") .setQuery(QueryBuilders.geoShapeQuery(defaultGeoFieldName, rectangle).relation(shapeRelation)) .get(); + fail("Expected " + shapeRelation + " query relation not supported for Field [" + defaultGeoFieldName + "]"); } catch (SearchPhaseExecutionException e) { assertThat( e.getCause().getMessage(), @@ -119,6 +120,7 @@ public void testQueryLine() throws Exception { try { client().prepareSearch("test").setQuery(QueryBuilders.geoShapeQuery(defaultGeoFieldName, line)).get(); + fail("Expected field [" + defaultGeoFieldName + "] does not support LINEARRING queries"); } catch (SearchPhaseExecutionException e) { assertThat(e.getCause().getMessage(), containsString("does not support " + GeoShapeType.LINESTRING + " queries")); } @@ -138,13 +140,12 @@ public void testQueryLinearRing() throws Exception { searchRequestBuilder.setQuery(queryBuilder); searchRequestBuilder.setIndices("test"); searchRequestBuilder.get(); + fail("Expected field [" + defaultGeoFieldName + "] does not support LINEARRING queries"); } catch (SearchPhaseExecutionException e) { assertThat( e.getCause().getMessage(), containsString("Field [" + defaultGeoFieldName + "] does not support LINEARRING queries") ); - } catch (UnsupportedOperationException e) { - assertThat(e.getMessage(), containsString("line ring cannot be serialized using GeoJson")); } } @@ -162,6 +163,7 @@ public void testQueryMultiLine() throws Exception { try { client().prepareSearch("test").setQuery(QueryBuilders.geoShapeQuery(defaultGeoFieldName, multiline)).get(); + fail("Expected field [" + defaultGeoFieldName + "] does not support " + GeoShapeType.MULTILINESTRING + " queries"); } catch (Exception e) { assertThat(e.getCause().getMessage(), containsString("does not support " + GeoShapeType.MULTILINESTRING + " queries")); } @@ -177,6 +179,7 @@ public void testQueryMultiPoint() throws Exception { try { client().prepareSearch("test").setQuery(QueryBuilders.geoShapeQuery(defaultGeoFieldName, multiPoint)).get(); + fail("Expected field [" + defaultGeoFieldName + "] does not support " + GeoShapeType.MULTIPOINT + " queries"); } catch (Exception e) { assertThat(e.getCause().getMessage(), containsString("does not support " + GeoShapeType.MULTIPOINT + " queries")); } @@ -192,6 +195,7 @@ public void testQueryPoint() throws Exception { try { client().prepareSearch("test").setQuery(QueryBuilders.geoShapeQuery(defaultGeoFieldName, point)).get(); + fail("Expected field [" + defaultGeoFieldName + "] does not support " + GeoShapeType.POINT + " queries"); } catch (Exception e) { assertThat(e.getCause().getMessage(), containsString("does not support " + GeoShapeType.POINT + " queries")); } From 13604c88cf31a402e3c771ed4ec90466b594fa1b Mon Sep 17 00:00:00 2001 From: Andrew Ross Date: Thu, 21 Mar 2024 20:49:15 -0500 Subject: [PATCH 29/38] Clear out 2.x section of changelog (#12847) The 2.13 release notes have been created. Signed-off-by: Andrew Ross --- CHANGELOG.md | 60 ---------------------------------------------------- 1 file changed, 60 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 9dced730cbcad..af20332c61146 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -101,76 +101,16 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), ## [Unreleased 2.x] ### Added -- [Tiered caching] Introducing cache plugins and exposing Ehcache as one of the pluggable disk cache option ([#11874](https://github.com/opensearch-project/OpenSearch/pull/11874)) -- Add support for dependencies in plugin descriptor properties with semver range ([#11441](https://github.com/opensearch-project/OpenSearch/pull/11441)) -- Add community_id ingest processor ([#12121](https://github.com/opensearch-project/OpenSearch/pull/12121)) -- Introduce query level setting `index.query.max_nested_depth` limiting nested queries ([#3268](https://github.com/opensearch-project/OpenSearch/issues/3268) -- Add toString methods to MultiSearchRequest, MultiGetRequest and CreateIndexRequest ([#12163](https://github.com/opensearch-project/OpenSearch/pull/12163)) -- Support for returning scores in matched queries ([#11626](https://github.com/opensearch-project/OpenSearch/pull/11626)) -- Add shard id property to SearchLookup for use in field types provided by plugins ([#1063](https://github.com/opensearch-project/OpenSearch/pull/1063)) -- Force merge API supports performing on primary shards only ([#11269](https://github.com/opensearch-project/OpenSearch/pull/11269)) -- [Tiered caching] Make IndicesRequestCache implementation configurable [EXPERIMENTAL] ([#12533](https://github.com/opensearch-project/OpenSearch/pull/12533)) -- Add kuromoji_completion analyzer and filter ([#4835](https://github.com/opensearch-project/OpenSearch/issues/4835)) -- The org.opensearch.bootstrap.Security should support codebase for JAR files with classifiers ([#12586](https://github.com/opensearch-project/OpenSearch/issues/12586)) -- [Metrics Framework] Adds support for asynchronous gauge metric type. ([#12642](https://github.com/opensearch-project/OpenSearch/issues/12642)) -- Make search query counters dynamic to support all query types ([#12601](https://github.com/opensearch-project/OpenSearch/pull/12601)) -- [Tiered caching] Add policies controlling which values can enter pluggable caches [EXPERIMENTAL] ([#12542](https://github.com/opensearch-project/OpenSearch/pull/12542)) -- [Tiered caching] Add Stale keys Management and CacheCleaner to IndicesRequestCache ([#12625](https://github.com/opensearch-project/OpenSearch/pull/12625)) -- [Tiered caching] Add serializer integration to allow ehcache disk cache to use non-primitive values ([#12709](https://github.com/opensearch-project/OpenSearch/pull/12709)) -- [Admission Control] Integrated IO Based AdmissionController to AdmissionControl Framework ([#12583](https://github.com/opensearch-project/OpenSearch/pull/12583)) -- Introduce a new setting `index.check_pending_flush.enabled` to expose the ability to disable the check for pending flushes by write threads ([#12710](https://github.com/opensearch-project/OpenSearch/pull/12710)) -- Built-in secure transports support ([#12435](https://github.com/opensearch-project/OpenSearch/pull/12435)) -- Lightweight Transport action to verify local term before fetching cluster-state from remote ([#12252](https://github.com/opensearch-project/OpenSearch/pull/12252/)) -- Integrate with admission controller for cluster-manager Read API. ([#12496](https://github.com/opensearch-project/OpenSearch/pull/12496)) ### Dependencies -- Bump `peter-evans/find-comment` from 2 to 3 ([#12288](https://github.com/opensearch-project/OpenSearch/pull/12288)) -- Bump `com.google.api.grpc:proto-google-common-protos` from 2.25.1 to 2.37.1 ([#12289](https://github.com/opensearch-project/OpenSearch/pull/12289), [#12365](https://github.com/opensearch-project/OpenSearch/pull/12365)) -- Bump `com.squareup.okio:okio` from 3.7.0 to 3.8.0 ([#12290](https://github.com/opensearch-project/OpenSearch/pull/12290)) -- Bump `gradle/wrapper-validation-action` from 1 to 2 ([#12367](https://github.com/opensearch-project/OpenSearch/pull/12367)) -- Bump `netty` from 4.1.106.Final to 4.1.107.Final ([#12372](https://github.com/opensearch-project/OpenSearch/pull/12372)) -- Bump `opentelemetry` from 1.34.1 to 1.36.0 ([#12388](https://github.com/opensearch-project/OpenSearch/pull/12388), [#12618](https://github.com/opensearch-project/OpenSearch/pull/12618)) -- Bump Apache Lucene from 9.9.2 to 9.10.0 ([#12392](https://github.com/opensearch-project/OpenSearch/pull/12392)) -- Bump `org.apache.logging.log4j:log4j-core` from 2.22.1 to 2.23.1 ([#12464](https://github.com/opensearch-project/OpenSearch/pull/12464), [#12587](https://github.com/opensearch-project/OpenSearch/pull/12587)) -- Bump `antlr4` from 4.11.1 to 4.13.1 ([#12445](https://github.com/opensearch-project/OpenSearch/pull/12445)) -- Bump `com.netflix.nebula.ospackage-base` from 11.8.0 to 11.8.1 ([#12461](https://github.com/opensearch-project/OpenSearch/pull/12461)) -- Bump `peter-evans/create-or-update-comment` from 3 to 4 ([#12462](https://github.com/opensearch-project/OpenSearch/pull/12462)) -- Bump `lycheeverse/lychee-action` from 1.9.1 to 1.9.3 ([#12521](https://github.com/opensearch-project/OpenSearch/pull/12521)) -- Bump `com.azure:azure-core` from 1.39.0 to 1.47.0 ([#12520](https://github.com/opensearch-project/OpenSearch/pull/12520)) -- Bump `ch.qos.logback:logback-core` from 1.2.13 to 1.5.3 ([#12519](https://github.com/opensearch-project/OpenSearch/pull/12519)) -- Bump `codecov/codecov-action` from 3 to 4 ([#12585](https://github.com/opensearch-project/OpenSearch/pull/12585)) -- Bump `org.apache.zookeeper:zookeeper` from 3.9.1 to 3.9.2 ([#12580](https://github.com/opensearch-project/OpenSearch/pull/12580)) -- Bump `org.codehaus.woodstox:stax2-api` from 4.2.1 to 4.2.2 ([#12579](https://github.com/opensearch-project/OpenSearch/pull/12579)) -- Bump Jackson version from 2.16.1 to 2.17.0 ([#12611](https://github.com/opensearch-project/OpenSearch/pull/12611), [#12662](https://github.com/opensearch-project/OpenSearch/pull/12662)) -- Bump `aws-sdk-java` from 2.20.55 to 2.20.86 ([#12251](https://github.com/opensearch-project/OpenSearch/pull/12251)) -- Bump `reactor-netty` from 1.1.15 to 1.1.17 ([#12633](https://github.com/opensearch-project/OpenSearch/pull/12633)) -- Bump `reactor` from 3.5.14 to 3.5.15 ([#12633](https://github.com/opensearch-project/OpenSearch/pull/12633)) -- Bump `peter-evans/create-pull-request` from 5 to 6 ([#12724](https://github.com/opensearch-project/OpenSearch/pull/12724)) -- Bump `org.apache.commons:commons-configuration2` from 2.9.0 to 2.10.0 ([#12721](https://github.com/opensearch-project/OpenSearch/pull/12721)) -- Bump `com.azure:azure-json` from 1.0.1 to 1.1.0 ([#12723](https://github.com/opensearch-project/OpenSearch/pull/12723)) ### Changed -- Allow composite aggregation to run under a parent filter aggregation ([#11499](https://github.com/opensearch-project/OpenSearch/pull/11499)) -- Quickly compute terms aggregations when the top-level query is functionally match-all for a segment ([#11643](https://github.com/opensearch-project/OpenSearch/pull/11643)) -- Mark fuzzy filter GA and remove experimental setting ([12631](https://github.com/opensearch-project/OpenSearch/pull/12631)) -- Keep the election scheduler open until cluster state has been applied ([#11699](https://github.com/opensearch-project/OpenSearch/pull/11699)) ### Deprecated ### Removed ### Fixed -- Fix for deserilization bug in weighted round-robin metadata ([#11679](https://github.com/opensearch-project/OpenSearch/pull/11679)) -- [Revert] [Bug] Check phase name before SearchRequestOperationsListener onPhaseStart ([#12035](https://github.com/opensearch-project/OpenSearch/pull/12035)) -- Add support of special WrappingSearchAsyncActionPhase so the onPhaseStart() will always be followed by onPhaseEnd() within AbstractSearchAsyncAction ([#12293](https://github.com/opensearch-project/OpenSearch/pull/12293)) -- Add a system property to configure YamlParser codepoint limits ([#12298](https://github.com/opensearch-project/OpenSearch/pull/12298)) -- Prevent read beyond slice boundary in ByteArrayIndexInput ([#10481](https://github.com/opensearch-project/OpenSearch/issues/10481)) -- Fix the "highlight.max_analyzer_offset" request parameter with "plain" highlighter ([#10919](https://github.com/opensearch-project/OpenSearch/pull/10919)) -- Prevent unnecessary fetch sub phase processor initialization during fetch phase execution ([#12503](https://github.com/opensearch-project/OpenSearch/pull/12503)) -- Warn about deprecated and ignored index.mapper.dynamic index setting ([#11193](https://github.com/opensearch-project/OpenSearch/pull/11193)) -- Fix `terms` query on `float` field when `doc_values` are turned off by reverting back to `FloatPoint` from `FloatField` ([#12499](https://github.com/opensearch-project/OpenSearch/pull/12499)) -- Fix get task API does not refresh resource stats ([#11531](https://github.com/opensearch-project/OpenSearch/pull/11531)) -- onShardResult and onShardFailure are executed on one shard causes opensearch jvm crashed ([#12158](https://github.com/opensearch-project/OpenSearch/pull/12158)) ### Security From 208520e2062bc675fd3a890a2b097e29b99d6479 Mon Sep 17 00:00:00 2001 From: Vikas Bansal <43470111+vikasvb90@users.noreply.github.com> Date: Fri, 22 Mar 2024 14:47:54 +0530 Subject: [PATCH 30/38] Implementing reload in encrypted blob store (#12826) Signed-off-by: vikasvb90 --- .../common/blobstore/EncryptedBlobStore.java | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/server/src/main/java/org/opensearch/common/blobstore/EncryptedBlobStore.java b/server/src/main/java/org/opensearch/common/blobstore/EncryptedBlobStore.java index a18ca8b9d5c39..c41641921c822 100644 --- a/server/src/main/java/org/opensearch/common/blobstore/EncryptedBlobStore.java +++ b/server/src/main/java/org/opensearch/common/blobstore/EncryptedBlobStore.java @@ -9,6 +9,7 @@ package org.opensearch.common.blobstore; import org.opensearch.cluster.metadata.CryptoMetadata; +import org.opensearch.cluster.metadata.RepositoryMetadata; import org.opensearch.common.crypto.CryptoHandler; import org.opensearch.crypto.CryptoHandlerRegistry; import org.opensearch.crypto.CryptoRegistryException; @@ -65,6 +66,15 @@ public BlobContainer blobContainer(BlobPath path) { return new EncryptedBlobContainer<>(blobContainer, cryptoHandler); } + /** + * Reoload blobstore metadata + * @param repositoryMetadata new repository metadata + */ + @Override + public void reload(RepositoryMetadata repositoryMetadata) { + blobStore.reload(repositoryMetadata); + } + /** * Retrieves statistics about the BlobStore. Delegates the call to the underlying BlobStore's stats() method. * From 122e944f1e8f2a774a783dc49db8ae73c1e33f9b Mon Sep 17 00:00:00 2001 From: Varun Bansal Date: Fri, 22 Mar 2024 15:50:42 +0530 Subject: [PATCH 31/38] add missing changelog entry to 2.13 release nodes (#12851) Signed-off-by: Varun Bansal --- release-notes/opensearch.release-notes-2.13.0.md | 1 + 1 file changed, 1 insertion(+) diff --git a/release-notes/opensearch.release-notes-2.13.0.md b/release-notes/opensearch.release-notes-2.13.0.md index 517092564545d..e55c22d6b851d 100644 --- a/release-notes/opensearch.release-notes-2.13.0.md +++ b/release-notes/opensearch.release-notes-2.13.0.md @@ -26,6 +26,7 @@ - Add Remote Store Migration Experimental flag and allow mixed mode clusters under same ([#11986](https://github.com/opensearch-project/OpenSearch/pull/11986)) - Built-in secure transports support ([#12435](https://github.com/opensearch-project/OpenSearch/pull/12435)) - Lightweight Transport action to verify local term before fetching cluster-state from remote ([#12252](https://github.com/opensearch-project/OpenSearch/pull/12252/)) +- Integrate with admission controller for cluster-manager Read API. ([#12496](https://github.com/opensearch-project/OpenSearch/pull/12496)) ### Dependencies - Bump `com.squareup.okio:okio` from 3.7.0 to 3.8.0 ([#12290](https://github.com/opensearch-project/OpenSearch/pull/12290)) From 4010ff129310eb84addd3ac07ff769ea8cc15f4e Mon Sep 17 00:00:00 2001 From: Neetika Singhal Date: Fri, 22 Mar 2024 05:41:57 -0700 Subject: [PATCH 32/38] Fix SimpleNestedExplainIT.testExplainMultipleDocs flakiness (#12776) Signed-off-by: Neetika Singhal --- .../search/nested/SimpleNestedExplainIT.java | 23 ++++++++++++++++++- 1 file changed, 22 insertions(+), 1 deletion(-) diff --git a/server/src/internalClusterTest/java/org/opensearch/search/nested/SimpleNestedExplainIT.java b/server/src/internalClusterTest/java/org/opensearch/search/nested/SimpleNestedExplainIT.java index a6554271a0bc5..2efec6a63e6c1 100644 --- a/server/src/internalClusterTest/java/org/opensearch/search/nested/SimpleNestedExplainIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/search/nested/SimpleNestedExplainIT.java @@ -30,6 +30,11 @@ */ public class SimpleNestedExplainIT extends OpenSearchIntegTestCase { + @Override + protected int numberOfShards() { + return 1; + } + /* * Tests the explain output for multiple docs. Concurrent search with multiple slices is tested * here as call to indexRandomForMultipleSlices is made and compared with explain output for @@ -70,7 +75,23 @@ public void testExplainMultipleDocs() throws Exception { .setRefreshPolicy(IMMEDIATE) .get(); - indexRandomForMultipleSlices("test"); + client().prepareIndex("test") + .setId("2") + .setSource( + jsonBuilder().startObject() + .field("field1", "value2") + .startArray("nested1") + .startObject() + .field("n_field1", "n_value2") + .endObject() + .startObject() + .field("n_field1", "n_value2") + .endObject() + .endArray() + .endObject() + ) + .setRefreshPolicy(IMMEDIATE) + .get(); // Turn off the concurrent search setting to test search with non-concurrent search client().admin() From c04dad58bbc941b8d5f18326d5c5cfb7ac312239 Mon Sep 17 00:00:00 2001 From: Zelin Hao Date: Fri, 22 Mar 2024 06:31:59 -0700 Subject: [PATCH 33/38] Add explicit dependency to validatePom and generatePom tasks. (#12807) * Add explicit dependency to PomValidation Signed-off-by: Zelin Hao * Update CHANGELOG Signed-off-by: Zelin Hao --------- Signed-off-by: Zelin Hao --- CHANGELOG.md | 1 + .../opensearch/gradle/pluginzip/Publish.java | 9 +++++---- .../precommit/PomValidationPrecommitPlugin.java | 17 ++++++++++++----- .../gradle/precommit/PomValidationTask.java | 1 + 4 files changed, 19 insertions(+), 9 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index af20332c61146..74042a06c73c3 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -19,6 +19,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Remote reindex: Add support for configurable retry mechanism ([#12561](https://github.com/opensearch-project/OpenSearch/pull/12561)) - [Admission Control] Integrate IO Usage Tracker to the Resource Usage Collector Service and Emit IO Usage Stats ([#11880](https://github.com/opensearch-project/OpenSearch/pull/11880)) - Tracing for deep search path ([#12103](https://github.com/opensearch-project/OpenSearch/pull/12103)) +- Add explicit dependency to validatePom and generatePom tasks ([#12103](https://github.com/opensearch-project/OpenSearch/pull/12807)) ### Dependencies - Bump `log4j-core` from 2.18.0 to 2.19.0 diff --git a/buildSrc/src/main/java/org/opensearch/gradle/pluginzip/Publish.java b/buildSrc/src/main/java/org/opensearch/gradle/pluginzip/Publish.java index 5d7e78589306f..599beb8649fcd 100644 --- a/buildSrc/src/main/java/org/opensearch/gradle/pluginzip/Publish.java +++ b/buildSrc/src/main/java/org/opensearch/gradle/pluginzip/Publish.java @@ -65,9 +65,6 @@ public void apply(Project project) { addLocalMavenRepo(project); addZipArtifact(project); Task validatePluginZipPom = project.getTasks().findByName("validatePluginZipPom"); - if (validatePluginZipPom != null) { - validatePluginZipPom.dependsOn("generatePomFileForNebulaPublication"); - } // There are number of tasks prefixed by 'publishPluginZipPublication', f.e.: // publishPluginZipPublicationToZipStagingRepository, publishPluginZipPublicationToMavenLocal @@ -76,7 +73,11 @@ public void apply(Project project) { .filter(t -> t.getName().startsWith("publishPluginZipPublicationTo")) .collect(Collectors.toSet()); if (!publishPluginZipPublicationToTasks.isEmpty()) { - publishPluginZipPublicationToTasks.forEach(t -> t.dependsOn("generatePomFileForNebulaPublication")); + if (validatePluginZipPom != null) { + publishPluginZipPublicationToTasks.forEach(t -> t.dependsOn(validatePluginZipPom)); + } else { + publishPluginZipPublicationToTasks.forEach(t -> t.dependsOn("generatePomFileForNebulaPublication")); + } } } else { project.getLogger() diff --git a/buildSrc/src/main/java/org/opensearch/gradle/precommit/PomValidationPrecommitPlugin.java b/buildSrc/src/main/java/org/opensearch/gradle/precommit/PomValidationPrecommitPlugin.java index 0e7a357dd5d18..62194af3c3350 100644 --- a/buildSrc/src/main/java/org/opensearch/gradle/precommit/PomValidationPrecommitPlugin.java +++ b/buildSrc/src/main/java/org/opensearch/gradle/precommit/PomValidationPrecommitPlugin.java @@ -48,17 +48,24 @@ public class PomValidationPrecommitPlugin extends PrecommitPlugin { public TaskProvider createTask(Project project) { TaskProvider validatePom = project.getTasks().register("validatePom"); PublishingExtension publishing = project.getExtensions().getByType(PublishingExtension.class); - publishing.getPublications().all(publication -> { + publishing.getPublications().configureEach(publication -> { String publicationName = Util.capitalize(publication.getName()); TaskProvider validateTask = project.getTasks() .register("validate" + publicationName + "Pom", PomValidationTask.class); validatePom.configure(t -> t.dependsOn(validateTask)); + TaskProvider generateMavenPom = project.getTasks() + .withType(GenerateMavenPom.class) + .named("generatePomFileFor" + publicationName + "Publication"); validateTask.configure(task -> { - GenerateMavenPom generateMavenPom = project.getTasks() - .withType(GenerateMavenPom.class) - .getByName("generatePomFileFor" + publicationName + "Publication"); task.dependsOn(generateMavenPom); - task.getPomFile().fileValue(generateMavenPom.getDestination()); + task.getPomFile().fileProvider(generateMavenPom.map(GenerateMavenPom::getDestination)); + publishing.getPublications().configureEach(publicationForPomGen -> { + task.mustRunAfter( + project.getTasks() + .withType(GenerateMavenPom.class) + .getByName("generatePomFileFor" + Util.capitalize(publicationForPomGen.getName()) + "Publication") + ); + }); }); }); diff --git a/buildSrc/src/main/java/org/opensearch/gradle/precommit/PomValidationTask.java b/buildSrc/src/main/java/org/opensearch/gradle/precommit/PomValidationTask.java index aca882fbb6477..b76e0d6dd93cf 100644 --- a/buildSrc/src/main/java/org/opensearch/gradle/precommit/PomValidationTask.java +++ b/buildSrc/src/main/java/org/opensearch/gradle/precommit/PomValidationTask.java @@ -106,6 +106,7 @@ private void validateNonNull(String element, T value, Runnable validator) { private void validateString(String element, String value) { validateNonNull(element, value, () -> validateNonEmpty(element, value, s -> s.trim().isEmpty())); + getLogger().info(element + " with value " + value + " is validated."); } private void validateCollection(String element, Collection value, Consumer validator) { From ac0bb8344a2dcd0b1c251e450710801c78b6249f Mon Sep 17 00:00:00 2001 From: Andriy Redko Date: Fri, 22 Mar 2024 12:42:50 -0400 Subject: [PATCH 34/38] Added missed API visibility annotations for public / experimental APIs (#12864) Signed-off-by: Andriy Redko --- .../java/org/opensearch/OpenSearchException.java | 4 +++- .../core/common/breaker/CircuitBreaker.java | 9 +++++++-- .../core/common/transport/BoundTransportAddress.java | 4 +++- .../opensearch/core/transport/TransportResponse.java | 4 +++- .../main/java/org/opensearch/semver/SemverRange.java | 4 ++++ .../clustermanager/ClusterManagerNodeRequest.java | 4 +++- .../org/opensearch/common/cache/RemovalListener.java | 5 ++++- .../common/cache/policy/CachedQueryResult.java | 4 ++++ .../common/cache/serializer/Serializer.java | 5 +++++ .../lucene/index/OpenSearchDirectoryReader.java | 7 ++++++- .../org/opensearch/common/metrics/MeanMetric.java | 5 ++++- .../src/main/java/org/opensearch/http/HttpInfo.java | 4 +++- .../org/opensearch/http/HttpServerTransport.java | 4 +++- .../src/main/java/org/opensearch/http/HttpStats.java | 4 +++- .../index/codec/fuzzy/LongArrayBackedBitSet.java | 2 +- .../store/RemoteSegmentStoreDirectoryFactory.java | 4 +++- .../java/org/opensearch/rest/BaseRestHandler.java | 5 +++++ .../main/java/org/opensearch/rest/RestChannel.java | 2 ++ .../main/java/org/opensearch/rest/RestHandler.java | 4 +++- .../main/java/org/opensearch/rest/RestResponse.java | 2 ++ .../org/opensearch/transport/ConnectionProfile.java | 4 +++- .../main/java/org/opensearch/transport/Header.java | 4 +++- .../org/opensearch/transport/InboundMessage.java | 4 +++- .../opensearch/transport/RequestHandlerRegistry.java | 4 +++- .../java/org/opensearch/transport/StatsTracker.java | 4 +++- .../java/org/opensearch/transport/TcpChannel.java | 7 +++++-- .../java/org/opensearch/transport/TcpTransport.java | 7 ++++++- .../java/org/opensearch/transport/Transport.java | 12 ++++++++++-- .../org/opensearch/transport/TransportChannel.java | 2 ++ .../transport/TransportMessageListener.java | 4 +++- .../transport/TransportRequestHandler.java | 4 +++- .../transport/TransportResponseHandler.java | 4 +++- .../org/opensearch/transport/TransportStats.java | 4 +++- 33 files changed, 121 insertions(+), 29 deletions(-) diff --git a/libs/core/src/main/java/org/opensearch/OpenSearchException.java b/libs/core/src/main/java/org/opensearch/OpenSearchException.java index cce86b452f698..dda3983fbb4d1 100644 --- a/libs/core/src/main/java/org/opensearch/OpenSearchException.java +++ b/libs/core/src/main/java/org/opensearch/OpenSearchException.java @@ -33,6 +33,7 @@ import org.opensearch.common.CheckedFunction; import org.opensearch.common.Nullable; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.collect.Tuple; import org.opensearch.core.ParseField; import org.opensearch.core.common.Strings; @@ -69,8 +70,9 @@ /** * A core library base class for all opensearch exceptions. * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public class OpenSearchException extends RuntimeException implements Writeable, ToXContentFragment { protected static final Version UNKNOWN_VERSION_ADDED = Version.fromId(0); diff --git a/libs/core/src/main/java/org/opensearch/core/common/breaker/CircuitBreaker.java b/libs/core/src/main/java/org/opensearch/core/common/breaker/CircuitBreaker.java index 846950ff17c63..9a09b3b38a5f2 100644 --- a/libs/core/src/main/java/org/opensearch/core/common/breaker/CircuitBreaker.java +++ b/libs/core/src/main/java/org/opensearch/core/common/breaker/CircuitBreaker.java @@ -32,14 +32,17 @@ package org.opensearch.core.common.breaker; +import org.opensearch.common.annotation.PublicApi; + import java.util.Locale; /** * Interface for an object that can be incremented, breaking after some * configured limit has been reached. * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public interface CircuitBreaker { /** @@ -72,8 +75,10 @@ public interface CircuitBreaker { /** * The type of breaker * can be {@link #MEMORY}, {@link #PARENT}, or {@link #NOOP} - * @opensearch.internal + * + * @opensearch.api */ + @PublicApi(since = "1.0.0") enum Type { /** A regular or ChildMemoryCircuitBreaker */ MEMORY, diff --git a/libs/core/src/main/java/org/opensearch/core/common/transport/BoundTransportAddress.java b/libs/core/src/main/java/org/opensearch/core/common/transport/BoundTransportAddress.java index 8908a172395f2..e2266339c058f 100644 --- a/libs/core/src/main/java/org/opensearch/core/common/transport/BoundTransportAddress.java +++ b/libs/core/src/main/java/org/opensearch/core/common/transport/BoundTransportAddress.java @@ -32,6 +32,7 @@ package org.opensearch.core.common.transport; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.network.InetAddresses; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; @@ -44,8 +45,9 @@ * the addresses the transport is bound to, and the other is the published one that represents the address clients * should communicate on. * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public class BoundTransportAddress implements Writeable { private TransportAddress[] boundAddresses; diff --git a/libs/core/src/main/java/org/opensearch/core/transport/TransportResponse.java b/libs/core/src/main/java/org/opensearch/core/transport/TransportResponse.java index 038069e93a51b..4ae01e140a89c 100644 --- a/libs/core/src/main/java/org/opensearch/core/transport/TransportResponse.java +++ b/libs/core/src/main/java/org/opensearch/core/transport/TransportResponse.java @@ -32,6 +32,7 @@ package org.opensearch.core.transport; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; @@ -40,8 +41,9 @@ /** * Response over the transport interface * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public abstract class TransportResponse extends TransportMessage { /** diff --git a/libs/core/src/main/java/org/opensearch/semver/SemverRange.java b/libs/core/src/main/java/org/opensearch/semver/SemverRange.java index da87acc7124aa..da8c06c07d8e5 100644 --- a/libs/core/src/main/java/org/opensearch/semver/SemverRange.java +++ b/libs/core/src/main/java/org/opensearch/semver/SemverRange.java @@ -10,6 +10,7 @@ import org.opensearch.Version; import org.opensearch.common.Nullable; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.core.xcontent.ToXContentFragment; import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.semver.expr.Caret; @@ -31,7 +32,10 @@ *
  • '~' Allows for patch version variability starting from the range version. For example, ~1.2.3 range would match versions greater than or equal to 1.2.3 but less than 1.3.0
  • *
  • '^' Allows for patch and minor version variability starting from the range version. For example, ^1.2.3 range would match versions greater than or equal to 1.2.3 but less than 2.0.0
  • * + * + * @opensearch.api */ +@PublicApi(since = "2.13.0") public class SemverRange implements ToXContentFragment { private final Version rangeVersion; diff --git a/server/src/main/java/org/opensearch/action/support/clustermanager/ClusterManagerNodeRequest.java b/server/src/main/java/org/opensearch/action/support/clustermanager/ClusterManagerNodeRequest.java index a43d6fb0b1e7a..03fc41e829e3d 100644 --- a/server/src/main/java/org/opensearch/action/support/clustermanager/ClusterManagerNodeRequest.java +++ b/server/src/main/java/org/opensearch/action/support/clustermanager/ClusterManagerNodeRequest.java @@ -33,6 +33,7 @@ package org.opensearch.action.support.clustermanager; import org.opensearch.action.ActionRequest; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.unit.TimeValue; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; @@ -42,8 +43,9 @@ /** * A based request for cluster-manager based operation. * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public abstract class ClusterManagerNodeRequest> extends ActionRequest { public static final TimeValue DEFAULT_CLUSTER_MANAGER_NODE_TIMEOUT = TimeValue.timeValueSeconds(30); diff --git a/server/src/main/java/org/opensearch/common/cache/RemovalListener.java b/server/src/main/java/org/opensearch/common/cache/RemovalListener.java index 369313f9f93f4..68e1cdf6139e2 100644 --- a/server/src/main/java/org/opensearch/common/cache/RemovalListener.java +++ b/server/src/main/java/org/opensearch/common/cache/RemovalListener.java @@ -32,11 +32,14 @@ package org.opensearch.common.cache; +import org.opensearch.common.annotation.ExperimentalApi; + /** * Listener for removing an element from the cache * - * @opensearch.internal + * @opensearch.experimental */ +@ExperimentalApi @FunctionalInterface public interface RemovalListener { void onRemoval(RemovalNotification notification); diff --git a/server/src/main/java/org/opensearch/common/cache/policy/CachedQueryResult.java b/server/src/main/java/org/opensearch/common/cache/policy/CachedQueryResult.java index 0a98542a05bb7..df698112c60d1 100644 --- a/server/src/main/java/org/opensearch/common/cache/policy/CachedQueryResult.java +++ b/server/src/main/java/org/opensearch/common/cache/policy/CachedQueryResult.java @@ -8,6 +8,7 @@ package org.opensearch.common.cache.policy; +import org.opensearch.common.annotation.ExperimentalApi; import org.opensearch.core.common.bytes.BytesReference; import org.opensearch.core.common.io.stream.NamedWriteableAwareStreamInput; import org.opensearch.core.common.io.stream.NamedWriteableRegistry; @@ -62,7 +63,10 @@ public void writeToNoId(StreamOutput out) throws IOException { /** * A class containing information needed for all cache policies * to decide whether to admit a given value. + * + * @opensearch.experimental */ + @ExperimentalApi public static class PolicyValues implements Writeable { final long tookTimeNanos; // More values can be added here as they're needed for future policies diff --git a/server/src/main/java/org/opensearch/common/cache/serializer/Serializer.java b/server/src/main/java/org/opensearch/common/cache/serializer/Serializer.java index 35e28707d1ca3..46a8ed5a72ccf 100644 --- a/server/src/main/java/org/opensearch/common/cache/serializer/Serializer.java +++ b/server/src/main/java/org/opensearch/common/cache/serializer/Serializer.java @@ -8,10 +8,15 @@ package org.opensearch.common.cache.serializer; +import org.opensearch.common.annotation.ExperimentalApi; + /** * Defines an interface for serializers, to be used by pluggable caches. * T is the class of the original object, and U is the serialized class. + * + * @opensearch.experimental */ +@ExperimentalApi public interface Serializer { /** * Serializes an object. diff --git a/server/src/main/java/org/opensearch/common/lucene/index/OpenSearchDirectoryReader.java b/server/src/main/java/org/opensearch/common/lucene/index/OpenSearchDirectoryReader.java index f9a87b9e74214..ec2cfde84ca5f 100644 --- a/server/src/main/java/org/opensearch/common/lucene/index/OpenSearchDirectoryReader.java +++ b/server/src/main/java/org/opensearch/common/lucene/index/OpenSearchDirectoryReader.java @@ -84,8 +84,10 @@ public DelegatingCacheHelper getDelegatingCacheHelper() { /** * Wraps existing IndexReader cache helper which internally provides a way to wrap CacheKey. - * @opensearch.internal + * + * @opensearch.api */ + @PublicApi(since = "2.13.0") public class DelegatingCacheHelper implements CacheHelper { private final CacheHelper cacheHelper; private final DelegatingCacheKey serializableCacheKey; @@ -113,7 +115,10 @@ public void addClosedListener(ClosedListener listener) { /** * Wraps internal IndexReader.CacheKey and attaches a uniqueId to it which can be eventually be used instead of * object itself for serialization purposes. + * + * @opensearch.api */ + @PublicApi(since = "2.13.0") public class DelegatingCacheKey { private final CacheKey cacheKey; private final String uniqueId; diff --git a/server/src/main/java/org/opensearch/common/metrics/MeanMetric.java b/server/src/main/java/org/opensearch/common/metrics/MeanMetric.java index 359facdce633b..94d44d5b35d74 100644 --- a/server/src/main/java/org/opensearch/common/metrics/MeanMetric.java +++ b/server/src/main/java/org/opensearch/common/metrics/MeanMetric.java @@ -32,13 +32,16 @@ package org.opensearch.common.metrics; +import org.opensearch.common.annotation.PublicApi; + import java.util.concurrent.atomic.LongAdder; /** * An average metric for tracking. * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public class MeanMetric implements Metric { private final LongAdder counter = new LongAdder(); diff --git a/server/src/main/java/org/opensearch/http/HttpInfo.java b/server/src/main/java/org/opensearch/http/HttpInfo.java index 10f2d50dacb14..4a39e40c471b1 100644 --- a/server/src/main/java/org/opensearch/http/HttpInfo.java +++ b/server/src/main/java/org/opensearch/http/HttpInfo.java @@ -32,6 +32,7 @@ package org.opensearch.http; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.logging.DeprecationLogger; import org.opensearch.common.network.InetAddresses; import org.opensearch.core.common.io.stream.StreamInput; @@ -47,8 +48,9 @@ /** * Information about an http connection * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public class HttpInfo implements ReportingService.Info { private static final DeprecationLogger deprecationLogger = DeprecationLogger.getLogger(HttpInfo.class); diff --git a/server/src/main/java/org/opensearch/http/HttpServerTransport.java b/server/src/main/java/org/opensearch/http/HttpServerTransport.java index 890136cb67e60..012b69c29c1d4 100644 --- a/server/src/main/java/org/opensearch/http/HttpServerTransport.java +++ b/server/src/main/java/org/opensearch/http/HttpServerTransport.java @@ -32,6 +32,7 @@ package org.opensearch.http; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.lifecycle.LifecycleComponent; import org.opensearch.common.util.concurrent.ThreadContext; import org.opensearch.core.common.transport.BoundTransportAddress; @@ -42,8 +43,9 @@ /** * HTTP Transport server * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public interface HttpServerTransport extends LifecycleComponent, ReportingService { String HTTP_SERVER_WORKER_THREAD_NAME_PREFIX = "http_server_worker"; diff --git a/server/src/main/java/org/opensearch/http/HttpStats.java b/server/src/main/java/org/opensearch/http/HttpStats.java index 078b84b7bc563..f69eff59e830d 100644 --- a/server/src/main/java/org/opensearch/http/HttpStats.java +++ b/server/src/main/java/org/opensearch/http/HttpStats.java @@ -32,6 +32,7 @@ package org.opensearch.http; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; import org.opensearch.core.common.io.stream.Writeable; @@ -43,8 +44,9 @@ /** * Stats for HTTP connections * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public class HttpStats implements Writeable, ToXContentFragment { private final long serverOpen; diff --git a/server/src/main/java/org/opensearch/index/codec/fuzzy/LongArrayBackedBitSet.java b/server/src/main/java/org/opensearch/index/codec/fuzzy/LongArrayBackedBitSet.java index bd4936aeec366..392a925c21143 100644 --- a/server/src/main/java/org/opensearch/index/codec/fuzzy/LongArrayBackedBitSet.java +++ b/server/src/main/java/org/opensearch/index/codec/fuzzy/LongArrayBackedBitSet.java @@ -39,7 +39,7 @@ class LongArrayBackedBitSet implements Accountable, Closeable { /** * Constructor which uses Lucene's IndexInput to read the bitset into a read-only buffer. * @param in IndexInput containing the serialized bitset. - * @throws IOException + * @throws IOException I/O exception */ LongArrayBackedBitSet(IndexInput in) throws IOException { underlyingArrayLength = in.readLong(); diff --git a/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryFactory.java b/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryFactory.java index eca8d9ec702e1..d6d3f1fca833c 100644 --- a/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryFactory.java +++ b/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryFactory.java @@ -9,6 +9,7 @@ package org.opensearch.index.store; import org.apache.lucene.store.Directory; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.blobstore.BlobPath; import org.opensearch.core.index.shard.ShardId; import org.opensearch.index.IndexSettings; @@ -28,8 +29,9 @@ /** * Factory for a remote store directory * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "2.3.0") public class RemoteSegmentStoreDirectoryFactory implements IndexStorePlugin.DirectoryFactory { private static final String SEGMENTS = "segments"; diff --git a/server/src/main/java/org/opensearch/rest/BaseRestHandler.java b/server/src/main/java/org/opensearch/rest/BaseRestHandler.java index e18a594236fc8..3552e32022b2c 100644 --- a/server/src/main/java/org/opensearch/rest/BaseRestHandler.java +++ b/server/src/main/java/org/opensearch/rest/BaseRestHandler.java @@ -40,6 +40,7 @@ import org.opensearch.action.support.clustermanager.ClusterManagerNodeRequest; import org.opensearch.client.node.NodeClient; import org.opensearch.common.CheckedConsumer; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.collect.Tuple; import org.opensearch.common.logging.DeprecationLogger; import org.opensearch.common.settings.Setting; @@ -73,6 +74,7 @@ * * @opensearch.api */ +@PublicApi(since = "1.0.0") public abstract class BaseRestHandler implements RestHandler { public static final Setting MULTI_ALLOW_EXPLICIT_INDEX = Setting.boolSetting( @@ -195,8 +197,11 @@ protected final String unrecognized( /** * REST requests are handled by preparing a channel consumer that represents the execution of * the request against a channel. + * + * @opensearch.api */ @FunctionalInterface + @PublicApi(since = "1.0.0") protected interface RestChannelConsumer extends CheckedConsumer {} /** diff --git a/server/src/main/java/org/opensearch/rest/RestChannel.java b/server/src/main/java/org/opensearch/rest/RestChannel.java index b8ce3e92e0098..b3ded1389f754 100644 --- a/server/src/main/java/org/opensearch/rest/RestChannel.java +++ b/server/src/main/java/org/opensearch/rest/RestChannel.java @@ -33,6 +33,7 @@ package org.opensearch.rest; import org.opensearch.common.Nullable; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.io.stream.BytesStreamOutput; import org.opensearch.core.xcontent.MediaType; import org.opensearch.core.xcontent.XContentBuilder; @@ -44,6 +45,7 @@ * * @opensearch.api */ +@PublicApi(since = "1.0.0") public interface RestChannel { XContentBuilder newBuilder() throws IOException; diff --git a/server/src/main/java/org/opensearch/rest/RestHandler.java b/server/src/main/java/org/opensearch/rest/RestHandler.java index 294dc3ffbe329..387e8b25a8274 100644 --- a/server/src/main/java/org/opensearch/rest/RestHandler.java +++ b/server/src/main/java/org/opensearch/rest/RestHandler.java @@ -33,6 +33,7 @@ package org.opensearch.rest; import org.opensearch.client.node.NodeClient; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.core.xcontent.XContent; import org.opensearch.rest.RestRequest.Method; @@ -180,8 +181,9 @@ public boolean allowSystemIndexAccessByDefault() { /** * Route for the request. * - * @opensearch.internal + * @opensearch.api */ + @PublicApi(since = "1.0.0") class Route { protected final String path; diff --git a/server/src/main/java/org/opensearch/rest/RestResponse.java b/server/src/main/java/org/opensearch/rest/RestResponse.java index 2eff746e8508c..482eb6b052e9b 100644 --- a/server/src/main/java/org/opensearch/rest/RestResponse.java +++ b/server/src/main/java/org/opensearch/rest/RestResponse.java @@ -33,6 +33,7 @@ package org.opensearch.rest; import org.opensearch.OpenSearchException; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.lease.Releasable; import org.opensearch.core.common.bytes.BytesReference; import org.opensearch.core.rest.RestStatus; @@ -49,6 +50,7 @@ * * @opensearch.api */ +@PublicApi(since = "1.0.0") public abstract class RestResponse { private Map> customHeaders; diff --git a/server/src/main/java/org/opensearch/transport/ConnectionProfile.java b/server/src/main/java/org/opensearch/transport/ConnectionProfile.java index b9764c0c53f4a..931707e4a1cdc 100644 --- a/server/src/main/java/org/opensearch/transport/ConnectionProfile.java +++ b/server/src/main/java/org/opensearch/transport/ConnectionProfile.java @@ -33,6 +33,7 @@ import org.opensearch.cluster.node.DiscoveryNode; import org.opensearch.common.Nullable; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.settings.Settings; import org.opensearch.common.unit.TimeValue; @@ -49,8 +50,9 @@ * A connection profile describes how many connection are established to specific node for each of the available request types. * ({@link org.opensearch.transport.TransportRequestOptions.Type}). This allows to tailor a connection towards a specific usage. * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public final class ConnectionProfile { /** diff --git a/server/src/main/java/org/opensearch/transport/Header.java b/server/src/main/java/org/opensearch/transport/Header.java index a179cfb35288e..57c1da6f46aec 100644 --- a/server/src/main/java/org/opensearch/transport/Header.java +++ b/server/src/main/java/org/opensearch/transport/Header.java @@ -33,6 +33,7 @@ package org.opensearch.transport; import org.opensearch.Version; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.collect.Tuple; import org.opensearch.common.util.concurrent.ThreadContext; import org.opensearch.core.common.io.stream.StreamInput; @@ -47,8 +48,9 @@ /** * Transport Header * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public class Header { private static final String RESPONSE_NAME = "NO_ACTION_NAME_FOR_RESPONSES"; diff --git a/server/src/main/java/org/opensearch/transport/InboundMessage.java b/server/src/main/java/org/opensearch/transport/InboundMessage.java index a1ed682ff7d7f..71c4d6973505d 100644 --- a/server/src/main/java/org/opensearch/transport/InboundMessage.java +++ b/server/src/main/java/org/opensearch/transport/InboundMessage.java @@ -32,6 +32,7 @@ package org.opensearch.transport; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.bytes.ReleasableBytesReference; import org.opensearch.common.lease.Releasable; import org.opensearch.common.lease.Releasables; @@ -43,8 +44,9 @@ /** * Inbound data as a message * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public class InboundMessage implements Releasable { private final Header header; diff --git a/server/src/main/java/org/opensearch/transport/RequestHandlerRegistry.java b/server/src/main/java/org/opensearch/transport/RequestHandlerRegistry.java index 98c182c562928..4368dbdece6cf 100644 --- a/server/src/main/java/org/opensearch/transport/RequestHandlerRegistry.java +++ b/server/src/main/java/org/opensearch/transport/RequestHandlerRegistry.java @@ -32,6 +32,7 @@ package org.opensearch.transport; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.lease.Releasable; import org.opensearch.common.lease.Releasables; import org.opensearch.common.util.concurrent.ThreadContext; @@ -47,8 +48,9 @@ /** * Registry for OpenSearch RequestHandlers * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public final class RequestHandlerRegistry { private final String action; diff --git a/server/src/main/java/org/opensearch/transport/StatsTracker.java b/server/src/main/java/org/opensearch/transport/StatsTracker.java index 5548d2d558ae2..02bc0a51c9330 100644 --- a/server/src/main/java/org/opensearch/transport/StatsTracker.java +++ b/server/src/main/java/org/opensearch/transport/StatsTracker.java @@ -32,6 +32,7 @@ package org.opensearch.transport; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.metrics.MeanMetric; import java.util.concurrent.atomic.LongAdder; @@ -39,8 +40,9 @@ /** * Tracks transport statistics * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public final class StatsTracker { private final LongAdder bytesRead = new LongAdder(); diff --git a/server/src/main/java/org/opensearch/transport/TcpChannel.java b/server/src/main/java/org/opensearch/transport/TcpChannel.java index f98b65d0a4df1..7d4515de85d80 100644 --- a/server/src/main/java/org/opensearch/transport/TcpChannel.java +++ b/server/src/main/java/org/opensearch/transport/TcpChannel.java @@ -32,6 +32,7 @@ package org.opensearch.transport; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.network.CloseableChannel; import org.opensearch.common.unit.TimeValue; import org.opensearch.core.action.ActionListener; @@ -45,8 +46,9 @@ * abstraction used by the {@link TcpTransport} and {@link TransportService}. All tcp transport * implementations must return channels that adhere to the required method contracts. * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public interface TcpChannel extends CloseableChannel { /** @@ -114,8 +116,9 @@ default Optional get(String name, Class clazz) { /** * Channel statistics * - * @opensearch.internal + * @opensearch.api */ + @PublicApi(since = "1.0.0") class ChannelStats { private volatile long lastAccessedTime; diff --git a/server/src/main/java/org/opensearch/transport/TcpTransport.java b/server/src/main/java/org/opensearch/transport/TcpTransport.java index d0e6516973382..7d45152089f37 100644 --- a/server/src/main/java/org/opensearch/transport/TcpTransport.java +++ b/server/src/main/java/org/opensearch/transport/TcpTransport.java @@ -39,6 +39,7 @@ import org.opensearch.action.support.ThreadedActionListener; import org.opensearch.cluster.node.DiscoveryNode; import org.opensearch.common.Booleans; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.lifecycle.AbstractLifecycleComponent; import org.opensearch.common.lifecycle.Lifecycle; import org.opensearch.common.metrics.MeanMetric; @@ -111,8 +112,9 @@ /** * The TCP Transport layer * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public abstract class TcpTransport extends AbstractLifecycleComponent implements Transport { private static final Logger logger = LogManager.getLogger(TcpTransport.class); @@ -966,7 +968,10 @@ public static Set getProfileSettings(Settings settings) { /** * Representation of a transport profile settings for a {@code transport.profiles.$profilename.*} + * + * @opensearch.api */ + @PublicApi(since = "1.0.0") public static final class ProfileSettings { public final String profileName; public final boolean tcpNoDelay; diff --git a/server/src/main/java/org/opensearch/transport/Transport.java b/server/src/main/java/org/opensearch/transport/Transport.java index 8abedff37db14..b89393615c95f 100644 --- a/server/src/main/java/org/opensearch/transport/Transport.java +++ b/server/src/main/java/org/opensearch/transport/Transport.java @@ -58,8 +58,9 @@ /** * OpenSearch Transport Interface * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public interface Transport extends LifecycleComponent { /** @@ -166,7 +167,10 @@ default Object getCacheKey() { /** * This class represents a response context that encapsulates the actual response handler, the action and the connection it was * executed on. + * + * @opensearch.api */ + @PublicApi(since = "1.0.0") final class ResponseContext { private final TransportResponseHandler handler; @@ -196,7 +200,10 @@ public String action() { /** * This class is a registry that allows + * + * @opensearch.api */ + @PublicApi(since = "1.0.0") final class ResponseHandlers { private final ConcurrentMapLong> handlers = ConcurrentCollections .newConcurrentMapLongWithAggressiveConcurrency(); @@ -276,8 +283,9 @@ public TransportResponseHandler onResponseReceived( /** * Request handler implementations * - * @opensearch.internal + * @opensearch.api */ + @PublicApi(since = "1.0.0") final class RequestHandlers { private volatile Map> requestHandlers = Collections.emptyMap(); diff --git a/server/src/main/java/org/opensearch/transport/TransportChannel.java b/server/src/main/java/org/opensearch/transport/TransportChannel.java index f84ee5dc745c3..7b6715ff2c73d 100644 --- a/server/src/main/java/org/opensearch/transport/TransportChannel.java +++ b/server/src/main/java/org/opensearch/transport/TransportChannel.java @@ -36,6 +36,7 @@ import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; import org.opensearch.Version; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.core.transport.TransportResponse; import java.io.IOException; @@ -46,6 +47,7 @@ * * @opensearch.internal */ +@PublicApi(since = "1.0.0") public interface TransportChannel { Logger logger = LogManager.getLogger(TransportChannel.class); diff --git a/server/src/main/java/org/opensearch/transport/TransportMessageListener.java b/server/src/main/java/org/opensearch/transport/TransportMessageListener.java index dfcd7acce3706..284c4646655c5 100644 --- a/server/src/main/java/org/opensearch/transport/TransportMessageListener.java +++ b/server/src/main/java/org/opensearch/transport/TransportMessageListener.java @@ -32,13 +32,15 @@ package org.opensearch.transport; import org.opensearch.cluster.node.DiscoveryNode; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.core.transport.TransportResponse; /** * Listens for transport messages * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public interface TransportMessageListener { TransportMessageListener NOOP_LISTENER = new TransportMessageListener() { diff --git a/server/src/main/java/org/opensearch/transport/TransportRequestHandler.java b/server/src/main/java/org/opensearch/transport/TransportRequestHandler.java index 54ee1b68fc9aa..0419c0b82029b 100644 --- a/server/src/main/java/org/opensearch/transport/TransportRequestHandler.java +++ b/server/src/main/java/org/opensearch/transport/TransportRequestHandler.java @@ -32,13 +32,15 @@ package org.opensearch.transport; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.tasks.Task; /** * Handles transport requests * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public interface TransportRequestHandler { void messageReceived(T request, TransportChannel channel, Task task) throws Exception; diff --git a/server/src/main/java/org/opensearch/transport/TransportResponseHandler.java b/server/src/main/java/org/opensearch/transport/TransportResponseHandler.java index 8992af18edb48..748d2a4d867ec 100644 --- a/server/src/main/java/org/opensearch/transport/TransportResponseHandler.java +++ b/server/src/main/java/org/opensearch/transport/TransportResponseHandler.java @@ -32,6 +32,7 @@ package org.opensearch.transport; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.Writeable; import org.opensearch.core.transport.TransportResponse; @@ -42,8 +43,9 @@ /** * Handles transport responses * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public interface TransportResponseHandler extends Writeable.Reader { void handleResponse(T response); diff --git a/server/src/main/java/org/opensearch/transport/TransportStats.java b/server/src/main/java/org/opensearch/transport/TransportStats.java index e3c4773f4a472..01980ce529caa 100644 --- a/server/src/main/java/org/opensearch/transport/TransportStats.java +++ b/server/src/main/java/org/opensearch/transport/TransportStats.java @@ -32,6 +32,7 @@ package org.opensearch.transport; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; import org.opensearch.core.common.io.stream.Writeable; @@ -44,8 +45,9 @@ /** * Stats for transport activity * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public class TransportStats implements Writeable, ToXContentFragment { private final long serverOpen; From 8e332b6cfee70221a62426d4ae282df8dc205981 Mon Sep 17 00:00:00 2001 From: Andriy Redko Date: Fri, 22 Mar 2024 16:36:17 -0400 Subject: [PATCH 35/38] Added missed API visibility annotations for public APIs and enable the check at the build time (#12872) * Added missed API visibility annotations for public APIs and enable the check at the build time Signed-off-by: Andriy Redko * Address code review comments Signed-off-by: Andriy Redko --------- Signed-off-by: Andriy Redko --- CHANGELOG.md | 1 + server/build.gradle | 2 +- .../main/java/org/opensearch/action/ActionModule.java | 4 +++- .../main/java/org/opensearch/action/ActionRequest.java | 2 ++ .../src/main/java/org/opensearch/action/ActionType.java | 2 ++ .../org/opensearch/action/support/TransportAction.java | 4 +++- .../main/java/org/opensearch/client/node/NodeClient.java | 4 +++- .../main/java/org/opensearch/common/inject/Binder.java | 4 +++- .../main/java/org/opensearch/common/inject/Binding.java | 4 +++- .../opensearch/common/inject/ConfigurationException.java | 4 +++- .../main/java/org/opensearch/common/inject/Injector.java | 5 ++++- .../src/main/java/org/opensearch/common/inject/Key.java | 4 +++- .../org/opensearch/common/inject/MembersInjector.java | 5 ++++- .../main/java/org/opensearch/common/inject/Module.java | 5 ++++- .../java/org/opensearch/common/inject/PrivateBinder.java | 4 +++- .../main/java/org/opensearch/common/inject/Provider.java | 5 ++++- .../main/java/org/opensearch/common/inject/Scope.java | 5 ++++- .../main/java/org/opensearch/common/inject/Stage.java | 5 ++++- .../java/org/opensearch/common/inject/TypeLiteral.java | 4 +++- .../common/inject/binder/AnnotatedBindingBuilder.java | 5 ++++- .../inject/binder/AnnotatedConstantBindingBuilder.java | 5 ++++- .../common/inject/binder/AnnotatedElementBuilder.java | 5 ++++- .../common/inject/binder/ConstantBindingBuilder.java | 5 ++++- .../common/inject/binder/LinkedBindingBuilder.java | 4 +++- .../common/inject/binder/ScopedBindingBuilder.java | 4 +++- .../org/opensearch/common/inject/matcher/Matcher.java | 5 ++++- .../common/inject/spi/BindingScopingVisitor.java | 4 +++- .../common/inject/spi/BindingTargetVisitor.java | 5 ++++- .../opensearch/common/inject/spi/ConstructorBinding.java | 4 +++- .../common/inject/spi/ConvertedConstantBinding.java | 4 +++- .../org/opensearch/common/inject/spi/Dependency.java | 4 +++- .../java/org/opensearch/common/inject/spi/Element.java | 4 +++- .../org/opensearch/common/inject/spi/ElementVisitor.java | 4 +++- .../org/opensearch/common/inject/spi/ExposedBinding.java | 4 +++- .../opensearch/common/inject/spi/InjectionListener.java | 5 ++++- .../org/opensearch/common/inject/spi/InjectionPoint.java | 4 +++- .../opensearch/common/inject/spi/InjectionRequest.java | 4 +++- .../opensearch/common/inject/spi/InstanceBinding.java | 4 +++- .../opensearch/common/inject/spi/LinkedKeyBinding.java | 4 +++- .../common/inject/spi/MembersInjectorLookup.java | 4 +++- .../java/org/opensearch/common/inject/spi/Message.java | 4 +++- .../opensearch/common/inject/spi/PrivateElements.java | 4 +++- .../opensearch/common/inject/spi/ProviderBinding.java | 4 +++- .../common/inject/spi/ProviderInstanceBinding.java | 4 +++- .../opensearch/common/inject/spi/ProviderKeyBinding.java | 4 +++- .../org/opensearch/common/inject/spi/ProviderLookup.java | 4 +++- .../org/opensearch/common/inject/spi/ScopeBinding.java | 4 +++- .../common/inject/spi/StaticInjectionRequest.java | 4 +++- .../org/opensearch/common/inject/spi/TypeConverter.java | 4 +++- .../common/inject/spi/TypeConverterBinding.java | 4 +++- .../org/opensearch/common/inject/spi/TypeEncounter.java | 4 +++- .../org/opensearch/common/inject/spi/TypeListener.java | 4 +++- .../common/inject/spi/TypeListenerBinding.java | 4 +++- .../opensearch/common/inject/spi/UntargettedBinding.java | 4 +++- .../extensions/rest/RestSendToExtensionAction.java | 4 ++++ server/src/main/java/org/opensearch/rest/NamedRoute.java | 4 +++- .../src/main/java/org/opensearch/rest/RestHandler.java | 7 +++++++ .../src/main/java/org/opensearch/tasks/TaskListener.java | 5 ++++- .../org/opensearch/transport/RemoteClusterService.java | 4 +++- .../org/opensearch/transport/RemoteConnectionInfo.java | 7 +++++-- .../opensearch/transport/RemoteConnectionStrategy.java | 9 ++++++++- 61 files changed, 203 insertions(+), 57 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 74042a06c73c3..346913f025f4d 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -106,6 +106,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), ### Dependencies ### Changed +- [BWC and API enforcement] Enforcing the presence of API annotations at build time ([#12872](https://github.com/opensearch-project/OpenSearch/pull/12872)) ### Deprecated diff --git a/server/build.gradle b/server/build.gradle index e36498bf1038b..bbbe93bd6e517 100644 --- a/server/build.gradle +++ b/server/build.gradle @@ -141,7 +141,7 @@ tasks.withType(JavaCompile).configureEach { compileJava { options.compilerArgs += ['-processor', ['org.apache.logging.log4j.core.config.plugins.processor.PluginProcessor', - 'org.opensearch.common.annotation.processor.ApiAnnotationProcessor'].join(','), '-AcontinueOnFailingChecks'] + 'org.opensearch.common.annotation.processor.ApiAnnotationProcessor'].join(',')] } tasks.named("internalClusterTest").configure { diff --git a/server/src/main/java/org/opensearch/action/ActionModule.java b/server/src/main/java/org/opensearch/action/ActionModule.java index f827b7f3f0097..716e248b09692 100644 --- a/server/src/main/java/org/opensearch/action/ActionModule.java +++ b/server/src/main/java/org/opensearch/action/ActionModule.java @@ -294,6 +294,7 @@ import org.opensearch.cluster.metadata.IndexNameExpressionResolver; import org.opensearch.cluster.node.DiscoveryNodes; import org.opensearch.common.NamedRegistry; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.inject.AbstractModule; import org.opensearch.common.inject.TypeLiteral; import org.opensearch.common.inject.multibindings.MapBinder; @@ -1052,8 +1053,9 @@ public RestController getRestController() { *

    * This class is modeled after {@link NamedRegistry} but provides both register and unregister capabilities. * - * @opensearch.internal + * @opensearch.api */ + @PublicApi(since = "2.7.0") public static class DynamicActionRegistry { // This is the unmodifiable actions map created during node bootstrap, which // will continue to link ActionType and TransportAction pairs from core and plugin diff --git a/server/src/main/java/org/opensearch/action/ActionRequest.java b/server/src/main/java/org/opensearch/action/ActionRequest.java index 5313a05ad6fae..7ab87065bef7e 100644 --- a/server/src/main/java/org/opensearch/action/ActionRequest.java +++ b/server/src/main/java/org/opensearch/action/ActionRequest.java @@ -32,6 +32,7 @@ package org.opensearch.action; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; import org.opensearch.transport.TransportRequest; @@ -43,6 +44,7 @@ * * @opensearch.api */ +@PublicApi(since = "1.0.0") public abstract class ActionRequest extends TransportRequest { public ActionRequest() { diff --git a/server/src/main/java/org/opensearch/action/ActionType.java b/server/src/main/java/org/opensearch/action/ActionType.java index dae931bdd1891..559dad73536e1 100644 --- a/server/src/main/java/org/opensearch/action/ActionType.java +++ b/server/src/main/java/org/opensearch/action/ActionType.java @@ -32,6 +32,7 @@ package org.opensearch.action; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.settings.Settings; import org.opensearch.core.action.ActionResponse; import org.opensearch.core.common.io.stream.StreamInput; @@ -43,6 +44,7 @@ * * @opensearch.api */ +@PublicApi(since = "1.0.0") public class ActionType { private final String name; diff --git a/server/src/main/java/org/opensearch/action/support/TransportAction.java b/server/src/main/java/org/opensearch/action/support/TransportAction.java index 72aae210d61ae..f71347f6f1d07 100644 --- a/server/src/main/java/org/opensearch/action/support/TransportAction.java +++ b/server/src/main/java/org/opensearch/action/support/TransportAction.java @@ -36,6 +36,7 @@ import org.apache.logging.log4j.Logger; import org.opensearch.action.ActionRequest; import org.opensearch.action.ActionRequestValidationException; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.lease.Releasable; import org.opensearch.common.lease.Releasables; import org.opensearch.common.util.concurrent.ThreadContext; @@ -52,8 +53,9 @@ /** * Base class for a transport action * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public abstract class TransportAction { public final String actionName; diff --git a/server/src/main/java/org/opensearch/client/node/NodeClient.java b/server/src/main/java/org/opensearch/client/node/NodeClient.java index 6e1bb6ce79349..5780e4c1e648a 100644 --- a/server/src/main/java/org/opensearch/client/node/NodeClient.java +++ b/server/src/main/java/org/opensearch/client/node/NodeClient.java @@ -39,6 +39,7 @@ import org.opensearch.client.Client; import org.opensearch.client.support.AbstractClient; import org.opensearch.cluster.node.DiscoveryNode; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.settings.Settings; import org.opensearch.core.action.ActionListener; import org.opensearch.core.action.ActionResponse; @@ -53,8 +54,9 @@ /** * Client that executes actions on the local node. * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public class NodeClient extends AbstractClient { private DynamicActionRegistry actionRegistry; diff --git a/server/src/main/java/org/opensearch/common/inject/Binder.java b/server/src/main/java/org/opensearch/common/inject/Binder.java index a733a19608ac1..a9d16becfb5ab 100644 --- a/server/src/main/java/org/opensearch/common/inject/Binder.java +++ b/server/src/main/java/org/opensearch/common/inject/Binder.java @@ -29,6 +29,7 @@ package org.opensearch.common.inject; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.inject.binder.AnnotatedBindingBuilder; import org.opensearch.common.inject.binder.AnnotatedConstantBindingBuilder; import org.opensearch.common.inject.binder.LinkedBindingBuilder; @@ -198,8 +199,9 @@ * @author jessewilson@google.com (Jesse Wilson) * @author kevinb@google.com (Kevin Bourrillion) * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public interface Binder { /** diff --git a/server/src/main/java/org/opensearch/common/inject/Binding.java b/server/src/main/java/org/opensearch/common/inject/Binding.java index 53d02e37502af..a42237697a1d2 100644 --- a/server/src/main/java/org/opensearch/common/inject/Binding.java +++ b/server/src/main/java/org/opensearch/common/inject/Binding.java @@ -29,6 +29,7 @@ package org.opensearch.common.inject; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.inject.spi.BindingScopingVisitor; import org.opensearch.common.inject.spi.BindingTargetVisitor; import org.opensearch.common.inject.spi.Element; @@ -69,8 +70,9 @@ * @author crazybob@google.com (Bob Lee) * @author jessewilson@google.com (Jesse Wilson) * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public interface Binding extends Element { /** diff --git a/server/src/main/java/org/opensearch/common/inject/ConfigurationException.java b/server/src/main/java/org/opensearch/common/inject/ConfigurationException.java index 4379a93482560..e3a32754a1bdb 100644 --- a/server/src/main/java/org/opensearch/common/inject/ConfigurationException.java +++ b/server/src/main/java/org/opensearch/common/inject/ConfigurationException.java @@ -29,6 +29,7 @@ package org.opensearch.common.inject; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.inject.internal.Errors; import org.opensearch.common.inject.spi.Message; @@ -46,8 +47,9 @@ * @author jessewilson@google.com (Jesse Wilson) * @since 2.0 * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public final class ConfigurationException extends RuntimeException { private final Set messages; private Object partialValue = null; diff --git a/server/src/main/java/org/opensearch/common/inject/Injector.java b/server/src/main/java/org/opensearch/common/inject/Injector.java index ff212c6313371..772578dd6bb2c 100644 --- a/server/src/main/java/org/opensearch/common/inject/Injector.java +++ b/server/src/main/java/org/opensearch/common/inject/Injector.java @@ -29,6 +29,8 @@ package org.opensearch.common.inject; +import org.opensearch.common.annotation.PublicApi; + import java.util.List; /** @@ -54,8 +56,9 @@ * @author crazybob@google.com (Bob Lee) * @author jessewilson@google.com (Jesse Wilson) * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public interface Injector { /** diff --git a/server/src/main/java/org/opensearch/common/inject/Key.java b/server/src/main/java/org/opensearch/common/inject/Key.java index cd305353a555d..32f168d18e523 100644 --- a/server/src/main/java/org/opensearch/common/inject/Key.java +++ b/server/src/main/java/org/opensearch/common/inject/Key.java @@ -29,6 +29,7 @@ package org.opensearch.common.inject; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.inject.internal.Annotations; import org.opensearch.common.inject.internal.MoreTypes; import org.opensearch.common.inject.internal.ToStringBuilder; @@ -59,8 +60,9 @@ * * @author crazybob@google.com (Bob Lee) * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public class Key { private final AnnotationStrategy annotationStrategy; diff --git a/server/src/main/java/org/opensearch/common/inject/MembersInjector.java b/server/src/main/java/org/opensearch/common/inject/MembersInjector.java index 891762375d5a2..872ae883e246b 100644 --- a/server/src/main/java/org/opensearch/common/inject/MembersInjector.java +++ b/server/src/main/java/org/opensearch/common/inject/MembersInjector.java @@ -29,6 +29,8 @@ package org.opensearch.common.inject; +import org.opensearch.common.annotation.PublicApi; + /** * Injects dependencies into the fields and methods on instances of type {@code T}. Ignores the * presence or absence of an injectable constructor. @@ -38,8 +40,9 @@ * @author jessewilson@google.com (Jesse Wilson) * @since 2.0 * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public interface MembersInjector { /** diff --git a/server/src/main/java/org/opensearch/common/inject/Module.java b/server/src/main/java/org/opensearch/common/inject/Module.java index b1fc031192ea0..e66044ff26c40 100644 --- a/server/src/main/java/org/opensearch/common/inject/Module.java +++ b/server/src/main/java/org/opensearch/common/inject/Module.java @@ -29,6 +29,8 @@ package org.opensearch.common.inject; +import org.opensearch.common.annotation.PublicApi; + /** * A module contributes configuration information, typically interface * bindings, which will be used to create an {@link Injector}. A Guice-based @@ -43,8 +45,9 @@ * Use scope and binding annotations on these methods to configure the * bindings. * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public interface Module { /** diff --git a/server/src/main/java/org/opensearch/common/inject/PrivateBinder.java b/server/src/main/java/org/opensearch/common/inject/PrivateBinder.java index 87635880e29d8..2b6b2e0aad146 100644 --- a/server/src/main/java/org/opensearch/common/inject/PrivateBinder.java +++ b/server/src/main/java/org/opensearch/common/inject/PrivateBinder.java @@ -29,6 +29,7 @@ package org.opensearch.common.inject; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.inject.binder.AnnotatedElementBuilder; /** @@ -38,8 +39,9 @@ * @author jessewilson@google.com (Jesse Wilson) * @since 2.0 * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public interface PrivateBinder extends Binder { /** diff --git a/server/src/main/java/org/opensearch/common/inject/Provider.java b/server/src/main/java/org/opensearch/common/inject/Provider.java index 97f9e9ae503cd..988143b328828 100644 --- a/server/src/main/java/org/opensearch/common/inject/Provider.java +++ b/server/src/main/java/org/opensearch/common/inject/Provider.java @@ -29,6 +29,8 @@ package org.opensearch.common.inject; +import org.opensearch.common.annotation.PublicApi; + /** * An object capable of providing instances of type {@code T}. Providers are used in numerous ways * by Guice: @@ -50,8 +52,9 @@ * @param the type of object this provides * @author crazybob@google.com (Bob Lee) * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public interface Provider { /** diff --git a/server/src/main/java/org/opensearch/common/inject/Scope.java b/server/src/main/java/org/opensearch/common/inject/Scope.java index a21495f522d5e..6fb9f560981ef 100644 --- a/server/src/main/java/org/opensearch/common/inject/Scope.java +++ b/server/src/main/java/org/opensearch/common/inject/Scope.java @@ -29,6 +29,8 @@ package org.opensearch.common.inject; +import org.opensearch.common.annotation.PublicApi; + /** * A scope is a level of visibility that instances provided by Guice may have. * By default, an instance created by the {@link Injector} has no scope, @@ -42,8 +44,9 @@ * * @author crazybob@google.com (Bob Lee) * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public interface Scope { /** diff --git a/server/src/main/java/org/opensearch/common/inject/Stage.java b/server/src/main/java/org/opensearch/common/inject/Stage.java index d5996bd1363e9..fbb6e389ef43f 100644 --- a/server/src/main/java/org/opensearch/common/inject/Stage.java +++ b/server/src/main/java/org/opensearch/common/inject/Stage.java @@ -29,13 +29,16 @@ package org.opensearch.common.inject; +import org.opensearch.common.annotation.PublicApi; + /** * The stage we're running in. * * @author crazybob@google.com (Bob Lee) * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public enum Stage { /** diff --git a/server/src/main/java/org/opensearch/common/inject/TypeLiteral.java b/server/src/main/java/org/opensearch/common/inject/TypeLiteral.java index f0cca2990b407..8ac04e5d0ac1d 100644 --- a/server/src/main/java/org/opensearch/common/inject/TypeLiteral.java +++ b/server/src/main/java/org/opensearch/common/inject/TypeLiteral.java @@ -29,6 +29,7 @@ package org.opensearch.common.inject; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.inject.internal.MoreTypes; import org.opensearch.common.inject.util.Types; @@ -77,8 +78,9 @@ * @author crazybob@google.com (Bob Lee) * @author jessewilson@google.com (Jesse Wilson) * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public class TypeLiteral { final Class rawType; diff --git a/server/src/main/java/org/opensearch/common/inject/binder/AnnotatedBindingBuilder.java b/server/src/main/java/org/opensearch/common/inject/binder/AnnotatedBindingBuilder.java index bcd593a8cbf7b..5c3c6eac9bd3a 100644 --- a/server/src/main/java/org/opensearch/common/inject/binder/AnnotatedBindingBuilder.java +++ b/server/src/main/java/org/opensearch/common/inject/binder/AnnotatedBindingBuilder.java @@ -29,6 +29,8 @@ package org.opensearch.common.inject.binder; +import org.opensearch.common.annotation.PublicApi; + import java.lang.annotation.Annotation; /** @@ -36,8 +38,9 @@ * * @author crazybob@google.com (Bob Lee) * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public interface AnnotatedBindingBuilder extends LinkedBindingBuilder { /** diff --git a/server/src/main/java/org/opensearch/common/inject/binder/AnnotatedConstantBindingBuilder.java b/server/src/main/java/org/opensearch/common/inject/binder/AnnotatedConstantBindingBuilder.java index 42c208a2b37ea..71ea1ba0a5207 100644 --- a/server/src/main/java/org/opensearch/common/inject/binder/AnnotatedConstantBindingBuilder.java +++ b/server/src/main/java/org/opensearch/common/inject/binder/AnnotatedConstantBindingBuilder.java @@ -29,6 +29,8 @@ package org.opensearch.common.inject.binder; +import org.opensearch.common.annotation.PublicApi; + import java.lang.annotation.Annotation; /** @@ -36,8 +38,9 @@ * * @author crazybob@google.com (Bob Lee) * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public interface AnnotatedConstantBindingBuilder { /** diff --git a/server/src/main/java/org/opensearch/common/inject/binder/AnnotatedElementBuilder.java b/server/src/main/java/org/opensearch/common/inject/binder/AnnotatedElementBuilder.java index f2d0916790b6b..54fcb915d83c9 100644 --- a/server/src/main/java/org/opensearch/common/inject/binder/AnnotatedElementBuilder.java +++ b/server/src/main/java/org/opensearch/common/inject/binder/AnnotatedElementBuilder.java @@ -29,6 +29,8 @@ package org.opensearch.common.inject.binder; +import org.opensearch.common.annotation.PublicApi; + import java.lang.annotation.Annotation; /** @@ -37,8 +39,9 @@ * @author jessewilson@google.com (Jesse Wilson) * @since 2.0 * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public interface AnnotatedElementBuilder { /** diff --git a/server/src/main/java/org/opensearch/common/inject/binder/ConstantBindingBuilder.java b/server/src/main/java/org/opensearch/common/inject/binder/ConstantBindingBuilder.java index 595c477d3e28b..feaee3ed59f46 100644 --- a/server/src/main/java/org/opensearch/common/inject/binder/ConstantBindingBuilder.java +++ b/server/src/main/java/org/opensearch/common/inject/binder/ConstantBindingBuilder.java @@ -29,11 +29,14 @@ package org.opensearch.common.inject.binder; +import org.opensearch.common.annotation.PublicApi; + /** * Binds to a constant value. * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public interface ConstantBindingBuilder { /** diff --git a/server/src/main/java/org/opensearch/common/inject/binder/LinkedBindingBuilder.java b/server/src/main/java/org/opensearch/common/inject/binder/LinkedBindingBuilder.java index 2368fef16471c..e8c4b197253b5 100644 --- a/server/src/main/java/org/opensearch/common/inject/binder/LinkedBindingBuilder.java +++ b/server/src/main/java/org/opensearch/common/inject/binder/LinkedBindingBuilder.java @@ -29,6 +29,7 @@ package org.opensearch.common.inject.binder; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.inject.Key; import org.opensearch.common.inject.Provider; import org.opensearch.common.inject.TypeLiteral; @@ -38,8 +39,9 @@ * * @author crazybob@google.com (Bob Lee) * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public interface LinkedBindingBuilder extends ScopedBindingBuilder { /** diff --git a/server/src/main/java/org/opensearch/common/inject/binder/ScopedBindingBuilder.java b/server/src/main/java/org/opensearch/common/inject/binder/ScopedBindingBuilder.java index 73dd4414f17a2..c360b9571bc4a 100644 --- a/server/src/main/java/org/opensearch/common/inject/binder/ScopedBindingBuilder.java +++ b/server/src/main/java/org/opensearch/common/inject/binder/ScopedBindingBuilder.java @@ -29,6 +29,7 @@ package org.opensearch.common.inject.binder; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.inject.Scope; import java.lang.annotation.Annotation; @@ -38,8 +39,9 @@ * * @author crazybob@google.com (Bob Lee) * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public interface ScopedBindingBuilder { /** diff --git a/server/src/main/java/org/opensearch/common/inject/matcher/Matcher.java b/server/src/main/java/org/opensearch/common/inject/matcher/Matcher.java index 21bb63cfef097..4e254f8641350 100644 --- a/server/src/main/java/org/opensearch/common/inject/matcher/Matcher.java +++ b/server/src/main/java/org/opensearch/common/inject/matcher/Matcher.java @@ -29,13 +29,16 @@ package org.opensearch.common.inject.matcher; +import org.opensearch.common.annotation.PublicApi; + /** * Returns {@code true} or {@code false} for a given input. * * @author crazybob@google.com (Bob Lee) * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public interface Matcher { /** diff --git a/server/src/main/java/org/opensearch/common/inject/spi/BindingScopingVisitor.java b/server/src/main/java/org/opensearch/common/inject/spi/BindingScopingVisitor.java index d7c7d9d65051d..b4fbdf2fdb72b 100644 --- a/server/src/main/java/org/opensearch/common/inject/spi/BindingScopingVisitor.java +++ b/server/src/main/java/org/opensearch/common/inject/spi/BindingScopingVisitor.java @@ -29,6 +29,7 @@ package org.opensearch.common.inject.spi; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.inject.Scope; import java.lang.annotation.Annotation; @@ -40,8 +41,9 @@ * {@code return null} if no return type is needed. * @since 2.0 * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public interface BindingScopingVisitor { /** diff --git a/server/src/main/java/org/opensearch/common/inject/spi/BindingTargetVisitor.java b/server/src/main/java/org/opensearch/common/inject/spi/BindingTargetVisitor.java index 91df812b58ac4..9543e731308bd 100644 --- a/server/src/main/java/org/opensearch/common/inject/spi/BindingTargetVisitor.java +++ b/server/src/main/java/org/opensearch/common/inject/spi/BindingTargetVisitor.java @@ -29,6 +29,8 @@ package org.opensearch.common.inject.spi; +import org.opensearch.common.annotation.PublicApi; + /** * Visits each of the strategies used to find an instance to satisfy an injection. * @@ -36,8 +38,9 @@ * {@code return null} if no return type is needed. * @since 2.0 * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public interface BindingTargetVisitor { /** diff --git a/server/src/main/java/org/opensearch/common/inject/spi/ConstructorBinding.java b/server/src/main/java/org/opensearch/common/inject/spi/ConstructorBinding.java index 997bf78234fd1..8eec6cefe53c7 100644 --- a/server/src/main/java/org/opensearch/common/inject/spi/ConstructorBinding.java +++ b/server/src/main/java/org/opensearch/common/inject/spi/ConstructorBinding.java @@ -29,6 +29,7 @@ package org.opensearch.common.inject.spi; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.inject.Binding; import java.util.Set; @@ -40,8 +41,9 @@ * @author jessewilson@google.com (Jesse Wilson) * @since 2.0 * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public interface ConstructorBinding extends Binding, HasDependencies { /** diff --git a/server/src/main/java/org/opensearch/common/inject/spi/ConvertedConstantBinding.java b/server/src/main/java/org/opensearch/common/inject/spi/ConvertedConstantBinding.java index e8d6b346f8596..a07da68a88931 100644 --- a/server/src/main/java/org/opensearch/common/inject/spi/ConvertedConstantBinding.java +++ b/server/src/main/java/org/opensearch/common/inject/spi/ConvertedConstantBinding.java @@ -29,6 +29,7 @@ package org.opensearch.common.inject.spi; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.inject.Binding; import org.opensearch.common.inject.Key; @@ -41,8 +42,9 @@ * @author jessewilson@google.com (Jesse Wilson) * @since 2.0 * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public interface ConvertedConstantBinding extends Binding, HasDependencies { /** diff --git a/server/src/main/java/org/opensearch/common/inject/spi/Dependency.java b/server/src/main/java/org/opensearch/common/inject/spi/Dependency.java index be1336ad0f297..e541ba0b73bf5 100644 --- a/server/src/main/java/org/opensearch/common/inject/spi/Dependency.java +++ b/server/src/main/java/org/opensearch/common/inject/spi/Dependency.java @@ -29,6 +29,7 @@ package org.opensearch.common.inject.spi; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.inject.Key; import java.util.HashSet; @@ -47,8 +48,9 @@ * @author jessewilson@google.com (Jesse Wilson) * @since 2.0 * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public final class Dependency { private final InjectionPoint injectionPoint; private final Key key; diff --git a/server/src/main/java/org/opensearch/common/inject/spi/Element.java b/server/src/main/java/org/opensearch/common/inject/spi/Element.java index 660aca1bd45ab..58a696fb7ffa9 100644 --- a/server/src/main/java/org/opensearch/common/inject/spi/Element.java +++ b/server/src/main/java/org/opensearch/common/inject/spi/Element.java @@ -29,6 +29,7 @@ package org.opensearch.common.inject.spi; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.inject.Binder; /** @@ -43,8 +44,9 @@ * @author crazybob@google.com (Bob Lee) * @since 2.0 * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public interface Element { /** diff --git a/server/src/main/java/org/opensearch/common/inject/spi/ElementVisitor.java b/server/src/main/java/org/opensearch/common/inject/spi/ElementVisitor.java index d415560fc03c8..b88f11b9378aa 100644 --- a/server/src/main/java/org/opensearch/common/inject/spi/ElementVisitor.java +++ b/server/src/main/java/org/opensearch/common/inject/spi/ElementVisitor.java @@ -29,6 +29,7 @@ package org.opensearch.common.inject.spi; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.inject.Binding; /** @@ -38,8 +39,9 @@ * {@code return null} if no return type is needed. * @since 2.0 * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public interface ElementVisitor { /** diff --git a/server/src/main/java/org/opensearch/common/inject/spi/ExposedBinding.java b/server/src/main/java/org/opensearch/common/inject/spi/ExposedBinding.java index d2563bc2728cd..6c1679432abe5 100644 --- a/server/src/main/java/org/opensearch/common/inject/spi/ExposedBinding.java +++ b/server/src/main/java/org/opensearch/common/inject/spi/ExposedBinding.java @@ -29,6 +29,7 @@ package org.opensearch.common.inject.spi; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.inject.Binder; import org.opensearch.common.inject.Binding; @@ -38,8 +39,9 @@ * @author jessewilson@google.com (Jesse Wilson) * @since 2.0 * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public interface ExposedBinding extends Binding, HasDependencies { /** diff --git a/server/src/main/java/org/opensearch/common/inject/spi/InjectionListener.java b/server/src/main/java/org/opensearch/common/inject/spi/InjectionListener.java index 7a760d2b84e9f..878e919cda4cc 100644 --- a/server/src/main/java/org/opensearch/common/inject/spi/InjectionListener.java +++ b/server/src/main/java/org/opensearch/common/inject/spi/InjectionListener.java @@ -29,6 +29,8 @@ package org.opensearch.common.inject.spi; +import org.opensearch.common.annotation.PublicApi; + /** * Listens for injections into instances of type {@code I}. Useful for performing further * injections, post-injection initialization, and more. @@ -37,8 +39,9 @@ * @author jessewilson@google.com (Jesse Wilson) * @since 2.0 * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public interface InjectionListener { /** diff --git a/server/src/main/java/org/opensearch/common/inject/spi/InjectionPoint.java b/server/src/main/java/org/opensearch/common/inject/spi/InjectionPoint.java index c88b2281107ed..542cbd780a8b6 100644 --- a/server/src/main/java/org/opensearch/common/inject/spi/InjectionPoint.java +++ b/server/src/main/java/org/opensearch/common/inject/spi/InjectionPoint.java @@ -29,6 +29,7 @@ package org.opensearch.common.inject.spi; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.inject.ConfigurationException; import org.opensearch.common.inject.Inject; import org.opensearch.common.inject.Key; @@ -66,8 +67,9 @@ * @author crazybob@google.com (Bob Lee) * @since 2.0 * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public final class InjectionPoint { private final boolean optional; diff --git a/server/src/main/java/org/opensearch/common/inject/spi/InjectionRequest.java b/server/src/main/java/org/opensearch/common/inject/spi/InjectionRequest.java index 6ce5febbb6711..a5faca6264424 100644 --- a/server/src/main/java/org/opensearch/common/inject/spi/InjectionRequest.java +++ b/server/src/main/java/org/opensearch/common/inject/spi/InjectionRequest.java @@ -29,6 +29,7 @@ package org.opensearch.common.inject.spi; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.inject.Binder; import org.opensearch.common.inject.ConfigurationException; import org.opensearch.common.inject.TypeLiteral; @@ -46,8 +47,9 @@ * @author mikeward@google.com (Mike Ward) * @since 2.0 * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public final class InjectionRequest implements Element { private final Object source; diff --git a/server/src/main/java/org/opensearch/common/inject/spi/InstanceBinding.java b/server/src/main/java/org/opensearch/common/inject/spi/InstanceBinding.java index fd7c1303ed6fc..f73b284ae2e8c 100644 --- a/server/src/main/java/org/opensearch/common/inject/spi/InstanceBinding.java +++ b/server/src/main/java/org/opensearch/common/inject/spi/InstanceBinding.java @@ -29,6 +29,7 @@ package org.opensearch.common.inject.spi; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.inject.Binding; import java.util.Set; @@ -39,8 +40,9 @@ * @author jessewilson@google.com (Jesse Wilson) * @since 2.0 * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public interface InstanceBinding extends Binding, HasDependencies { /** diff --git a/server/src/main/java/org/opensearch/common/inject/spi/LinkedKeyBinding.java b/server/src/main/java/org/opensearch/common/inject/spi/LinkedKeyBinding.java index 10b270e499603..01da905f8da47 100644 --- a/server/src/main/java/org/opensearch/common/inject/spi/LinkedKeyBinding.java +++ b/server/src/main/java/org/opensearch/common/inject/spi/LinkedKeyBinding.java @@ -29,6 +29,7 @@ package org.opensearch.common.inject.spi; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.inject.Binding; import org.opensearch.common.inject.Key; @@ -38,8 +39,9 @@ * @author jessewilson@google.com (Jesse Wilson) * @since 2.0 * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public interface LinkedKeyBinding extends Binding { /** diff --git a/server/src/main/java/org/opensearch/common/inject/spi/MembersInjectorLookup.java b/server/src/main/java/org/opensearch/common/inject/spi/MembersInjectorLookup.java index 1f652708de875..b8a07146812c1 100644 --- a/server/src/main/java/org/opensearch/common/inject/spi/MembersInjectorLookup.java +++ b/server/src/main/java/org/opensearch/common/inject/spi/MembersInjectorLookup.java @@ -29,6 +29,7 @@ package org.opensearch.common.inject.spi; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.inject.Binder; import org.opensearch.common.inject.MembersInjector; import org.opensearch.common.inject.TypeLiteral; @@ -45,8 +46,9 @@ * @author crazybob@google.com (Bob Lee) * @since 2.0 * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public final class MembersInjectorLookup implements Element { private final Object source; diff --git a/server/src/main/java/org/opensearch/common/inject/spi/Message.java b/server/src/main/java/org/opensearch/common/inject/spi/Message.java index 78829e82c150e..13184a7d82f0c 100644 --- a/server/src/main/java/org/opensearch/common/inject/spi/Message.java +++ b/server/src/main/java/org/opensearch/common/inject/spi/Message.java @@ -29,6 +29,7 @@ package org.opensearch.common.inject.spi; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.inject.Binder; import org.opensearch.common.inject.internal.Errors; import org.opensearch.common.inject.internal.SourceProvider; @@ -50,8 +51,9 @@ * * @author crazybob@google.com (Bob Lee) * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public final class Message implements Element { private final String message; private final Throwable cause; diff --git a/server/src/main/java/org/opensearch/common/inject/spi/PrivateElements.java b/server/src/main/java/org/opensearch/common/inject/spi/PrivateElements.java index e4d86a356cd53..6330cbe33de58 100644 --- a/server/src/main/java/org/opensearch/common/inject/spi/PrivateElements.java +++ b/server/src/main/java/org/opensearch/common/inject/spi/PrivateElements.java @@ -29,6 +29,7 @@ package org.opensearch.common.inject.spi; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.inject.Injector; import org.opensearch.common.inject.Key; @@ -42,8 +43,9 @@ * @author jessewilson@google.com (Jesse Wilson) * @since 2.0 * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public interface PrivateElements extends Element { /** diff --git a/server/src/main/java/org/opensearch/common/inject/spi/ProviderBinding.java b/server/src/main/java/org/opensearch/common/inject/spi/ProviderBinding.java index 0a63fefc0a9e9..dd55e9805843f 100644 --- a/server/src/main/java/org/opensearch/common/inject/spi/ProviderBinding.java +++ b/server/src/main/java/org/opensearch/common/inject/spi/ProviderBinding.java @@ -29,6 +29,7 @@ package org.opensearch.common.inject.spi; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.inject.Binding; import org.opensearch.common.inject.Key; import org.opensearch.common.inject.Provider; @@ -40,8 +41,9 @@ * @author jessewilson@google.com (Jesse Wilson) * @since 2.0 * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public interface ProviderBinding> extends Binding { /** diff --git a/server/src/main/java/org/opensearch/common/inject/spi/ProviderInstanceBinding.java b/server/src/main/java/org/opensearch/common/inject/spi/ProviderInstanceBinding.java index 654f40e627e4b..25bac3b5df34c 100644 --- a/server/src/main/java/org/opensearch/common/inject/spi/ProviderInstanceBinding.java +++ b/server/src/main/java/org/opensearch/common/inject/spi/ProviderInstanceBinding.java @@ -29,6 +29,7 @@ package org.opensearch.common.inject.spi; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.inject.Binding; import org.opensearch.common.inject.Provider; @@ -41,8 +42,9 @@ * @author jessewilson@google.com (Jesse Wilson) * @since 2.0 * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public interface ProviderInstanceBinding extends Binding, HasDependencies { /** diff --git a/server/src/main/java/org/opensearch/common/inject/spi/ProviderKeyBinding.java b/server/src/main/java/org/opensearch/common/inject/spi/ProviderKeyBinding.java index 6f1ae8f2b9a03..f68e1662ad124 100644 --- a/server/src/main/java/org/opensearch/common/inject/spi/ProviderKeyBinding.java +++ b/server/src/main/java/org/opensearch/common/inject/spi/ProviderKeyBinding.java @@ -29,6 +29,7 @@ package org.opensearch.common.inject.spi; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.inject.Binding; import org.opensearch.common.inject.Key; import org.opensearch.common.inject.Provider; @@ -40,8 +41,9 @@ * @author jessewilson@google.com (Jesse Wilson) * @since 2.0 * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public interface ProviderKeyBinding extends Binding { /** diff --git a/server/src/main/java/org/opensearch/common/inject/spi/ProviderLookup.java b/server/src/main/java/org/opensearch/common/inject/spi/ProviderLookup.java index 16060ddd3e222..6afe7346a1431 100644 --- a/server/src/main/java/org/opensearch/common/inject/spi/ProviderLookup.java +++ b/server/src/main/java/org/opensearch/common/inject/spi/ProviderLookup.java @@ -29,6 +29,7 @@ package org.opensearch.common.inject.spi; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.inject.Binder; import org.opensearch.common.inject.Key; import org.opensearch.common.inject.Provider; @@ -45,8 +46,9 @@ * @author jessewilson@google.com (Jesse Wilson) * @since 2.0 * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public final class ProviderLookup implements Element { /** diff --git a/server/src/main/java/org/opensearch/common/inject/spi/ScopeBinding.java b/server/src/main/java/org/opensearch/common/inject/spi/ScopeBinding.java index 7a619456e06e3..ca03f4291a062 100644 --- a/server/src/main/java/org/opensearch/common/inject/spi/ScopeBinding.java +++ b/server/src/main/java/org/opensearch/common/inject/spi/ScopeBinding.java @@ -29,6 +29,7 @@ package org.opensearch.common.inject.spi; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.inject.Binder; import org.opensearch.common.inject.Scope; @@ -46,8 +47,9 @@ * @author jessewilson@google.com (Jesse Wilson) * @since 2.0 * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public final class ScopeBinding implements Element { private final Object source; private final Class annotationType; diff --git a/server/src/main/java/org/opensearch/common/inject/spi/StaticInjectionRequest.java b/server/src/main/java/org/opensearch/common/inject/spi/StaticInjectionRequest.java index 494e35e6c4490..c426639d85cab 100644 --- a/server/src/main/java/org/opensearch/common/inject/spi/StaticInjectionRequest.java +++ b/server/src/main/java/org/opensearch/common/inject/spi/StaticInjectionRequest.java @@ -29,6 +29,7 @@ package org.opensearch.common.inject.spi; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.inject.Binder; import org.opensearch.common.inject.ConfigurationException; @@ -45,8 +46,9 @@ * @author jessewilson@google.com (Jesse Wilson) * @since 2.0 * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public final class StaticInjectionRequest implements Element { private final Object source; private final Class type; diff --git a/server/src/main/java/org/opensearch/common/inject/spi/TypeConverter.java b/server/src/main/java/org/opensearch/common/inject/spi/TypeConverter.java index 93a0f607ddc27..2386c1e528db6 100644 --- a/server/src/main/java/org/opensearch/common/inject/spi/TypeConverter.java +++ b/server/src/main/java/org/opensearch/common/inject/spi/TypeConverter.java @@ -29,6 +29,7 @@ package org.opensearch.common.inject.spi; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.inject.TypeLiteral; /** @@ -37,8 +38,9 @@ * @author crazybob@google.com (Bob Lee) * @since 2.0 * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public interface TypeConverter { /** diff --git a/server/src/main/java/org/opensearch/common/inject/spi/TypeConverterBinding.java b/server/src/main/java/org/opensearch/common/inject/spi/TypeConverterBinding.java index 00b8c7c013b5a..59311de0fb3f5 100644 --- a/server/src/main/java/org/opensearch/common/inject/spi/TypeConverterBinding.java +++ b/server/src/main/java/org/opensearch/common/inject/spi/TypeConverterBinding.java @@ -29,6 +29,7 @@ package org.opensearch.common.inject.spi; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.inject.Binder; import org.opensearch.common.inject.TypeLiteral; import org.opensearch.common.inject.matcher.Matcher; @@ -45,8 +46,9 @@ * @author jessewilson@google.com (Jesse Wilson) * @since 2.0 * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public final class TypeConverterBinding implements Element { private final Object source; private final Matcher> typeMatcher; diff --git a/server/src/main/java/org/opensearch/common/inject/spi/TypeEncounter.java b/server/src/main/java/org/opensearch/common/inject/spi/TypeEncounter.java index e06751668c0f1..61756a5bcad95 100644 --- a/server/src/main/java/org/opensearch/common/inject/spi/TypeEncounter.java +++ b/server/src/main/java/org/opensearch/common/inject/spi/TypeEncounter.java @@ -29,6 +29,7 @@ package org.opensearch.common.inject.spi; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.inject.Key; import org.opensearch.common.inject.MembersInjector; import org.opensearch.common.inject.Provider; @@ -43,8 +44,9 @@ * @param the injectable type encountered * @since 2.0 * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") @SuppressWarnings("overloads") public interface TypeEncounter { diff --git a/server/src/main/java/org/opensearch/common/inject/spi/TypeListener.java b/server/src/main/java/org/opensearch/common/inject/spi/TypeListener.java index fd7004aa80df0..3157fa15f471b 100644 --- a/server/src/main/java/org/opensearch/common/inject/spi/TypeListener.java +++ b/server/src/main/java/org/opensearch/common/inject/spi/TypeListener.java @@ -29,6 +29,7 @@ package org.opensearch.common.inject.spi; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.inject.TypeLiteral; /** @@ -43,8 +44,9 @@ * * @since 2.0 * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public interface TypeListener { /** diff --git a/server/src/main/java/org/opensearch/common/inject/spi/TypeListenerBinding.java b/server/src/main/java/org/opensearch/common/inject/spi/TypeListenerBinding.java index 505028f09232d..4ddcf3fc11bc1 100644 --- a/server/src/main/java/org/opensearch/common/inject/spi/TypeListenerBinding.java +++ b/server/src/main/java/org/opensearch/common/inject/spi/TypeListenerBinding.java @@ -29,6 +29,7 @@ package org.opensearch.common.inject.spi; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.inject.Binder; import org.opensearch.common.inject.TypeLiteral; import org.opensearch.common.inject.matcher.Matcher; @@ -42,8 +43,9 @@ * @author jessewilson@google.com (Jesse Wilson) * @since 2.0 * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public final class TypeListenerBinding implements Element { private final Object source; diff --git a/server/src/main/java/org/opensearch/common/inject/spi/UntargettedBinding.java b/server/src/main/java/org/opensearch/common/inject/spi/UntargettedBinding.java index 37e40d45cb5a9..56890efdfcd8d 100644 --- a/server/src/main/java/org/opensearch/common/inject/spi/UntargettedBinding.java +++ b/server/src/main/java/org/opensearch/common/inject/spi/UntargettedBinding.java @@ -29,6 +29,7 @@ package org.opensearch.common.inject.spi; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.inject.Binding; /** @@ -38,6 +39,7 @@ * @author jessewilson@google.com (Jesse Wilson) * @since 2.0 * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public interface UntargettedBinding extends Binding {} diff --git a/server/src/main/java/org/opensearch/extensions/rest/RestSendToExtensionAction.java b/server/src/main/java/org/opensearch/extensions/rest/RestSendToExtensionAction.java index 41783b89ccc69..f4503ce55e6bc 100644 --- a/server/src/main/java/org/opensearch/extensions/rest/RestSendToExtensionAction.java +++ b/server/src/main/java/org/opensearch/extensions/rest/RestSendToExtensionAction.java @@ -12,6 +12,7 @@ import org.apache.logging.log4j.Logger; import org.opensearch.action.ActionModule.DynamicActionRegistry; import org.opensearch.client.node.NodeClient; +import org.opensearch.common.annotation.ExperimentalApi; import org.opensearch.core.common.bytes.BytesReference; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.rest.RestStatus; @@ -52,7 +53,10 @@ /** * An action that forwards REST requests to an extension + * + * @opensearch.experimental */ +@ExperimentalApi public class RestSendToExtensionAction extends BaseRestHandler { private static final String SEND_TO_EXTENSION_ACTION = "send_to_extension_action"; diff --git a/server/src/main/java/org/opensearch/rest/NamedRoute.java b/server/src/main/java/org/opensearch/rest/NamedRoute.java index 109f688a4924e..c2b3ea5fdeaaf 100644 --- a/server/src/main/java/org/opensearch/rest/NamedRoute.java +++ b/server/src/main/java/org/opensearch/rest/NamedRoute.java @@ -9,6 +9,7 @@ package org.opensearch.rest; import org.opensearch.OpenSearchException; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.transport.TransportService; import java.util.HashSet; @@ -20,8 +21,9 @@ /** * A named Route * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "2.7.0") public class NamedRoute extends RestHandler.Route { private static final String VALID_ACTION_NAME_PATTERN = "^[a-zA-Z0-9:/*_]*$"; diff --git a/server/src/main/java/org/opensearch/rest/RestHandler.java b/server/src/main/java/org/opensearch/rest/RestHandler.java index 387e8b25a8274..877afdd951088 100644 --- a/server/src/main/java/org/opensearch/rest/RestHandler.java +++ b/server/src/main/java/org/opensearch/rest/RestHandler.java @@ -47,6 +47,7 @@ * * @opensearch.api */ +@PublicApi(since = "1.0.0") @FunctionalInterface public interface RestHandler { @@ -233,7 +234,10 @@ public boolean equals(Object o) { /** * Represents an API that has been deprecated and is slated for removal. + * + * @opensearch.api */ + @PublicApi(since = "1.0.0") class DeprecatedRoute extends Route { private final String deprecationMessage; @@ -251,7 +255,10 @@ public String getDeprecationMessage() { /** * Represents an API that has had its {@code path} or {@code method} changed. Holds both the * new and previous {@code path} and {@code method} combination. + * + * @opensearch.api */ + @PublicApi(since = "1.0.0") class ReplacedRoute extends Route { private final String deprecatedPath; diff --git a/server/src/main/java/org/opensearch/tasks/TaskListener.java b/server/src/main/java/org/opensearch/tasks/TaskListener.java index 97df8eacee584..7ec08984a7f07 100644 --- a/server/src/main/java/org/opensearch/tasks/TaskListener.java +++ b/server/src/main/java/org/opensearch/tasks/TaskListener.java @@ -32,11 +32,14 @@ package org.opensearch.tasks; +import org.opensearch.common.annotation.PublicApi; + /** * Listener for Task success or failure. * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public interface TaskListener { /** * Handle task response. This response may constitute a failure or a success diff --git a/server/src/main/java/org/opensearch/transport/RemoteClusterService.java b/server/src/main/java/org/opensearch/transport/RemoteClusterService.java index 87786fb22f22e..dc729053c2148 100644 --- a/server/src/main/java/org/opensearch/transport/RemoteClusterService.java +++ b/server/src/main/java/org/opensearch/transport/RemoteClusterService.java @@ -40,6 +40,7 @@ import org.opensearch.client.Client; import org.opensearch.cluster.node.DiscoveryNode; import org.opensearch.cluster.node.DiscoveryNodeRole; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.settings.ClusterSettings; import org.opensearch.common.settings.Setting; import org.opensearch.common.settings.Settings; @@ -73,8 +74,9 @@ /** * Basic service for accessing remote clusters via gateway nodes * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public final class RemoteClusterService extends RemoteClusterAware implements Closeable { private final Logger logger = LogManager.getLogger(RemoteClusterService.class); diff --git a/server/src/main/java/org/opensearch/transport/RemoteConnectionInfo.java b/server/src/main/java/org/opensearch/transport/RemoteConnectionInfo.java index 280dd958358fd..b5f356490d3d4 100644 --- a/server/src/main/java/org/opensearch/transport/RemoteConnectionInfo.java +++ b/server/src/main/java/org/opensearch/transport/RemoteConnectionInfo.java @@ -32,6 +32,7 @@ package org.opensearch.transport; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.unit.TimeValue; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; @@ -46,8 +47,9 @@ * This class encapsulates all remote cluster information to be rendered on * {@code _remote/info} requests. * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public final class RemoteConnectionInfo implements ToXContentFragment, Writeable { final ModeInfo modeInfo; @@ -132,8 +134,9 @@ public int hashCode() { /** * Mode information * - * @opensearch.internal + * @opensearch.api */ + @PublicApi(since = "1.0.0") public interface ModeInfo extends ToXContentFragment, Writeable { boolean isConnected(); diff --git a/server/src/main/java/org/opensearch/transport/RemoteConnectionStrategy.java b/server/src/main/java/org/opensearch/transport/RemoteConnectionStrategy.java index f0b37c617725e..f2c159d1380e8 100644 --- a/server/src/main/java/org/opensearch/transport/RemoteConnectionStrategy.java +++ b/server/src/main/java/org/opensearch/transport/RemoteConnectionStrategy.java @@ -38,6 +38,7 @@ import org.apache.lucene.store.AlreadyClosedException; import org.opensearch.action.support.ContextPreservingActionListener; import org.opensearch.cluster.node.DiscoveryNode; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.settings.Setting; import org.opensearch.common.settings.Settings; import org.opensearch.common.unit.TimeValue; @@ -75,7 +76,13 @@ */ public abstract class RemoteConnectionStrategy implements TransportConnectionListener, Closeable { - enum ConnectionStrategy { + /** + * Strategy to connect to remote nodes + * + * @opensearch.api + */ + @PublicApi(since = "1.0.0") + public enum ConnectionStrategy { SNIFF( SniffConnectionStrategy.CHANNELS_PER_CONNECTION, SniffConnectionStrategy::enablementSettings, From 5b4b4aa4c282d06a93de72a5c07a54a1524b04ff Mon Sep 17 00:00:00 2001 From: Peter Nied Date: Fri, 22 Mar 2024 20:19:03 -0500 Subject: [PATCH 36/38] Make GetTermVersionAction internal (#12866) * Make GetTermVersionAction internal GetTermVersion action has been added to check if cluster state needs to be updated as a performance improvement on all cluster state checks. The Authorization systems in the Security Plugin check all actions that do not start with `internal:` for permissions causing users without `cluster:monitor/*` permissions to start getting 403 exceptions. This change signals that this action does not require an authorization check by any security systems since it cannot be called via REST APIs. Signed-off-by: Peter Nied * Move TermVerson action outside of admin/cluster namespace Signed-off-by: Peter Nied --------- Signed-off-by: Peter Nied --- .../support/clustermanager/term}/FetchByTermVersionIT.java | 4 +--- .../src/main/java/org/opensearch/action/ActionModule.java | 4 ++-- .../clustermanager/TransportClusterManagerNodeAction.java | 6 +++--- .../clustermanager}/term/GetTermVersionAction.java | 4 ++-- .../clustermanager}/term/GetTermVersionRequest.java | 2 +- .../clustermanager}/term/GetTermVersionResponse.java | 2 +- .../clustermanager}/term/TransportGetTermVersionAction.java | 2 +- .../state => support/clustermanager}/term/package-info.java | 2 +- .../TransportClusterManagerTermCheckTests.java | 6 +++--- .../clustermanager}/term/ClusterTermVersionIT.java | 2 +- .../clustermanager}/term/ClusterTermVersionTests.java | 2 +- .../org/opensearch/snapshots/SnapshotResiliencyTests.java | 4 ++-- 12 files changed, 19 insertions(+), 21 deletions(-) rename server/src/internalClusterTest/java/org/opensearch/{cluster/state => action/support/clustermanager/term}/FetchByTermVersionIT.java (97%) rename server/src/main/java/org/opensearch/action/{admin/cluster/state => support/clustermanager}/term/GetTermVersionAction.java (83%) rename server/src/main/java/org/opensearch/action/{admin/cluster/state => support/clustermanager}/term/GetTermVersionRequest.java (93%) rename server/src/main/java/org/opensearch/action/{admin/cluster/state => support/clustermanager}/term/GetTermVersionResponse.java (96%) rename server/src/main/java/org/opensearch/action/{admin/cluster/state => support/clustermanager}/term/TransportGetTermVersionAction.java (97%) rename server/src/main/java/org/opensearch/action/{admin/cluster/state => support/clustermanager}/term/package-info.java (80%) rename server/src/test/java/org/opensearch/action/{admin/cluster/state => support/clustermanager}/term/ClusterTermVersionIT.java (98%) rename server/src/test/java/org/opensearch/action/{admin/cluster/state => support/clustermanager}/term/ClusterTermVersionTests.java (93%) diff --git a/server/src/internalClusterTest/java/org/opensearch/cluster/state/FetchByTermVersionIT.java b/server/src/internalClusterTest/java/org/opensearch/action/support/clustermanager/term/FetchByTermVersionIT.java similarity index 97% rename from server/src/internalClusterTest/java/org/opensearch/cluster/state/FetchByTermVersionIT.java rename to server/src/internalClusterTest/java/org/opensearch/action/support/clustermanager/term/FetchByTermVersionIT.java index cef184b3fddf9..72dcc98dcdc12 100644 --- a/server/src/internalClusterTest/java/org/opensearch/cluster/state/FetchByTermVersionIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/action/support/clustermanager/term/FetchByTermVersionIT.java @@ -6,12 +6,10 @@ * compatible open source license. */ -package org.opensearch.cluster.state; +package org.opensearch.action.support.clustermanager.term; import org.opensearch.action.admin.cluster.state.ClusterStateRequest; import org.opensearch.action.admin.cluster.state.ClusterStateResponse; -import org.opensearch.action.admin.cluster.state.term.GetTermVersionAction; -import org.opensearch.action.admin.cluster.state.term.GetTermVersionResponse; import org.opensearch.action.admin.indices.mapping.put.PutMappingRequest; import org.opensearch.cluster.ClusterName; import org.opensearch.cluster.coordination.ClusterStateTermVersion; diff --git a/server/src/main/java/org/opensearch/action/ActionModule.java b/server/src/main/java/org/opensearch/action/ActionModule.java index 716e248b09692..5e2b62614fc47 100644 --- a/server/src/main/java/org/opensearch/action/ActionModule.java +++ b/server/src/main/java/org/opensearch/action/ActionModule.java @@ -107,8 +107,6 @@ import org.opensearch.action.admin.cluster.snapshots.status.TransportSnapshotsStatusAction; import org.opensearch.action.admin.cluster.state.ClusterStateAction; import org.opensearch.action.admin.cluster.state.TransportClusterStateAction; -import org.opensearch.action.admin.cluster.state.term.GetTermVersionAction; -import org.opensearch.action.admin.cluster.state.term.TransportGetTermVersionAction; import org.opensearch.action.admin.cluster.stats.ClusterStatsAction; import org.opensearch.action.admin.cluster.stats.TransportClusterStatsAction; import org.opensearch.action.admin.cluster.storedscripts.DeleteStoredScriptAction; @@ -283,6 +281,8 @@ import org.opensearch.action.support.AutoCreateIndex; import org.opensearch.action.support.DestructiveOperations; import org.opensearch.action.support.TransportAction; +import org.opensearch.action.support.clustermanager.term.GetTermVersionAction; +import org.opensearch.action.support.clustermanager.term.TransportGetTermVersionAction; import org.opensearch.action.termvectors.MultiTermVectorsAction; import org.opensearch.action.termvectors.TermVectorsAction; import org.opensearch.action.termvectors.TransportMultiTermVectorsAction; diff --git a/server/src/main/java/org/opensearch/action/support/clustermanager/TransportClusterManagerNodeAction.java b/server/src/main/java/org/opensearch/action/support/clustermanager/TransportClusterManagerNodeAction.java index 5f57658e33924..080b0d607e991 100644 --- a/server/src/main/java/org/opensearch/action/support/clustermanager/TransportClusterManagerNodeAction.java +++ b/server/src/main/java/org/opensearch/action/support/clustermanager/TransportClusterManagerNodeAction.java @@ -37,13 +37,13 @@ import org.apache.logging.log4j.message.ParameterizedMessage; import org.opensearch.action.ActionListenerResponseHandler; import org.opensearch.action.ActionRunnable; -import org.opensearch.action.admin.cluster.state.term.GetTermVersionAction; -import org.opensearch.action.admin.cluster.state.term.GetTermVersionRequest; -import org.opensearch.action.admin.cluster.state.term.GetTermVersionResponse; import org.opensearch.action.bulk.BackoffPolicy; import org.opensearch.action.support.ActionFilters; import org.opensearch.action.support.HandledTransportAction; import org.opensearch.action.support.RetryableAction; +import org.opensearch.action.support.clustermanager.term.GetTermVersionAction; +import org.opensearch.action.support.clustermanager.term.GetTermVersionRequest; +import org.opensearch.action.support.clustermanager.term.GetTermVersionResponse; import org.opensearch.cluster.ClusterManagerNodeChangePredicate; import org.opensearch.cluster.ClusterState; import org.opensearch.cluster.ClusterStateObserver; diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/state/term/GetTermVersionAction.java b/server/src/main/java/org/opensearch/action/support/clustermanager/term/GetTermVersionAction.java similarity index 83% rename from server/src/main/java/org/opensearch/action/admin/cluster/state/term/GetTermVersionAction.java rename to server/src/main/java/org/opensearch/action/support/clustermanager/term/GetTermVersionAction.java index 3344fd549b23f..2401dddd0cab3 100644 --- a/server/src/main/java/org/opensearch/action/admin/cluster/state/term/GetTermVersionAction.java +++ b/server/src/main/java/org/opensearch/action/support/clustermanager/term/GetTermVersionAction.java @@ -6,7 +6,7 @@ * compatible open source license. */ -package org.opensearch.action.admin.cluster.state.term; +package org.opensearch.action.support.clustermanager.term; import org.opensearch.action.ActionType; @@ -18,7 +18,7 @@ public class GetTermVersionAction extends ActionType { public static final GetTermVersionAction INSTANCE = new GetTermVersionAction(); - public static final String NAME = "cluster:monitor/term"; + public static final String NAME = "internal:monitor/term"; private GetTermVersionAction() { super(NAME, GetTermVersionResponse::new); diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/state/term/GetTermVersionRequest.java b/server/src/main/java/org/opensearch/action/support/clustermanager/term/GetTermVersionRequest.java similarity index 93% rename from server/src/main/java/org/opensearch/action/admin/cluster/state/term/GetTermVersionRequest.java rename to server/src/main/java/org/opensearch/action/support/clustermanager/term/GetTermVersionRequest.java index b099f8087bd15..507997a1f7e7a 100644 --- a/server/src/main/java/org/opensearch/action/admin/cluster/state/term/GetTermVersionRequest.java +++ b/server/src/main/java/org/opensearch/action/support/clustermanager/term/GetTermVersionRequest.java @@ -6,7 +6,7 @@ * compatible open source license. */ -package org.opensearch.action.admin.cluster.state.term; +package org.opensearch.action.support.clustermanager.term; import org.opensearch.action.ActionRequestValidationException; import org.opensearch.action.support.clustermanager.ClusterManagerNodeReadRequest; diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/state/term/GetTermVersionResponse.java b/server/src/main/java/org/opensearch/action/support/clustermanager/term/GetTermVersionResponse.java similarity index 96% rename from server/src/main/java/org/opensearch/action/admin/cluster/state/term/GetTermVersionResponse.java rename to server/src/main/java/org/opensearch/action/support/clustermanager/term/GetTermVersionResponse.java index 16b355a80d1f2..0906abe57d547 100644 --- a/server/src/main/java/org/opensearch/action/admin/cluster/state/term/GetTermVersionResponse.java +++ b/server/src/main/java/org/opensearch/action/support/clustermanager/term/GetTermVersionResponse.java @@ -6,7 +6,7 @@ * compatible open source license. */ -package org.opensearch.action.admin.cluster.state.term; +package org.opensearch.action.support.clustermanager.term; import org.opensearch.cluster.ClusterState; import org.opensearch.cluster.coordination.ClusterStateTermVersion; diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/state/term/TransportGetTermVersionAction.java b/server/src/main/java/org/opensearch/action/support/clustermanager/term/TransportGetTermVersionAction.java similarity index 97% rename from server/src/main/java/org/opensearch/action/admin/cluster/state/term/TransportGetTermVersionAction.java rename to server/src/main/java/org/opensearch/action/support/clustermanager/term/TransportGetTermVersionAction.java index 88305252aa99c..4752a99c910e4 100644 --- a/server/src/main/java/org/opensearch/action/admin/cluster/state/term/TransportGetTermVersionAction.java +++ b/server/src/main/java/org/opensearch/action/support/clustermanager/term/TransportGetTermVersionAction.java @@ -6,7 +6,7 @@ * compatible open source license. */ -package org.opensearch.action.admin.cluster.state.term; +package org.opensearch.action.support.clustermanager.term; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/state/term/package-info.java b/server/src/main/java/org/opensearch/action/support/clustermanager/term/package-info.java similarity index 80% rename from server/src/main/java/org/opensearch/action/admin/cluster/state/term/package-info.java rename to server/src/main/java/org/opensearch/action/support/clustermanager/term/package-info.java index 0ee559c527d7d..229c405df2d7c 100644 --- a/server/src/main/java/org/opensearch/action/admin/cluster/state/term/package-info.java +++ b/server/src/main/java/org/opensearch/action/support/clustermanager/term/package-info.java @@ -7,4 +7,4 @@ */ /** Cluster Term transport handler. */ -package org.opensearch.action.admin.cluster.state.term; +package org.opensearch.action.support.clustermanager.term; diff --git a/server/src/test/java/org/opensearch/action/support/clustermanager/TransportClusterManagerTermCheckTests.java b/server/src/test/java/org/opensearch/action/support/clustermanager/TransportClusterManagerTermCheckTests.java index 8c7b7a0940c82..6d118cbccb42d 100644 --- a/server/src/test/java/org/opensearch/action/support/clustermanager/TransportClusterManagerTermCheckTests.java +++ b/server/src/test/java/org/opensearch/action/support/clustermanager/TransportClusterManagerTermCheckTests.java @@ -33,10 +33,10 @@ import org.opensearch.Version; import org.opensearch.action.ActionRequestValidationException; -import org.opensearch.action.admin.cluster.state.term.GetTermVersionResponse; import org.opensearch.action.support.ActionFilters; import org.opensearch.action.support.PlainActionFuture; import org.opensearch.action.support.ThreadedActionListener; +import org.opensearch.action.support.clustermanager.term.GetTermVersionResponse; import org.opensearch.action.support.replication.ClusterStateCreationUtils; import org.opensearch.cluster.ClusterState; import org.opensearch.cluster.block.ClusterBlockException; @@ -223,7 +223,7 @@ public void testTermCheckMatchWithClusterManager() throws ExecutionException, In assertThat(transport.capturedRequests().length, equalTo(1)); CapturingTransport.CapturedRequest capturedRequest = transport.capturedRequests()[0]; assertTrue(capturedRequest.node.isClusterManagerNode()); - assertThat(capturedRequest.action, equalTo("cluster:monitor/term")); + assertThat(capturedRequest.action, equalTo("internal:monitor/term")); GetTermVersionResponse response = new GetTermVersionResponse( new ClusterStateTermVersion( clusterService.state().getClusterName(), @@ -249,7 +249,7 @@ public void testTermCheckNoMatchWithClusterManager() throws ExecutionException, assertThat(transport.capturedRequests().length, equalTo(1)); CapturingTransport.CapturedRequest termCheckRequest = transport.capturedRequests()[0]; assertTrue(termCheckRequest.node.isClusterManagerNode()); - assertThat(termCheckRequest.action, equalTo("cluster:monitor/term")); + assertThat(termCheckRequest.action, equalTo("internal:monitor/term")); GetTermVersionResponse noMatchResponse = new GetTermVersionResponse( new ClusterStateTermVersion( clusterService.state().getClusterName(), diff --git a/server/src/test/java/org/opensearch/action/admin/cluster/state/term/ClusterTermVersionIT.java b/server/src/test/java/org/opensearch/action/support/clustermanager/term/ClusterTermVersionIT.java similarity index 98% rename from server/src/test/java/org/opensearch/action/admin/cluster/state/term/ClusterTermVersionIT.java rename to server/src/test/java/org/opensearch/action/support/clustermanager/term/ClusterTermVersionIT.java index fa2a6121af349..7b783e025a575 100644 --- a/server/src/test/java/org/opensearch/action/admin/cluster/state/term/ClusterTermVersionIT.java +++ b/server/src/test/java/org/opensearch/action/support/clustermanager/term/ClusterTermVersionIT.java @@ -6,7 +6,7 @@ * compatible open source license. */ -package org.opensearch.action.admin.cluster.state.term; +package org.opensearch.action.support.clustermanager.term; import org.opensearch.action.admin.cluster.state.ClusterStateAction; import org.opensearch.action.admin.cluster.state.ClusterStateRequest; diff --git a/server/src/test/java/org/opensearch/action/admin/cluster/state/term/ClusterTermVersionTests.java b/server/src/test/java/org/opensearch/action/support/clustermanager/term/ClusterTermVersionTests.java similarity index 93% rename from server/src/test/java/org/opensearch/action/admin/cluster/state/term/ClusterTermVersionTests.java rename to server/src/test/java/org/opensearch/action/support/clustermanager/term/ClusterTermVersionTests.java index 22d9623eebdbe..23ae8c6a58776 100644 --- a/server/src/test/java/org/opensearch/action/admin/cluster/state/term/ClusterTermVersionTests.java +++ b/server/src/test/java/org/opensearch/action/support/clustermanager/term/ClusterTermVersionTests.java @@ -6,7 +6,7 @@ * compatible open source license. */ -package org.opensearch.action.admin.cluster.state.term; +package org.opensearch.action.support.clustermanager.term; import org.opensearch.cluster.service.ClusterService; import org.opensearch.test.OpenSearchSingleNodeTestCase; diff --git a/server/src/test/java/org/opensearch/snapshots/SnapshotResiliencyTests.java b/server/src/test/java/org/opensearch/snapshots/SnapshotResiliencyTests.java index 58315ba031b84..e9bbf3d861408 100644 --- a/server/src/test/java/org/opensearch/snapshots/SnapshotResiliencyTests.java +++ b/server/src/test/java/org/opensearch/snapshots/SnapshotResiliencyTests.java @@ -66,8 +66,6 @@ import org.opensearch.action.admin.cluster.state.ClusterStateRequest; import org.opensearch.action.admin.cluster.state.ClusterStateResponse; import org.opensearch.action.admin.cluster.state.TransportClusterStateAction; -import org.opensearch.action.admin.cluster.state.term.GetTermVersionAction; -import org.opensearch.action.admin.cluster.state.term.TransportGetTermVersionAction; import org.opensearch.action.admin.indices.create.CreateIndexAction; import org.opensearch.action.admin.indices.create.CreateIndexRequest; import org.opensearch.action.admin.indices.create.CreateIndexResponse; @@ -105,6 +103,8 @@ import org.opensearch.action.support.PlainActionFuture; import org.opensearch.action.support.TransportAction; import org.opensearch.action.support.WriteRequest; +import org.opensearch.action.support.clustermanager.term.GetTermVersionAction; +import org.opensearch.action.support.clustermanager.term.TransportGetTermVersionAction; import org.opensearch.action.support.master.AcknowledgedResponse; import org.opensearch.action.update.UpdateHelper; import org.opensearch.client.AdminClient; From 9267170e77c3e523a877fae8d7688be72ebccffe Mon Sep 17 00:00:00 2001 From: Zelin Hao Date: Mon, 25 Mar 2024 06:03:54 -0700 Subject: [PATCH 37/38] Replace configureEach with all for publication iteration (#12876) * Change from configureEach to all to resolve incompatibility Signed-off-by: Zelin Hao * Update Changelog Signed-off-by: Zelin Hao --------- Signed-off-by: Zelin Hao --- CHANGELOG.md | 3 ++- .../gradle/precommit/PomValidationPrecommitPlugin.java | 4 ++-- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 346913f025f4d..882a46d60a191 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -19,7 +19,8 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Remote reindex: Add support for configurable retry mechanism ([#12561](https://github.com/opensearch-project/OpenSearch/pull/12561)) - [Admission Control] Integrate IO Usage Tracker to the Resource Usage Collector Service and Emit IO Usage Stats ([#11880](https://github.com/opensearch-project/OpenSearch/pull/11880)) - Tracing for deep search path ([#12103](https://github.com/opensearch-project/OpenSearch/pull/12103)) -- Add explicit dependency to validatePom and generatePom tasks ([#12103](https://github.com/opensearch-project/OpenSearch/pull/12807)) +- Add explicit dependency to validatePom and generatePom tasks ([#12807](https://github.com/opensearch-project/OpenSearch/pull/12807)) +- Replace configureEach with all for publication iteration ([#12876](https://github.com/opensearch-project/OpenSearch/pull/12876)) ### Dependencies - Bump `log4j-core` from 2.18.0 to 2.19.0 diff --git a/buildSrc/src/main/java/org/opensearch/gradle/precommit/PomValidationPrecommitPlugin.java b/buildSrc/src/main/java/org/opensearch/gradle/precommit/PomValidationPrecommitPlugin.java index 62194af3c3350..d3f173c9c02ea 100644 --- a/buildSrc/src/main/java/org/opensearch/gradle/precommit/PomValidationPrecommitPlugin.java +++ b/buildSrc/src/main/java/org/opensearch/gradle/precommit/PomValidationPrecommitPlugin.java @@ -48,7 +48,7 @@ public class PomValidationPrecommitPlugin extends PrecommitPlugin { public TaskProvider createTask(Project project) { TaskProvider validatePom = project.getTasks().register("validatePom"); PublishingExtension publishing = project.getExtensions().getByType(PublishingExtension.class); - publishing.getPublications().configureEach(publication -> { + publishing.getPublications().all(publication -> { String publicationName = Util.capitalize(publication.getName()); TaskProvider validateTask = project.getTasks() .register("validate" + publicationName + "Pom", PomValidationTask.class); @@ -59,7 +59,7 @@ public TaskProvider createTask(Project project) { validateTask.configure(task -> { task.dependsOn(generateMavenPom); task.getPomFile().fileProvider(generateMavenPom.map(GenerateMavenPom::getDestination)); - publishing.getPublications().configureEach(publicationForPomGen -> { + publishing.getPublications().all(publicationForPomGen -> { task.mustRunAfter( project.getTasks() .withType(GenerateMavenPom.class) From 968f99752a313edb241cd64162b456e7042784bb Mon Sep 17 00:00:00 2001 From: Craig Perkins Date: Mon, 25 Mar 2024 10:56:29 -0400 Subject: [PATCH 38/38] Bump OpenSearch version in README for open issues (#12895) Signed-off-by: Craig Perkins --- README.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/README.md b/README.md index b5fc45509b002..748f8a366ecc8 100644 --- a/README.md +++ b/README.md @@ -7,8 +7,8 @@ [![Security Vulnerabilities](https://img.shields.io/github/issues/opensearch-project/OpenSearch/security%20vulnerability?labelColor=red)](https://github.com/opensearch-project/OpenSearch/issues?q=is%3Aissue+is%3Aopen+label%3A"security%20vulnerability") [![Open Issues](https://img.shields.io/github/issues/opensearch-project/OpenSearch)](https://github.com/opensearch-project/OpenSearch/issues) [![Open Pull Requests](https://img.shields.io/github/issues-pr/opensearch-project/OpenSearch)](https://github.com/opensearch-project/OpenSearch/pulls) -[![2.10 Open Issues](https://img.shields.io/github/issues/opensearch-project/OpenSearch/v2.10.0)](https://github.com/opensearch-project/OpenSearch/issues?q=is%3Aissue+is%3Aopen+label%3A"v2.10.0") -[![3.0 Open Issues](https://img.shields.io/github/issues/opensearch-project/OpenSearch/v3.0.0)](https://github.com/opensearch-project/OpenSearch/issues?q=is%3Aissue+is%3Aopen+label%3A"v3.0.0") +[![2.14.0 Open Issues](https://img.shields.io/github/issues/opensearch-project/OpenSearch/v2.14.0)](https://github.com/opensearch-project/OpenSearch/issues?q=is%3Aissue+is%3Aopen+label%3A"v2.14.0") +[![3.0.0 Open Issues](https://img.shields.io/github/issues/opensearch-project/OpenSearch/v3.0.0)](https://github.com/opensearch-project/OpenSearch/issues?q=is%3Aissue+is%3Aopen+label%3A"v3.0.0") [![GHA gradle check](https://github.com/opensearch-project/OpenSearch/actions/workflows/gradle-check.yml/badge.svg)](https://github.com/opensearch-project/OpenSearch/actions/workflows/gradle-check.yml) [![GHA validate pull request](https://github.com/opensearch-project/OpenSearch/actions/workflows/wrapper.yml/badge.svg)](https://github.com/opensearch-project/OpenSearch/actions/workflows/wrapper.yml) [![GHA precommit](https://github.com/opensearch-project/OpenSearch/actions/workflows/precommit.yml/badge.svg)](https://github.com/opensearch-project/OpenSearch/actions/workflows/precommit.yml)