From 1dbb63a8ee04812bd5f5ef5bc4995eac8b090438 Mon Sep 17 00:00:00 2001 From: Andrew Ross Date: Fri, 26 Aug 2022 12:58:47 -0700 Subject: [PATCH 1/3] Do not fail replica shard due to primary closure (#4133) This commit prevents a replica shard from being failed in the case that a replication action to a replica is canceled due to the primary shard being closed. Signed-off-by: Andrew Ross Signed-off-by: Andrew Ross --- CHANGELOG.md | 1 + .../index/store/CorruptedFileIT.java | 101 ++++++++++++++++++ .../org/opensearch/OpenSearchException.java | 7 ++ .../PendingReplicationActions.java | 14 ++- .../replication/TransportWriteAction.java | 24 +++-- .../shard/PrimaryShardClosedException.java | 26 +++++ .../ExceptionSerializationTests.java | 2 + .../PendingReplicationActionsTests.java | 3 +- .../TransportWriteActionTests.java | 45 ++++++++ 9 files changed, 205 insertions(+), 18 deletions(-) create mode 100644 server/src/main/java/org/opensearch/index/shard/PrimaryShardClosedException.java diff --git a/CHANGELOG.md b/CHANGELOG.md index c258100894555..e988435a688da 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -14,6 +14,7 @@ Inspired from [Keep a Changelog](https://keepachangelog.com/en/1.0.0/) ### Fixed - `opensearch-service.bat start` and `opensearch-service.bat manager` failing to run ([#4289](https://github.com/opensearch-project/OpenSearch/pull/4289)) - PR reference to checkout code for changelog verifier ([#4296](https://github.com/opensearch-project/OpenSearch/pull/4296)) +- Do not fail replica shard due to primary closure ([#4133](https://github.com/opensearch-project/OpenSearch/pull/4133)) ### Security diff --git a/server/src/internalClusterTest/java/org/opensearch/index/store/CorruptedFileIT.java b/server/src/internalClusterTest/java/org/opensearch/index/store/CorruptedFileIT.java index ee2067c591cef..960e17b76acb5 100644 --- a/server/src/internalClusterTest/java/org/opensearch/index/store/CorruptedFileIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/index/store/CorruptedFileIT.java @@ -40,6 +40,7 @@ import org.apache.lucene.store.FSDirectory; import org.apache.lucene.util.BytesRef; +import org.hamcrest.MatcherAssert; import org.opensearch.action.admin.cluster.health.ClusterHealthResponse; import org.opensearch.action.admin.cluster.node.stats.NodeStats; import org.opensearch.action.admin.cluster.node.stats.NodesStatsResponse; @@ -48,6 +49,7 @@ import org.opensearch.action.admin.indices.shards.IndicesShardStoresResponse; import org.opensearch.action.index.IndexRequestBuilder; import org.opensearch.action.search.SearchResponse; +import org.opensearch.action.support.replication.TransportReplicationAction; import org.opensearch.client.Requests; import org.opensearch.cluster.ClusterState; import org.opensearch.cluster.health.ClusterHealthStatus; @@ -108,6 +110,7 @@ import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.stream.Collectors; import static org.opensearch.action.admin.cluster.node.stats.NodesStatsRequest.Metric.FS; import static org.opensearch.common.util.CollectionUtils.iterableAsArrayList; @@ -698,6 +701,104 @@ public void testReplicaCorruption() throws Exception { ensureGreen(TimeValue.timeValueSeconds(60)); } + public void testPrimaryCorruptionDuringReplicationDoesNotFailReplicaShard() throws Exception { + internalCluster().ensureAtLeastNumDataNodes(2); + final NodesStatsResponse nodeStats = client().admin().cluster().prepareNodesStats().get(); + final List dataNodeStats = nodeStats.getNodes() + .stream() + .filter(stat -> stat.getNode().isDataNode()) + .collect(Collectors.toUnmodifiableList()); + MatcherAssert.assertThat(dataNodeStats.size(), greaterThanOrEqualTo(2)); + + final NodeStats primaryNode = dataNodeStats.get(0); + final NodeStats replicaNode = dataNodeStats.get(1); + assertAcked( + prepareCreate("test").setSettings( + Settings.builder() + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, "0") + .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) + .put("index.routing.allocation.include._name", primaryNode.getNode().getName()) + .put(EnableAllocationDecider.INDEX_ROUTING_REBALANCE_ENABLE_SETTING.getKey(), EnableAllocationDecider.Rebalance.NONE) + .put("index.allocation.max_retries", Integer.MAX_VALUE) // keep on retrying + + ) + ); + ensureGreen(); + + // Add custom send behavior between primary and replica that will + // count down a latch to indicate that a replication operation is + // currently in flight, and then block on a second latch that will + // be released once the primary shard has been corrupted. + final CountDownLatch indexingInFlight = new CountDownLatch(1); + final CountDownLatch corruptionHasHappened = new CountDownLatch(1); + final MockTransportService mockTransportService = ((MockTransportService) internalCluster().getInstance( + TransportService.class, + primaryNode.getNode().getName() + )); + mockTransportService.addSendBehavior( + internalCluster().getInstance(TransportService.class, replicaNode.getNode().getName()), + (connection, requestId, action, request, options) -> { + if (request instanceof TransportReplicationAction.ConcreteShardRequest) { + indexingInFlight.countDown(); + try { + corruptionHasHappened.await(); + } catch (InterruptedException e) { + logger.info("Interrupted while waiting for corruption"); + } + } + connection.sendRequest(requestId, action, request, options); + } + ); + + // Configure the modified data node as a replica + final Settings build = Settings.builder() + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, "1") + .put("index.routing.allocation.include._name", primaryNode.getNode().getName() + "," + replicaNode.getNode().getName()) + .build(); + client().admin().indices().prepareUpdateSettings("test").setSettings(build).get(); + client().admin().cluster().prepareReroute().get(); + ensureGreen(); + + // Create a snapshot repository. This repo is used to take a snapshot after + // corrupting a file, which causes the node to notice the corrupt data and + // close the shard. + assertAcked( + client().admin() + .cluster() + .preparePutRepository("test-repo") + .setType("fs") + .setSettings( + Settings.builder() + .put("location", randomRepoPath().toAbsolutePath()) + .put("compress", randomBoolean()) + .put("chunk_size", randomIntBetween(100, 1000), ByteSizeUnit.BYTES) + ) + ); + + client().prepareIndex("test").setSource("field", "value").execute(); + indexingInFlight.await(); + + // Corrupt a file on the primary then take a snapshot. Snapshot should + // finish in the PARTIAL state since the corrupted file will cause a checksum + // validation failure. + final ShardRouting corruptedShardRouting = corruptRandomPrimaryFile(); + logger.info("--> {} corrupted", corruptedShardRouting); + final CreateSnapshotResponse createSnapshotResponse = client().admin() + .cluster() + .prepareCreateSnapshot("test-repo", "test-snap") + .setWaitForCompletion(true) + .setIndices("test") + .get(); + final SnapshotState snapshotState = createSnapshotResponse.getSnapshotInfo().state(); + MatcherAssert.assertThat("Expect file corruption to cause PARTIAL snapshot state", snapshotState, equalTo(SnapshotState.PARTIAL)); + + // Unblock the blocked indexing thread now that corruption on the primary has been confirmed + corruptionHasHappened.countDown(); + + // Assert the cluster returns to green status because the replica will be promoted to primary + ensureGreen(); + } + private int numShards(String... index) { ClusterState state = client().admin().cluster().prepareState().get().getState(); GroupShardsIterator shardIterators = state.getRoutingTable().activePrimaryShardsGrouped(index, false); diff --git a/server/src/main/java/org/opensearch/OpenSearchException.java b/server/src/main/java/org/opensearch/OpenSearchException.java index 4ebcd9622ce38..87efc03734d26 100644 --- a/server/src/main/java/org/opensearch/OpenSearchException.java +++ b/server/src/main/java/org/opensearch/OpenSearchException.java @@ -68,6 +68,7 @@ import static java.util.Collections.singletonMap; import static java.util.Collections.unmodifiableMap; import static org.opensearch.Version.V_2_1_0; +import static org.opensearch.Version.V_3_0_0; import static org.opensearch.cluster.metadata.IndexMetadata.INDEX_UUID_NA_VALUE; import static org.opensearch.common.xcontent.XContentParserUtils.ensureExpectedToken; import static org.opensearch.common.xcontent.XContentParserUtils.ensureFieldName; @@ -1601,6 +1602,12 @@ private enum OpenSearchExceptionHandle { org.opensearch.indices.replication.common.ReplicationFailedException::new, 161, V_2_1_0 + ), + PRIMARY_SHARD_CLOSED_EXCEPTION( + org.opensearch.index.shard.PrimaryShardClosedException.class, + org.opensearch.index.shard.PrimaryShardClosedException::new, + 162, + V_3_0_0 ); final Class exceptionClass; diff --git a/server/src/main/java/org/opensearch/action/support/replication/PendingReplicationActions.java b/server/src/main/java/org/opensearch/action/support/replication/PendingReplicationActions.java index b305c4c8c83a7..7087b64758888 100644 --- a/server/src/main/java/org/opensearch/action/support/replication/PendingReplicationActions.java +++ b/server/src/main/java/org/opensearch/action/support/replication/PendingReplicationActions.java @@ -35,6 +35,7 @@ import org.opensearch.action.support.RetryableAction; import org.opensearch.common.lease.Releasable; import org.opensearch.common.util.concurrent.ConcurrentCollections; +import org.opensearch.index.shard.PrimaryShardClosedException; import org.opensearch.index.shard.IndexShardClosedException; import org.opensearch.index.shard.ReplicationGroup; import org.opensearch.index.shard.ShardId; @@ -45,6 +46,7 @@ import java.util.Map; import java.util.Set; import java.util.function.Consumer; +import java.util.function.Supplier; /** * Pending Replication Actions @@ -121,7 +123,7 @@ synchronized void acceptNewTrackedAllocationIds(Set trackedAllocationIds } } - cancelActions(toCancel, "Replica left ReplicationGroup"); + cancelActions(toCancel, () -> new IndexShardClosedException(shardId, "Replica left ReplicationGroup")); } @Override @@ -129,15 +131,11 @@ public synchronized void close() { ArrayList>> toCancel = new ArrayList<>(onGoingReplicationActions.values()); onGoingReplicationActions.clear(); - cancelActions(toCancel, "Primary closed."); + cancelActions(toCancel, () -> new PrimaryShardClosedException(shardId)); } - private void cancelActions(ArrayList>> toCancel, String message) { + private void cancelActions(ArrayList>> toCancel, Supplier exceptionSupplier) { threadPool.executor(ThreadPool.Names.GENERIC) - .execute( - () -> toCancel.stream() - .flatMap(Collection::stream) - .forEach(action -> action.cancel(new IndexShardClosedException(shardId, message))) - ); + .execute(() -> toCancel.stream().flatMap(Collection::stream).forEach(action -> action.cancel(exceptionSupplier.get()))); } } diff --git a/server/src/main/java/org/opensearch/action/support/replication/TransportWriteAction.java b/server/src/main/java/org/opensearch/action/support/replication/TransportWriteAction.java index 39fb89bc48568..7fc810808f560 100644 --- a/server/src/main/java/org/opensearch/action/support/replication/TransportWriteAction.java +++ b/server/src/main/java/org/opensearch/action/support/replication/TransportWriteAction.java @@ -52,6 +52,7 @@ import org.opensearch.index.IndexingPressureService; import org.opensearch.index.engine.Engine; import org.opensearch.index.mapper.MapperParsingException; +import org.opensearch.index.shard.PrimaryShardClosedException; import org.opensearch.index.shard.IndexShard; import org.opensearch.index.shard.ShardId; import org.opensearch.index.translog.Translog; @@ -514,15 +515,20 @@ public void failShardIfNeeded( if (TransportActions.isShardNotAvailableException(exception) == false) { logger.warn(new ParameterizedMessage("[{}] {}", replica.shardId(), message), exception); } - shardStateAction.remoteShardFailed( - replica.shardId(), - replica.allocationId().getId(), - primaryTerm, - true, - message, - exception, - listener - ); + // If a write action fails due to the closure of the primary shard + // then the replicas should not be marked as failed since they are + // still up-to-date with the (now closed) primary shard + if (exception instanceof PrimaryShardClosedException == false) { + shardStateAction.remoteShardFailed( + replica.shardId(), + replica.allocationId().getId(), + primaryTerm, + true, + message, + exception, + listener + ); + } } @Override diff --git a/server/src/main/java/org/opensearch/index/shard/PrimaryShardClosedException.java b/server/src/main/java/org/opensearch/index/shard/PrimaryShardClosedException.java new file mode 100644 index 0000000000000..d1b2bf9079289 --- /dev/null +++ b/server/src/main/java/org/opensearch/index/shard/PrimaryShardClosedException.java @@ -0,0 +1,26 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.index.shard; + +import java.io.IOException; + +import org.opensearch.common.io.stream.StreamInput; + +/** + * Exception to indicate failures are caused due to the closure of the primary + * shard. + * + * @opensearch.internal + */ +public class PrimaryShardClosedException extends IndexShardClosedException { + public PrimaryShardClosedException(ShardId shardId) { + super(shardId, "Primary closed"); + } + + public PrimaryShardClosedException(StreamInput in) throws IOException { + super(in); + } +} diff --git a/server/src/test/java/org/opensearch/ExceptionSerializationTests.java b/server/src/test/java/org/opensearch/ExceptionSerializationTests.java index 5a93d7c0bd86e..26b0ce7e9e20c 100644 --- a/server/src/test/java/org/opensearch/ExceptionSerializationTests.java +++ b/server/src/test/java/org/opensearch/ExceptionSerializationTests.java @@ -80,6 +80,7 @@ import org.opensearch.index.seqno.RetentionLeaseNotFoundException; import org.opensearch.index.shard.IllegalIndexShardStateException; import org.opensearch.index.shard.IndexShardState; +import org.opensearch.index.shard.PrimaryShardClosedException; import org.opensearch.index.shard.ShardId; import org.opensearch.index.shard.ShardNotInPrimaryModeException; import org.opensearch.indices.IndexTemplateMissingException; @@ -858,6 +859,7 @@ public void testIds() { ids.put(159, NodeHealthCheckFailureException.class); ids.put(160, NoSeedNodeLeftException.class); ids.put(161, ReplicationFailedException.class); + ids.put(162, PrimaryShardClosedException.class); Map, Integer> reverse = new HashMap<>(); for (Map.Entry> entry : ids.entrySet()) { diff --git a/server/src/test/java/org/opensearch/action/support/replication/PendingReplicationActionsTests.java b/server/src/test/java/org/opensearch/action/support/replication/PendingReplicationActionsTests.java index ec0cefed842cd..66d3b843529ab 100644 --- a/server/src/test/java/org/opensearch/action/support/replication/PendingReplicationActionsTests.java +++ b/server/src/test/java/org/opensearch/action/support/replication/PendingReplicationActionsTests.java @@ -38,6 +38,7 @@ import org.opensearch.common.UUIDs; import org.opensearch.common.unit.TimeValue; import org.opensearch.index.shard.IndexShardClosedException; +import org.opensearch.index.shard.PrimaryShardClosedException; import org.opensearch.index.shard.ShardId; import org.opensearch.test.OpenSearchTestCase; import org.opensearch.threadpool.TestThreadPool; @@ -102,7 +103,7 @@ public void testAllocationIdActionWillBeCancelledOnClose() { pendingReplication.addPendingAction(allocationId, action); action.run(); pendingReplication.close(); - expectThrows(IndexShardClosedException.class, future::actionGet); + expectThrows(PrimaryShardClosedException.class, future::actionGet); } private class TestAction extends RetryableAction { diff --git a/server/src/test/java/org/opensearch/action/support/replication/TransportWriteActionTests.java b/server/src/test/java/org/opensearch/action/support/replication/TransportWriteActionTests.java index 4da32a890fd0e..137aca4966936 100644 --- a/server/src/test/java/org/opensearch/action/support/replication/TransportWriteActionTests.java +++ b/server/src/test/java/org/opensearch/action/support/replication/TransportWriteActionTests.java @@ -32,6 +32,7 @@ package org.opensearch.action.support.replication; +import org.hamcrest.MatcherAssert; import org.opensearch.OpenSearchException; import org.opensearch.action.ActionListener; import org.opensearch.action.support.ActionFilters; @@ -57,6 +58,7 @@ import org.opensearch.index.IndexService; import org.opensearch.index.IndexingPressureService; import org.opensearch.index.shard.IndexShard; +import org.opensearch.index.shard.PrimaryShardClosedException; import org.opensearch.index.shard.ShardId; import org.opensearch.index.shard.ShardNotFoundException; import org.opensearch.index.translog.Translog; @@ -91,6 +93,7 @@ import java.util.stream.Collectors; import static java.util.Collections.emptyMap; +import static org.hamcrest.Matchers.emptyArray; import static org.opensearch.test.ClusterServiceUtils.createClusterService; import static org.hamcrest.Matchers.arrayWithSize; import static org.hamcrest.Matchers.equalTo; @@ -395,6 +398,48 @@ public void testReplicaProxy() throws InterruptedException, ExecutionException { } } + public void testPrimaryClosedDoesNotFailShard() { + final CapturingTransport transport = new CapturingTransport(); + final TransportService transportService = transport.createTransportService( + clusterService.getSettings(), + threadPool, + TransportService.NOOP_TRANSPORT_INTERCEPTOR, + x -> clusterService.localNode(), + null, + Collections.emptySet() + ); + transportService.start(); + transportService.acceptIncomingRequests(); + final ShardStateAction shardStateAction = new ShardStateAction(clusterService, transportService, null, null, threadPool); + final TestAction action = new TestAction( + Settings.EMPTY, + "internal:testAction", + transportService, + clusterService, + shardStateAction, + threadPool + ); + final String index = "test"; + final ShardId shardId = new ShardId(index, "_na_", 0); + final ClusterState state = ClusterStateCreationUtils.stateWithActivePrimary(index, true, 1, 0); + ClusterServiceUtils.setState(clusterService, state); + final long primaryTerm = state.metadata().index(index).primaryTerm(0); + final ShardRouting shardRouting = state.routingTable().shardRoutingTable(shardId).replicaShards().get(0); + + // Assert that failShardIfNeeded is a no-op for the PrimaryShardClosedException failure + final AtomicInteger callbackCount = new AtomicInteger(0); + action.newReplicasProxy() + .failShardIfNeeded( + shardRouting, + primaryTerm, + "test", + new PrimaryShardClosedException(shardId), + ActionListener.wrap(callbackCount::incrementAndGet) + ); + MatcherAssert.assertThat(transport.getCapturedRequestsAndClear(), emptyArray()); + MatcherAssert.assertThat(callbackCount.get(), equalTo(0)); + } + private class TestAction extends TransportWriteAction { private final boolean withDocumentFailureOnPrimary; From c62cecb048bafe8b79709660956fe4cba5548872 Mon Sep 17 00:00:00 2001 From: Andriy Redko Date: Fri, 26 Aug 2022 17:46:01 -0400 Subject: [PATCH 2/3] Some dependency updates (#4308) Signed-off-by: Andriy Redko Signed-off-by: Andriy Redko --- CHANGELOG.md | 1 + buildSrc/version.properties | 10 +++++----- client/rest/licenses/commons-codec-1.13.jar.sha1 | 1 - client/rest/licenses/commons-codec-1.15.jar.sha1 | 1 + client/rest/licenses/httpasyncclient-4.1.4.jar.sha1 | 1 - client/rest/licenses/httpasyncclient-4.1.5.jar.sha1 | 1 + client/rest/licenses/httpcore-4.4.12.jar.sha1 | 1 - client/rest/licenses/httpcore-4.4.15.jar.sha1 | 1 + client/rest/licenses/httpcore-nio-4.4.12.jar.sha1 | 1 - client/rest/licenses/httpcore-nio-4.4.15.jar.sha1 | 1 + client/sniffer/licenses/commons-codec-1.13.jar.sha1 | 1 - client/sniffer/licenses/commons-codec-1.15.jar.sha1 | 1 + client/sniffer/licenses/httpcore-4.4.12.jar.sha1 | 1 - client/sniffer/licenses/httpcore-4.4.15.jar.sha1 | 1 + .../licenses/commons-codec-1.13.jar.sha1 | 1 - .../licenses/commons-codec-1.15.jar.sha1 | 1 + .../licenses/commons-codec-1.13.jar.sha1 | 1 - .../licenses/commons-codec-1.15.jar.sha1 | 1 + .../licenses/httpcore-4.4.12.jar.sha1 | 1 - .../licenses/httpcore-4.4.15.jar.sha1 | 1 + .../discovery-ec2/licenses/commons-codec-1.13.jar.sha1 | 1 - .../discovery-ec2/licenses/commons-codec-1.15.jar.sha1 | 1 + .../discovery-ec2/licenses/httpcore-4.4.12.jar.sha1 | 1 - .../discovery-ec2/licenses/httpcore-4.4.15.jar.sha1 | 1 + .../discovery-gce/licenses/commons-codec-1.13.jar.sha1 | 1 - .../discovery-gce/licenses/commons-codec-1.15.jar.sha1 | 1 + .../discovery-gce/licenses/httpcore-4.4.12.jar.sha1 | 1 - .../discovery-gce/licenses/httpcore-4.4.15.jar.sha1 | 1 + .../licenses/commons-codec-1.13.jar.sha1 | 1 - .../licenses/commons-codec-1.15.jar.sha1 | 1 + .../licenses/slf4j-api-1.6.2.jar.sha1 | 1 - .../licenses/slf4j-api-1.7.36.jar.sha1 | 1 + .../repository-azure/licenses/slf4j-api-1.6.2.jar.sha1 | 1 - .../licenses/slf4j-api-1.7.36.jar.sha1 | 1 + .../licenses/commons-codec-1.13.jar.sha1 | 1 - .../licenses/commons-codec-1.15.jar.sha1 | 1 + .../licenses/commons-codec-1.13.jar.sha1 | 1 - .../licenses/commons-codec-1.15.jar.sha1 | 1 + .../repository-hdfs/licenses/slf4j-api-1.6.2.jar.sha1 | 1 - .../repository-hdfs/licenses/slf4j-api-1.7.36.jar.sha1 | 1 + .../repository-s3/licenses/commons-codec-1.13.jar.sha1 | 1 - .../repository-s3/licenses/commons-codec-1.15.jar.sha1 | 1 + .../repository-s3/licenses/httpcore-4.4.12.jar.sha1 | 1 - .../repository-s3/licenses/httpcore-4.4.15.jar.sha1 | 1 + 44 files changed, 27 insertions(+), 26 deletions(-) delete mode 100644 client/rest/licenses/commons-codec-1.13.jar.sha1 create mode 100644 client/rest/licenses/commons-codec-1.15.jar.sha1 delete mode 100644 client/rest/licenses/httpasyncclient-4.1.4.jar.sha1 create mode 100644 client/rest/licenses/httpasyncclient-4.1.5.jar.sha1 delete mode 100644 client/rest/licenses/httpcore-4.4.12.jar.sha1 create mode 100644 client/rest/licenses/httpcore-4.4.15.jar.sha1 delete mode 100644 client/rest/licenses/httpcore-nio-4.4.12.jar.sha1 create mode 100644 client/rest/licenses/httpcore-nio-4.4.15.jar.sha1 delete mode 100644 client/sniffer/licenses/commons-codec-1.13.jar.sha1 create mode 100644 client/sniffer/licenses/commons-codec-1.15.jar.sha1 delete mode 100644 client/sniffer/licenses/httpcore-4.4.12.jar.sha1 create mode 100644 client/sniffer/licenses/httpcore-4.4.15.jar.sha1 delete mode 100644 plugins/analysis-phonetic/licenses/commons-codec-1.13.jar.sha1 create mode 100644 plugins/analysis-phonetic/licenses/commons-codec-1.15.jar.sha1 delete mode 100644 plugins/discovery-azure-classic/licenses/commons-codec-1.13.jar.sha1 create mode 100644 plugins/discovery-azure-classic/licenses/commons-codec-1.15.jar.sha1 delete mode 100644 plugins/discovery-azure-classic/licenses/httpcore-4.4.12.jar.sha1 create mode 100644 plugins/discovery-azure-classic/licenses/httpcore-4.4.15.jar.sha1 delete mode 100644 plugins/discovery-ec2/licenses/commons-codec-1.13.jar.sha1 create mode 100644 plugins/discovery-ec2/licenses/commons-codec-1.15.jar.sha1 delete mode 100644 plugins/discovery-ec2/licenses/httpcore-4.4.12.jar.sha1 create mode 100644 plugins/discovery-ec2/licenses/httpcore-4.4.15.jar.sha1 delete mode 100644 plugins/discovery-gce/licenses/commons-codec-1.13.jar.sha1 create mode 100644 plugins/discovery-gce/licenses/commons-codec-1.15.jar.sha1 delete mode 100644 plugins/discovery-gce/licenses/httpcore-4.4.12.jar.sha1 create mode 100644 plugins/discovery-gce/licenses/httpcore-4.4.15.jar.sha1 delete mode 100644 plugins/ingest-attachment/licenses/commons-codec-1.13.jar.sha1 create mode 100644 plugins/ingest-attachment/licenses/commons-codec-1.15.jar.sha1 delete mode 100644 plugins/ingest-attachment/licenses/slf4j-api-1.6.2.jar.sha1 create mode 100644 plugins/ingest-attachment/licenses/slf4j-api-1.7.36.jar.sha1 delete mode 100644 plugins/repository-azure/licenses/slf4j-api-1.6.2.jar.sha1 create mode 100644 plugins/repository-azure/licenses/slf4j-api-1.7.36.jar.sha1 delete mode 100644 plugins/repository-gcs/licenses/commons-codec-1.13.jar.sha1 create mode 100644 plugins/repository-gcs/licenses/commons-codec-1.15.jar.sha1 delete mode 100644 plugins/repository-hdfs/licenses/commons-codec-1.13.jar.sha1 create mode 100644 plugins/repository-hdfs/licenses/commons-codec-1.15.jar.sha1 delete mode 100644 plugins/repository-hdfs/licenses/slf4j-api-1.6.2.jar.sha1 create mode 100644 plugins/repository-hdfs/licenses/slf4j-api-1.7.36.jar.sha1 delete mode 100644 plugins/repository-s3/licenses/commons-codec-1.13.jar.sha1 create mode 100644 plugins/repository-s3/licenses/commons-codec-1.15.jar.sha1 delete mode 100644 plugins/repository-s3/licenses/httpcore-4.4.12.jar.sha1 create mode 100644 plugins/repository-s3/licenses/httpcore-4.4.15.jar.sha1 diff --git a/CHANGELOG.md b/CHANGELOG.md index e988435a688da..26ff011609635 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -6,6 +6,7 @@ Inspired from [Keep a Changelog](https://keepachangelog.com/en/1.0.0/) - Github workflow for changelog verification ([#4085](https://github.com/opensearch-project/OpenSearch/pull/4085)) ### Changed + - Dependency updates (httpcore, mockito, slf4j, httpasyncclient, commons-codec) ([#4308](https://github.com/opensearch-project/OpenSearch/pull/4308)) ### Deprecated diff --git a/buildSrc/version.properties b/buildSrc/version.properties index 4af1acfed0ab2..876910d5351d0 100644 --- a/buildSrc/version.properties +++ b/buildSrc/version.properties @@ -15,7 +15,7 @@ snakeyaml = 1.26 icu4j = 70.1 supercsv = 2.4.0 log4j = 2.17.1 -slf4j = 1.6.2 +slf4j = 1.7.36 asm = 9.3 # when updating the JNA version, also update the version in buildSrc/build.gradle @@ -26,10 +26,10 @@ joda = 2.10.13 # client dependencies httpclient = 4.5.13 -httpcore = 4.4.12 -httpasyncclient = 4.1.4 +httpcore = 4.4.15 +httpasyncclient = 4.1.5 commonslogging = 1.2 -commonscodec = 1.13 +commonscodec = 1.15 # plugin dependencies aws = 1.12.270 @@ -42,7 +42,7 @@ bouncycastle=1.70 randomizedrunner = 2.7.1 junit = 4.13.2 hamcrest = 2.1 -mockito = 4.6.1 +mockito = 4.7.0 objenesis = 3.2 bytebuddy = 1.12.12 diff --git a/client/rest/licenses/commons-codec-1.13.jar.sha1 b/client/rest/licenses/commons-codec-1.13.jar.sha1 deleted file mode 100644 index 66b72c414d63a..0000000000000 --- a/client/rest/licenses/commons-codec-1.13.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -3f18e1aa31031d89db6f01ba05d501258ce69d2c \ No newline at end of file diff --git a/client/rest/licenses/commons-codec-1.15.jar.sha1 b/client/rest/licenses/commons-codec-1.15.jar.sha1 new file mode 100644 index 0000000000000..62d99837b87e1 --- /dev/null +++ b/client/rest/licenses/commons-codec-1.15.jar.sha1 @@ -0,0 +1 @@ +49d94806b6e3dc933dacbd8acb0fdbab8ebd1e5d \ No newline at end of file diff --git a/client/rest/licenses/httpasyncclient-4.1.4.jar.sha1 b/client/rest/licenses/httpasyncclient-4.1.4.jar.sha1 deleted file mode 100644 index 8360ab45c7ab3..0000000000000 --- a/client/rest/licenses/httpasyncclient-4.1.4.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -f3a3240681faae3fa46b573a4c7e50cec9db0d86 \ No newline at end of file diff --git a/client/rest/licenses/httpasyncclient-4.1.5.jar.sha1 b/client/rest/licenses/httpasyncclient-4.1.5.jar.sha1 new file mode 100644 index 0000000000000..366a9e31069a6 --- /dev/null +++ b/client/rest/licenses/httpasyncclient-4.1.5.jar.sha1 @@ -0,0 +1 @@ +cd18227f1eb8e9a263286c1d7362ceb24f6f9b32 \ No newline at end of file diff --git a/client/rest/licenses/httpcore-4.4.12.jar.sha1 b/client/rest/licenses/httpcore-4.4.12.jar.sha1 deleted file mode 100644 index 3c046171b30da..0000000000000 --- a/client/rest/licenses/httpcore-4.4.12.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -21ebaf6d532bc350ba95bd81938fa5f0e511c132 \ No newline at end of file diff --git a/client/rest/licenses/httpcore-4.4.15.jar.sha1 b/client/rest/licenses/httpcore-4.4.15.jar.sha1 new file mode 100644 index 0000000000000..42a03b5d7a376 --- /dev/null +++ b/client/rest/licenses/httpcore-4.4.15.jar.sha1 @@ -0,0 +1 @@ +7f2e0c573eaa7a74bac2e89b359e1f73d92a0a1d \ No newline at end of file diff --git a/client/rest/licenses/httpcore-nio-4.4.12.jar.sha1 b/client/rest/licenses/httpcore-nio-4.4.12.jar.sha1 deleted file mode 100644 index 4de932dc5aca0..0000000000000 --- a/client/rest/licenses/httpcore-nio-4.4.12.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -84cd29eca842f31db02987cfedea245af020198b \ No newline at end of file diff --git a/client/rest/licenses/httpcore-nio-4.4.15.jar.sha1 b/client/rest/licenses/httpcore-nio-4.4.15.jar.sha1 new file mode 100644 index 0000000000000..251b35ab6a1a5 --- /dev/null +++ b/client/rest/licenses/httpcore-nio-4.4.15.jar.sha1 @@ -0,0 +1 @@ +85d2b6825d42db909a1474f0ffbd6328429b7a32 \ No newline at end of file diff --git a/client/sniffer/licenses/commons-codec-1.13.jar.sha1 b/client/sniffer/licenses/commons-codec-1.13.jar.sha1 deleted file mode 100644 index 66b72c414d63a..0000000000000 --- a/client/sniffer/licenses/commons-codec-1.13.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -3f18e1aa31031d89db6f01ba05d501258ce69d2c \ No newline at end of file diff --git a/client/sniffer/licenses/commons-codec-1.15.jar.sha1 b/client/sniffer/licenses/commons-codec-1.15.jar.sha1 new file mode 100644 index 0000000000000..62d99837b87e1 --- /dev/null +++ b/client/sniffer/licenses/commons-codec-1.15.jar.sha1 @@ -0,0 +1 @@ +49d94806b6e3dc933dacbd8acb0fdbab8ebd1e5d \ No newline at end of file diff --git a/client/sniffer/licenses/httpcore-4.4.12.jar.sha1 b/client/sniffer/licenses/httpcore-4.4.12.jar.sha1 deleted file mode 100644 index 3c046171b30da..0000000000000 --- a/client/sniffer/licenses/httpcore-4.4.12.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -21ebaf6d532bc350ba95bd81938fa5f0e511c132 \ No newline at end of file diff --git a/client/sniffer/licenses/httpcore-4.4.15.jar.sha1 b/client/sniffer/licenses/httpcore-4.4.15.jar.sha1 new file mode 100644 index 0000000000000..42a03b5d7a376 --- /dev/null +++ b/client/sniffer/licenses/httpcore-4.4.15.jar.sha1 @@ -0,0 +1 @@ +7f2e0c573eaa7a74bac2e89b359e1f73d92a0a1d \ No newline at end of file diff --git a/plugins/analysis-phonetic/licenses/commons-codec-1.13.jar.sha1 b/plugins/analysis-phonetic/licenses/commons-codec-1.13.jar.sha1 deleted file mode 100644 index 66b72c414d63a..0000000000000 --- a/plugins/analysis-phonetic/licenses/commons-codec-1.13.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -3f18e1aa31031d89db6f01ba05d501258ce69d2c \ No newline at end of file diff --git a/plugins/analysis-phonetic/licenses/commons-codec-1.15.jar.sha1 b/plugins/analysis-phonetic/licenses/commons-codec-1.15.jar.sha1 new file mode 100644 index 0000000000000..62d99837b87e1 --- /dev/null +++ b/plugins/analysis-phonetic/licenses/commons-codec-1.15.jar.sha1 @@ -0,0 +1 @@ +49d94806b6e3dc933dacbd8acb0fdbab8ebd1e5d \ No newline at end of file diff --git a/plugins/discovery-azure-classic/licenses/commons-codec-1.13.jar.sha1 b/plugins/discovery-azure-classic/licenses/commons-codec-1.13.jar.sha1 deleted file mode 100644 index 66b72c414d63a..0000000000000 --- a/plugins/discovery-azure-classic/licenses/commons-codec-1.13.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -3f18e1aa31031d89db6f01ba05d501258ce69d2c \ No newline at end of file diff --git a/plugins/discovery-azure-classic/licenses/commons-codec-1.15.jar.sha1 b/plugins/discovery-azure-classic/licenses/commons-codec-1.15.jar.sha1 new file mode 100644 index 0000000000000..62d99837b87e1 --- /dev/null +++ b/plugins/discovery-azure-classic/licenses/commons-codec-1.15.jar.sha1 @@ -0,0 +1 @@ +49d94806b6e3dc933dacbd8acb0fdbab8ebd1e5d \ No newline at end of file diff --git a/plugins/discovery-azure-classic/licenses/httpcore-4.4.12.jar.sha1 b/plugins/discovery-azure-classic/licenses/httpcore-4.4.12.jar.sha1 deleted file mode 100644 index 3c046171b30da..0000000000000 --- a/plugins/discovery-azure-classic/licenses/httpcore-4.4.12.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -21ebaf6d532bc350ba95bd81938fa5f0e511c132 \ No newline at end of file diff --git a/plugins/discovery-azure-classic/licenses/httpcore-4.4.15.jar.sha1 b/plugins/discovery-azure-classic/licenses/httpcore-4.4.15.jar.sha1 new file mode 100644 index 0000000000000..42a03b5d7a376 --- /dev/null +++ b/plugins/discovery-azure-classic/licenses/httpcore-4.4.15.jar.sha1 @@ -0,0 +1 @@ +7f2e0c573eaa7a74bac2e89b359e1f73d92a0a1d \ No newline at end of file diff --git a/plugins/discovery-ec2/licenses/commons-codec-1.13.jar.sha1 b/plugins/discovery-ec2/licenses/commons-codec-1.13.jar.sha1 deleted file mode 100644 index 66b72c414d63a..0000000000000 --- a/plugins/discovery-ec2/licenses/commons-codec-1.13.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -3f18e1aa31031d89db6f01ba05d501258ce69d2c \ No newline at end of file diff --git a/plugins/discovery-ec2/licenses/commons-codec-1.15.jar.sha1 b/plugins/discovery-ec2/licenses/commons-codec-1.15.jar.sha1 new file mode 100644 index 0000000000000..62d99837b87e1 --- /dev/null +++ b/plugins/discovery-ec2/licenses/commons-codec-1.15.jar.sha1 @@ -0,0 +1 @@ +49d94806b6e3dc933dacbd8acb0fdbab8ebd1e5d \ No newline at end of file diff --git a/plugins/discovery-ec2/licenses/httpcore-4.4.12.jar.sha1 b/plugins/discovery-ec2/licenses/httpcore-4.4.12.jar.sha1 deleted file mode 100644 index 3c046171b30da..0000000000000 --- a/plugins/discovery-ec2/licenses/httpcore-4.4.12.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -21ebaf6d532bc350ba95bd81938fa5f0e511c132 \ No newline at end of file diff --git a/plugins/discovery-ec2/licenses/httpcore-4.4.15.jar.sha1 b/plugins/discovery-ec2/licenses/httpcore-4.4.15.jar.sha1 new file mode 100644 index 0000000000000..42a03b5d7a376 --- /dev/null +++ b/plugins/discovery-ec2/licenses/httpcore-4.4.15.jar.sha1 @@ -0,0 +1 @@ +7f2e0c573eaa7a74bac2e89b359e1f73d92a0a1d \ No newline at end of file diff --git a/plugins/discovery-gce/licenses/commons-codec-1.13.jar.sha1 b/plugins/discovery-gce/licenses/commons-codec-1.13.jar.sha1 deleted file mode 100644 index 66b72c414d63a..0000000000000 --- a/plugins/discovery-gce/licenses/commons-codec-1.13.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -3f18e1aa31031d89db6f01ba05d501258ce69d2c \ No newline at end of file diff --git a/plugins/discovery-gce/licenses/commons-codec-1.15.jar.sha1 b/plugins/discovery-gce/licenses/commons-codec-1.15.jar.sha1 new file mode 100644 index 0000000000000..62d99837b87e1 --- /dev/null +++ b/plugins/discovery-gce/licenses/commons-codec-1.15.jar.sha1 @@ -0,0 +1 @@ +49d94806b6e3dc933dacbd8acb0fdbab8ebd1e5d \ No newline at end of file diff --git a/plugins/discovery-gce/licenses/httpcore-4.4.12.jar.sha1 b/plugins/discovery-gce/licenses/httpcore-4.4.12.jar.sha1 deleted file mode 100644 index 3c046171b30da..0000000000000 --- a/plugins/discovery-gce/licenses/httpcore-4.4.12.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -21ebaf6d532bc350ba95bd81938fa5f0e511c132 \ No newline at end of file diff --git a/plugins/discovery-gce/licenses/httpcore-4.4.15.jar.sha1 b/plugins/discovery-gce/licenses/httpcore-4.4.15.jar.sha1 new file mode 100644 index 0000000000000..42a03b5d7a376 --- /dev/null +++ b/plugins/discovery-gce/licenses/httpcore-4.4.15.jar.sha1 @@ -0,0 +1 @@ +7f2e0c573eaa7a74bac2e89b359e1f73d92a0a1d \ No newline at end of file diff --git a/plugins/ingest-attachment/licenses/commons-codec-1.13.jar.sha1 b/plugins/ingest-attachment/licenses/commons-codec-1.13.jar.sha1 deleted file mode 100644 index 66b72c414d63a..0000000000000 --- a/plugins/ingest-attachment/licenses/commons-codec-1.13.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -3f18e1aa31031d89db6f01ba05d501258ce69d2c \ No newline at end of file diff --git a/plugins/ingest-attachment/licenses/commons-codec-1.15.jar.sha1 b/plugins/ingest-attachment/licenses/commons-codec-1.15.jar.sha1 new file mode 100644 index 0000000000000..62d99837b87e1 --- /dev/null +++ b/plugins/ingest-attachment/licenses/commons-codec-1.15.jar.sha1 @@ -0,0 +1 @@ +49d94806b6e3dc933dacbd8acb0fdbab8ebd1e5d \ No newline at end of file diff --git a/plugins/ingest-attachment/licenses/slf4j-api-1.6.2.jar.sha1 b/plugins/ingest-attachment/licenses/slf4j-api-1.6.2.jar.sha1 deleted file mode 100644 index a2f93ea55802b..0000000000000 --- a/plugins/ingest-attachment/licenses/slf4j-api-1.6.2.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -8619e95939167fb37245b5670135e4feb0ec7d50 \ No newline at end of file diff --git a/plugins/ingest-attachment/licenses/slf4j-api-1.7.36.jar.sha1 b/plugins/ingest-attachment/licenses/slf4j-api-1.7.36.jar.sha1 new file mode 100644 index 0000000000000..77b9917528382 --- /dev/null +++ b/plugins/ingest-attachment/licenses/slf4j-api-1.7.36.jar.sha1 @@ -0,0 +1 @@ +6c62681a2f655b49963a5983b8b0950a6120ae14 \ No newline at end of file diff --git a/plugins/repository-azure/licenses/slf4j-api-1.6.2.jar.sha1 b/plugins/repository-azure/licenses/slf4j-api-1.6.2.jar.sha1 deleted file mode 100644 index a2f93ea55802b..0000000000000 --- a/plugins/repository-azure/licenses/slf4j-api-1.6.2.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -8619e95939167fb37245b5670135e4feb0ec7d50 \ No newline at end of file diff --git a/plugins/repository-azure/licenses/slf4j-api-1.7.36.jar.sha1 b/plugins/repository-azure/licenses/slf4j-api-1.7.36.jar.sha1 new file mode 100644 index 0000000000000..77b9917528382 --- /dev/null +++ b/plugins/repository-azure/licenses/slf4j-api-1.7.36.jar.sha1 @@ -0,0 +1 @@ +6c62681a2f655b49963a5983b8b0950a6120ae14 \ No newline at end of file diff --git a/plugins/repository-gcs/licenses/commons-codec-1.13.jar.sha1 b/plugins/repository-gcs/licenses/commons-codec-1.13.jar.sha1 deleted file mode 100644 index 66b72c414d63a..0000000000000 --- a/plugins/repository-gcs/licenses/commons-codec-1.13.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -3f18e1aa31031d89db6f01ba05d501258ce69d2c \ No newline at end of file diff --git a/plugins/repository-gcs/licenses/commons-codec-1.15.jar.sha1 b/plugins/repository-gcs/licenses/commons-codec-1.15.jar.sha1 new file mode 100644 index 0000000000000..62d99837b87e1 --- /dev/null +++ b/plugins/repository-gcs/licenses/commons-codec-1.15.jar.sha1 @@ -0,0 +1 @@ +49d94806b6e3dc933dacbd8acb0fdbab8ebd1e5d \ No newline at end of file diff --git a/plugins/repository-hdfs/licenses/commons-codec-1.13.jar.sha1 b/plugins/repository-hdfs/licenses/commons-codec-1.13.jar.sha1 deleted file mode 100644 index 66b72c414d63a..0000000000000 --- a/plugins/repository-hdfs/licenses/commons-codec-1.13.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -3f18e1aa31031d89db6f01ba05d501258ce69d2c \ No newline at end of file diff --git a/plugins/repository-hdfs/licenses/commons-codec-1.15.jar.sha1 b/plugins/repository-hdfs/licenses/commons-codec-1.15.jar.sha1 new file mode 100644 index 0000000000000..62d99837b87e1 --- /dev/null +++ b/plugins/repository-hdfs/licenses/commons-codec-1.15.jar.sha1 @@ -0,0 +1 @@ +49d94806b6e3dc933dacbd8acb0fdbab8ebd1e5d \ No newline at end of file diff --git a/plugins/repository-hdfs/licenses/slf4j-api-1.6.2.jar.sha1 b/plugins/repository-hdfs/licenses/slf4j-api-1.6.2.jar.sha1 deleted file mode 100644 index a2f93ea55802b..0000000000000 --- a/plugins/repository-hdfs/licenses/slf4j-api-1.6.2.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -8619e95939167fb37245b5670135e4feb0ec7d50 \ No newline at end of file diff --git a/plugins/repository-hdfs/licenses/slf4j-api-1.7.36.jar.sha1 b/plugins/repository-hdfs/licenses/slf4j-api-1.7.36.jar.sha1 new file mode 100644 index 0000000000000..77b9917528382 --- /dev/null +++ b/plugins/repository-hdfs/licenses/slf4j-api-1.7.36.jar.sha1 @@ -0,0 +1 @@ +6c62681a2f655b49963a5983b8b0950a6120ae14 \ No newline at end of file diff --git a/plugins/repository-s3/licenses/commons-codec-1.13.jar.sha1 b/plugins/repository-s3/licenses/commons-codec-1.13.jar.sha1 deleted file mode 100644 index 66b72c414d63a..0000000000000 --- a/plugins/repository-s3/licenses/commons-codec-1.13.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -3f18e1aa31031d89db6f01ba05d501258ce69d2c \ No newline at end of file diff --git a/plugins/repository-s3/licenses/commons-codec-1.15.jar.sha1 b/plugins/repository-s3/licenses/commons-codec-1.15.jar.sha1 new file mode 100644 index 0000000000000..62d99837b87e1 --- /dev/null +++ b/plugins/repository-s3/licenses/commons-codec-1.15.jar.sha1 @@ -0,0 +1 @@ +49d94806b6e3dc933dacbd8acb0fdbab8ebd1e5d \ No newline at end of file diff --git a/plugins/repository-s3/licenses/httpcore-4.4.12.jar.sha1 b/plugins/repository-s3/licenses/httpcore-4.4.12.jar.sha1 deleted file mode 100644 index 3c046171b30da..0000000000000 --- a/plugins/repository-s3/licenses/httpcore-4.4.12.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -21ebaf6d532bc350ba95bd81938fa5f0e511c132 \ No newline at end of file diff --git a/plugins/repository-s3/licenses/httpcore-4.4.15.jar.sha1 b/plugins/repository-s3/licenses/httpcore-4.4.15.jar.sha1 new file mode 100644 index 0000000000000..42a03b5d7a376 --- /dev/null +++ b/plugins/repository-s3/licenses/httpcore-4.4.15.jar.sha1 @@ -0,0 +1 @@ +7f2e0c573eaa7a74bac2e89b359e1f73d92a0a1d \ No newline at end of file From 65f966ed71ff9bc0a53490ee801943869c0f536d Mon Sep 17 00:00:00 2001 From: Tianli Feng Date: Fri, 26 Aug 2022 15:42:31 -0700 Subject: [PATCH 3/3] Restore using the class ClusterInfoRequest and ClusterInfoRequestBuilder from package 'org.opensearch.action.support.master.info' for subclasses (#4307) * Restore using the class ClusterInfoRequest and ClusterInfoRequestBuilder from package 'org.opensearch.action.support.master.info' for subclasses Signed-off-by: Tianli Feng * Add changelog Signed-off-by: Tianli Feng Signed-off-by: Tianli Feng Co-authored-by: Andrew Ross --- CHANGELOG.md | 1 + .../admin/indices/get/GetIndexRequest.java | 2 +- .../indices/get/GetIndexRequestBuilder.java | 2 +- .../mapping/get/GetMappingsRequest.java | 2 +- .../get/GetMappingsRequestBuilder.java | 2 +- .../indices/get/GetIndexRequestTests.java | 21 +++++++++++++++++++ 6 files changed, 26 insertions(+), 4 deletions(-) create mode 100644 server/src/test/java/org/opensearch/action/admin/indices/get/GetIndexRequestTests.java diff --git a/CHANGELOG.md b/CHANGELOG.md index 26ff011609635..8132c1281e412 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -15,6 +15,7 @@ Inspired from [Keep a Changelog](https://keepachangelog.com/en/1.0.0/) ### Fixed - `opensearch-service.bat start` and `opensearch-service.bat manager` failing to run ([#4289](https://github.com/opensearch-project/OpenSearch/pull/4289)) - PR reference to checkout code for changelog verifier ([#4296](https://github.com/opensearch-project/OpenSearch/pull/4296)) +- Restore using the class ClusterInfoRequest and ClusterInfoRequestBuilder from package 'org.opensearch.action.support.master.info' for subclasses ([#4307](https://github.com/opensearch-project/OpenSearch/pull/4307)) - Do not fail replica shard due to primary closure ([#4133](https://github.com/opensearch-project/OpenSearch/pull/4133)) ### Security diff --git a/server/src/main/java/org/opensearch/action/admin/indices/get/GetIndexRequest.java b/server/src/main/java/org/opensearch/action/admin/indices/get/GetIndexRequest.java index ee0b204c77aa3..9a7fae9f84a98 100644 --- a/server/src/main/java/org/opensearch/action/admin/indices/get/GetIndexRequest.java +++ b/server/src/main/java/org/opensearch/action/admin/indices/get/GetIndexRequest.java @@ -33,7 +33,7 @@ package org.opensearch.action.admin.indices.get; import org.opensearch.action.ActionRequestValidationException; -import org.opensearch.action.support.clustermanager.info.ClusterInfoRequest; +import org.opensearch.action.support.master.info.ClusterInfoRequest; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.io.stream.StreamOutput; import org.opensearch.common.util.ArrayUtils; diff --git a/server/src/main/java/org/opensearch/action/admin/indices/get/GetIndexRequestBuilder.java b/server/src/main/java/org/opensearch/action/admin/indices/get/GetIndexRequestBuilder.java index ed106c44ea36a..3019191e5570e 100644 --- a/server/src/main/java/org/opensearch/action/admin/indices/get/GetIndexRequestBuilder.java +++ b/server/src/main/java/org/opensearch/action/admin/indices/get/GetIndexRequestBuilder.java @@ -32,7 +32,7 @@ package org.opensearch.action.admin.indices.get; -import org.opensearch.action.support.clustermanager.info.ClusterInfoRequestBuilder; +import org.opensearch.action.support.master.info.ClusterInfoRequestBuilder; import org.opensearch.client.OpenSearchClient; /** diff --git a/server/src/main/java/org/opensearch/action/admin/indices/mapping/get/GetMappingsRequest.java b/server/src/main/java/org/opensearch/action/admin/indices/mapping/get/GetMappingsRequest.java index 1fd9323edd2f8..2c9bec8398b66 100644 --- a/server/src/main/java/org/opensearch/action/admin/indices/mapping/get/GetMappingsRequest.java +++ b/server/src/main/java/org/opensearch/action/admin/indices/mapping/get/GetMappingsRequest.java @@ -33,7 +33,7 @@ package org.opensearch.action.admin.indices.mapping.get; import org.opensearch.action.ActionRequestValidationException; -import org.opensearch.action.support.clustermanager.info.ClusterInfoRequest; +import org.opensearch.action.support.master.info.ClusterInfoRequest; import org.opensearch.common.io.stream.StreamInput; import java.io.IOException; diff --git a/server/src/main/java/org/opensearch/action/admin/indices/mapping/get/GetMappingsRequestBuilder.java b/server/src/main/java/org/opensearch/action/admin/indices/mapping/get/GetMappingsRequestBuilder.java index 0a6d7cac79133..85bf8c2ffd9c6 100644 --- a/server/src/main/java/org/opensearch/action/admin/indices/mapping/get/GetMappingsRequestBuilder.java +++ b/server/src/main/java/org/opensearch/action/admin/indices/mapping/get/GetMappingsRequestBuilder.java @@ -32,7 +32,7 @@ package org.opensearch.action.admin.indices.mapping.get; -import org.opensearch.action.support.clustermanager.info.ClusterInfoRequestBuilder; +import org.opensearch.action.support.master.info.ClusterInfoRequestBuilder; import org.opensearch.client.OpenSearchClient; /** diff --git a/server/src/test/java/org/opensearch/action/admin/indices/get/GetIndexRequestTests.java b/server/src/test/java/org/opensearch/action/admin/indices/get/GetIndexRequestTests.java new file mode 100644 index 0000000000000..f0d3db71c27b7 --- /dev/null +++ b/server/src/test/java/org/opensearch/action/admin/indices/get/GetIndexRequestTests.java @@ -0,0 +1,21 @@ +/* + * 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.indices.get; + +import org.opensearch.action.support.master.info.ClusterInfoRequest; +import org.opensearch.test.OpenSearchTestCase; + +import static org.hamcrest.Matchers.is; + +public class GetIndexRequestTests extends OpenSearchTestCase { + public void testGetIndexRequestExtendsClusterInfoRequestOfDeprecatedClassPath() { + GetIndexRequest getIndexRequest = new GetIndexRequest().indices("test"); + assertThat(getIndexRequest instanceof ClusterInfoRequest, is(true)); + } +}